forked from grafana/loki
-
Notifications
You must be signed in to change notification settings - Fork 0
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
* begins speccing out TSDB Head * auto incrementing series ref + mempostings * mintime/maxtime methods * tsdb head IndexReader impl * head correctly populates ref lookup * tsdb head tests * adds prometheus license to tsdb head * linting * [WIP] speccing out tsdb head wal * fix length check and adds tsdb wal encoding tests * exposes wal structs & removes closed semantics * logs start time in the tsdb wal * wal interface + testing * exports walrecord + returns ref when appending * specs out head manager * tsdb head manager wal initialization * tsdb wal rotation * wals dont use node name, but tsdb files do * cleans up fn signature * multi tsdb idx now just wraps Index interfaces * no longer sorts indices when creating multi-idx * tenantHeads & HeadManger index impls * head mgr tests * bugfixes & head manager tests * tsdb dir selection now helper fns * period utility * pulls out more code to helpers, fixes some var races * head recovery is more generic * tsdb manager builds from wals * pulls more helpers out of headmanager * lockedIdx, Close() on idx, tsdbManager update * removes mmap from index reader implementation * tsdb file * adds tsdb shipper config and refactors initStore * removes unused tsdbManager code * implements stores.Index and stores.ChunkWriter for tsdb * chunk.Data now supports an Entries() method * moves walreader to new util/wal pkg to avoid circular dep + tsdb storage alignment * tsdb store * passes indexWriter to chunkWriter * build a tsdb per index bucket in according with shipper conventions * dont open tsdb files until necessary for indexshipper * tsdbManager Index impl * tsdb defaults + initStore fix for invalid looping * fixes UsingTSDB helper * disables deleteRequestStore when using TSDB * pass limits to tsdb store * always start headmanager for tsdb Signed-off-by: Owen Diehl <ow.diehl@gmail.com> * fixes copy bug Signed-off-by: Owen Diehl <ow.diehl@gmail.com> * more logging Signed-off-by: Owen Diehl <ow.diehl@gmail.com> * fixes duplicate tenant label bug Signed-off-by: Owen Diehl <ow.diehl@gmail.com> * debug logs, uses label builder, removes __name__=logs for tsdb Signed-off-by: Owen Diehl <ow.diehl@gmail.com> * tsdb fixes labels at earlier pt Signed-off-by: Owen Diehl <ow.diehl@gmail.com> * account for setting tenant label in head manager test * changing tsdb dir names * identifier interface, builder to tsdb pkg * tsdb version path prefix * fixes buildfromwals identifier * fixes tsdb shipper paths * split buckets once per user set * refactors combining single and multi tenant tsdb indices on shipper reads * indexshipper ignores old gzip logic * method name refactor * remove unused record type * removes v1 prefix in tsdb paths and refactores indices method * ignores double optimization in tsdb looking for multitenant idx, shipper handles this * removes 5-ln requirement on shipper tablename regexp * groups identifiers, begins removing multitenant prefix in shipped files * passses open fn to indexshipper * exposes RealByteSlice * TSDBFile no longer needs a file descriptor, parses gzip extensions * method signature fixing * stop masquerading as compressed indices post-download in indexshipper * variable bucket regexp * removes accidental configs committed * label matcher handling for multitenancy and metricname in tsdb * explicitly require fingerprint when creating tsdb index * only add tenant label when creating multitenant tsdb write fingerprints without synthetic tenant label strip out tenant labels from queries * linting + unused removal * more linting :( * goimports * removes uploadername from indexshipper * maxuint32 for arm32 builds * tsdb chunk filterer support * always set ingester name when using object storage index Co-authored-by: Sandeep Sukhani <sandeep.d.sukhani@gmail.com>
1 parent
03153e8
commit b45efd4
Showing
49 changed files
with
2,926 additions
and
297 deletions.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,100 @@ | ||
package tsdb | ||
|
||
import ( | ||
"context" | ||
|
||
"github.com/go-kit/log/level" | ||
"github.com/pkg/errors" | ||
"github.com/prometheus/common/model" | ||
"github.com/prometheus/prometheus/model/labels" | ||
|
||
"github.com/grafana/loki/pkg/storage/chunk" | ||
"github.com/grafana/loki/pkg/storage/chunk/fetcher" | ||
"github.com/grafana/loki/pkg/storage/config" | ||
"github.com/grafana/loki/pkg/storage/stores/series" | ||
"github.com/grafana/loki/pkg/storage/stores/tsdb/index" | ||
"github.com/grafana/loki/pkg/util/spanlogger" | ||
) | ||
|
||
type IndexWriter interface { | ||
Append(userID string, ls labels.Labels, chks index.ChunkMetas) error | ||
} | ||
|
||
type ChunkWriter struct { | ||
schemaCfg config.SchemaConfig | ||
fetcher *fetcher.Fetcher | ||
indexWriter IndexWriter | ||
} | ||
|
||
func NewChunkWriter( | ||
fetcher *fetcher.Fetcher, | ||
pd config.PeriodConfig, | ||
indexWriter IndexWriter, | ||
) *ChunkWriter { | ||
return &ChunkWriter{ | ||
schemaCfg: config.SchemaConfig{ | ||
Configs: []config.PeriodConfig{pd}, | ||
}, | ||
fetcher: fetcher, | ||
indexWriter: indexWriter, | ||
} | ||
} | ||
|
||
func (w *ChunkWriter) Put(ctx context.Context, chunks []chunk.Chunk) error { | ||
for _, chunk := range chunks { | ||
if err := w.PutOne(ctx, chunk.From, chunk.Through, chunk); err != nil { | ||
return err | ||
} | ||
} | ||
return nil | ||
} | ||
|
||
func (w *ChunkWriter) PutOne(ctx context.Context, from, through model.Time, chk chunk.Chunk) error { | ||
log, ctx := spanlogger.New(ctx, "SeriesStore.PutOne") | ||
defer log.Finish() | ||
|
||
// with local TSDB indices, we _always_ write the index entry | ||
// to avoid data loss if we lose an ingester's disk | ||
// but we can skip writing the chunk if another replica | ||
// has already written it to storage. | ||
writeChunk := true | ||
|
||
// If this chunk is in cache it must already be in the database so we don't need to write it again | ||
found, _, _, _ := w.fetcher.Cache().Fetch(ctx, []string{w.schemaCfg.ExternalKey(chk.ChunkRef)}) | ||
|
||
if len(found) > 0 { | ||
writeChunk = false | ||
series.DedupedChunksTotal.Inc() | ||
} | ||
|
||
chunks := []chunk.Chunk{chk} | ||
|
||
c := w.fetcher.Client() | ||
if writeChunk { | ||
if err := c.PutChunks(ctx, chunks); err != nil { | ||
return errors.Wrap(err, "writing chunk") | ||
} | ||
} | ||
|
||
// Always write the index to benefit durability via replication factor. | ||
metas := index.ChunkMetas{ | ||
{ | ||
Checksum: chk.ChunkRef.Checksum, | ||
MinTime: int64(chk.ChunkRef.From), | ||
MaxTime: int64(chk.ChunkRef.Through), | ||
KB: uint32(chk.Size()) / (1 << 10), | ||
Entries: uint32(chk.Data.Entries()), | ||
}, | ||
} | ||
if err := w.indexWriter.Append(chk.UserID, chk.Metric, metas); err != nil { | ||
return errors.Wrap(err, "writing index entry") | ||
} | ||
|
||
if writeChunk { | ||
if cacheErr := w.fetcher.WriteBackCache(ctx, chunks); cacheErr != nil { | ||
level.Warn(log).Log("msg", "could not store chunks in chunk cache", "err", cacheErr) | ||
} | ||
} | ||
|
||
return nil | ||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Large diffs are not rendered by default.
Oops, something went wrong.
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,310 @@ | ||
package tsdb | ||
|
||
import ( | ||
"context" | ||
"math" | ||
"testing" | ||
"time" | ||
|
||
"github.com/go-kit/log" | ||
"github.com/prometheus/common/model" | ||
"github.com/prometheus/prometheus/model/labels" | ||
"github.com/prometheus/prometheus/tsdb/chunks" | ||
"github.com/prometheus/prometheus/tsdb/record" | ||
"github.com/stretchr/testify/require" | ||
|
||
"github.com/grafana/loki/pkg/storage/chunk/client/util" | ||
"github.com/grafana/loki/pkg/storage/stores/tsdb/index" | ||
) | ||
|
||
type noopTSDBManager struct{ NoopIndex } | ||
|
||
func (noopTSDBManager) BuildFromWALs(_ time.Time, _ []WALIdentifier) error { return nil } | ||
|
||
func chunkMetasToChunkRefs(user string, fp uint64, xs index.ChunkMetas) (res []ChunkRef) { | ||
for _, x := range xs { | ||
res = append(res, ChunkRef{ | ||
User: user, | ||
Fingerprint: model.Fingerprint(fp), | ||
Start: x.From(), | ||
End: x.Through(), | ||
Checksum: x.Checksum, | ||
}) | ||
} | ||
return | ||
} | ||
|
||
// Test append | ||
func Test_TenantHeads_Append(t *testing.T) { | ||
h := newTenantHeads(time.Now(), defaultHeadManagerStripeSize, NewMetrics(nil), log.NewNopLogger()) | ||
ls := mustParseLabels(`{foo="bar"}`) | ||
chks := []index.ChunkMeta{ | ||
{ | ||
Checksum: 0, | ||
MinTime: 1, | ||
MaxTime: 10, | ||
KB: 2, | ||
Entries: 30, | ||
}, | ||
} | ||
_ = h.Append("fake", ls, chks) | ||
|
||
found, err := h.GetChunkRefs( | ||
context.Background(), | ||
"fake", | ||
0, | ||
100, | ||
nil, nil, | ||
labels.MustNewMatcher(labels.MatchEqual, "foo", "bar"), | ||
) | ||
require.Nil(t, err) | ||
require.Equal(t, chunkMetasToChunkRefs("fake", ls.Hash(), chks), found) | ||
|
||
} | ||
|
||
// Test multitenant reads | ||
func Test_TenantHeads_MultiRead(t *testing.T) { | ||
h := newTenantHeads(time.Now(), defaultHeadManagerStripeSize, NewMetrics(nil), log.NewNopLogger()) | ||
ls := mustParseLabels(`{foo="bar"}`) | ||
chks := []index.ChunkMeta{ | ||
{ | ||
Checksum: 0, | ||
MinTime: 1, | ||
MaxTime: 10, | ||
KB: 2, | ||
Entries: 30, | ||
}, | ||
} | ||
|
||
tenants := []struct { | ||
user string | ||
ls labels.Labels | ||
}{ | ||
{ | ||
user: "tenant1", | ||
ls: append(ls.Copy(), labels.Label{ | ||
Name: "tenant", | ||
Value: "tenant1", | ||
}), | ||
}, | ||
{ | ||
user: "tenant2", | ||
ls: append(ls.Copy(), labels.Label{ | ||
Name: "tenant", | ||
Value: "tenant2", | ||
}), | ||
}, | ||
} | ||
|
||
// add data for both tenants | ||
for _, tenant := range tenants { | ||
_ = h.Append(tenant.user, tenant.ls, chks) | ||
|
||
} | ||
|
||
// ensure we're only returned the data from the correct tenant | ||
for _, tenant := range tenants { | ||
found, err := h.GetChunkRefs( | ||
context.Background(), | ||
tenant.user, | ||
0, | ||
100, | ||
nil, nil, | ||
labels.MustNewMatcher(labels.MatchEqual, "foo", "bar"), | ||
) | ||
require.Nil(t, err) | ||
require.Equal(t, chunkMetasToChunkRefs(tenant.user, tenant.ls.Hash(), chks), found) | ||
} | ||
|
||
} | ||
|
||
// test head recover from wal | ||
func Test_HeadManager_RecoverHead(t *testing.T) { | ||
now := time.Now() | ||
dir := t.TempDir() | ||
cases := []struct { | ||
Labels labels.Labels | ||
Chunks []index.ChunkMeta | ||
User string | ||
}{ | ||
{ | ||
User: "tenant1", | ||
Labels: mustParseLabels(`{foo="bar", bazz="buzz"}`), | ||
Chunks: []index.ChunkMeta{ | ||
{ | ||
MinTime: 1, | ||
MaxTime: 10, | ||
Checksum: 3, | ||
}, | ||
}, | ||
}, | ||
{ | ||
User: "tenant2", | ||
Labels: mustParseLabels(`{foo="bard", bazz="bozz", bonk="borb"}`), | ||
Chunks: []index.ChunkMeta{ | ||
{ | ||
MinTime: 1, | ||
MaxTime: 7, | ||
Checksum: 4, | ||
}, | ||
}, | ||
}, | ||
} | ||
|
||
mgr := NewHeadManager(log.NewNopLogger(), dir, nil, noopTSDBManager{}) | ||
// This bit is normally handled by the Start() fn, but we're testing a smaller surface area | ||
// so ensure our dirs exist | ||
for _, d := range managerRequiredDirs(dir) { | ||
require.Nil(t, util.EnsureDirectory(d)) | ||
} | ||
|
||
// Call Rotate() to ensure the new head tenant heads exist, etc | ||
require.Nil(t, mgr.Rotate(now)) | ||
|
||
// now build a WAL independently to test recovery | ||
w, err := newHeadWAL(log.NewNopLogger(), walPath(mgr.dir, now), now) | ||
require.Nil(t, err) | ||
|
||
for i, c := range cases { | ||
require.Nil(t, w.Log(&WALRecord{ | ||
UserID: c.User, | ||
Series: record.RefSeries{ | ||
Ref: chunks.HeadSeriesRef(i), | ||
Labels: c.Labels, | ||
}, | ||
Chks: ChunkMetasRecord{ | ||
Chks: c.Chunks, | ||
Ref: uint64(i), | ||
}, | ||
})) | ||
} | ||
|
||
require.Nil(t, w.Stop()) | ||
|
||
grp, ok, err := walsForPeriod(mgr.dir, mgr.period, mgr.period.PeriodFor(now)) | ||
require.Nil(t, err) | ||
require.True(t, ok) | ||
require.Equal(t, 1, len(grp.wals)) | ||
require.Nil(t, recoverHead(mgr.dir, mgr.activeHeads, grp.wals)) | ||
|
||
for _, c := range cases { | ||
refs, err := mgr.GetChunkRefs( | ||
context.Background(), | ||
c.User, | ||
0, math.MaxInt64, | ||
nil, nil, | ||
labels.MustNewMatcher(labels.MatchRegexp, "foo", ".+"), | ||
) | ||
require.Nil(t, err) | ||
require.Equal(t, chunkMetasToChunkRefs(c.User, c.Labels.Hash(), c.Chunks), refs) | ||
} | ||
|
||
} | ||
|
||
// test mgr recover from multiple wals across multiple periods | ||
func Test_HeadManager_Lifecycle(t *testing.T) { | ||
dir := t.TempDir() | ||
curPeriod := time.Now() | ||
cases := []struct { | ||
Labels labels.Labels | ||
Chunks []index.ChunkMeta | ||
User string | ||
}{ | ||
{ | ||
User: "tenant1", | ||
Labels: mustParseLabels(`{foo="bar", bazz="buzz"}`), | ||
Chunks: []index.ChunkMeta{ | ||
{ | ||
MinTime: 1, | ||
MaxTime: 10, | ||
Checksum: 3, | ||
}, | ||
}, | ||
}, | ||
{ | ||
User: "tenant2", | ||
Labels: mustParseLabels(`{foo="bard", bazz="bozz", bonk="borb"}`), | ||
Chunks: []index.ChunkMeta{ | ||
{ | ||
MinTime: 1, | ||
MaxTime: 7, | ||
Checksum: 4, | ||
}, | ||
}, | ||
}, | ||
} | ||
|
||
mgr := NewHeadManager(log.NewNopLogger(), dir, nil, noopTSDBManager{}) | ||
w, err := newHeadWAL(log.NewNopLogger(), walPath(mgr.dir, curPeriod), curPeriod) | ||
require.Nil(t, err) | ||
|
||
// Write old WALs | ||
for i, c := range cases { | ||
require.Nil(t, w.Log(&WALRecord{ | ||
UserID: c.User, | ||
Series: record.RefSeries{ | ||
Ref: chunks.HeadSeriesRef(i), | ||
Labels: c.Labels, | ||
}, | ||
Chks: ChunkMetasRecord{ | ||
Chks: c.Chunks, | ||
Ref: uint64(i), | ||
}, | ||
})) | ||
} | ||
|
||
require.Nil(t, w.Stop()) | ||
|
||
// Start, ensuring recovery from old WALs | ||
require.Nil(t, mgr.Start()) | ||
// Ensure old WAL data is queryable | ||
for _, c := range cases { | ||
refs, err := mgr.GetChunkRefs( | ||
context.Background(), | ||
c.User, | ||
0, math.MaxInt64, | ||
nil, nil, | ||
labels.MustNewMatcher(labels.MatchRegexp, "foo", ".+"), | ||
) | ||
require.Nil(t, err) | ||
|
||
lbls := labels.NewBuilder(c.Labels) | ||
lbls.Set(TenantLabel, c.User) | ||
require.Equal(t, chunkMetasToChunkRefs(c.User, c.Labels.Hash(), c.Chunks), refs) | ||
} | ||
|
||
// Add data | ||
newCase := struct { | ||
Labels labels.Labels | ||
Chunks []index.ChunkMeta | ||
User string | ||
}{ | ||
User: "tenant3", | ||
Labels: mustParseLabels(`{foo="bard", other="hi"}`), | ||
Chunks: []index.ChunkMeta{ | ||
{ | ||
MinTime: 1, | ||
MaxTime: 7, | ||
Checksum: 4, | ||
}, | ||
}, | ||
} | ||
|
||
require.Nil(t, mgr.Append(newCase.User, newCase.Labels, newCase.Chunks)) | ||
|
||
// Ensure old + new data is queryable | ||
for _, c := range append(cases, newCase) { | ||
refs, err := mgr.GetChunkRefs( | ||
context.Background(), | ||
c.User, | ||
0, math.MaxInt64, | ||
nil, nil, | ||
labels.MustNewMatcher(labels.MatchRegexp, "foo", ".+"), | ||
) | ||
require.Nil(t, err) | ||
|
||
lbls := labels.NewBuilder(c.Labels) | ||
lbls.Set(TenantLabel, c.User) | ||
require.Equal(t, chunkMetasToChunkRefs(c.User, c.Labels.Hash(), c.Chunks), refs) | ||
} | ||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,213 @@ | ||
package tsdb | ||
|
||
import ( | ||
"time" | ||
|
||
"github.com/go-kit/log" | ||
"github.com/pkg/errors" | ||
"github.com/prometheus/prometheus/tsdb/record" | ||
"github.com/prometheus/prometheus/tsdb/wal" | ||
|
||
"github.com/grafana/loki/pkg/storage/stores/tsdb/index" | ||
"github.com/grafana/loki/pkg/util/encoding" | ||
) | ||
|
||
type WAL interface { | ||
Start(time.Time) error | ||
Log(*WALRecord) error | ||
Stop() error | ||
} | ||
|
||
// TODO(owen-d): There are probably some performance gains to be had by utilizing | ||
// pools here, but in the interest of implementation time and given chunks aren't | ||
// flushed often (generally ~5/s), this seems fine. | ||
// This may also be applicable to varint encoding. | ||
|
||
// 128KB | ||
// The segment sizes are kept small for the TSDB Head here because | ||
// we only store chunk references | ||
const walSegmentSize = 128 << 10 | ||
|
||
type RecordType byte | ||
|
||
// By prefixing records with versions, we can easily update our wal schema | ||
const ( | ||
// FirstWrite is a special record type written once | ||
// at the beginning of every WAL. It records the system time | ||
// when the WAL was created. This is used to determine when to rotate | ||
// WALs and persists across restarts. | ||
WalRecordSeries RecordType = iota | ||
WalRecordChunks | ||
) | ||
|
||
type WALRecord struct { | ||
UserID string | ||
Series record.RefSeries | ||
Chks ChunkMetasRecord | ||
} | ||
|
||
type ChunkMetasRecord struct { | ||
Chks index.ChunkMetas | ||
Ref uint64 | ||
} | ||
|
||
func (r *WALRecord) encodeSeries(b []byte) []byte { | ||
buf := encoding.EncWith(b) | ||
buf.PutByte(byte(WalRecordSeries)) | ||
buf.PutUvarintStr(r.UserID) | ||
|
||
var enc record.Encoder | ||
// The 'encoded' already has the type header and userID here, hence re-using | ||
// the remaining part of the slice (i.e. encoded[len(encoded):])) to encode the series. | ||
encoded := buf.Get() | ||
encoded = append(encoded, enc.Series([]record.RefSeries{r.Series}, encoded[len(encoded):])...) | ||
|
||
return encoded | ||
} | ||
|
||
func (r *WALRecord) encodeChunks(b []byte) []byte { | ||
buf := encoding.EncWith(b) | ||
buf.PutByte(byte(WalRecordChunks)) | ||
buf.PutUvarintStr(r.UserID) | ||
buf.PutBE64(r.Chks.Ref) | ||
buf.PutUvarint(len(r.Chks.Chks)) | ||
|
||
for _, chk := range r.Chks.Chks { | ||
buf.PutBE64(uint64(chk.MinTime)) | ||
buf.PutBE64(uint64(chk.MaxTime)) | ||
buf.PutBE32(chk.Checksum) | ||
buf.PutBE32(chk.KB) | ||
buf.PutBE32(chk.Entries) | ||
} | ||
|
||
return buf.Get() | ||
} | ||
|
||
func decodeChunks(b []byte, rec *WALRecord) error { | ||
if len(b) == 0 { | ||
return nil | ||
} | ||
|
||
dec := encoding.DecWith(b) | ||
|
||
rec.Chks.Ref = dec.Be64() | ||
if err := dec.Err(); err != nil { | ||
return errors.Wrap(err, "decoding series ref") | ||
} | ||
|
||
ln := dec.Uvarint() | ||
if err := dec.Err(); err != nil { | ||
return errors.Wrap(err, "decoding number of chunks") | ||
} | ||
// allocate space for the required number of chunks | ||
rec.Chks.Chks = make(index.ChunkMetas, 0, ln) | ||
|
||
for len(dec.B) > 0 && dec.Err() == nil { | ||
rec.Chks.Chks = append(rec.Chks.Chks, index.ChunkMeta{ | ||
MinTime: dec.Be64int64(), | ||
MaxTime: dec.Be64int64(), | ||
Checksum: dec.Be32(), | ||
KB: dec.Be32(), | ||
Entries: dec.Be32(), | ||
}) | ||
} | ||
|
||
if err := dec.Err(); err != nil { | ||
return errors.Wrap(err, "decoding chunk metas") | ||
} | ||
|
||
return nil | ||
} | ||
|
||
func decodeWALRecord(b []byte, walRec *WALRecord) error { | ||
var ( | ||
userID string | ||
dec record.Decoder | ||
|
||
decbuf = encoding.DecWith(b) | ||
t = RecordType(decbuf.Byte()) | ||
) | ||
|
||
switch t { | ||
case WalRecordSeries: | ||
userID = decbuf.UvarintStr() | ||
rSeries, err := dec.Series(decbuf.B, nil) | ||
if err != nil { | ||
return errors.Wrap(err, "decoding head series") | ||
} | ||
// unlike tsdb, we only add one series per record. | ||
if len(rSeries) > 1 { | ||
return errors.New("more than one series detected in tsdb head wal record") | ||
} | ||
if len(rSeries) == 1 { | ||
walRec.Series = rSeries[0] | ||
} | ||
case WalRecordChunks: | ||
userID = decbuf.UvarintStr() | ||
if err := decodeChunks(decbuf.B, walRec); err != nil { | ||
return err | ||
} | ||
default: | ||
return errors.New("unknown record type") | ||
} | ||
|
||
if decbuf.Err() != nil { | ||
return decbuf.Err() | ||
} | ||
|
||
walRec.UserID = userID | ||
return nil | ||
} | ||
|
||
// the headWAL, unlike Head, is multi-tenant. This is just to avoid the need to maintain | ||
// an open segment per tenant (potentially thousands of them) | ||
type headWAL struct { | ||
initialized time.Time | ||
log log.Logger | ||
wal *wal.WAL | ||
} | ||
|
||
func newHeadWAL(log log.Logger, dir string, t time.Time) (*headWAL, error) { | ||
// NB: if we use a non-nil Prometheus Registerer, ensure | ||
// that the underlying metrics won't conflict with existing WAL metrics in the ingester. | ||
// Likely, this can be done by adding extra label(s) | ||
wal, err := wal.NewSize(log, nil, dir, walSegmentSize, false) | ||
if err != nil { | ||
return nil, err | ||
} | ||
|
||
return &headWAL{ | ||
initialized: t, | ||
log: log, | ||
wal: wal, | ||
}, nil | ||
} | ||
|
||
func (w *headWAL) Stop() error { | ||
return w.wal.Close() | ||
} | ||
|
||
func (w *headWAL) Log(record *WALRecord) error { | ||
if record == nil { | ||
return nil | ||
} | ||
|
||
var buf []byte | ||
|
||
// Always write series before chunks | ||
if len(record.Series.Labels) > 0 { | ||
buf = record.encodeSeries(buf[:0]) | ||
if err := w.wal.Log(buf); err != nil { | ||
return err | ||
} | ||
} | ||
|
||
if len(record.Chks.Chks) > 0 { | ||
buf = record.encodeChunks(buf[:0]) | ||
if err := w.wal.Log(buf); err != nil { | ||
return err | ||
} | ||
} | ||
|
||
return nil | ||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,102 @@ | ||
package tsdb | ||
|
||
import ( | ||
"testing" | ||
"time" | ||
|
||
"github.com/go-kit/log" | ||
"github.com/prometheus/prometheus/tsdb/chunks" | ||
"github.com/prometheus/prometheus/tsdb/record" | ||
"github.com/stretchr/testify/require" | ||
|
||
"github.com/grafana/loki/pkg/storage/stores/tsdb/index" | ||
) | ||
|
||
func Test_Encoding_Series(t *testing.T) { | ||
record := &WALRecord{ | ||
UserID: "foo", | ||
Series: record.RefSeries{ | ||
Ref: chunks.HeadSeriesRef(1), | ||
Labels: mustParseLabels(`{foo="bar"}`), | ||
}, | ||
} | ||
buf := record.encodeSeries(nil) | ||
decoded := &WALRecord{} | ||
|
||
err := decodeWALRecord(buf, decoded) | ||
require.Nil(t, err) | ||
require.Equal(t, record, decoded) | ||
} | ||
|
||
func Test_Encoding_Chunks(t *testing.T) { | ||
record := &WALRecord{ | ||
UserID: "foo", | ||
Chks: ChunkMetasRecord{ | ||
Ref: 1, | ||
Chks: index.ChunkMetas{ | ||
{ | ||
Checksum: 1, | ||
MinTime: 1, | ||
MaxTime: 4, | ||
KB: 5, | ||
Entries: 6, | ||
}, | ||
{ | ||
Checksum: 2, | ||
MinTime: 5, | ||
MaxTime: 10, | ||
KB: 7, | ||
Entries: 8, | ||
}, | ||
}, | ||
}, | ||
} | ||
buf := record.encodeChunks(nil) | ||
decoded := &WALRecord{} | ||
|
||
err := decodeWALRecord(buf, decoded) | ||
require.Nil(t, err) | ||
require.Equal(t, record, decoded) | ||
} | ||
|
||
func Test_HeadWALLog(t *testing.T) { | ||
dir := t.TempDir() | ||
w, err := newHeadWAL(log.NewNopLogger(), dir, time.Now()) | ||
require.Nil(t, err) | ||
|
||
newSeries := &WALRecord{ | ||
UserID: "foo", | ||
Series: record.RefSeries{Ref: 1, Labels: mustParseLabels(`{foo="bar"}`)}, | ||
Chks: ChunkMetasRecord{ | ||
Chks: []index.ChunkMeta{ | ||
{ | ||
Checksum: 1, | ||
MinTime: 1, | ||
MaxTime: 10, | ||
KB: 5, | ||
Entries: 50, | ||
}, | ||
}, | ||
Ref: 1, | ||
}, | ||
} | ||
require.Nil(t, w.Log(newSeries)) | ||
|
||
chunksOnly := &WALRecord{ | ||
UserID: "foo", | ||
Chks: ChunkMetasRecord{ | ||
Chks: []index.ChunkMeta{ | ||
{ | ||
Checksum: 2, | ||
MinTime: 5, | ||
MaxTime: 100, | ||
KB: 3, | ||
Entries: 25, | ||
}, | ||
}, | ||
Ref: 1, | ||
}, | ||
} | ||
require.Nil(t, w.Log(chunksOnly)) | ||
require.Nil(t, w.Stop()) | ||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,202 @@ | ||
package tsdb | ||
|
||
import ( | ||
"fmt" | ||
"path" | ||
"path/filepath" | ||
"strconv" | ||
"strings" | ||
"time" | ||
|
||
"github.com/prometheus/common/model" | ||
|
||
"github.com/grafana/loki/pkg/storage/stores/tsdb/index" | ||
) | ||
|
||
// Identifier can resolve an index to a name (in object storage) | ||
// and a path (on disk) | ||
type Identifier interface { | ||
Name() string | ||
Path() string | ||
} | ||
|
||
// identifierFromPath will detect whether this is a single or multitenant TSDB | ||
func identifierFromPath(p string) (Identifier, error) { | ||
multiID, multitenant := parseMultitenantTSDBPath(p) | ||
if multitenant { | ||
parent := filepath.Dir(p) | ||
return newPrefixedIdentifier(multiID, parent, ""), nil | ||
} | ||
|
||
id, parent, ok := parseSingleTenantTSDBPath(p) | ||
if !ok { | ||
return nil, fmt.Errorf("invalid tsdb path: %s", p) | ||
} | ||
|
||
return newPrefixedIdentifier(id, parent, ""), nil | ||
} | ||
|
||
func newPrefixedIdentifier(id Identifier, path, name string) prefixedIdentifier { | ||
return prefixedIdentifier{ | ||
Identifier: id, | ||
parentPath: path, | ||
parentName: name, | ||
} | ||
} | ||
|
||
// parentIdentifier wraps an Identifier and prepends to its methods | ||
type prefixedIdentifier struct { | ||
parentPath, parentName string | ||
Identifier | ||
} | ||
|
||
func (p prefixedIdentifier) Path() string { | ||
return filepath.Join(p.parentPath, p.Identifier.Path()) | ||
} | ||
|
||
func (p prefixedIdentifier) Name() string { | ||
return path.Join(p.parentName, p.Identifier.Name()) | ||
} | ||
|
||
func newSuffixedIdentifier(id Identifier, pathSuffix string) suffixedIdentifier { | ||
return suffixedIdentifier{ | ||
pathSuffix: pathSuffix, | ||
Identifier: id, | ||
} | ||
} | ||
|
||
// Generally useful for gzip extensions | ||
type suffixedIdentifier struct { | ||
pathSuffix string | ||
Identifier | ||
} | ||
|
||
func (s suffixedIdentifier) Path() string { | ||
return s.Identifier.Path() + s.pathSuffix | ||
} | ||
|
||
// Identifier has all the information needed to resolve a TSDB index | ||
// Notably this abstracts away OS path separators, etc. | ||
type SingleTenantTSDBIdentifier struct { | ||
Tenant string | ||
From, Through model.Time | ||
Checksum uint32 | ||
} | ||
|
||
func (i SingleTenantTSDBIdentifier) str() string { | ||
return fmt.Sprintf( | ||
"%s-%d-%d-%x.tsdb", | ||
index.IndexFilename, | ||
i.From, | ||
i.Through, | ||
i.Checksum, | ||
) | ||
} | ||
|
||
func (i SingleTenantTSDBIdentifier) Name() string { | ||
return path.Join(i.Tenant, i.str()) | ||
} | ||
|
||
func (i SingleTenantTSDBIdentifier) Path() string { | ||
return filepath.Join(i.Tenant, i.str()) | ||
} | ||
|
||
func parseSingleTenantTSDBPath(p string) (id SingleTenantTSDBIdentifier, parent string, ok bool) { | ||
// parsing as multitenant didn't work, so try single tenant | ||
file := filepath.Base(p) | ||
parents := filepath.Dir(p) | ||
pathPrefix := filepath.Dir(parents) | ||
tenant := filepath.Base(parents) | ||
|
||
// no tenant was provided | ||
if tenant == "." { | ||
return | ||
} | ||
|
||
// incorrect suffix | ||
trimmed := strings.TrimSuffix(file, ".tsdb") | ||
if trimmed == file { | ||
return | ||
} | ||
|
||
elems := strings.Split(trimmed, "-") | ||
if len(elems) != 4 { | ||
return | ||
} | ||
|
||
if elems[0] != index.IndexFilename { | ||
return | ||
} | ||
|
||
from, err := strconv.ParseInt(elems[1], 10, 64) | ||
if err != nil { | ||
return | ||
} | ||
|
||
through, err := strconv.ParseInt(elems[2], 10, 64) | ||
if err != nil { | ||
return | ||
} | ||
|
||
checksum, err := strconv.ParseInt(elems[3], 16, 32) | ||
if err != nil { | ||
return | ||
} | ||
|
||
return SingleTenantTSDBIdentifier{ | ||
Tenant: tenant, | ||
From: model.Time(from), | ||
Through: model.Time(through), | ||
Checksum: uint32(checksum), | ||
}, pathPrefix, true | ||
|
||
} | ||
|
||
type MultitenantTSDBIdentifier struct { | ||
nodeName string | ||
ts time.Time | ||
} | ||
|
||
func (id MultitenantTSDBIdentifier) Name() string { | ||
return fmt.Sprintf("%d-%s.tsdb", id.ts.Unix(), id.nodeName) | ||
} | ||
|
||
func (id MultitenantTSDBIdentifier) Path() string { | ||
// There are no directories, so reuse name | ||
return id.Name() | ||
} | ||
|
||
func parseMultitenantTSDBPath(p string) (id MultitenantTSDBIdentifier, ok bool) { | ||
cleaned := filepath.Base(p) | ||
return parseMultitenantTSDBNameFromBase(cleaned) | ||
} | ||
|
||
func parseMultitenantTSDBName(p string) (id MultitenantTSDBIdentifier, ok bool) { | ||
cleaned := path.Base(p) | ||
return parseMultitenantTSDBNameFromBase(cleaned) | ||
} | ||
|
||
func parseMultitenantTSDBNameFromBase(name string) (res MultitenantTSDBIdentifier, ok bool) { | ||
|
||
trimmed := strings.TrimSuffix(name, ".tsdb") | ||
|
||
// incorrect suffix | ||
if trimmed == name { | ||
return | ||
} | ||
|
||
xs := strings.Split(trimmed, "-") | ||
if len(xs) != 2 { | ||
return | ||
} | ||
|
||
ts, err := strconv.Atoi(xs[0]) | ||
if err != nil { | ||
return | ||
} | ||
|
||
return MultitenantTSDBIdentifier{ | ||
ts: time.Unix(int64(ts), 0), | ||
nodeName: xs[1], | ||
}, true | ||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,57 @@ | ||
package tsdb | ||
|
||
import ( | ||
"testing" | ||
|
||
"github.com/stretchr/testify/require" | ||
) | ||
|
||
func TestParseSingleTenantTSDBPath(t *testing.T) { | ||
for _, tc := range []struct { | ||
desc string | ||
input string | ||
id SingleTenantTSDBIdentifier | ||
parent string | ||
ok bool | ||
}{ | ||
{ | ||
desc: "simple_works", | ||
input: "parent/fake/index-1-10-ff.tsdb", | ||
id: SingleTenantTSDBIdentifier{ | ||
Tenant: "fake", | ||
From: 1, | ||
Through: 10, | ||
Checksum: 255, | ||
}, | ||
parent: "parent", | ||
ok: true, | ||
}, | ||
{ | ||
desc: "no tenant dir", | ||
input: "index-1-10-ff.tsdb", | ||
ok: false, | ||
}, | ||
{ | ||
desc: "wrong index name", | ||
input: "fake/notindex-1-10-ff.tsdb", | ||
ok: false, | ||
}, | ||
{ | ||
desc: "wrong argument len", | ||
input: "fake/index-10-ff.tsdb", | ||
ok: false, | ||
}, | ||
{ | ||
desc: "wrong argument encoding", | ||
input: "fake/index-ff-10-ff.tsdb", | ||
ok: false, | ||
}, | ||
} { | ||
t.Run(tc.desc, func(t *testing.T) { | ||
id, parent, ok := parseSingleTenantTSDBPath(tc.input) | ||
require.Equal(t, tc.id, id) | ||
require.Equal(t, tc.parent, parent) | ||
require.Equal(t, tc.ok, ok) | ||
}) | ||
} | ||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,118 @@ | ||
package tsdb | ||
|
||
import ( | ||
"context" | ||
|
||
"github.com/prometheus/common/model" | ||
"github.com/prometheus/prometheus/model/labels" | ||
|
||
"github.com/grafana/loki/pkg/logproto" | ||
"github.com/grafana/loki/pkg/querier/astmapper" | ||
"github.com/grafana/loki/pkg/storage/chunk" | ||
"github.com/grafana/loki/pkg/storage/config" | ||
"github.com/grafana/loki/pkg/storage/stores/tsdb/index" | ||
) | ||
|
||
// implements stores.Index | ||
type IndexClient struct { | ||
schema config.SchemaConfig | ||
idx Index | ||
} | ||
|
||
func NewIndexClient(idx Index, pd config.PeriodConfig) *IndexClient { | ||
return &IndexClient{ | ||
schema: config.SchemaConfig{ | ||
Configs: []config.PeriodConfig{pd}, | ||
}, | ||
idx: idx, | ||
} | ||
} | ||
|
||
// TODO(owen-d): This is a hack for compatibility with how the current query-mapping works. | ||
// Historically, Loki will read the index shard factor and the query planner will inject shard | ||
// labels accordingly. | ||
// In the future, we should use dynamic sharding in TSDB to determine the shard factors | ||
// and we may no longer wish to send a shard label inside the queries, | ||
// but rather expose it as part of the stores.Index interface | ||
func (c *IndexClient) shard(matchers ...*labels.Matcher) ([]*labels.Matcher, *index.ShardAnnotation, error) { | ||
s, shardLabelIndex, err := astmapper.ShardFromMatchers(matchers) | ||
if err != nil { | ||
return nil, nil, err | ||
} | ||
|
||
var shard *index.ShardAnnotation | ||
if s != nil { | ||
matchers = append(matchers[:shardLabelIndex], matchers[shardLabelIndex+1:]...) | ||
shard = &index.ShardAnnotation{ | ||
Shard: uint32(s.Shard), | ||
Of: uint32(s.Of), | ||
} | ||
} | ||
|
||
return matchers, shard, err | ||
|
||
} | ||
|
||
// TODO(owen-d): synchronize logproto.ChunkRef and tsdb.ChunkRef so we don't have to convert. | ||
// They share almost the same fields, so we can add the missing `KB` field to the proto and then | ||
// use that within the tsdb package. | ||
func (c *IndexClient) GetChunkRefs(ctx context.Context, userID string, from, through model.Time, matchers ...*labels.Matcher) ([]logproto.ChunkRef, error) { | ||
matchers, shard, err := c.shard(matchers...) | ||
if err != nil { | ||
return nil, err | ||
} | ||
|
||
// TODO(owen-d): use a pool to reduce allocs here | ||
chks, err := c.idx.GetChunkRefs(ctx, userID, from, through, nil, shard, matchers...) | ||
if err != nil { | ||
return nil, err | ||
} | ||
|
||
refs := make([]logproto.ChunkRef, 0, len(chks)) | ||
for _, chk := range chks { | ||
refs = append(refs, logproto.ChunkRef{ | ||
Fingerprint: uint64(chk.Fingerprint), | ||
UserID: chk.User, | ||
From: chk.Start, | ||
Through: chk.End, | ||
Checksum: chk.Checksum, | ||
}) | ||
} | ||
|
||
return refs, err | ||
} | ||
|
||
func (c *IndexClient) GetSeries(ctx context.Context, userID string, from, through model.Time, matchers ...*labels.Matcher) ([]labels.Labels, error) { | ||
matchers, shard, err := c.shard(matchers...) | ||
if err != nil { | ||
return nil, err | ||
} | ||
|
||
xs, err := c.idx.Series(ctx, userID, from, through, nil, shard, matchers...) | ||
if err != nil { | ||
return nil, err | ||
} | ||
|
||
res := make([]labels.Labels, 0, len(xs)) | ||
for _, x := range xs { | ||
res = append(res, x.Labels) | ||
} | ||
return res, nil | ||
} | ||
|
||
// tsdb no longer uses the __metric_name__="logs" hack, so we can ignore metric names! | ||
func (c *IndexClient) LabelValuesForMetricName(ctx context.Context, userID string, from, through model.Time, metricName string, labelName string, matchers ...*labels.Matcher) ([]string, error) { | ||
return c.idx.LabelValues(ctx, userID, from, through, labelName, matchers...) | ||
} | ||
|
||
// tsdb no longer uses the __metric_name__="logs" hack, so we can ignore metric names! | ||
func (c *IndexClient) LabelNamesForMetricName(ctx context.Context, userID string, from, through model.Time, metricName string) ([]string, error) { | ||
return c.idx.LabelNames(ctx, userID, from, through) | ||
} | ||
|
||
// SetChunkFilterer sets a chunk filter to be used when retrieving chunks. | ||
// This is only used for GetSeries implementation. | ||
// Todo we might want to pass it as a parameter to GetSeries instead. | ||
func (c *IndexClient) SetChunkFilterer(chunkFilter chunk.RequestChunkFilterer) { | ||
c.idx.SetChunkFilterer(chunkFilter) | ||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,66 @@ | ||
package tsdb | ||
|
||
import ( | ||
"context" | ||
|
||
"github.com/prometheus/common/model" | ||
"github.com/prometheus/prometheus/model/labels" | ||
|
||
"github.com/grafana/loki/pkg/storage/chunk" | ||
"github.com/grafana/loki/pkg/storage/stores/tsdb/index" | ||
) | ||
|
||
// Index adapter for a function which returns an index when queried. | ||
type LazyIndex func() (Index, error) | ||
|
||
func (f LazyIndex) Bounds() (model.Time, model.Time) { | ||
i, err := f() | ||
if err != nil { | ||
return 0, 0 | ||
} | ||
return i.Bounds() | ||
} | ||
|
||
func (f LazyIndex) SetChunkFilterer(chunkFilter chunk.RequestChunkFilterer) { | ||
i, err := f() | ||
if err == nil { | ||
i.SetChunkFilterer(chunkFilter) | ||
} | ||
} | ||
|
||
func (f LazyIndex) Close() error { | ||
i, err := f() | ||
if err != nil { | ||
return err | ||
} | ||
return i.Close() | ||
} | ||
|
||
func (f LazyIndex) GetChunkRefs(ctx context.Context, userID string, from, through model.Time, res []ChunkRef, shard *index.ShardAnnotation, matchers ...*labels.Matcher) ([]ChunkRef, error) { | ||
i, err := f() | ||
if err != nil { | ||
return nil, err | ||
} | ||
return i.GetChunkRefs(ctx, userID, from, through, res, shard, matchers...) | ||
} | ||
func (f LazyIndex) Series(ctx context.Context, userID string, from, through model.Time, res []Series, shard *index.ShardAnnotation, matchers ...*labels.Matcher) ([]Series, error) { | ||
i, err := f() | ||
if err != nil { | ||
return nil, err | ||
} | ||
return i.Series(ctx, userID, from, through, res, shard, matchers...) | ||
} | ||
func (f LazyIndex) LabelNames(ctx context.Context, userID string, from, through model.Time, matchers ...*labels.Matcher) ([]string, error) { | ||
i, err := f() | ||
if err != nil { | ||
return nil, err | ||
} | ||
return i.LabelNames(ctx, userID, from, through, matchers...) | ||
} | ||
func (f LazyIndex) LabelValues(ctx context.Context, userID string, from, through model.Time, name string, matchers ...*labels.Matcher) ([]string, error) { | ||
i, err := f() | ||
if err != nil { | ||
return nil, err | ||
} | ||
return i.LabelValues(ctx, userID, from, through, name, matchers...) | ||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,284 @@ | ||
package tsdb | ||
|
||
import ( | ||
"context" | ||
"fmt" | ||
"math" | ||
"path/filepath" | ||
"sync" | ||
"time" | ||
|
||
"github.com/go-kit/log" | ||
"github.com/go-kit/log/level" | ||
"github.com/pkg/errors" | ||
"github.com/prometheus/common/model" | ||
"github.com/prometheus/prometheus/model/labels" | ||
|
||
"github.com/grafana/loki/pkg/storage/chunk" | ||
"github.com/grafana/loki/pkg/storage/stores/indexshipper" | ||
shipper_index "github.com/grafana/loki/pkg/storage/stores/indexshipper/index" | ||
"github.com/grafana/loki/pkg/storage/stores/tsdb/index" | ||
) | ||
|
||
// nolint:revive | ||
// TSDBManager wraps the index shipper and writes/manages | ||
// TSDB files on disk | ||
type TSDBManager interface { | ||
Index | ||
// Builds a new TSDB file from a set of WALs | ||
BuildFromWALs(time.Time, []WALIdentifier) error | ||
} | ||
|
||
/* | ||
tsdbManager is responsible for: | ||
* Turning WALs into optimized multi-tenant TSDBs when requested | ||
* Serving reads from these TSDBs | ||
* Shipping them to remote storage | ||
* Keeping them available for querying | ||
* Removing old TSDBs which are no longer needed | ||
*/ | ||
type tsdbManager struct { | ||
indexPeriod time.Duration | ||
nodeName string // node name | ||
log log.Logger | ||
dir string | ||
metrics *Metrics | ||
|
||
sync.RWMutex | ||
|
||
chunkFilter chunk.RequestChunkFilterer | ||
shipper indexshipper.IndexShipper | ||
} | ||
|
||
func NewTSDBManager( | ||
nodeName, | ||
dir string, | ||
shipper indexshipper.IndexShipper, | ||
indexPeriod time.Duration, | ||
logger log.Logger, | ||
metrics *Metrics, | ||
) TSDBManager { | ||
return &tsdbManager{ | ||
indexPeriod: indexPeriod, | ||
nodeName: nodeName, | ||
log: log.With(logger, "component", "tsdb-manager"), | ||
dir: dir, | ||
metrics: metrics, | ||
shipper: shipper, | ||
} | ||
} | ||
|
||
func (m *tsdbManager) BuildFromWALs(t time.Time, ids []WALIdentifier) (err error) { | ||
level.Debug(m.log).Log("msg", "building WALs", "n", len(ids), "ts", t) | ||
// get relevant wals | ||
// iterate them, build tsdb in scratch dir | ||
defer func() { | ||
m.metrics.tsdbCreationsTotal.Inc() | ||
if err != nil { | ||
m.metrics.tsdbCreationFailures.Inc() | ||
} | ||
}() | ||
|
||
level.Debug(m.log).Log("msg", "recovering tenant heads") | ||
tmp := newTenantHeads(t, defaultHeadManagerStripeSize, m.metrics, m.log) | ||
if err = recoverHead(m.dir, tmp, ids); err != nil { | ||
return errors.Wrap(err, "building TSDB from WALs") | ||
} | ||
|
||
periods := make(map[int]*Builder) | ||
|
||
if err := tmp.forAll(func(user string, ls labels.Labels, chks index.ChunkMetas) { | ||
|
||
// chunks may overlap index period bounds, in which case they're written to multiple | ||
pds := make(map[int]index.ChunkMetas) | ||
for _, chk := range chks { | ||
for _, bucket := range indexBuckets(m.indexPeriod, chk.From(), chk.Through()) { | ||
pds[bucket] = append(pds[bucket], chk) | ||
} | ||
} | ||
|
||
// Embed the tenant label into TSDB | ||
lb := labels.NewBuilder(ls) | ||
lb.Set(TenantLabel, user) | ||
withTenant := lb.Labels() | ||
|
||
// Add the chunks to all relevant builders | ||
for pd, matchingChks := range pds { | ||
b, ok := periods[pd] | ||
if !ok { | ||
b = NewBuilder() | ||
periods[pd] = b | ||
} | ||
|
||
b.AddSeries( | ||
withTenant, | ||
// use the fingerprint without the added tenant label | ||
// so queries route to the chunks which actually exist. | ||
model.Fingerprint(ls.Hash()), | ||
matchingChks, | ||
) | ||
} | ||
|
||
}); err != nil { | ||
level.Error(m.log).Log("err", err.Error(), "msg", "building TSDB from WALs") | ||
return err | ||
} | ||
|
||
for p, b := range periods { | ||
|
||
dstDir := filepath.Join(managerMultitenantDir(m.dir), fmt.Sprint(p)) | ||
dst := newPrefixedIdentifier( | ||
MultitenantTSDBIdentifier{ | ||
nodeName: m.nodeName, | ||
ts: t, | ||
}, | ||
dstDir, | ||
"", | ||
) | ||
|
||
level.Debug(m.log).Log("msg", "building tsdb for period", "pd", p, "dst", dst.Path()) | ||
// build+move tsdb to multitenant dir | ||
start := time.Now() | ||
_, err = b.Build( | ||
context.Background(), | ||
managerScratchDir(m.dir), | ||
func(from, through model.Time, checksum uint32) Identifier { | ||
return dst | ||
}, | ||
) | ||
if err != nil { | ||
return err | ||
} | ||
|
||
level.Debug(m.log).Log("msg", "finished building tsdb for period", "pd", p, "dst", dst.Path(), "duration", time.Since(start)) | ||
|
||
loaded, err := NewShippableTSDBFile(dst, false) | ||
if err != nil { | ||
return err | ||
} | ||
|
||
if err := m.shipper.AddIndex(fmt.Sprintf("%d", p), "", loaded); err != nil { | ||
return err | ||
} | ||
} | ||
|
||
return nil | ||
} | ||
|
||
func indexBuckets(indexPeriod time.Duration, from, through model.Time) (res []int) { | ||
start := from.Time().UnixNano() / int64(indexPeriod) | ||
end := through.Time().UnixNano() / int64(indexPeriod) | ||
for cur := start; cur <= end; cur++ { | ||
res = append(res, int(cur)) | ||
} | ||
return | ||
} | ||
|
||
func (m *tsdbManager) indices(ctx context.Context, from, through model.Time, user string) (Index, error) { | ||
var indices []Index | ||
|
||
// Ensure we query both per tenant and multitenant TSDBs | ||
|
||
for _, bkt := range indexBuckets(m.indexPeriod, from, through) { | ||
if err := m.shipper.ForEach(ctx, fmt.Sprintf("%d", bkt), user, func(idx shipper_index.Index) error { | ||
_, multitenant := parseMultitenantTSDBName(idx.Name()) | ||
impl, ok := idx.(Index) | ||
if !ok { | ||
return fmt.Errorf("unexpected shipper index type: %T", idx) | ||
} | ||
if multitenant { | ||
indices = append(indices, NewMultiTenantIndex(impl)) | ||
} else { | ||
indices = append(indices, impl) | ||
} | ||
return nil | ||
}); err != nil { | ||
return nil, err | ||
} | ||
|
||
} | ||
|
||
if len(indices) == 0 { | ||
return NoopIndex{}, nil | ||
} | ||
idx, err := NewMultiIndex(indices...) | ||
if err != nil { | ||
return nil, err | ||
} | ||
|
||
if m.chunkFilter != nil { | ||
idx.SetChunkFilterer(m.chunkFilter) | ||
} | ||
return idx, nil | ||
} | ||
|
||
// TODO(owen-d): how to better implement this? | ||
// setting 0->maxint will force the tsdbmanager to always query | ||
// underlying tsdbs, which is safe, but can we optimize this? | ||
func (m *tsdbManager) Bounds() (model.Time, model.Time) { | ||
return 0, math.MaxInt64 | ||
} | ||
|
||
func (m *tsdbManager) SetChunkFilterer(chunkFilter chunk.RequestChunkFilterer) { | ||
m.chunkFilter = chunkFilter | ||
} | ||
|
||
// Close implements Index.Close, but we offload this responsibility | ||
// to the index shipper | ||
func (m *tsdbManager) Close() error { | ||
return nil | ||
} | ||
|
||
func (m *tsdbManager) GetChunkRefs(ctx context.Context, userID string, from, through model.Time, res []ChunkRef, shard *index.ShardAnnotation, matchers ...*labels.Matcher) ([]ChunkRef, error) { | ||
idx, err := m.indices(ctx, from, through, userID) | ||
if err != nil { | ||
return nil, err | ||
} | ||
matchers = withoutNameLabel(matchers) | ||
return idx.GetChunkRefs(ctx, userID, from, through, res, shard, matchers...) | ||
} | ||
|
||
func (m *tsdbManager) Series(ctx context.Context, userID string, from, through model.Time, res []Series, shard *index.ShardAnnotation, matchers ...*labels.Matcher) ([]Series, error) { | ||
idx, err := m.indices(ctx, from, through, userID) | ||
if err != nil { | ||
return nil, err | ||
} | ||
matchers = withoutNameLabel(matchers) | ||
return idx.Series(ctx, userID, from, through, res, shard, matchers...) | ||
} | ||
|
||
func (m *tsdbManager) LabelNames(ctx context.Context, userID string, from, through model.Time, matchers ...*labels.Matcher) ([]string, error) { | ||
idx, err := m.indices(ctx, from, through, userID) | ||
if err != nil { | ||
return nil, err | ||
} | ||
matchers = withoutNameLabel(matchers) | ||
return idx.LabelNames(ctx, userID, from, through, matchers...) | ||
} | ||
|
||
func (m *tsdbManager) LabelValues(ctx context.Context, userID string, from, through model.Time, name string, matchers ...*labels.Matcher) ([]string, error) { | ||
idx, err := m.indices(ctx, from, through, userID) | ||
if err != nil { | ||
return nil, err | ||
} | ||
matchers = withoutNameLabel(matchers) | ||
return idx.LabelValues(ctx, userID, from, through, name, matchers...) | ||
} | ||
|
||
// TODO(owen-d): in the future, handle this by preventing passing the __name__="logs" label | ||
// to TSDB indices at all. | ||
func withoutNameLabel(matchers []*labels.Matcher) []*labels.Matcher { | ||
if len(matchers) == 0 { | ||
return nil | ||
} | ||
|
||
dst := make([]*labels.Matcher, 0, len(matchers)-1) | ||
for _, m := range matchers { | ||
if m.Name == labels.MetricName { | ||
continue | ||
} | ||
dst = append(dst, m) | ||
} | ||
|
||
return dst | ||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,74 @@ | ||
package tsdb | ||
|
||
import ( | ||
"context" | ||
|
||
"github.com/prometheus/common/model" | ||
"github.com/prometheus/prometheus/model/labels" | ||
|
||
"github.com/grafana/loki/pkg/storage/chunk" | ||
"github.com/grafana/loki/pkg/storage/stores/tsdb/index" | ||
) | ||
|
||
// TenantLabel is part of the reserved label namespace (__ prefix) | ||
// It's used to create multi-tenant TSDBs (which do not have a tenancy concept) | ||
// These labels are stripped out during compaction to single-tenant TSDBs | ||
const TenantLabel = "__loki_tenant__" | ||
|
||
// MultiTenantIndex will inject a tenant label to it's queries | ||
// This works with pre-compacted TSDBs which aren't yet per tenant. | ||
type MultiTenantIndex struct { | ||
idx Index | ||
} | ||
|
||
func NewMultiTenantIndex(idx Index) *MultiTenantIndex { | ||
return &MultiTenantIndex{idx: idx} | ||
} | ||
|
||
func withTenantLabel(userID string, matchers []*labels.Matcher) []*labels.Matcher { | ||
cpy := make([]*labels.Matcher, len(matchers)) | ||
copy(cpy, matchers) | ||
cpy = append(cpy, labels.MustNewMatcher(labels.MatchEqual, TenantLabel, userID)) | ||
return cpy | ||
} | ||
|
||
func withoutTenantLabel(ls labels.Labels) labels.Labels { | ||
for i, l := range ls { | ||
if l.Name == TenantLabel { | ||
ls = append(ls[:i], ls[i+1:]...) | ||
break | ||
} | ||
} | ||
return ls | ||
} | ||
|
||
func (m *MultiTenantIndex) Bounds() (model.Time, model.Time) { return m.idx.Bounds() } | ||
|
||
func (m *MultiTenantIndex) SetChunkFilterer(chunkFilter chunk.RequestChunkFilterer) { | ||
m.idx.SetChunkFilterer(chunkFilter) | ||
} | ||
|
||
func (m *MultiTenantIndex) Close() error { return m.idx.Close() } | ||
|
||
func (m *MultiTenantIndex) GetChunkRefs(ctx context.Context, userID string, from, through model.Time, res []ChunkRef, shard *index.ShardAnnotation, matchers ...*labels.Matcher) ([]ChunkRef, error) { | ||
return m.idx.GetChunkRefs(ctx, userID, from, through, res, shard, withTenantLabel(userID, matchers)...) | ||
} | ||
|
||
func (m *MultiTenantIndex) Series(ctx context.Context, userID string, from, through model.Time, res []Series, shard *index.ShardAnnotation, matchers ...*labels.Matcher) ([]Series, error) { | ||
xs, err := m.idx.Series(ctx, userID, from, through, res, shard, withTenantLabel(userID, matchers)...) | ||
if err != nil { | ||
return nil, err | ||
} | ||
for i := range xs { | ||
xs[i].Labels = withoutTenantLabel(xs[i].Labels) | ||
} | ||
return xs, nil | ||
} | ||
|
||
func (m *MultiTenantIndex) LabelNames(ctx context.Context, userID string, from, through model.Time, matchers ...*labels.Matcher) ([]string, error) { | ||
return m.idx.LabelNames(ctx, userID, from, through, withTenantLabel(userID, matchers)...) | ||
} | ||
|
||
func (m *MultiTenantIndex) LabelValues(ctx context.Context, userID string, from, through model.Time, name string, matchers ...*labels.Matcher) ([]string, error) { | ||
return m.idx.LabelValues(ctx, userID, from, through, name, withTenantLabel(userID, matchers)...) | ||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Oops, something went wrong.