mirror of https://github.com/grafana/loki
Experimental TSDB index (#5376)
* moves encoding to own pkg * decwrap impl * forks tsdb index * renames tsdb_enc * wraps all tsdb decbufs * use loki encbuf * custom chunk meta replacing ref-id for uint32 checksum * handle overlapping chunks in tsdb encoding * tsdb builder impl * tsdb querier beginnings to use modified tsdb * fixes nonexistent reference * lint/remove unused * adds tsdb testware * lintpull/5407/head
parent
9b6f17f168
commit
1837c9e0b2
@ -0,0 +1,89 @@ |
||||
package index |
||||
|
||||
import ( |
||||
"context" |
||||
"sort" |
||||
|
||||
"github.com/prometheus/prometheus/model/labels" |
||||
"github.com/prometheus/prometheus/storage" |
||||
) |
||||
|
||||
// Builder is a helper used to create tsdb indices.
|
||||
// It can accept streams in any order and will create the tsdb
|
||||
// index appropriately via `Build()`
|
||||
// It can even receive multiple writes for the same stream with the caveat
|
||||
// that chunks must be added in order and not duplicated
|
||||
type Builder struct { |
||||
streams map[string]*stream |
||||
} |
||||
|
||||
func NewBuilder() *Builder { |
||||
return &Builder{streams: make(map[string]*stream)} |
||||
} |
||||
|
||||
func (b *Builder) AddSeries(ls labels.Labels, chks []ChunkMeta) { |
||||
id := ls.String() |
||||
s, ok := b.streams[id] |
||||
if !ok { |
||||
s = &stream{ |
||||
labels: ls, |
||||
} |
||||
b.streams[id] = s |
||||
} |
||||
|
||||
s.chunks = append(s.chunks, chks...) |
||||
} |
||||
|
||||
func (b *Builder) Build(ctx context.Context, dir string) error { |
||||
writer, err := NewWriter(ctx, dir) |
||||
if err != nil { |
||||
return err |
||||
} |
||||
// TODO(owen-d): multithread
|
||||
|
||||
// Sort series
|
||||
streams := make([]*stream, 0, len(b.streams)) |
||||
for _, s := range b.streams { |
||||
streams = append(streams, s) |
||||
} |
||||
sort.Slice(streams, func(i, j int) bool { |
||||
return labels.Compare(streams[i].labels, streams[j].labels) < 0 |
||||
}) |
||||
|
||||
// Build symbols
|
||||
symbolsMap := make(map[string]struct{}) |
||||
for _, s := range streams { |
||||
for _, l := range s.labels { |
||||
symbolsMap[l.Name] = struct{}{} |
||||
symbolsMap[l.Value] = struct{}{} |
||||
} |
||||
} |
||||
|
||||
// Sort symbols
|
||||
symbols := make([]string, 0, len(symbolsMap)) |
||||
for s := range symbolsMap { |
||||
symbols = append(symbols, s) |
||||
} |
||||
sort.Strings(symbols) |
||||
|
||||
// Add symbols
|
||||
for _, symbol := range symbols { |
||||
if err := writer.AddSymbol(symbol); err != nil { |
||||
return err |
||||
} |
||||
} |
||||
|
||||
// Add series
|
||||
for i, s := range streams { |
||||
if err := writer.AddSeries(storage.SeriesRef(i), s.labels, s.chunks...); err != nil { |
||||
return err |
||||
} |
||||
} |
||||
|
||||
return writer.Close() |
||||
} |
||||
|
||||
type stream struct { |
||||
labels labels.Labels |
||||
chunks []ChunkMeta |
||||
} |
@ -0,0 +1,10 @@ |
||||
package index |
||||
|
||||
// Meta holds information about a chunk of data.
|
||||
type ChunkMeta struct { |
||||
Checksum uint32 |
||||
|
||||
// Time range the data covers.
|
||||
// When MaxTime == math.MaxInt64 the chunk is still open and being appended to.
|
||||
MinTime, MaxTime int64 |
||||
} |
File diff suppressed because it is too large
Load Diff
@ -0,0 +1,545 @@ |
||||
// Copyright 2017 The Prometheus Authors
|
||||
// Licensed under the Apache License, Version 2.0 (the "License");
|
||||
// you may not use this file except in compliance with the License.
|
||||
// You may obtain a copy of the License at
|
||||
//
|
||||
// http://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing, software
|
||||
// distributed under the License is distributed on an "AS IS" BASIS,
|
||||
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
package index |
||||
|
||||
import ( |
||||
"context" |
||||
"fmt" |
||||
"hash/crc32" |
||||
"io/ioutil" |
||||
"math/rand" |
||||
"os" |
||||
"path/filepath" |
||||
"sort" |
||||
"testing" |
||||
|
||||
"github.com/pkg/errors" |
||||
"github.com/stretchr/testify/require" |
||||
"go.uber.org/goleak" |
||||
|
||||
"github.com/prometheus/prometheus/model/labels" |
||||
"github.com/prometheus/prometheus/storage" |
||||
"github.com/prometheus/prometheus/tsdb/encoding" |
||||
"github.com/prometheus/prometheus/util/testutil" |
||||
) |
||||
|
||||
func TestMain(m *testing.M) { |
||||
goleak.VerifyTestMain(m) |
||||
} |
||||
|
||||
type series struct { |
||||
l labels.Labels |
||||
chunks []ChunkMeta |
||||
} |
||||
|
||||
type mockIndex struct { |
||||
series map[storage.SeriesRef]series |
||||
postings map[labels.Label][]storage.SeriesRef |
||||
symbols map[string]struct{} |
||||
} |
||||
|
||||
func newMockIndex() mockIndex { |
||||
ix := mockIndex{ |
||||
series: make(map[storage.SeriesRef]series), |
||||
postings: make(map[labels.Label][]storage.SeriesRef), |
||||
symbols: make(map[string]struct{}), |
||||
} |
||||
ix.postings[allPostingsKey] = []storage.SeriesRef{} |
||||
return ix |
||||
} |
||||
|
||||
func (m mockIndex) Symbols() (map[string]struct{}, error) { |
||||
return m.symbols, nil |
||||
} |
||||
|
||||
func (m mockIndex) AddSeries(ref storage.SeriesRef, l labels.Labels, chunks ...ChunkMeta) error { |
||||
if _, ok := m.series[ref]; ok { |
||||
return errors.Errorf("series with reference %d already added", ref) |
||||
} |
||||
for _, lbl := range l { |
||||
m.symbols[lbl.Name] = struct{}{} |
||||
m.symbols[lbl.Value] = struct{}{} |
||||
if _, ok := m.postings[lbl]; !ok { |
||||
m.postings[lbl] = []storage.SeriesRef{} |
||||
} |
||||
m.postings[lbl] = append(m.postings[lbl], ref) |
||||
} |
||||
m.postings[allPostingsKey] = append(m.postings[allPostingsKey], ref) |
||||
|
||||
s := series{l: l} |
||||
// Actual chunk data is not stored in the index.
|
||||
s.chunks = append(s.chunks, chunks...) |
||||
m.series[ref] = s |
||||
|
||||
return nil |
||||
} |
||||
|
||||
func (m mockIndex) Close() error { |
||||
return nil |
||||
} |
||||
|
||||
func (m mockIndex) LabelValues(name string) ([]string, error) { |
||||
values := []string{} |
||||
for l := range m.postings { |
||||
if l.Name == name { |
||||
values = append(values, l.Value) |
||||
} |
||||
} |
||||
return values, nil |
||||
} |
||||
|
||||
func (m mockIndex) Postings(name string, values ...string) (Postings, error) { |
||||
p := []Postings{} |
||||
for _, value := range values { |
||||
l := labels.Label{Name: name, Value: value} |
||||
p = append(p, m.SortedPostings(NewListPostings(m.postings[l]))) |
||||
} |
||||
return Merge(p...), nil |
||||
} |
||||
|
||||
func (m mockIndex) SortedPostings(p Postings) Postings { |
||||
ep, err := ExpandPostings(p) |
||||
if err != nil { |
||||
return ErrPostings(errors.Wrap(err, "expand postings")) |
||||
} |
||||
|
||||
sort.Slice(ep, func(i, j int) bool { |
||||
return labels.Compare(m.series[ep[i]].l, m.series[ep[j]].l) < 0 |
||||
}) |
||||
return NewListPostings(ep) |
||||
} |
||||
|
||||
func (m mockIndex) Series(ref storage.SeriesRef, lset *labels.Labels, chks *[]ChunkMeta) error { |
||||
s, ok := m.series[ref] |
||||
if !ok { |
||||
return errors.New("not found") |
||||
} |
||||
*lset = append((*lset)[:0], s.l...) |
||||
*chks = append((*chks)[:0], s.chunks...) |
||||
|
||||
return nil |
||||
} |
||||
|
||||
func TestIndexRW_Create_Open(t *testing.T) { |
||||
dir := t.TempDir() |
||||
|
||||
fn := filepath.Join(dir, indexFilename) |
||||
|
||||
// An empty index must still result in a readable file.
|
||||
iw, err := NewWriter(context.Background(), fn) |
||||
require.NoError(t, err) |
||||
require.NoError(t, iw.Close()) |
||||
|
||||
ir, err := NewFileReader(fn) |
||||
require.NoError(t, err) |
||||
require.NoError(t, ir.Close()) |
||||
|
||||
// Modify magic header must cause open to fail.
|
||||
f, err := os.OpenFile(fn, os.O_WRONLY, 0o666) |
||||
require.NoError(t, err) |
||||
_, err = f.WriteAt([]byte{0, 0}, 0) |
||||
require.NoError(t, err) |
||||
f.Close() |
||||
|
||||
_, err = NewFileReader(dir) |
||||
require.Error(t, err) |
||||
} |
||||
|
||||
func TestIndexRW_Postings(t *testing.T) { |
||||
dir := t.TempDir() |
||||
|
||||
fn := filepath.Join(dir, indexFilename) |
||||
|
||||
iw, err := NewWriter(context.Background(), fn) |
||||
require.NoError(t, err) |
||||
|
||||
series := []labels.Labels{ |
||||
labels.FromStrings("a", "1", "b", "1"), |
||||
labels.FromStrings("a", "1", "b", "2"), |
||||
labels.FromStrings("a", "1", "b", "3"), |
||||
labels.FromStrings("a", "1", "b", "4"), |
||||
} |
||||
|
||||
require.NoError(t, iw.AddSymbol("1")) |
||||
require.NoError(t, iw.AddSymbol("2")) |
||||
require.NoError(t, iw.AddSymbol("3")) |
||||
require.NoError(t, iw.AddSymbol("4")) |
||||
require.NoError(t, iw.AddSymbol("a")) |
||||
require.NoError(t, iw.AddSymbol("b")) |
||||
|
||||
// Postings lists are only written if a series with the respective
|
||||
// reference was added before.
|
||||
require.NoError(t, iw.AddSeries(1, series[0])) |
||||
require.NoError(t, iw.AddSeries(2, series[1])) |
||||
require.NoError(t, iw.AddSeries(3, series[2])) |
||||
require.NoError(t, iw.AddSeries(4, series[3])) |
||||
|
||||
require.NoError(t, iw.Close()) |
||||
|
||||
ir, err := NewFileReader(fn) |
||||
require.NoError(t, err) |
||||
|
||||
p, err := ir.Postings("a", "1") |
||||
require.NoError(t, err) |
||||
|
||||
var l labels.Labels |
||||
var c []ChunkMeta |
||||
|
||||
for i := 0; p.Next(); i++ { |
||||
err := ir.Series(p.At(), &l, &c) |
||||
|
||||
require.NoError(t, err) |
||||
require.Equal(t, 0, len(c)) |
||||
require.Equal(t, series[i], l) |
||||
} |
||||
require.NoError(t, p.Err()) |
||||
|
||||
// The label indices are no longer used, so test them by hand here.
|
||||
labelIndices := map[string][]string{} |
||||
require.NoError(t, ReadOffsetTable(ir.b, ir.toc.LabelIndicesTable, func(key []string, off uint64, _ int) error { |
||||
if len(key) != 1 { |
||||
return errors.Errorf("unexpected key length for label indices table %d", len(key)) |
||||
} |
||||
|
||||
d := encoding.NewDecbufAt(ir.b, int(off), castagnoliTable) |
||||
vals := []string{} |
||||
nc := d.Be32int() |
||||
if nc != 1 { |
||||
return errors.Errorf("unexpected number of label indices table names %d", nc) |
||||
} |
||||
for i := d.Be32(); i > 0; i-- { |
||||
v, err := ir.lookupSymbol(d.Be32()) |
||||
if err != nil { |
||||
return err |
||||
} |
||||
vals = append(vals, v) |
||||
} |
||||
labelIndices[key[0]] = vals |
||||
return d.Err() |
||||
})) |
||||
require.Equal(t, map[string][]string{ |
||||
"a": {"1"}, |
||||
"b": {"1", "2", "3", "4"}, |
||||
}, labelIndices) |
||||
|
||||
require.NoError(t, ir.Close()) |
||||
} |
||||
|
||||
func TestPostingsMany(t *testing.T) { |
||||
dir := t.TempDir() |
||||
|
||||
fn := filepath.Join(dir, indexFilename) |
||||
|
||||
iw, err := NewWriter(context.Background(), fn) |
||||
require.NoError(t, err) |
||||
|
||||
// Create a label in the index which has 999 values.
|
||||
symbols := map[string]struct{}{} |
||||
series := []labels.Labels{} |
||||
for i := 1; i < 1000; i++ { |
||||
v := fmt.Sprintf("%03d", i) |
||||
series = append(series, labels.FromStrings("i", v, "foo", "bar")) |
||||
symbols[v] = struct{}{} |
||||
} |
||||
symbols["i"] = struct{}{} |
||||
symbols["foo"] = struct{}{} |
||||
symbols["bar"] = struct{}{} |
||||
syms := []string{} |
||||
for s := range symbols { |
||||
syms = append(syms, s) |
||||
} |
||||
sort.Strings(syms) |
||||
for _, s := range syms { |
||||
require.NoError(t, iw.AddSymbol(s)) |
||||
} |
||||
|
||||
for i, s := range series { |
||||
require.NoError(t, iw.AddSeries(storage.SeriesRef(i), s)) |
||||
} |
||||
require.NoError(t, iw.Close()) |
||||
|
||||
ir, err := NewFileReader(fn) |
||||
require.NoError(t, err) |
||||
defer func() { require.NoError(t, ir.Close()) }() |
||||
|
||||
cases := []struct { |
||||
in []string |
||||
}{ |
||||
// Simple cases, everything is present.
|
||||
{in: []string{"002"}}, |
||||
{in: []string{"031", "032", "033"}}, |
||||
{in: []string{"032", "033"}}, |
||||
{in: []string{"127", "128"}}, |
||||
{in: []string{"127", "128", "129"}}, |
||||
{in: []string{"127", "129"}}, |
||||
{in: []string{"128", "129"}}, |
||||
{in: []string{"998", "999"}}, |
||||
{in: []string{"999"}}, |
||||
// Before actual values.
|
||||
{in: []string{"000"}}, |
||||
{in: []string{"000", "001"}}, |
||||
{in: []string{"000", "002"}}, |
||||
// After actual values.
|
||||
{in: []string{"999a"}}, |
||||
{in: []string{"999", "999a"}}, |
||||
{in: []string{"998", "999", "999a"}}, |
||||
// In the middle of actual values.
|
||||
{in: []string{"126a", "127", "128"}}, |
||||
{in: []string{"127", "127a", "128"}}, |
||||
{in: []string{"127", "127a", "128", "128a", "129"}}, |
||||
{in: []string{"127", "128a", "129"}}, |
||||
{in: []string{"128", "128a", "129"}}, |
||||
{in: []string{"128", "129", "129a"}}, |
||||
{in: []string{"126a", "126b", "127", "127a", "127b", "128", "128a", "128b", "129", "129a", "129b"}}, |
||||
} |
||||
|
||||
for _, c := range cases { |
||||
it, err := ir.Postings("i", c.in...) |
||||
require.NoError(t, err) |
||||
|
||||
got := []string{} |
||||
var lbls labels.Labels |
||||
var metas []ChunkMeta |
||||
for it.Next() { |
||||
require.NoError(t, ir.Series(it.At(), &lbls, &metas)) |
||||
got = append(got, lbls.Get("i")) |
||||
} |
||||
require.NoError(t, it.Err()) |
||||
exp := []string{} |
||||
for _, e := range c.in { |
||||
if _, ok := symbols[e]; ok && e != "l" { |
||||
exp = append(exp, e) |
||||
} |
||||
} |
||||
require.Equal(t, exp, got, fmt.Sprintf("input: %v", c.in)) |
||||
} |
||||
} |
||||
|
||||
func TestPersistence_index_e2e(t *testing.T) { |
||||
dir := t.TempDir() |
||||
|
||||
lbls, err := labels.ReadLabels(filepath.Join("..", "testdata", "20kseries.json"), 20000) |
||||
require.NoError(t, err) |
||||
|
||||
// Sort labels as the index writer expects series in sorted order.
|
||||
sort.Sort(labels.Slice(lbls)) |
||||
|
||||
symbols := map[string]struct{}{} |
||||
for _, lset := range lbls { |
||||
for _, l := range lset { |
||||
symbols[l.Name] = struct{}{} |
||||
symbols[l.Value] = struct{}{} |
||||
} |
||||
} |
||||
|
||||
var input indexWriterSeriesSlice |
||||
|
||||
// Generate ChunkMetas for every label set.
|
||||
for i, lset := range lbls { |
||||
var metas []ChunkMeta |
||||
|
||||
for j := 0; j <= (i % 20); j++ { |
||||
metas = append(metas, ChunkMeta{ |
||||
MinTime: int64(j * 10000), |
||||
MaxTime: int64((j + 1) * 10000), |
||||
Checksum: rand.Uint32(), |
||||
}) |
||||
} |
||||
input = append(input, &indexWriterSeries{ |
||||
labels: lset, |
||||
chunks: metas, |
||||
}) |
||||
} |
||||
|
||||
iw, err := NewWriter(context.Background(), filepath.Join(dir, indexFilename)) |
||||
require.NoError(t, err) |
||||
|
||||
syms := []string{} |
||||
for s := range symbols { |
||||
syms = append(syms, s) |
||||
} |
||||
sort.Strings(syms) |
||||
for _, s := range syms { |
||||
require.NoError(t, iw.AddSymbol(s)) |
||||
} |
||||
|
||||
// Population procedure as done by compaction.
|
||||
var ( |
||||
postings = NewMemPostings() |
||||
values = map[string]map[string]struct{}{} |
||||
) |
||||
|
||||
mi := newMockIndex() |
||||
|
||||
for i, s := range input { |
||||
err = iw.AddSeries(storage.SeriesRef(i), s.labels, s.chunks...) |
||||
require.NoError(t, err) |
||||
require.NoError(t, mi.AddSeries(storage.SeriesRef(i), s.labels, s.chunks...)) |
||||
|
||||
for _, l := range s.labels { |
||||
valset, ok := values[l.Name] |
||||
if !ok { |
||||
valset = map[string]struct{}{} |
||||
values[l.Name] = valset |
||||
} |
||||
valset[l.Value] = struct{}{} |
||||
} |
||||
postings.Add(storage.SeriesRef(i), s.labels) |
||||
} |
||||
|
||||
err = iw.Close() |
||||
require.NoError(t, err) |
||||
|
||||
ir, err := NewFileReader(filepath.Join(dir, indexFilename)) |
||||
require.NoError(t, err) |
||||
|
||||
for p := range mi.postings { |
||||
gotp, err := ir.Postings(p.Name, p.Value) |
||||
require.NoError(t, err) |
||||
|
||||
expp, err := mi.Postings(p.Name, p.Value) |
||||
require.NoError(t, err) |
||||
|
||||
var lset, explset labels.Labels |
||||
var chks, expchks []ChunkMeta |
||||
|
||||
for gotp.Next() { |
||||
require.True(t, expp.Next()) |
||||
|
||||
ref := gotp.At() |
||||
|
||||
err := ir.Series(ref, &lset, &chks) |
||||
require.NoError(t, err) |
||||
|
||||
err = mi.Series(expp.At(), &explset, &expchks) |
||||
require.NoError(t, err) |
||||
require.Equal(t, explset, lset) |
||||
require.Equal(t, expchks, chks) |
||||
} |
||||
require.False(t, expp.Next(), "Expected no more postings for %q=%q", p.Name, p.Value) |
||||
require.NoError(t, gotp.Err()) |
||||
} |
||||
|
||||
labelPairs := map[string][]string{} |
||||
for l := range mi.postings { |
||||
labelPairs[l.Name] = append(labelPairs[l.Name], l.Value) |
||||
} |
||||
for k, v := range labelPairs { |
||||
sort.Strings(v) |
||||
|
||||
res, err := ir.SortedLabelValues(k) |
||||
require.NoError(t, err) |
||||
|
||||
require.Equal(t, len(v), len(res)) |
||||
for i := 0; i < len(v); i++ { |
||||
require.Equal(t, v[i], res[i]) |
||||
} |
||||
} |
||||
|
||||
gotSymbols := []string{} |
||||
it := ir.Symbols() |
||||
for it.Next() { |
||||
gotSymbols = append(gotSymbols, it.At()) |
||||
} |
||||
require.NoError(t, it.Err()) |
||||
expSymbols := []string{} |
||||
for s := range mi.symbols { |
||||
expSymbols = append(expSymbols, s) |
||||
} |
||||
sort.Strings(expSymbols) |
||||
require.Equal(t, expSymbols, gotSymbols) |
||||
|
||||
require.NoError(t, ir.Close()) |
||||
} |
||||
|
||||
func TestDecbufUvarintWithInvalidBuffer(t *testing.T) { |
||||
b := realByteSlice([]byte{0x81, 0x81, 0x81, 0x81, 0x81, 0x81}) |
||||
|
||||
db := encoding.NewDecbufUvarintAt(b, 0, castagnoliTable) |
||||
require.Error(t, db.Err()) |
||||
} |
||||
|
||||
func TestReaderWithInvalidBuffer(t *testing.T) { |
||||
b := realByteSlice([]byte{0x81, 0x81, 0x81, 0x81, 0x81, 0x81}) |
||||
|
||||
_, err := NewReader(b) |
||||
require.Error(t, err) |
||||
} |
||||
|
||||
// TestNewFileReaderErrorNoOpenFiles ensures that in case of an error no file remains open.
|
||||
func TestNewFileReaderErrorNoOpenFiles(t *testing.T) { |
||||
dir := testutil.NewTemporaryDirectory("block", t) |
||||
|
||||
idxName := filepath.Join(dir.Path(), "index") |
||||
err := ioutil.WriteFile(idxName, []byte("corrupted contents"), 0o666) |
||||
require.NoError(t, err) |
||||
|
||||
_, err = NewFileReader(idxName) |
||||
require.Error(t, err) |
||||
|
||||
// dir.Close will fail on Win if idxName fd is not closed on error path.
|
||||
dir.Close() |
||||
} |
||||
|
||||
func TestSymbols(t *testing.T) { |
||||
buf := encoding.Encbuf{} |
||||
|
||||
// Add prefix to the buffer to simulate symbols as part of larger buffer.
|
||||
buf.PutUvarintStr("something") |
||||
|
||||
symbolsStart := buf.Len() |
||||
buf.PutBE32int(204) // Length of symbols table.
|
||||
buf.PutBE32int(100) // Number of symbols.
|
||||
for i := 0; i < 100; i++ { |
||||
// i represents index in unicode characters table.
|
||||
buf.PutUvarintStr(string(rune(i))) // Symbol.
|
||||
} |
||||
checksum := crc32.Checksum(buf.Get()[symbolsStart+4:], castagnoliTable) |
||||
buf.PutBE32(checksum) // Check sum at the end.
|
||||
|
||||
s, err := NewSymbols(realByteSlice(buf.Get()), FormatV2, symbolsStart) |
||||
require.NoError(t, err) |
||||
|
||||
// We store only 4 offsets to symbols.
|
||||
require.Equal(t, 32, s.Size()) |
||||
|
||||
for i := 99; i >= 0; i-- { |
||||
s, err := s.Lookup(uint32(i)) |
||||
require.NoError(t, err) |
||||
require.Equal(t, string(rune(i)), s) |
||||
} |
||||
_, err = s.Lookup(100) |
||||
require.Error(t, err) |
||||
|
||||
for i := 99; i >= 0; i-- { |
||||
r, err := s.ReverseLookup(string(rune(i))) |
||||
require.NoError(t, err) |
||||
require.Equal(t, uint32(i), r) |
||||
} |
||||
_, err = s.ReverseLookup(string(rune(100))) |
||||
require.Error(t, err) |
||||
|
||||
iter := s.Iter() |
||||
i := 0 |
||||
for iter.Next() { |
||||
require.Equal(t, string(rune(i)), iter.At()) |
||||
i++ |
||||
} |
||||
require.NoError(t, iter.Err()) |
||||
} |
||||
|
||||
func TestDecoder_Postings_WrongInput(t *testing.T) { |
||||
_, _, err := (&Decoder{}).Postings([]byte("the cake is a lie")) |
||||
require.Error(t, err) |
||||
} |
@ -0,0 +1,833 @@ |
||||
// Copyright 2017 The Prometheus Authors
|
||||
// Licensed under the Apache License, Version 2.0 (the "License");
|
||||
// you may not use this file except in compliance with the License.
|
||||
// You may obtain a copy of the License at
|
||||
//
|
||||
// http://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing, software
|
||||
// distributed under the License is distributed on an "AS IS" BASIS,
|
||||
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
package index |
||||
|
||||
import ( |
||||
"container/heap" |
||||
"encoding/binary" |
||||
"runtime" |
||||
"sort" |
||||
"sync" |
||||
|
||||
"github.com/prometheus/prometheus/model/labels" |
||||
"github.com/prometheus/prometheus/storage" |
||||
) |
||||
|
||||
var allPostingsKey = labels.Label{} |
||||
|
||||
// AllPostingsKey returns the label key that is used to store the postings list of all existing IDs.
|
||||
func AllPostingsKey() (name, value string) { |
||||
return allPostingsKey.Name, allPostingsKey.Value |
||||
} |
||||
|
||||
// ensureOrderBatchSize is the max number of postings passed to a worker in a single batch in MemPostings.EnsureOrder().
|
||||
const ensureOrderBatchSize = 1024 |
||||
|
||||
// ensureOrderBatchPool is a pool used to recycle batches passed to workers in MemPostings.EnsureOrder().
|
||||
var ensureOrderBatchPool = sync.Pool{ |
||||
New: func() interface{} { |
||||
return make([][]storage.SeriesRef, 0, ensureOrderBatchSize) |
||||
}, |
||||
} |
||||
|
||||
// MemPostings holds postings list for series ID per label pair. They may be written
|
||||
// to out of order.
|
||||
// EnsureOrder() must be called once before any reads are done. This allows for quick
|
||||
// unordered batch fills on startup.
|
||||
type MemPostings struct { |
||||
mtx sync.RWMutex |
||||
m map[string]map[string][]storage.SeriesRef |
||||
ordered bool |
||||
} |
||||
|
||||
// NewMemPostings returns a memPostings that's ready for reads and writes.
|
||||
func NewMemPostings() *MemPostings { |
||||
return &MemPostings{ |
||||
m: make(map[string]map[string][]storage.SeriesRef, 512), |
||||
ordered: true, |
||||
} |
||||
} |
||||
|
||||
// NewUnorderedMemPostings returns a memPostings that is not safe to be read from
|
||||
// until EnsureOrder() was called once.
|
||||
func NewUnorderedMemPostings() *MemPostings { |
||||
return &MemPostings{ |
||||
m: make(map[string]map[string][]storage.SeriesRef, 512), |
||||
ordered: false, |
||||
} |
||||
} |
||||
|
||||
// Symbols returns an iterator over all unique name and value strings, in order.
|
||||
func (p *MemPostings) Symbols() StringIter { |
||||
p.mtx.RLock() |
||||
|
||||
// Add all the strings to a map to de-duplicate.
|
||||
symbols := make(map[string]struct{}, 512) |
||||
for n, e := range p.m { |
||||
symbols[n] = struct{}{} |
||||
for v := range e { |
||||
symbols[v] = struct{}{} |
||||
} |
||||
} |
||||
p.mtx.RUnlock() |
||||
|
||||
res := make([]string, 0, len(symbols)) |
||||
for k := range symbols { |
||||
res = append(res, k) |
||||
} |
||||
|
||||
sort.Strings(res) |
||||
return NewStringListIter(res) |
||||
} |
||||
|
||||
// SortedKeys returns a list of sorted label keys of the postings.
|
||||
func (p *MemPostings) SortedKeys() []labels.Label { |
||||
p.mtx.RLock() |
||||
keys := make([]labels.Label, 0, len(p.m)) |
||||
|
||||
for n, e := range p.m { |
||||
for v := range e { |
||||
keys = append(keys, labels.Label{Name: n, Value: v}) |
||||
} |
||||
} |
||||
p.mtx.RUnlock() |
||||
|
||||
sort.Slice(keys, func(i, j int) bool { |
||||
if keys[i].Name != keys[j].Name { |
||||
return keys[i].Name < keys[j].Name |
||||
} |
||||
return keys[i].Value < keys[j].Value |
||||
}) |
||||
return keys |
||||
} |
||||
|
||||
// LabelNames returns all the unique label names.
|
||||
func (p *MemPostings) LabelNames() []string { |
||||
p.mtx.RLock() |
||||
defer p.mtx.RUnlock() |
||||
n := len(p.m) |
||||
if n == 0 { |
||||
return nil |
||||
} |
||||
|
||||
names := make([]string, 0, n-1) |
||||
for name := range p.m { |
||||
if name != allPostingsKey.Name { |
||||
names = append(names, name) |
||||
} |
||||
} |
||||
return names |
||||
} |
||||
|
||||
// LabelValues returns label values for the given name.
|
||||
func (p *MemPostings) LabelValues(name string) []string { |
||||
p.mtx.RLock() |
||||
defer p.mtx.RUnlock() |
||||
|
||||
values := make([]string, 0, len(p.m[name])) |
||||
for v := range p.m[name] { |
||||
values = append(values, v) |
||||
} |
||||
return values |
||||
} |
||||
|
||||
// PostingsStats contains cardinality based statistics for postings.
|
||||
type PostingsStats struct { |
||||
CardinalityMetricsStats []Stat |
||||
CardinalityLabelStats []Stat |
||||
LabelValueStats []Stat |
||||
LabelValuePairsStats []Stat |
||||
NumLabelPairs int |
||||
} |
||||
|
||||
// Stats calculates the cardinality statistics from postings.
|
||||
func (p *MemPostings) Stats(label string) *PostingsStats { |
||||
const maxNumOfRecords = 10 |
||||
var size uint64 |
||||
|
||||
p.mtx.RLock() |
||||
|
||||
metrics := &maxHeap{} |
||||
labels := &maxHeap{} |
||||
labelValueLength := &maxHeap{} |
||||
labelValuePairs := &maxHeap{} |
||||
numLabelPairs := 0 |
||||
|
||||
metrics.init(maxNumOfRecords) |
||||
labels.init(maxNumOfRecords) |
||||
labelValueLength.init(maxNumOfRecords) |
||||
labelValuePairs.init(maxNumOfRecords) |
||||
|
||||
for n, e := range p.m { |
||||
if n == "" { |
||||
continue |
||||
} |
||||
labels.push(Stat{Name: n, Count: uint64(len(e))}) |
||||
numLabelPairs += len(e) |
||||
size = 0 |
||||
for name, values := range e { |
||||
if n == label { |
||||
metrics.push(Stat{Name: name, Count: uint64(len(values))}) |
||||
} |
||||
labelValuePairs.push(Stat{Name: n + "=" + name, Count: uint64(len(values))}) |
||||
size += uint64(len(name)) |
||||
} |
||||
labelValueLength.push(Stat{Name: n, Count: size}) |
||||
} |
||||
|
||||
p.mtx.RUnlock() |
||||
|
||||
return &PostingsStats{ |
||||
CardinalityMetricsStats: metrics.get(), |
||||
CardinalityLabelStats: labels.get(), |
||||
LabelValueStats: labelValueLength.get(), |
||||
LabelValuePairsStats: labelValuePairs.get(), |
||||
NumLabelPairs: numLabelPairs, |
||||
} |
||||
} |
||||
|
||||
// Get returns a postings list for the given label pair.
|
||||
func (p *MemPostings) Get(name, value string) Postings { |
||||
var lp []storage.SeriesRef |
||||
p.mtx.RLock() |
||||
l := p.m[name] |
||||
if l != nil { |
||||
lp = l[value] |
||||
} |
||||
p.mtx.RUnlock() |
||||
|
||||
if lp == nil { |
||||
return EmptyPostings() |
||||
} |
||||
return newListPostings(lp...) |
||||
} |
||||
|
||||
// All returns a postings list over all documents ever added.
|
||||
func (p *MemPostings) All() Postings { |
||||
return p.Get(AllPostingsKey()) |
||||
} |
||||
|
||||
// EnsureOrder ensures that all postings lists are sorted. After it returns all further
|
||||
// calls to add and addFor will insert new IDs in a sorted manner.
|
||||
func (p *MemPostings) EnsureOrder() { |
||||
p.mtx.Lock() |
||||
defer p.mtx.Unlock() |
||||
|
||||
if p.ordered { |
||||
return |
||||
} |
||||
|
||||
n := runtime.GOMAXPROCS(0) |
||||
workc := make(chan [][]storage.SeriesRef) |
||||
|
||||
var wg sync.WaitGroup |
||||
wg.Add(n) |
||||
|
||||
for i := 0; i < n; i++ { |
||||
go func() { |
||||
for job := range workc { |
||||
for _, l := range job { |
||||
sort.Sort(seriesRefSlice(l)) |
||||
} |
||||
|
||||
job = job[:0] |
||||
ensureOrderBatchPool.Put(job) //nolint:staticcheck // Ignore SA6002 safe to ignore and actually fixing it has some performance penalty.
|
||||
} |
||||
wg.Done() |
||||
}() |
||||
} |
||||
|
||||
nextJob := ensureOrderBatchPool.Get().([][]storage.SeriesRef) |
||||
for _, e := range p.m { |
||||
for _, l := range e { |
||||
nextJob = append(nextJob, l) |
||||
|
||||
if len(nextJob) >= ensureOrderBatchSize { |
||||
workc <- nextJob |
||||
nextJob = ensureOrderBatchPool.Get().([][]storage.SeriesRef) |
||||
} |
||||
} |
||||
} |
||||
|
||||
// If the last job was partially filled, we need to push it to workers too.
|
||||
if len(nextJob) > 0 { |
||||
workc <- nextJob |
||||
} |
||||
|
||||
close(workc) |
||||
wg.Wait() |
||||
|
||||
p.ordered = true |
||||
} |
||||
|
||||
// Delete removes all ids in the given map from the postings lists.
|
||||
func (p *MemPostings) Delete(deleted map[storage.SeriesRef]struct{}) { |
||||
var keys, vals []string |
||||
|
||||
// Collect all keys relevant for deletion once. New keys added afterwards
|
||||
// can by definition not be affected by any of the given deletes.
|
||||
p.mtx.RLock() |
||||
for n := range p.m { |
||||
keys = append(keys, n) |
||||
} |
||||
p.mtx.RUnlock() |
||||
|
||||
for _, n := range keys { |
||||
p.mtx.RLock() |
||||
vals = vals[:0] |
||||
for v := range p.m[n] { |
||||
vals = append(vals, v) |
||||
} |
||||
p.mtx.RUnlock() |
||||
|
||||
// For each posting we first analyse whether the postings list is affected by the deletes.
|
||||
// If yes, we actually reallocate a new postings list.
|
||||
for _, l := range vals { |
||||
// Only lock for processing one postings list so we don't block reads for too long.
|
||||
p.mtx.Lock() |
||||
|
||||
found := false |
||||
for _, id := range p.m[n][l] { |
||||
if _, ok := deleted[id]; ok { |
||||
found = true |
||||
break |
||||
} |
||||
} |
||||
if !found { |
||||
p.mtx.Unlock() |
||||
continue |
||||
} |
||||
repl := make([]storage.SeriesRef, 0, len(p.m[n][l])) |
||||
|
||||
for _, id := range p.m[n][l] { |
||||
if _, ok := deleted[id]; !ok { |
||||
repl = append(repl, id) |
||||
} |
||||
} |
||||
if len(repl) > 0 { |
||||
p.m[n][l] = repl |
||||
} else { |
||||
delete(p.m[n], l) |
||||
} |
||||
p.mtx.Unlock() |
||||
} |
||||
p.mtx.Lock() |
||||
if len(p.m[n]) == 0 { |
||||
delete(p.m, n) |
||||
} |
||||
p.mtx.Unlock() |
||||
} |
||||
} |
||||
|
||||
// Iter calls f for each postings list. It aborts if f returns an error and returns it.
|
||||
func (p *MemPostings) Iter(f func(labels.Label, Postings) error) error { |
||||
p.mtx.RLock() |
||||
defer p.mtx.RUnlock() |
||||
|
||||
for n, e := range p.m { |
||||
for v, p := range e { |
||||
if err := f(labels.Label{Name: n, Value: v}, newListPostings(p...)); err != nil { |
||||
return err |
||||
} |
||||
} |
||||
} |
||||
return nil |
||||
} |
||||
|
||||
// Add a label set to the postings index.
|
||||
func (p *MemPostings) Add(id storage.SeriesRef, lset labels.Labels) { |
||||
p.mtx.Lock() |
||||
|
||||
for _, l := range lset { |
||||
p.addFor(id, l) |
||||
} |
||||
p.addFor(id, allPostingsKey) |
||||
|
||||
p.mtx.Unlock() |
||||
} |
||||
|
||||
func (p *MemPostings) addFor(id storage.SeriesRef, l labels.Label) { |
||||
nm, ok := p.m[l.Name] |
||||
if !ok { |
||||
nm = map[string][]storage.SeriesRef{} |
||||
p.m[l.Name] = nm |
||||
} |
||||
list := append(nm[l.Value], id) |
||||
nm[l.Value] = list |
||||
|
||||
if !p.ordered { |
||||
return |
||||
} |
||||
// There is no guarantee that no higher ID was inserted before as they may
|
||||
// be generated independently before adding them to postings.
|
||||
// We repair order violations on insert. The invariant is that the first n-1
|
||||
// items in the list are already sorted.
|
||||
for i := len(list) - 1; i >= 1; i-- { |
||||
if list[i] >= list[i-1] { |
||||
break |
||||
} |
||||
list[i], list[i-1] = list[i-1], list[i] |
||||
} |
||||
} |
||||
|
||||
// ExpandPostings returns the postings expanded as a slice.
|
||||
func ExpandPostings(p Postings) (res []storage.SeriesRef, err error) { |
||||
for p.Next() { |
||||
res = append(res, p.At()) |
||||
} |
||||
return res, p.Err() |
||||
} |
||||
|
||||
// Postings provides iterative access over a postings list.
|
||||
type Postings interface { |
||||
// Next advances the iterator and returns true if another value was found.
|
||||
Next() bool |
||||
|
||||
// Seek advances the iterator to value v or greater and returns
|
||||
// true if a value was found.
|
||||
Seek(v storage.SeriesRef) bool |
||||
|
||||
// At returns the value at the current iterator position.
|
||||
At() storage.SeriesRef |
||||
|
||||
// Err returns the last error of the iterator.
|
||||
Err() error |
||||
} |
||||
|
||||
// errPostings is an empty iterator that always errors.
|
||||
type errPostings struct { |
||||
err error |
||||
} |
||||
|
||||
func (e errPostings) Next() bool { return false } |
||||
func (e errPostings) Seek(storage.SeriesRef) bool { return false } |
||||
func (e errPostings) At() storage.SeriesRef { return 0 } |
||||
func (e errPostings) Err() error { return e.err } |
||||
|
||||
var emptyPostings = errPostings{} |
||||
|
||||
// EmptyPostings returns a postings list that's always empty.
|
||||
// NOTE: Returning EmptyPostings sentinel when Postings struct has no postings is recommended.
|
||||
// It triggers optimized flow in other functions like Intersect, Without etc.
|
||||
func EmptyPostings() Postings { |
||||
return emptyPostings |
||||
} |
||||
|
||||
// ErrPostings returns new postings that immediately error.
|
||||
func ErrPostings(err error) Postings { |
||||
return errPostings{err} |
||||
} |
||||
|
||||
// Intersect returns a new postings list over the intersection of the
|
||||
// input postings.
|
||||
func Intersect(its ...Postings) Postings { |
||||
if len(its) == 0 { |
||||
return EmptyPostings() |
||||
} |
||||
if len(its) == 1 { |
||||
return its[0] |
||||
} |
||||
for _, p := range its { |
||||
if p == EmptyPostings() { |
||||
return EmptyPostings() |
||||
} |
||||
} |
||||
|
||||
return newIntersectPostings(its...) |
||||
} |
||||
|
||||
type intersectPostings struct { |
||||
arr []Postings |
||||
cur storage.SeriesRef |
||||
} |
||||
|
||||
func newIntersectPostings(its ...Postings) *intersectPostings { |
||||
return &intersectPostings{arr: its} |
||||
} |
||||
|
||||
func (it *intersectPostings) At() storage.SeriesRef { |
||||
return it.cur |
||||
} |
||||
|
||||
func (it *intersectPostings) doNext() bool { |
||||
Loop: |
||||
for { |
||||
for _, p := range it.arr { |
||||
if !p.Seek(it.cur) { |
||||
return false |
||||
} |
||||
if p.At() > it.cur { |
||||
it.cur = p.At() |
||||
continue Loop |
||||
} |
||||
} |
||||
return true |
||||
} |
||||
} |
||||
|
||||
func (it *intersectPostings) Next() bool { |
||||
for _, p := range it.arr { |
||||
if !p.Next() { |
||||
return false |
||||
} |
||||
if p.At() > it.cur { |
||||
it.cur = p.At() |
||||
} |
||||
} |
||||
return it.doNext() |
||||
} |
||||
|
||||
func (it *intersectPostings) Seek(id storage.SeriesRef) bool { |
||||
it.cur = id |
||||
return it.doNext() |
||||
} |
||||
|
||||
func (it *intersectPostings) Err() error { |
||||
for _, p := range it.arr { |
||||
if p.Err() != nil { |
||||
return p.Err() |
||||
} |
||||
} |
||||
return nil |
||||
} |
||||
|
||||
// Merge returns a new iterator over the union of the input iterators.
|
||||
func Merge(its ...Postings) Postings { |
||||
if len(its) == 0 { |
||||
return EmptyPostings() |
||||
} |
||||
if len(its) == 1 { |
||||
return its[0] |
||||
} |
||||
|
||||
p, ok := newMergedPostings(its) |
||||
if !ok { |
||||
return EmptyPostings() |
||||
} |
||||
return p |
||||
} |
||||
|
||||
type postingsHeap []Postings |
||||
|
||||
func (h postingsHeap) Len() int { return len(h) } |
||||
func (h postingsHeap) Less(i, j int) bool { return h[i].At() < h[j].At() } |
||||
func (h *postingsHeap) Swap(i, j int) { (*h)[i], (*h)[j] = (*h)[j], (*h)[i] } |
||||
|
||||
func (h *postingsHeap) Push(x interface{}) { |
||||
*h = append(*h, x.(Postings)) |
||||
} |
||||
|
||||
func (h *postingsHeap) Pop() interface{} { |
||||
old := *h |
||||
n := len(old) |
||||
x := old[n-1] |
||||
*h = old[0 : n-1] |
||||
return x |
||||
} |
||||
|
||||
type mergedPostings struct { |
||||
h postingsHeap |
||||
initialized bool |
||||
cur storage.SeriesRef |
||||
err error |
||||
} |
||||
|
||||
func newMergedPostings(p []Postings) (m *mergedPostings, nonEmpty bool) { |
||||
ph := make(postingsHeap, 0, len(p)) |
||||
|
||||
for _, it := range p { |
||||
// NOTE: mergedPostings struct requires the user to issue an initial Next.
|
||||
if it.Next() { |
||||
ph = append(ph, it) |
||||
} else { |
||||
if it.Err() != nil { |
||||
return &mergedPostings{err: it.Err()}, true |
||||
} |
||||
} |
||||
} |
||||
|
||||
if len(ph) == 0 { |
||||
return nil, false |
||||
} |
||||
return &mergedPostings{h: ph}, true |
||||
} |
||||
|
||||
func (it *mergedPostings) Next() bool { |
||||
if it.h.Len() == 0 || it.err != nil { |
||||
return false |
||||
} |
||||
|
||||
// The user must issue an initial Next.
|
||||
if !it.initialized { |
||||
heap.Init(&it.h) |
||||
it.cur = it.h[0].At() |
||||
it.initialized = true |
||||
return true |
||||
} |
||||
|
||||
for { |
||||
cur := it.h[0] |
||||
if !cur.Next() { |
||||
heap.Pop(&it.h) |
||||
if cur.Err() != nil { |
||||
it.err = cur.Err() |
||||
return false |
||||
} |
||||
if it.h.Len() == 0 { |
||||
return false |
||||
} |
||||
} else { |
||||
// Value of top of heap has changed, re-heapify.
|
||||
heap.Fix(&it.h, 0) |
||||
} |
||||
|
||||
if it.h[0].At() != it.cur { |
||||
it.cur = it.h[0].At() |
||||
return true |
||||
} |
||||
} |
||||
} |
||||
|
||||
func (it *mergedPostings) Seek(id storage.SeriesRef) bool { |
||||
if it.h.Len() == 0 || it.err != nil { |
||||
return false |
||||
} |
||||
if !it.initialized { |
||||
if !it.Next() { |
||||
return false |
||||
} |
||||
} |
||||
for it.cur < id { |
||||
cur := it.h[0] |
||||
if !cur.Seek(id) { |
||||
heap.Pop(&it.h) |
||||
if cur.Err() != nil { |
||||
it.err = cur.Err() |
||||
return false |
||||
} |
||||
if it.h.Len() == 0 { |
||||
return false |
||||
} |
||||
} else { |
||||
// Value of top of heap has changed, re-heapify.
|
||||
heap.Fix(&it.h, 0) |
||||
} |
||||
|
||||
it.cur = it.h[0].At() |
||||
} |
||||
return true |
||||
} |
||||
|
||||
func (it mergedPostings) At() storage.SeriesRef { |
||||
return it.cur |
||||
} |
||||
|
||||
func (it mergedPostings) Err() error { |
||||
return it.err |
||||
} |
||||
|
||||
// Without returns a new postings list that contains all elements from the full list that
|
||||
// are not in the drop list.
|
||||
func Without(full, drop Postings) Postings { |
||||
if full == EmptyPostings() { |
||||
return EmptyPostings() |
||||
} |
||||
|
||||
if drop == EmptyPostings() { |
||||
return full |
||||
} |
||||
return newRemovedPostings(full, drop) |
||||
} |
||||
|
||||
type removedPostings struct { |
||||
full, remove Postings |
||||
|
||||
cur storage.SeriesRef |
||||
|
||||
initialized bool |
||||
fok, rok bool |
||||
} |
||||
|
||||
func newRemovedPostings(full, remove Postings) *removedPostings { |
||||
return &removedPostings{ |
||||
full: full, |
||||
remove: remove, |
||||
} |
||||
} |
||||
|
||||
func (rp *removedPostings) At() storage.SeriesRef { |
||||
return rp.cur |
||||
} |
||||
|
||||
func (rp *removedPostings) Next() bool { |
||||
if !rp.initialized { |
||||
rp.fok = rp.full.Next() |
||||
rp.rok = rp.remove.Next() |
||||
rp.initialized = true |
||||
} |
||||
for { |
||||
if !rp.fok { |
||||
return false |
||||
} |
||||
|
||||
if !rp.rok { |
||||
rp.cur = rp.full.At() |
||||
rp.fok = rp.full.Next() |
||||
return true |
||||
} |
||||
|
||||
fcur, rcur := rp.full.At(), rp.remove.At() |
||||
if fcur < rcur { |
||||
rp.cur = fcur |
||||
rp.fok = rp.full.Next() |
||||
|
||||
return true |
||||
} else if rcur < fcur { |
||||
// Forward the remove postings to the right position.
|
||||
rp.rok = rp.remove.Seek(fcur) |
||||
} else { |
||||
// Skip the current posting.
|
||||
rp.fok = rp.full.Next() |
||||
} |
||||
} |
||||
} |
||||
|
||||
func (rp *removedPostings) Seek(id storage.SeriesRef) bool { |
||||
if rp.cur >= id { |
||||
return true |
||||
} |
||||
|
||||
rp.fok = rp.full.Seek(id) |
||||
rp.rok = rp.remove.Seek(id) |
||||
rp.initialized = true |
||||
|
||||
return rp.Next() |
||||
} |
||||
|
||||
func (rp *removedPostings) Err() error { |
||||
if rp.full.Err() != nil { |
||||
return rp.full.Err() |
||||
} |
||||
|
||||
return rp.remove.Err() |
||||
} |
||||
|
||||
// ListPostings implements the Postings interface over a plain list.
|
||||
type ListPostings struct { |
||||
list []storage.SeriesRef |
||||
cur storage.SeriesRef |
||||
} |
||||
|
||||
func NewListPostings(list []storage.SeriesRef) Postings { |
||||
return newListPostings(list...) |
||||
} |
||||
|
||||
func newListPostings(list ...storage.SeriesRef) *ListPostings { |
||||
return &ListPostings{list: list} |
||||
} |
||||
|
||||
func (it *ListPostings) At() storage.SeriesRef { |
||||
return it.cur |
||||
} |
||||
|
||||
func (it *ListPostings) Next() bool { |
||||
if len(it.list) > 0 { |
||||
it.cur = it.list[0] |
||||
it.list = it.list[1:] |
||||
return true |
||||
} |
||||
it.cur = 0 |
||||
return false |
||||
} |
||||
|
||||
func (it *ListPostings) Seek(x storage.SeriesRef) bool { |
||||
// If the current value satisfies, then return.
|
||||
if it.cur >= x { |
||||
return true |
||||
} |
||||
if len(it.list) == 0 { |
||||
return false |
||||
} |
||||
|
||||
// Do binary search between current position and end.
|
||||
i := sort.Search(len(it.list), func(i int) bool { |
||||
return it.list[i] >= x |
||||
}) |
||||
if i < len(it.list) { |
||||
it.cur = it.list[i] |
||||
it.list = it.list[i+1:] |
||||
return true |
||||
} |
||||
it.list = nil |
||||
return false |
||||
} |
||||
|
||||
func (it *ListPostings) Err() error { |
||||
return nil |
||||
} |
||||
|
||||
// bigEndianPostings implements the Postings interface over a byte stream of
|
||||
// big endian numbers.
|
||||
type bigEndianPostings struct { |
||||
list []byte |
||||
cur uint32 |
||||
} |
||||
|
||||
func newBigEndianPostings(list []byte) *bigEndianPostings { |
||||
return &bigEndianPostings{list: list} |
||||
} |
||||
|
||||
func (it *bigEndianPostings) At() storage.SeriesRef { |
||||
return storage.SeriesRef(it.cur) |
||||
} |
||||
|
||||
func (it *bigEndianPostings) Next() bool { |
||||
if len(it.list) >= 4 { |
||||
it.cur = binary.BigEndian.Uint32(it.list) |
||||
it.list = it.list[4:] |
||||
return true |
||||
} |
||||
return false |
||||
} |
||||
|
||||
func (it *bigEndianPostings) Seek(x storage.SeriesRef) bool { |
||||
if storage.SeriesRef(it.cur) >= x { |
||||
return true |
||||
} |
||||
|
||||
num := len(it.list) / 4 |
||||
// Do binary search between current position and end.
|
||||
i := sort.Search(num, func(i int) bool { |
||||
return binary.BigEndian.Uint32(it.list[i*4:]) >= uint32(x) |
||||
}) |
||||
if i < num { |
||||
j := i * 4 |
||||
it.cur = binary.BigEndian.Uint32(it.list[j:]) |
||||
it.list = it.list[j+4:] |
||||
return true |
||||
} |
||||
it.list = nil |
||||
return false |
||||
} |
||||
|
||||
func (it *bigEndianPostings) Err() error { |
||||
return nil |
||||
} |
||||
|
||||
// seriesRefSlice attaches the methods of sort.Interface to []storage.SeriesRef, sorting in increasing order.
|
||||
type seriesRefSlice []storage.SeriesRef |
||||
|
||||
func (x seriesRefSlice) Len() int { return len(x) } |
||||
func (x seriesRefSlice) Less(i, j int) bool { return x[i] < x[j] } |
||||
func (x seriesRefSlice) Swap(i, j int) { x[i], x[j] = x[j], x[i] } |
@ -0,0 +1,931 @@ |
||||
// Copyright 2017 The Prometheus Authors
|
||||
// Licensed under the Apache License, Version 2.0 (the "License");
|
||||
// you may not use this file except in compliance with the License.
|
||||
// You may obtain a copy of the License at
|
||||
//
|
||||
// http://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing, software
|
||||
// distributed under the License is distributed on an "AS IS" BASIS,
|
||||
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
package index |
||||
|
||||
import ( |
||||
"encoding/binary" |
||||
"fmt" |
||||
"math/rand" |
||||
"sort" |
||||
"strconv" |
||||
"testing" |
||||
|
||||
"github.com/stretchr/testify/require" |
||||
|
||||
"github.com/prometheus/prometheus/model/labels" |
||||
"github.com/prometheus/prometheus/storage" |
||||
) |
||||
|
||||
func TestMemPostings_addFor(t *testing.T) { |
||||
p := NewMemPostings() |
||||
p.m[allPostingsKey.Name] = map[string][]storage.SeriesRef{} |
||||
p.m[allPostingsKey.Name][allPostingsKey.Value] = []storage.SeriesRef{1, 2, 3, 4, 6, 7, 8} |
||||
|
||||
p.addFor(5, allPostingsKey) |
||||
|
||||
require.Equal(t, []storage.SeriesRef{1, 2, 3, 4, 5, 6, 7, 8}, p.m[allPostingsKey.Name][allPostingsKey.Value]) |
||||
} |
||||
|
||||
func TestMemPostings_ensureOrder(t *testing.T) { |
||||
p := NewUnorderedMemPostings() |
||||
p.m["a"] = map[string][]storage.SeriesRef{} |
||||
|
||||
for i := 0; i < 100; i++ { |
||||
l := make([]storage.SeriesRef, 100) |
||||
for j := range l { |
||||
l[j] = storage.SeriesRef(rand.Uint64()) |
||||
} |
||||
v := fmt.Sprintf("%d", i) |
||||
|
||||
p.m["a"][v] = l |
||||
} |
||||
|
||||
p.EnsureOrder() |
||||
|
||||
for _, e := range p.m { |
||||
for _, l := range e { |
||||
ok := sort.SliceIsSorted(l, func(i, j int) bool { |
||||
return l[i] < l[j] |
||||
}) |
||||
if !ok { |
||||
t.Fatalf("postings list %v is not sorted", l) |
||||
} |
||||
} |
||||
} |
||||
} |
||||
|
||||
func BenchmarkMemPostings_ensureOrder(b *testing.B) { |
||||
tests := map[string]struct { |
||||
numLabels int |
||||
numValuesPerLabel int |
||||
numRefsPerValue int |
||||
}{ |
||||
"many values per label": { |
||||
numLabels: 100, |
||||
numValuesPerLabel: 10000, |
||||
numRefsPerValue: 100, |
||||
}, |
||||
"few values per label": { |
||||
numLabels: 1000000, |
||||
numValuesPerLabel: 1, |
||||
numRefsPerValue: 100, |
||||
}, |
||||
"few refs per label value": { |
||||
numLabels: 1000, |
||||
numValuesPerLabel: 1000, |
||||
numRefsPerValue: 10, |
||||
}, |
||||
} |
||||
|
||||
for testName, testData := range tests { |
||||
b.Run(testName, func(b *testing.B) { |
||||
p := NewUnorderedMemPostings() |
||||
|
||||
// Generate postings.
|
||||
for l := 0; l < testData.numLabels; l++ { |
||||
labelName := strconv.Itoa(l) |
||||
p.m[labelName] = map[string][]storage.SeriesRef{} |
||||
|
||||
for v := 0; v < testData.numValuesPerLabel; v++ { |
||||
refs := make([]storage.SeriesRef, testData.numRefsPerValue) |
||||
for j := range refs { |
||||
refs[j] = storage.SeriesRef(rand.Uint64()) |
||||
} |
||||
|
||||
labelValue := strconv.Itoa(v) |
||||
p.m[labelName][labelValue] = refs |
||||
} |
||||
} |
||||
|
||||
b.ResetTimer() |
||||
|
||||
for n := 0; n < b.N; n++ { |
||||
p.EnsureOrder() |
||||
p.ordered = false |
||||
} |
||||
}) |
||||
} |
||||
} |
||||
|
||||
func TestIntersect(t *testing.T) { |
||||
a := newListPostings(1, 2, 3) |
||||
b := newListPostings(2, 3, 4) |
||||
|
||||
cases := []struct { |
||||
in []Postings |
||||
|
||||
res Postings |
||||
}{ |
||||
{ |
||||
in: []Postings{}, |
||||
res: EmptyPostings(), |
||||
}, |
||||
{ |
||||
in: []Postings{a, b, EmptyPostings()}, |
||||
res: EmptyPostings(), |
||||
}, |
||||
{ |
||||
in: []Postings{b, a, EmptyPostings()}, |
||||
res: EmptyPostings(), |
||||
}, |
||||
{ |
||||
in: []Postings{EmptyPostings(), b, a}, |
||||
res: EmptyPostings(), |
||||
}, |
||||
{ |
||||
in: []Postings{EmptyPostings(), a, b}, |
||||
res: EmptyPostings(), |
||||
}, |
||||
{ |
||||
in: []Postings{a, EmptyPostings(), b}, |
||||
res: EmptyPostings(), |
||||
}, |
||||
{ |
||||
in: []Postings{b, EmptyPostings(), a}, |
||||
res: EmptyPostings(), |
||||
}, |
||||
{ |
||||
in: []Postings{b, EmptyPostings(), a, a, b, a, a, a}, |
||||
res: EmptyPostings(), |
||||
}, |
||||
{ |
||||
in: []Postings{ |
||||
newListPostings(1, 2, 3, 4, 5), |
||||
newListPostings(6, 7, 8, 9, 10), |
||||
}, |
||||
res: newListPostings(), |
||||
}, |
||||
{ |
||||
in: []Postings{ |
||||
newListPostings(1, 2, 3, 4, 5), |
||||
newListPostings(4, 5, 6, 7, 8), |
||||
}, |
||||
res: newListPostings(4, 5), |
||||
}, |
||||
{ |
||||
in: []Postings{ |
||||
newListPostings(1, 2, 3, 4, 9, 10), |
||||
newListPostings(1, 4, 5, 6, 7, 8, 10, 11), |
||||
}, |
||||
res: newListPostings(1, 4, 10), |
||||
}, |
||||
{ |
||||
in: []Postings{ |
||||
newListPostings(1), |
||||
newListPostings(0, 1), |
||||
}, |
||||
res: newListPostings(1), |
||||
}, |
||||
{ |
||||
in: []Postings{ |
||||
newListPostings(1), |
||||
}, |
||||
res: newListPostings(1), |
||||
}, |
||||
{ |
||||
in: []Postings{ |
||||
newListPostings(1), |
||||
newListPostings(), |
||||
}, |
||||
res: newListPostings(), |
||||
}, |
||||
{ |
||||
in: []Postings{ |
||||
newListPostings(), |
||||
newListPostings(), |
||||
}, |
||||
res: newListPostings(), |
||||
}, |
||||
} |
||||
|
||||
for _, c := range cases { |
||||
t.Run("", func(t *testing.T) { |
||||
if c.res == nil { |
||||
t.Fatal("intersect result expectancy cannot be nil") |
||||
} |
||||
|
||||
expected, err := ExpandPostings(c.res) |
||||
require.NoError(t, err) |
||||
|
||||
i := Intersect(c.in...) |
||||
|
||||
if c.res == EmptyPostings() { |
||||
require.Equal(t, EmptyPostings(), i) |
||||
return |
||||
} |
||||
|
||||
if i == EmptyPostings() { |
||||
t.Fatal("intersect unexpected result: EmptyPostings sentinel") |
||||
} |
||||
|
||||
res, err := ExpandPostings(i) |
||||
require.NoError(t, err) |
||||
require.Equal(t, expected, res) |
||||
}) |
||||
} |
||||
} |
||||
|
||||
func TestMultiIntersect(t *testing.T) { |
||||
cases := []struct { |
||||
p [][]storage.SeriesRef |
||||
res []storage.SeriesRef |
||||
}{ |
||||
{ |
||||
p: [][]storage.SeriesRef{ |
||||
{1, 2, 3, 4, 5, 6, 1000, 1001}, |
||||
{2, 4, 5, 6, 7, 8, 999, 1001}, |
||||
{1, 2, 5, 6, 7, 8, 1001, 1200}, |
||||
}, |
||||
res: []storage.SeriesRef{2, 5, 6, 1001}, |
||||
}, |
||||
// One of the reproducible cases for:
|
||||
// https://github.com/prometheus/prometheus/issues/2616
|
||||
// The initialisation of intersectPostings was moving the iterator forward
|
||||
// prematurely making us miss some postings.
|
||||
{ |
||||
p: [][]storage.SeriesRef{ |
||||
{1, 2}, |
||||
{1, 2}, |
||||
{1, 2}, |
||||
{2}, |
||||
}, |
||||
res: []storage.SeriesRef{2}, |
||||
}, |
||||
} |
||||
|
||||
for _, c := range cases { |
||||
ps := make([]Postings, 0, len(c.p)) |
||||
for _, postings := range c.p { |
||||
ps = append(ps, newListPostings(postings...)) |
||||
} |
||||
|
||||
res, err := ExpandPostings(Intersect(ps...)) |
||||
|
||||
require.NoError(t, err) |
||||
require.Equal(t, c.res, res) |
||||
} |
||||
} |
||||
|
||||
func BenchmarkIntersect(t *testing.B) { |
||||
t.Run("LongPostings1", func(bench *testing.B) { |
||||
var a, b, c, d []storage.SeriesRef |
||||
|
||||
for i := 0; i < 10000000; i += 2 { |
||||
a = append(a, storage.SeriesRef(i)) |
||||
} |
||||
for i := 5000000; i < 5000100; i += 4 { |
||||
b = append(b, storage.SeriesRef(i)) |
||||
} |
||||
for i := 5090000; i < 5090600; i += 4 { |
||||
b = append(b, storage.SeriesRef(i)) |
||||
} |
||||
for i := 4990000; i < 5100000; i++ { |
||||
c = append(c, storage.SeriesRef(i)) |
||||
} |
||||
for i := 4000000; i < 6000000; i++ { |
||||
d = append(d, storage.SeriesRef(i)) |
||||
} |
||||
|
||||
i1 := newListPostings(a...) |
||||
i2 := newListPostings(b...) |
||||
i3 := newListPostings(c...) |
||||
i4 := newListPostings(d...) |
||||
|
||||
bench.ResetTimer() |
||||
bench.ReportAllocs() |
||||
for i := 0; i < bench.N; i++ { |
||||
if _, err := ExpandPostings(Intersect(i1, i2, i3, i4)); err != nil { |
||||
bench.Fatal(err) |
||||
} |
||||
} |
||||
}) |
||||
|
||||
t.Run("LongPostings2", func(bench *testing.B) { |
||||
var a, b, c, d []storage.SeriesRef |
||||
|
||||
for i := 0; i < 12500000; i++ { |
||||
a = append(a, storage.SeriesRef(i)) |
||||
} |
||||
for i := 7500000; i < 12500000; i++ { |
||||
b = append(b, storage.SeriesRef(i)) |
||||
} |
||||
for i := 9000000; i < 20000000; i++ { |
||||
c = append(c, storage.SeriesRef(i)) |
||||
} |
||||
for i := 10000000; i < 12000000; i++ { |
||||
d = append(d, storage.SeriesRef(i)) |
||||
} |
||||
|
||||
i1 := newListPostings(a...) |
||||
i2 := newListPostings(b...) |
||||
i3 := newListPostings(c...) |
||||
i4 := newListPostings(d...) |
||||
|
||||
bench.ResetTimer() |
||||
bench.ReportAllocs() |
||||
for i := 0; i < bench.N; i++ { |
||||
if _, err := ExpandPostings(Intersect(i1, i2, i3, i4)); err != nil { |
||||
bench.Fatal(err) |
||||
} |
||||
} |
||||
}) |
||||
|
||||
// Many matchers(k >> n).
|
||||
t.Run("ManyPostings", func(bench *testing.B) { |
||||
var its []Postings |
||||
|
||||
// 100000 matchers(k=100000).
|
||||
for i := 0; i < 100000; i++ { |
||||
var temp []storage.SeriesRef |
||||
for j := storage.SeriesRef(1); j < 100; j++ { |
||||
temp = append(temp, j) |
||||
} |
||||
its = append(its, newListPostings(temp...)) |
||||
} |
||||
|
||||
bench.ResetTimer() |
||||
bench.ReportAllocs() |
||||
for i := 0; i < bench.N; i++ { |
||||
if _, err := ExpandPostings(Intersect(its...)); err != nil { |
||||
bench.Fatal(err) |
||||
} |
||||
} |
||||
}) |
||||
} |
||||
|
||||
func TestMultiMerge(t *testing.T) { |
||||
i1 := newListPostings(1, 2, 3, 4, 5, 6, 1000, 1001) |
||||
i2 := newListPostings(2, 4, 5, 6, 7, 8, 999, 1001) |
||||
i3 := newListPostings(1, 2, 5, 6, 7, 8, 1001, 1200) |
||||
|
||||
res, err := ExpandPostings(Merge(i1, i2, i3)) |
||||
require.NoError(t, err) |
||||
require.Equal(t, []storage.SeriesRef{1, 2, 3, 4, 5, 6, 7, 8, 999, 1000, 1001, 1200}, res) |
||||
} |
||||
|
||||
func TestMergedPostings(t *testing.T) { |
||||
cases := []struct { |
||||
in []Postings |
||||
|
||||
res Postings |
||||
}{ |
||||
{ |
||||
in: []Postings{}, |
||||
res: EmptyPostings(), |
||||
}, |
||||
{ |
||||
in: []Postings{ |
||||
newListPostings(), |
||||
newListPostings(), |
||||
}, |
||||
res: EmptyPostings(), |
||||
}, |
||||
{ |
||||
in: []Postings{ |
||||
newListPostings(), |
||||
}, |
||||
res: newListPostings(), |
||||
}, |
||||
{ |
||||
in: []Postings{ |
||||
EmptyPostings(), |
||||
EmptyPostings(), |
||||
EmptyPostings(), |
||||
EmptyPostings(), |
||||
}, |
||||
res: EmptyPostings(), |
||||
}, |
||||
{ |
||||
in: []Postings{ |
||||
newListPostings(1, 2, 3, 4, 5), |
||||
newListPostings(6, 7, 8, 9, 10), |
||||
}, |
||||
res: newListPostings(1, 2, 3, 4, 5, 6, 7, 8, 9, 10), |
||||
}, |
||||
{ |
||||
in: []Postings{ |
||||
newListPostings(1, 2, 3, 4, 5), |
||||
newListPostings(4, 5, 6, 7, 8), |
||||
}, |
||||
res: newListPostings(1, 2, 3, 4, 5, 6, 7, 8), |
||||
}, |
||||
{ |
||||
in: []Postings{ |
||||
newListPostings(1, 2, 3, 4, 9, 10), |
||||
newListPostings(1, 4, 5, 6, 7, 8, 10, 11), |
||||
}, |
||||
res: newListPostings(1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11), |
||||
}, |
||||
{ |
||||
in: []Postings{ |
||||
newListPostings(1, 2, 3, 4, 9, 10), |
||||
EmptyPostings(), |
||||
newListPostings(1, 4, 5, 6, 7, 8, 10, 11), |
||||
}, |
||||
res: newListPostings(1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11), |
||||
}, |
||||
{ |
||||
in: []Postings{ |
||||
newListPostings(1, 2), |
||||
newListPostings(), |
||||
}, |
||||
res: newListPostings(1, 2), |
||||
}, |
||||
{ |
||||
in: []Postings{ |
||||
newListPostings(1, 2), |
||||
EmptyPostings(), |
||||
}, |
||||
res: newListPostings(1, 2), |
||||
}, |
||||
} |
||||
|
||||
for _, c := range cases { |
||||
t.Run("", func(t *testing.T) { |
||||
if c.res == nil { |
||||
t.Fatal("merge result expectancy cannot be nil") |
||||
} |
||||
|
||||
expected, err := ExpandPostings(c.res) |
||||
require.NoError(t, err) |
||||
|
||||
m := Merge(c.in...) |
||||
|
||||
if c.res == EmptyPostings() { |
||||
require.Equal(t, EmptyPostings(), m) |
||||
return |
||||
} |
||||
|
||||
if m == EmptyPostings() { |
||||
t.Fatal("merge unexpected result: EmptyPostings sentinel") |
||||
} |
||||
|
||||
res, err := ExpandPostings(m) |
||||
require.NoError(t, err) |
||||
require.Equal(t, expected, res) |
||||
}) |
||||
} |
||||
} |
||||
|
||||
func TestMergedPostingsSeek(t *testing.T) { |
||||
cases := []struct { |
||||
a, b []storage.SeriesRef |
||||
|
||||
seek storage.SeriesRef |
||||
success bool |
||||
res []storage.SeriesRef |
||||
}{ |
||||
{ |
||||
a: []storage.SeriesRef{2, 3, 4, 5}, |
||||
b: []storage.SeriesRef{6, 7, 8, 9, 10}, |
||||
|
||||
seek: 1, |
||||
success: true, |
||||
res: []storage.SeriesRef{2, 3, 4, 5, 6, 7, 8, 9, 10}, |
||||
}, |
||||
{ |
||||
a: []storage.SeriesRef{1, 2, 3, 4, 5}, |
||||
b: []storage.SeriesRef{6, 7, 8, 9, 10}, |
||||
|
||||
seek: 2, |
||||
success: true, |
||||
res: []storage.SeriesRef{2, 3, 4, 5, 6, 7, 8, 9, 10}, |
||||
}, |
||||
{ |
||||
a: []storage.SeriesRef{1, 2, 3, 4, 5}, |
||||
b: []storage.SeriesRef{4, 5, 6, 7, 8}, |
||||
|
||||
seek: 9, |
||||
success: false, |
||||
res: nil, |
||||
}, |
||||
{ |
||||
a: []storage.SeriesRef{1, 2, 3, 4, 9, 10}, |
||||
b: []storage.SeriesRef{1, 4, 5, 6, 7, 8, 10, 11}, |
||||
|
||||
seek: 10, |
||||
success: true, |
||||
res: []storage.SeriesRef{10, 11}, |
||||
}, |
||||
} |
||||
|
||||
for _, c := range cases { |
||||
a := newListPostings(c.a...) |
||||
b := newListPostings(c.b...) |
||||
|
||||
p := Merge(a, b) |
||||
|
||||
require.Equal(t, c.success, p.Seek(c.seek)) |
||||
|
||||
// After Seek(), At() should be called.
|
||||
if c.success { |
||||
start := p.At() |
||||
lst, err := ExpandPostings(p) |
||||
require.NoError(t, err) |
||||
|
||||
lst = append([]storage.SeriesRef{start}, lst...) |
||||
require.Equal(t, c.res, lst) |
||||
} |
||||
} |
||||
} |
||||
|
||||
func TestRemovedPostings(t *testing.T) { |
||||
cases := []struct { |
||||
a, b []storage.SeriesRef |
||||
res []storage.SeriesRef |
||||
}{ |
||||
{ |
||||
a: nil, |
||||
b: nil, |
||||
res: []storage.SeriesRef(nil), |
||||
}, |
||||
{ |
||||
a: []storage.SeriesRef{1, 2, 3, 4}, |
||||
b: nil, |
||||
res: []storage.SeriesRef{1, 2, 3, 4}, |
||||
}, |
||||
{ |
||||
a: nil, |
||||
b: []storage.SeriesRef{1, 2, 3, 4}, |
||||
res: []storage.SeriesRef(nil), |
||||
}, |
||||
{ |
||||
a: []storage.SeriesRef{1, 2, 3, 4, 5}, |
||||
b: []storage.SeriesRef{6, 7, 8, 9, 10}, |
||||
res: []storage.SeriesRef{1, 2, 3, 4, 5}, |
||||
}, |
||||
{ |
||||
a: []storage.SeriesRef{1, 2, 3, 4, 5}, |
||||
b: []storage.SeriesRef{4, 5, 6, 7, 8}, |
||||
res: []storage.SeriesRef{1, 2, 3}, |
||||
}, |
||||
{ |
||||
a: []storage.SeriesRef{1, 2, 3, 4, 9, 10}, |
||||
b: []storage.SeriesRef{1, 4, 5, 6, 7, 8, 10, 11}, |
||||
res: []storage.SeriesRef{2, 3, 9}, |
||||
}, |
||||
{ |
||||
a: []storage.SeriesRef{1, 2, 3, 4, 9, 10}, |
||||
b: []storage.SeriesRef{1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11}, |
||||
res: []storage.SeriesRef(nil), |
||||
}, |
||||
} |
||||
|
||||
for _, c := range cases { |
||||
a := newListPostings(c.a...) |
||||
b := newListPostings(c.b...) |
||||
|
||||
res, err := ExpandPostings(newRemovedPostings(a, b)) |
||||
require.NoError(t, err) |
||||
require.Equal(t, c.res, res) |
||||
} |
||||
} |
||||
|
||||
func TestRemovedNextStackoverflow(t *testing.T) { |
||||
var full []storage.SeriesRef |
||||
var remove []storage.SeriesRef |
||||
|
||||
var i storage.SeriesRef |
||||
for i = 0; i < 1e7; i++ { |
||||
full = append(full, i) |
||||
remove = append(remove, i) |
||||
} |
||||
|
||||
flp := newListPostings(full...) |
||||
rlp := newListPostings(remove...) |
||||
rp := newRemovedPostings(flp, rlp) |
||||
gotElem := false |
||||
for rp.Next() { |
||||
gotElem = true |
||||
} |
||||
|
||||
require.NoError(t, rp.Err()) |
||||
require.False(t, gotElem) |
||||
} |
||||
|
||||
func TestRemovedPostingsSeek(t *testing.T) { |
||||
cases := []struct { |
||||
a, b []storage.SeriesRef |
||||
|
||||
seek storage.SeriesRef |
||||
success bool |
||||
res []storage.SeriesRef |
||||
}{ |
||||
{ |
||||
a: []storage.SeriesRef{2, 3, 4, 5}, |
||||
b: []storage.SeriesRef{6, 7, 8, 9, 10}, |
||||
|
||||
seek: 1, |
||||
success: true, |
||||
res: []storage.SeriesRef{2, 3, 4, 5}, |
||||
}, |
||||
{ |
||||
a: []storage.SeriesRef{1, 2, 3, 4, 5}, |
||||
b: []storage.SeriesRef{6, 7, 8, 9, 10}, |
||||
|
||||
seek: 2, |
||||
success: true, |
||||
res: []storage.SeriesRef{2, 3, 4, 5}, |
||||
}, |
||||
{ |
||||
a: []storage.SeriesRef{1, 2, 3, 4, 5}, |
||||
b: []storage.SeriesRef{4, 5, 6, 7, 8}, |
||||
|
||||
seek: 9, |
||||
success: false, |
||||
res: nil, |
||||
}, |
||||
{ |
||||
a: []storage.SeriesRef{1, 2, 3, 4, 9, 10}, |
||||
b: []storage.SeriesRef{1, 4, 5, 6, 7, 8, 10, 11}, |
||||
|
||||
seek: 10, |
||||
success: false, |
||||
res: nil, |
||||
}, |
||||
{ |
||||
a: []storage.SeriesRef{1, 2, 3, 4, 9, 10}, |
||||
b: []storage.SeriesRef{1, 4, 5, 6, 7, 8, 11}, |
||||
|
||||
seek: 4, |
||||
success: true, |
||||
res: []storage.SeriesRef{9, 10}, |
||||
}, |
||||
{ |
||||
a: []storage.SeriesRef{1, 2, 3, 4, 9, 10}, |
||||
b: []storage.SeriesRef{1, 4, 5, 6, 7, 8, 11}, |
||||
|
||||
seek: 5, |
||||
success: true, |
||||
res: []storage.SeriesRef{9, 10}, |
||||
}, |
||||
{ |
||||
a: []storage.SeriesRef{1, 2, 3, 4, 9, 10}, |
||||
b: []storage.SeriesRef{1, 4, 5, 6, 7, 8, 11}, |
||||
|
||||
seek: 10, |
||||
success: true, |
||||
res: []storage.SeriesRef{10}, |
||||
}, |
||||
} |
||||
|
||||
for _, c := range cases { |
||||
a := newListPostings(c.a...) |
||||
b := newListPostings(c.b...) |
||||
|
||||
p := newRemovedPostings(a, b) |
||||
|
||||
require.Equal(t, c.success, p.Seek(c.seek)) |
||||
|
||||
// After Seek(), At() should be called.
|
||||
if c.success { |
||||
start := p.At() |
||||
lst, err := ExpandPostings(p) |
||||
require.NoError(t, err) |
||||
|
||||
lst = append([]storage.SeriesRef{start}, lst...) |
||||
require.Equal(t, c.res, lst) |
||||
} |
||||
} |
||||
} |
||||
|
||||
func TestBigEndian(t *testing.T) { |
||||
num := 1000 |
||||
// mock a list as postings
|
||||
ls := make([]uint32, num) |
||||
ls[0] = 2 |
||||
for i := 1; i < num; i++ { |
||||
ls[i] = ls[i-1] + uint32(rand.Int31n(25)) + 2 |
||||
} |
||||
|
||||
beLst := make([]byte, num*4) |
||||
for i := 0; i < num; i++ { |
||||
b := beLst[i*4 : i*4+4] |
||||
binary.BigEndian.PutUint32(b, ls[i]) |
||||
} |
||||
|
||||
t.Run("Iteration", func(t *testing.T) { |
||||
bep := newBigEndianPostings(beLst) |
||||
for i := 0; i < num; i++ { |
||||
require.True(t, bep.Next()) |
||||
require.Equal(t, storage.SeriesRef(ls[i]), bep.At()) |
||||
} |
||||
|
||||
require.False(t, bep.Next()) |
||||
require.NoError(t, bep.Err()) |
||||
}) |
||||
|
||||
t.Run("Seek", func(t *testing.T) { |
||||
table := []struct { |
||||
seek uint32 |
||||
val uint32 |
||||
found bool |
||||
}{ |
||||
{ |
||||
ls[0] - 1, ls[0], true, |
||||
}, |
||||
{ |
||||
ls[4], ls[4], true, |
||||
}, |
||||
{ |
||||
ls[500] - 1, ls[500], true, |
||||
}, |
||||
{ |
||||
ls[600] + 1, ls[601], true, |
||||
}, |
||||
{ |
||||
ls[600] + 1, ls[601], true, |
||||
}, |
||||
{ |
||||
ls[600] + 1, ls[601], true, |
||||
}, |
||||
{ |
||||
ls[0], ls[601], true, |
||||
}, |
||||
{ |
||||
ls[600], ls[601], true, |
||||
}, |
||||
{ |
||||
ls[999], ls[999], true, |
||||
}, |
||||
{ |
||||
ls[999] + 10, ls[999], false, |
||||
}, |
||||
} |
||||
|
||||
bep := newBigEndianPostings(beLst) |
||||
|
||||
for _, v := range table { |
||||
require.Equal(t, v.found, bep.Seek(storage.SeriesRef(v.seek))) |
||||
require.Equal(t, storage.SeriesRef(v.val), bep.At()) |
||||
require.NoError(t, bep.Err()) |
||||
} |
||||
}) |
||||
} |
||||
|
||||
func TestIntersectWithMerge(t *testing.T) { |
||||
// One of the reproducible cases for:
|
||||
// https://github.com/prometheus/prometheus/issues/2616
|
||||
a := newListPostings(21, 22, 23, 24, 25, 30) |
||||
|
||||
b := Merge( |
||||
newListPostings(10, 20, 30), |
||||
newListPostings(15, 26, 30), |
||||
) |
||||
|
||||
p := Intersect(a, b) |
||||
res, err := ExpandPostings(p) |
||||
|
||||
require.NoError(t, err) |
||||
require.Equal(t, []storage.SeriesRef{30}, res) |
||||
} |
||||
|
||||
func TestWithoutPostings(t *testing.T) { |
||||
cases := []struct { |
||||
base Postings |
||||
drop Postings |
||||
|
||||
res Postings |
||||
}{ |
||||
{ |
||||
base: EmptyPostings(), |
||||
drop: EmptyPostings(), |
||||
|
||||
res: EmptyPostings(), |
||||
}, |
||||
{ |
||||
base: EmptyPostings(), |
||||
drop: newListPostings(1, 2), |
||||
|
||||
res: EmptyPostings(), |
||||
}, |
||||
{ |
||||
base: newListPostings(1, 2), |
||||
drop: EmptyPostings(), |
||||
|
||||
res: newListPostings(1, 2), |
||||
}, |
||||
{ |
||||
base: newListPostings(), |
||||
drop: newListPostings(), |
||||
|
||||
res: newListPostings(), |
||||
}, |
||||
{ |
||||
base: newListPostings(1, 2, 3), |
||||
drop: newListPostings(), |
||||
|
||||
res: newListPostings(1, 2, 3), |
||||
}, |
||||
{ |
||||
base: newListPostings(1, 2, 3), |
||||
drop: newListPostings(4, 5, 6), |
||||
|
||||
res: newListPostings(1, 2, 3), |
||||
}, |
||||
{ |
||||
base: newListPostings(1, 2, 3), |
||||
drop: newListPostings(3, 4, 5), |
||||
|
||||
res: newListPostings(1, 2), |
||||
}, |
||||
} |
||||
|
||||
for _, c := range cases { |
||||
t.Run("", func(t *testing.T) { |
||||
if c.res == nil { |
||||
t.Fatal("without result expectancy cannot be nil") |
||||
} |
||||
|
||||
expected, err := ExpandPostings(c.res) |
||||
require.NoError(t, err) |
||||
|
||||
w := Without(c.base, c.drop) |
||||
|
||||
if c.res == EmptyPostings() { |
||||
require.Equal(t, EmptyPostings(), w) |
||||
return |
||||
} |
||||
|
||||
if w == EmptyPostings() { |
||||
t.Fatal("without unexpected result: EmptyPostings sentinel") |
||||
} |
||||
|
||||
res, err := ExpandPostings(w) |
||||
require.NoError(t, err) |
||||
require.Equal(t, expected, res) |
||||
}) |
||||
} |
||||
} |
||||
|
||||
func BenchmarkPostings_Stats(b *testing.B) { |
||||
p := NewMemPostings() |
||||
|
||||
var seriesID storage.SeriesRef |
||||
|
||||
createPostingsLabelValues := func(name, valuePrefix string, count int) { |
||||
for n := 1; n < count; n++ { |
||||
value := fmt.Sprintf("%s-%d", valuePrefix, n) |
||||
p.Add(seriesID, labels.FromStrings(name, value)) |
||||
seriesID++ |
||||
} |
||||
} |
||||
createPostingsLabelValues("__name__", "metrics_name_can_be_very_big_and_bad", 1e3) |
||||
for i := 0; i < 20; i++ { |
||||
createPostingsLabelValues(fmt.Sprintf("host-%d", i), "metrics_name_can_be_very_big_and_bad", 1e3) |
||||
createPostingsLabelValues(fmt.Sprintf("instance-%d", i), "10.0.IP.", 1e3) |
||||
createPostingsLabelValues(fmt.Sprintf("job-%d", i), "Small_Job_name", 1e3) |
||||
createPostingsLabelValues(fmt.Sprintf("err-%d", i), "avg_namespace-", 1e3) |
||||
createPostingsLabelValues(fmt.Sprintf("team-%d", i), "team-", 1e3) |
||||
createPostingsLabelValues(fmt.Sprintf("container_name-%d", i), "pod-", 1e3) |
||||
createPostingsLabelValues(fmt.Sprintf("cluster-%d", i), "newcluster-", 1e3) |
||||
createPostingsLabelValues(fmt.Sprintf("uid-%d", i), "123412312312312311-", 1e3) |
||||
createPostingsLabelValues(fmt.Sprintf("area-%d", i), "new_area_of_work-", 1e3) |
||||
createPostingsLabelValues(fmt.Sprintf("request_id-%d", i), "owner_name_work-", 1e3) |
||||
} |
||||
b.ResetTimer() |
||||
for n := 0; n < b.N; n++ { |
||||
p.Stats("__name__") |
||||
} |
||||
} |
||||
|
||||
func TestMemPostings_Delete(t *testing.T) { |
||||
p := NewMemPostings() |
||||
p.Add(1, labels.FromStrings("lbl1", "a")) |
||||
p.Add(2, labels.FromStrings("lbl1", "b")) |
||||
p.Add(3, labels.FromStrings("lbl2", "a")) |
||||
|
||||
before := p.Get(allPostingsKey.Name, allPostingsKey.Value) |
||||
p.Delete(map[storage.SeriesRef]struct{}{ |
||||
2: {}, |
||||
}) |
||||
after := p.Get(allPostingsKey.Name, allPostingsKey.Value) |
||||
|
||||
// Make sure postings gotten before the delete have the old data when
|
||||
// iterated over.
|
||||
expanded, err := ExpandPostings(before) |
||||
require.NoError(t, err) |
||||
require.Equal(t, []storage.SeriesRef{1, 2, 3}, expanded) |
||||
|
||||
// Make sure postings gotten after the delete have the new data when
|
||||
// iterated over.
|
||||
expanded, err = ExpandPostings(after) |
||||
require.NoError(t, err) |
||||
require.Equal(t, []storage.SeriesRef{1, 3}, expanded) |
||||
|
||||
deleted := p.Get("lbl1", "b") |
||||
expanded, err = ExpandPostings(deleted) |
||||
require.NoError(t, err) |
||||
require.Equal(t, 0, len(expanded), "expected empty postings, got %v", expanded) |
||||
} |
@ -0,0 +1,69 @@ |
||||
// Copyright 2019 The Prometheus Authors
|
||||
// Licensed under the Apache License, Version 2.0 (the "License");
|
||||
// you may not use this file except in compliance with the License.
|
||||
// You may obtain a copy of the License at
|
||||
//
|
||||
// http://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing, software
|
||||
// distributed under the License is distributed on an "AS IS" BASIS,
|
||||
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
package index |
||||
|
||||
import ( |
||||
"math" |
||||
"sort" |
||||
) |
||||
|
||||
// Stat holds values for a single cardinality statistic.
|
||||
type Stat struct { |
||||
Name string |
||||
Count uint64 |
||||
} |
||||
|
||||
type maxHeap struct { |
||||
maxLength int |
||||
minValue uint64 |
||||
minIndex int |
||||
Items []Stat |
||||
} |
||||
|
||||
func (m *maxHeap) init(len int) { |
||||
m.maxLength = len |
||||
m.minValue = math.MaxUint64 |
||||
m.Items = make([]Stat, 0, len) |
||||
} |
||||
|
||||
func (m *maxHeap) push(item Stat) { |
||||
if len(m.Items) < m.maxLength { |
||||
if item.Count < m.minValue { |
||||
m.minValue = item.Count |
||||
m.minIndex = len(m.Items) |
||||
} |
||||
m.Items = append(m.Items, item) |
||||
return |
||||
} |
||||
if item.Count < m.minValue { |
||||
return |
||||
} |
||||
|
||||
m.Items[m.minIndex] = item |
||||
m.minValue = item.Count |
||||
|
||||
for i, stat := range m.Items { |
||||
if stat.Count < m.minValue { |
||||
m.minValue = stat.Count |
||||
m.minIndex = i |
||||
} |
||||
} |
||||
} |
||||
|
||||
func (m *maxHeap) get() []Stat { |
||||
sort.Slice(m.Items, func(i, j int) bool { |
||||
return m.Items[i].Count > m.Items[j].Count |
||||
}) |
||||
return m.Items |
||||
} |
@ -0,0 +1,74 @@ |
||||
// Copyright 2019 The Prometheus Authors
|
||||
// Licensed under the Apache License, Version 2.0 (the "License");
|
||||
// you may not use this file except in compliance with the License.
|
||||
// You may obtain a copy of the License at
|
||||
//
|
||||
// http://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing, software
|
||||
// distributed under the License is distributed on an "AS IS" BASIS,
|
||||
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
package index |
||||
|
||||
import ( |
||||
"testing" |
||||
|
||||
"github.com/stretchr/testify/require" |
||||
) |
||||
|
||||
func TestPostingsStats(t *testing.T) { |
||||
stats := &maxHeap{} |
||||
max := 3000000 |
||||
heapLength := 10 |
||||
stats.init(heapLength) |
||||
for i := 0; i < max; i++ { |
||||
item := Stat{ |
||||
Name: "Label-da", |
||||
Count: uint64(i), |
||||
} |
||||
stats.push(item) |
||||
} |
||||
stats.push(Stat{Name: "Stuff", Count: 3000000}) |
||||
|
||||
data := stats.get() |
||||
require.Equal(t, 10, len(data)) |
||||
for i := 0; i < heapLength; i++ { |
||||
require.Equal(t, uint64(max-i), data[i].Count) |
||||
} |
||||
} |
||||
|
||||
func TestPostingsStats2(t *testing.T) { |
||||
stats := &maxHeap{} |
||||
heapLength := 10 |
||||
|
||||
stats.init(heapLength) |
||||
stats.push(Stat{Name: "Stuff", Count: 10}) |
||||
stats.push(Stat{Name: "Stuff", Count: 11}) |
||||
stats.push(Stat{Name: "Stuff", Count: 1}) |
||||
stats.push(Stat{Name: "Stuff", Count: 6}) |
||||
|
||||
data := stats.get() |
||||
|
||||
require.Equal(t, 4, len(data)) |
||||
require.Equal(t, uint64(11), data[0].Count) |
||||
} |
||||
|
||||
func BenchmarkPostingStatsMaxHep(b *testing.B) { |
||||
stats := &maxHeap{} |
||||
max := 9000000 |
||||
heapLength := 10 |
||||
b.ResetTimer() |
||||
for n := 0; n < b.N; n++ { |
||||
stats.init(heapLength) |
||||
for i := 0; i < max; i++ { |
||||
item := Stat{ |
||||
Name: "Label-da", |
||||
Count: uint64(i), |
||||
} |
||||
stats.push(item) |
||||
} |
||||
stats.get() |
||||
} |
||||
} |
@ -0,0 +1,19 @@ |
||||
package index |
||||
|
||||
import ( |
||||
"github.com/prometheus/prometheus/model/labels" |
||||
) |
||||
|
||||
type indexWriterSeries struct { |
||||
labels labels.Labels |
||||
chunks []ChunkMeta // series file offset of chunks
|
||||
} |
||||
|
||||
type indexWriterSeriesSlice []*indexWriterSeries |
||||
|
||||
func (s indexWriterSeriesSlice) Len() int { return len(s) } |
||||
func (s indexWriterSeriesSlice) Swap(i, j int) { s[i], s[j] = s[j], s[i] } |
||||
|
||||
func (s indexWriterSeriesSlice) Less(i, j int) bool { |
||||
return labels.Compare(s[i].labels, s[j].labels) < 0 |
||||
} |
@ -0,0 +1,278 @@ |
||||
// Copyright 2017 The Prometheus Authors
|
||||
// Licensed under the Apache License, Version 2.0 (the "License");
|
||||
// you may not use this file except in compliance with the License.
|
||||
// You may obtain a copy of the License at
|
||||
//
|
||||
// http://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing, software
|
||||
// distributed under the License is distributed on an "AS IS" BASIS,
|
||||
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
package tsdb |
||||
|
||||
import ( |
||||
"sort" |
||||
"strings" |
||||
"unicode/utf8" |
||||
|
||||
"github.com/prometheus/prometheus/model/labels" |
||||
"github.com/prometheus/prometheus/storage" |
||||
|
||||
"github.com/grafana/loki/pkg/storage/tsdb/index" |
||||
) |
||||
|
||||
// Bitmap used by func isRegexMetaCharacter to check whether a character needs to be escaped.
|
||||
var regexMetaCharacterBytes [16]byte |
||||
|
||||
// isRegexMetaCharacter reports whether byte b needs to be escaped.
|
||||
func isRegexMetaCharacter(b byte) bool { |
||||
return b < utf8.RuneSelf && regexMetaCharacterBytes[b%16]&(1<<(b/16)) != 0 |
||||
} |
||||
|
||||
func init() { |
||||
for _, b := range []byte(`.+*?()|[]{}^$`) { |
||||
regexMetaCharacterBytes[b%16] |= 1 << (b / 16) |
||||
} |
||||
} |
||||
|
||||
// IndexReader provides reading access of serialized index data.
|
||||
type IndexReader interface { |
||||
// Symbols return an iterator over sorted string symbols that may occur in
|
||||
// series' labels and indices. It is not safe to use the returned strings
|
||||
// beyond the lifetime of the index reader.
|
||||
Symbols() index.StringIter |
||||
|
||||
// SortedLabelValues returns sorted possible label values.
|
||||
SortedLabelValues(name string, matchers ...*labels.Matcher) ([]string, error) |
||||
|
||||
// LabelValues returns possible label values which may not be sorted.
|
||||
LabelValues(name string, matchers ...*labels.Matcher) ([]string, error) |
||||
|
||||
// Postings returns the postings list iterator for the label pairs.
|
||||
// The Postings here contain the offsets to the series inside the index.
|
||||
// Found IDs are not strictly required to point to a valid Series, e.g.
|
||||
// during background garbage collections. Input values must be sorted.
|
||||
Postings(name string, values ...string) (index.Postings, error) |
||||
|
||||
// SortedPostings returns a postings list that is reordered to be sorted
|
||||
// by the label set of the underlying series.
|
||||
SortedPostings(index.Postings) index.Postings |
||||
|
||||
// Series populates the given labels and chunk metas for the series identified
|
||||
// by the reference.
|
||||
// Returns storage.ErrNotFound if the ref does not resolve to a known series.
|
||||
Series(ref storage.SeriesRef, lset *labels.Labels, chks *[]index.ChunkMeta) error |
||||
|
||||
// LabelNames returns all the unique label names present in the index in sorted order.
|
||||
LabelNames(matchers ...*labels.Matcher) ([]string, error) |
||||
|
||||
// LabelValueFor returns label value for the given label name in the series referred to by ID.
|
||||
// If the series couldn't be found or the series doesn't have the requested label a
|
||||
// storage.ErrNotFound is returned as error.
|
||||
LabelValueFor(id storage.SeriesRef, label string) (string, error) |
||||
|
||||
// LabelNamesFor returns all the label names for the series referred to by IDs.
|
||||
// The names returned are sorted.
|
||||
LabelNamesFor(ids ...storage.SeriesRef) ([]string, error) |
||||
|
||||
// Close releases the underlying resources of the reader.
|
||||
Close() error |
||||
} |
||||
|
||||
// PostingsForMatchers assembles a single postings iterator against the index reader
|
||||
// based on the given matchers. The resulting postings are not ordered by series.
|
||||
func PostingsForMatchers(ix IndexReader, ms ...*labels.Matcher) (index.Postings, error) { |
||||
var its, notIts []index.Postings |
||||
// See which label must be non-empty.
|
||||
// Optimization for case like {l=~".", l!="1"}.
|
||||
labelMustBeSet := make(map[string]bool, len(ms)) |
||||
for _, m := range ms { |
||||
if !m.Matches("") { |
||||
labelMustBeSet[m.Name] = true |
||||
} |
||||
} |
||||
|
||||
for _, m := range ms { |
||||
if labelMustBeSet[m.Name] { |
||||
// If this matcher must be non-empty, we can be smarter.
|
||||
matchesEmpty := m.Matches("") |
||||
isNot := m.Type == labels.MatchNotEqual || m.Type == labels.MatchNotRegexp |
||||
if isNot && matchesEmpty { // l!="foo"
|
||||
// If the label can't be empty and is a Not and the inner matcher
|
||||
// doesn't match empty, then subtract it out at the end.
|
||||
inverse, err := m.Inverse() |
||||
if err != nil { |
||||
return nil, err |
||||
} |
||||
|
||||
it, err := postingsForMatcher(ix, inverse) |
||||
if err != nil { |
||||
return nil, err |
||||
} |
||||
notIts = append(notIts, it) |
||||
} else if isNot && !matchesEmpty { // l!=""
|
||||
// If the label can't be empty and is a Not, but the inner matcher can
|
||||
// be empty we need to use inversePostingsForMatcher.
|
||||
inverse, err := m.Inverse() |
||||
if err != nil { |
||||
return nil, err |
||||
} |
||||
|
||||
it, err := inversePostingsForMatcher(ix, inverse) |
||||
if err != nil { |
||||
return nil, err |
||||
} |
||||
its = append(its, it) |
||||
} else { // l="a"
|
||||
// Non-Not matcher, use normal postingsForMatcher.
|
||||
it, err := postingsForMatcher(ix, m) |
||||
if err != nil { |
||||
return nil, err |
||||
} |
||||
its = append(its, it) |
||||
} |
||||
} else { // l=""
|
||||
// If the matchers for a labelname selects an empty value, it selects all
|
||||
// the series which don't have the label name set too. See:
|
||||
// https://github.com/prometheus/prometheus/issues/3575 and
|
||||
// https://github.com/prometheus/prometheus/pull/3578#issuecomment-351653555
|
||||
it, err := inversePostingsForMatcher(ix, m) |
||||
if err != nil { |
||||
return nil, err |
||||
} |
||||
notIts = append(notIts, it) |
||||
} |
||||
} |
||||
|
||||
// If there's nothing to subtract from, add in everything and remove the notIts later.
|
||||
if len(its) == 0 && len(notIts) != 0 { |
||||
k, v := index.AllPostingsKey() |
||||
allPostings, err := ix.Postings(k, v) |
||||
if err != nil { |
||||
return nil, err |
||||
} |
||||
its = append(its, allPostings) |
||||
} |
||||
|
||||
it := index.Intersect(its...) |
||||
|
||||
for _, n := range notIts { |
||||
it = index.Without(it, n) |
||||
} |
||||
|
||||
return it, nil |
||||
} |
||||
|
||||
func postingsForMatcher(ix IndexReader, m *labels.Matcher) (index.Postings, error) { |
||||
// This method will not return postings for missing labels.
|
||||
|
||||
// Fast-path for equal matching.
|
||||
if m.Type == labels.MatchEqual { |
||||
return ix.Postings(m.Name, m.Value) |
||||
} |
||||
|
||||
// Fast-path for set matching.
|
||||
if m.Type == labels.MatchRegexp { |
||||
setMatches := findSetMatches(m.GetRegexString()) |
||||
if len(setMatches) > 0 { |
||||
sort.Strings(setMatches) |
||||
return ix.Postings(m.Name, setMatches...) |
||||
} |
||||
} |
||||
|
||||
vals, err := ix.LabelValues(m.Name) |
||||
if err != nil { |
||||
return nil, err |
||||
} |
||||
|
||||
var res []string |
||||
lastVal, isSorted := "", true |
||||
for _, val := range vals { |
||||
if m.Matches(val) { |
||||
res = append(res, val) |
||||
if isSorted && val < lastVal { |
||||
isSorted = false |
||||
} |
||||
lastVal = val |
||||
} |
||||
} |
||||
|
||||
if len(res) == 0 { |
||||
return index.EmptyPostings(), nil |
||||
} |
||||
|
||||
if !isSorted { |
||||
sort.Strings(res) |
||||
} |
||||
return ix.Postings(m.Name, res...) |
||||
} |
||||
|
||||
// inversePostingsForMatcher returns the postings for the series with the label name set but not matching the matcher.
|
||||
func inversePostingsForMatcher(ix IndexReader, m *labels.Matcher) (index.Postings, error) { |
||||
vals, err := ix.LabelValues(m.Name) |
||||
if err != nil { |
||||
return nil, err |
||||
} |
||||
|
||||
var res []string |
||||
lastVal, isSorted := "", true |
||||
for _, val := range vals { |
||||
if !m.Matches(val) { |
||||
res = append(res, val) |
||||
if isSorted && val < lastVal { |
||||
isSorted = false |
||||
} |
||||
lastVal = val |
||||
} |
||||
} |
||||
|
||||
if !isSorted { |
||||
sort.Strings(res) |
||||
} |
||||
return ix.Postings(m.Name, res...) |
||||
} |
||||
|
||||
func findSetMatches(pattern string) []string { |
||||
// Return empty matches if the wrapper from Prometheus is missing.
|
||||
if len(pattern) < 6 || pattern[:4] != "^(?:" || pattern[len(pattern)-2:] != ")$" { |
||||
return nil |
||||
} |
||||
escaped := false |
||||
sets := []*strings.Builder{{}} |
||||
for i := 4; i < len(pattern)-2; i++ { |
||||
if escaped { |
||||
switch { |
||||
case isRegexMetaCharacter(pattern[i]): |
||||
sets[len(sets)-1].WriteByte(pattern[i]) |
||||
case pattern[i] == '\\': |
||||
sets[len(sets)-1].WriteByte('\\') |
||||
default: |
||||
return nil |
||||
} |
||||
escaped = false |
||||
} else { |
||||
switch { |
||||
case isRegexMetaCharacter(pattern[i]): |
||||
if pattern[i] == '|' { |
||||
sets = append(sets, &strings.Builder{}) |
||||
} else { |
||||
return nil |
||||
} |
||||
case pattern[i] == '\\': |
||||
escaped = true |
||||
default: |
||||
sets[len(sets)-1].WriteByte(pattern[i]) |
||||
} |
||||
} |
||||
} |
||||
matches := make([]string, 0, len(sets)) |
||||
for _, s := range sets { |
||||
if s.Len() > 0 { |
||||
matches = append(matches, s.String()) |
||||
} |
||||
} |
||||
return matches |
||||
} |
@ -0,0 +1,104 @@ |
||||
package tsdb |
||||
|
||||
import ( |
||||
"context" |
||||
"testing" |
||||
|
||||
"github.com/prometheus/prometheus/model/labels" |
||||
"github.com/stretchr/testify/require" |
||||
|
||||
"github.com/grafana/loki/pkg/logql" |
||||
"github.com/grafana/loki/pkg/storage/tsdb/index" |
||||
) |
||||
|
||||
func mustParseLabels(s string) labels.Labels { |
||||
ls, err := logql.ParseLabels(s) |
||||
if err != nil { |
||||
panic(err) |
||||
} |
||||
return ls |
||||
} |
||||
|
||||
func TestQueryIndex(t *testing.T) { |
||||
dir := t.TempDir() |
||||
b := index.NewBuilder() |
||||
cases := []struct { |
||||
labels labels.Labels |
||||
chunks []index.ChunkMeta |
||||
}{ |
||||
{ |
||||
labels: mustParseLabels(`{foo="bar"}`), |
||||
chunks: []index.ChunkMeta{ |
||||
{ |
||||
Checksum: 1, |
||||
MinTime: 1, |
||||
MaxTime: 10, |
||||
}, |
||||
{ |
||||
Checksum: 2, |
||||
MinTime: 5, |
||||
MaxTime: 15, |
||||
}, |
||||
}, |
||||
}, |
||||
{ |
||||
labels: mustParseLabels(`{foo="bar", bazz="buzz"}`), |
||||
chunks: []index.ChunkMeta{ |
||||
{ |
||||
Checksum: 3, |
||||
MinTime: 20, |
||||
MaxTime: 30, |
||||
}, |
||||
{ |
||||
Checksum: 4, |
||||
MinTime: 40, |
||||
MaxTime: 50, |
||||
}, |
||||
}, |
||||
}, |
||||
{ |
||||
labels: mustParseLabels(`{unrelated="true"}`), |
||||
chunks: []index.ChunkMeta{ |
||||
{ |
||||
Checksum: 1, |
||||
MinTime: 1, |
||||
MaxTime: 10, |
||||
}, |
||||
{ |
||||
Checksum: 2, |
||||
MinTime: 5, |
||||
MaxTime: 15, |
||||
}, |
||||
}, |
||||
}, |
||||
} |
||||
for _, s := range cases { |
||||
b.AddSeries(s.labels, s.chunks) |
||||
} |
||||
|
||||
require.Nil(t, b.Build(context.Background(), dir)) |
||||
|
||||
reader, err := index.NewFileReader(dir) |
||||
require.Nil(t, err) |
||||
|
||||
p, err := PostingsForMatchers(reader, labels.MustNewMatcher(labels.MatchEqual, "foo", "bar")) |
||||
require.Nil(t, err) |
||||
|
||||
var ( |
||||
chks []index.ChunkMeta |
||||
ls labels.Labels |
||||
) |
||||
|
||||
require.True(t, p.Next()) |
||||
require.Nil(t, reader.Series(p.At(), &ls, &chks)) |
||||
// the second series should be the first returned as it's lexicographically sorted
|
||||
// and bazz < foo
|
||||
require.Equal(t, cases[1].labels.String(), ls.String()) |
||||
require.Equal(t, cases[1].chunks, chks) |
||||
require.True(t, p.Next()) |
||||
require.Nil(t, reader.Series(p.At(), &ls, &chks)) |
||||
// Now we should encounter the series "added" first.
|
||||
require.Equal(t, cases[0].labels.String(), ls.String()) |
||||
require.Equal(t, cases[0].chunks, chks) |
||||
require.False(t, p.Next()) |
||||
} |
File diff suppressed because it is too large
Load Diff
@ -0,0 +1,42 @@ |
||||
package encoding |
||||
|
||||
import "github.com/prometheus/prometheus/tsdb/encoding" |
||||
|
||||
func EncWith(b []byte) (res Encbuf) { |
||||
res.B = b |
||||
return res |
||||
} |
||||
|
||||
func EncWrap(inner encoding.Encbuf) Encbuf { return Encbuf{Encbuf: inner} } |
||||
|
||||
// Encbuf extends encoding.Encbuf with support for multi byte encoding
|
||||
type Encbuf struct { |
||||
encoding.Encbuf |
||||
} |
||||
|
||||
func (e *Encbuf) PutString(s string) { e.B = append(e.B, s...) } |
||||
|
||||
func DecWith(b []byte) (res Decbuf) { |
||||
res.B = b |
||||
return res |
||||
} |
||||
|
||||
func DecWrap(inner encoding.Decbuf) Decbuf { return Decbuf{Decbuf: inner} } |
||||
|
||||
// Decbuf extends encoding.Decbuf with support for multi byte decoding
|
||||
type Decbuf struct { |
||||
encoding.Decbuf |
||||
} |
||||
|
||||
func (d *Decbuf) Bytes(n int) []byte { |
||||
if d.E != nil { |
||||
return nil |
||||
} |
||||
if len(d.B) < n { |
||||
d.E = encoding.ErrInvalidSize |
||||
return nil |
||||
} |
||||
x := d.B[:n] |
||||
d.B = d.B[n:] |
||||
return x |
||||
} |
Loading…
Reference in new issue