From 723118124cefe2fa6ab135b2d8d7afb456d425a9 Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Fri, 8 Apr 2022 15:47:15 -0400 Subject: [PATCH 01/85] begins speccing out TSDB Head --- pkg/storage/tsdb/head.go | 171 ++++++++++++++++++++++++++++++++ pkg/storage/tsdb/index/chunk.go | 22 ++++ 2 files changed, 193 insertions(+) create mode 100644 pkg/storage/tsdb/head.go diff --git a/pkg/storage/tsdb/head.go b/pkg/storage/tsdb/head.go new file mode 100644 index 0000000000000..9ebba91011570 --- /dev/null +++ b/pkg/storage/tsdb/head.go @@ -0,0 +1,171 @@ +package tsdb + +import ( + "sync" + + "github.com/go-kit/log" + "github.com/grafana/loki/pkg/storage/tsdb/index" + "github.com/prometheus/client_golang/prometheus" + "github.com/prometheus/common/model" + "github.com/prometheus/prometheus/model/labels" + "go.uber.org/atomic" +) + +const ( + // Note, this is significantly less than the stripe values used by Prometheus' stripeSeries. + // This is for two reasons. + // 1) Heads are per-tenant in Loki + // 2) Loki tends to have a few orders of magnitude less series per node than + // Prometheus|Cortex|Mimir. + defaultSeriesMapShards = 64 +) + +/* +Head is a per-tenant accumulator for index entries in memory. +It can be queried as an IndexReader and consumed to generate a TSDB index. +These are written to on the ingester component when chunks are flushed, +then written to disk as per tenant TSDB indices at the end of the WAL checkpointing cycle. +Every n cycles, they are compacted together and written to object storage. + +In turn, many `Head`s may be wrapped into a multi-tenant head. +This allows Loki to serve `GetChunkRefs` requests for _chunks_ which have been flushed +whereas the corresponding index has not yet been uploaded to object storage, +guaranteeing we maintain querying consistency for the entire data lifecycle. +*/ + +type HeadMetrics struct{} + +func NewHeadMetrics(r prometheus.Registerer) *HeadMetrics { + return &HeadMetrics{} +} + +type Head struct { + tenant string + numSeries atomic.Uint64 + minTime, maxTime atomic.Int64 // Current min and max of the samples included in the head. + + metrics *HeadMetrics + logger log.Logger + + series *seriesMap + + postings *index.MemPostings // Postings lists for terms. + + closedMtx sync.Mutex + closed bool +} + +func NewHead(tenant string, metrics *HeadMetrics, logger log.Logger) *Head { + return &Head{ + tenant: tenant, + metrics: metrics, + logger: logger, + series: newSeriesMap(), + postings: index.NewMemPostings(), + closedMtx: sync.Mutex{}, + closed: false, + } +} + +func (h *Head) updateMinMaxTime(mint, maxt int64) { + for { + lt := h.minTime.Load() + if mint >= lt { + break + } + if h.minTime.CAS(lt, mint) { + break + } + } + for { + ht := h.maxTime.Load() + if maxt <= ht { + break + } + if h.maxTime.CAS(ht, maxt) { + break + } + } +} + +// Note: chks must not be nil or zero-length +func (h *Head) Append(ls labels.Labels, chks index.ChunkMetas) { + from, through := chks.Bounds() + created := h.series.Append(ls, chks) + h.updateMinMaxTime(int64(from), int64(through)) + + if !created { + return + } + h.numSeries.Add(1) +} + +type seriesMap struct { + shards int + locks []sync.RWMutex + series []map[uint64]memSeriesList +} + +func newSeriesMap() *seriesMap { + s := &seriesMap{ + shards: defaultSeriesMapShards, + locks: make([]sync.RWMutex, defaultSeriesMapShards), + series: make([]map[uint64]memSeriesList, defaultSeriesMapShards), + } + for i := range s.series { + s.series[i] = map[uint64]memSeriesList{} + } + return s +} + +// Append adds chunks to the correct series and returns whether a new series was added +func (s *seriesMap) Append(ls labels.Labels, chks index.ChunkMetas) (created bool) { + fp := ls.Hash() + i := fp & uint64(s.shards) + mtx := &s.locks[i] + + mtx.Lock() + xs, ok := s.series[i][fp] + if !ok { + xs = memSeriesList{} + } + + series, ok := xs.Find(ls) + if !ok { + series = newMemSeries(ls, model.Fingerprint(fp)) + s.series[i][fp] = append(xs, series) + created = true + } + // Safe to unlock the seriesMap shard. + // We'll be using the series' mutex from now on. + mtx.Unlock() + series.Lock() + series.chks = append(series.chks, chks...) + series.Unlock() + return +} + +type memSeriesList []*memSeries + +func (ms memSeriesList) Find(ls labels.Labels) (*memSeries, bool) { + for _, x := range ms { + if labels.Equal(x.ls, ls) { + return x, true + } + } + return nil, false +} + +type memSeries struct { + sync.RWMutex + ls labels.Labels + fp model.Fingerprint + chks index.ChunkMetas +} + +func newMemSeries(ls labels.Labels, fp model.Fingerprint) *memSeries { + return &memSeries{ + ls: ls, + fp: fp, + } +} diff --git a/pkg/storage/tsdb/index/chunk.go b/pkg/storage/tsdb/index/chunk.go index fe5564072ef43..1b29340910ee8 100644 --- a/pkg/storage/tsdb/index/chunk.go +++ b/pkg/storage/tsdb/index/chunk.go @@ -26,6 +26,28 @@ type ChunkMetas []ChunkMeta func (c ChunkMetas) Len() int { return len(c) } func (c ChunkMetas) Swap(i, j int) { c[i], c[j] = c[j], c[i] } +func (c ChunkMetas) Bounds() (mint, maxt model.Time) { + ln := len(c) + if ln == 0 { + return + } + + mint, maxt = model.Time(c[0].MinTime), model.Time(c[ln-1].MaxTime) + // even when sorted, we need to check all chunks for maxt + // since we sort by (min, max, checksum). Therefore + // check mint here as well to ensure this works on unordered ChunkMetas too + for _, chk := range c { + from, through := chk.Bounds() + if mint > from { + mint = from + } + + if maxt < through { + maxt = through + } + } + return +} // Sort by (MinTime, MaxTime, Checksum) func (c ChunkMetas) Less(i, j int) bool { From 5e20970f7118b22fa38eaa0f098cce2c99a731be Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Fri, 8 Apr 2022 16:48:55 -0400 Subject: [PATCH 02/85] auto incrementing series ref + mempostings --- pkg/storage/tsdb/head.go | 138 +++++++++++++++++++++++++++------------ 1 file changed, 98 insertions(+), 40 deletions(-) diff --git a/pkg/storage/tsdb/head.go b/pkg/storage/tsdb/head.go index 9ebba91011570..fc9dedbecf71d 100644 --- a/pkg/storage/tsdb/head.go +++ b/pkg/storage/tsdb/head.go @@ -8,16 +8,22 @@ import ( "github.com/prometheus/client_golang/prometheus" "github.com/prometheus/common/model" "github.com/prometheus/prometheus/model/labels" + "github.com/prometheus/prometheus/storage" "go.uber.org/atomic" ) +/* +Disclaimer: This is largely inspired from Prometheus' TSDB Head, albeit +with significant changes (generally reductions rather than additions) to accommodate Loki +*/ + const ( // Note, this is significantly less than the stripe values used by Prometheus' stripeSeries. // This is for two reasons. // 1) Heads are per-tenant in Loki // 2) Loki tends to have a few orders of magnitude less series per node than // Prometheus|Cortex|Mimir. - defaultSeriesMapShards = 64 + defaultStripeSize = 64 ) /* @@ -33,6 +39,7 @@ whereas the corresponding index has not yet been uploaded to object storage, guaranteeing we maintain querying consistency for the entire data lifecycle. */ +// TODO(owen-d) type HeadMetrics struct{} func NewHeadMetrics(r prometheus.Registerer) *HeadMetrics { @@ -44,10 +51,14 @@ type Head struct { numSeries atomic.Uint64 minTime, maxTime atomic.Int64 // Current min and max of the samples included in the head. + // auto incrementing counter to uniquely identify series. This is also used + // in the MemPostings, but is eventually discarded when we create a real TSDB index. + lastSeriesID atomic.Uint64 + metrics *HeadMetrics logger log.Logger - series *seriesMap + series *stripeSeries postings *index.MemPostings // Postings lists for terms. @@ -60,7 +71,7 @@ func NewHead(tenant string, metrics *HeadMetrics, logger log.Logger) *Head { tenant: tenant, metrics: metrics, logger: logger, - series: newSeriesMap(), + series: newStripeSeries(), postings: index.NewMemPostings(), closedMtx: sync.Mutex{}, closed: false, @@ -91,81 +102,128 @@ func (h *Head) updateMinMaxTime(mint, maxt int64) { // Note: chks must not be nil or zero-length func (h *Head) Append(ls labels.Labels, chks index.ChunkMetas) { from, through := chks.Bounds() - created := h.series.Append(ls, chks) + var id uint64 + created := h.series.Append(ls, chks, func() *memSeries { + id = h.lastSeriesID.Inc() + return newMemSeries(id, ls) + }) h.updateMinMaxTime(int64(from), int64(through)) if !created { return } - h.numSeries.Add(1) + h.postings.Add(storage.SeriesRef(id), ls) + h.numSeries.Inc() +} + +// seriesHashmap is a simple hashmap for memSeries by their label set. It is built +// on top of a regular hashmap and holds a slice of series to resolve hash collisions. +// Its methods require the hash to be submitted with it to avoid re-computations throughout +// the code. +type seriesHashmap map[uint64][]*memSeries + +func (m seriesHashmap) get(hash uint64, ls labels.Labels) *memSeries { + for _, s := range m[hash] { + if labels.Equal(s.ls, ls) { + return s + } + } + return nil +} + +func (m seriesHashmap) set(hash uint64, s *memSeries) { + l := m[hash] + for i, prev := range l { + if labels.Equal(prev.ls, s.ls) { + l[i] = s + return + } + } + m[hash] = append(l, s) } -type seriesMap struct { +type stripeSeries struct { shards int locks []sync.RWMutex - series []map[uint64]memSeriesList + hashes []seriesHashmap + // Sharded by ref. A series ref is the value of `size` when the series was being newly added. + series []map[uint64]*memSeries } -func newSeriesMap() *seriesMap { - s := &seriesMap{ - shards: defaultSeriesMapShards, - locks: make([]sync.RWMutex, defaultSeriesMapShards), - series: make([]map[uint64]memSeriesList, defaultSeriesMapShards), +func newStripeSeries() *stripeSeries { + s := &stripeSeries{ + shards: defaultStripeSize, + locks: make([]sync.RWMutex, defaultStripeSize), + hashes: make([]seriesHashmap, defaultStripeSize), + series: make([]map[uint64]*memSeries, defaultStripeSize), + } + for i := range s.hashes { + s.hashes[i] = seriesHashmap{} } for i := range s.series { - s.series[i] = map[uint64]memSeriesList{} + s.series[i] = map[uint64]*memSeries{} } return s } +func (s *stripeSeries) getByID(id uint64) *memSeries { + i := id & uint64(s.shards-1) + + s.locks[i].RLock() + series := s.series[i][id] + s.locks[i].RUnlock() + + return series +} + +func (s *stripeSeries) getByHash(hash uint64, lset labels.Labels) *memSeries { + i := hash & uint64(s.shards-1) + + s.locks[i].RLock() + series := s.hashes[i].get(hash, lset) + s.locks[i].RUnlock() + + return series +} + // Append adds chunks to the correct series and returns whether a new series was added -func (s *seriesMap) Append(ls labels.Labels, chks index.ChunkMetas) (created bool) { +func (s *stripeSeries) Append( + ls labels.Labels, + chks index.ChunkMetas, + createFn func() *memSeries, +) (created bool) { fp := ls.Hash() - i := fp & uint64(s.shards) + i := fp & uint64(s.shards-1) mtx := &s.locks[i] mtx.Lock() - xs, ok := s.series[i][fp] - if !ok { - xs = memSeriesList{} - } - - series, ok := xs.Find(ls) - if !ok { - series = newMemSeries(ls, model.Fingerprint(fp)) - s.series[i][fp] = append(xs, series) + series := s.hashes[i].get(fp, ls) + if series == nil { + series = createFn() + s.hashes[i].set(fp, series) created = true } - // Safe to unlock the seriesMap shard. - // We'll be using the series' mutex from now on. mtx.Unlock() + series.Lock() series.chks = append(series.chks, chks...) series.Unlock() - return -} - -type memSeriesList []*memSeries -func (ms memSeriesList) Find(ls labels.Labels) (*memSeries, bool) { - for _, x := range ms { - if labels.Equal(x.ls, ls) { - return x, true - } - } - return nil, false + return } type memSeries struct { sync.RWMutex + ref uint64 // The unique reference within a *Head ls labels.Labels fp model.Fingerprint chks index.ChunkMetas } -func newMemSeries(ls labels.Labels, fp model.Fingerprint) *memSeries { +func newMemSeries(ref uint64, ls labels.Labels) *memSeries { return &memSeries{ - ls: ls, - fp: fp, + ref: ref, + ls: ls, + fp: model.Fingerprint(ls.Hash()), } } From a04a2fd5e2b3c45f12566c97467d695a04618e5d Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Fri, 8 Apr 2022 18:33:10 -0400 Subject: [PATCH 03/85] mintime/maxtime methods --- pkg/storage/tsdb/head.go | 14 ++++++++++++-- 1 file changed, 12 insertions(+), 2 deletions(-) diff --git a/pkg/storage/tsdb/head.go b/pkg/storage/tsdb/head.go index fc9dedbecf71d..fb155426f88a4 100644 --- a/pkg/storage/tsdb/head.go +++ b/pkg/storage/tsdb/head.go @@ -78,9 +78,19 @@ func NewHead(tenant string, metrics *HeadMetrics, logger log.Logger) *Head { } } +// MinTime returns the lowest time bound on visible data in the head. +func (h *Head) MinTime() int64 { + return h.minTime.Load() +} + +// MaxTime returns the highest timestamp seen in data of the head. +func (h *Head) MaxTime() int64 { + return h.maxTime.Load() +} + func (h *Head) updateMinMaxTime(mint, maxt int64) { for { - lt := h.minTime.Load() + lt := h.MinTime() if mint >= lt { break } @@ -89,7 +99,7 @@ func (h *Head) updateMinMaxTime(mint, maxt int64) { } } for { - ht := h.maxTime.Load() + ht := h.MaxTime() if maxt <= ht { break } From 0b67b764dac47ebdf0f088051497b9334dc03d3f Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Sat, 9 Apr 2022 10:18:29 -0400 Subject: [PATCH 04/85] tsdb head IndexReader impl --- pkg/storage/tsdb/head.go | 17 +- pkg/storage/tsdb/head_read.go | 174 +++++++++++++++++++++ pkg/storage/tsdb/index/fingerprint.go | 4 +- pkg/storage/tsdb/index/fingerprint_test.go | 2 +- pkg/storage/tsdb/index/index.go | 10 +- pkg/storage/tsdb/index/postings.go | 14 +- 6 files changed, 201 insertions(+), 20 deletions(-) create mode 100644 pkg/storage/tsdb/head_read.go diff --git a/pkg/storage/tsdb/head.go b/pkg/storage/tsdb/head.go index fb155426f88a4..cf938114f2245 100644 --- a/pkg/storage/tsdb/head.go +++ b/pkg/storage/tsdb/head.go @@ -6,7 +6,7 @@ import ( "github.com/go-kit/log" "github.com/grafana/loki/pkg/storage/tsdb/index" "github.com/prometheus/client_golang/prometheus" - "github.com/prometheus/common/model" + "github.com/prometheus/client_golang/prometheus/promauto" "github.com/prometheus/prometheus/model/labels" "github.com/prometheus/prometheus/storage" "go.uber.org/atomic" @@ -40,10 +40,17 @@ guaranteeing we maintain querying consistency for the entire data lifecycle. */ // TODO(owen-d) -type HeadMetrics struct{} +type HeadMetrics struct { + seriesNotFound prometheus.Counter +} func NewHeadMetrics(r prometheus.Registerer) *HeadMetrics { - return &HeadMetrics{} + return &HeadMetrics{ + seriesNotFound: promauto.With(r).NewCounter(prometheus.CounterOpts{ + Name: "loki_tsdb_head_series_not_found_total", + Help: "Total number of requests for series that were not found.", + }), + } } type Head struct { @@ -226,7 +233,7 @@ type memSeries struct { sync.RWMutex ref uint64 // The unique reference within a *Head ls labels.Labels - fp model.Fingerprint + fp uint64 chks index.ChunkMetas } @@ -234,6 +241,6 @@ func newMemSeries(ref uint64, ls labels.Labels) *memSeries { return &memSeries{ ref: ref, ls: ls, - fp: model.Fingerprint(ls.Hash()), + fp: ls.Hash(), } } diff --git a/pkg/storage/tsdb/head_read.go b/pkg/storage/tsdb/head_read.go new file mode 100644 index 0000000000000..61b1908fa8735 --- /dev/null +++ b/pkg/storage/tsdb/head_read.go @@ -0,0 +1,174 @@ +// Copyright 2021 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 ( + "math" + "sort" + + "github.com/prometheus/prometheus/model/labels" + "github.com/prometheus/prometheus/storage" + + "github.com/grafana/loki/pkg/storage/tsdb/index" +) + +// Index returns an IndexReader against the block. +func (h *Head) Index() (IndexReader, error) { + return h.indexRange(math.MinInt64, math.MaxInt64), nil +} + +func (h *Head) indexRange(mint, maxt int64) *headIndexReader { + if hmin := h.MinTime(); hmin > mint { + mint = hmin + } + return &headIndexReader{head: h, mint: mint, maxt: maxt} +} + +type headIndexReader struct { + head *Head + mint, maxt int64 +} + +func (h *headIndexReader) Bounds() (int64, int64) { + return h.head.MinTime(), h.head.MaxTime() +} + +func (h *headIndexReader) Checksum() uint32 { return 0 } + +func (h *headIndexReader) Close() error { + return nil +} + +func (h *headIndexReader) Symbols() index.StringIter { + return h.head.postings.Symbols() +} + +// SortedLabelValues returns label values present in the head for the +// specific label name that are within the time range mint to maxt. +// If matchers are specified the returned result set is reduced +// to label values of metrics matching the matchers. +func (h *headIndexReader) SortedLabelValues(name string, matchers ...*labels.Matcher) ([]string, error) { + values, err := h.LabelValues(name, matchers...) + if err == nil { + sort.Strings(values) + } + return values, err +} + +// LabelValues returns label values present in the head for the +// specific label name that are within the time range mint to maxt. +// If matchers are specified the returned result set is reduced +// to label values of metrics matching the matchers. +func (h *headIndexReader) LabelValues(name string, matchers ...*labels.Matcher) ([]string, error) { + if h.maxt < h.head.MinTime() || h.mint > h.head.MaxTime() { + return []string{}, nil + } + + if len(matchers) == 0 { + return h.head.postings.LabelValues(name), nil + } + + return labelValuesWithMatchers(h, name, matchers...) +} + +// LabelNames returns all the unique label names present in the head +// that are within the time range mint to maxt. +func (h *headIndexReader) LabelNames(matchers ...*labels.Matcher) ([]string, error) { + if h.maxt < h.head.MinTime() || h.mint > h.head.MaxTime() { + return []string{}, nil + } + + if len(matchers) == 0 { + labelNames := h.head.postings.LabelNames() + sort.Strings(labelNames) + return labelNames, nil + } + + return labelNamesWithMatchers(h, matchers...) +} + +// Postings returns the postings list iterator for the label pairs. +func (h *headIndexReader) Postings(name string, shard *index.ShardAnnotation, values ...string) (index.Postings, error) { + var p index.Postings + switch len(values) { + case 0: + p = index.EmptyPostings() + case 1: + p = h.head.postings.Get(name, values[0]) + default: + res := make([]index.Postings, 0, len(values)) + for _, value := range values { + res = append(res, h.head.postings.Get(name, value)) + } + p = index.Merge(res...) + } + + if shard != nil { + return index.NewShardedPostings(p, *shard, nil), nil + } + return p, nil +} + +// Series returns the series for the given reference. +func (h *headIndexReader) Series(ref storage.SeriesRef, lbls *labels.Labels, chks *[]index.ChunkMeta) (uint64, error) { + s := h.head.series.getByID(uint64(ref)) + + if s == nil { + h.head.metrics.seriesNotFound.Inc() + return 0, storage.ErrNotFound + } + *lbls = append((*lbls)[:0], s.ls...) + + s.Lock() + *chks = append((*chks)[:0], s.chks...) + s.Unlock() + + return s.fp, nil +} + +// LabelValueFor returns label value for the given label name in the series referred to by ID. +func (h *headIndexReader) LabelValueFor(id storage.SeriesRef, label string) (string, error) { + memSeries := h.head.series.getByID(uint64(id)) + if memSeries == nil { + return "", storage.ErrNotFound + } + + value := memSeries.ls.Get(label) + if value == "" { + return "", storage.ErrNotFound + } + + return value, nil +} + +// LabelNamesFor returns all the label names for the series referred to by IDs. +// The names returned are sorted. +func (h *headIndexReader) LabelNamesFor(ids ...storage.SeriesRef) ([]string, error) { + namesMap := make(map[string]struct{}) + for _, id := range ids { + memSeries := h.head.series.getByID(uint64(id)) + if memSeries == nil { + return nil, storage.ErrNotFound + } + for _, lbl := range memSeries.ls { + namesMap[lbl.Name] = struct{}{} + } + } + names := make([]string, 0, len(namesMap)) + for name := range namesMap { + names = append(names, name) + } + sort.Strings(names) + return names, nil +} diff --git a/pkg/storage/tsdb/index/fingerprint.go b/pkg/storage/tsdb/index/fingerprint.go index 0f46ed27a562b..c29f7357542e4 100644 --- a/pkg/storage/tsdb/index/fingerprint.go +++ b/pkg/storage/tsdb/index/fingerprint.go @@ -6,9 +6,9 @@ import ( ) // (SeriesRef, Fingerprint) tuples -type fingerprintOffsets [][2]uint64 +type FingerprintOffsets [][2]uint64 -func (xs fingerprintOffsets) Range(shard ShardAnnotation) (minOffset, maxOffset uint64) { +func (xs FingerprintOffsets) Range(shard ShardAnnotation) (minOffset, maxOffset uint64) { from, through := shard.Bounds() lower := sort.Search(len(xs), func(i int) bool { diff --git a/pkg/storage/tsdb/index/fingerprint_test.go b/pkg/storage/tsdb/index/fingerprint_test.go index 1485c247a2ad5..250355cd792ac 100644 --- a/pkg/storage/tsdb/index/fingerprint_test.go +++ b/pkg/storage/tsdb/index/fingerprint_test.go @@ -9,7 +9,7 @@ import ( ) func Test_FingerprintOffsetRange(t *testing.T) { - offsets := fingerprintOffsets{ + offsets := FingerprintOffsets{ {1, 1}, // 00 prefix {2, 1 << 62}, // 01 prefix {3, 1 << 63}, // 10 prefix diff --git a/pkg/storage/tsdb/index/index.go b/pkg/storage/tsdb/index/index.go index ddbca4a815b2b..0d34f96aaf95a 100644 --- a/pkg/storage/tsdb/index/index.go +++ b/pkg/storage/tsdb/index/index.go @@ -131,7 +131,7 @@ type Writer struct { labelIndexes []labelIndexHashEntry // Label index offsets. labelNames map[string]uint64 // Label names, and their usage. // Keeps track of the fingerprint/offset for every n series - fingerprintOffsets fingerprintOffsets + fingerprintOffsets FingerprintOffsets // Hold last series to validate that clients insert new series in order. lastSeries labels.Labels @@ -1149,7 +1149,7 @@ type Reader struct { nameSymbols map[uint32]string // Cache of the label name symbol lookups, // as there are not many and they are half of all lookups. - fingerprintOffsets fingerprintOffsets + fingerprintOffsets FingerprintOffsets dec *Decoder @@ -1522,10 +1522,10 @@ func ReadOffsetTable(bs ByteSlice, off uint64, f func([]string, uint64, int) err return d.Err() } -func ReadFingerprintOffsetsTable(bs ByteSlice, off uint64) (fingerprintOffsets, error) { +func ReadFingerprintOffsetsTable(bs ByteSlice, off uint64) (FingerprintOffsets, error) { d := encoding.DecWrap(tsdb_enc.NewDecbufAt(bs, int(off), castagnoliTable)) cnt := d.Be32() - res := make(fingerprintOffsets, 0, int(cnt)) + res := make(FingerprintOffsets, 0, int(cnt)) for d.Err() == nil && d.Len() > 0 && cnt > 0 { res = append(res, [2]uint64{d.Be64(), d.Be64()}) @@ -1824,7 +1824,7 @@ func (r *Reader) Postings(name string, shard *ShardAnnotation, values ...string) merged := Merge(res...) if shard != nil { - return newShardedPostings(merged, *shard, r.fingerprintOffsets), nil + return NewShardedPostings(merged, *shard, r.fingerprintOffsets), nil } return merged, nil diff --git a/pkg/storage/tsdb/index/postings.go b/pkg/storage/tsdb/index/postings.go index f45fccf80223f..c144efa55f583 100644 --- a/pkg/storage/tsdb/index/postings.go +++ b/pkg/storage/tsdb/index/postings.go @@ -832,7 +832,7 @@ 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] } -type shardedPostings struct { +type ShardedPostings struct { p Postings minOffset, maxOffset uint64 initialized bool @@ -845,9 +845,9 @@ type shardedPostings struct { // For example (below), given a shard, we'll likely return a slight superset of offsets surrounding the shard. // ---[shard0]--- # Shard membership // -[--shard0--]- # Series returned by shardedPostings -func newShardedPostings(p Postings, shard ShardAnnotation, offsets fingerprintOffsets) *shardedPostings { +func NewShardedPostings(p Postings, shard ShardAnnotation, offsets FingerprintOffsets) *ShardedPostings { min, max := offsets.Range(shard) - return &shardedPostings{ + return &ShardedPostings{ p: p, minOffset: min, maxOffset: max, @@ -855,7 +855,7 @@ func newShardedPostings(p Postings, shard ShardAnnotation, offsets fingerprintOf } // Next advances the iterator and returns true if another value was found. -func (sp *shardedPostings) Next() bool { +func (sp *ShardedPostings) Next() bool { // fast forward to the point we know we'll have to start checking if !sp.initialized { sp.initialized = true @@ -880,7 +880,7 @@ func (sp *shardedPostings) Next() bool { // Seek advances the iterator to value v or greater and returns // true if a value was found. -func (sp *shardedPostings) Seek(v storage.SeriesRef) (res bool) { +func (sp *ShardedPostings) Seek(v storage.SeriesRef) (res bool) { if v > storage.SeriesRef(sp.maxOffset) { return false } @@ -891,11 +891,11 @@ func (sp *shardedPostings) Seek(v storage.SeriesRef) (res bool) { } // At returns the value at the current iterator position. -func (sp *shardedPostings) At() storage.SeriesRef { +func (sp *ShardedPostings) At() storage.SeriesRef { return sp.p.At() } // Err returns the last error of the iterator. -func (sp *shardedPostings) Err() (err error) { +func (sp *ShardedPostings) Err() (err error) { return sp.p.Err() } From 9019d10c4fe5cb85120f6e8ca7f0f8befe0263af Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Mon, 11 Apr 2022 09:41:59 -0400 Subject: [PATCH 05/85] head correctly populates ref lookup --- pkg/storage/tsdb/head.go | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/pkg/storage/tsdb/head.go b/pkg/storage/tsdb/head.go index cf938114f2245..eaadc5b8b478c 100644 --- a/pkg/storage/tsdb/head.go +++ b/pkg/storage/tsdb/head.go @@ -218,6 +218,10 @@ func (s *stripeSeries) Append( if series == nil { series = createFn() s.hashes[i].set(fp, series) + + // the series locks are modulo'd by the ref, not fingerprint + refIdx := series.ref & uint64(s.shards-1) + s.series[refIdx][series.ref] = series created = true } mtx.Unlock() From 35402a0152361506ee1650edffb9371907aef6b0 Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Mon, 11 Apr 2022 09:42:10 -0400 Subject: [PATCH 06/85] tsdb head tests --- pkg/storage/tsdb/single_file_index_test.go | 255 ++++++++++++--------- 1 file changed, 141 insertions(+), 114 deletions(-) diff --git a/pkg/storage/tsdb/single_file_index_test.go b/pkg/storage/tsdb/single_file_index_test.go index 4c40ea03156a5..5159b9da8d087 100644 --- a/pkg/storage/tsdb/single_file_index_test.go +++ b/pkg/storage/tsdb/single_file_index_test.go @@ -4,11 +4,11 @@ import ( "context" "testing" + "github.com/go-kit/kit/log" + "github.com/grafana/loki/pkg/storage/tsdb/index" "github.com/prometheus/common/model" "github.com/prometheus/prometheus/model/labels" "github.com/stretchr/testify/require" - - "github.com/grafana/loki/pkg/storage/tsdb/index" ) func TestSingleIdx(t *testing.T) { @@ -55,118 +55,145 @@ func TestSingleIdx(t *testing.T) { }, } - idx := BuildIndex(t, t.TempDir(), "fake", cases) - - t.Run("GetChunkRefs", func(t *testing.T) { - refs, err := idx.GetChunkRefs(context.Background(), "fake", 1, 5, nil, nil, labels.MustNewMatcher(labels.MatchEqual, "foo", "bar")) - require.Nil(t, err) - - expected := []ChunkRef{ - { - User: "fake", - Fingerprint: model.Fingerprint(mustParseLabels(`{foo="bar"}`).Hash()), - Start: 0, - End: 3, - Checksum: 0, - }, - { - User: "fake", - Fingerprint: model.Fingerprint(mustParseLabels(`{foo="bar"}`).Hash()), - Start: 1, - End: 4, - Checksum: 1, - }, - { - User: "fake", - Fingerprint: model.Fingerprint(mustParseLabels(`{foo="bar"}`).Hash()), - Start: 2, - End: 5, - Checksum: 2, - }, - { - User: "fake", - Fingerprint: model.Fingerprint(mustParseLabels(`{foo="bar", bazz="buzz"}`).Hash()), - Start: 1, - End: 10, - Checksum: 3, - }, - } - require.Equal(t, expected, refs) - }) - - t.Run("GetChunkRefsSharded", func(t *testing.T) { - shard := index.ShardAnnotation{ - Shard: 1, - Of: 2, - } - shardedRefs, err := idx.GetChunkRefs(context.Background(), "fake", 1, 5, nil, &shard, labels.MustNewMatcher(labels.MatchEqual, "foo", "bar")) - - require.Nil(t, err) - - require.Equal(t, []ChunkRef{{ - User: "fake", - Fingerprint: model.Fingerprint(mustParseLabels(`{foo="bar", bazz="buzz"}`).Hash()), - Start: 1, - End: 10, - Checksum: 3, - }}, shardedRefs) - - }) - - t.Run("Series", func(t *testing.T) { - xs, err := idx.Series(context.Background(), "fake", 8, 9, nil, nil, labels.MustNewMatcher(labels.MatchEqual, "foo", "bar")) - require.Nil(t, err) - - expected := []Series{ - { - Labels: mustParseLabels(`{foo="bar", bazz="buzz"}`), - Fingerprint: model.Fingerprint(mustParseLabels(`{foo="bar", bazz="buzz"}`).Hash()), + for _, variant := range []struct { + desc string + fn func() *TSDBIndex + }{ + { + desc: "file", + fn: func() *TSDBIndex { + return BuildIndex(t, t.TempDir(), "fake", cases) }, - } - require.Equal(t, expected, xs) - }) - - t.Run("SeriesSharded", func(t *testing.T) { - shard := index.ShardAnnotation{ - Shard: 0, - Of: 2, - } - - xs, err := idx.Series(context.Background(), "fake", 0, 10, nil, &shard, labels.MustNewMatcher(labels.MatchEqual, "foo", "bar")) - require.Nil(t, err) - - expected := []Series{ - { - Labels: mustParseLabels(`{foo="bar"}`), - Fingerprint: model.Fingerprint(mustParseLabels(`{foo="bar"}`).Hash()), + }, + { + desc: "head", + fn: func() *TSDBIndex { + head := NewHead("fake", NewHeadMetrics(nil), log.NewNopLogger()) + for _, x := range cases { + head.Append(x.Labels, x.Chunks) + } + reader, err := head.Index() + require.Nil(t, err) + return NewTSDBIndex(reader) }, - } - require.Equal(t, expected, xs) - }) - - t.Run("LabelNames", func(t *testing.T) { - // request data at the end of the tsdb range, but it should return all labels present - ls, err := idx.LabelNames(context.Background(), "fake", 9, 10) - require.Nil(t, err) - require.Equal(t, []string{"bazz", "bonk", "foo"}, ls) - }) - - t.Run("LabelNamesWithMatchers", func(t *testing.T) { - // request data at the end of the tsdb range, but it should return all labels present - ls, err := idx.LabelNames(context.Background(), "fake", 9, 10, labels.MustNewMatcher(labels.MatchEqual, "bazz", "buzz")) - require.Nil(t, err) - require.Equal(t, []string{"bazz", "foo"}, ls) - }) - - t.Run("LabelValues", func(t *testing.T) { - vs, err := idx.LabelValues(context.Background(), "fake", 9, 10, "foo") - require.Nil(t, err) - require.Equal(t, []string{"bar", "bard"}, vs) - }) - - t.Run("LabelValuesWithMatchers", func(t *testing.T) { - vs, err := idx.LabelValues(context.Background(), "fake", 9, 10, "foo", labels.MustNewMatcher(labels.MatchEqual, "bazz", "buzz")) - require.Nil(t, err) - require.Equal(t, []string{"bar"}, vs) - }) + }, + } { + t.Run(variant.desc, func(t *testing.T) { + idx := variant.fn() + t.Run("GetChunkRefs", func(t *testing.T) { + refs, err := idx.GetChunkRefs(context.Background(), "fake", 1, 5, nil, nil, labels.MustNewMatcher(labels.MatchEqual, "foo", "bar")) + require.Nil(t, err) + + expected := []ChunkRef{ + { + User: "fake", + Fingerprint: model.Fingerprint(mustParseLabels(`{foo="bar"}`).Hash()), + Start: 0, + End: 3, + Checksum: 0, + }, + { + User: "fake", + Fingerprint: model.Fingerprint(mustParseLabels(`{foo="bar"}`).Hash()), + Start: 1, + End: 4, + Checksum: 1, + }, + { + User: "fake", + Fingerprint: model.Fingerprint(mustParseLabels(`{foo="bar"}`).Hash()), + Start: 2, + End: 5, + Checksum: 2, + }, + { + User: "fake", + Fingerprint: model.Fingerprint(mustParseLabels(`{foo="bar", bazz="buzz"}`).Hash()), + Start: 1, + End: 10, + Checksum: 3, + }, + } + require.Equal(t, expected, refs) + }) + + t.Run("GetChunkRefsSharded", func(t *testing.T) { + shard := index.ShardAnnotation{ + Shard: 1, + Of: 2, + } + shardedRefs, err := idx.GetChunkRefs(context.Background(), "fake", 1, 5, nil, &shard, labels.MustNewMatcher(labels.MatchEqual, "foo", "bar")) + + require.Nil(t, err) + + require.Equal(t, []ChunkRef{{ + User: "fake", + Fingerprint: model.Fingerprint(mustParseLabels(`{foo="bar", bazz="buzz"}`).Hash()), + Start: 1, + End: 10, + Checksum: 3, + }}, shardedRefs) + + }) + + t.Run("Series", func(t *testing.T) { + xs, err := idx.Series(context.Background(), "fake", 8, 9, nil, nil, labels.MustNewMatcher(labels.MatchEqual, "foo", "bar")) + require.Nil(t, err) + + expected := []Series{ + { + Labels: mustParseLabels(`{foo="bar", bazz="buzz"}`), + Fingerprint: model.Fingerprint(mustParseLabels(`{foo="bar", bazz="buzz"}`).Hash()), + }, + } + require.Equal(t, expected, xs) + }) + + t.Run("SeriesSharded", func(t *testing.T) { + shard := index.ShardAnnotation{ + Shard: 0, + Of: 2, + } + + xs, err := idx.Series(context.Background(), "fake", 0, 10, nil, &shard, labels.MustNewMatcher(labels.MatchEqual, "foo", "bar")) + require.Nil(t, err) + + expected := []Series{ + { + Labels: mustParseLabels(`{foo="bar"}`), + Fingerprint: model.Fingerprint(mustParseLabels(`{foo="bar"}`).Hash()), + }, + } + require.Equal(t, expected, xs) + }) + + t.Run("LabelNames", func(t *testing.T) { + // request data at the end of the tsdb range, but it should return all labels present + ls, err := idx.LabelNames(context.Background(), "fake", 9, 10) + require.Nil(t, err) + require.Equal(t, []string{"bazz", "bonk", "foo"}, ls) + }) + + t.Run("LabelNamesWithMatchers", func(t *testing.T) { + // request data at the end of the tsdb range, but it should return all labels present + ls, err := idx.LabelNames(context.Background(), "fake", 9, 10, labels.MustNewMatcher(labels.MatchEqual, "bazz", "buzz")) + require.Nil(t, err) + require.Equal(t, []string{"bazz", "foo"}, ls) + }) + + t.Run("LabelValues", func(t *testing.T) { + vs, err := idx.LabelValues(context.Background(), "fake", 9, 10, "foo") + require.Nil(t, err) + require.Equal(t, []string{"bar", "bard"}, vs) + }) + + t.Run("LabelValuesWithMatchers", func(t *testing.T) { + vs, err := idx.LabelValues(context.Background(), "fake", 9, 10, "foo", labels.MustNewMatcher(labels.MatchEqual, "bazz", "buzz")) + require.Nil(t, err) + require.Equal(t, []string{"bar"}, vs) + }) + + }) + } + } From e69729e8e6575ec502a8e12fa467fb6357c163f5 Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Mon, 11 Apr 2022 09:43:57 -0400 Subject: [PATCH 07/85] adds prometheus license to tsdb head --- pkg/storage/tsdb/head.go | 12 ++++++++++++ 1 file changed, 12 insertions(+) diff --git a/pkg/storage/tsdb/head.go b/pkg/storage/tsdb/head.go index eaadc5b8b478c..69824af4654ae 100644 --- a/pkg/storage/tsdb/head.go +++ b/pkg/storage/tsdb/head.go @@ -1,3 +1,15 @@ +// Copyright 2021 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 ( From 968e617760058e2a59c00cf9f521588c2fa5f684 Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Mon, 11 Apr 2022 12:00:12 -0400 Subject: [PATCH 08/85] linting --- pkg/storage/stores/tsdb/head.go | 10 ---------- pkg/storage/stores/tsdb/single_file_index_test.go | 2 +- 2 files changed, 1 insertion(+), 11 deletions(-) diff --git a/pkg/storage/stores/tsdb/head.go b/pkg/storage/stores/tsdb/head.go index db0b2e32c0876..ef4a3c0316d15 100644 --- a/pkg/storage/stores/tsdb/head.go +++ b/pkg/storage/stores/tsdb/head.go @@ -206,16 +206,6 @@ func (s *stripeSeries) getByID(id uint64) *memSeries { return series } -func (s *stripeSeries) getByHash(hash uint64, lset labels.Labels) *memSeries { - i := hash & uint64(s.shards-1) - - s.locks[i].RLock() - series := s.hashes[i].get(hash, lset) - s.locks[i].RUnlock() - - return series -} - // Append adds chunks to the correct series and returns whether a new series was added func (s *stripeSeries) Append( ls labels.Labels, diff --git a/pkg/storage/stores/tsdb/single_file_index_test.go b/pkg/storage/stores/tsdb/single_file_index_test.go index cd2b1ab6106c6..d9b49fd4be00e 100644 --- a/pkg/storage/stores/tsdb/single_file_index_test.go +++ b/pkg/storage/stores/tsdb/single_file_index_test.go @@ -4,7 +4,7 @@ import ( "context" "testing" - "github.com/go-kit/kit/log" + "github.com/go-kit/log" "github.com/prometheus/common/model" "github.com/prometheus/prometheus/model/labels" "github.com/stretchr/testify/require" From e051eee9b3309fb47d58011af99b9e0cb10cd435 Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Tue, 12 Apr 2022 11:03:30 -0400 Subject: [PATCH 09/85] [WIP] speccing out tsdb head wal --- pkg/storage/stores/tsdb/head_wal.go | 185 ++++++++++++++++++++++++++++ 1 file changed, 185 insertions(+) create mode 100644 pkg/storage/stores/tsdb/head_wal.go diff --git a/pkg/storage/stores/tsdb/head_wal.go b/pkg/storage/stores/tsdb/head_wal.go new file mode 100644 index 0000000000000..c1a6b3c4755d6 --- /dev/null +++ b/pkg/storage/stores/tsdb/head_wal.go @@ -0,0 +1,185 @@ +package tsdb + +import ( + "sync" + + "github.com/go-kit/log" + "github.com/grafana/loki/pkg/storage/stores/tsdb/index" + "github.com/grafana/loki/pkg/util/encoding" + "github.com/pkg/errors" + "github.com/prometheus/prometheus/tsdb/record" + "github.com/prometheus/prometheus/tsdb/wal" +) + +// 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 ( + 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, version RecordType, 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) (err 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) > 0 { + 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() + err = decodeChunks(decbuf.B, t, walRec) + default: + return errors.New("unknown record type") + } + + // We reach here only if its a record with type header. + if decbuf.Err() != nil { + return decbuf.Err() + } + + if err != nil { + return 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 { + log log.Logger + wal *wal.WAL + + closeMtx sync.RWMutex + closed bool +} + +func newHeadWAL(log log.Logger, dir string) (*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{ + log: log, + wal: wal, + }, nil +} + +func (w *headWAL) Stop() error { + w.closeMtx.Lock() + err := w.wal.Close() + w.closed = true + w.closeMtx.Unlock() + return err +} From 7f207c74a85d94930337bf2b9d292ea19ccd44c2 Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Tue, 12 Apr 2022 13:22:07 -0400 Subject: [PATCH 10/85] fix length check and adds tsdb wal encoding tests --- pkg/storage/stores/tsdb/head_wal.go | 2 +- pkg/storage/stores/tsdb/head_wal_test.go | 57 ++++++++++++++++++++++++ 2 files changed, 58 insertions(+), 1 deletion(-) create mode 100644 pkg/storage/stores/tsdb/head_wal_test.go diff --git a/pkg/storage/stores/tsdb/head_wal.go b/pkg/storage/stores/tsdb/head_wal.go index c1a6b3c4755d6..515ec9178bfe0 100644 --- a/pkg/storage/stores/tsdb/head_wal.go +++ b/pkg/storage/stores/tsdb/head_wal.go @@ -125,7 +125,7 @@ func decodeWALRecord(b []byte, walRec *walRecord) (err error) { return errors.Wrap(err, "decoding head series") } // unlike tsdb, we only add one series per record. - if len(rSeries) > 0 { + if len(rSeries) > 1 { return errors.New("more than one series detected in tsdb head wal record") } if len(rSeries) == 1 { diff --git a/pkg/storage/stores/tsdb/head_wal_test.go b/pkg/storage/stores/tsdb/head_wal_test.go new file mode 100644 index 0000000000000..d0cf7d10751d9 --- /dev/null +++ b/pkg/storage/stores/tsdb/head_wal_test.go @@ -0,0 +1,57 @@ +package tsdb + +import ( + "testing" + + "github.com/grafana/loki/pkg/storage/stores/tsdb/index" + "github.com/prometheus/prometheus/tsdb/chunks" + "github.com/prometheus/prometheus/tsdb/record" + "github.com/stretchr/testify/require" +) + +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) +} From 3da9db8fc446866c61d2df70997ad15b913b8657 Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Tue, 12 Apr 2022 13:41:41 -0400 Subject: [PATCH 11/85] exposes wal structs & removes closed semantics --- pkg/storage/stores/tsdb/head_wal.go | 80 ++++++++++++++---------- pkg/storage/stores/tsdb/head_wal_test.go | 20 +++--- 2 files changed, 58 insertions(+), 42 deletions(-) diff --git a/pkg/storage/stores/tsdb/head_wal.go b/pkg/storage/stores/tsdb/head_wal.go index 515ec9178bfe0..99c5eb5c0353d 100644 --- a/pkg/storage/stores/tsdb/head_wal.go +++ b/pkg/storage/stores/tsdb/head_wal.go @@ -1,8 +1,6 @@ package tsdb import ( - "sync" - "github.com/go-kit/log" "github.com/grafana/loki/pkg/storage/stores/tsdb/index" "github.com/grafana/loki/pkg/util/encoding" @@ -29,39 +27,39 @@ const ( walRecordChunks ) -type walRecord struct { - userID string - series record.RefSeries - chks chunkMetasRecord +type WalRecord struct { + UserID string + Series record.RefSeries + Chks ChunkMetasRecord } -type chunkMetasRecord struct { - chks index.ChunkMetas - ref uint64 +type ChunkMetasRecord struct { + Chks index.ChunkMetas + Ref uint64 } -func (r *walRecord) encodeSeries(b []byte) []byte { +func (r *WalRecord) encodeSeries(b []byte) []byte { buf := encoding.EncWith(b) buf.PutByte(byte(walRecordSeries)) - buf.PutUvarintStr(r.userID) + 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):])...) + encoded = append(encoded, enc.Series([]record.RefSeries{r.Series}, encoded[len(encoded):])...) return encoded } -func (r *walRecord) encodeChunks(b []byte) []byte { +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)) + buf.PutUvarintStr(r.UserID) + buf.PutBE64(r.Chks.Ref) + buf.PutUvarint(len(r.Chks.Chks)) - for _, chk := range r.chks.chks { + for _, chk := range r.Chks.Chks { buf.PutBE64(uint64(chk.MinTime)) buf.PutBE64(uint64(chk.MaxTime)) buf.PutBE32(chk.Checksum) @@ -72,14 +70,14 @@ func (r *walRecord) encodeChunks(b []byte) []byte { return buf.Get() } -func decodeChunks(b []byte, version RecordType, rec *walRecord) error { +func decodeChunks(b []byte, version RecordType, rec *WalRecord) error { if len(b) == 0 { return nil } dec := encoding.DecWith(b) - rec.chks.ref = dec.Be64() + rec.Chks.Ref = dec.Be64() if err := dec.Err(); err != nil { return errors.Wrap(err, "decoding series ref") } @@ -89,10 +87,10 @@ func decodeChunks(b []byte, version RecordType, rec *walRecord) error { return errors.Wrap(err, "decoding number of chunks") } // allocate space for the required number of chunks - rec.chks.chks = make(index.ChunkMetas, 0, ln) + 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{ + rec.Chks.Chks = append(rec.Chks.Chks, index.ChunkMeta{ MinTime: dec.Be64int64(), MaxTime: dec.Be64int64(), Checksum: dec.Be32(), @@ -108,7 +106,7 @@ func decodeChunks(b []byte, version RecordType, rec *walRecord) error { return nil } -func decodeWALRecord(b []byte, walRec *walRecord) (err error) { +func decodeWALRecord(b []byte, walRec *WalRecord) (err error) { var ( userID string dec record.Decoder @@ -129,7 +127,7 @@ func decodeWALRecord(b []byte, walRec *walRecord) (err error) { return errors.New("more than one series detected in tsdb head wal record") } if len(rSeries) == 1 { - walRec.series = rSeries[0] + walRec.Series = rSeries[0] } case walRecordChunks: userID = decbuf.UvarintStr() @@ -147,7 +145,7 @@ func decodeWALRecord(b []byte, walRec *walRecord) (err error) { return err } - walRec.userID = userID + walRec.UserID = userID return nil } @@ -156,9 +154,6 @@ func decodeWALRecord(b []byte, walRec *walRecord) (err error) { type headWAL struct { log log.Logger wal *wal.WAL - - closeMtx sync.RWMutex - closed bool } func newHeadWAL(log log.Logger, dir string) (*headWAL, error) { @@ -177,9 +172,30 @@ func newHeadWAL(log log.Logger, dir string) (*headWAL, error) { } func (w *headWAL) Stop() error { - w.closeMtx.Lock() - err := w.wal.Close() - w.closed = true - w.closeMtx.Unlock() - return err + 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) + 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 } diff --git a/pkg/storage/stores/tsdb/head_wal_test.go b/pkg/storage/stores/tsdb/head_wal_test.go index d0cf7d10751d9..d7d801f427b3c 100644 --- a/pkg/storage/stores/tsdb/head_wal_test.go +++ b/pkg/storage/stores/tsdb/head_wal_test.go @@ -10,15 +10,15 @@ import ( ) func Test_Encoding_Series(t *testing.T) { - record := &walRecord{ - userID: "foo", - series: record.RefSeries{ + record := &WalRecord{ + UserID: "foo", + Series: record.RefSeries{ Ref: chunks.HeadSeriesRef(1), Labels: mustParseLabels(`{foo="bar"}`), }, } buf := record.encodeSeries(nil) - decoded := &walRecord{} + decoded := &WalRecord{} err := decodeWALRecord(buf, decoded) require.Nil(t, err) @@ -26,11 +26,11 @@ func Test_Encoding_Series(t *testing.T) { } func Test_Encoding_Chunks(t *testing.T) { - record := &walRecord{ - userID: "foo", - chks: chunkMetasRecord{ - ref: 1, - chks: index.ChunkMetas{ + record := &WalRecord{ + UserID: "foo", + Chks: ChunkMetasRecord{ + Ref: 1, + Chks: index.ChunkMetas{ { Checksum: 1, MinTime: 1, @@ -49,7 +49,7 @@ func Test_Encoding_Chunks(t *testing.T) { }, } buf := record.encodeChunks(nil) - decoded := &walRecord{} + decoded := &WalRecord{} err := decodeWALRecord(buf, decoded) require.Nil(t, err) From 734a106abeea34747d4d79afeff84d2b10f06439 Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Tue, 12 Apr 2022 15:15:38 -0400 Subject: [PATCH 12/85] logs start time in the tsdb wal --- pkg/storage/stores/tsdb/head_wal.go | 76 ++++++++++++++++++------ pkg/storage/stores/tsdb/head_wal_test.go | 13 ++++ 2 files changed, 70 insertions(+), 19 deletions(-) diff --git a/pkg/storage/stores/tsdb/head_wal.go b/pkg/storage/stores/tsdb/head_wal.go index 99c5eb5c0353d..39710483e4a51 100644 --- a/pkg/storage/stores/tsdb/head_wal.go +++ b/pkg/storage/stores/tsdb/head_wal.go @@ -1,6 +1,8 @@ package tsdb import ( + "time" + "github.com/go-kit/log" "github.com/grafana/loki/pkg/storage/stores/tsdb/index" "github.com/grafana/loki/pkg/util/encoding" @@ -23,14 +25,20 @@ type RecordType byte // By prefixing records with versions, we can easily update our wal schema const ( - walRecordSeries RecordType = iota - walRecordChunks + // 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. + WALRecordFirstWrite RecordType = iota + WalRecordSeries + WalRecordChunks ) type WalRecord struct { - UserID string - Series record.RefSeries - Chks ChunkMetasRecord + UserID string + StartTime int64 // UnixNano + Series record.RefSeries + Chks ChunkMetasRecord } type ChunkMetasRecord struct { @@ -40,7 +48,7 @@ type ChunkMetasRecord struct { func (r *WalRecord) encodeSeries(b []byte) []byte { buf := encoding.EncWith(b) - buf.PutByte(byte(walRecordSeries)) + buf.PutByte(byte(WalRecordSeries)) buf.PutUvarintStr(r.UserID) var enc record.Encoder @@ -54,7 +62,7 @@ func (r *WalRecord) encodeSeries(b []byte) []byte { func (r *WalRecord) encodeChunks(b []byte) []byte { buf := encoding.EncWith(b) - buf.PutByte(byte(walRecordChunks)) + buf.PutByte(byte(WalRecordChunks)) buf.PutUvarintStr(r.UserID) buf.PutBE64(r.Chks.Ref) buf.PutUvarint(len(r.Chks.Chks)) @@ -106,7 +114,20 @@ func decodeChunks(b []byte, version RecordType, rec *WalRecord) error { return nil } -func decodeWALRecord(b []byte, walRec *WalRecord) (err error) { +func (r *WalRecord) encodeStartTime(b []byte) []byte { + buf := encoding.EncWith(b) + buf.PutByte(byte(WALRecordFirstWrite)) + buf.PutBE64int64(r.StartTime) + return buf.Get() +} + +func decodeStartTime(b []byte, rec *WalRecord) error { + dec := encoding.DecWith(b) + rec.StartTime = dec.Be64int64() + return dec.Err() +} + +func decodeWALRecord(b []byte, walRec *WalRecord) error { var ( userID string dec record.Decoder @@ -116,7 +137,11 @@ func decodeWALRecord(b []byte, walRec *WalRecord) (err error) { ) switch t { - case walRecordSeries: + case WALRecordFirstWrite: + if err := decodeStartTime(decbuf.B, walRec); err != nil { + return errors.Wrap(err, "decoding tsdb wal start time") + } + case WalRecordSeries: userID = decbuf.UvarintStr() rSeries, err := dec.Series(decbuf.B, nil) if err != nil { @@ -129,22 +154,19 @@ func decodeWALRecord(b []byte, walRec *WalRecord) (err error) { if len(rSeries) == 1 { walRec.Series = rSeries[0] } - case walRecordChunks: + case WalRecordChunks: userID = decbuf.UvarintStr() - err = decodeChunks(decbuf.B, t, walRec) + if err := decodeChunks(decbuf.B, t, walRec); err != nil { + return err + } default: return errors.New("unknown record type") } - // We reach here only if its a record with type header. if decbuf.Err() != nil { return decbuf.Err() } - if err != nil { - return err - } - walRec.UserID = userID return nil } @@ -152,8 +174,9 @@ func decodeWALRecord(b []byte, walRec *WalRecord) (err error) { // 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 { - log log.Logger - wal *wal.WAL + start time.Time + log log.Logger + wal *wal.WAL } func newHeadWAL(log log.Logger, dir string) (*headWAL, error) { @@ -171,6 +194,11 @@ func newHeadWAL(log log.Logger, dir string) (*headWAL, error) { }, nil } +// Start logs a record containing the time at which this WAL became active. +func (w *headWAL) Start() error { + return w.Log(&WalRecord{StartTime: time.Now().UnixNano()}) +} + func (w *headWAL) Stop() error { return w.wal.Close() } @@ -182,9 +210,19 @@ func (w *headWAL) Log(record *WalRecord) error { var buf []byte + // This only happens once when the wal is first written to. + // It could be refactored out of this function to avoid cpu cycles, + // but this is simpler and not on the hot path, so I've (owen-d) kept it. + if record.StartTime != 0 { + buf = record.encodeStartTime(buf[:0]) + if err := w.wal.Log(buf); err != nil { + return err + } + } + // Always write series before chunks if len(record.Series.Labels) > 0 { - buf = record.encodeSeries(buf) + buf = record.encodeSeries(buf[:0]) if err := w.wal.Log(buf); err != nil { return err } diff --git a/pkg/storage/stores/tsdb/head_wal_test.go b/pkg/storage/stores/tsdb/head_wal_test.go index d7d801f427b3c..a37c445be823b 100644 --- a/pkg/storage/stores/tsdb/head_wal_test.go +++ b/pkg/storage/stores/tsdb/head_wal_test.go @@ -2,6 +2,7 @@ package tsdb import ( "testing" + "time" "github.com/grafana/loki/pkg/storage/stores/tsdb/index" "github.com/prometheus/prometheus/tsdb/chunks" @@ -55,3 +56,15 @@ func Test_Encoding_Chunks(t *testing.T) { require.Nil(t, err) require.Equal(t, record, decoded) } + +func Test_Encoding_StartTime(t *testing.T) { + record := &WalRecord{ + StartTime: time.Now().UnixNano(), + } + buf := record.encodeStartTime(nil) + decoded := &WalRecord{} + + err := decodeWALRecord(buf, decoded) + require.Nil(t, err) + require.Equal(t, record, decoded) +} From dbb306740a537d243695b9f949a56dff8e347488 Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Tue, 12 Apr 2022 15:33:38 -0400 Subject: [PATCH 13/85] wal interface + testing --- pkg/storage/stores/tsdb/head_wal.go | 15 ++++++-- pkg/storage/stores/tsdb/head_wal_test.go | 47 ++++++++++++++++++++++++ 2 files changed, 58 insertions(+), 4 deletions(-) diff --git a/pkg/storage/stores/tsdb/head_wal.go b/pkg/storage/stores/tsdb/head_wal.go index 39710483e4a51..585c30b7a73a6 100644 --- a/pkg/storage/stores/tsdb/head_wal.go +++ b/pkg/storage/stores/tsdb/head_wal.go @@ -4,13 +4,20 @@ import ( "time" "github.com/go-kit/log" - "github.com/grafana/loki/pkg/storage/stores/tsdb/index" - "github.com/grafana/loki/pkg/util/encoding" "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. @@ -195,8 +202,8 @@ func newHeadWAL(log log.Logger, dir string) (*headWAL, error) { } // Start logs a record containing the time at which this WAL became active. -func (w *headWAL) Start() error { - return w.Log(&WalRecord{StartTime: time.Now().UnixNano()}) +func (w *headWAL) Start(t time.Time) error { + return w.Log(&WalRecord{StartTime: t.UnixNano()}) } func (w *headWAL) Stop() error { diff --git a/pkg/storage/stores/tsdb/head_wal_test.go b/pkg/storage/stores/tsdb/head_wal_test.go index a37c445be823b..6500706615d68 100644 --- a/pkg/storage/stores/tsdb/head_wal_test.go +++ b/pkg/storage/stores/tsdb/head_wal_test.go @@ -4,6 +4,7 @@ import ( "testing" "time" + "github.com/go-kit/kit/log" "github.com/grafana/loki/pkg/storage/stores/tsdb/index" "github.com/prometheus/prometheus/tsdb/chunks" "github.com/prometheus/prometheus/tsdb/record" @@ -68,3 +69,49 @@ func Test_Encoding_StartTime(t *testing.T) { require.Nil(t, err) require.Equal(t, record, decoded) } + +func Test_HeadWALLog(t *testing.T) { + dir := t.TempDir() + start := time.Now() + w, err := newHeadWAL(log.NewNopLogger(), dir) + require.Nil(t, err) + require.Nil(t, w.Start(start)) + + newSeries := &WalRecord{ + UserID: "foo", + StartTime: 0, + 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", + StartTime: 0, + 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()) +} From cab102002137f73dba923903cb5cfa238b7780d6 Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Wed, 13 Apr 2022 16:36:05 -0400 Subject: [PATCH 14/85] exports walrecord + returns ref when appending --- pkg/storage/stores/tsdb/head.go | 12 +++++++---- pkg/storage/stores/tsdb/head_wal.go | 21 ++++++++++--------- pkg/storage/stores/tsdb/head_wal_test.go | 16 +++++++------- .../stores/tsdb/single_file_index_test.go | 2 +- 4 files changed, 28 insertions(+), 23 deletions(-) diff --git a/pkg/storage/stores/tsdb/head.go b/pkg/storage/stores/tsdb/head.go index ef4a3c0316d15..aa6f7c0c82399 100644 --- a/pkg/storage/stores/tsdb/head.go +++ b/pkg/storage/stores/tsdb/head.go @@ -36,6 +36,8 @@ const ( // 1) Heads are per-tenant in Loki // 2) Loki tends to have a few orders of magnitude less series per node than // Prometheus|Cortex|Mimir. + // Do not specify without bit shifting. This allows us to + // do shard index calcuations via bitwise & rather than modulos. defaultStripeSize = 64 ) @@ -130,10 +132,10 @@ func (h *Head) updateMinMaxTime(mint, maxt int64) { } // Note: chks must not be nil or zero-length -func (h *Head) Append(ls labels.Labels, chks index.ChunkMetas) { +func (h *Head) Append(ls labels.Labels, chks index.ChunkMetas) (created bool, refID uint64) { from, through := chks.Bounds() var id uint64 - created := h.series.Append(ls, chks, func() *memSeries { + created, refID = h.series.Append(ls, chks, func() *memSeries { id = h.lastSeriesID.Inc() return newMemSeries(id, ls) }) @@ -144,6 +146,7 @@ func (h *Head) Append(ls labels.Labels, chks index.ChunkMetas) { } h.postings.Add(storage.SeriesRef(id), ls) h.numSeries.Inc() + return } // seriesHashmap is a simple hashmap for memSeries by their label set. It is built @@ -211,7 +214,7 @@ func (s *stripeSeries) Append( ls labels.Labels, chks index.ChunkMetas, createFn func() *memSeries, -) (created bool) { +) (created bool, refID uint64) { fp := ls.Hash() i := fp & uint64(s.shards-1) mtx := &s.locks[i] @@ -222,7 +225,7 @@ func (s *stripeSeries) Append( series = createFn() s.hashes[i].set(fp, series) - // the series locks are modulo'd by the ref, not fingerprint + // the series locks are determined by the ref, not fingerprint refIdx := series.ref & uint64(s.shards-1) s.series[refIdx][series.ref] = series created = true @@ -231,6 +234,7 @@ func (s *stripeSeries) Append( series.Lock() series.chks = append(series.chks, chks...) + refID = series.ref series.Unlock() return diff --git a/pkg/storage/stores/tsdb/head_wal.go b/pkg/storage/stores/tsdb/head_wal.go index 585c30b7a73a6..2ba0d72d9fe78 100644 --- a/pkg/storage/stores/tsdb/head_wal.go +++ b/pkg/storage/stores/tsdb/head_wal.go @@ -14,7 +14,7 @@ import ( type WAL interface { Start(time.Time) error - Log(*WalRecord) error + Log(*WALRecord) error Stop() error } @@ -41,7 +41,7 @@ const ( WalRecordChunks ) -type WalRecord struct { +type WALRecord struct { UserID string StartTime int64 // UnixNano Series record.RefSeries @@ -53,7 +53,7 @@ type ChunkMetasRecord struct { Ref uint64 } -func (r *WalRecord) encodeSeries(b []byte) []byte { +func (r *WALRecord) encodeSeries(b []byte) []byte { buf := encoding.EncWith(b) buf.PutByte(byte(WalRecordSeries)) buf.PutUvarintStr(r.UserID) @@ -67,7 +67,7 @@ func (r *WalRecord) encodeSeries(b []byte) []byte { return encoded } -func (r *WalRecord) encodeChunks(b []byte) []byte { +func (r *WALRecord) encodeChunks(b []byte) []byte { buf := encoding.EncWith(b) buf.PutByte(byte(WalRecordChunks)) buf.PutUvarintStr(r.UserID) @@ -85,7 +85,7 @@ func (r *WalRecord) encodeChunks(b []byte) []byte { return buf.Get() } -func decodeChunks(b []byte, version RecordType, rec *WalRecord) error { +func decodeChunks(b []byte, version RecordType, rec *WALRecord) error { if len(b) == 0 { return nil } @@ -121,20 +121,20 @@ func decodeChunks(b []byte, version RecordType, rec *WalRecord) error { return nil } -func (r *WalRecord) encodeStartTime(b []byte) []byte { +func (r *WALRecord) encodeStartTime(b []byte) []byte { buf := encoding.EncWith(b) buf.PutByte(byte(WALRecordFirstWrite)) buf.PutBE64int64(r.StartTime) return buf.Get() } -func decodeStartTime(b []byte, rec *WalRecord) error { +func decodeStartTime(b []byte, rec *WALRecord) error { dec := encoding.DecWith(b) rec.StartTime = dec.Be64int64() return dec.Err() } -func decodeWALRecord(b []byte, walRec *WalRecord) error { +func decodeWALRecord(b []byte, walRec *WALRecord) error { var ( userID string dec record.Decoder @@ -203,14 +203,15 @@ func newHeadWAL(log log.Logger, dir string) (*headWAL, error) { // Start logs a record containing the time at which this WAL became active. func (w *headWAL) Start(t time.Time) error { - return w.Log(&WalRecord{StartTime: t.UnixNano()}) + w.start = t + return w.Log(&WALRecord{StartTime: t.UnixNano()}) } func (w *headWAL) Stop() error { return w.wal.Close() } -func (w *headWAL) Log(record *WalRecord) error { +func (w *headWAL) Log(record *WALRecord) error { if record == nil { return nil } diff --git a/pkg/storage/stores/tsdb/head_wal_test.go b/pkg/storage/stores/tsdb/head_wal_test.go index 6500706615d68..154f6b1fc2f6c 100644 --- a/pkg/storage/stores/tsdb/head_wal_test.go +++ b/pkg/storage/stores/tsdb/head_wal_test.go @@ -12,7 +12,7 @@ import ( ) func Test_Encoding_Series(t *testing.T) { - record := &WalRecord{ + record := &WALRecord{ UserID: "foo", Series: record.RefSeries{ Ref: chunks.HeadSeriesRef(1), @@ -20,7 +20,7 @@ func Test_Encoding_Series(t *testing.T) { }, } buf := record.encodeSeries(nil) - decoded := &WalRecord{} + decoded := &WALRecord{} err := decodeWALRecord(buf, decoded) require.Nil(t, err) @@ -28,7 +28,7 @@ func Test_Encoding_Series(t *testing.T) { } func Test_Encoding_Chunks(t *testing.T) { - record := &WalRecord{ + record := &WALRecord{ UserID: "foo", Chks: ChunkMetasRecord{ Ref: 1, @@ -51,7 +51,7 @@ func Test_Encoding_Chunks(t *testing.T) { }, } buf := record.encodeChunks(nil) - decoded := &WalRecord{} + decoded := &WALRecord{} err := decodeWALRecord(buf, decoded) require.Nil(t, err) @@ -59,11 +59,11 @@ func Test_Encoding_Chunks(t *testing.T) { } func Test_Encoding_StartTime(t *testing.T) { - record := &WalRecord{ + record := &WALRecord{ StartTime: time.Now().UnixNano(), } buf := record.encodeStartTime(nil) - decoded := &WalRecord{} + decoded := &WALRecord{} err := decodeWALRecord(buf, decoded) require.Nil(t, err) @@ -77,7 +77,7 @@ func Test_HeadWALLog(t *testing.T) { require.Nil(t, err) require.Nil(t, w.Start(start)) - newSeries := &WalRecord{ + newSeries := &WALRecord{ UserID: "foo", StartTime: 0, Series: record.RefSeries{Ref: 1, Labels: mustParseLabels(`{foo="bar"}`)}, @@ -96,7 +96,7 @@ func Test_HeadWALLog(t *testing.T) { } require.Nil(t, w.Log(newSeries)) - chunksOnly := &WalRecord{ + chunksOnly := &WALRecord{ UserID: "foo", StartTime: 0, Chks: ChunkMetasRecord{ diff --git a/pkg/storage/stores/tsdb/single_file_index_test.go b/pkg/storage/stores/tsdb/single_file_index_test.go index d9b49fd4be00e..454babbf2d721 100644 --- a/pkg/storage/stores/tsdb/single_file_index_test.go +++ b/pkg/storage/stores/tsdb/single_file_index_test.go @@ -71,7 +71,7 @@ func TestSingleIdx(t *testing.T) { fn: func() *TSDBIndex { head := NewHead("fake", NewHeadMetrics(nil), log.NewNopLogger()) for _, x := range cases { - head.Append(x.Labels, x.Chunks) + _, _ = head.Append(x.Labels, x.Chunks) } reader, err := head.Index() require.Nil(t, err) From 60bfe0b354285ad4d03b0f0469cdb34bd66acc9c Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Wed, 13 Apr 2022 16:36:14 -0400 Subject: [PATCH 15/85] specs out head manager --- pkg/storage/stores/tsdb/head_manager.go | 226 ++++++++++++++++++++++++ 1 file changed, 226 insertions(+) create mode 100644 pkg/storage/stores/tsdb/head_manager.go diff --git a/pkg/storage/stores/tsdb/head_manager.go b/pkg/storage/stores/tsdb/head_manager.go new file mode 100644 index 0000000000000..700671999f9d1 --- /dev/null +++ b/pkg/storage/stores/tsdb/head_manager.go @@ -0,0 +1,226 @@ +package tsdb + +import ( + "fmt" + "path/filepath" + "sync" + "time" + + "github.com/cespare/xxhash" + "github.com/go-kit/log" + "github.com/go-kit/log/level" + "github.com/grafana/loki/pkg/storage/stores/tsdb/index" + "github.com/prometheus/client_golang/prometheus" + "github.com/prometheus/prometheus/model/labels" + "github.com/prometheus/prometheus/tsdb/chunks" + "github.com/prometheus/prometheus/tsdb/record" +) + +const defaultRotationPeriod = 2 * time.Hour + +// Do not specify without bit shifting. This allows us to +// do shard index calcuations via bitwise & rather than modulos. +const defaultHeadManagerStripeSize = 1 << 7 + +// HeadManager both accepts flushed chunk writes +// and exposes the index interface for multiple tenants. +// It also handles updating an underlying WAL and periodically +// rotates both the tenant Heads and the underlying WAL, using +// the old versions to build TSDB files. +type HeadManager struct { + name string + log log.Logger + dir string + metrics *HeadMetrics + + // RLocked for all writes/reads, + // Locked before rotating heads/wal + mtx sync.RWMutex + + // how often WALs should be rotated and TSDBs cut + period time.Duration + + active *headWAL + + shards int + activeHeads *tenantHeads + oldHeads []*tenantHeads + + readyCh chan struct{} + flusher flusher +} + +func NewHeadManager(log log.Logger, dir string, reg prometheus.Registerer, name string, flusher flusher) *HeadManager { + shards := 128 + metrics := NewHeadMetrics(reg) + return &HeadManager{ + name: name, + log: log, + dir: dir, + metrics: metrics, + + period: defaultRotationPeriod, + shards: shards, + flusher: flusher, + readyCh: make(chan struct{}), + } + +} + +func (m *HeadManager) Run() { + // Run once immediately + if err := m.Rotate(); err != nil { + level.Error(m.log).Log("msg", "failed to rotate TSDB WAL", "error", err.Error()) + } + + ticker := time.NewTicker(m.period) + for { + <-ticker.C + if err := m.Rotate(); err != nil { + level.Error(m.log).Log("msg", "failed to rotate TSDB WAL", "error", err.Error()) + } + } +} + +func (m *HeadManager) Ready() <-chan struct{} { return m.readyCh } + +func (m *HeadManager) ensureReady() { + // if m wasn't ready before, mark it ready + select { + case <-m.Ready(): + default: + close(m.readyCh) + } +} + +func (m *HeadManager) Append(userID string, ls labels.Labels, chks index.ChunkMetas) error { + m.mtx.RLock() + defer m.mtx.RUnlock() + rec := m.activeHeads.Append(userID, ls, chks) + return m.active.Log(rec) +} + +/* +Rotate starts using a new WAL and builds a TSDB index from the old one. +Once a WAL is shipped successfully to storage, it's moved into the shipped directory. +tsdb/ + wal/ + pending/ + - + shipped/ + - +*/ +func (m *HeadManager) Rotate() (err error) { + // First, set up the next WAL + + t := time.Now() + nextPeriod := t.UnixNano() / m.period.Nanoseconds() + + if m.activeHeads != nil && m.activeHeads.period == nextPeriod { + return nil + } + + nextWAL, err := newHeadWAL(m.log, m.pendingPath(nextPeriod)) + if err != nil { + return err + } + if err := nextWAL.Start(t); err != nil { + return err + } + + nextHeads := newTenantHeads(nextPeriod, m.shards, m.metrics, m.log) + + m.mtx.Lock() + oldWAL := m.active + oldHeads := m.activeHeads + + m.active = nextWAL + m.activeHeads = nextHeads + m.mtx.Unlock() + + m.ensureReady() + + if err := oldWAL.Stop(); err != nil { + return err + } + + return m.flusher.Flush(oldHeads.period, oldWAL) +} + +// TODO(owen-d): placeholder for shipper +type flusher interface { + Flush(period int64, wal *headWAL) error +} + +func (m *HeadManager) pendingPath(period int64) string { + return filepath.Join(m.dir, "pending", fmt.Sprintf("%d-%s", period, m.name)) +} + +func (m *HeadManager) shippedPath(period int64) string { + return filepath.Join(m.dir, "shipped", fmt.Sprintf("%d-%s", period, m.name)) +} + +type tenantHeads struct { + period int64 + shards int + locks []sync.RWMutex + tenants []map[string]*Head + log log.Logger + metrics *HeadMetrics +} + +func newTenantHeads(period int64, shards int, metrics *HeadMetrics, log log.Logger) *tenantHeads { + res := &tenantHeads{ + period: period, + shards: shards, + locks: make([]sync.RWMutex, shards), + tenants: make([]map[string]*Head, shards), + log: log, + metrics: metrics, + } + for i := range res.tenants { + res.tenants[i] = make(map[string]*Head) + } + return res +} + +func (t *tenantHeads) Append(userID string, ls labels.Labels, chks index.ChunkMetas) *WALRecord { + idx := xxhash.Sum64String(userID) & uint64(t.shards-1) + + // First, check if this tenant has been created + var ( + mtx = &t.locks[idx] + newStream bool + refID uint64 + ) + mtx.RLock() + if head, ok := t.tenants[idx][userID]; ok { + newStream, refID = head.Append(ls, chks) + mtx.RUnlock() + } else { + // tenant does not exist, so acquire write lock to insert it + mtx.RUnlock() + mtx.Lock() + head := NewHead(userID, t.metrics, t.log) + t.tenants[idx][userID] = head + newStream, refID = head.Append(ls, chks) + mtx.Unlock() + } + + rec := &WALRecord{ + UserID: userID, + Chks: ChunkMetasRecord{ + Ref: refID, + Chks: chks, + }, + } + + if newStream { + rec.Series = record.RefSeries{ + Ref: chunks.HeadSeriesRef(refID), + Labels: ls, + } + } + + return rec +} From 5a07e53c05b5c18450a289a6c80613b7531c58f9 Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Thu, 14 Apr 2022 17:14:02 -0400 Subject: [PATCH 16/85] tsdb head manager wal initialization --- pkg/ingester/ingester.go | 2 +- pkg/ingester/recovery.go | 2 +- pkg/storage/stores/tsdb/head_manager.go | 406 ++++++++++++++++++------ pkg/storage/stores/tsdb/head_wal.go | 45 +-- pkg/storage/stores/tsdb/manager.go | 12 + 5 files changed, 336 insertions(+), 131 deletions(-) create mode 100644 pkg/storage/stores/tsdb/manager.go diff --git a/pkg/ingester/ingester.go b/pkg/ingester/ingester.go index 25b6cfe6d3b87..2b49f5c460651 100644 --- a/pkg/ingester/ingester.go +++ b/pkg/ingester/ingester.go @@ -420,7 +420,7 @@ func (i *Ingester) starting(ctx context.Context) error { ) level.Info(util_log.Logger).Log("msg", "recovering from WAL") - segmentReader, segmentCloser, err := newWalReader(i.cfg.WAL.Dir, -1) + segmentReader, segmentCloser, err := NewWalReader(i.cfg.WAL.Dir, -1) if err != nil { return err } diff --git a/pkg/ingester/recovery.go b/pkg/ingester/recovery.go index cb27a14a27e90..861b52a85d9bd 100644 --- a/pkg/ingester/recovery.go +++ b/pkg/ingester/recovery.go @@ -31,7 +31,7 @@ func (NoopWALReader) Record() []byte { return nil } func (NoopWALReader) Close() error { return nil } // If startSegment is <0, it means all the segments. -func newWalReader(dir string, startSegment int) (*wal.Reader, io.Closer, error) { +func NewWalReader(dir string, startSegment int) (*wal.Reader, io.Closer, error) { var ( segmentReader io.ReadCloser err error diff --git a/pkg/storage/stores/tsdb/head_manager.go b/pkg/storage/stores/tsdb/head_manager.go index 700671999f9d1..1cb2733529fea 100644 --- a/pkg/storage/stores/tsdb/head_manager.go +++ b/pkg/storage/stores/tsdb/head_manager.go @@ -2,31 +2,82 @@ package tsdb import ( "fmt" + "io/ioutil" + "os" "path/filepath" + "sort" + "strconv" + "strings" "sync" "time" "github.com/cespare/xxhash" "github.com/go-kit/log" - "github.com/go-kit/log/level" - "github.com/grafana/loki/pkg/storage/stores/tsdb/index" + "github.com/pkg/errors" "github.com/prometheus/client_golang/prometheus" "github.com/prometheus/prometheus/model/labels" "github.com/prometheus/prometheus/tsdb/chunks" "github.com/prometheus/prometheus/tsdb/record" + + "github.com/grafana/loki/pkg/ingester" + "github.com/grafana/loki/pkg/storage/chunk/client/util" + "github.com/grafana/loki/pkg/storage/stores/tsdb/index" ) -const defaultRotationPeriod = 2 * time.Hour +const defaultRotationPeriod = 15 * time.Minute // Do not specify without bit shifting. This allows us to // do shard index calcuations via bitwise & rather than modulos. const defaultHeadManagerStripeSize = 1 << 7 -// HeadManager both accepts flushed chunk writes -// and exposes the index interface for multiple tenants. -// It also handles updating an underlying WAL and periodically -// rotates both the tenant Heads and the underlying WAL, using -// the old versions to build TSDB files. +/* +HeadManager both accepts flushed chunk writes +and exposes the index interface for multiple tenants. +It also handles updating an underlying WAL and periodically +rotates both the tenant Heads and the underlying WAL, using +the old versions to build + upload TSDB files. + +The basic algorithm is: +# Initialization +- clear scratch dir +- remove any old shipped TSDBs +- get list of all WALs + - Build/ship a TSDB for anything before currPeriod + - Delete WALs after + - Recover currPeriod WALs into memory. + - Load currPeriod-1 from TSDB dir + +# Rotation +- if currPeriod > activePeriod, + - create new wal+tenantHeads, make it active + - push old into prev + - build tsdb from prev + - under mtx, load tsdb-prev from file instead of tenantHeads and remove prev. + - remove any older tsdbs than activePeriod-1 + +On disk, it looks like: + +tsdb/ + # scratch directory used for temp tsdb files during build stage + scratch/ + wal/ + - + # multitenant tsdb files which are created on the ingesters/shipped + multitenant/ + # contains built TSDBs + built/ + -.tsdb + # once shipped successfully, they're moved here and can be safely deleted later + shipped/ + -.tsdb + compacted/ + # post-compaction tenant tsdbs which are grouped per + # period bucket + / + / + index---.tsdb +*/ + type HeadManager struct { name string log log.Logger @@ -40,128 +91,305 @@ type HeadManager struct { // how often WALs should be rotated and TSDBs cut period time.Duration - active *headWAL + tsdbManager TSDBManager + active *headWAL - shards int - activeHeads *tenantHeads - oldHeads []*tenantHeads - - readyCh chan struct{} - flusher flusher + shards int + activeHeads, prevHeads *tenantHeads } -func NewHeadManager(log log.Logger, dir string, reg prometheus.Registerer, name string, flusher flusher) *HeadManager { - shards := 128 +func NewHeadManager(log log.Logger, dir string, reg prometheus.Registerer, name string, tsdbManager TSDBManager) *HeadManager { + shards := defaultHeadManagerStripeSize metrics := NewHeadMetrics(reg) return &HeadManager{ - name: name, - log: log, - dir: dir, - metrics: metrics, - - period: defaultRotationPeriod, - shards: shards, - flusher: flusher, - readyCh: make(chan struct{}), + name: name, + log: log, + dir: dir, + metrics: metrics, + tsdbManager: tsdbManager, + + period: defaultRotationPeriod, + shards: shards, } } -func (m *HeadManager) Run() { - // Run once immediately - if err := m.Rotate(); err != nil { - level.Error(m.log).Log("msg", "failed to rotate TSDB WAL", "error", err.Error()) - } - - ticker := time.NewTicker(m.period) - for { - <-ticker.C +func (m *HeadManager) Append(userID string, ls labels.Labels, chks index.ChunkMetas) error { + m.mtx.RLock() + if m.PeriodFor(time.Now()) > m.PeriodFor(m.activeHeads.start) { + m.mtx.RUnlock() if err := m.Rotate(); err != nil { - level.Error(m.log).Log("msg", "failed to rotate TSDB WAL", "error", err.Error()) + return errors.Wrap(err, "rotating TSDB Head") } + m.mtx.RLock() } -} - -func (m *HeadManager) Ready() <-chan struct{} { return m.readyCh } - -func (m *HeadManager) ensureReady() { - // if m wasn't ready before, mark it ready - select { - case <-m.Ready(): - default: - close(m.readyCh) - } -} - -func (m *HeadManager) Append(userID string, ls labels.Labels, chks index.ChunkMetas) error { - m.mtx.RLock() defer m.mtx.RUnlock() rec := m.activeHeads.Append(userID, ls, chks) return m.active.Log(rec) } -/* -Rotate starts using a new WAL and builds a TSDB index from the old one. -Once a WAL is shipped successfully to storage, it's moved into the shipped directory. -tsdb/ - wal/ - pending/ - - - shipped/ - - -*/ -func (m *HeadManager) Rotate() (err error) { - // First, set up the next WAL +func (m *HeadManager) PeriodFor(t time.Time) int { + return int(t.UnixNano() / int64(m.period)) +} + +func (m *HeadManager) TimeForPeriod(period int) time.Time { + return time.Unix(0, int64(m.period)*int64(period)) +} - t := time.Now() - nextPeriod := t.UnixNano() / m.period.Nanoseconds() +func (m *HeadManager) Start() error { + if err := os.RemoveAll(filepath.Join(m.dir, "scratch")); err != nil { + return errors.Wrap(err, "removing tsdb scratch dir") + } - if m.activeHeads != nil && m.activeHeads.period == nextPeriod { - return nil + for _, d := range m.RequiredDirs() { + if err := util.EnsureDirectory(d); err != nil { + return errors.Wrapf(err, "ensuring required directory exists: %s", d) + } } - nextWAL, err := newHeadWAL(m.log, m.pendingPath(nextPeriod)) + now := time.Now() + curPeriod := m.PeriodFor(now) + + toRemove, err := m.shippedTSDBsBeforePeriod(curPeriod) if err != nil { return err } - if err := nextWAL.Start(t); err != nil { + + for _, x := range toRemove { + if err := os.RemoveAll(x); err != nil { + return errors.Wrapf(err, "removing tsdb: %s", x) + } + } + + walsByPeriod, err := m.walsByPeriod() + if err != nil { return err } - nextHeads := newTenantHeads(nextPeriod, m.shards, m.metrics, m.log) + m.activeHeads = newTenantHeads(now, m.shards, m.metrics, m.log) + + for _, group := range walsByPeriod { + if group.period < (curPeriod) { + if err := m.tsdbManager.BuildFromWALs( + m.TimeForPeriod(group.period), + group.wals, + ); err != nil { + return errors.Wrap(err, "building tsdb") + } + // Now that we've built tsdbs of this data, we can safely remove the WALs + if err := m.removeWALGroup(group); err != nil { + return errors.Wrapf(err, "removing wals for period %d", group.period) + } + } - m.mtx.Lock() - oldWAL := m.active - oldHeads := m.activeHeads + if group.period == curPeriod { + if err := m.recoverHead(group); err != nil { + return errors.Wrap(err, "recovering tsdb head from wal") + } + } + } + nextWALPath := m.walPath(now) + nextWAL, err := newHeadWAL(m.log, nextWALPath) + if err != nil { + return errors.Wrapf(err, "creating tsdb wal: %s", nextWALPath) + } m.active = nextWAL - m.activeHeads = nextHeads - m.mtx.Unlock() - m.ensureReady() + return nil +} - if err := oldWAL.Stop(); err != nil { - return err +func (m *HeadManager) RequiredDirs() []string { + return []string{ + m.scratchDir(), + m.walDir(), + m.builtDir(), + m.shippedDir(), + } +} +func (m *HeadManager) scratchDir() string { return filepath.Join(m.dir, "scratch") } +func (m *HeadManager) walDir() string { return filepath.Join(m.dir, "wal") } +func (m *HeadManager) builtDir() string { return filepath.Join(m.dir, "multitenant", "built") } +func (m *HeadManager) shippedDir() string { return filepath.Join(m.dir, "multitenant", "shipped") } + +func (m *HeadManager) Rotate() error { + panic("unimplemented") +} + +func (m *HeadManager) shippedTSDBsBeforePeriod(period int) (res []string, err error) { + files, err := ioutil.ReadDir(m.shippedDir()) + if err != nil { + return nil, err + } + for _, f := range files { + if id, ok := parseTSDBPath(f.Name()); ok { + if found := m.PeriodFor(id.ts); found < period { + res = append(res, f.Name()) + } + } + } + return +} + +type walGroup struct { + period int + wals []WALIdentifier +} + +func (m *HeadManager) walsByPeriod() ([]walGroup, error) { + files, err := ioutil.ReadDir(m.walDir()) + if err != nil { + return nil, err + } + + groupsMap := map[int]*walGroup{} + + for _, f := range files { + if id, ok := parseWALPath(f.Name()); ok { + pd := m.PeriodFor(id.ts) + grp, ok := groupsMap[pd] + if !ok { + grp := walGroup{ + period: pd, + } + groupsMap[pd] = &grp + } + grp.wals = append(grp.wals, id) + } + } + + res := make([]walGroup, 0, len(groupsMap)) + for _, grp := range groupsMap { + // Ensure the earliest wals are seen first + sort.Slice(grp.wals, func(i, j int) bool { + return grp.wals[i].ts.Before(grp.wals[j].ts) + }) + res = append(res, *grp) + } + // Ensure the earliers periods are seen first + sort.Slice(res, func(i, j int) bool { + return res[i].period < res[j].period + }) + return res, nil +} + +func (m *HeadManager) removeWALGroup(grp walGroup) error { + for _, wal := range grp.wals { + if err := os.RemoveAll(m.walPath(wal.ts)); err != nil { + return errors.Wrapf(err, "removing tsdb wal: %s", m.walPath(wal.ts)) + } } + return nil +} - return m.flusher.Flush(oldHeads.period, oldWAL) +func (m *HeadManager) walPath(t time.Time) string { + return filepath.Join( + m.walDir(), + fmt.Sprintf("%d-%s", t.Unix(), m.name), + ) +} + +// recoverHead recovers from all WALs belonging to some period +// and inserts it into the active *tenantHeads +func (m *HeadManager) recoverHead(grp walGroup) error { + for _, id := range grp.wals { + + // use anonymous function for ease of cleanup + if err := func() error { + reader, closer, err := ingester.NewWalReader(m.walPath(id.ts), -1) + if err != nil { + return err + } + defer closer.Close() + + // map of users -> ref -> series. + // Keep track of which ref corresponds to which series + // for each WAL so we replay into the correct series + seriesMap := make(map[string]map[uint64]labels.Labels) + + for reader.Next() { + rec := &WALRecord{} + if err := decodeWALRecord(reader.Record(), rec); err != nil { + return err + } + + if len(rec.Series.Labels) > 0 { + tenant, ok := seriesMap[rec.UserID] + if !ok { + tenant = make(map[uint64]labels.Labels) + seriesMap[rec.UserID] = tenant + } + tenant[uint64(rec.Series.Ref)] = rec.Series.Labels + } + + if len(rec.Chks.Chks) > 0 { + tenant, ok := seriesMap[rec.UserID] + if !ok { + return errors.New("found tsdb chunk metas without user in WAL replay") + } + ls, ok := tenant[rec.Chks.Ref] + if !ok { + return errors.New("found tsdb chunk metas without series in WAL replay") + } + _ = m.activeHeads.Append(rec.UserID, ls, rec.Chks.Chks) + } + } + return reader.Err() + + }(); err != nil { + return errors.Wrapf( + err, + "error recovering from TSDB WAL: %s", + m.walPath(id.ts), + ) + } + } } -// TODO(owen-d): placeholder for shipper -type flusher interface { - Flush(period int64, wal *headWAL) error +type WALIdentifier struct { + nodeName string + ts time.Time } +type MultitenantTSDBIdentifier WALIdentifier -func (m *HeadManager) pendingPath(period int64) string { - return filepath.Join(m.dir, "pending", fmt.Sprintf("%d-%s", period, m.name)) +func parseWALPath(p string) (id WALIdentifier, ok bool) { + xs := strings.Split(p, "-") + if len(xs) != 2 { + return + } + + // require node name isn't empty + if len(xs[1]) == 0 { + return + } + + period, err := strconv.Atoi(xs[0]) + if err != nil { + return + } + + return WALIdentifier{ + ts: time.Unix(int64(period), 0), + nodeName: xs[1], + }, true } -func (m *HeadManager) shippedPath(period int64) string { - return filepath.Join(m.dir, "shipped", fmt.Sprintf("%d-%s", period, m.name)) +func parseTSDBPath(p string) (id MultitenantTSDBIdentifier, ok bool) { + trimmed := strings.TrimSuffix(p, ".tsdb") + + // incorrect suffix + if trimmed == p { + return + } + + if found, ok := parseWALPath(trimmed); ok { + return MultitenantTSDBIdentifier(found), true + } + return } type tenantHeads struct { - period int64 + start time.Time shards int locks []sync.RWMutex tenants []map[string]*Head @@ -169,9 +397,9 @@ type tenantHeads struct { metrics *HeadMetrics } -func newTenantHeads(period int64, shards int, metrics *HeadMetrics, log log.Logger) *tenantHeads { +func newTenantHeads(start time.Time, shards int, metrics *HeadMetrics, log log.Logger) *tenantHeads { res := &tenantHeads{ - period: period, + start: start, shards: shards, locks: make([]sync.RWMutex, shards), tenants: make([]map[string]*Head, shards), diff --git a/pkg/storage/stores/tsdb/head_wal.go b/pkg/storage/stores/tsdb/head_wal.go index 2ba0d72d9fe78..4c78dae0d9074 100644 --- a/pkg/storage/stores/tsdb/head_wal.go +++ b/pkg/storage/stores/tsdb/head_wal.go @@ -42,10 +42,9 @@ const ( ) type WALRecord struct { - UserID string - StartTime int64 // UnixNano - Series record.RefSeries - Chks ChunkMetasRecord + UserID string + Series record.RefSeries + Chks ChunkMetasRecord } type ChunkMetasRecord struct { @@ -121,19 +120,6 @@ func decodeChunks(b []byte, version RecordType, rec *WALRecord) error { return nil } -func (r *WALRecord) encodeStartTime(b []byte) []byte { - buf := encoding.EncWith(b) - buf.PutByte(byte(WALRecordFirstWrite)) - buf.PutBE64int64(r.StartTime) - return buf.Get() -} - -func decodeStartTime(b []byte, rec *WALRecord) error { - dec := encoding.DecWith(b) - rec.StartTime = dec.Be64int64() - return dec.Err() -} - func decodeWALRecord(b []byte, walRec *WALRecord) error { var ( userID string @@ -144,10 +130,6 @@ func decodeWALRecord(b []byte, walRec *WALRecord) error { ) switch t { - case WALRecordFirstWrite: - if err := decodeStartTime(decbuf.B, walRec); err != nil { - return errors.Wrap(err, "decoding tsdb wal start time") - } case WalRecordSeries: userID = decbuf.UvarintStr() rSeries, err := dec.Series(decbuf.B, nil) @@ -181,9 +163,8 @@ func decodeWALRecord(b []byte, walRec *WALRecord) error { // 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 { - start time.Time - log log.Logger - wal *wal.WAL + log log.Logger + wal *wal.WAL } func newHeadWAL(log log.Logger, dir string) (*headWAL, error) { @@ -201,12 +182,6 @@ func newHeadWAL(log log.Logger, dir string) (*headWAL, error) { }, nil } -// Start logs a record containing the time at which this WAL became active. -func (w *headWAL) Start(t time.Time) error { - w.start = t - return w.Log(&WALRecord{StartTime: t.UnixNano()}) -} - func (w *headWAL) Stop() error { return w.wal.Close() } @@ -218,16 +193,6 @@ func (w *headWAL) Log(record *WALRecord) error { var buf []byte - // This only happens once when the wal is first written to. - // It could be refactored out of this function to avoid cpu cycles, - // but this is simpler and not on the hot path, so I've (owen-d) kept it. - if record.StartTime != 0 { - buf = record.encodeStartTime(buf[:0]) - if err := w.wal.Log(buf); err != nil { - return err - } - } - // Always write series before chunks if len(record.Series.Labels) > 0 { buf = record.encodeSeries(buf[:0]) diff --git a/pkg/storage/stores/tsdb/manager.go b/pkg/storage/stores/tsdb/manager.go new file mode 100644 index 0000000000000..dad2e98caa2f7 --- /dev/null +++ b/pkg/storage/stores/tsdb/manager.go @@ -0,0 +1,12 @@ +package tsdb + +import "time" + +type TSDBManager interface { + Index + + // Builds a new TSDB file from a set of WALs + BuildFromWALs(time.Time, []WALIdentifier) error +} + +type tsdbManger struct{} From d83b86eceeca88e13dd3a46ec9bc16f96be7758b Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Thu, 14 Apr 2022 18:33:33 -0400 Subject: [PATCH 17/85] tsdb wal rotation --- pkg/storage/stores/tsdb/head_manager.go | 132 +++++++++++++++++------ pkg/storage/stores/tsdb/head_wal.go | 12 ++- pkg/storage/stores/tsdb/head_wal_test.go | 24 +---- 3 files changed, 111 insertions(+), 57 deletions(-) diff --git a/pkg/storage/stores/tsdb/head_manager.go b/pkg/storage/stores/tsdb/head_manager.go index 1cb2733529fea..d5ec5922c5de1 100644 --- a/pkg/storage/stores/tsdb/head_manager.go +++ b/pkg/storage/stores/tsdb/head_manager.go @@ -13,6 +13,7 @@ import ( "github.com/cespare/xxhash" "github.com/go-kit/log" + "github.com/go-kit/log/level" "github.com/pkg/errors" "github.com/prometheus/client_golang/prometheus" "github.com/prometheus/prometheus/model/labels" @@ -37,24 +38,6 @@ It also handles updating an underlying WAL and periodically rotates both the tenant Heads and the underlying WAL, using the old versions to build + upload TSDB files. -The basic algorithm is: -# Initialization -- clear scratch dir -- remove any old shipped TSDBs -- get list of all WALs - - Build/ship a TSDB for anything before currPeriod - - Delete WALs after - - Recover currPeriod WALs into memory. - - Load currPeriod-1 from TSDB dir - -# Rotation -- if currPeriod > activePeriod, - - create new wal+tenantHeads, make it active - - push old into prev - - build tsdb from prev - - under mtx, load tsdb-prev from file instead of tenantHeads and remove prev. - - remove any older tsdbs than activePeriod-1 - On disk, it looks like: tsdb/ @@ -91,8 +74,8 @@ type HeadManager struct { // how often WALs should be rotated and TSDBs cut period time.Duration - tsdbManager TSDBManager - active *headWAL + tsdbManager TSDBManager + active, prev *headWAL shards int activeHeads, prevHeads *tenantHeads @@ -116,9 +99,10 @@ func NewHeadManager(log log.Logger, dir string, reg prometheus.Registerer, name func (m *HeadManager) Append(userID string, ls labels.Labels, chks index.ChunkMetas) error { m.mtx.RLock() - if m.PeriodFor(time.Now()) > m.PeriodFor(m.activeHeads.start) { + now := time.Now() + if m.PeriodFor(now) > m.PeriodFor(m.activeHeads.start) { m.mtx.RUnlock() - if err := m.Rotate(); err != nil { + if err := m.Rotate(now); err != nil { return errors.Wrap(err, "rotating TSDB Head") } m.mtx.RLock() @@ -190,7 +174,7 @@ func (m *HeadManager) Start() error { } nextWALPath := m.walPath(now) - nextWAL, err := newHeadWAL(m.log, nextWALPath) + nextWAL, err := newHeadWAL(m.log, nextWALPath, now) if err != nil { return errors.Wrapf(err, "creating tsdb wal: %s", nextWALPath) } @@ -212,8 +196,66 @@ func (m *HeadManager) walDir() string { return filepath.Join(m.dir, "wal") } func (m *HeadManager) builtDir() string { return filepath.Join(m.dir, "multitenant", "built") } func (m *HeadManager) shippedDir() string { return filepath.Join(m.dir, "multitenant", "shipped") } -func (m *HeadManager) Rotate() error { - panic("unimplemented") +func (m *HeadManager) Rotate(t time.Time) error { + // create new wal + nextWALPath := m.walPath(t) + nextWAL, err := newHeadWAL(m.log, nextWALPath, t) + if err != nil { + return errors.Wrapf(err, "creating tsdb wal: %s during rotation", nextWALPath) + } + + // create new tenant heads + nextHeads := newTenantHeads(t, m.shards, m.metrics, m.log) + + stopPrev := func(s string) { + if m.prev != nil { + if err := m.prev.Stop(); err != nil { + level.Error(m.log).Log( + "msg", "failed stopping wal", + "period", m.PeriodFor(m.prev.initialized), + "err", err, + "wal", s, + ) + } + } + } + + stopPrev("previous cycle") // stop the previous wal if it hasn't been cleaned up yet + m.mtx.Lock() + m.prev = m.active + m.prevHeads = m.activeHeads + m.active = nextWAL + m.activeHeads = nextHeads + m.mtx.Unlock() + stopPrev("freshly rotated") // stop the newly rotated-out wal + + // build tsdb from rotated-out period + grp, _, err := m.walsForPeriod(m.PeriodFor(m.prev.initialized)) + if err != nil { + return errors.Wrap(err, "listing wals") + } + + // TODO(owen-d): It's probably faster to build this from the *tenantHeads instead, + // but we already need to impl BuildFromWALs to ensure we can correctly build/ship + // TSDBs from orphaned WALs of previous periods during startup. + // we use the m.prev.initialized timestamp here for the filename to ensure it can't clobber + // an existing file from a previous cycle. I don't think this is possible, but + // perhaps in some unusual crashlooping it could be, so let's be safe and protect ourselves. + if err := m.tsdbManager.BuildFromWALs(m.prev.initialized, grp.wals); err != nil { + return errors.Wrapf(err, "building TSDB from prevHeads WALs for period %d", grp.period) + } + + // Now that a TSDB has been created from this group, it's safe to remove them + if err := m.removeWALGroup(grp); err != nil { + return errors.Wrapf(err, "removing prev TSDB WALs for period %d", grp.period) + } + + // Now that the tsdbManager has the updated TSDBs, we can remove our references + m.mtx.Lock() + m.prevHeads = nil + m.prev = nil + m.mtx.Unlock() + return nil } func (m *HeadManager) shippedTSDBsBeforePeriod(period int) (res []string, err error) { @@ -237,6 +279,23 @@ type walGroup struct { } func (m *HeadManager) walsByPeriod() ([]walGroup, error) { + + groupsMap, err := m.walGroups() + if err != nil { + return nil, err + } + res := make([]walGroup, 0, len(groupsMap)) + for _, grp := range groupsMap { + res = append(res, *grp) + } + // Ensure the earliers periods are seen first + sort.Slice(res, func(i, j int) bool { + return res[i].period < res[j].period + }) + return res, nil +} + +func (m *HeadManager) walGroups() (map[int]*walGroup, error) { files, err := ioutil.ReadDir(m.walDir()) if err != nil { return nil, err @@ -258,19 +317,27 @@ func (m *HeadManager) walsByPeriod() ([]walGroup, error) { } } - res := make([]walGroup, 0, len(groupsMap)) for _, grp := range groupsMap { // Ensure the earliest wals are seen first sort.Slice(grp.wals, func(i, j int) bool { return grp.wals[i].ts.Before(grp.wals[j].ts) }) - res = append(res, *grp) } - // Ensure the earliers periods are seen first - sort.Slice(res, func(i, j int) bool { - return res[i].period < res[j].period - }) - return res, nil + return groupsMap, nil +} + +func (m *HeadManager) walsForPeriod(period int) (walGroup, bool, error) { + groupsMap, err := m.walGroups() + if err != nil { + return walGroup{}, false, err + } + + grp, ok := groupsMap[period] + if !ok { + return walGroup{}, false, nil + } + + return *grp, true, nil } func (m *HeadManager) removeWALGroup(grp walGroup) error { @@ -344,6 +411,7 @@ func (m *HeadManager) recoverHead(grp walGroup) error { ) } } + return nil } type WALIdentifier struct { diff --git a/pkg/storage/stores/tsdb/head_wal.go b/pkg/storage/stores/tsdb/head_wal.go index 4c78dae0d9074..d307215d0ce6e 100644 --- a/pkg/storage/stores/tsdb/head_wal.go +++ b/pkg/storage/stores/tsdb/head_wal.go @@ -163,11 +163,12 @@ func decodeWALRecord(b []byte, walRec *WALRecord) error { // 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 { - log log.Logger - wal *wal.WAL + initialized time.Time + log log.Logger + wal *wal.WAL } -func newHeadWAL(log log.Logger, dir string) (*headWAL, error) { +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) @@ -177,8 +178,9 @@ func newHeadWAL(log log.Logger, dir string) (*headWAL, error) { } return &headWAL{ - log: log, - wal: wal, + initialized: t, + log: log, + wal: wal, }, nil } diff --git a/pkg/storage/stores/tsdb/head_wal_test.go b/pkg/storage/stores/tsdb/head_wal_test.go index 154f6b1fc2f6c..7dae49c9ca850 100644 --- a/pkg/storage/stores/tsdb/head_wal_test.go +++ b/pkg/storage/stores/tsdb/head_wal_test.go @@ -58,29 +58,14 @@ func Test_Encoding_Chunks(t *testing.T) { require.Equal(t, record, decoded) } -func Test_Encoding_StartTime(t *testing.T) { - record := &WALRecord{ - StartTime: time.Now().UnixNano(), - } - buf := record.encodeStartTime(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() - start := time.Now() - w, err := newHeadWAL(log.NewNopLogger(), dir) + w, err := newHeadWAL(log.NewNopLogger(), dir, time.Now()) require.Nil(t, err) - require.Nil(t, w.Start(start)) newSeries := &WALRecord{ - UserID: "foo", - StartTime: 0, - Series: record.RefSeries{Ref: 1, Labels: mustParseLabels(`{foo="bar"}`)}, + UserID: "foo", + Series: record.RefSeries{Ref: 1, Labels: mustParseLabels(`{foo="bar"}`)}, Chks: ChunkMetasRecord{ Chks: []index.ChunkMeta{ { @@ -97,8 +82,7 @@ func Test_HeadWALLog(t *testing.T) { require.Nil(t, w.Log(newSeries)) chunksOnly := &WALRecord{ - UserID: "foo", - StartTime: 0, + UserID: "foo", Chks: ChunkMetasRecord{ Chks: []index.ChunkMeta{ { From c629a2bca6db869ba60cc83909de007e01e27167 Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Thu, 14 Apr 2022 18:52:46 -0400 Subject: [PATCH 18/85] wals dont use node name, but tsdb files do --- pkg/storage/stores/tsdb/head_manager.go | 52 +++++++++++++++---------- 1 file changed, 31 insertions(+), 21 deletions(-) diff --git a/pkg/storage/stores/tsdb/head_manager.go b/pkg/storage/stores/tsdb/head_manager.go index d5ec5922c5de1..dc2d6d8bc1650 100644 --- a/pkg/storage/stores/tsdb/head_manager.go +++ b/pkg/storage/stores/tsdb/head_manager.go @@ -43,8 +43,11 @@ On disk, it looks like: tsdb/ # scratch directory used for temp tsdb files during build stage scratch/ + # wal directory used to store WALs being written on the ingester. + # These are eventually shipped to storage as multi-tenant TSDB files + # and compacted into per tenant indices wal/ - - + # multitenant tsdb files which are created on the ingesters/shipped multitenant/ # contains built TSDBs @@ -352,7 +355,7 @@ func (m *HeadManager) removeWALGroup(grp walGroup) error { func (m *HeadManager) walPath(t time.Time) string { return filepath.Join( m.walDir(), - fmt.Sprintf("%d-%s", t.Unix(), m.name), + fmt.Sprintf("%d", t.Unix()), ) } @@ -415,33 +418,25 @@ func (m *HeadManager) recoverHead(grp walGroup) error { } type WALIdentifier struct { - nodeName string - ts time.Time + ts time.Time } -type MultitenantTSDBIdentifier WALIdentifier func parseWALPath(p string) (id WALIdentifier, ok bool) { - xs := strings.Split(p, "-") - if len(xs) != 2 { - return - } - - // require node name isn't empty - if len(xs[1]) == 0 { - return - } - - period, err := strconv.Atoi(xs[0]) + ts, err := strconv.Atoi(p) if err != nil { return } return WALIdentifier{ - ts: time.Unix(int64(period), 0), - nodeName: xs[1], + ts: time.Unix(int64(ts), 0), }, true } +type MultitenantTSDBIdentifier struct { + nodeName string + ts time.Time +} + func parseTSDBPath(p string) (id MultitenantTSDBIdentifier, ok bool) { trimmed := strings.TrimSuffix(p, ".tsdb") @@ -450,10 +445,25 @@ func parseTSDBPath(p string) (id MultitenantTSDBIdentifier, ok bool) { return } - if found, ok := parseWALPath(trimmed); ok { - return MultitenantTSDBIdentifier(found), true + xs := strings.Split(trimmed, "-") + if len(xs) != 2 { + return } - return + + // require node name isn't empty + if len(xs[1]) == 0 { + return + } + + ts, err := strconv.Atoi(xs[0]) + if err != nil { + return + } + + return MultitenantTSDBIdentifier{ + ts: time.Unix(int64(ts), 0), + nodeName: xs[1], + }, true } type tenantHeads struct { From c115627f0f5d27efeb3d73a3000f6fa2cf11c05d Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Thu, 14 Apr 2022 19:02:18 -0400 Subject: [PATCH 19/85] cleans up fn signature --- pkg/storage/stores/tsdb/head_wal.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/pkg/storage/stores/tsdb/head_wal.go b/pkg/storage/stores/tsdb/head_wal.go index d307215d0ce6e..dee7d1c9506d1 100644 --- a/pkg/storage/stores/tsdb/head_wal.go +++ b/pkg/storage/stores/tsdb/head_wal.go @@ -84,7 +84,7 @@ func (r *WALRecord) encodeChunks(b []byte) []byte { return buf.Get() } -func decodeChunks(b []byte, version RecordType, rec *WALRecord) error { +func decodeChunks(b []byte, rec *WALRecord) error { if len(b) == 0 { return nil } @@ -145,7 +145,7 @@ func decodeWALRecord(b []byte, walRec *WALRecord) error { } case WalRecordChunks: userID = decbuf.UvarintStr() - if err := decodeChunks(decbuf.B, t, walRec); err != nil { + if err := decodeChunks(decbuf.B, walRec); err != nil { return err } default: From 4eaa1f3c78ca2b76e53bb6ccd8a0da72e87fa5b4 Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Fri, 15 Apr 2022 09:34:49 -0400 Subject: [PATCH 20/85] multi tsdb idx now just wraps Index interfaces --- pkg/storage/stores/tsdb/compact.go | 16 ++++++++++++++-- pkg/storage/stores/tsdb/multi_file_index.go | 14 +++++++------- pkg/storage/stores/tsdb/multi_file_index_test.go | 2 +- 3 files changed, 22 insertions(+), 10 deletions(-) diff --git a/pkg/storage/stores/tsdb/compact.go b/pkg/storage/stores/tsdb/compact.go index a5df265ff8252..397be46102d42 100644 --- a/pkg/storage/stores/tsdb/compact.go +++ b/pkg/storage/stores/tsdb/compact.go @@ -2,6 +2,7 @@ package tsdb import ( "context" + "fmt" "github.com/prometheus/common/model" "github.com/prometheus/prometheus/model/labels" @@ -29,18 +30,29 @@ func (c *Compactor) Compact(ctx context.Context, indices ...*TSDBIndex) (res ind return indices[0].Identifier(c.tenant), nil } + ifcs := make([]Index, 0, len(indices)) + for _, idx := range indices { + ifcs = append(ifcs, idx) + } + b := index.NewBuilder() - multi, err := NewMultiIndex(indices...) + multi, err := NewMultiIndex(ifcs...) if err != nil { return res, err } + // TODO(owen-d): introduce parallelism + // Until then, // Instead of using the MultiIndex.forIndices helper, we loop over each sub-index manually. // The index builder is single threaded, so we avoid races. // Additionally, this increases the likelihood we add chunks in order // by processing the indices in ascending order. for _, idx := range multi.indices { - if err := idx.forSeries( + casted, ok := idx.(*TSDBIndex) + if !ok { + return index.Identifier{}, fmt.Errorf("expected tsdb index to compact, found :%T", idx) + } + if err := casted.forSeries( nil, func(ls labels.Labels, _ model.Fingerprint, chks []index.ChunkMeta) { // AddSeries copies chks into it's own slice diff --git a/pkg/storage/stores/tsdb/multi_file_index.go b/pkg/storage/stores/tsdb/multi_file_index.go index daa6240f58bc8..0dbc7ef0cec16 100644 --- a/pkg/storage/stores/tsdb/multi_file_index.go +++ b/pkg/storage/stores/tsdb/multi_file_index.go @@ -13,10 +13,10 @@ import ( ) type MultiIndex struct { - indices []*TSDBIndex + indices []Index } -func NewMultiIndex(indices ...*TSDBIndex) (*MultiIndex, error) { +func NewMultiIndex(indices ...Index) (*MultiIndex, error) { if len(indices) == 0 { return nil, errors.New("must supply at least one index") } @@ -51,7 +51,7 @@ func (i *MultiIndex) Bounds() (model.Time, model.Time) { return lowest, highest } -func (i *MultiIndex) forIndices(ctx context.Context, from, through model.Time, fn func(context.Context, *TSDBIndex) (interface{}, error)) ([]interface{}, error) { +func (i *MultiIndex) forIndices(ctx context.Context, from, through model.Time, fn func(context.Context, Index) (interface{}, error)) ([]interface{}, error) { queryBounds := newBounds(from, through) g, ctx := errgroup.WithContext(ctx) @@ -94,7 +94,7 @@ func (i *MultiIndex) GetChunkRefs(ctx context.Context, userID string, from, thro } res = res[:0] - groups, err := i.forIndices(ctx, from, through, func(ctx context.Context, idx *TSDBIndex) (interface{}, error) { + groups, err := i.forIndices(ctx, from, through, func(ctx context.Context, idx Index) (interface{}, error) { return idx.GetChunkRefs(ctx, userID, from, through, nil, shard, matchers...) }) if err != nil { @@ -128,7 +128,7 @@ func (i *MultiIndex) Series(ctx context.Context, userID string, from, through mo } res = res[:0] - groups, err := i.forIndices(ctx, from, through, func(ctx context.Context, idx *TSDBIndex) (interface{}, error) { + groups, err := i.forIndices(ctx, from, through, func(ctx context.Context, idx Index) (interface{}, error) { return idx.Series(ctx, userID, from, through, nil, shard, matchers...) }) if err != nil { @@ -154,7 +154,7 @@ func (i *MultiIndex) Series(ctx context.Context, userID string, from, through mo } func (i *MultiIndex) LabelNames(ctx context.Context, userID string, from, through model.Time, matchers ...*labels.Matcher) ([]string, error) { - groups, err := i.forIndices(ctx, from, through, func(ctx context.Context, idx *TSDBIndex) (interface{}, error) { + groups, err := i.forIndices(ctx, from, through, func(ctx context.Context, idx Index) (interface{}, error) { return idx.LabelNames(ctx, userID, from, through, matchers...) }) if err != nil { @@ -189,7 +189,7 @@ func (i *MultiIndex) LabelNames(ctx context.Context, userID string, from, throug } func (i *MultiIndex) LabelValues(ctx context.Context, userID string, from, through model.Time, name string, matchers ...*labels.Matcher) ([]string, error) { - groups, err := i.forIndices(ctx, from, through, func(ctx context.Context, idx *TSDBIndex) (interface{}, error) { + groups, err := i.forIndices(ctx, from, through, func(ctx context.Context, idx Index) (interface{}, error) { return idx.LabelValues(ctx, userID, from, through, name, matchers...) }) if err != nil { diff --git a/pkg/storage/stores/tsdb/multi_file_index_test.go b/pkg/storage/stores/tsdb/multi_file_index_test.go index fc5b0253ce57b..aaf00324b65e7 100644 --- a/pkg/storage/stores/tsdb/multi_file_index_test.go +++ b/pkg/storage/stores/tsdb/multi_file_index_test.go @@ -58,7 +58,7 @@ func TestMultiIndex(t *testing.T) { // group 5 indices together, all with duplicate data n := 5 - var indices []*TSDBIndex + var indices []Index dir := t.TempDir() for i := 0; i < n; i++ { indices = append(indices, BuildIndex(t, dir, "fake", cases)) From 9dc723481ebffc1c526197cd75c7be41fc14914e Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Fri, 15 Apr 2022 09:38:36 -0400 Subject: [PATCH 21/85] no longer sorts indices when creating multi-idx --- pkg/storage/stores/tsdb/multi_file_index.go | 12 ------------ 1 file changed, 12 deletions(-) diff --git a/pkg/storage/stores/tsdb/multi_file_index.go b/pkg/storage/stores/tsdb/multi_file_index.go index 0dbc7ef0cec16..24c1c6996829c 100644 --- a/pkg/storage/stores/tsdb/multi_file_index.go +++ b/pkg/storage/stores/tsdb/multi_file_index.go @@ -3,7 +3,6 @@ package tsdb import ( "context" "errors" - "sort" "github.com/prometheus/common/model" "github.com/prometheus/prometheus/model/labels" @@ -21,17 +20,6 @@ func NewMultiIndex(indices ...Index) (*MultiIndex, error) { return nil, errors.New("must supply at least one index") } - sort.Slice(indices, func(i, j int) bool { - aFrom, aThrough := indices[i].Bounds() - bFrom, bThrough := indices[j].Bounds() - - if aFrom != bFrom { - return aFrom < bFrom - } - // tiebreaker uses through - return aThrough <= bThrough - }) - return &MultiIndex{indices: indices}, nil } From bf4b125385633c62094899982600d4f3d6ea3727 Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Fri, 15 Apr 2022 10:34:14 -0400 Subject: [PATCH 22/85] tenantHeads & HeadManger index impls --- pkg/storage/stores/tsdb/head.go | 15 ++-- pkg/storage/stores/tsdb/head_manager.go | 96 ++++++++++++++++++++++++- 2 files changed, 103 insertions(+), 8 deletions(-) diff --git a/pkg/storage/stores/tsdb/head.go b/pkg/storage/stores/tsdb/head.go index aa6f7c0c82399..8e8fffa0673aa 100644 --- a/pkg/storage/stores/tsdb/head.go +++ b/pkg/storage/stores/tsdb/head.go @@ -110,22 +110,23 @@ func (h *Head) MaxTime() int64 { return h.maxTime.Load() } -func (h *Head) updateMinMaxTime(mint, maxt int64) { +// Will CAS until successfully updates bounds or the condition is no longer valid +func updateMintMaxt(mint, maxt int64, mintSrc, maxtSrc *atomic.Int64) { for { - lt := h.MinTime() - if mint >= lt { + lt := mintSrc.Load() + if mint >= lt && lt != 0 { break } - if h.minTime.CAS(lt, mint) { + if mintSrc.CAS(lt, mint) { break } } for { - ht := h.MaxTime() + ht := maxtSrc.Load() if maxt <= ht { break } - if h.maxTime.CAS(ht, maxt) { + if maxtSrc.CAS(ht, maxt) { break } } @@ -139,7 +140,7 @@ func (h *Head) Append(ls labels.Labels, chks index.ChunkMetas) (created bool, re id = h.lastSeriesID.Inc() return newMemSeries(id, ls) }) - h.updateMinMaxTime(int64(from), int64(through)) + updateMintMaxt(int64(from), int64(through), &h.minTime, &h.maxTime) if !created { return diff --git a/pkg/storage/stores/tsdb/head_manager.go b/pkg/storage/stores/tsdb/head_manager.go index dc2d6d8bc1650..25269933763d9 100644 --- a/pkg/storage/stores/tsdb/head_manager.go +++ b/pkg/storage/stores/tsdb/head_manager.go @@ -1,6 +1,7 @@ package tsdb import ( + "context" "fmt" "io/ioutil" "os" @@ -16,9 +17,11 @@ import ( "github.com/go-kit/log/level" "github.com/pkg/errors" "github.com/prometheus/client_golang/prometheus" + "github.com/prometheus/common/model" "github.com/prometheus/prometheus/model/labels" "github.com/prometheus/prometheus/tsdb/chunks" "github.com/prometheus/prometheus/tsdb/record" + "go.uber.org/atomic" "github.com/grafana/loki/pkg/ingester" "github.com/grafana/loki/pkg/storage/chunk/client/util" @@ -261,6 +264,21 @@ func (m *HeadManager) Rotate(t time.Time) error { return nil } +func (m *HeadManager) Index() (Index, error) { + m.mtx.RLock() + defer m.mtx.RUnlock() + + indices := []Index{m.tsdbManager} + if m.prev != nil { + indices = append(indices, m.prevHeads) + } + if m.active != nil { + indices = append(indices, m.activeHeads) + } + + return NewMultiIndex(indices...) +} + func (m *HeadManager) shippedTSDBsBeforePeriod(period int) (res []string, err error) { files, err := ioutil.ReadDir(m.shippedDir()) if err != nil { @@ -467,6 +485,8 @@ func parseTSDBPath(p string) (id MultitenantTSDBIdentifier, ok bool) { } type tenantHeads struct { + mint, maxt atomic.Int64 // easy lookup for Bounds() impl + start time.Time shards int locks []sync.RWMutex @@ -491,7 +511,19 @@ func newTenantHeads(start time.Time, shards int, metrics *HeadMetrics, log log.L } func (t *tenantHeads) Append(userID string, ls labels.Labels, chks index.ChunkMetas) *WALRecord { - idx := xxhash.Sum64String(userID) & uint64(t.shards-1) + idx := t.shardForTenant(userID) + + var mint, maxt int64 + for _, chk := range chks { + if chk.MinTime < mint || mint == 0 { + mint = chk.MinTime + } + + if chk.MaxTime > maxt { + maxt = chk.MaxTime + } + } + updateMintMaxt(mint, maxt, &t.mint, &t.maxt) // First, check if this tenant has been created var ( @@ -530,3 +562,65 @@ func (t *tenantHeads) Append(userID string, ls labels.Labels, chks index.ChunkMe return rec } + +func (t *tenantHeads) shardForTenant(userID string) uint64 { + return xxhash.Sum64String(userID) & uint64(t.shards-1) +} + +func (t *tenantHeads) Bounds() (model.Time, model.Time) { + return model.Time(t.mint.Load()), model.Time(t.maxt.Load()) +} + +func (t *tenantHeads) tenantIndex(userID string, from, through model.Time) (idx Index, unlock func(), ok bool) { + i := t.shardForTenant(userID) + t.locks[i].RLock() + tenant, ok := t.tenants[i][userID] + if !ok { + t.locks[i].RUnlock() + return + } + + return NewTSDBIndex(tenant.indexRange(int64(from), int64(through))), t.locks[i].RUnlock, true + +} + +func (t *tenantHeads) GetChunkRefs(ctx context.Context, userID string, from, through model.Time, res []ChunkRef, shard *index.ShardAnnotation, matchers ...*labels.Matcher) ([]ChunkRef, error) { + idx, unlock, ok := t.tenantIndex(userID, from, through) + if !ok { + return nil, nil + } + defer unlock() + return idx.GetChunkRefs(ctx, userID, from, through, nil, shard, matchers...) + +} + +// Series follows the same semantics regarding the passed slice and shard as GetChunkRefs. +func (t *tenantHeads) Series(ctx context.Context, userID string, from, through model.Time, res []Series, shard *index.ShardAnnotation, matchers ...*labels.Matcher) ([]Series, error) { + idx, unlock, ok := t.tenantIndex(userID, from, through) + if !ok { + return nil, nil + } + defer unlock() + return idx.Series(ctx, userID, from, through, nil, shard, matchers...) + +} + +func (t *tenantHeads) LabelNames(ctx context.Context, userID string, from, through model.Time, matchers ...*labels.Matcher) ([]string, error) { + idx, unlock, ok := t.tenantIndex(userID, from, through) + if !ok { + return nil, nil + } + defer unlock() + return idx.LabelNames(ctx, userID, from, through, matchers...) + +} + +func (t *tenantHeads) LabelValues(ctx context.Context, userID string, from, through model.Time, name string, matchers ...*labels.Matcher) ([]string, error) { + idx, unlock, ok := t.tenantIndex(userID, from, through) + if !ok { + return nil, nil + } + defer unlock() + return idx.LabelValues(ctx, userID, from, through, name, matchers...) + +} From 5d9e66ccf4124a44147fd3542f554bf5364a30f5 Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Fri, 15 Apr 2022 14:33:20 -0400 Subject: [PATCH 23/85] head mgr tests --- pkg/storage/stores/tsdb/head_manager.go | 6 + pkg/storage/stores/tsdb/head_manager_test.go | 118 +++++++++++++++++++ pkg/storage/stores/tsdb/index.go | 18 +++ pkg/storage/stores/tsdb/manager.go | 2 - 4 files changed, 142 insertions(+), 2 deletions(-) create mode 100644 pkg/storage/stores/tsdb/head_manager_test.go diff --git a/pkg/storage/stores/tsdb/head_manager.go b/pkg/storage/stores/tsdb/head_manager.go index 25269933763d9..d0cacaa74c73d 100644 --- a/pkg/storage/stores/tsdb/head_manager.go +++ b/pkg/storage/stores/tsdb/head_manager.go @@ -103,6 +103,12 @@ func NewHeadManager(log log.Logger, dir string, reg prometheus.Registerer, name } +func (m *HeadManager) Stop() error { + m.mtx.Lock() + defer m.mtx.Unlock() + return m.active.Stop() +} + func (m *HeadManager) Append(userID string, ls labels.Labels, chks index.ChunkMetas) error { m.mtx.RLock() now := time.Now() diff --git a/pkg/storage/stores/tsdb/head_manager_test.go b/pkg/storage/stores/tsdb/head_manager_test.go new file mode 100644 index 0000000000000..ba4312d82968b --- /dev/null +++ b/pkg/storage/stores/tsdb/head_manager_test.go @@ -0,0 +1,118 @@ +package tsdb + +import ( + "context" + "testing" + "time" + + "github.com/go-kit/log" + "github.com/grafana/loki/pkg/storage/stores/tsdb/index" + "github.com/prometheus/common/model" + "github.com/prometheus/prometheus/model/labels" + "github.com/stretchr/testify/require" +) + +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, NewHeadMetrics(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, NewHeadMetrics(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 + +// test mgr recover from multiple wals across multiple periods diff --git a/pkg/storage/stores/tsdb/index.go b/pkg/storage/stores/tsdb/index.go index 6b60734b2d0b8..c95380e94ec45 100644 --- a/pkg/storage/stores/tsdb/index.go +++ b/pkg/storage/stores/tsdb/index.go @@ -48,3 +48,21 @@ type Index interface { LabelNames(ctx context.Context, userID string, from, through model.Time, matchers ...*labels.Matcher) ([]string, error) LabelValues(ctx context.Context, userID string, from, through model.Time, name string, matchers ...*labels.Matcher) ([]string, error) } + +type NoopIndex struct{} + +func (NoopIndex) Bounds() (from, through model.Time) { return } +func (NoopIndex) GetChunkRefs(ctx context.Context, userID string, from, through model.Time, res []ChunkRef, shard *index.ShardAnnotation, matchers ...*labels.Matcher) ([]ChunkRef, error) { + return nil, nil +} + +// Series follows the same semantics regarding the passed slice and shard as GetChunkRefs. +func (NoopIndex) Series(ctx context.Context, userID string, from, through model.Time, res []Series, shard *index.ShardAnnotation, matchers ...*labels.Matcher) ([]Series, error) { + return nil, nil +} +func (NoopIndex) LabelNames(ctx context.Context, userID string, from, through model.Time, matchers ...*labels.Matcher) ([]string, error) { + return nil, nil +} +func (NoopIndex) LabelValues(ctx context.Context, userID string, from, through model.Time, name string, matchers ...*labels.Matcher) ([]string, error) { + return nil, nil +} diff --git a/pkg/storage/stores/tsdb/manager.go b/pkg/storage/stores/tsdb/manager.go index dad2e98caa2f7..64c76bfc116d1 100644 --- a/pkg/storage/stores/tsdb/manager.go +++ b/pkg/storage/stores/tsdb/manager.go @@ -8,5 +8,3 @@ type TSDBManager interface { // Builds a new TSDB file from a set of WALs BuildFromWALs(time.Time, []WALIdentifier) error } - -type tsdbManger struct{} From 3ccd7ee7fde54bd9e47a682a7f5fe1681fde0519 Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Fri, 15 Apr 2022 15:59:02 -0400 Subject: [PATCH 24/85] bugfixes & head manager tests --- pkg/storage/stores/tsdb/head_manager.go | 39 ++-- pkg/storage/stores/tsdb/head_manager_test.go | 192 +++++++++++++++++++ 2 files changed, 213 insertions(+), 18 deletions(-) diff --git a/pkg/storage/stores/tsdb/head_manager.go b/pkg/storage/stores/tsdb/head_manager.go index d0cacaa74c73d..a924bd516f4b4 100644 --- a/pkg/storage/stores/tsdb/head_manager.go +++ b/pkg/storage/stores/tsdb/head_manager.go @@ -242,24 +242,27 @@ func (m *HeadManager) Rotate(t time.Time) error { stopPrev("freshly rotated") // stop the newly rotated-out wal // build tsdb from rotated-out period - grp, _, err := m.walsForPeriod(m.PeriodFor(m.prev.initialized)) - if err != nil { - return errors.Wrap(err, "listing wals") - } + if m.prev != nil { + grp, _, err := m.walsForPeriod(m.PeriodFor(m.prev.initialized)) + if err != nil { + return errors.Wrap(err, "listing wals") + } - // TODO(owen-d): It's probably faster to build this from the *tenantHeads instead, - // but we already need to impl BuildFromWALs to ensure we can correctly build/ship - // TSDBs from orphaned WALs of previous periods during startup. - // we use the m.prev.initialized timestamp here for the filename to ensure it can't clobber - // an existing file from a previous cycle. I don't think this is possible, but - // perhaps in some unusual crashlooping it could be, so let's be safe and protect ourselves. - if err := m.tsdbManager.BuildFromWALs(m.prev.initialized, grp.wals); err != nil { - return errors.Wrapf(err, "building TSDB from prevHeads WALs for period %d", grp.period) - } + // TODO(owen-d): It's probably faster to build this from the *tenantHeads instead, + // but we already need to impl BuildFromWALs to ensure we can correctly build/ship + // TSDBs from orphaned WALs of previous periods during startup. + // we use the m.prev.initialized timestamp here for the filename to ensure it can't clobber + // an existing file from a previous cycle. I don't think this is possible, but + // perhaps in some unusual crashlooping it could be, so let's be safe and protect ourselves. + if err := m.tsdbManager.BuildFromWALs(m.prev.initialized, grp.wals); err != nil { + return errors.Wrapf(err, "building TSDB from prevHeads WALs for period %d", grp.period) + } + + // Now that a TSDB has been created from this group, it's safe to remove them + if err := m.removeWALGroup(grp); err != nil { + return errors.Wrapf(err, "removing prev TSDB WALs for period %d", grp.period) + } - // Now that a TSDB has been created from this group, it's safe to remove them - if err := m.removeWALGroup(grp); err != nil { - return errors.Wrapf(err, "removing prev TSDB WALs for period %d", grp.period) } // Now that the tsdbManager has the updated TSDBs, we can remove our references @@ -335,10 +338,10 @@ func (m *HeadManager) walGroups() (map[int]*walGroup, error) { pd := m.PeriodFor(id.ts) grp, ok := groupsMap[pd] if !ok { - grp := walGroup{ + grp = &walGroup{ period: pd, } - groupsMap[pd] = &grp + groupsMap[pd] = grp } grp.wals = append(grp.wals, id) } diff --git a/pkg/storage/stores/tsdb/head_manager_test.go b/pkg/storage/stores/tsdb/head_manager_test.go index ba4312d82968b..0c186c0b14908 100644 --- a/pkg/storage/stores/tsdb/head_manager_test.go +++ b/pkg/storage/stores/tsdb/head_manager_test.go @@ -2,13 +2,17 @@ package tsdb import ( "context" + "math" "testing" "time" "github.com/go-kit/log" + "github.com/grafana/loki/pkg/storage/chunk/client/util" "github.com/grafana/loki/pkg/storage/stores/tsdb/index" "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" ) @@ -114,5 +118,193 @@ func Test_TenantHeads_MultiRead(t *testing.T) { } // 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, "tsdb-mgr-test", 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 mgr.RequiredDirs() { + 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(), mgr.walPath(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 := mgr.walsForPeriod(mgr.PeriodFor(now)) + require.Nil(t, err) + require.True(t, ok) + require.Equal(t, 1, len(grp.wals)) + require.Nil(t, mgr.recoverHead(grp)) + + for _, c := range cases { + idx, err := mgr.Index() + require.Nil(t, err) + refs, err := idx.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, "tsdb-mgr-test", noopTSDBManager{}) + w, err := newHeadWAL(log.NewNopLogger(), mgr.walPath(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 { + idx, err := mgr.Index() + require.Nil(t, err) + refs, err := idx.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) + } + + // 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) { + idx, err := mgr.Index() + require.Nil(t, err) + refs, err := idx.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) + } + +} From cd39afb42ebe9eb15daaf20185bfa264ed5887db Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Fri, 15 Apr 2022 16:13:13 -0400 Subject: [PATCH 25/85] tsdb dir selection now helper fns --- pkg/storage/stores/tsdb/head_manager.go | 26 ++++++++++---------- pkg/storage/stores/tsdb/head_manager_test.go | 2 +- pkg/storage/stores/tsdb/manager.go | 10 ++++++++ 3 files changed, 24 insertions(+), 14 deletions(-) diff --git a/pkg/storage/stores/tsdb/head_manager.go b/pkg/storage/stores/tsdb/head_manager.go index a924bd516f4b4..47db68efbc068 100644 --- a/pkg/storage/stores/tsdb/head_manager.go +++ b/pkg/storage/stores/tsdb/head_manager.go @@ -137,7 +137,7 @@ func (m *HeadManager) Start() error { return errors.Wrap(err, "removing tsdb scratch dir") } - for _, d := range m.RequiredDirs() { + for _, d := range managerRequiredDirs(m.dir) { if err := util.EnsureDirectory(d); err != nil { return errors.Wrapf(err, "ensuring required directory exists: %s", d) } @@ -195,18 +195,18 @@ func (m *HeadManager) Start() error { return nil } -func (m *HeadManager) RequiredDirs() []string { +func managerRequiredDirs(parent string) []string { return []string{ - m.scratchDir(), - m.walDir(), - m.builtDir(), - m.shippedDir(), + managerScratchDir(parent), + managerWalDir(parent), + managerBuiltDir(parent), + managerShippedDir(parent), } } -func (m *HeadManager) scratchDir() string { return filepath.Join(m.dir, "scratch") } -func (m *HeadManager) walDir() string { return filepath.Join(m.dir, "wal") } -func (m *HeadManager) builtDir() string { return filepath.Join(m.dir, "multitenant", "built") } -func (m *HeadManager) shippedDir() string { return filepath.Join(m.dir, "multitenant", "shipped") } +func managerScratchDir(parent string) string { return filepath.Join(parent, "scratch") } +func managerWalDir(parent string) string { return filepath.Join(parent, "wal") } +func managerBuiltDir(parent string) string { return filepath.Join(parent, "multitenant", "built") } +func managerShippedDir(parent string) string { return filepath.Join(parent, "multitenant", "shipped") } func (m *HeadManager) Rotate(t time.Time) error { // create new wal @@ -289,7 +289,7 @@ func (m *HeadManager) Index() (Index, error) { } func (m *HeadManager) shippedTSDBsBeforePeriod(period int) (res []string, err error) { - files, err := ioutil.ReadDir(m.shippedDir()) + files, err := ioutil.ReadDir(managerShippedDir(m.dir)) if err != nil { return nil, err } @@ -326,7 +326,7 @@ func (m *HeadManager) walsByPeriod() ([]walGroup, error) { } func (m *HeadManager) walGroups() (map[int]*walGroup, error) { - files, err := ioutil.ReadDir(m.walDir()) + files, err := ioutil.ReadDir(managerWalDir(m.dir)) if err != nil { return nil, err } @@ -381,7 +381,7 @@ func (m *HeadManager) removeWALGroup(grp walGroup) error { func (m *HeadManager) walPath(t time.Time) string { return filepath.Join( - m.walDir(), + managerWalDir(m.dir), fmt.Sprintf("%d", t.Unix()), ) } diff --git a/pkg/storage/stores/tsdb/head_manager_test.go b/pkg/storage/stores/tsdb/head_manager_test.go index 0c186c0b14908..f394d02fd49ef 100644 --- a/pkg/storage/stores/tsdb/head_manager_test.go +++ b/pkg/storage/stores/tsdb/head_manager_test.go @@ -153,7 +153,7 @@ func Test_HeadManager_RecoverHead(t *testing.T) { mgr := NewHeadManager(log.NewNopLogger(), dir, nil, "tsdb-mgr-test", 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 mgr.RequiredDirs() { + for _, d := range managerRequiredDirs(dir) { require.Nil(t, util.EnsureDirectory(d)) } diff --git a/pkg/storage/stores/tsdb/manager.go b/pkg/storage/stores/tsdb/manager.go index 64c76bfc116d1..596d8b00e29fe 100644 --- a/pkg/storage/stores/tsdb/manager.go +++ b/pkg/storage/stores/tsdb/manager.go @@ -8,3 +8,13 @@ type TSDBManager interface { // 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{} From b556d90c6a6957f1eb5368d514050b8904a40399 Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Fri, 15 Apr 2022 16:21:54 -0400 Subject: [PATCH 26/85] period utility --- pkg/storage/stores/tsdb/head_manager.go | 28 +++++++++++--------- pkg/storage/stores/tsdb/head_manager_test.go | 2 +- pkg/storage/stores/tsdb/manager.go | 4 ++- 3 files changed, 19 insertions(+), 15 deletions(-) diff --git a/pkg/storage/stores/tsdb/head_manager.go b/pkg/storage/stores/tsdb/head_manager.go index 47db68efbc068..ac20203d41e9c 100644 --- a/pkg/storage/stores/tsdb/head_manager.go +++ b/pkg/storage/stores/tsdb/head_manager.go @@ -28,7 +28,9 @@ import ( "github.com/grafana/loki/pkg/storage/stores/tsdb/index" ) -const defaultRotationPeriod = 15 * time.Minute +type period time.Duration + +const defaultRotationPeriod = period(15 * time.Minute) // Do not specify without bit shifting. This allows us to // do shard index calcuations via bitwise & rather than modulos. @@ -78,7 +80,7 @@ type HeadManager struct { mtx sync.RWMutex // how often WALs should be rotated and TSDBs cut - period time.Duration + period period tsdbManager TSDBManager active, prev *headWAL @@ -112,7 +114,7 @@ func (m *HeadManager) Stop() error { func (m *HeadManager) Append(userID string, ls labels.Labels, chks index.ChunkMetas) error { m.mtx.RLock() now := time.Now() - if m.PeriodFor(now) > m.PeriodFor(m.activeHeads.start) { + if m.period.PeriodFor(now) > m.period.PeriodFor(m.activeHeads.start) { m.mtx.RUnlock() if err := m.Rotate(now); err != nil { return errors.Wrap(err, "rotating TSDB Head") @@ -124,12 +126,12 @@ func (m *HeadManager) Append(userID string, ls labels.Labels, chks index.ChunkMe return m.active.Log(rec) } -func (m *HeadManager) PeriodFor(t time.Time) int { - return int(t.UnixNano() / int64(m.period)) +func (p period) PeriodFor(t time.Time) int { + return int(t.UnixNano() / int64(p)) } -func (m *HeadManager) TimeForPeriod(period int) time.Time { - return time.Unix(0, int64(m.period)*int64(period)) +func (p period) TimeForPeriod(n int) time.Time { + return time.Unix(0, int64(p)*int64(n)) } func (m *HeadManager) Start() error { @@ -144,7 +146,7 @@ func (m *HeadManager) Start() error { } now := time.Now() - curPeriod := m.PeriodFor(now) + curPeriod := m.period.PeriodFor(now) toRemove, err := m.shippedTSDBsBeforePeriod(curPeriod) if err != nil { @@ -167,7 +169,7 @@ func (m *HeadManager) Start() error { for _, group := range walsByPeriod { if group.period < (curPeriod) { if err := m.tsdbManager.BuildFromWALs( - m.TimeForPeriod(group.period), + m.period.TimeForPeriod(group.period), group.wals, ); err != nil { return errors.Wrap(err, "building tsdb") @@ -224,7 +226,7 @@ func (m *HeadManager) Rotate(t time.Time) error { if err := m.prev.Stop(); err != nil { level.Error(m.log).Log( "msg", "failed stopping wal", - "period", m.PeriodFor(m.prev.initialized), + "period", m.period.PeriodFor(m.prev.initialized), "err", err, "wal", s, ) @@ -243,7 +245,7 @@ func (m *HeadManager) Rotate(t time.Time) error { // build tsdb from rotated-out period if m.prev != nil { - grp, _, err := m.walsForPeriod(m.PeriodFor(m.prev.initialized)) + grp, _, err := m.walsForPeriod(m.period.PeriodFor(m.prev.initialized)) if err != nil { return errors.Wrap(err, "listing wals") } @@ -295,7 +297,7 @@ func (m *HeadManager) shippedTSDBsBeforePeriod(period int) (res []string, err er } for _, f := range files { if id, ok := parseTSDBPath(f.Name()); ok { - if found := m.PeriodFor(id.ts); found < period { + if found := m.period.PeriodFor(id.ts); found < period { res = append(res, f.Name()) } } @@ -335,7 +337,7 @@ func (m *HeadManager) walGroups() (map[int]*walGroup, error) { for _, f := range files { if id, ok := parseWALPath(f.Name()); ok { - pd := m.PeriodFor(id.ts) + pd := m.period.PeriodFor(id.ts) grp, ok := groupsMap[pd] if !ok { grp = &walGroup{ diff --git a/pkg/storage/stores/tsdb/head_manager_test.go b/pkg/storage/stores/tsdb/head_manager_test.go index f394d02fd49ef..6c795834637b9 100644 --- a/pkg/storage/stores/tsdb/head_manager_test.go +++ b/pkg/storage/stores/tsdb/head_manager_test.go @@ -180,7 +180,7 @@ func Test_HeadManager_RecoverHead(t *testing.T) { require.Nil(t, w.Stop()) - grp, ok, err := mgr.walsForPeriod(mgr.PeriodFor(now)) + grp, ok, err := mgr.walsForPeriod(mgr.period.PeriodFor(now)) require.Nil(t, err) require.True(t, ok) require.Equal(t, 1, len(grp.wals)) diff --git a/pkg/storage/stores/tsdb/manager.go b/pkg/storage/stores/tsdb/manager.go index 596d8b00e29fe..5bc3a843a2b83 100644 --- a/pkg/storage/stores/tsdb/manager.go +++ b/pkg/storage/stores/tsdb/manager.go @@ -17,4 +17,6 @@ tsdbManager is responsible for: * Keeping them available for querying * Removing old TSDBs which are no longer needed */ -type tsdbManager struct{} +type tsdbManager struct { + period time.Duration // period to retain old tsdbs +} From ce6a27c5b8a973e1a41de3c9af133bb64b68ac54 Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Fri, 15 Apr 2022 16:45:44 -0400 Subject: [PATCH 27/85] pulls out more code to helpers, fixes some var races --- pkg/storage/stores/tsdb/head_manager.go | 23 ++++++++++---------- pkg/storage/stores/tsdb/head_manager_test.go | 4 ++-- 2 files changed, 13 insertions(+), 14 deletions(-) diff --git a/pkg/storage/stores/tsdb/head_manager.go b/pkg/storage/stores/tsdb/head_manager.go index ac20203d41e9c..379c0d7a8da8f 100644 --- a/pkg/storage/stores/tsdb/head_manager.go +++ b/pkg/storage/stores/tsdb/head_manager.go @@ -187,7 +187,7 @@ func (m *HeadManager) Start() error { } } - nextWALPath := m.walPath(now) + nextWALPath := walPath(m.dir, now) nextWAL, err := newHeadWAL(m.log, nextWALPath, now) if err != nil { return errors.Wrapf(err, "creating tsdb wal: %s", nextWALPath) @@ -212,7 +212,7 @@ func managerShippedDir(parent string) string { return filepath.Join(parent, "mul func (m *HeadManager) Rotate(t time.Time) error { // create new wal - nextWALPath := m.walPath(t) + nextWALPath := walPath(m.dir, t) nextWAL, err := newHeadWAL(m.log, nextWALPath, t) if err != nil { return errors.Wrapf(err, "creating tsdb wal: %s during rotation", nextWALPath) @@ -374,16 +374,16 @@ func (m *HeadManager) walsForPeriod(period int) (walGroup, bool, error) { func (m *HeadManager) removeWALGroup(grp walGroup) error { for _, wal := range grp.wals { - if err := os.RemoveAll(m.walPath(wal.ts)); err != nil { - return errors.Wrapf(err, "removing tsdb wal: %s", m.walPath(wal.ts)) + if err := os.RemoveAll(walPath(m.dir, wal.ts)); err != nil { + return errors.Wrapf(err, "removing tsdb wal: %s", walPath(m.dir, wal.ts)) } } return nil } -func (m *HeadManager) walPath(t time.Time) string { +func walPath(parent string, t time.Time) string { return filepath.Join( - managerWalDir(m.dir), + managerWalDir(parent), fmt.Sprintf("%d", t.Unix()), ) } @@ -394,8 +394,8 @@ func (m *HeadManager) recoverHead(grp walGroup) error { for _, id := range grp.wals { // use anonymous function for ease of cleanup - if err := func() error { - reader, closer, err := ingester.NewWalReader(m.walPath(id.ts), -1) + if err := func(id WALIdentifier) error { + reader, closer, err := ingester.NewWalReader(walPath(m.dir, id.ts), -1) if err != nil { return err } @@ -435,11 +435,10 @@ func (m *HeadManager) recoverHead(grp walGroup) error { } return reader.Err() - }(); err != nil { - return errors.Wrapf( + }(id); err != nil { + return errors.Wrap( err, - "error recovering from TSDB WAL: %s", - m.walPath(id.ts), + "error recovering from TSDB WAL", ) } } diff --git a/pkg/storage/stores/tsdb/head_manager_test.go b/pkg/storage/stores/tsdb/head_manager_test.go index 6c795834637b9..3d2a021190755 100644 --- a/pkg/storage/stores/tsdb/head_manager_test.go +++ b/pkg/storage/stores/tsdb/head_manager_test.go @@ -161,7 +161,7 @@ func Test_HeadManager_RecoverHead(t *testing.T) { require.Nil(t, mgr.Rotate(now)) // now build a WAL independently to test recovery - w, err := newHeadWAL(log.NewNopLogger(), mgr.walPath(now), now) + w, err := newHeadWAL(log.NewNopLogger(), walPath(mgr.dir, now), now) require.Nil(t, err) for i, c := range cases { @@ -236,7 +236,7 @@ func Test_HeadManager_Lifecycle(t *testing.T) { } mgr := NewHeadManager(log.NewNopLogger(), dir, nil, "tsdb-mgr-test", noopTSDBManager{}) - w, err := newHeadWAL(log.NewNopLogger(), mgr.walPath(curPeriod), curPeriod) + w, err := newHeadWAL(log.NewNopLogger(), walPath(mgr.dir, curPeriod), curPeriod) require.Nil(t, err) // Write old WALs From 2fd813e637778a83948605a0711ddfdbb7c391f9 Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Fri, 15 Apr 2022 16:56:50 -0400 Subject: [PATCH 28/85] head recovery is more generic --- pkg/storage/stores/tsdb/head.go | 10 +++++----- pkg/storage/stores/tsdb/head_manager.go | 19 ++++++++++++------- pkg/storage/stores/tsdb/head_manager_test.go | 2 +- 3 files changed, 18 insertions(+), 13 deletions(-) diff --git a/pkg/storage/stores/tsdb/head.go b/pkg/storage/stores/tsdb/head.go index 8e8fffa0673aa..bfcd5e0d448cf 100644 --- a/pkg/storage/stores/tsdb/head.go +++ b/pkg/storage/stores/tsdb/head.go @@ -55,12 +55,12 @@ guaranteeing we maintain querying consistency for the entire data lifecycle. */ // TODO(owen-d) -type HeadMetrics struct { +type Metrics struct { seriesNotFound prometheus.Counter } -func NewHeadMetrics(r prometheus.Registerer) *HeadMetrics { - return &HeadMetrics{ +func NewHeadMetrics(r prometheus.Registerer) *Metrics { + return &Metrics{ seriesNotFound: promauto.With(r).NewCounter(prometheus.CounterOpts{ Name: "loki_tsdb_head_series_not_found_total", Help: "Total number of requests for series that were not found.", @@ -77,7 +77,7 @@ type Head struct { // in the MemPostings, but is eventually discarded when we create a real TSDB index. lastSeriesID atomic.Uint64 - metrics *HeadMetrics + metrics *Metrics logger log.Logger series *stripeSeries @@ -88,7 +88,7 @@ type Head struct { closed bool } -func NewHead(tenant string, metrics *HeadMetrics, logger log.Logger) *Head { +func NewHead(tenant string, metrics *Metrics, logger log.Logger) *Head { return &Head{ tenant: tenant, metrics: metrics, diff --git a/pkg/storage/stores/tsdb/head_manager.go b/pkg/storage/stores/tsdb/head_manager.go index 379c0d7a8da8f..b6883789dfcc3 100644 --- a/pkg/storage/stores/tsdb/head_manager.go +++ b/pkg/storage/stores/tsdb/head_manager.go @@ -32,6 +32,11 @@ type period time.Duration const defaultRotationPeriod = period(15 * time.Minute) +// 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__" + // Do not specify without bit shifting. This allows us to // do shard index calcuations via bitwise & rather than modulos. const defaultHeadManagerStripeSize = 1 << 7 @@ -73,7 +78,7 @@ type HeadManager struct { name string log log.Logger dir string - metrics *HeadMetrics + metrics *Metrics // RLocked for all writes/reads, // Locked before rotating heads/wal @@ -181,7 +186,7 @@ func (m *HeadManager) Start() error { } if group.period == curPeriod { - if err := m.recoverHead(group); err != nil { + if err := recoverHead(m.dir, m.activeHeads, group, false); err != nil { return errors.Wrap(err, "recovering tsdb head from wal") } } @@ -390,12 +395,12 @@ func walPath(parent string, t time.Time) string { // recoverHead recovers from all WALs belonging to some period // and inserts it into the active *tenantHeads -func (m *HeadManager) recoverHead(grp walGroup) error { +func recoverHead(dir string, heads *tenantHeads, grp walGroup, addTenantLabels bool) error { for _, id := range grp.wals { // use anonymous function for ease of cleanup if err := func(id WALIdentifier) error { - reader, closer, err := ingester.NewWalReader(walPath(m.dir, id.ts), -1) + reader, closer, err := ingester.NewWalReader(walPath(dir, id.ts), -1) if err != nil { return err } @@ -430,7 +435,7 @@ func (m *HeadManager) recoverHead(grp walGroup) error { if !ok { return errors.New("found tsdb chunk metas without series in WAL replay") } - _ = m.activeHeads.Append(rec.UserID, ls, rec.Chks.Chks) + _ = heads.Append(rec.UserID, ls, rec.Chks.Chks) } } return reader.Err() @@ -502,10 +507,10 @@ type tenantHeads struct { locks []sync.RWMutex tenants []map[string]*Head log log.Logger - metrics *HeadMetrics + metrics *Metrics } -func newTenantHeads(start time.Time, shards int, metrics *HeadMetrics, log log.Logger) *tenantHeads { +func newTenantHeads(start time.Time, shards int, metrics *Metrics, log log.Logger) *tenantHeads { res := &tenantHeads{ start: start, shards: shards, diff --git a/pkg/storage/stores/tsdb/head_manager_test.go b/pkg/storage/stores/tsdb/head_manager_test.go index 3d2a021190755..9bcc4492671eb 100644 --- a/pkg/storage/stores/tsdb/head_manager_test.go +++ b/pkg/storage/stores/tsdb/head_manager_test.go @@ -184,7 +184,7 @@ func Test_HeadManager_RecoverHead(t *testing.T) { require.Nil(t, err) require.True(t, ok) require.Equal(t, 1, len(grp.wals)) - require.Nil(t, mgr.recoverHead(grp)) + require.Nil(t, recoverHead(mgr.dir, mgr.activeHeads, grp, false)) for _, c := range cases { idx, err := mgr.Index() From 47500792f88aebd32d033cb8239b295192086907 Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Fri, 15 Apr 2022 18:47:45 -0400 Subject: [PATCH 29/85] tsdb manager builds from wals --- pkg/storage/stores/tsdb/compact.go | 14 ++- pkg/storage/stores/tsdb/head.go | 12 ++- pkg/storage/stores/tsdb/head_manager.go | 75 ++++++++++++--- pkg/storage/stores/tsdb/head_manager_test.go | 2 +- pkg/storage/stores/tsdb/head_read.go | 4 +- pkg/storage/stores/tsdb/index/builder.go | 29 +++--- pkg/storage/stores/tsdb/manager.go | 92 ++++++++++++++++++- pkg/storage/stores/tsdb/querier_test.go | 11 ++- .../stores/tsdb/single_file_index_test.go | 3 +- pkg/storage/stores/tsdb/util_test.go | 11 ++- 10 files changed, 216 insertions(+), 37 deletions(-) diff --git a/pkg/storage/stores/tsdb/compact.go b/pkg/storage/stores/tsdb/compact.go index 397be46102d42..d7e8abe51dd55 100644 --- a/pkg/storage/stores/tsdb/compact.go +++ b/pkg/storage/stores/tsdb/compact.go @@ -64,5 +64,17 @@ func (c *Compactor) Compact(ctx context.Context, indices ...*TSDBIndex) (res ind } } - return b.Build(ctx, c.parentDir, c.tenant) + return b.Build( + ctx, + c.parentDir, + func(from, through model.Time, checksum uint32) (index.Identifier, string) { + id := index.Identifier{ + Tenant: c.tenant, + From: from, + Through: through, + Checksum: checksum, + } + return id, id.FilePath(c.parentDir) + }, + ) } diff --git a/pkg/storage/stores/tsdb/head.go b/pkg/storage/stores/tsdb/head.go index bfcd5e0d448cf..5d272c1ce91d0 100644 --- a/pkg/storage/stores/tsdb/head.go +++ b/pkg/storage/stores/tsdb/head.go @@ -56,7 +56,9 @@ guaranteeing we maintain querying consistency for the entire data lifecycle. // TODO(owen-d) type Metrics struct { - seriesNotFound prometheus.Counter + seriesNotFound prometheus.Counter + tsdbCreationsTotal prometheus.Counter + tsdbCreationFailures prometheus.Counter } func NewHeadMetrics(r prometheus.Registerer) *Metrics { @@ -65,6 +67,14 @@ func NewHeadMetrics(r prometheus.Registerer) *Metrics { Name: "loki_tsdb_head_series_not_found_total", Help: "Total number of requests for series that were not found.", }), + tsdbCreationsTotal: promauto.With(r).NewCounter(prometheus.CounterOpts{ + Name: "loki_tsdb_creations_total", + Help: "Total number of tsdb creations attempted", + }), + tsdbCreationFailures: promauto.With(r).NewCounter(prometheus.CounterOpts{ + Name: "loki_tsdb_creations_failed_total", + Help: "Total number of tsdb creations failed", + }), } } diff --git a/pkg/storage/stores/tsdb/head_manager.go b/pkg/storage/stores/tsdb/head_manager.go index b6883789dfcc3..0f6c646cdb087 100644 --- a/pkg/storage/stores/tsdb/head_manager.go +++ b/pkg/storage/stores/tsdb/head_manager.go @@ -186,7 +186,7 @@ func (m *HeadManager) Start() error { } if group.period == curPeriod { - if err := recoverHead(m.dir, m.activeHeads, group, false); err != nil { + if err := recoverHead(m.dir, m.activeHeads, group.wals, false); err != nil { return errors.Wrap(err, "recovering tsdb head from wal") } } @@ -310,18 +310,18 @@ func (m *HeadManager) shippedTSDBsBeforePeriod(period int) (res []string, err er return } -type walGroup struct { +type WalGroup struct { period int wals []WALIdentifier } -func (m *HeadManager) walsByPeriod() ([]walGroup, error) { +func (m *HeadManager) walsByPeriod() ([]WalGroup, error) { groupsMap, err := m.walGroups() if err != nil { return nil, err } - res := make([]walGroup, 0, len(groupsMap)) + res := make([]WalGroup, 0, len(groupsMap)) for _, grp := range groupsMap { res = append(res, *grp) } @@ -332,20 +332,20 @@ func (m *HeadManager) walsByPeriod() ([]walGroup, error) { return res, nil } -func (m *HeadManager) walGroups() (map[int]*walGroup, error) { +func (m *HeadManager) walGroups() (map[int]*WalGroup, error) { files, err := ioutil.ReadDir(managerWalDir(m.dir)) if err != nil { return nil, err } - groupsMap := map[int]*walGroup{} + groupsMap := map[int]*WalGroup{} for _, f := range files { if id, ok := parseWALPath(f.Name()); ok { pd := m.period.PeriodFor(id.ts) grp, ok := groupsMap[pd] if !ok { - grp = &walGroup{ + grp = &WalGroup{ period: pd, } groupsMap[pd] = grp @@ -363,21 +363,21 @@ func (m *HeadManager) walGroups() (map[int]*walGroup, error) { return groupsMap, nil } -func (m *HeadManager) walsForPeriod(period int) (walGroup, bool, error) { +func (m *HeadManager) walsForPeriod(period int) (WalGroup, bool, error) { groupsMap, err := m.walGroups() if err != nil { - return walGroup{}, false, err + return WalGroup{}, false, err } grp, ok := groupsMap[period] if !ok { - return walGroup{}, false, nil + return WalGroup{}, false, nil } return *grp, true, nil } -func (m *HeadManager) removeWALGroup(grp walGroup) error { +func (m *HeadManager) removeWALGroup(grp WalGroup) error { for _, wal := range grp.wals { if err := os.RemoveAll(walPath(m.dir, wal.ts)); err != nil { return errors.Wrapf(err, "removing tsdb wal: %s", walPath(m.dir, wal.ts)) @@ -395,8 +395,8 @@ func walPath(parent string, t time.Time) string { // recoverHead recovers from all WALs belonging to some period // and inserts it into the active *tenantHeads -func recoverHead(dir string, heads *tenantHeads, grp walGroup, addTenantLabels bool) error { - for _, id := range grp.wals { +func recoverHead(dir string, heads *tenantHeads, wals []WALIdentifier, addTenantLabels bool) error { + for _, id := range wals { // use anonymous function for ease of cleanup if err := func(id WALIdentifier) error { @@ -435,6 +435,12 @@ func recoverHead(dir string, heads *tenantHeads, grp walGroup, addTenantLabels b if !ok { return errors.New("found tsdb chunk metas without series in WAL replay") } + if addTenantLabels { + ls = append(ls, labels.Label{ + Name: TenantLabel, + Value: rec.UserID, + }) + } _ = heads.Append(rec.UserID, ls, rec.Chks.Chks) } } @@ -470,6 +476,10 @@ type MultitenantTSDBIdentifier struct { ts time.Time } +func (id MultitenantTSDBIdentifier) Name() string { + return fmt.Sprintf("%d-%s.tsdb", id.ts.Unix(), id.nodeName) +} + func parseTSDBPath(p string) (id MultitenantTSDBIdentifier, ok bool) { trimmed := strings.TrimSuffix(p, ".tsdb") @@ -639,3 +649,42 @@ func (t *tenantHeads) LabelValues(ctx context.Context, userID string, from, thro return idx.LabelValues(ctx, userID, from, through, name, matchers...) } + +// helper only used in building TSDBs +func (t *tenantHeads) forAll(fn func(user string, ls labels.Labels, chks index.ChunkMetas)) error { + for i, shard := range t.tenants { + t.locks[i].RLock() + defer t.locks[i].RUnlock() + + var ( + ls labels.Labels + chks []index.ChunkMeta + ) + + for user, tenant := range shard { + idx := tenant.Index() + ps, err := postingsForMatcher(idx, nil, labels.MustNewMatcher(labels.MatchEqual, "", "")) + if err != nil { + return err + } + + for ps.Next() { + _, err := idx.Series(ps.At(), &ls, &chks) + + if err != nil { + return errors.Wrapf(err, "iterating postings for tenant: %s", user) + } + + // We'll be reusing the slices, so copy before calling fn + dst := make(index.ChunkMetas, 0, len(chks)) + _ = copy(dst, chks) + fn(user, ls.Copy(), dst) + + ls = ls[:0] + chks = chks[:0] + } + } + } + + return nil +} diff --git a/pkg/storage/stores/tsdb/head_manager_test.go b/pkg/storage/stores/tsdb/head_manager_test.go index 9bcc4492671eb..68768fe2aeb16 100644 --- a/pkg/storage/stores/tsdb/head_manager_test.go +++ b/pkg/storage/stores/tsdb/head_manager_test.go @@ -184,7 +184,7 @@ func Test_HeadManager_RecoverHead(t *testing.T) { require.Nil(t, err) require.True(t, ok) require.Equal(t, 1, len(grp.wals)) - require.Nil(t, recoverHead(mgr.dir, mgr.activeHeads, grp, false)) + require.Nil(t, recoverHead(mgr.dir, mgr.activeHeads, grp.wals, false)) for _, c := range cases { idx, err := mgr.Index() diff --git a/pkg/storage/stores/tsdb/head_read.go b/pkg/storage/stores/tsdb/head_read.go index 0f6e06410733d..5ff524115c100 100644 --- a/pkg/storage/stores/tsdb/head_read.go +++ b/pkg/storage/stores/tsdb/head_read.go @@ -24,8 +24,8 @@ import ( ) // Index returns an IndexReader against the block. -func (h *Head) Index() (IndexReader, error) { - return h.indexRange(math.MinInt64, math.MaxInt64), nil +func (h *Head) Index() IndexReader { + return h.indexRange(math.MinInt64, math.MaxInt64) } func (h *Head) indexRange(mint, maxt int64) *headIndexReader { diff --git a/pkg/storage/stores/tsdb/index/builder.go b/pkg/storage/stores/tsdb/index/builder.go index 99cd1ec9ea005..915c853d6ae69 100644 --- a/pkg/storage/stores/tsdb/index/builder.go +++ b/pkg/storage/stores/tsdb/index/builder.go @@ -71,11 +71,18 @@ func (b *Builder) AddSeries(ls labels.Labels, chks []ChunkMeta) { s.chunks = append(s.chunks, chks...) } -func (b *Builder) Build(ctx context.Context, dir, tenant string) (id Identifier, err error) { +func (b *Builder) Build( + ctx context.Context, + scratchDir string, + // Determines how to create the resulting Identifier and file name. + // This is variable as we use Builder for multiple reasons, + // such as building multi-tenant tsdbs on the ingester + // and per tenant ones during compaction + createFn func(from, through model.Time, checksum uint32) (Identifier, string), +) (id Identifier, err error) { // Ensure the parent dir exists (i.e. index///) - parent := filepath.Join(dir, tenant) - if parent != "" { - if err := chunk_util.EnsureDirectory(parent); err != nil { + if scratchDir != "" { + if err := chunk_util.EnsureDirectory(scratchDir); err != nil { return id, err } } @@ -83,7 +90,7 @@ func (b *Builder) Build(ctx context.Context, dir, tenant string) (id Identifier, // First write tenant/index-bounds-random.staging rng := rand.Int63() name := fmt.Sprintf("%s-%x.staging", IndexFilename, rng) - tmpPath := filepath.Join(parent, name) + tmpPath := filepath.Join(scratchDir, name) writer, err := NewWriter(ctx, tmpPath) if err != nil { @@ -145,12 +152,7 @@ func (b *Builder) Build(ctx context.Context, dir, tenant string) (id Identifier, from, through := reader.Bounds() // load the newly compacted index to grab checksum, promptly close - id = Identifier{ - Tenant: tenant, - From: model.Time(from), - Through: model.Time(through), - Checksum: reader.Checksum(), - } + id, dst := createFn(model.Time(from), model.Time(through), reader.Checksum()) reader.Close() defer func() { @@ -159,8 +161,9 @@ func (b *Builder) Build(ctx context.Context, dir, tenant string) (id Identifier, } }() - dst := id.FilePath(dir) - + if err := chunk_util.EnsureDirectory(filepath.Dir(dst)); err != nil { + return id, err + } if err := os.Rename(tmpPath, dst); err != nil { return id, err } diff --git a/pkg/storage/stores/tsdb/manager.go b/pkg/storage/stores/tsdb/manager.go index 5bc3a843a2b83..000b4b7ea89b2 100644 --- a/pkg/storage/stores/tsdb/manager.go +++ b/pkg/storage/stores/tsdb/manager.go @@ -1,6 +1,18 @@ package tsdb -import "time" +import ( + "context" + "path/filepath" + "sync" + "time" + + "github.com/go-kit/log" + "github.com/go-kit/log/level" + "github.com/grafana/loki/pkg/storage/stores/tsdb/index" + "github.com/pkg/errors" + "github.com/prometheus/common/model" + "github.com/prometheus/prometheus/model/labels" +) type TSDBManager interface { Index @@ -9,6 +21,15 @@ type TSDBManager interface { BuildFromWALs(time.Time, []WALIdentifier) error } +// placeholder for index shipper +type shipper interface { + Ship(context.Context, MultitenantTSDBIdentifier) error +} + +type noopShipper struct{} + +func (noopShipper) Ship(_ context.Context, _ MultitenantTSDBIdentifier) error { return nil } + /* tsdbManager is responsible for: * Turning WALs into optimized multi-tenant TSDBs when requested @@ -18,5 +39,72 @@ tsdbManager is responsible for: * Removing old TSDBs which are no longer needed */ type tsdbManager struct { - period time.Duration // period to retain old tsdbs + name string // node name + log log.Logger + period period // period to retain old tsdbs + dir string + metrics *Metrics + + sync.RWMutex + + shipper shipper +} + +func (m *tsdbManager) BuildFromWALs(t time.Time, ids []WALIdentifier) (err error) { + // get relevant wals + // iterate them, build tsdb in scratch dir + defer func() { + m.metrics.tsdbCreationsTotal.Inc() + if err != nil { + m.metrics.tsdbCreationFailures.Inc() + } + }() + + tmp := newTenantHeads(t, defaultHeadManagerStripeSize, m.metrics, m.log) + if err = recoverHead(m.dir, tmp, ids, true); err != nil { + return errors.Wrap(err, "building TSDB from WALs") + } + b := index.NewBuilder() + + if err := tmp.forAll(func(user string, ls labels.Labels, chks index.ChunkMetas) { + b.AddSeries( + append(ls, labels.Label{ + Name: TenantLabel, + Value: user, + }), + chks, + ) + }); err != nil { + level.Error(m.log).Log("err", err.Error(), "msg", "building TSDB from WALs") + return err + } + + desired := MultitenantTSDBIdentifier{ + nodeName: m.name, + ts: t, + } + + dstFile := filepath.Join(managerBuiltDir(m.dir), desired.Name()) + + // build/move tsdb to multitenant/built dir + _, err = b.Build( + context.TODO(), + managerScratchDir(m.dir), + func(from, through model.Time, checksum uint32) (index.Identifier, string) { + + // We don't use the resulting ID b/c this isn't compaction. + // Instead we'll discard this and use our own override. + return index.Identifier{}, dstFile + }, + ) + if err != nil { + return err + } + + // TODO(owen-d): lock mtx, load file into list, unlock, start ship process + panic("unimplemented") +} + +func (m *tsdbManager) loop() { + // continually remove shipped tsdbs over 1 period old } diff --git a/pkg/storage/stores/tsdb/querier_test.go b/pkg/storage/stores/tsdb/querier_test.go index 013aed0a6e5af..f3c8eaba0cfe7 100644 --- a/pkg/storage/stores/tsdb/querier_test.go +++ b/pkg/storage/stores/tsdb/querier_test.go @@ -4,6 +4,7 @@ import ( "context" "testing" + "github.com/prometheus/common/model" "github.com/prometheus/prometheus/model/labels" "github.com/stretchr/testify/require" @@ -88,7 +89,15 @@ func TestQueryIndex(t *testing.T) { b.AddSeries(s.labels, s.chunks) } - dst, err := b.Build(context.Background(), dir, "fake") + dst, err := b.Build(context.Background(), dir, func(from, through model.Time, checksum uint32) (index.Identifier, string) { + id := index.Identifier{ + Tenant: "fake", + From: from, + Through: through, + Checksum: checksum, + } + return id, id.FilePath(dir) + }) require.Nil(t, err) reader, err := index.NewFileReader(dst.FilePath(dir)) diff --git a/pkg/storage/stores/tsdb/single_file_index_test.go b/pkg/storage/stores/tsdb/single_file_index_test.go index 454babbf2d721..cce37b8734bf3 100644 --- a/pkg/storage/stores/tsdb/single_file_index_test.go +++ b/pkg/storage/stores/tsdb/single_file_index_test.go @@ -73,8 +73,7 @@ func TestSingleIdx(t *testing.T) { for _, x := range cases { _, _ = head.Append(x.Labels, x.Chunks) } - reader, err := head.Index() - require.Nil(t, err) + reader := head.Index() return NewTSDBIndex(reader) }, }, diff --git a/pkg/storage/stores/tsdb/util_test.go b/pkg/storage/stores/tsdb/util_test.go index 22b30189f8634..f9c34118de0f1 100644 --- a/pkg/storage/stores/tsdb/util_test.go +++ b/pkg/storage/stores/tsdb/util_test.go @@ -4,6 +4,7 @@ import ( "context" "testing" + "github.com/prometheus/common/model" "github.com/prometheus/prometheus/model/labels" "github.com/stretchr/testify/require" @@ -22,7 +23,15 @@ func BuildIndex(t *testing.T, dir, tenant string, cases []LoadableSeries) *TSDBI b.AddSeries(s.Labels, s.Chunks) } - dst, err := b.Build(context.Background(), dir, tenant) + dst, err := b.Build(context.Background(), dir, func(from, through model.Time, checksum uint32) (index.Identifier, string) { + id := index.Identifier{ + Tenant: tenant, + From: from, + Through: through, + Checksum: checksum, + } + return id, id.FilePath(dir) + }) require.Nil(t, err) location := dst.FilePath(dir) From c5eda36b71833721d92d9273a9af945eecaf13be Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Sat, 16 Apr 2022 13:35:00 -0400 Subject: [PATCH 30/85] pulls more helpers out of headmanager --- pkg/storage/stores/tsdb/head_manager.go | 20 +++++++++---------- pkg/storage/stores/tsdb/head_manager_test.go | 2 +- pkg/storage/stores/tsdb/manager.go | 5 +++++ .../stores/tsdb/single_file_index_test.go | 2 ++ 4 files changed, 18 insertions(+), 11 deletions(-) diff --git a/pkg/storage/stores/tsdb/head_manager.go b/pkg/storage/stores/tsdb/head_manager.go index 0f6c646cdb087..515285f0b601a 100644 --- a/pkg/storage/stores/tsdb/head_manager.go +++ b/pkg/storage/stores/tsdb/head_manager.go @@ -164,7 +164,7 @@ func (m *HeadManager) Start() error { } } - walsByPeriod, err := m.walsByPeriod() + walsByPeriod, err := walsByPeriod(m.dir, m.period) if err != nil { return err } @@ -250,7 +250,7 @@ func (m *HeadManager) Rotate(t time.Time) error { // build tsdb from rotated-out period if m.prev != nil { - grp, _, err := m.walsForPeriod(m.period.PeriodFor(m.prev.initialized)) + grp, _, err := walsForPeriod(m.dir, m.period, m.period.PeriodFor(m.prev.initialized)) if err != nil { return errors.Wrap(err, "listing wals") } @@ -315,9 +315,9 @@ type WalGroup struct { wals []WALIdentifier } -func (m *HeadManager) walsByPeriod() ([]WalGroup, error) { +func walsByPeriod(dir string, period period) ([]WalGroup, error) { - groupsMap, err := m.walGroups() + groupsMap, err := walGroups(dir, period) if err != nil { return nil, err } @@ -332,8 +332,8 @@ func (m *HeadManager) walsByPeriod() ([]WalGroup, error) { return res, nil } -func (m *HeadManager) walGroups() (map[int]*WalGroup, error) { - files, err := ioutil.ReadDir(managerWalDir(m.dir)) +func walGroups(dir string, period period) (map[int]*WalGroup, error) { + files, err := ioutil.ReadDir(managerWalDir(dir)) if err != nil { return nil, err } @@ -342,7 +342,7 @@ func (m *HeadManager) walGroups() (map[int]*WalGroup, error) { for _, f := range files { if id, ok := parseWALPath(f.Name()); ok { - pd := m.period.PeriodFor(id.ts) + pd := period.PeriodFor(id.ts) grp, ok := groupsMap[pd] if !ok { grp = &WalGroup{ @@ -363,13 +363,13 @@ func (m *HeadManager) walGroups() (map[int]*WalGroup, error) { return groupsMap, nil } -func (m *HeadManager) walsForPeriod(period int) (WalGroup, bool, error) { - groupsMap, err := m.walGroups() +func walsForPeriod(dir string, period period, offset int) (WalGroup, bool, error) { + groupsMap, err := walGroups(dir, period) if err != nil { return WalGroup{}, false, err } - grp, ok := groupsMap[period] + grp, ok := groupsMap[offset] if !ok { return WalGroup{}, false, nil } diff --git a/pkg/storage/stores/tsdb/head_manager_test.go b/pkg/storage/stores/tsdb/head_manager_test.go index 68768fe2aeb16..0c9a5f5848854 100644 --- a/pkg/storage/stores/tsdb/head_manager_test.go +++ b/pkg/storage/stores/tsdb/head_manager_test.go @@ -180,7 +180,7 @@ func Test_HeadManager_RecoverHead(t *testing.T) { require.Nil(t, w.Stop()) - grp, ok, err := mgr.walsForPeriod(mgr.period.PeriodFor(now)) + 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)) diff --git a/pkg/storage/stores/tsdb/manager.go b/pkg/storage/stores/tsdb/manager.go index 000b4b7ea89b2..3d6b84392da6c 100644 --- a/pkg/storage/stores/tsdb/manager.go +++ b/pkg/storage/stores/tsdb/manager.go @@ -105,6 +105,11 @@ func (m *tsdbManager) BuildFromWALs(t time.Time, ids []WALIdentifier) (err error panic("unimplemented") } +func (m *tsdbManager) Start() { + go m.loop() +} + func (m *tsdbManager) loop() { + // continually ship built indices to storage then move them to the shipped directory // continually remove shipped tsdbs over 1 period old } diff --git a/pkg/storage/stores/tsdb/single_file_index_test.go b/pkg/storage/stores/tsdb/single_file_index_test.go index cce37b8734bf3..688c9506854d2 100644 --- a/pkg/storage/stores/tsdb/single_file_index_test.go +++ b/pkg/storage/stores/tsdb/single_file_index_test.go @@ -2,6 +2,7 @@ package tsdb import ( "context" + "sort" "testing" "github.com/go-kit/log" @@ -184,6 +185,7 @@ func TestSingleIdx(t *testing.T) { t.Run("LabelValues", func(t *testing.T) { vs, err := idx.LabelValues(context.Background(), "fake", 9, 10, "foo") require.Nil(t, err) + sort.Strings(vs) require.Equal(t, []string{"bar", "bard"}, vs) }) From c6b074759e3f5677ae856580a8f4ad6f4ea9c24d Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Sat, 16 Apr 2022 14:15:01 -0400 Subject: [PATCH 31/85] lockedIdx, Close() on idx, tsdbManager update --- pkg/storage/stores/tsdb/head.go | 16 ++++- pkg/storage/stores/tsdb/head_manager.go | 6 +- pkg/storage/stores/tsdb/index.go | 45 ++++++++++++ pkg/storage/stores/tsdb/manager.go | 77 ++++++++++++++++++++- pkg/storage/stores/tsdb/multi_file_index.go | 12 ++++ 5 files changed, 149 insertions(+), 7 deletions(-) diff --git a/pkg/storage/stores/tsdb/head.go b/pkg/storage/stores/tsdb/head.go index 5d272c1ce91d0..431917e003bf1 100644 --- a/pkg/storage/stores/tsdb/head.go +++ b/pkg/storage/stores/tsdb/head.go @@ -56,9 +56,11 @@ guaranteeing we maintain querying consistency for the entire data lifecycle. // TODO(owen-d) type Metrics struct { - seriesNotFound prometheus.Counter - tsdbCreationsTotal prometheus.Counter - tsdbCreationFailures prometheus.Counter + seriesNotFound prometheus.Counter + tsdbCreationsTotal prometheus.Counter + tsdbCreationFailures prometheus.Counter + tsdbManagerUpdatesTotal prometheus.Counter + tsdbManagerUpdatesFailedTotal prometheus.Counter } func NewHeadMetrics(r prometheus.Registerer) *Metrics { @@ -75,6 +77,14 @@ func NewHeadMetrics(r prometheus.Registerer) *Metrics { Name: "loki_tsdb_creations_failed_total", Help: "Total number of tsdb creations failed", }), + tsdbManagerUpdatesTotal: promauto.With(r).NewCounter(prometheus.CounterOpts{ + Name: "loki_tsdb_manager_updates_total", + Help: "Total number of tsdb manager updates (loading/rotating tsdbs in mem)", + }), + tsdbManagerUpdatesFailedTotal: promauto.With(r).NewCounter(prometheus.CounterOpts{ + Name: "loki_tsdb_manager_updates_failed_total", + Help: "Total number of tsdb manager update failures (loading/rotating tsdbs in mem)", + }), } } diff --git a/pkg/storage/stores/tsdb/head_manager.go b/pkg/storage/stores/tsdb/head_manager.go index 515285f0b601a..c9858cacc6dd5 100644 --- a/pkg/storage/stores/tsdb/head_manager.go +++ b/pkg/storage/stores/tsdb/head_manager.go @@ -301,7 +301,7 @@ func (m *HeadManager) shippedTSDBsBeforePeriod(period int) (res []string, err er return nil, err } for _, f := range files { - if id, ok := parseTSDBPath(f.Name()); ok { + if id, ok := parseMultitenantTSDBPath(f.Name()); ok { if found := m.period.PeriodFor(id.ts); found < period { res = append(res, f.Name()) } @@ -480,7 +480,7 @@ func (id MultitenantTSDBIdentifier) Name() string { return fmt.Sprintf("%d-%s.tsdb", id.ts.Unix(), id.nodeName) } -func parseTSDBPath(p string) (id MultitenantTSDBIdentifier, ok bool) { +func parseMultitenantTSDBPath(p string) (id MultitenantTSDBIdentifier, ok bool) { trimmed := strings.TrimSuffix(p, ".tsdb") // incorrect suffix @@ -592,6 +592,8 @@ func (t *tenantHeads) shardForTenant(userID string) uint64 { return xxhash.Sum64String(userID) & uint64(t.shards-1) } +func (t *tenantHeads) Close() error { return nil } + func (t *tenantHeads) Bounds() (model.Time, model.Time) { return model.Time(t.mint.Load()), model.Time(t.maxt.Load()) } diff --git a/pkg/storage/stores/tsdb/index.go b/pkg/storage/stores/tsdb/index.go index c95380e94ec45..ae471e65a326d 100644 --- a/pkg/storage/stores/tsdb/index.go +++ b/pkg/storage/stores/tsdb/index.go @@ -2,6 +2,7 @@ package tsdb import ( "context" + "sync" "github.com/prometheus/common/model" "github.com/prometheus/prometheus/model/labels" @@ -32,6 +33,7 @@ func (r ChunkRef) Less(x ChunkRef) bool { type Index interface { Bounded + Close() error // GetChunkRefs accepts an optional []ChunkRef argument. // If not nil, it will use that slice to build the result, // allowing us to avoid unnecessary allocations at the caller's discretion. @@ -51,6 +53,7 @@ type Index interface { type NoopIndex struct{} +func (NoopIndex) Close() error { return nil } func (NoopIndex) Bounds() (from, through model.Time) { return } func (NoopIndex) GetChunkRefs(ctx context.Context, userID string, from, through model.Time, res []ChunkRef, shard *index.ShardAnnotation, matchers ...*labels.Matcher) ([]ChunkRef, error) { return nil, nil @@ -66,3 +69,45 @@ func (NoopIndex) LabelNames(ctx context.Context, userID string, from, through mo func (NoopIndex) LabelValues(ctx context.Context, userID string, from, through model.Time, name string, matchers ...*labels.Matcher) ([]string, error) { return nil, nil } + +// LockedIndex wraps an index+RWMutex and only calls index methods under rlock +type LockedIndex struct { + mtx *sync.RWMutex + idx Index +} + +func NewLockedMutex(mtx *sync.RWMutex, idx Index) *LockedIndex { + return &LockedIndex{ + mtx: mtx, + idx: idx, + } +} + +func (i *LockedIndex) Close() error { return i.Close() } +func (i *LockedIndex) Bounds() (from, through model.Time) { + i.mtx.RLock() + defer i.mtx.RUnlock() + return i.idx.Bounds() +} +func (i *LockedIndex) GetChunkRefs(ctx context.Context, userID string, from, through model.Time, res []ChunkRef, shard *index.ShardAnnotation, matchers ...*labels.Matcher) ([]ChunkRef, error) { + i.mtx.RLock() + defer i.mtx.RUnlock() + return i.idx.GetChunkRefs(ctx, userID, from, through, res, shard, matchers...) +} + +func (i *LockedIndex) Series(ctx context.Context, userID string, from, through model.Time, res []Series, shard *index.ShardAnnotation, matchers ...*labels.Matcher) ([]Series, error) { + i.mtx.RLock() + defer i.mtx.RUnlock() + return i.idx.Series(ctx, userID, from, through, res, shard, matchers...) +} +func (i *LockedIndex) LabelNames(ctx context.Context, userID string, from, through model.Time, matchers ...*labels.Matcher) ([]string, error) { + i.mtx.RLock() + defer i.mtx.RUnlock() + return i.idx.LabelNames(ctx, userID, from, through, matchers...) +} +func (i *LockedIndex) LabelValues(ctx context.Context, userID string, from, through model.Time, name string, matchers ...*labels.Matcher) ([]string, error) { + i.mtx.RLock() + defer i.mtx.RUnlock() + return i.idx.LabelValues(ctx, userID, from, through, name, matchers...) + +} diff --git a/pkg/storage/stores/tsdb/manager.go b/pkg/storage/stores/tsdb/manager.go index 3d6b84392da6c..134b85dfbeba2 100644 --- a/pkg/storage/stores/tsdb/manager.go +++ b/pkg/storage/stores/tsdb/manager.go @@ -2,6 +2,7 @@ package tsdb import ( "context" + "io/ioutil" "path/filepath" "sync" "time" @@ -47,6 +48,8 @@ type tsdbManager struct { sync.RWMutex + Index + shipper shipper } @@ -101,8 +104,78 @@ func (m *tsdbManager) BuildFromWALs(t time.Time, ids []WALIdentifier) (err error return err } - // TODO(owen-d): lock mtx, load file into list, unlock, start ship process - panic("unimplemented") + return m.updateIndices() +} + +// updateIndices replaces the *tsdbManager's list of indices +// with those on disk. +func (m *tsdbManager) updateIndices() (err error) { + defer func() { + m.metrics.tsdbManagerUpdatesTotal.Inc() + if err != nil { + m.metrics.tsdbManagerUpdatesFailedTotal.Inc() + } + }() + var indices []Index + + // lock mtx, load file into list, unlock, start ship process + built, err := m.listMultiTenantTSDBs(managerBuiltDir(m.dir)) + if err != nil { + return err + } + for _, x := range built { + idx, err := LoadTSDB(filepath.Join(managerBuiltDir(m.dir), x.Name())) + if err != nil { + return err + } + indices = append(indices, idx) + } + + shipped, err := m.listMultiTenantTSDBs(managerShippedDir(m.dir)) + if err != nil { + return err + } + for _, x := range shipped { + idx, err := LoadTSDB(filepath.Join(managerShippedDir(m.dir), x.Name())) + if err != nil { + return err + } + indices = append(indices, idx) + } + + var newIdx Index + if len(indices) == 0 { + newIdx = NoopIndex{} + } else { + newIdx, err = NewMultiIndex(indices...) + if err != nil { + return err + } + } + + m.Lock() + defer m.Unlock() + if err := m.Index.Close(); err != nil { + return err + } + + m.Index = NewLockedMutex(&m.RWMutex, newIdx) + return nil +} + +func (m *tsdbManager) listMultiTenantTSDBs(dir string) (res []MultitenantTSDBIdentifier, err error) { + files, err := ioutil.ReadDir(dir) + if err != nil { + return nil, err + } + + for _, f := range files { + if id, ok := parseMultitenantTSDBPath(filepath.Base(f.Name())); ok { + res = append(res, id) + } + } + + return } func (m *tsdbManager) Start() { diff --git a/pkg/storage/stores/tsdb/multi_file_index.go b/pkg/storage/stores/tsdb/multi_file_index.go index 24c1c6996829c..98ead43eedc19 100644 --- a/pkg/storage/stores/tsdb/multi_file_index.go +++ b/pkg/storage/stores/tsdb/multi_file_index.go @@ -8,6 +8,7 @@ import ( "github.com/prometheus/prometheus/model/labels" "golang.org/x/sync/errgroup" + "github.com/grafana/dskit/multierror" "github.com/grafana/loki/pkg/storage/stores/tsdb/index" ) @@ -39,6 +40,17 @@ func (i *MultiIndex) Bounds() (model.Time, model.Time) { return lowest, highest } +func (i *MultiIndex) Close() error { + var errs multierror.MultiError + for _, idx := range i.indices { + if err := idx.Close(); err != nil { + errs = append(errs, err) + } + } + return errs.Err() + +} + func (i *MultiIndex) forIndices(ctx context.Context, from, through model.Time, fn func(context.Context, Index) (interface{}, error)) ([]interface{}, error) { queryBounds := newBounds(from, through) g, ctx := errgroup.WithContext(ctx) From 3ece1421ad503a1dab267f2e2dd7dd7f14b1d64a Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Mon, 18 Apr 2022 12:06:57 -0400 Subject: [PATCH 32/85] removes mmap from index reader implementation --- pkg/storage/stores/tsdb/index/index.go | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/pkg/storage/stores/tsdb/index/index.go b/pkg/storage/stores/tsdb/index/index.go index 0d34f96aaf95a..f2059a18584c2 100644 --- a/pkg/storage/stores/tsdb/index/index.go +++ b/pkg/storage/stores/tsdb/index/index.go @@ -33,7 +33,6 @@ import ( "github.com/prometheus/prometheus/model/labels" "github.com/prometheus/prometheus/storage" tsdb_enc "github.com/prometheus/prometheus/tsdb/encoding" - tsdb_errors "github.com/prometheus/prometheus/tsdb/errors" "github.com/prometheus/prometheus/tsdb/fileutil" "github.com/grafana/loki/pkg/util/encoding" @@ -1187,18 +1186,19 @@ func NewReader(b ByteSlice) (*Reader, error) { return newReader(b, ioutil.NopCloser(nil)) } +type nopCloser struct{} + +func (_ nopCloser) Close() error { return nil } + // NewFileReader returns a new index reader against the given index file. func NewFileReader(path string) (*Reader, error) { - f, err := fileutil.OpenMmapFile(path) + b, err := ioutil.ReadFile(path) if err != nil { return nil, err } - r, err := newReader(realByteSlice(f.Bytes()), f) + r, err := newReader(realByteSlice(b), nopCloser{}) if err != nil { - return nil, tsdb_errors.NewMulti( - err, - f.Close(), - ).Err() + return r, err } return r, nil From 8b4c2b497298f5047dbd56264baa1b2077247a76 Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Mon, 18 Apr 2022 13:57:23 -0400 Subject: [PATCH 33/85] tsdb file --- pkg/storage/stores/tsdb/compact_test.go | 2 +- pkg/storage/stores/tsdb/head_manager.go | 9 ++- pkg/storage/stores/tsdb/manager.go | 28 +++++---- pkg/storage/stores/tsdb/multi_file_index.go | 22 ++++--- pkg/storage/stores/tsdb/single_file_index.go | 63 +++++++++++++++++-- .../stores/tsdb/single_file_index_test.go | 6 +- pkg/storage/stores/tsdb/util_test.go | 4 +- 7 files changed, 99 insertions(+), 35 deletions(-) diff --git a/pkg/storage/stores/tsdb/compact_test.go b/pkg/storage/stores/tsdb/compact_test.go index 143b1fa4c8383..81d43029512ce 100644 --- a/pkg/storage/stores/tsdb/compact_test.go +++ b/pkg/storage/stores/tsdb/compact_test.go @@ -356,7 +356,7 @@ func TestCompactor(t *testing.T) { for _, cases := range tc.input { idx := BuildIndex(t, dir, "fake", cases) defer idx.Close() - indices = append(indices, idx) + indices = append(indices, idx.TSDBIndex) } out, err := c.Compact(context.Background(), indices...) diff --git a/pkg/storage/stores/tsdb/head_manager.go b/pkg/storage/stores/tsdb/head_manager.go index c9858cacc6dd5..bc9c501aaf8b9 100644 --- a/pkg/storage/stores/tsdb/head_manager.go +++ b/pkg/storage/stores/tsdb/head_manager.go @@ -249,6 +249,7 @@ func (m *HeadManager) Rotate(t time.Time) error { stopPrev("freshly rotated") // stop the newly rotated-out wal // build tsdb from rotated-out period + // TODO(owen-d): don't block Append() waiting for tsdb building. Use a work channel/etc if m.prev != nil { grp, _, err := walsForPeriod(m.dir, m.period, m.period.PeriodFor(m.prev.initialized)) if err != nil { @@ -284,14 +285,16 @@ func (m *HeadManager) Index() (Index, error) { m.mtx.RLock() defer m.mtx.RUnlock() - indices := []Index{m.tsdbManager} - if m.prev != nil { + var indices []Index + if m.prevHeads != nil { indices = append(indices, m.prevHeads) } - if m.active != nil { + if m.activeHeads != nil { indices = append(indices, m.activeHeads) } + indices = append(indices, m.tsdbManager) + return NewMultiIndex(indices...) } diff --git a/pkg/storage/stores/tsdb/manager.go b/pkg/storage/stores/tsdb/manager.go index 134b85dfbeba2..fe4ba5c25d786 100644 --- a/pkg/storage/stores/tsdb/manager.go +++ b/pkg/storage/stores/tsdb/manager.go @@ -2,6 +2,7 @@ package tsdb import ( "context" + "fmt" "io/ioutil" "path/filepath" "sync" @@ -9,6 +10,7 @@ import ( "github.com/go-kit/log" "github.com/go-kit/log/level" + "github.com/grafana/loki/pkg/storage/stores/indexshipper" "github.com/grafana/loki/pkg/storage/stores/tsdb/index" "github.com/pkg/errors" "github.com/prometheus/common/model" @@ -22,15 +24,6 @@ type TSDBManager interface { BuildFromWALs(time.Time, []WALIdentifier) error } -// placeholder for index shipper -type shipper interface { - Ship(context.Context, MultitenantTSDBIdentifier) error -} - -type noopShipper struct{} - -func (noopShipper) Ship(_ context.Context, _ MultitenantTSDBIdentifier) error { return nil } - /* tsdbManager is responsible for: * Turning WALs into optimized multi-tenant TSDBs when requested @@ -40,9 +33,9 @@ tsdbManager is responsible for: * Removing old TSDBs which are no longer needed */ type tsdbManager struct { + period period name string // node name log log.Logger - period period // period to retain old tsdbs dir string metrics *Metrics @@ -50,7 +43,7 @@ type tsdbManager struct { Index - shipper shipper + shipper indexshipper.IndexShipper } func (m *tsdbManager) BuildFromWALs(t time.Time, ids []WALIdentifier) (err error) { @@ -104,6 +97,15 @@ func (m *tsdbManager) BuildFromWALs(t time.Time, ids []WALIdentifier) (err error return err } + loaded, err := NewShippableTSDBFile(dstFile) + if err != nil { + return err + } + + if err := m.shipper.AddIndex(fmt.Sprintf("%d", m.period.PeriodFor(t)), "", loaded); err != nil { + return err + } + return m.updateIndices() } @@ -124,7 +126,7 @@ func (m *tsdbManager) updateIndices() (err error) { return err } for _, x := range built { - idx, err := LoadTSDB(filepath.Join(managerBuiltDir(m.dir), x.Name())) + idx, err := NewShippableTSDBFile(filepath.Join(managerBuiltDir(m.dir), x.Name())) if err != nil { return err } @@ -136,7 +138,7 @@ func (m *tsdbManager) updateIndices() (err error) { return err } for _, x := range shipped { - idx, err := LoadTSDB(filepath.Join(managerShippedDir(m.dir), x.Name())) + idx, err := NewShippableTSDBFile(filepath.Join(managerShippedDir(m.dir), x.Name())) if err != nil { return err } diff --git a/pkg/storage/stores/tsdb/multi_file_index.go b/pkg/storage/stores/tsdb/multi_file_index.go index 98ead43eedc19..1cf2daa32ccfc 100644 --- a/pkg/storage/stores/tsdb/multi_file_index.go +++ b/pkg/storage/stores/tsdb/multi_file_index.go @@ -64,14 +64,20 @@ func (i *MultiIndex) forIndices(ctx context.Context, from, through model.Time, f if Overlap(queryBounds, idx) { // run all queries in linked goroutines (cancel after first err), // bounded by parallelism controls if applicable. - g.Go(func() error { - got, err := fn(ctx, idx) - if err != nil { - return err - } - ch <- got - return nil - }) + + // must wrap g.Go in anonymous function to capture + // idx variable during iteration + func(idx Index) { + g.Go(func() error { + got, err := fn(ctx, idx) + if err != nil { + return err + } + ch <- got + return nil + }) + }(idx) + } } diff --git a/pkg/storage/stores/tsdb/single_file_index.go b/pkg/storage/stores/tsdb/single_file_index.go index ddbe8080ee353..a3c2ab14c114b 100644 --- a/pkg/storage/stores/tsdb/single_file_index.go +++ b/pkg/storage/stores/tsdb/single_file_index.go @@ -2,31 +2,84 @@ package tsdb import ( "context" + "io" + "os" "github.com/prometheus/common/model" "github.com/prometheus/prometheus/model/labels" + "github.com/grafana/dskit/multierror" "github.com/grafana/loki/pkg/storage/stores/tsdb/index" ) -func LoadTSDBIdentifier(dir string, id index.Identifier) (*TSDBIndex, error) { - return LoadTSDB(id.FilePath(dir)) +func LoadTSDBIdentifier(dir string, id index.Identifier) (*TSDBFile, error) { + return NewShippableTSDBFile(id.FilePath(dir)) } -func LoadTSDB(name string) (*TSDBIndex, error) { - reader, err := index.NewFileReader(name) +// nolint +// TSDBFile is backed by an actual file and implements the indexshipper/index.Index interface +type TSDBFile struct { + path string + + // reuse TSDBIndex for reading + *TSDBIndex + + // open the read only fd + // to sastisfy Reader() and Close() methods + f *os.File +} + +func NewShippableTSDBFile(location string) (*TSDBFile, error) { + idx, err := NewTSDBIndexFromFile(location) if err != nil { return nil, err } - return NewTSDBIndex(reader), nil + f, err := os.Open(location) + if err != nil { + return nil, err + } + + return &TSDBFile{ + path: location, + TSDBIndex: idx, + f: f, + }, err +} + +// TODO(owen-d): not yet sure how name vs path differ +func (f *TSDBFile) Name() string { + return f.path +} +func (f *TSDBFile) Path() string { return f.path } + +func (f *TSDBFile) Close() error { + var errs multierror.MultiError + errs.Add(f.TSDBIndex.Close()) + errs.Add(f.f.Close()) + return errs.Err() +} + +func (f *TSDBFile) Reader() (io.ReadSeeker, error) { + return f.f, nil } // nolint +// TSDBIndex is backed by an IndexReader +// and translates the IndexReader to an Index implementation +// It loads the file into memory and doesn't keep a file descriptor open type TSDBIndex struct { reader IndexReader } +func NewTSDBIndexFromFile(location string) (*TSDBIndex, error) { + reader, err := index.NewFileReader(location) + if err != nil { + return nil, err + } + return NewTSDBIndex(reader), nil +} + func NewTSDBIndex(reader IndexReader) *TSDBIndex { return &TSDBIndex{ reader: reader, diff --git a/pkg/storage/stores/tsdb/single_file_index_test.go b/pkg/storage/stores/tsdb/single_file_index_test.go index 688c9506854d2..2f85d0806d0e0 100644 --- a/pkg/storage/stores/tsdb/single_file_index_test.go +++ b/pkg/storage/stores/tsdb/single_file_index_test.go @@ -59,17 +59,17 @@ func TestSingleIdx(t *testing.T) { for _, variant := range []struct { desc string - fn func() *TSDBIndex + fn func() Index }{ { desc: "file", - fn: func() *TSDBIndex { + fn: func() Index { return BuildIndex(t, t.TempDir(), "fake", cases) }, }, { desc: "head", - fn: func() *TSDBIndex { + fn: func() Index { head := NewHead("fake", NewHeadMetrics(nil), log.NewNopLogger()) for _, x := range cases { _, _ = head.Append(x.Labels, x.Chunks) diff --git a/pkg/storage/stores/tsdb/util_test.go b/pkg/storage/stores/tsdb/util_test.go index f9c34118de0f1..9e3aade6b7ed8 100644 --- a/pkg/storage/stores/tsdb/util_test.go +++ b/pkg/storage/stores/tsdb/util_test.go @@ -16,7 +16,7 @@ type LoadableSeries struct { Chunks index.ChunkMetas } -func BuildIndex(t *testing.T, dir, tenant string, cases []LoadableSeries) *TSDBIndex { +func BuildIndex(t *testing.T, dir, tenant string, cases []LoadableSeries) *TSDBFile { b := index.NewBuilder() for _, s := range cases { @@ -35,7 +35,7 @@ func BuildIndex(t *testing.T, dir, tenant string, cases []LoadableSeries) *TSDBI require.Nil(t, err) location := dst.FilePath(dir) - idx, err := LoadTSDB(location) + idx, err := NewShippableTSDBFile(location) require.Nil(t, err) return idx } From ee6eda9bae434bbe773e0677c934d11be29be386 Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Mon, 18 Apr 2022 18:07:13 -0400 Subject: [PATCH 34/85] adds tsdb shipper config and refactors initStore --- pkg/loki/modules.go | 130 +++++++++++++----- pkg/storage/config/schema_config.go | 39 +++++- pkg/storage/factory.go | 2 + .../stores/shipper/shipper_index_client.go | 24 ++-- 4 files changed, 145 insertions(+), 50 deletions(-) diff --git a/pkg/loki/modules.go b/pkg/loki/modules.go index cb9a80644ea9c..c992fd04e6404 100644 --- a/pkg/loki/modules.go +++ b/pkg/loki/modules.go @@ -382,19 +382,27 @@ func (t *Loki) initTableManager() (services.Service, error) { } func (t *Loki) initStore() (_ services.Service, err error) { + // Always set these configs + for _, cfg := range []*shipper.Config{ + &t.Cfg.StorageConfig.BoltDBShipperConfig, + &t.Cfg.StorageConfig.TSDBShipperConfig, + } { + cfg.IndexGatewayClientConfig.Mode = t.Cfg.IndexGateway.Mode + cfg.IndexGatewayClientConfig.Ring = t.indexGatewayRing + } + // If RF > 1 and current or upcoming index type is boltdb-shipper then disable index dedupe and write dedupe cache. // This is to ensure that index entries are replicated to all the boltdb files in ingesters flushing replicated data. - if t.Cfg.Ingester.LifecyclerConfig.RingConfig.ReplicationFactor > 1 && config.UsingBoltdbShipper(t.Cfg.SchemaConfig.Configs) { + if t.Cfg.Ingester.LifecyclerConfig.RingConfig.ReplicationFactor > 1 && config.UsingObjectStorageIndex(t.Cfg.SchemaConfig.Configs) { t.Cfg.ChunkStoreConfig.DisableIndexDeduplication = true t.Cfg.ChunkStoreConfig.WriteDedupeCacheConfig = cache.Config{} } - if config.UsingBoltdbShipper(t.Cfg.SchemaConfig.Configs) { - t.Cfg.StorageConfig.BoltDBShipperConfig.IngesterName = t.Cfg.Ingester.LifecyclerConfig.ID + // Set configs pertaining to object storage based indices + if config.UsingObjectStorageIndex(t.Cfg.SchemaConfig.Configs) { + switch true { case t.Cfg.isModuleEnabled(Ingester), t.Cfg.isModuleEnabled(Write): - // We do not want ingester to unnecessarily keep downloading files - t.Cfg.StorageConfig.BoltDBShipperConfig.Mode = shipper.ModeWriteOnly // Use fifo cache for caching index in memory, this also significantly helps performance. t.Cfg.StorageConfig.IndexQueriesCacheConfig = cache.Config{ EnableFifoCache: true, @@ -412,22 +420,64 @@ func (t *Loki) initStore() (_ services.Service, err error) { // have query gaps on chunks flushed after an index entry is cached by keeping them retained in the ingester // and queried as part of live data until the cache TTL expires on the index entry. t.Cfg.Ingester.RetainPeriod = t.Cfg.StorageConfig.IndexCacheValidity + 1*time.Minute - t.Cfg.StorageConfig.BoltDBShipperConfig.IngesterDBRetainPeriod = boltdbShipperQuerierIndexUpdateDelay(t.Cfg) + 2*time.Minute + + for _, cfg := range []*shipper.Config{ + &t.Cfg.StorageConfig.BoltDBShipperConfig, + &t.Cfg.StorageConfig.TSDBShipperConfig, + } { + // We do not want ingester to unnecessarily keep downloading files + cfg.Mode = shipper.ModeWriteOnly + + cfg.IngesterName = t.Cfg.Ingester.LifecyclerConfig.ID + + cfg.IngesterDBRetainPeriod = shipperQuerierIndexUpdateDelay(t.Cfg.StorageConfig.IndexCacheValidity, cfg.ResyncInterval) + 2*time.Minute + } + case t.Cfg.isModuleEnabled(Querier), t.Cfg.isModuleEnabled(Ruler), t.Cfg.isModuleEnabled(Read), t.isModuleActive(IndexGateway): - // We do not want query to do any updates to index - t.Cfg.StorageConfig.BoltDBShipperConfig.Mode = shipper.ModeReadOnly + for _, cfg := range []*shipper.Config{ + &t.Cfg.StorageConfig.BoltDBShipperConfig, + &t.Cfg.StorageConfig.TSDBShipperConfig, + } { + // We do not want query to do any updates to index + cfg.Mode = shipper.ModeReadOnly + } default: - t.Cfg.StorageConfig.BoltDBShipperConfig.Mode = shipper.ModeReadWrite - t.Cfg.StorageConfig.BoltDBShipperConfig.IngesterDBRetainPeriod = boltdbShipperQuerierIndexUpdateDelay(t.Cfg) + 2*time.Minute + for _, cfg := range []*shipper.Config{ + &t.Cfg.StorageConfig.BoltDBShipperConfig, + &t.Cfg.StorageConfig.TSDBShipperConfig, + } { + cfg.Mode = shipper.ModeReadWrite + cfg.IngesterDBRetainPeriod = shipperQuerierIndexUpdateDelay(t.Cfg.StorageConfig.IndexCacheValidity, cfg.ResyncInterval) + 2*time.Minute + } } } - t.Cfg.StorageConfig.BoltDBShipperConfig.IndexGatewayClientConfig.Mode = t.Cfg.IndexGateway.Mode - t.Cfg.StorageConfig.BoltDBShipperConfig.IndexGatewayClientConfig.Ring = t.indexGatewayRing + if config.UsingObjectStorageIndex(t.Cfg.SchemaConfig.Configs) { + var asyncStore bool + + shipperConfigIdx := config.ActivePeriodConfig(t.Cfg.SchemaConfig.Configs) + iTy := t.Cfg.SchemaConfig.Configs[shipperConfigIdx].IndexType + if iTy != config.BoltDBShipperType && iTy != config.TSDBType { + shipperConfigIdx++ + } + + // TODO(owen-d): make helper more agnostic between boltdb|tsdb + var resyncInterval time.Duration + switch t.Cfg.SchemaConfig.Configs[shipperConfigIdx].IndexType { + case config.BoltDBShipperType: + resyncInterval = t.Cfg.StorageConfig.BoltDBShipperConfig.ResyncInterval + case config.TSDBType: + resyncInterval = t.Cfg.StorageConfig.TSDBShipperConfig.ResyncInterval + } + + minIngesterQueryStoreDuration := shipperMinIngesterQueryStoreDuration( + t.Cfg.Ingester.MaxChunkAge, + shipperQuerierIndexUpdateDelay( + t.Cfg.StorageConfig.IndexCacheValidity, + resyncInterval, + ), + ) - var asyncStore bool - if config.UsingBoltdbShipper(t.Cfg.SchemaConfig.Configs) { - boltdbShipperMinIngesterQueryStoreDuration := boltdbShipperMinIngesterQueryStoreDuration(t.Cfg) switch true { case t.Cfg.isModuleEnabled(Querier), t.Cfg.isModuleEnabled(Ruler), t.Cfg.isModuleEnabled(Read): // Do not use the AsyncStore if the querier is configured with QueryStoreOnly set to true @@ -439,30 +489,38 @@ func (t *Loki) initStore() (_ services.Service, err error) { asyncStore = true case t.Cfg.isModuleEnabled(IndexGateway): // we want to use the actual storage when running the index-gateway, so we remove the Addr from the config - t.Cfg.StorageConfig.BoltDBShipperConfig.IndexGatewayClientConfig.Disabled = true + for _, cfg := range []*shipper.Config{ + &t.Cfg.StorageConfig.BoltDBShipperConfig, + &t.Cfg.StorageConfig.TSDBShipperConfig, + } { + cfg.IndexGatewayClientConfig.Disabled = true + } case t.Cfg.isModuleEnabled(All): // We want ingester to also query the store when using boltdb-shipper but only when running with target All. // We do not want to use AsyncStore otherwise it would start spiraling around doing queries over and over again to the ingesters and store. // ToDo: See if we can avoid doing this when not running loki in clustered mode. t.Cfg.Ingester.QueryStore = true - boltdbShipperConfigIdx := config.ActivePeriodConfig(t.Cfg.SchemaConfig.Configs) - if t.Cfg.SchemaConfig.Configs[boltdbShipperConfigIdx].IndexType != config.BoltDBShipperType { - boltdbShipperConfigIdx++ - } - mlb, err := calculateMaxLookBack(t.Cfg.SchemaConfig.Configs[boltdbShipperConfigIdx], t.Cfg.Ingester.QueryStoreMaxLookBackPeriod, - boltdbShipperMinIngesterQueryStoreDuration) + + mlb, err := calculateMaxLookBack( + t.Cfg.SchemaConfig.Configs[shipperConfigIdx], + t.Cfg.Ingester.QueryStoreMaxLookBackPeriod, + minIngesterQueryStoreDuration, + ) if err != nil { return nil, err } t.Cfg.Ingester.QueryStoreMaxLookBackPeriod = mlb } - } - if asyncStore { - t.Cfg.StorageConfig.EnableAsyncStore = true - t.Cfg.StorageConfig.AsyncStoreConfig = storage.AsyncStoreCfg{ - IngesterQuerier: t.ingesterQuerier, - QueryIngestersWithin: calculateAsyncStoreQueryIngestersWithin(t.Cfg.Querier.QueryIngestersWithin, boltdbShipperMinIngesterQueryStoreDuration(t.Cfg)), + if asyncStore { + t.Cfg.StorageConfig.EnableAsyncStore = true + t.Cfg.StorageConfig.AsyncStoreConfig = storage.AsyncStoreCfg{ + IngesterQuerier: t.ingesterQuerier, + QueryIngestersWithin: calculateAsyncStoreQueryIngestersWithin( + t.Cfg.Querier.QueryIngestersWithin, + minIngesterQueryStoreDuration, + ), + } } } @@ -921,22 +979,22 @@ func calculateAsyncStoreQueryIngestersWithin(queryIngestersWithinConfig, minDura return queryIngestersWithinConfig } -// boltdbShipperQuerierIndexUpdateDelay returns duration it could take for queriers to serve the index since it was uploaded. +// shipperQuerierIndexUpdateDelay returns duration it could take for queriers to serve the index since it was uploaded. // It also considers index cache validity because a querier could have cached index just before it was going to resync which means // it would keep serving index until the cache entries expire. -func boltdbShipperQuerierIndexUpdateDelay(cfg Config) time.Duration { - return cfg.StorageConfig.IndexCacheValidity + cfg.StorageConfig.BoltDBShipperConfig.ResyncInterval +func shipperQuerierIndexUpdateDelay(cacheValidity, resyncInterval time.Duration) time.Duration { + return cacheValidity + resyncInterval } -// boltdbShipperIngesterIndexUploadDelay returns duration it could take for an index file containing id of a chunk to be uploaded to the shared store since it got flushed. -func boltdbShipperIngesterIndexUploadDelay() time.Duration { +// shipperIngesterIndexUploadDelay returns duration it could take for an index file containing id of a chunk to be uploaded to the shared store since it got flushed. +func shipperIngesterIndexUploadDelay() time.Duration { return uploads.ShardDBsByDuration + shipper.UploadInterval } -// boltdbShipperMinIngesterQueryStoreDuration returns minimum duration(with some buffer) ingesters should query their stores to +// shipperMinIngesterQueryStoreDuration returns minimum duration(with some buffer) ingesters should query their stores to // avoid missing any logs or chunk ids due to async nature of BoltDB Shipper. -func boltdbShipperMinIngesterQueryStoreDuration(cfg Config) time.Duration { - return cfg.Ingester.MaxChunkAge + boltdbShipperIngesterIndexUploadDelay() + boltdbShipperQuerierIndexUpdateDelay(cfg) + 2*time.Minute +func shipperMinIngesterQueryStoreDuration(maxChunkAge, querierUpdateDelay time.Duration) time.Duration { + return maxChunkAge + shipperIngesterIndexUploadDelay() + querierUpdateDelay + 2*time.Minute } // NewServerService constructs service from Server component. diff --git a/pkg/storage/config/schema_config.go b/pkg/storage/config/schema_config.go index e5081460c6fbc..0864fd2db632c 100644 --- a/pkg/storage/config/schema_config.go +++ b/pkg/storage/config/schema_config.go @@ -39,6 +39,7 @@ const ( StorageTypeSwift = "swift" // BoltDBShipperType holds the index type for using boltdb with shipper which keeps flushing them to a shared storage BoltDBShipperType = "boltdb-shipper" + TSDBType = "tsdb" ) var ( @@ -184,17 +185,47 @@ func ActivePeriodConfig(configs []PeriodConfig) int { return i } -// UsingBoltdbShipper checks whether current or the next index type is boltdb-shipper, returns true if yes. -func UsingBoltdbShipper(configs []PeriodConfig) bool { +func usingForPeriodConfigs(configs []PeriodConfig, fn func(PeriodConfig) bool) bool { activePCIndex := ActivePeriodConfig(configs) - if configs[activePCIndex].IndexType == BoltDBShipperType || - (len(configs)-1 > activePCIndex && configs[activePCIndex+1].IndexType == BoltDBShipperType) { + + if fn(configs[activePCIndex]) || + (len(configs)-1 > activePCIndex && fn(configs[activePCIndex+1])) { return true } return false } +func UsingObjectStorageIndex(configs []PeriodConfig) bool { + fn := func(cfg PeriodConfig) bool { + switch cfg.IndexType { + case BoltDBShipperType, TSDBType: + return true + default: + return false + } + } + + return usingForPeriodConfigs(configs, fn) +} + +// UsingBoltdbShipper checks whether current or the next index type is boltdb-shipper, returns true if yes. +func UsingBoltdbShipper(configs []PeriodConfig) bool { + fn := func(cfg PeriodConfig) bool { + return cfg.IndexType == BoltDBShipperType + } + + return usingForPeriodConfigs(configs, fn) +} + +func UsingTSDB(configs []PeriodConfig) bool { + fn := func(cfg PeriodConfig) bool { + return cfg.IndexType == BoltDBShipperType + } + + return usingForPeriodConfigs(configs, fn) +} + func defaultRowShards(schema string) uint32 { switch schema { case "v1", "v2", "v3", "v4", "v5", "v6", "v9": diff --git a/pkg/storage/factory.go b/pkg/storage/factory.go index d22fee324b265..c4c90a473a4cd 100644 --- a/pkg/storage/factory.go +++ b/pkg/storage/factory.go @@ -63,6 +63,7 @@ type Config struct { MaxChunkBatchSize int `yaml:"max_chunk_batch_size"` BoltDBShipperConfig shipper.Config `yaml:"boltdb_shipper"` + TSDBShipperConfig shipper.Config `yaml:"tsdb_shipper"` // Config for using AsyncStore when using async index stores like `boltdb-shipper`. // It is required for getting chunk ids of recently flushed chunks from the ingesters. @@ -89,6 +90,7 @@ func (cfg *Config) RegisterFlags(f *flag.FlagSet) { f.IntVar(&cfg.MaxParallelGetChunk, "store.max-parallel-get-chunk", 150, "Maximum number of parallel chunk reads.") cfg.BoltDBShipperConfig.RegisterFlags(f) f.IntVar(&cfg.MaxChunkBatchSize, "store.max-chunk-batch-size", 50, "The maximum number of chunks to fetch per batch.") + cfg.TSDBShipperConfig.RegisterFlagsWithPrefix("tsdb.", f) } // Validate config and returns error on failure diff --git a/pkg/storage/stores/shipper/shipper_index_client.go b/pkg/storage/stores/shipper/shipper_index_client.go index d3b326f095878..cfd68afc6387c 100644 --- a/pkg/storage/stores/shipper/shipper_index_client.go +++ b/pkg/storage/stores/shipper/shipper_index_client.go @@ -66,16 +66,20 @@ type Config struct { // RegisterFlags registers flags. func (cfg *Config) RegisterFlags(f *flag.FlagSet) { - cfg.IndexGatewayClientConfig.RegisterFlagsWithPrefix("boltdb.shipper.index-gateway-client", f) - - f.StringVar(&cfg.ActiveIndexDirectory, "boltdb.shipper.active-index-directory", "", "Directory where ingesters would write boltdb files which would then be uploaded by shipper to configured storage") - f.StringVar(&cfg.SharedStoreType, "boltdb.shipper.shared-store", "", "Shared store for keeping boltdb files. Supported types: gcs, s3, azure, filesystem") - f.StringVar(&cfg.SharedStoreKeyPrefix, "boltdb.shipper.shared-store.key-prefix", "index/", "Prefix to add to Object Keys in Shared store. Path separator(if any) should always be a '/'. Prefix should never start with a separator but should always end with it") - f.StringVar(&cfg.CacheLocation, "boltdb.shipper.cache-location", "", "Cache location for restoring boltDB files for queries") - f.DurationVar(&cfg.CacheTTL, "boltdb.shipper.cache-ttl", 24*time.Hour, "TTL for boltDB files restored in cache for queries") - f.DurationVar(&cfg.ResyncInterval, "boltdb.shipper.resync-interval", 5*time.Minute, "Resync downloaded files with the storage") - f.IntVar(&cfg.QueryReadyNumDays, "boltdb.shipper.query-ready-num-days", 0, "Number of days of common index to be kept downloaded for queries. For per tenant index query readiness, use limits overrides config.") - f.BoolVar(&cfg.BuildPerTenantIndex, "boltdb.shipper.build-per-tenant-index", false, "Build per tenant index files") + cfg.RegisterFlagsWithPrefix("boltdb.", f) +} + +func (cfg *Config) RegisterFlagsWithPrefix(prefix string, f *flag.FlagSet) { + cfg.IndexGatewayClientConfig.RegisterFlagsWithPrefix(prefix+"shipper.index-gateway-client", f) + + f.StringVar(&cfg.ActiveIndexDirectory, prefix+"shipper.active-index-directory", "", "Directory where ingesters would write boltdb files which would then be uploaded by shipper to configured storage") + f.StringVar(&cfg.SharedStoreType, prefix+"shipper.shared-store", "", "Shared store for keeping boltdb files. Supported types: gcs, s3, azure, filesystem") + f.StringVar(&cfg.SharedStoreKeyPrefix, prefix+"shipper.shared-store.key-prefix", "index/", "Prefix to add to Object Keys in Shared store. Path separator(if any) should always be a '/'. Prefix should never start with a separator but should always end with it") + f.StringVar(&cfg.CacheLocation, prefix+"shipper.cache-location", "", "Cache location for restoring boltDB files for queries") + f.DurationVar(&cfg.CacheTTL, prefix+"shipper.cache-ttl", 24*time.Hour, "TTL for boltDB files restored in cache for queries") + f.DurationVar(&cfg.ResyncInterval, prefix+"shipper.resync-interval", 5*time.Minute, "Resync downloaded files with the storage") + f.IntVar(&cfg.QueryReadyNumDays, prefix+"shipper.query-ready-num-days", 0, "Number of days of common index to be kept downloaded for queries. For per tenant index query readiness, use limits overrides config.") + f.BoolVar(&cfg.BuildPerTenantIndex, prefix+"shipper.build-per-tenant-index", false, "Build per tenant index files") } func (cfg *Config) Validate() error { From b844d85e0763d77b2f012f737122b2ec15b28d41 Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Mon, 18 Apr 2022 18:07:24 -0400 Subject: [PATCH 35/85] removes unused tsdbManager code --- pkg/storage/stores/tsdb/manager.go | 83 ------------------------------ 1 file changed, 83 deletions(-) diff --git a/pkg/storage/stores/tsdb/manager.go b/pkg/storage/stores/tsdb/manager.go index fe4ba5c25d786..0e9bac30f5c8d 100644 --- a/pkg/storage/stores/tsdb/manager.go +++ b/pkg/storage/stores/tsdb/manager.go @@ -3,7 +3,6 @@ package tsdb import ( "context" "fmt" - "io/ioutil" "path/filepath" "sync" "time" @@ -41,8 +40,6 @@ type tsdbManager struct { sync.RWMutex - Index - shipper indexshipper.IndexShipper } @@ -106,85 +103,5 @@ func (m *tsdbManager) BuildFromWALs(t time.Time, ids []WALIdentifier) (err error return err } - return m.updateIndices() -} - -// updateIndices replaces the *tsdbManager's list of indices -// with those on disk. -func (m *tsdbManager) updateIndices() (err error) { - defer func() { - m.metrics.tsdbManagerUpdatesTotal.Inc() - if err != nil { - m.metrics.tsdbManagerUpdatesFailedTotal.Inc() - } - }() - var indices []Index - - // lock mtx, load file into list, unlock, start ship process - built, err := m.listMultiTenantTSDBs(managerBuiltDir(m.dir)) - if err != nil { - return err - } - for _, x := range built { - idx, err := NewShippableTSDBFile(filepath.Join(managerBuiltDir(m.dir), x.Name())) - if err != nil { - return err - } - indices = append(indices, idx) - } - - shipped, err := m.listMultiTenantTSDBs(managerShippedDir(m.dir)) - if err != nil { - return err - } - for _, x := range shipped { - idx, err := NewShippableTSDBFile(filepath.Join(managerShippedDir(m.dir), x.Name())) - if err != nil { - return err - } - indices = append(indices, idx) - } - - var newIdx Index - if len(indices) == 0 { - newIdx = NoopIndex{} - } else { - newIdx, err = NewMultiIndex(indices...) - if err != nil { - return err - } - } - - m.Lock() - defer m.Unlock() - if err := m.Index.Close(); err != nil { - return err - } - - m.Index = NewLockedMutex(&m.RWMutex, newIdx) return nil } - -func (m *tsdbManager) listMultiTenantTSDBs(dir string) (res []MultitenantTSDBIdentifier, err error) { - files, err := ioutil.ReadDir(dir) - if err != nil { - return nil, err - } - - for _, f := range files { - if id, ok := parseMultitenantTSDBPath(filepath.Base(f.Name())); ok { - res = append(res, id) - } - } - - return -} - -func (m *tsdbManager) Start() { - go m.loop() -} - -func (m *tsdbManager) loop() { - // continually ship built indices to storage then move them to the shipped directory - // continually remove shipped tsdbs over 1 period old -} From eb31b3265851aa41faac7e98177c879953577387 Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Tue, 19 Apr 2022 13:26:02 -0400 Subject: [PATCH 36/85] implements stores.Index and stores.ChunkWriter for tsdb --- pkg/storage/stores/tsdb/chunkwriter.go | 98 +++++++++++++++++++++ pkg/storage/stores/tsdb/index_client.go | 109 ++++++++++++++++++++++++ pkg/storage/stores/tsdb/manager.go | 2 + 3 files changed, 209 insertions(+) create mode 100644 pkg/storage/stores/tsdb/chunkwriter.go create mode 100644 pkg/storage/stores/tsdb/index_client.go diff --git a/pkg/storage/stores/tsdb/chunkwriter.go b/pkg/storage/stores/tsdb/chunkwriter.go new file mode 100644 index 0000000000000..3e75c69fde25e --- /dev/null +++ b/pkg/storage/stores/tsdb/chunkwriter.go @@ -0,0 +1,98 @@ +package tsdb + +import ( + "context" + + "github.com/go-kit/log/level" + "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" + "github.com/pkg/errors" + "github.com/prometheus/common/model" + "github.com/prometheus/prometheus/model/labels" +) + +type indexWriter interface { + Append(userID string, ls labels.Labels, chks index.ChunkMetas) error +} + +// satisfied by HeadManager +var _ indexWriter = &HeadManager{} + +type ChunkWriter struct { + schemaCfg config.SchemaConfig + fetcher *fetcher.Fetcher + indexWriter indexWriter +} + +func NewChunkWriter( + fetcher *fetcher.Fetcher, + schemaCfg config.SchemaConfig, +) *ChunkWriter { + return &ChunkWriter{ + schemaCfg: schemaCfg, + fetcher: fetcher, + } +} + +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 +} diff --git a/pkg/storage/stores/tsdb/index_client.go b/pkg/storage/stores/tsdb/index_client.go new file mode 100644 index 0000000000000..76eea97dedd13 --- /dev/null +++ b/pkg/storage/stores/tsdb/index_client.go @@ -0,0 +1,109 @@ +package tsdb + +import ( + "context" + + "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" + "github.com/prometheus/common/model" + "github.com/prometheus/prometheus/model/labels" +) + +// implements stores.Index +type IndexClient struct { + schema config.SchemaConfig + idx Index +} + +// 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) { + // TODO(owen-d): handle this + panic("unimplemented") +} diff --git a/pkg/storage/stores/tsdb/manager.go b/pkg/storage/stores/tsdb/manager.go index 0e9bac30f5c8d..02862e530ee40 100644 --- a/pkg/storage/stores/tsdb/manager.go +++ b/pkg/storage/stores/tsdb/manager.go @@ -16,6 +16,8 @@ import ( "github.com/prometheus/prometheus/model/labels" ) +// TSDBManager wraps the index shipper and writes/manages +// TSDB files on disk type TSDBManager interface { Index From 32218b9a9e1e2119c0d4b6367113182a552530f4 Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Tue, 19 Apr 2022 13:26:46 -0400 Subject: [PATCH 37/85] chunk.Data now supports an Entries() method --- pkg/chunkenc/facade.go | 11 ++++++++++- pkg/storage/chunk/bigchunk.go | 4 ++++ pkg/storage/chunk/interface.go | 2 ++ 3 files changed, 16 insertions(+), 1 deletion(-) diff --git a/pkg/chunkenc/facade.go b/pkg/chunkenc/facade.go index 36c6331aafe07..fa214fda8aa30 100644 --- a/pkg/chunkenc/facade.go +++ b/pkg/chunkenc/facade.go @@ -72,7 +72,9 @@ func (f Facade) Utilization() float64 { return f.c.Utilization() } -// Size implements encoding.Chunk. +// Size implements encoding.Chunk, which unfortunately uses +// the Size method to refer to the byte size and not the entry count +// like chunkenc.Chunk does. func (f Facade) Size() int { if f.c == nil { return 0 @@ -81,6 +83,13 @@ func (f Facade) Size() int { return f.c.CompressedSize() } +func (f Facade) Entries() int { + if f.c == nil { + return 0 + } + return f.c.Size() +} + // LokiChunk returns the chunkenc.Chunk. func (f Facade) LokiChunk() Chunk { return f.c diff --git a/pkg/storage/chunk/bigchunk.go b/pkg/storage/chunk/bigchunk.go index 70b188faaad24..1f3c0957a39af 100644 --- a/pkg/storage/chunk/bigchunk.go +++ b/pkg/storage/chunk/bigchunk.go @@ -32,6 +32,10 @@ func newBigchunk() *bigchunk { return &bigchunk{} } +// TODO(owen-d): remove bigchunk from our code, we don't use it. +// Hack an Entries() impl +func (b *bigchunk) Entries() int { return 0 } + func (b *bigchunk) Add(sample model.SamplePair) (Data, error) { if b.remainingSamples == 0 { if bigchunkSizeCapBytes > 0 && b.Size() > bigchunkSizeCapBytes { diff --git a/pkg/storage/chunk/interface.go b/pkg/storage/chunk/interface.go index be5f83211076b..ccc5ce6764043 100644 --- a/pkg/storage/chunk/interface.go +++ b/pkg/storage/chunk/interface.go @@ -50,6 +50,8 @@ type Data interface { Rebound(start, end model.Time) (Data, error) // Size returns the approximate length of the chunk in bytes. Size() int + // Entries returns the number of entries in a chunk + Entries() int Utilization() float64 } From 7d20b5eb8ca41236ee4dcc85e6941d50a4e95d6b Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Tue, 19 Apr 2022 14:15:24 -0400 Subject: [PATCH 38/85] moves walreader to new util/wal pkg to avoid circular dep + tsdb storage alignment --- pkg/ingester/ingester.go | 3 +- pkg/ingester/recovery.go | 34 ----------- pkg/storage/stores/tsdb/chunkwriter.go | 8 ++- pkg/storage/stores/tsdb/head.go | 2 +- pkg/storage/stores/tsdb/head_manager.go | 50 ++++++++--------- pkg/storage/stores/tsdb/head_manager_test.go | 20 +++---- pkg/storage/stores/tsdb/index_client.go | 9 +++ pkg/storage/stores/tsdb/lazy_index.go | 56 +++++++++++++++++++ pkg/storage/stores/tsdb/manager.go | 30 ++++++++-- .../stores/tsdb/single_file_index_test.go | 2 +- pkg/util/wal/reader.go | 42 ++++++++++++++ 11 files changed, 172 insertions(+), 84 deletions(-) create mode 100644 pkg/storage/stores/tsdb/lazy_index.go create mode 100644 pkg/util/wal/reader.go diff --git a/pkg/ingester/ingester.go b/pkg/ingester/ingester.go index 2b49f5c460651..4859f684e3b56 100644 --- a/pkg/ingester/ingester.go +++ b/pkg/ingester/ingester.go @@ -38,6 +38,7 @@ import ( "github.com/grafana/loki/pkg/util" errUtil "github.com/grafana/loki/pkg/util" util_log "github.com/grafana/loki/pkg/util/log" + "github.com/grafana/loki/pkg/util/wal" "github.com/grafana/loki/pkg/validation" ) @@ -420,7 +421,7 @@ func (i *Ingester) starting(ctx context.Context) error { ) level.Info(util_log.Logger).Log("msg", "recovering from WAL") - segmentReader, segmentCloser, err := NewWalReader(i.cfg.WAL.Dir, -1) + segmentReader, segmentCloser, err := wal.NewWalReader(i.cfg.WAL.Dir, -1) if err != nil { return err } diff --git a/pkg/ingester/recovery.go b/pkg/ingester/recovery.go index 861b52a85d9bd..86fc6305b13d4 100644 --- a/pkg/ingester/recovery.go +++ b/pkg/ingester/recovery.go @@ -30,40 +30,6 @@ func (NoopWALReader) Err() error { return nil } func (NoopWALReader) Record() []byte { return nil } func (NoopWALReader) Close() error { return nil } -// If startSegment is <0, it means all the segments. -func NewWalReader(dir string, startSegment int) (*wal.Reader, io.Closer, error) { - var ( - segmentReader io.ReadCloser - err error - ) - if startSegment < 0 { - segmentReader, err = wal.NewSegmentsReader(dir) - if err != nil { - return nil, nil, err - } - } else { - first, last, err := wal.Segments(dir) - if err != nil { - return nil, nil, err - } - if startSegment > last { - return nil, nil, errors.New("start segment is beyond the last WAL segment") - } - if first > startSegment { - startSegment = first - } - segmentReader, err = wal.NewSegmentsRangeReader(wal.SegmentRange{ - Dir: dir, - First: startSegment, - Last: -1, // Till the end. - }) - if err != nil { - return nil, nil, err - } - } - return wal.NewReader(segmentReader), segmentReader, nil -} - func newCheckpointReader(dir string) (WALReader, io.Closer, error) { lastCheckpointDir, idx, err := lastCheckpoint(dir) if err != nil { diff --git a/pkg/storage/stores/tsdb/chunkwriter.go b/pkg/storage/stores/tsdb/chunkwriter.go index 3e75c69fde25e..bcd80e00032e3 100644 --- a/pkg/storage/stores/tsdb/chunkwriter.go +++ b/pkg/storage/stores/tsdb/chunkwriter.go @@ -30,11 +30,13 @@ type ChunkWriter struct { func NewChunkWriter( fetcher *fetcher.Fetcher, - schemaCfg config.SchemaConfig, + pd config.PeriodConfig, ) *ChunkWriter { return &ChunkWriter{ - schemaCfg: schemaCfg, - fetcher: fetcher, + schemaCfg: config.SchemaConfig{ + Configs: []config.PeriodConfig{pd}, + }, + fetcher: fetcher, } } diff --git a/pkg/storage/stores/tsdb/head.go b/pkg/storage/stores/tsdb/head.go index 431917e003bf1..3e074ac372b7d 100644 --- a/pkg/storage/stores/tsdb/head.go +++ b/pkg/storage/stores/tsdb/head.go @@ -63,7 +63,7 @@ type Metrics struct { tsdbManagerUpdatesFailedTotal prometheus.Counter } -func NewHeadMetrics(r prometheus.Registerer) *Metrics { +func NewMetrics(r prometheus.Registerer) *Metrics { return &Metrics{ seriesNotFound: promauto.With(r).NewCounter(prometheus.CounterOpts{ Name: "loki_tsdb_head_series_not_found_total", diff --git a/pkg/storage/stores/tsdb/head_manager.go b/pkg/storage/stores/tsdb/head_manager.go index bc9c501aaf8b9..8fe1234920614 100644 --- a/pkg/storage/stores/tsdb/head_manager.go +++ b/pkg/storage/stores/tsdb/head_manager.go @@ -16,16 +16,15 @@ import ( "github.com/go-kit/log" "github.com/go-kit/log/level" "github.com/pkg/errors" - "github.com/prometheus/client_golang/prometheus" "github.com/prometheus/common/model" "github.com/prometheus/prometheus/model/labels" "github.com/prometheus/prometheus/tsdb/chunks" "github.com/prometheus/prometheus/tsdb/record" "go.uber.org/atomic" - "github.com/grafana/loki/pkg/ingester" "github.com/grafana/loki/pkg/storage/chunk/client/util" "github.com/grafana/loki/pkg/storage/stores/tsdb/index" + "github.com/grafana/loki/pkg/util/wal" ) type period time.Duration @@ -75,7 +74,6 @@ tsdb/ */ type HeadManager struct { - name string log log.Logger dir string metrics *Metrics @@ -92,13 +90,13 @@ type HeadManager struct { shards int activeHeads, prevHeads *tenantHeads + + Index } -func NewHeadManager(log log.Logger, dir string, reg prometheus.Registerer, name string, tsdbManager TSDBManager) *HeadManager { +func NewHeadManager(log log.Logger, dir string, metrics *Metrics, tsdbManager TSDBManager) *HeadManager { shards := defaultHeadManagerStripeSize - metrics := NewHeadMetrics(reg) - return &HeadManager{ - name: name, + m := &HeadManager{ log: log, dir: dir, metrics: metrics, @@ -108,6 +106,25 @@ func NewHeadManager(log log.Logger, dir string, reg prometheus.Registerer, name shards: shards, } + m.Index = LazyIndex(func() (Index, error) { + m.mtx.RLock() + defer m.mtx.RUnlock() + + var indices []Index + if m.prevHeads != nil { + indices = append(indices, m.prevHeads) + } + if m.activeHeads != nil { + indices = append(indices, m.activeHeads) + } + + indices = append(indices, m.tsdbManager) + + return NewMultiIndex(indices...) + + }) + + return m } func (m *HeadManager) Stop() error { @@ -281,23 +298,6 @@ func (m *HeadManager) Rotate(t time.Time) error { return nil } -func (m *HeadManager) Index() (Index, error) { - m.mtx.RLock() - defer m.mtx.RUnlock() - - var indices []Index - if m.prevHeads != nil { - indices = append(indices, m.prevHeads) - } - if m.activeHeads != nil { - indices = append(indices, m.activeHeads) - } - - indices = append(indices, m.tsdbManager) - - return NewMultiIndex(indices...) -} - func (m *HeadManager) shippedTSDBsBeforePeriod(period int) (res []string, err error) { files, err := ioutil.ReadDir(managerShippedDir(m.dir)) if err != nil { @@ -403,7 +403,7 @@ func recoverHead(dir string, heads *tenantHeads, wals []WALIdentifier, addTenant // use anonymous function for ease of cleanup if err := func(id WALIdentifier) error { - reader, closer, err := ingester.NewWalReader(walPath(dir, id.ts), -1) + reader, closer, err := wal.NewWalReader(walPath(dir, id.ts), -1) if err != nil { return err } diff --git a/pkg/storage/stores/tsdb/head_manager_test.go b/pkg/storage/stores/tsdb/head_manager_test.go index 0c9a5f5848854..ad7c34466e613 100644 --- a/pkg/storage/stores/tsdb/head_manager_test.go +++ b/pkg/storage/stores/tsdb/head_manager_test.go @@ -35,7 +35,7 @@ func chunkMetasToChunkRefs(user string, fp uint64, xs index.ChunkMetas) (res []C // Test append func Test_TenantHeads_Append(t *testing.T) { - h := newTenantHeads(time.Now(), defaultHeadManagerStripeSize, NewHeadMetrics(nil), log.NewNopLogger()) + h := newTenantHeads(time.Now(), defaultHeadManagerStripeSize, NewMetrics(nil), log.NewNopLogger()) ls := mustParseLabels(`{foo="bar"}`) chks := []index.ChunkMeta{ { @@ -63,7 +63,7 @@ func Test_TenantHeads_Append(t *testing.T) { // Test multitenant reads func Test_TenantHeads_MultiRead(t *testing.T) { - h := newTenantHeads(time.Now(), defaultHeadManagerStripeSize, NewHeadMetrics(nil), log.NewNopLogger()) + h := newTenantHeads(time.Now(), defaultHeadManagerStripeSize, NewMetrics(nil), log.NewNopLogger()) ls := mustParseLabels(`{foo="bar"}`) chks := []index.ChunkMeta{ { @@ -150,7 +150,7 @@ func Test_HeadManager_RecoverHead(t *testing.T) { }, } - mgr := NewHeadManager(log.NewNopLogger(), dir, nil, "tsdb-mgr-test", noopTSDBManager{}) + 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) { @@ -187,9 +187,7 @@ func Test_HeadManager_RecoverHead(t *testing.T) { require.Nil(t, recoverHead(mgr.dir, mgr.activeHeads, grp.wals, false)) for _, c := range cases { - idx, err := mgr.Index() - require.Nil(t, err) - refs, err := idx.GetChunkRefs( + refs, err := mgr.GetChunkRefs( context.Background(), c.User, 0, math.MaxInt64, @@ -235,7 +233,7 @@ func Test_HeadManager_Lifecycle(t *testing.T) { }, } - mgr := NewHeadManager(log.NewNopLogger(), dir, nil, "tsdb-mgr-test", noopTSDBManager{}) + mgr := NewHeadManager(log.NewNopLogger(), dir, nil, noopTSDBManager{}) w, err := newHeadWAL(log.NewNopLogger(), walPath(mgr.dir, curPeriod), curPeriod) require.Nil(t, err) @@ -260,9 +258,7 @@ func Test_HeadManager_Lifecycle(t *testing.T) { require.Nil(t, mgr.Start()) // Ensure old WAL data is queryable for _, c := range cases { - idx, err := mgr.Index() - require.Nil(t, err) - refs, err := idx.GetChunkRefs( + refs, err := mgr.GetChunkRefs( context.Background(), c.User, 0, math.MaxInt64, @@ -294,9 +290,7 @@ func Test_HeadManager_Lifecycle(t *testing.T) { // Ensure old + new data is queryable for _, c := range append(cases, newCase) { - idx, err := mgr.Index() - require.Nil(t, err) - refs, err := idx.GetChunkRefs( + refs, err := mgr.GetChunkRefs( context.Background(), c.User, 0, math.MaxInt64, diff --git a/pkg/storage/stores/tsdb/index_client.go b/pkg/storage/stores/tsdb/index_client.go index 76eea97dedd13..69d0cc8dd44a7 100644 --- a/pkg/storage/stores/tsdb/index_client.go +++ b/pkg/storage/stores/tsdb/index_client.go @@ -18,6 +18,15 @@ type IndexClient struct { 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. diff --git a/pkg/storage/stores/tsdb/lazy_index.go b/pkg/storage/stores/tsdb/lazy_index.go new file mode 100644 index 0000000000000..ecc3f60fbc2fa --- /dev/null +++ b/pkg/storage/stores/tsdb/lazy_index.go @@ -0,0 +1,56 @@ +package tsdb + +import ( + "context" + + "github.com/grafana/loki/pkg/storage/stores/tsdb/index" + "github.com/prometheus/common/model" + "github.com/prometheus/prometheus/model/labels" +) + +// 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) 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...) +} diff --git a/pkg/storage/stores/tsdb/manager.go b/pkg/storage/stores/tsdb/manager.go index 02862e530ee40..8cbfb2515c2a8 100644 --- a/pkg/storage/stores/tsdb/manager.go +++ b/pkg/storage/stores/tsdb/manager.go @@ -34,17 +34,35 @@ tsdbManager is responsible for: * Removing old TSDBs which are no longer needed */ type tsdbManager struct { - period period - name string // node name - log log.Logger - dir string - metrics *Metrics + Index // placeholder until I implement + period period + nodeName string // node name + log log.Logger + dir string + metrics *Metrics sync.RWMutex shipper indexshipper.IndexShipper } +func NewTSDBManager( + nodeName, + dir string, + shipper indexshipper.IndexShipper, + log log.Logger, + metrics *Metrics, +) TSDBManager { + return &tsdbManager{ + period: defaultRotationPeriod, + nodeName: nodeName, + log: log, + dir: dir, + metrics: metrics, + shipper: shipper, + } +} + func (m *tsdbManager) BuildFromWALs(t time.Time, ids []WALIdentifier) (err error) { // get relevant wals // iterate them, build tsdb in scratch dir @@ -75,7 +93,7 @@ func (m *tsdbManager) BuildFromWALs(t time.Time, ids []WALIdentifier) (err error } desired := MultitenantTSDBIdentifier{ - nodeName: m.name, + nodeName: m.nodeName, ts: t, } diff --git a/pkg/storage/stores/tsdb/single_file_index_test.go b/pkg/storage/stores/tsdb/single_file_index_test.go index a84830b10569e..0105afcda38db 100644 --- a/pkg/storage/stores/tsdb/single_file_index_test.go +++ b/pkg/storage/stores/tsdb/single_file_index_test.go @@ -70,7 +70,7 @@ func TestSingleIdx(t *testing.T) { { desc: "head", fn: func() Index { - head := NewHead("fake", NewHeadMetrics(nil), log.NewNopLogger()) + head := NewHead("fake", NewMetrics(nil), log.NewNopLogger()) for _, x := range cases { _, _ = head.Append(x.Labels, x.Chunks) } diff --git a/pkg/util/wal/reader.go b/pkg/util/wal/reader.go new file mode 100644 index 0000000000000..f94b10b84b9cd --- /dev/null +++ b/pkg/util/wal/reader.go @@ -0,0 +1,42 @@ +package wal + +import ( + "errors" + "io" + + "github.com/prometheus/prometheus/tsdb/wal" +) + +// If startSegment is <0, it means all the segments. +func NewWalReader(dir string, startSegment int) (*wal.Reader, io.Closer, error) { + var ( + segmentReader io.ReadCloser + err error + ) + if startSegment < 0 { + segmentReader, err = wal.NewSegmentsReader(dir) + if err != nil { + return nil, nil, err + } + } else { + first, last, err := wal.Segments(dir) + if err != nil { + return nil, nil, err + } + if startSegment > last { + return nil, nil, errors.New("start segment is beyond the last WAL segment") + } + if first > startSegment { + startSegment = first + } + segmentReader, err = wal.NewSegmentsRangeReader(wal.SegmentRange{ + Dir: dir, + First: startSegment, + Last: -1, // Till the end. + }) + if err != nil { + return nil, nil, err + } + } + return wal.NewReader(segmentReader), segmentReader, nil +} From f6f557e57be3ecc210222d0e49f444e24e18c2f3 Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Tue, 19 Apr 2022 16:10:11 -0400 Subject: [PATCH 39/85] tsdb store --- pkg/storage/factory.go | 9 ++-- pkg/storage/store.go | 52 +++++++++++++++++-- .../stores/series/series_store_write.go | 8 +-- 3 files changed, 57 insertions(+), 12 deletions(-) diff --git a/pkg/storage/factory.go b/pkg/storage/factory.go index c4c90a473a4cd..e800dbf7fa1e8 100644 --- a/pkg/storage/factory.go +++ b/pkg/storage/factory.go @@ -23,6 +23,7 @@ import ( "github.com/grafana/loki/pkg/storage/chunk/client/openstack" "github.com/grafana/loki/pkg/storage/chunk/client/testutils" "github.com/grafana/loki/pkg/storage/config" + "github.com/grafana/loki/pkg/storage/stores/indexshipper" "github.com/grafana/loki/pkg/storage/stores/series/index" "github.com/grafana/loki/pkg/storage/stores/shipper" "github.com/grafana/loki/pkg/storage/stores/shipper/downloads" @@ -61,9 +62,9 @@ type Config struct { DisableBroadIndexQueries bool `yaml:"disable_broad_index_queries"` MaxParallelGetChunk int `yaml:"max_parallel_get_chunk"` - MaxChunkBatchSize int `yaml:"max_chunk_batch_size"` - BoltDBShipperConfig shipper.Config `yaml:"boltdb_shipper"` - TSDBShipperConfig shipper.Config `yaml:"tsdb_shipper"` + MaxChunkBatchSize int `yaml:"max_chunk_batch_size"` + BoltDBShipperConfig shipper.Config `yaml:"boltdb_shipper"` + TSDBShipperConfig indexshipper.Config `yaml:"tsdb_shipper"` // Config for using AsyncStore when using async index stores like `boltdb-shipper`. // It is required for getting chunk ids of recently flushed chunks from the ingesters. @@ -152,7 +153,7 @@ func NewIndexClient(name string, cfg Config, schemaCfg config.SchemaConfig, limi return boltDBIndexClientWithShipper, nil } - if shouldUseIndexGatewayClient(cfg) { + if shouldUseBoltDBIndexGatewayClient(cfg) { gateway, err := shipper.NewGatewayClient(cfg.BoltDBShipperConfig.IndexGatewayClientConfig, registerer, util_log.Logger) if err != nil { return nil, err diff --git a/pkg/storage/store.go b/pkg/storage/store.go index 926a535cabb6a..bea578e46276f 100644 --- a/pkg/storage/store.go +++ b/pkg/storage/store.go @@ -23,13 +23,16 @@ import ( "github.com/grafana/loki/pkg/storage/chunk/fetcher" "github.com/grafana/loki/pkg/storage/config" "github.com/grafana/loki/pkg/storage/stores" + "github.com/grafana/loki/pkg/storage/stores/indexshipper" "github.com/grafana/loki/pkg/storage/stores/series" "github.com/grafana/loki/pkg/storage/stores/series/index" "github.com/grafana/loki/pkg/storage/stores/shipper" "github.com/grafana/loki/pkg/storage/stores/shipper/indexgateway" + "github.com/grafana/loki/pkg/storage/stores/tsdb" "github.com/grafana/loki/pkg/usagestats" "github.com/grafana/loki/pkg/util" "github.com/grafana/loki/pkg/util/deletion" + util_log "github.com/grafana/loki/pkg/util/log" ) var ( @@ -184,7 +187,7 @@ func (s *store) chunkClientForPeriod(p config.PeriodConfig) (client.Client, erro return chunks, nil } -func shouldUseIndexGatewayClient(cfg Config) bool { +func shouldUseBoltDBIndexGatewayClient(cfg Config) bool { if cfg.BoltDBShipperConfig.Mode != shipper.ModeReadOnly || cfg.BoltDBShipperConfig.IndexGatewayClientConfig.Disabled { return false } @@ -198,11 +201,52 @@ func shouldUseIndexGatewayClient(cfg Config) bool { } func (s *store) storeForPeriod(p config.PeriodConfig, chunkClient client.Client, f *fetcher.Fetcher) (stores.ChunkWriter, stores.Index, func(), error) { - // todo switch tsdb. - indexClientReg := prometheus.WrapRegistererWith( prometheus.Labels{"component": "index-store-" + p.From.String()}, s.registerer) + if p.IndexType == config.TSDBType { + var ( + nodeName = s.cfg.TSDBShipperConfig.IngesterName + dir = s.cfg.TSDBShipperConfig.ActiveIndexDirectory + ) + writer := tsdb.NewChunkWriter(f, p) + tsdbMetrics := tsdb.NewMetrics(indexClientReg) + objectClient, err := NewObjectClient(s.cfg.TSDBShipperConfig.SharedStoreType, s.cfg, s.clientMetrics) + if err != nil { + return nil, nil, nil, err + } + shpr, err := indexshipper.NewIndexShipper( + s.cfg.TSDBShipperConfig, + objectClient, + nil, + ) + if err != nil { + return nil, nil, nil, err + } + tsdbManager := tsdb.NewTSDBManager( + nodeName, + dir, + shpr, + util_log.Logger, + tsdbMetrics, + ) + headManager := tsdb.NewHeadManager( + util_log.Logger, + dir, + tsdbMetrics, + tsdbManager, + ) + idx := tsdb.NewIndexClient(headManager, p) + + // TODO(owen-d): add TSDB index-gateway support + + return writer, idx, + func() { + chunkClient.Stop() + f.Stop() + }, nil + } + idx, err := NewIndexClient(p.IndexType, s.cfg, s.schemaCfg, s.limits, s.clientMetrics, indexClientReg) if err != nil { return nil, nil, nil, errors.Wrap(err, "error creating index client") @@ -222,7 +266,7 @@ func (s *store) storeForPeriod(p config.PeriodConfig, chunkClient client.Client, index stores.Index = seriesdIndex ) - if shouldUseIndexGatewayClient(s.cfg) { + if shouldUseBoltDBIndexGatewayClient(s.cfg) { // inject the index-gateway client into the index store gw, err := shipper.NewGatewayClient(s.cfg.BoltDBShipperConfig.IndexGatewayClientConfig, indexClientReg, s.logger) if err != nil { diff --git a/pkg/storage/stores/series/series_store_write.go b/pkg/storage/stores/series/series_store_write.go index 73e47f21fe622..790d520a586cc 100644 --- a/pkg/storage/stores/series/series_store_write.go +++ b/pkg/storage/stores/series/series_store_write.go @@ -20,13 +20,13 @@ import ( ) var ( - dedupedChunksTotal = promauto.NewCounter(prometheus.CounterOpts{ + DedupedChunksTotal = promauto.NewCounter(prometheus.CounterOpts{ Namespace: "loki", Name: "chunk_store_deduped_chunks_total", Help: "Count of chunks which were not stored because they have already been stored by another replica.", }) - indexEntriesPerChunk = promauto.NewHistogram(prometheus.HistogramOpts{ + IndexEntriesPerChunk = promauto.NewHistogram(prometheus.HistogramOpts{ Namespace: "loki", Name: "chunk_store_index_entries_per_chunk", Help: "Number of entries written to storage per chunk.", @@ -86,7 +86,7 @@ func (c *Writer) PutOne(ctx context.Context, from, through model.Time, chk chunk if len(found) > 0 { writeChunk = false - dedupedChunksTotal.Inc() + DedupedChunksTotal.Inc() } // If we dont have to write the chunk and DisableIndexDeduplication is false, we do not have to do anything. @@ -170,7 +170,7 @@ func (c *Writer) calculateIndexEntries(ctx context.Context, from, through model. } entries = append(entries, chunkEntries...) - indexEntriesPerChunk.Observe(float64(len(entries))) + IndexEntriesPerChunk.Observe(float64(len(entries))) // Remove duplicate entries based on tableName:hashValue:rangeValue result := c.indexWriter.NewWriteBatch() From 3fbf31671c2f6ad4e40edb173e119de45d3eadf3 Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Thu, 21 Apr 2022 14:12:24 -0400 Subject: [PATCH 40/85] passes indexWriter to chunkWriter --- pkg/storage/store.go | 4 +++- pkg/storage/stores/tsdb/chunkwriter.go | 11 +++++------ 2 files changed, 8 insertions(+), 7 deletions(-) diff --git a/pkg/storage/store.go b/pkg/storage/store.go index bea578e46276f..6c0b858674818 100644 --- a/pkg/storage/store.go +++ b/pkg/storage/store.go @@ -209,7 +209,6 @@ func (s *store) storeForPeriod(p config.PeriodConfig, chunkClient client.Client, nodeName = s.cfg.TSDBShipperConfig.IngesterName dir = s.cfg.TSDBShipperConfig.ActiveIndexDirectory ) - writer := tsdb.NewChunkWriter(f, p) tsdbMetrics := tsdb.NewMetrics(indexClientReg) objectClient, err := NewObjectClient(s.cfg.TSDBShipperConfig.SharedStoreType, s.cfg, s.clientMetrics) if err != nil { @@ -230,6 +229,8 @@ func (s *store) storeForPeriod(p config.PeriodConfig, chunkClient client.Client, util_log.Logger, tsdbMetrics, ) + // TODO(owen-d): Only need HeadManager + // on the ingester. Otherwise, the TSDBManager is sufficient headManager := tsdb.NewHeadManager( util_log.Logger, dir, @@ -237,6 +238,7 @@ func (s *store) storeForPeriod(p config.PeriodConfig, chunkClient client.Client, tsdbManager, ) idx := tsdb.NewIndexClient(headManager, p) + writer := tsdb.NewChunkWriter(f, p, headManager) // TODO(owen-d): add TSDB index-gateway support diff --git a/pkg/storage/stores/tsdb/chunkwriter.go b/pkg/storage/stores/tsdb/chunkwriter.go index bcd80e00032e3..2c79c095ea714 100644 --- a/pkg/storage/stores/tsdb/chunkwriter.go +++ b/pkg/storage/stores/tsdb/chunkwriter.go @@ -15,28 +15,27 @@ import ( "github.com/prometheus/prometheus/model/labels" ) -type indexWriter interface { +type IndexWriter interface { Append(userID string, ls labels.Labels, chks index.ChunkMetas) error } -// satisfied by HeadManager -var _ indexWriter = &HeadManager{} - type ChunkWriter struct { schemaCfg config.SchemaConfig fetcher *fetcher.Fetcher - indexWriter indexWriter + indexWriter IndexWriter } func NewChunkWriter( fetcher *fetcher.Fetcher, pd config.PeriodConfig, + indexWriter IndexWriter, ) *ChunkWriter { return &ChunkWriter{ schemaCfg: config.SchemaConfig{ Configs: []config.PeriodConfig{pd}, }, - fetcher: fetcher, + fetcher: fetcher, + indexWriter: indexWriter, } } From 5ad1cf503f4c8155122c6c00f6a1e3dba16eaf24 Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Thu, 21 Apr 2022 15:43:23 -0400 Subject: [PATCH 41/85] build a tsdb per index bucket in according with shipper conventions --- pkg/storage/store.go | 1 + pkg/storage/stores/tsdb/manager.go | 120 ++++++++++++++++++----------- 2 files changed, 77 insertions(+), 44 deletions(-) diff --git a/pkg/storage/store.go b/pkg/storage/store.go index 6c0b858674818..af45250b60b29 100644 --- a/pkg/storage/store.go +++ b/pkg/storage/store.go @@ -226,6 +226,7 @@ func (s *store) storeForPeriod(p config.PeriodConfig, chunkClient client.Client, nodeName, dir, shpr, + p.IndexTables.Period, util_log.Logger, tsdbMetrics, ) diff --git a/pkg/storage/stores/tsdb/manager.go b/pkg/storage/stores/tsdb/manager.go index 8cbfb2515c2a8..3f4c5e46d3419 100644 --- a/pkg/storage/stores/tsdb/manager.go +++ b/pkg/storage/stores/tsdb/manager.go @@ -16,6 +16,7 @@ import ( "github.com/prometheus/prometheus/model/labels" ) +// nolint:revive // TSDBManager wraps the index shipper and writes/manages // TSDB files on disk type TSDBManager interface { @@ -34,12 +35,12 @@ tsdbManager is responsible for: * Removing old TSDBs which are no longer needed */ type tsdbManager struct { - Index // placeholder until I implement - period period - nodeName string // node name - log log.Logger - dir string - metrics *Metrics + Index // placeholder until I implement + indexPeriod time.Duration + nodeName string // node name + log log.Logger + dir string + metrics *Metrics sync.RWMutex @@ -50,19 +51,29 @@ func NewTSDBManager( nodeName, dir string, shipper indexshipper.IndexShipper, + indexPeriod time.Duration, log log.Logger, metrics *Metrics, ) TSDBManager { return &tsdbManager{ - period: defaultRotationPeriod, - nodeName: nodeName, - log: log, - dir: dir, - metrics: metrics, - shipper: shipper, + indexPeriod: indexPeriod, + nodeName: nodeName, + log: log, + dir: dir, + metrics: metrics, + shipper: shipper, } } +func indexBuckets(indexPeriod time.Duration, from, through model.Time) []int { + start := from.Time().UnixNano() / int64(indexPeriod) + end := through.Time().UnixNano() / int64(indexPeriod) + if start == end { + return []int{int(start)} + } + return []int{int(start), int(end)} +} + func (m *tsdbManager) BuildFromWALs(t time.Time, ids []WALIdentifier) (err error) { // get relevant wals // iterate them, build tsdb in scratch dir @@ -77,50 +88,71 @@ func (m *tsdbManager) BuildFromWALs(t time.Time, ids []WALIdentifier) (err error if err = recoverHead(m.dir, tmp, ids, true); err != nil { return errors.Wrap(err, "building TSDB from WALs") } - b := index.NewBuilder() + + periods := make(map[int]*index.Builder) if err := tmp.forAll(func(user string, ls labels.Labels, chks index.ChunkMetas) { - b.AddSeries( - append(ls, labels.Label{ - Name: TenantLabel, - Value: user, - }), - chks, - ) + // 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) + } + } + + // Add the chunks to all relevant builders + for pd, matchingChks := range pds { + b, ok := periods[pd] + if !ok { + b = index.NewBuilder() + periods[pd] = b + } + + b.AddSeries( + append(ls, labels.Label{ + Name: TenantLabel, + Value: user, + }), + matchingChks, + ) + } + }); err != nil { level.Error(m.log).Log("err", err.Error(), "msg", "building TSDB from WALs") return err } - desired := MultitenantTSDBIdentifier{ - nodeName: m.nodeName, - ts: t, - } + for p, b := range periods { + desired := MultitenantTSDBIdentifier{ + nodeName: m.nodeName, + ts: t, + } - dstFile := filepath.Join(managerBuiltDir(m.dir), desired.Name()) + dstFile := filepath.Join(managerBuiltDir(m.dir), desired.Name()) - // build/move tsdb to multitenant/built dir - _, err = b.Build( - context.TODO(), - managerScratchDir(m.dir), - func(from, through model.Time, checksum uint32) (index.Identifier, string) { + // build/move tsdb to multitenant/built dir + _, err = b.Build( + context.TODO(), + managerScratchDir(m.dir), + func(from, through model.Time, checksum uint32) (index.Identifier, string) { - // We don't use the resulting ID b/c this isn't compaction. - // Instead we'll discard this and use our own override. - return index.Identifier{}, dstFile - }, - ) - if err != nil { - return err - } + // We don't use the resulting ID b/c this isn't compaction. + // Instead we'll discard this and use our own override. + return index.Identifier{}, dstFile + }, + ) + if err != nil { + return err + } - loaded, err := NewShippableTSDBFile(dstFile) - if err != nil { - return err - } + loaded, err := NewShippableTSDBFile(dstFile) + if err != nil { + return err + } - if err := m.shipper.AddIndex(fmt.Sprintf("%d", m.period.PeriodFor(t)), "", loaded); err != nil { - return err + if err := m.shipper.AddIndex(fmt.Sprintf("%d-%s.tsdb", p, m.nodeName), "", loaded); err != nil { + return err + } } return nil From 52c4ea376e1f137d7473cd9115f01648deb4721f Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Thu, 21 Apr 2022 16:08:44 -0400 Subject: [PATCH 42/85] dont open tsdb files until necessary for indexshipper --- pkg/storage/stores/tsdb/single_file_index.go | 26 ++++++++++++++------ 1 file changed, 19 insertions(+), 7 deletions(-) diff --git a/pkg/storage/stores/tsdb/single_file_index.go b/pkg/storage/stores/tsdb/single_file_index.go index a3c2ab14c114b..d0c73df32ed2f 100644 --- a/pkg/storage/stores/tsdb/single_file_index.go +++ b/pkg/storage/stores/tsdb/single_file_index.go @@ -4,6 +4,7 @@ import ( "context" "io" "os" + "sync" "github.com/prometheus/common/model" "github.com/prometheus/prometheus/model/labels" @@ -19,6 +20,8 @@ func LoadTSDBIdentifier(dir string, id index.Identifier) (*TSDBFile, error) { // nolint // TSDBFile is backed by an actual file and implements the indexshipper/index.Index interface type TSDBFile struct { + sync.Mutex + path string // reuse TSDBIndex for reading @@ -35,15 +38,9 @@ func NewShippableTSDBFile(location string) (*TSDBFile, error) { return nil, err } - f, err := os.Open(location) - if err != nil { - return nil, err - } - return &TSDBFile{ path: location, TSDBIndex: idx, - f: f, }, err } @@ -54,13 +51,28 @@ func (f *TSDBFile) Name() string { func (f *TSDBFile) Path() string { return f.path } func (f *TSDBFile) Close() error { + f.Lock() + defer f.Unlock() var errs multierror.MultiError errs.Add(f.TSDBIndex.Close()) - errs.Add(f.f.Close()) + if f.f != nil { + errs.Add(f.f.Close()) + f.f = nil + } return errs.Err() } func (f *TSDBFile) Reader() (io.ReadSeeker, error) { + f.Lock() + defer f.Unlock() + if f.f == nil { + fd, err := os.Open(f.path) + if err != nil { + return nil, err + } + f.f = fd + } + return f.f, nil } From e28b9411cf3c9424835809472b9865c5b9c8f855 Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Thu, 21 Apr 2022 18:42:40 -0400 Subject: [PATCH 43/85] tsdbManager Index impl --- pkg/storage/stores/tsdb/compact_test.go | 4 +- pkg/storage/stores/tsdb/manager.go | 110 +++++++++++++++++-- pkg/storage/stores/tsdb/multitenant.go | 43 ++++++++ pkg/storage/stores/tsdb/single_file_index.go | 8 +- 4 files changed, 148 insertions(+), 17 deletions(-) create mode 100644 pkg/storage/stores/tsdb/multitenant.go diff --git a/pkg/storage/stores/tsdb/compact_test.go b/pkg/storage/stores/tsdb/compact_test.go index 81d43029512ce..78e5950d8d417 100644 --- a/pkg/storage/stores/tsdb/compact_test.go +++ b/pkg/storage/stores/tsdb/compact_test.go @@ -356,7 +356,9 @@ func TestCompactor(t *testing.T) { for _, cases := range tc.input { idx := BuildIndex(t, dir, "fake", cases) defer idx.Close() - indices = append(indices, idx.TSDBIndex) + casted, ok := idx.Index.(*TSDBIndex) + require.Equal(t, true, ok) + indices = append(indices, casted) } out, err := c.Compact(context.Background(), indices...) diff --git a/pkg/storage/stores/tsdb/manager.go b/pkg/storage/stores/tsdb/manager.go index 3f4c5e46d3419..4730996ec74e3 100644 --- a/pkg/storage/stores/tsdb/manager.go +++ b/pkg/storage/stores/tsdb/manager.go @@ -3,6 +3,7 @@ package tsdb import ( "context" "fmt" + "math" "path/filepath" "sync" "time" @@ -10,6 +11,7 @@ import ( "github.com/go-kit/log" "github.com/go-kit/log/level" "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" "github.com/pkg/errors" "github.com/prometheus/common/model" @@ -21,7 +23,6 @@ import ( // TSDB files on disk type TSDBManager interface { Index - // Builds a new TSDB file from a set of WALs BuildFromWALs(time.Time, []WALIdentifier) error } @@ -35,7 +36,6 @@ tsdbManager is responsible for: * Removing old TSDBs which are no longer needed */ type tsdbManager struct { - Index // placeholder until I implement indexPeriod time.Duration nodeName string // node name log log.Logger @@ -65,15 +65,6 @@ func NewTSDBManager( } } -func indexBuckets(indexPeriod time.Duration, from, through model.Time) []int { - start := from.Time().UnixNano() / int64(indexPeriod) - end := through.Time().UnixNano() / int64(indexPeriod) - if start == end { - return []int{int(start)} - } - return []int{int(start), int(end)} -} - func (m *tsdbManager) BuildFromWALs(t time.Time, ids []WALIdentifier) (err error) { // get relevant wals // iterate them, build tsdb in scratch dir @@ -150,10 +141,105 @@ func (m *tsdbManager) BuildFromWALs(t time.Time, ids []WALIdentifier) (err error return err } - if err := m.shipper.AddIndex(fmt.Sprintf("%d-%s.tsdb", p, m.nodeName), "", loaded); err != nil { + 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, userIDs ...string) ([]Index, error) { + var indices []Index + for _, user := range userIDs { + 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 { + impl, ok := idx.(Index) + if !ok { + return fmt.Errorf("unexpected shipper index type: %T", idx) + } + indices = append(indices, impl) + return nil + }); err != nil { + return nil, err + } + } + } + return indices, nil +} + +func (m *tsdbManager) indexForTenant(ctx context.Context, from, through model.Time, userID string) (Index, error) { + // Get all the indices with an empty user id. They're multitenant indices. + multitenants, err := m.indices(ctx, from, through, "") + for i, idx := range multitenants { + multitenants[i] = NewMultiTenantIndex(idx) + } + + if err != nil { + return nil, err + } + tenant, err := m.indices(ctx, from, through, userID) + if err != nil { + return nil, err + } + + combined := append(multitenants, tenant...) + if len(combined) == 0 { + return NoopIndex{}, nil + } + return NewMultiIndex(combined...) +} + +// 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 +} + +// 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.indexForTenant(ctx, from, through, userID) + if err != nil { + return nil, err + } + 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.indexForTenant(ctx, from, through, userID) + if err != nil { + return nil, err + } + 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.indexForTenant(ctx, from, through, userID) + if err != nil { + return nil, err + } + 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.indexForTenant(ctx, from, through, userID) + if err != nil { + return nil, err + } + return idx.LabelValues(ctx, userID, from, through, name, matchers...) +} diff --git a/pkg/storage/stores/tsdb/multitenant.go b/pkg/storage/stores/tsdb/multitenant.go new file mode 100644 index 0000000000000..fafda2d655f31 --- /dev/null +++ b/pkg/storage/stores/tsdb/multitenant.go @@ -0,0 +1,43 @@ +package tsdb + +import ( + "context" + + "github.com/grafana/loki/pkg/storage/stores/tsdb/index" + "github.com/prometheus/common/model" + "github.com/prometheus/prometheus/model/labels" +) + +// 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 { + return append(matchers, labels.MustNewMatcher(labels.MatchEqual, TenantLabel, userID)) +} + +func (m *MultiTenantIndex) Bounds() (model.Time, model.Time) { return m.idx.Bounds() } + +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) { + return m.idx.Series(ctx, userID, from, through, res, shard, withTenantLabel(userID, matchers)...) +} + +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)...) +} diff --git a/pkg/storage/stores/tsdb/single_file_index.go b/pkg/storage/stores/tsdb/single_file_index.go index d0c73df32ed2f..b6d489464796f 100644 --- a/pkg/storage/stores/tsdb/single_file_index.go +++ b/pkg/storage/stores/tsdb/single_file_index.go @@ -25,7 +25,7 @@ type TSDBFile struct { path string // reuse TSDBIndex for reading - *TSDBIndex + Index // open the read only fd // to sastisfy Reader() and Close() methods @@ -39,8 +39,8 @@ func NewShippableTSDBFile(location string) (*TSDBFile, error) { } return &TSDBFile{ - path: location, - TSDBIndex: idx, + path: location, + Index: idx, }, err } @@ -54,7 +54,7 @@ func (f *TSDBFile) Close() error { f.Lock() defer f.Unlock() var errs multierror.MultiError - errs.Add(f.TSDBIndex.Close()) + errs.Add(f.Index.Close()) if f.f != nil { errs.Add(f.f.Close()) f.f = nil From 15667a7fd3c84e497993f8e1dc37edebd0c8e847 Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Fri, 22 Apr 2022 17:10:41 -0400 Subject: [PATCH 44/85] tsdb defaults + initStore fix for invalid looping --- pkg/loki/config_wrapper.go | 29 +++++++++++++++++++ pkg/loki/modules.go | 57 +++++++++++++------------------------- 2 files changed, 49 insertions(+), 37 deletions(-) diff --git a/pkg/loki/config_wrapper.go b/pkg/loki/config_wrapper.go index 0427d5bf486b0..2f1e19cf6fa98 100644 --- a/pkg/loki/config_wrapper.go +++ b/pkg/loki/config_wrapper.go @@ -103,6 +103,10 @@ func (c *ConfigWrapper) ApplyDynamicConfig() cfg.Source { betterBoltdbShipperDefaults(r, &defaults) } + if len(r.SchemaConfig.Configs) > 0 && config.UsingTSDB(r.SchemaConfig.Configs) { + betterTSDBShipperDefaults(r, &defaults) + } + applyFIFOCacheConfig(r) applyIngesterFinalSleep(r) applyIngesterReplicationFactor(r) @@ -497,6 +501,31 @@ func betterBoltdbShipperDefaults(cfg, defaults *ConfigWrapper) { } } +func betterTSDBShipperDefaults(cfg, defaults *ConfigWrapper) { + currentSchemaIdx := config.ActivePeriodConfig(cfg.SchemaConfig.Configs) + currentSchema := cfg.SchemaConfig.Configs[currentSchemaIdx] + + if cfg.StorageConfig.TSDBShipperConfig.SharedStoreType == defaults.StorageConfig.TSDBShipperConfig.SharedStoreType { + cfg.StorageConfig.TSDBShipperConfig.SharedStoreType = currentSchema.ObjectType + } + + if cfg.CompactorConfig.SharedStoreType == defaults.CompactorConfig.SharedStoreType { + cfg.CompactorConfig.SharedStoreType = currentSchema.ObjectType + } + + if cfg.Common.PathPrefix != "" { + prefix := strings.TrimSuffix(cfg.Common.PathPrefix, "/") + + if cfg.StorageConfig.TSDBShipperConfig.ActiveIndexDirectory == "" { + cfg.StorageConfig.TSDBShipperConfig.ActiveIndexDirectory = fmt.Sprintf("%s/tsdb-shipper-active", prefix) + } + + if cfg.StorageConfig.TSDBShipperConfig.CacheLocation == "" { + cfg.StorageConfig.TSDBShipperConfig.CacheLocation = fmt.Sprintf("%s/tsdb-shipper-cache", prefix) + } + } +} + // applyFIFOCacheConfig turns on FIFO cache for the chunk store and for the query range results, // but only if no other cache storage is configured (redis or memcache). // diff --git a/pkg/loki/modules.go b/pkg/loki/modules.go index c992fd04e6404..31f239e4c07fc 100644 --- a/pkg/loki/modules.go +++ b/pkg/loki/modules.go @@ -49,6 +49,7 @@ import ( "github.com/grafana/loki/pkg/storage/chunk/cache" chunk_util "github.com/grafana/loki/pkg/storage/chunk/client/util" "github.com/grafana/loki/pkg/storage/config" + "github.com/grafana/loki/pkg/storage/stores/indexshipper" "github.com/grafana/loki/pkg/storage/stores/series/index" "github.com/grafana/loki/pkg/storage/stores/shipper" "github.com/grafana/loki/pkg/storage/stores/shipper/compactor" @@ -383,13 +384,9 @@ func (t *Loki) initTableManager() (services.Service, error) { func (t *Loki) initStore() (_ services.Service, err error) { // Always set these configs - for _, cfg := range []*shipper.Config{ - &t.Cfg.StorageConfig.BoltDBShipperConfig, - &t.Cfg.StorageConfig.TSDBShipperConfig, - } { - cfg.IndexGatewayClientConfig.Mode = t.Cfg.IndexGateway.Mode - cfg.IndexGatewayClientConfig.Ring = t.indexGatewayRing - } + // TODO(owen-d): Do the same for TSDB when we add IndexGatewayClientConfig + t.Cfg.StorageConfig.BoltDBShipperConfig.IndexGatewayClientConfig.Mode = t.Cfg.IndexGateway.Mode + t.Cfg.StorageConfig.BoltDBShipperConfig.IndexGatewayClientConfig.Ring = t.indexGatewayRing // If RF > 1 and current or upcoming index type is boltdb-shipper then disable index dedupe and write dedupe cache. // This is to ensure that index entries are replicated to all the boltdb files in ingesters flushing replicated data. @@ -421,34 +418,24 @@ func (t *Loki) initStore() (_ services.Service, err error) { // and queried as part of live data until the cache TTL expires on the index entry. t.Cfg.Ingester.RetainPeriod = t.Cfg.StorageConfig.IndexCacheValidity + 1*time.Minute - for _, cfg := range []*shipper.Config{ - &t.Cfg.StorageConfig.BoltDBShipperConfig, - &t.Cfg.StorageConfig.TSDBShipperConfig, - } { - // We do not want ingester to unnecessarily keep downloading files - cfg.Mode = shipper.ModeWriteOnly - - cfg.IngesterName = t.Cfg.Ingester.LifecyclerConfig.ID + // We do not want ingester to unnecessarily keep downloading files + t.Cfg.StorageConfig.BoltDBShipperConfig.Mode = shipper.ModeWriteOnly + t.Cfg.StorageConfig.BoltDBShipperConfig.IngesterName = t.Cfg.Ingester.LifecyclerConfig.ID + t.Cfg.StorageConfig.BoltDBShipperConfig.IngesterDBRetainPeriod = shipperQuerierIndexUpdateDelay(t.Cfg.StorageConfig.IndexCacheValidity, t.Cfg.StorageConfig.BoltDBShipperConfig.ResyncInterval) + 2*time.Minute - cfg.IngesterDBRetainPeriod = shipperQuerierIndexUpdateDelay(t.Cfg.StorageConfig.IndexCacheValidity, cfg.ResyncInterval) + 2*time.Minute - } + t.Cfg.StorageConfig.TSDBShipperConfig.Mode = indexshipper.ModeWriteOnly + t.Cfg.StorageConfig.TSDBShipperConfig.IngesterName = t.Cfg.Ingester.LifecyclerConfig.ID + t.Cfg.StorageConfig.TSDBShipperConfig.IngesterDBRetainPeriod = shipperQuerierIndexUpdateDelay(t.Cfg.StorageConfig.IndexCacheValidity, t.Cfg.StorageConfig.TSDBShipperConfig.ResyncInterval) + 2*time.Minute case t.Cfg.isModuleEnabled(Querier), t.Cfg.isModuleEnabled(Ruler), t.Cfg.isModuleEnabled(Read), t.isModuleActive(IndexGateway): - for _, cfg := range []*shipper.Config{ - &t.Cfg.StorageConfig.BoltDBShipperConfig, - &t.Cfg.StorageConfig.TSDBShipperConfig, - } { - // We do not want query to do any updates to index - cfg.Mode = shipper.ModeReadOnly - } + // We do not want query to do any updates to index + t.Cfg.StorageConfig.BoltDBShipperConfig.Mode = shipper.ModeReadOnly + t.Cfg.StorageConfig.TSDBShipperConfig.Mode = indexshipper.ModeReadOnly default: - for _, cfg := range []*shipper.Config{ - &t.Cfg.StorageConfig.BoltDBShipperConfig, - &t.Cfg.StorageConfig.TSDBShipperConfig, - } { - cfg.Mode = shipper.ModeReadWrite - cfg.IngesterDBRetainPeriod = shipperQuerierIndexUpdateDelay(t.Cfg.StorageConfig.IndexCacheValidity, cfg.ResyncInterval) + 2*time.Minute - } + t.Cfg.StorageConfig.BoltDBShipperConfig.Mode = shipper.ModeReadWrite + t.Cfg.StorageConfig.BoltDBShipperConfig.IngesterDBRetainPeriod = shipperQuerierIndexUpdateDelay(t.Cfg.StorageConfig.IndexCacheValidity, t.Cfg.StorageConfig.BoltDBShipperConfig.ResyncInterval) + 2*time.Minute + t.Cfg.StorageConfig.TSDBShipperConfig.Mode = indexshipper.ModeReadWrite + t.Cfg.StorageConfig.TSDBShipperConfig.IngesterDBRetainPeriod = shipperQuerierIndexUpdateDelay(t.Cfg.StorageConfig.IndexCacheValidity, t.Cfg.StorageConfig.TSDBShipperConfig.ResyncInterval) + 2*time.Minute } } @@ -489,12 +476,8 @@ func (t *Loki) initStore() (_ services.Service, err error) { asyncStore = true case t.Cfg.isModuleEnabled(IndexGateway): // we want to use the actual storage when running the index-gateway, so we remove the Addr from the config - for _, cfg := range []*shipper.Config{ - &t.Cfg.StorageConfig.BoltDBShipperConfig, - &t.Cfg.StorageConfig.TSDBShipperConfig, - } { - cfg.IndexGatewayClientConfig.Disabled = true - } + // TODO(owen-d): Do the same for TSDB when we add IndexGatewayClientConfig + t.Cfg.StorageConfig.BoltDBShipperConfig.IndexGatewayClientConfig.Disabled = true case t.Cfg.isModuleEnabled(All): // We want ingester to also query the store when using boltdb-shipper but only when running with target All. // We do not want to use AsyncStore otherwise it would start spiraling around doing queries over and over again to the ingesters and store. From 47757985deacdc2035d12d976530f550f4f4722d Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Fri, 22 Apr 2022 17:37:02 -0400 Subject: [PATCH 45/85] fixes UsingTSDB helper --- pkg/storage/config/schema_config.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pkg/storage/config/schema_config.go b/pkg/storage/config/schema_config.go index 0864fd2db632c..18b48433b55dd 100644 --- a/pkg/storage/config/schema_config.go +++ b/pkg/storage/config/schema_config.go @@ -220,7 +220,7 @@ func UsingBoltdbShipper(configs []PeriodConfig) bool { func UsingTSDB(configs []PeriodConfig) bool { fn := func(cfg PeriodConfig) bool { - return cfg.IndexType == BoltDBShipperType + return cfg.IndexType == TSDBType } return usingForPeriodConfigs(configs, fn) From 7a51edae80878be5efc8a872ed9fe1f8c7f7ad72 Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Fri, 22 Apr 2022 17:37:17 -0400 Subject: [PATCH 46/85] disables deleteRequestStore when using TSDB --- pkg/loki/modules.go | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/pkg/loki/modules.go b/pkg/loki/modules.go index 31f239e4c07fc..e769d17a4433b 100644 --- a/pkg/loki/modules.go +++ b/pkg/loki/modules.go @@ -916,6 +916,11 @@ func (t *Loki) initUsageReport() (services.Service, error) { } func (t *Loki) deleteRequestsStore() (deletion.DeleteRequestsStore, error) { + // TODO(owen-d): enable delete request storage in tsdb + if config.UsingTSDB(t.Cfg.SchemaConfig.Configs) { + return deletion.NewNoOpDeleteRequestsStore(), nil + } + filteringEnabled, err := deletion.FilteringEnabled(t.Cfg.CompactorConfig.DeletionMode) if err != nil { return nil, err From 5fa3b4af121d19ae320cff596704b0ac9b7a97a9 Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Fri, 22 Apr 2022 17:43:35 -0400 Subject: [PATCH 47/85] pass limits to tsdb store --- pkg/storage/store.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pkg/storage/store.go b/pkg/storage/store.go index af45250b60b29..a57262d742075 100644 --- a/pkg/storage/store.go +++ b/pkg/storage/store.go @@ -217,7 +217,7 @@ func (s *store) storeForPeriod(p config.PeriodConfig, chunkClient client.Client, shpr, err := indexshipper.NewIndexShipper( s.cfg.TSDBShipperConfig, objectClient, - nil, + s.limits, ) if err != nil { return nil, nil, nil, err From c88d742735bbcc1ae3c579e666e0856b90df0a32 Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Mon, 25 Apr 2022 15:45:05 -0400 Subject: [PATCH 48/85] always start headmanager for tsdb Signed-off-by: Owen Diehl --- pkg/storage/store.go | 3 +++ 1 file changed, 3 insertions(+) diff --git a/pkg/storage/store.go b/pkg/storage/store.go index a57262d742075..2a4a52e6f3ed1 100644 --- a/pkg/storage/store.go +++ b/pkg/storage/store.go @@ -238,6 +238,9 @@ func (s *store) storeForPeriod(p config.PeriodConfig, chunkClient client.Client, tsdbMetrics, tsdbManager, ) + if err := headManager.Start(); err != nil { + return nil, nil, nil, err + } idx := tsdb.NewIndexClient(headManager, p) writer := tsdb.NewChunkWriter(f, p, headManager) From ed6071096057081c67932a991b3a15d725f3f66d Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Mon, 25 Apr 2022 16:17:04 -0400 Subject: [PATCH 49/85] fixes copy bug Signed-off-by: Owen Diehl --- pkg/storage/stores/tsdb/head_manager.go | 30 +++++++++++-------------- 1 file changed, 13 insertions(+), 17 deletions(-) diff --git a/pkg/storage/stores/tsdb/head_manager.go b/pkg/storage/stores/tsdb/head_manager.go index 8fe1234920614..7152e76751ebd 100644 --- a/pkg/storage/stores/tsdb/head_manager.go +++ b/pkg/storage/stores/tsdb/head_manager.go @@ -94,10 +94,10 @@ type HeadManager struct { Index } -func NewHeadManager(log log.Logger, dir string, metrics *Metrics, tsdbManager TSDBManager) *HeadManager { +func NewHeadManager(logger log.Logger, dir string, metrics *Metrics, tsdbManager TSDBManager) *HeadManager { shards := defaultHeadManagerStripeSize m := &HeadManager{ - log: log, + log: log.With(logger, "component", "tsdb-head-manager"), dir: dir, metrics: metrics, tsdbManager: tsdbManager, @@ -268,10 +268,12 @@ func (m *HeadManager) Rotate(t time.Time) error { // build tsdb from rotated-out period // TODO(owen-d): don't block Append() waiting for tsdb building. Use a work channel/etc if m.prev != nil { + level.Debug(m.log).Log("msg", "combining tsdb WALs") grp, _, err := walsForPeriod(m.dir, m.period, m.period.PeriodFor(m.prev.initialized)) if err != nil { return errors.Wrap(err, "listing wals") } + level.Debug(m.log).Log("msg", "listed WALs", "pd", grp.period, "n", len(grp.wals)) // TODO(owen-d): It's probably faster to build this from the *tenantHeads instead, // but we already need to impl BuildFromWALs to ensure we can correctly build/ship @@ -287,6 +289,7 @@ func (m *HeadManager) Rotate(t time.Time) error { if err := m.removeWALGroup(grp); err != nil { return errors.Wrapf(err, "removing prev TSDB WALs for period %d", grp.period) } + level.Debug(m.log).Log("msg", "removing wals", "pd", grp.period, "n", len(grp.wals)) } @@ -400,7 +403,6 @@ func walPath(parent string, t time.Time) string { // and inserts it into the active *tenantHeads func recoverHead(dir string, heads *tenantHeads, wals []WALIdentifier, addTenantLabels bool) error { for _, id := range wals { - // use anonymous function for ease of cleanup if err := func(id WALIdentifier) error { reader, closer, err := wal.NewWalReader(walPath(dir, id.ts), -1) @@ -523,13 +525,13 @@ type tenantHeads struct { metrics *Metrics } -func newTenantHeads(start time.Time, shards int, metrics *Metrics, log log.Logger) *tenantHeads { +func newTenantHeads(start time.Time, shards int, metrics *Metrics, logger log.Logger) *tenantHeads { res := &tenantHeads{ start: start, shards: shards, locks: make([]sync.RWMutex, shards), tenants: make([]map[string]*Head, shards), - log: log, + log: log.With(logger, "component", "tenant-heads"), metrics: metrics, } for i := range res.tenants { @@ -661,11 +663,6 @@ func (t *tenantHeads) forAll(fn func(user string, ls labels.Labels, chks index.C t.locks[i].RLock() defer t.locks[i].RUnlock() - var ( - ls labels.Labels - chks []index.ChunkMeta - ) - for user, tenant := range shard { idx := tenant.Index() ps, err := postingsForMatcher(idx, nil, labels.MustNewMatcher(labels.MatchEqual, "", "")) @@ -674,19 +671,18 @@ func (t *tenantHeads) forAll(fn func(user string, ls labels.Labels, chks index.C } for ps.Next() { + var ( + ls labels.Labels + chks []index.ChunkMeta + ) + _, err := idx.Series(ps.At(), &ls, &chks) if err != nil { return errors.Wrapf(err, "iterating postings for tenant: %s", user) } - // We'll be reusing the slices, so copy before calling fn - dst := make(index.ChunkMetas, 0, len(chks)) - _ = copy(dst, chks) - fn(user, ls.Copy(), dst) - - ls = ls[:0] - chks = chks[:0] + fn(user, ls, chks) } } } From cacbea8f98b64e19b1f92003d53b85a816d3cde7 Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Mon, 25 Apr 2022 18:58:27 -0400 Subject: [PATCH 50/85] more logging Signed-off-by: Owen Diehl --- pkg/storage/stores/tsdb/manager.go | 13 +++++++++---- 1 file changed, 9 insertions(+), 4 deletions(-) diff --git a/pkg/storage/stores/tsdb/manager.go b/pkg/storage/stores/tsdb/manager.go index 4730996ec74e3..ae058d8925ff0 100644 --- a/pkg/storage/stores/tsdb/manager.go +++ b/pkg/storage/stores/tsdb/manager.go @@ -52,13 +52,13 @@ func NewTSDBManager( dir string, shipper indexshipper.IndexShipper, indexPeriod time.Duration, - log log.Logger, + logger log.Logger, metrics *Metrics, ) TSDBManager { return &tsdbManager{ indexPeriod: indexPeriod, nodeName: nodeName, - log: log, + log: log.With(logger, "component", "tsdb-manager"), dir: dir, metrics: metrics, shipper: shipper, @@ -66,6 +66,7 @@ func NewTSDBManager( } func (m *tsdbManager) BuildFromWALs(t time.Time, ids []WALIdentifier) (err error) { + level.Info(m.log).Log("msg", "building WALs", "n", len(ids), "ts", t) // get relevant wals // iterate them, build tsdb in scratch dir defer func() { @@ -119,11 +120,13 @@ func (m *tsdbManager) BuildFromWALs(t time.Time, ids []WALIdentifier) (err error ts: t, } - dstFile := filepath.Join(managerBuiltDir(m.dir), desired.Name()) + dstFile := filepath.Join(managerBuiltDir(m.dir), fmt.Sprint(p), desired.Name()) + level.Info(m.log).Log("msg", "building tsdb for period", "pd", p, "dst", dstFile) // build/move tsdb to multitenant/built dir + start := time.Now() _, err = b.Build( - context.TODO(), + context.Background(), managerScratchDir(m.dir), func(from, through model.Time, checksum uint32) (index.Identifier, string) { @@ -136,6 +139,8 @@ func (m *tsdbManager) BuildFromWALs(t time.Time, ids []WALIdentifier) (err error return err } + level.Info(m.log).Log("msg", "finished building tsdb for period", "pd", p, "dst", dstFile, "duration", time.Since(start)) + loaded, err := NewShippableTSDBFile(dstFile) if err != nil { return err From afecd217bf2a47a50f92d3653874c7c25a701ca0 Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Mon, 25 Apr 2022 19:20:55 -0400 Subject: [PATCH 51/85] fixes duplicate tenant label bug Signed-off-by: Owen Diehl --- pkg/storage/stores/tsdb/head_manager.go | 10 ++-------- pkg/storage/stores/tsdb/head_manager_test.go | 2 +- pkg/storage/stores/tsdb/manager.go | 2 +- 3 files changed, 4 insertions(+), 10 deletions(-) diff --git a/pkg/storage/stores/tsdb/head_manager.go b/pkg/storage/stores/tsdb/head_manager.go index 7152e76751ebd..20e9c46f66118 100644 --- a/pkg/storage/stores/tsdb/head_manager.go +++ b/pkg/storage/stores/tsdb/head_manager.go @@ -203,7 +203,7 @@ func (m *HeadManager) Start() error { } if group.period == curPeriod { - if err := recoverHead(m.dir, m.activeHeads, group.wals, false); err != nil { + if err := recoverHead(m.dir, m.activeHeads, group.wals); err != nil { return errors.Wrap(err, "recovering tsdb head from wal") } } @@ -401,7 +401,7 @@ func walPath(parent string, t time.Time) string { // recoverHead recovers from all WALs belonging to some period // and inserts it into the active *tenantHeads -func recoverHead(dir string, heads *tenantHeads, wals []WALIdentifier, addTenantLabels bool) error { +func recoverHead(dir string, heads *tenantHeads, wals []WALIdentifier) error { for _, id := range wals { // use anonymous function for ease of cleanup if err := func(id WALIdentifier) error { @@ -440,12 +440,6 @@ func recoverHead(dir string, heads *tenantHeads, wals []WALIdentifier, addTenant if !ok { return errors.New("found tsdb chunk metas without series in WAL replay") } - if addTenantLabels { - ls = append(ls, labels.Label{ - Name: TenantLabel, - Value: rec.UserID, - }) - } _ = heads.Append(rec.UserID, ls, rec.Chks.Chks) } } diff --git a/pkg/storage/stores/tsdb/head_manager_test.go b/pkg/storage/stores/tsdb/head_manager_test.go index ad7c34466e613..64eb0745877c4 100644 --- a/pkg/storage/stores/tsdb/head_manager_test.go +++ b/pkg/storage/stores/tsdb/head_manager_test.go @@ -184,7 +184,7 @@ func Test_HeadManager_RecoverHead(t *testing.T) { 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, false)) + require.Nil(t, recoverHead(mgr.dir, mgr.activeHeads, grp.wals)) for _, c := range cases { refs, err := mgr.GetChunkRefs( diff --git a/pkg/storage/stores/tsdb/manager.go b/pkg/storage/stores/tsdb/manager.go index ae058d8925ff0..3080c9c4274f1 100644 --- a/pkg/storage/stores/tsdb/manager.go +++ b/pkg/storage/stores/tsdb/manager.go @@ -77,7 +77,7 @@ func (m *tsdbManager) BuildFromWALs(t time.Time, ids []WALIdentifier) (err error }() tmp := newTenantHeads(t, defaultHeadManagerStripeSize, m.metrics, m.log) - if err = recoverHead(m.dir, tmp, ids, true); err != nil { + if err = recoverHead(m.dir, tmp, ids); err != nil { return errors.Wrap(err, "building TSDB from WALs") } From 1f84ce0dc0957c7cbdf8276193ad44a76e1bdb80 Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Tue, 26 Apr 2022 09:51:51 -0400 Subject: [PATCH 52/85] debug logs, uses label builder, removes __name__=logs for tsdb Signed-off-by: Owen Diehl --- pkg/storage/stores/tsdb/manager.go | 19 ++++++++++++------- 1 file changed, 12 insertions(+), 7 deletions(-) diff --git a/pkg/storage/stores/tsdb/manager.go b/pkg/storage/stores/tsdb/manager.go index 3080c9c4274f1..a48a0d6487448 100644 --- a/pkg/storage/stores/tsdb/manager.go +++ b/pkg/storage/stores/tsdb/manager.go @@ -66,7 +66,7 @@ func NewTSDBManager( } func (m *tsdbManager) BuildFromWALs(t time.Time, ids []WALIdentifier) (err error) { - level.Info(m.log).Log("msg", "building WALs", "n", len(ids), "ts", t) + 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() { @@ -76,6 +76,7 @@ func (m *tsdbManager) BuildFromWALs(t time.Time, ids []WALIdentifier) (err error } }() + 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") @@ -84,6 +85,13 @@ func (m *tsdbManager) BuildFromWALs(t time.Time, ids []WALIdentifier) (err error periods := make(map[int]*index.Builder) if err := tmp.forAll(func(user string, ls labels.Labels, chks index.ChunkMetas) { + + labelsBuilder := labels.NewBuilder(ls) + // TSDB doesnt need the __name__="log" convention the old chunk store index used. + labelsBuilder.Del("__name__") + labelsBuilder.Set(TenantLabel, user) + metric := labelsBuilder.Labels() + // chunks may overlap index period bounds, in which case they're written to multiple pds := make(map[int]index.ChunkMetas) for _, chk := range chks { @@ -101,10 +109,7 @@ func (m *tsdbManager) BuildFromWALs(t time.Time, ids []WALIdentifier) (err error } b.AddSeries( - append(ls, labels.Label{ - Name: TenantLabel, - Value: user, - }), + metric, matchingChks, ) } @@ -121,7 +126,7 @@ func (m *tsdbManager) BuildFromWALs(t time.Time, ids []WALIdentifier) (err error } dstFile := filepath.Join(managerBuiltDir(m.dir), fmt.Sprint(p), desired.Name()) - level.Info(m.log).Log("msg", "building tsdb for period", "pd", p, "dst", dstFile) + level.Debug(m.log).Log("msg", "building tsdb for period", "pd", p, "dst", dstFile) // build/move tsdb to multitenant/built dir start := time.Now() @@ -139,7 +144,7 @@ func (m *tsdbManager) BuildFromWALs(t time.Time, ids []WALIdentifier) (err error return err } - level.Info(m.log).Log("msg", "finished building tsdb for period", "pd", p, "dst", dstFile, "duration", time.Since(start)) + level.Debug(m.log).Log("msg", "finished building tsdb for period", "pd", p, "dst", dstFile, "duration", time.Since(start)) loaded, err := NewShippableTSDBFile(dstFile) if err != nil { From 3b734cf4963de785dd3bb88f33ca7dddad2b5a5d Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Tue, 26 Apr 2022 16:13:55 -0400 Subject: [PATCH 53/85] tsdb fixes labels at earlier pt Signed-off-by: Owen Diehl --- pkg/storage/stores/tsdb/head_manager.go | 9 ++++++++- pkg/storage/stores/tsdb/manager.go | 8 +------- 2 files changed, 9 insertions(+), 8 deletions(-) diff --git a/pkg/storage/stores/tsdb/head_manager.go b/pkg/storage/stores/tsdb/head_manager.go index 20e9c46f66118..9f8d73c749656 100644 --- a/pkg/storage/stores/tsdb/head_manager.go +++ b/pkg/storage/stores/tsdb/head_manager.go @@ -134,6 +134,13 @@ func (m *HeadManager) Stop() error { } func (m *HeadManager) Append(userID string, ls labels.Labels, chks index.ChunkMetas) error { + labelsBuilder := labels.NewBuilder(ls) + // TSDB doesnt need the __name__="log" convention the old chunk store index used. + labelsBuilder.Del("__name__") + // userIDs are also included until compaction occurs in tsdb + labelsBuilder.Set(TenantLabel, userID) + metric := labelsBuilder.Labels() + m.mtx.RLock() now := time.Now() if m.period.PeriodFor(now) > m.period.PeriodFor(m.activeHeads.start) { @@ -144,7 +151,7 @@ func (m *HeadManager) Append(userID string, ls labels.Labels, chks index.ChunkMe m.mtx.RLock() } defer m.mtx.RUnlock() - rec := m.activeHeads.Append(userID, ls, chks) + rec := m.activeHeads.Append(userID, metric, chks) return m.active.Log(rec) } diff --git a/pkg/storage/stores/tsdb/manager.go b/pkg/storage/stores/tsdb/manager.go index a48a0d6487448..1c5d66414c855 100644 --- a/pkg/storage/stores/tsdb/manager.go +++ b/pkg/storage/stores/tsdb/manager.go @@ -86,12 +86,6 @@ func (m *tsdbManager) BuildFromWALs(t time.Time, ids []WALIdentifier) (err error if err := tmp.forAll(func(user string, ls labels.Labels, chks index.ChunkMetas) { - labelsBuilder := labels.NewBuilder(ls) - // TSDB doesnt need the __name__="log" convention the old chunk store index used. - labelsBuilder.Del("__name__") - labelsBuilder.Set(TenantLabel, user) - metric := labelsBuilder.Labels() - // chunks may overlap index period bounds, in which case they're written to multiple pds := make(map[int]index.ChunkMetas) for _, chk := range chks { @@ -109,7 +103,7 @@ func (m *tsdbManager) BuildFromWALs(t time.Time, ids []WALIdentifier) (err error } b.AddSeries( - metric, + ls, matchingChks, ) } From b88df5cafe2702323f43d803f5f437c2a8ee4455 Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Wed, 27 Apr 2022 10:58:51 -0400 Subject: [PATCH 54/85] account for setting tenant label in head manager test --- pkg/storage/stores/tsdb/head_manager_test.go | 14 +++++++++++--- 1 file changed, 11 insertions(+), 3 deletions(-) diff --git a/pkg/storage/stores/tsdb/head_manager_test.go b/pkg/storage/stores/tsdb/head_manager_test.go index 64eb0745877c4..3756284ce68be 100644 --- a/pkg/storage/stores/tsdb/head_manager_test.go +++ b/pkg/storage/stores/tsdb/head_manager_test.go @@ -239,11 +239,13 @@ func Test_HeadManager_Lifecycle(t *testing.T) { // Write old WALs for i, c := range cases { + lbls := labels.NewBuilder(c.Labels) + lbls.Set(TenantLabel, c.User) require.Nil(t, w.Log(&WALRecord{ UserID: c.User, Series: record.RefSeries{ Ref: chunks.HeadSeriesRef(i), - Labels: c.Labels, + Labels: lbls.Labels(), }, Chks: ChunkMetasRecord{ Chks: c.Chunks, @@ -266,7 +268,10 @@ func Test_HeadManager_Lifecycle(t *testing.T) { labels.MustNewMatcher(labels.MatchRegexp, "foo", ".+"), ) require.Nil(t, err) - require.Equal(t, chunkMetasToChunkRefs(c.User, c.Labels.Hash(), c.Chunks), refs) + + lbls := labels.NewBuilder(c.Labels) + lbls.Set(TenantLabel, c.User) + require.Equal(t, chunkMetasToChunkRefs(c.User, lbls.Labels().Hash(), c.Chunks), refs) } // Add data @@ -298,7 +303,10 @@ func Test_HeadManager_Lifecycle(t *testing.T) { labels.MustNewMatcher(labels.MatchRegexp, "foo", ".+"), ) require.Nil(t, err) - require.Equal(t, chunkMetasToChunkRefs(c.User, c.Labels.Hash(), c.Chunks), refs) + + lbls := labels.NewBuilder(c.Labels) + lbls.Set(TenantLabel, c.User) + require.Equal(t, chunkMetasToChunkRefs(c.User, lbls.Labels().Hash(), c.Chunks), refs) } } From cdb9cfd1c60d5de87a0fac8412e34ac0ee10a92f Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Wed, 27 Apr 2022 11:02:39 -0400 Subject: [PATCH 55/85] changing tsdb dir names --- pkg/storage/stores/tsdb/head_manager.go | 52 ++++++++++++------------- pkg/storage/stores/tsdb/manager.go | 2 +- 2 files changed, 25 insertions(+), 29 deletions(-) diff --git a/pkg/storage/stores/tsdb/head_manager.go b/pkg/storage/stores/tsdb/head_manager.go index 9f8d73c749656..f4e1c80f2222f 100644 --- a/pkg/storage/stores/tsdb/head_manager.go +++ b/pkg/storage/stores/tsdb/head_manager.go @@ -50,27 +50,23 @@ the old versions to build + upload TSDB files. On disk, it looks like: tsdb/ - # scratch directory used for temp tsdb files during build stage - scratch/ - # wal directory used to store WALs being written on the ingester. - # These are eventually shipped to storage as multi-tenant TSDB files - # and compacted into per tenant indices - wal/ - - # multitenant tsdb files which are created on the ingesters/shipped - multitenant/ - # contains built TSDBs - built/ - -.tsdb - # once shipped successfully, they're moved here and can be safely deleted later - shipped/ - -.tsdb - compacted/ - # post-compaction tenant tsdbs which are grouped per - # period bucket - / - / - index---.tsdb + v1/ + # scratch directory used for temp tsdb files during build stage + scratch/ + # wal directory used to store WALs being written on the ingester. + # These are eventually shipped to storage as multi-tenant TSDB files + # and compacted into per tenant indices + wal/ + + # multitenant tsdb files which are created on the ingesters/shipped + multitenant/ + -.tsdb + per_tenant/ + # post-compaction tenant tsdbs which are grouped per + # period bucket + / + / + index---.tsdb */ type HeadManager struct { @@ -230,14 +226,14 @@ func managerRequiredDirs(parent string) []string { return []string{ managerScratchDir(parent), managerWalDir(parent), - managerBuiltDir(parent), - managerShippedDir(parent), + managerMultitenantDir(parent), + managerPerTenantDir(parent), } } -func managerScratchDir(parent string) string { return filepath.Join(parent, "scratch") } -func managerWalDir(parent string) string { return filepath.Join(parent, "wal") } -func managerBuiltDir(parent string) string { return filepath.Join(parent, "multitenant", "built") } -func managerShippedDir(parent string) string { return filepath.Join(parent, "multitenant", "shipped") } +func managerScratchDir(parent string) string { return filepath.Join(parent, "v1", "scratch") } +func managerWalDir(parent string) string { return filepath.Join(parent, "v1", "wal") } +func managerMultitenantDir(parent string) string { return filepath.Join(parent, "v1", "multitenant") } +func managerPerTenantDir(parent string) string { return filepath.Join(parent, "v1", "per_tenant") } func (m *HeadManager) Rotate(t time.Time) error { // create new wal @@ -309,7 +305,7 @@ func (m *HeadManager) Rotate(t time.Time) error { } func (m *HeadManager) shippedTSDBsBeforePeriod(period int) (res []string, err error) { - files, err := ioutil.ReadDir(managerShippedDir(m.dir)) + files, err := ioutil.ReadDir(managerPerTenantDir(m.dir)) if err != nil { return nil, err } diff --git a/pkg/storage/stores/tsdb/manager.go b/pkg/storage/stores/tsdb/manager.go index 1c5d66414c855..57521acf7f95b 100644 --- a/pkg/storage/stores/tsdb/manager.go +++ b/pkg/storage/stores/tsdb/manager.go @@ -119,7 +119,7 @@ func (m *tsdbManager) BuildFromWALs(t time.Time, ids []WALIdentifier) (err error ts: t, } - dstFile := filepath.Join(managerBuiltDir(m.dir), fmt.Sprint(p), desired.Name()) + dstFile := filepath.Join(managerMultitenantDir(m.dir), fmt.Sprint(p), desired.Name()) level.Debug(m.log).Log("msg", "building tsdb for period", "pd", p, "dst", dstFile) // build/move tsdb to multitenant/built dir From 2c2dbb7a4c356f465f7d50877a64d225e85b9661 Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Wed, 27 Apr 2022 16:40:32 -0400 Subject: [PATCH 56/85] identifier interface, builder to tsdb pkg --- .../stores/tsdb/{index => }/builder.go | 54 ++++++++------- pkg/storage/stores/tsdb/compact.go | 19 ++++-- pkg/storage/stores/tsdb/compact_test.go | 3 +- .../stores/tsdb/fake/index-1-15-dc7af36e.tsdb | Bin 0 -> 496 bytes pkg/storage/stores/tsdb/head_manager.go | 24 +++++-- pkg/storage/stores/tsdb/index/chunk.go | 4 +- pkg/storage/stores/tsdb/manager.go | 62 ++++++++++++++---- pkg/storage/stores/tsdb/querier_test.go | 10 +-- pkg/storage/stores/tsdb/single_file_index.go | 27 +++----- pkg/storage/stores/tsdb/util_test.go | 11 ++-- 10 files changed, 134 insertions(+), 80 deletions(-) rename pkg/storage/stores/tsdb/{index => }/builder.go (74%) create mode 100644 pkg/storage/stores/tsdb/fake/index-1-15-dc7af36e.tsdb diff --git a/pkg/storage/stores/tsdb/index/builder.go b/pkg/storage/stores/tsdb/builder.go similarity index 74% rename from pkg/storage/stores/tsdb/index/builder.go rename to pkg/storage/stores/tsdb/builder.go index 915c853d6ae69..f75179349c728 100644 --- a/pkg/storage/stores/tsdb/index/builder.go +++ b/pkg/storage/stores/tsdb/builder.go @@ -1,10 +1,11 @@ -package index +package tsdb import ( "context" "fmt" "math/rand" "os" + "path" "path/filepath" "sort" @@ -13,31 +14,33 @@ import ( "github.com/prometheus/prometheus/storage" chunk_util "github.com/grafana/loki/pkg/storage/chunk/client/util" + "github.com/grafana/loki/pkg/storage/stores/tsdb/index" ) // Identifier has all the information needed to resolve a TSDB index // Notably this abstracts away OS path separators, etc. -type Identifier struct { +type SingleTenantTSDBIdentifier struct { Tenant string From, Through model.Time Checksum uint32 } -func (i Identifier) String() string { - return filepath.Join( - i.Tenant, - fmt.Sprintf( - "%s-%d-%d-%x.tsdb", - IndexFilename, - i.From, - i.Through, - i.Checksum, - ), +func (i SingleTenantTSDBIdentifier) str() string { + return fmt.Sprintf( + "%s-%d-%d-%x.tsdb", + index.IndexFilename, + i.From, + i.Through, + i.Checksum, ) } -func (i Identifier) FilePath(parentDir string) string { - return filepath.Join(parentDir, i.String()) +func (i SingleTenantTSDBIdentifier) Name() string { + return path.Join(i.Tenant, i.str()) +} + +func (i SingleTenantTSDBIdentifier) Path() string { + return filepath.Join(i.Tenant, i.str()) } // Builder is a helper used to create tsdb indices. @@ -51,14 +54,14 @@ type Builder struct { type stream struct { labels labels.Labels - chunks ChunkMetas + chunks index.ChunkMetas } func NewBuilder() *Builder { return &Builder{streams: make(map[string]*stream)} } -func (b *Builder) AddSeries(ls labels.Labels, chks []ChunkMeta) { +func (b *Builder) AddSeries(ls labels.Labels, chks []index.ChunkMeta) { id := ls.String() s, ok := b.streams[id] if !ok { @@ -78,7 +81,7 @@ func (b *Builder) Build( // This is variable as we use Builder for multiple reasons, // such as building multi-tenant tsdbs on the ingester // and per tenant ones during compaction - createFn func(from, through model.Time, checksum uint32) (Identifier, string), + createFn func(from, through model.Time, checksum uint32) Identifier, ) (id Identifier, err error) { // Ensure the parent dir exists (i.e. index///) if scratchDir != "" { @@ -89,10 +92,10 @@ func (b *Builder) Build( // First write tenant/index-bounds-random.staging rng := rand.Int63() - name := fmt.Sprintf("%s-%x.staging", IndexFilename, rng) + name := fmt.Sprintf("%s-%x.staging", index.IndexFilename, rng) tmpPath := filepath.Join(scratchDir, name) - writer, err := NewWriter(ctx, tmpPath) + writer, err := index.NewWriter(ctx, tmpPath) if err != nil { return id, err } @@ -135,7 +138,7 @@ func (b *Builder) Build( // Add series for i, s := range streams { - if err := writer.AddSeries(storage.SeriesRef(i), s.labels, s.chunks.finalize()...); err != nil { + if err := writer.AddSeries(storage.SeriesRef(i), s.labels, s.chunks.Finalize()...); err != nil { return id, err } } @@ -144,7 +147,7 @@ func (b *Builder) Build( return id, err } - reader, err := NewFileReader(tmpPath) + reader, err := index.NewFileReader(tmpPath) if err != nil { return id, err } @@ -152,7 +155,7 @@ func (b *Builder) Build( from, through := reader.Bounds() // load the newly compacted index to grab checksum, promptly close - id, dst := createFn(model.Time(from), model.Time(through), reader.Checksum()) + dst := createFn(model.Time(from), model.Time(through), reader.Checksum()) reader.Close() defer func() { @@ -161,12 +164,13 @@ func (b *Builder) Build( } }() - if err := chunk_util.EnsureDirectory(filepath.Dir(dst)); err != nil { + if err := chunk_util.EnsureDirectory(filepath.Dir(dst.Path())); err != nil { return id, err } - if err := os.Rename(tmpPath, dst); err != nil { + dstPath := dst.Path() + if err := os.Rename(tmpPath, dstPath); err != nil { return id, err } - return id, nil + return dst, nil } diff --git a/pkg/storage/stores/tsdb/compact.go b/pkg/storage/stores/tsdb/compact.go index d7e8abe51dd55..f19c29fee29f6 100644 --- a/pkg/storage/stores/tsdb/compact.go +++ b/pkg/storage/stores/tsdb/compact.go @@ -24,10 +24,15 @@ func NewCompactor(tenant, parentDir string) *Compactor { } } -func (c *Compactor) Compact(ctx context.Context, indices ...*TSDBIndex) (res index.Identifier, err error) { +func (c *Compactor) Compact(ctx context.Context, indices ...*TSDBIndex) (res Identifier, err error) { // No need to compact a single index file if len(indices) == 1 { - return indices[0].Identifier(c.tenant), nil + return newPrefixedIdentifier( + indices[0].Identifier(c.tenant), + c.parentDir, + c.parentDir, + ), + nil } ifcs := make([]Index, 0, len(indices)) @@ -35,7 +40,7 @@ func (c *Compactor) Compact(ctx context.Context, indices ...*TSDBIndex) (res ind ifcs = append(ifcs, idx) } - b := index.NewBuilder() + b := NewBuilder() multi, err := NewMultiIndex(ifcs...) if err != nil { return res, err @@ -50,7 +55,7 @@ func (c *Compactor) Compact(ctx context.Context, indices ...*TSDBIndex) (res ind for _, idx := range multi.indices { casted, ok := idx.(*TSDBIndex) if !ok { - return index.Identifier{}, fmt.Errorf("expected tsdb index to compact, found :%T", idx) + return nil, fmt.Errorf("expected tsdb index to compact, found :%T", idx) } if err := casted.forSeries( nil, @@ -67,14 +72,14 @@ func (c *Compactor) Compact(ctx context.Context, indices ...*TSDBIndex) (res ind return b.Build( ctx, c.parentDir, - func(from, through model.Time, checksum uint32) (index.Identifier, string) { - id := index.Identifier{ + func(from, through model.Time, checksum uint32) Identifier { + id := SingleTenantTSDBIdentifier{ Tenant: c.tenant, From: from, Through: through, Checksum: checksum, } - return id, id.FilePath(c.parentDir) + return newPrefixedIdentifier(id, c.parentDir, c.parentDir) }, ) } diff --git a/pkg/storage/stores/tsdb/compact_test.go b/pkg/storage/stores/tsdb/compact_test.go index 78e5950d8d417..500569aac3b8c 100644 --- a/pkg/storage/stores/tsdb/compact_test.go +++ b/pkg/storage/stores/tsdb/compact_test.go @@ -366,8 +366,9 @@ func TestCompactor(t *testing.T) { require.NotNil(t, err) return } + require.Nil(t, err) - idx, err := LoadTSDBIdentifier(dir, out) + idx, err := NewShippableTSDBFile(out) require.Nil(t, err) defer idx.Close() diff --git a/pkg/storage/stores/tsdb/fake/index-1-15-dc7af36e.tsdb b/pkg/storage/stores/tsdb/fake/index-1-15-dc7af36e.tsdb new file mode 100644 index 0000000000000000000000000000000000000000..02cb227674daf71fe52281ee3660af534137469d GIT binary patch literal 496 zcmdnB>N*1x0|SE;5VJ8RGA1!6B^I$HC013jB$ZZGF{kC{M|^NH0?I;w3Mb!|*!SW; zRTx>Bm>HNMg3J(_5lSpKqh7uRwhO!POvC**Tf_2AQlf0gNy)UQI-B(Xksi1S7$Fn6Jug4sqq7f2>>z3 zS`cOfV$Mf3DIP$Y1BgN9u}zv`bqrT3VSoj5>9ms# kKyeUYg-{IYP Date: Wed, 27 Apr 2022 16:52:13 -0400 Subject: [PATCH 57/85] tsdb version path prefix --- pkg/storage/stores/tsdb/manager.go | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/pkg/storage/stores/tsdb/manager.go b/pkg/storage/stores/tsdb/manager.go index 30d4201b3ca8f..d566142df7d3b 100644 --- a/pkg/storage/stores/tsdb/manager.go +++ b/pkg/storage/stores/tsdb/manager.go @@ -25,6 +25,10 @@ func (v VersionPrefix) String() string { return fmt.Sprintf("v%d", v) } +func (v VersionPrefix) PathPrefix() string { + return fmt.Sprintf("tsdb/%s", v.String()) +} + const ( _ VersionPrefix = iota V1 @@ -160,7 +164,8 @@ func (m *tsdbManager) BuildFromWALs(t time.Time, ids []WALIdentifier) (err error ts: t, } dstDir := filepath.Join(managerMultitenantDir(m.dir), fmt.Sprint(p)) - dst := newPrefixedIdentifier(desired, dstDir, dstDir) + dstName := filepath.Join(managerMultitenantDir(V1.PathPrefix()), fmt.Sprint(p)) + dst := newPrefixedIdentifier(desired, dstDir, dstName) level.Debug(m.log).Log("msg", "building tsdb for period", "pd", p, "dst", dst.Path()) // build/move tsdb to multitenant/built dir From 40cc48c9b7963ea4c0d29c0819c4a276057bd1aa Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Wed, 27 Apr 2022 17:21:26 -0400 Subject: [PATCH 58/85] fixes buildfromwals identifier --- pkg/storage/stores/tsdb/manager.go | 18 +++++++++++------- 1 file changed, 11 insertions(+), 7 deletions(-) diff --git a/pkg/storage/stores/tsdb/manager.go b/pkg/storage/stores/tsdb/manager.go index d566142df7d3b..f4fc1dbd5b90e 100644 --- a/pkg/storage/stores/tsdb/manager.go +++ b/pkg/storage/stores/tsdb/manager.go @@ -159,22 +159,26 @@ func (m *tsdbManager) BuildFromWALs(t time.Time, ids []WALIdentifier) (err error } for p, b := range periods { - desired := MultitenantTSDBIdentifier{ - nodeName: m.nodeName, - ts: t, - } + dstDir := filepath.Join(managerMultitenantDir(m.dir), fmt.Sprint(p)) dstName := filepath.Join(managerMultitenantDir(V1.PathPrefix()), fmt.Sprint(p)) - dst := newPrefixedIdentifier(desired, dstDir, dstName) + dst := newPrefixedIdentifier( + MultitenantTSDBIdentifier{ + nodeName: m.nodeName, + ts: t, + }, + dstDir, + dstName, + ) level.Debug(m.log).Log("msg", "building tsdb for period", "pd", p, "dst", dst.Path()) - // build/move tsdb to multitenant/built dir + // 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 MultitenantTSDBIdentifier{} + return dst }, ) if err != nil { From 8d2d033e7b83338c583f312191a906b78850df77 Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Thu, 28 Apr 2022 10:10:28 -0400 Subject: [PATCH 59/85] fixes tsdb shipper paths --- pkg/storage/stores/tsdb/head_manager.go | 9 +++++++++ pkg/storage/stores/tsdb/manager.go | 2 +- 2 files changed, 10 insertions(+), 1 deletion(-) diff --git a/pkg/storage/stores/tsdb/head_manager.go b/pkg/storage/stores/tsdb/head_manager.go index fc2786def8bb4..58db46eba8661 100644 --- a/pkg/storage/stores/tsdb/head_manager.go +++ b/pkg/storage/stores/tsdb/head_manager.go @@ -5,6 +5,7 @@ import ( "fmt" "io/ioutil" "os" + "path" "path/filepath" "sort" "strconv" @@ -242,10 +243,18 @@ func managerMultitenantDir(parent string) string { return filepath.Join(parent, V1.String(), "multitenant") } +func managerMultitenantName() string { + return path.Join(V1.String(), "multitenant") +} + func managerPerTenantDir(parent string) string { return filepath.Join(parent, V1.String(), "per_tenant") } +func managerPerTenantName() string { + return path.Join(V1.String(), "per_tenant") +} + func (m *HeadManager) Rotate(t time.Time) error { // create new wal nextWALPath := walPath(m.dir, t) diff --git a/pkg/storage/stores/tsdb/manager.go b/pkg/storage/stores/tsdb/manager.go index f4fc1dbd5b90e..2438e03ecc72b 100644 --- a/pkg/storage/stores/tsdb/manager.go +++ b/pkg/storage/stores/tsdb/manager.go @@ -161,7 +161,7 @@ func (m *tsdbManager) BuildFromWALs(t time.Time, ids []WALIdentifier) (err error for p, b := range periods { dstDir := filepath.Join(managerMultitenantDir(m.dir), fmt.Sprint(p)) - dstName := filepath.Join(managerMultitenantDir(V1.PathPrefix()), fmt.Sprint(p)) + dstName := filepath.Join(managerMultitenantName()) dst := newPrefixedIdentifier( MultitenantTSDBIdentifier{ nodeName: m.nodeName, From 08500ed8af46cef6891730b3687e330f17b24de3 Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Thu, 28 Apr 2022 11:48:23 -0400 Subject: [PATCH 60/85] split buckets once per user set --- pkg/storage/stores/tsdb/manager.go | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/pkg/storage/stores/tsdb/manager.go b/pkg/storage/stores/tsdb/manager.go index 2438e03ecc72b..0b4fa8eaf9607 100644 --- a/pkg/storage/stores/tsdb/manager.go +++ b/pkg/storage/stores/tsdb/manager.go @@ -211,8 +211,9 @@ func indexBuckets(indexPeriod time.Duration, from, through model.Time) (res []in func (m *tsdbManager) indices(ctx context.Context, from, through model.Time, userIDs ...string) ([]Index, error) { var indices []Index - for _, user := range userIDs { - for _, bkt := range indexBuckets(m.indexPeriod, from, through) { + + for _, bkt := range indexBuckets(m.indexPeriod, from, through) { + for _, user := range userIDs { if err := m.shipper.ForEach(ctx, fmt.Sprintf("%d", bkt), user, func(idx shipper_index.Index) error { impl, ok := idx.(Index) if !ok { @@ -224,6 +225,7 @@ func (m *tsdbManager) indices(ctx context.Context, from, through model.Time, use return nil, err } } + } return indices, nil } From f84af6acf183800e77c8c1cba39699e8937c5d08 Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Thu, 28 Apr 2022 12:41:02 -0400 Subject: [PATCH 61/85] refactors combining single and multi tenant tsdb indices on shipper reads --- pkg/storage/stores/tsdb/head_manager.go | 20 +++++++----- pkg/storage/stores/tsdb/manager.go | 41 +++++++++---------------- 2 files changed, 27 insertions(+), 34 deletions(-) diff --git a/pkg/storage/stores/tsdb/head_manager.go b/pkg/storage/stores/tsdb/head_manager.go index 58db46eba8661..7fc07415533e9 100644 --- a/pkg/storage/stores/tsdb/head_manager.go +++ b/pkg/storage/stores/tsdb/head_manager.go @@ -508,10 +508,21 @@ func (id MultitenantTSDBIdentifier) Path() string { } func parseMultitenantTSDBPath(p string) (id MultitenantTSDBIdentifier, ok bool) { - trimmed := strings.TrimSuffix(p, ".tsdb") + 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 == p { + if trimmed == name { return } @@ -520,11 +531,6 @@ func parseMultitenantTSDBPath(p string) (id MultitenantTSDBIdentifier, ok bool) return } - // require node name isn't empty - if len(xs[1]) == 0 { - return - } - ts, err := strconv.Atoi(xs[0]) if err != nil { return diff --git a/pkg/storage/stores/tsdb/manager.go b/pkg/storage/stores/tsdb/manager.go index 0b4fa8eaf9607..27f3370e720d9 100644 --- a/pkg/storage/stores/tsdb/manager.go +++ b/pkg/storage/stores/tsdb/manager.go @@ -161,7 +161,7 @@ func (m *tsdbManager) BuildFromWALs(t time.Time, ids []WALIdentifier) (err error for p, b := range periods { dstDir := filepath.Join(managerMultitenantDir(m.dir), fmt.Sprint(p)) - dstName := filepath.Join(managerMultitenantName()) + dstName := managerMultitenantName() dst := newPrefixedIdentifier( MultitenantTSDBIdentifier{ nodeName: m.nodeName, @@ -209,17 +209,22 @@ func indexBuckets(indexPeriod time.Duration, from, through model.Time) (res []in return } -func (m *tsdbManager) indices(ctx context.Context, from, through model.Time, userIDs ...string) ([]Index, error) { +func (m *tsdbManager) indices(ctx context.Context, from, through model.Time, userIDs ...string) (Index, error) { var indices []Index for _, bkt := range indexBuckets(m.indexPeriod, from, through) { for _, user := range userIDs { 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) } - indices = append(indices, impl) + if multitenant { + indices = append(indices, NewMultiTenantIndex(impl)) + } else { + indices = append(indices, impl) + } return nil }); err != nil { return nil, err @@ -227,29 +232,11 @@ func (m *tsdbManager) indices(ctx context.Context, from, through model.Time, use } } - return indices, nil -} - -func (m *tsdbManager) indexForTenant(ctx context.Context, from, through model.Time, userID string) (Index, error) { - // Get all the indices with an empty user id. They're multitenant indices. - multitenants, err := m.indices(ctx, from, through, "") - for i, idx := range multitenants { - multitenants[i] = NewMultiTenantIndex(idx) - } - - if err != nil { - return nil, err - } - tenant, err := m.indices(ctx, from, through, userID) - if err != nil { - return nil, err - } - combined := append(multitenants, tenant...) - if len(combined) == 0 { + if len(indices) == 0 { return NoopIndex{}, nil } - return NewMultiIndex(combined...) + return NewMultiIndex(indices...) } // TODO(owen-d): how to better implement this? @@ -266,7 +253,7 @@ func (m *tsdbManager) Close() error { } 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.indexForTenant(ctx, from, through, userID) + idx, err := m.indices(ctx, from, through, userID, "") if err != nil { return nil, err } @@ -274,7 +261,7 @@ func (m *tsdbManager) GetChunkRefs(ctx context.Context, userID string, from, thr } 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.indexForTenant(ctx, from, through, userID) + idx, err := m.indices(ctx, from, through, userID, "") if err != nil { return nil, err } @@ -282,7 +269,7 @@ func (m *tsdbManager) Series(ctx context.Context, userID string, from, through m } func (m *tsdbManager) LabelNames(ctx context.Context, userID string, from, through model.Time, matchers ...*labels.Matcher) ([]string, error) { - idx, err := m.indexForTenant(ctx, from, through, userID) + idx, err := m.indices(ctx, from, through, userID, "") if err != nil { return nil, err } @@ -290,7 +277,7 @@ func (m *tsdbManager) LabelNames(ctx context.Context, userID string, from, throu } func (m *tsdbManager) LabelValues(ctx context.Context, userID string, from, through model.Time, name string, matchers ...*labels.Matcher) ([]string, error) { - idx, err := m.indexForTenant(ctx, from, through, userID) + idx, err := m.indices(ctx, from, through, userID, "") if err != nil { return nil, err } From aa4633607ce4e64a65d5069e3ae8ac0bd000ea5c Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Thu, 28 Apr 2022 15:52:11 -0400 Subject: [PATCH 62/85] indexshipper ignores old gzip logic --- pkg/storage/stores/indexshipper/uploads/index_set.go | 11 +---------- 1 file changed, 1 insertion(+), 10 deletions(-) diff --git a/pkg/storage/stores/indexshipper/uploads/index_set.go b/pkg/storage/stores/indexshipper/uploads/index_set.go index 1ffdba8dd88b2..e636652dfc29b 100644 --- a/pkg/storage/stores/indexshipper/uploads/index_set.go +++ b/pkg/storage/stores/indexshipper/uploads/index_set.go @@ -238,14 +238,5 @@ func (t *indexSet) removeIndex(name string) error { } func (t *indexSet) buildFileName(indexName string) string { - // Files are stored with - - fileName := fmt.Sprintf("%s-%s", t.uploader, indexName) - - // if the file is a migrated one then don't add its name to the object key otherwise we would re-upload them again here with a different name. - // This is kept for historic reasons of boltdb-shipper. - if t.tableName == indexName { - fileName = t.uploader - } - - return fmt.Sprintf("%s.gz", fileName) + return fmt.Sprintf("%s.gz", indexName) } From f388e240b7cda29c36e3dc97a52c56ade5785565 Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Thu, 28 Apr 2022 16:09:56 -0400 Subject: [PATCH 63/85] method name refactor --- pkg/storage/stores/tsdb/index/chunk_test.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pkg/storage/stores/tsdb/index/chunk_test.go b/pkg/storage/stores/tsdb/index/chunk_test.go index 9da566e3b0ab5..f20613c828abc 100644 --- a/pkg/storage/stores/tsdb/index/chunk_test.go +++ b/pkg/storage/stores/tsdb/index/chunk_test.go @@ -135,7 +135,7 @@ func TestChunkMetasFinalize(t *testing.T) { }, } { t.Run(tc.desc, func(t *testing.T) { - require.Equal(t, tc.output, tc.input.finalize()) + require.Equal(t, tc.output, tc.input.Finalize()) }) } } From 4690dffdee6cd313b912b3a183763a1b5b23b961 Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Tue, 3 May 2022 07:57:49 -0400 Subject: [PATCH 64/85] remove unused record type --- pkg/storage/stores/tsdb/head_wal.go | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/pkg/storage/stores/tsdb/head_wal.go b/pkg/storage/stores/tsdb/head_wal.go index dee7d1c9506d1..ed7f72897c1db 100644 --- a/pkg/storage/stores/tsdb/head_wal.go +++ b/pkg/storage/stores/tsdb/head_wal.go @@ -36,8 +36,7 @@ const ( // 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. - WALRecordFirstWrite RecordType = iota - WalRecordSeries + WalRecordSeries RecordType = iota WalRecordChunks ) From a31c2034884ebd5236a1be2d3d37c0bf70a466c1 Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Tue, 3 May 2022 08:08:49 -0400 Subject: [PATCH 65/85] removes v1 prefix in tsdb paths and refactores indices method --- pkg/storage/stores/tsdb/head_manager.go | 12 +++++------ pkg/storage/stores/tsdb/manager.go | 28 +++++++------------------ 2 files changed, 14 insertions(+), 26 deletions(-) diff --git a/pkg/storage/stores/tsdb/head_manager.go b/pkg/storage/stores/tsdb/head_manager.go index 7fc07415533e9..72b308b28084c 100644 --- a/pkg/storage/stores/tsdb/head_manager.go +++ b/pkg/storage/stores/tsdb/head_manager.go @@ -232,27 +232,27 @@ func managerRequiredDirs(parent string) []string { } } func managerScratchDir(parent string) string { - return filepath.Join(parent, V1.String(), "scratch") + return filepath.Join(parent, "scratch") } func managerWalDir(parent string) string { - return filepath.Join(parent, V1.String(), "wal") + return filepath.Join(parent, "wal") } func managerMultitenantDir(parent string) string { - return filepath.Join(parent, V1.String(), "multitenant") + return filepath.Join(parent, "multitenant") } func managerMultitenantName() string { - return path.Join(V1.String(), "multitenant") + return "multitenant" } func managerPerTenantDir(parent string) string { - return filepath.Join(parent, V1.String(), "per_tenant") + return filepath.Join(parent, "per_tenant") } func managerPerTenantName() string { - return path.Join(V1.String(), "per_tenant") + return "per_tenant" } func (m *HeadManager) Rotate(t time.Time) error { diff --git a/pkg/storage/stores/tsdb/manager.go b/pkg/storage/stores/tsdb/manager.go index 27f3370e720d9..fa03aa04d842c 100644 --- a/pkg/storage/stores/tsdb/manager.go +++ b/pkg/storage/stores/tsdb/manager.go @@ -19,21 +19,6 @@ import ( "github.com/prometheus/prometheus/model/labels" ) -type VersionPrefix int - -func (v VersionPrefix) String() string { - return fmt.Sprintf("v%d", v) -} - -func (v VersionPrefix) PathPrefix() string { - return fmt.Sprintf("tsdb/%s", v.String()) -} - -const ( - _ VersionPrefix = iota - V1 -) - // nolint:revive // TSDBManager wraps the index shipper and writes/manages // TSDB files on disk @@ -209,9 +194,12 @@ func indexBuckets(indexPeriod time.Duration, from, through model.Time) (res []in return } -func (m *tsdbManager) indices(ctx context.Context, from, through model.Time, userIDs ...string) (Index, error) { +func (m *tsdbManager) indices(ctx context.Context, from, through model.Time, userID string) (Index, error) { var indices []Index + // Ensure we query both per tenant and multitenant TSDBs + userIDs := []string{userID, ""} + for _, bkt := range indexBuckets(m.indexPeriod, from, through) { for _, user := range userIDs { if err := m.shipper.ForEach(ctx, fmt.Sprintf("%d", bkt), user, func(idx shipper_index.Index) error { @@ -253,7 +241,7 @@ func (m *tsdbManager) Close() error { } 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, "") + idx, err := m.indices(ctx, from, through, userID) if err != nil { return nil, err } @@ -261,7 +249,7 @@ func (m *tsdbManager) GetChunkRefs(ctx context.Context, userID string, from, thr } 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, "") + idx, err := m.indices(ctx, from, through, userID) if err != nil { return nil, err } @@ -269,7 +257,7 @@ func (m *tsdbManager) Series(ctx context.Context, userID string, from, through m } 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, "") + idx, err := m.indices(ctx, from, through, userID) if err != nil { return nil, err } @@ -277,7 +265,7 @@ func (m *tsdbManager) LabelNames(ctx context.Context, userID string, from, throu } 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, "") + idx, err := m.indices(ctx, from, through, userID) if err != nil { return nil, err } From 8e6956fae74d18642d6e177ef2eddc3b886070ae Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Tue, 3 May 2022 09:11:46 -0400 Subject: [PATCH 66/85] ignores double optimization in tsdb looking for multitenant idx, shipper handles this --- pkg/storage/stores/tsdb/manager.go | 31 ++++++++++++++---------------- 1 file changed, 14 insertions(+), 17 deletions(-) diff --git a/pkg/storage/stores/tsdb/manager.go b/pkg/storage/stores/tsdb/manager.go index fa03aa04d842c..32e7ac8e5c9a5 100644 --- a/pkg/storage/stores/tsdb/manager.go +++ b/pkg/storage/stores/tsdb/manager.go @@ -194,29 +194,26 @@ func indexBuckets(indexPeriod time.Duration, from, through model.Time) (res []in return } -func (m *tsdbManager) indices(ctx context.Context, from, through model.Time, userID string) (Index, error) { +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 - userIDs := []string{userID, ""} for _, bkt := range indexBuckets(m.indexPeriod, from, through) { - for _, user := range userIDs { - 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 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 } } From e928519a75bfdf12572b5f17fa7c9cb440a79e02 Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Tue, 3 May 2022 09:12:45 -0400 Subject: [PATCH 67/85] removes 5-ln requirement on shipper tablename regexp --- pkg/storage/stores/indexshipper/downloads/table_manager.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/pkg/storage/stores/indexshipper/downloads/table_manager.go b/pkg/storage/stores/indexshipper/downloads/table_manager.go index 423a71baab1ec..bb9f78e36e7e9 100644 --- a/pkg/storage/stores/indexshipper/downloads/table_manager.go +++ b/pkg/storage/stores/indexshipper/downloads/table_manager.go @@ -242,7 +242,7 @@ func (tm *tableManager) ensureQueryReadiness(ctx context.Context) error { } // regex for finding daily tables which have a 5 digit number at the end. - re, err := regexp.Compile(`.+[0-9]{5}$`) + re, err := regexp.Compile(`[0-9]+$`) if err != nil { return err } @@ -252,7 +252,7 @@ func (tm *tableManager) ensureQueryReadiness(ctx context.Context) error { continue } - tableNumber, err := strconv.ParseInt(tableName[len(tableName)-5:], 10, 64) + tableNumber, err := strconv.ParseInt(tableName, 10, 64) if err != nil { return err } From bc0fe93bb8c7aed02f9384cf2093537d348ef896 Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Tue, 3 May 2022 10:09:38 -0400 Subject: [PATCH 68/85] groups identifiers, begins removing multitenant prefix in shipped files --- pkg/storage/stores/tsdb/builder.go | 27 --- pkg/storage/stores/tsdb/head_manager.go | 59 ------ pkg/storage/stores/tsdb/identifier.go | 184 +++++++++++++++++++ pkg/storage/stores/tsdb/identifier_test.go | 57 ++++++ pkg/storage/stores/tsdb/manager.go | 33 +--- pkg/storage/stores/tsdb/single_file_index.go | 10 + 6 files changed, 252 insertions(+), 118 deletions(-) create mode 100644 pkg/storage/stores/tsdb/identifier.go create mode 100644 pkg/storage/stores/tsdb/identifier_test.go diff --git a/pkg/storage/stores/tsdb/builder.go b/pkg/storage/stores/tsdb/builder.go index f75179349c728..22bf3a64839db 100644 --- a/pkg/storage/stores/tsdb/builder.go +++ b/pkg/storage/stores/tsdb/builder.go @@ -5,7 +5,6 @@ import ( "fmt" "math/rand" "os" - "path" "path/filepath" "sort" @@ -17,32 +16,6 @@ import ( "github.com/grafana/loki/pkg/storage/stores/tsdb/index" ) -// 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()) -} - // 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()` diff --git a/pkg/storage/stores/tsdb/head_manager.go b/pkg/storage/stores/tsdb/head_manager.go index 72b308b28084c..2f69caf2ca2d9 100644 --- a/pkg/storage/stores/tsdb/head_manager.go +++ b/pkg/storage/stores/tsdb/head_manager.go @@ -5,11 +5,9 @@ import ( "fmt" "io/ioutil" "os" - "path" "path/filepath" "sort" "strconv" - "strings" "sync" "time" @@ -243,18 +241,10 @@ func managerMultitenantDir(parent string) string { return filepath.Join(parent, "multitenant") } -func managerMultitenantName() string { - return "multitenant" -} - func managerPerTenantDir(parent string) string { return filepath.Join(parent, "per_tenant") } -func managerPerTenantName() string { - return "per_tenant" -} - func (m *HeadManager) Rotate(t time.Time) error { // create new wal nextWALPath := walPath(m.dir, t) @@ -493,55 +483,6 @@ func parseWALPath(p string) (id WALIdentifier, ok bool) { }, 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 -} - type tenantHeads struct { mint, maxt atomic.Int64 // easy lookup for Bounds() impl diff --git a/pkg/storage/stores/tsdb/identifier.go b/pkg/storage/stores/tsdb/identifier.go new file mode 100644 index 0000000000000..98c653a4ebab2 --- /dev/null +++ b/pkg/storage/stores/tsdb/identifier.go @@ -0,0 +1,184 @@ +package tsdb + +import ( + "fmt" + "path" + "path/filepath" + "strconv" + "strings" + "time" + + "github.com/grafana/loki/pkg/storage/stores/tsdb/index" + "github.com/prometheus/common/model" +) + +// 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()) +} + +// 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 +} diff --git a/pkg/storage/stores/tsdb/identifier_test.go b/pkg/storage/stores/tsdb/identifier_test.go new file mode 100644 index 0000000000000..619651cd2364f --- /dev/null +++ b/pkg/storage/stores/tsdb/identifier_test.go @@ -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) + }) + } +} diff --git a/pkg/storage/stores/tsdb/manager.go b/pkg/storage/stores/tsdb/manager.go index 32e7ac8e5c9a5..47f019931777b 100644 --- a/pkg/storage/stores/tsdb/manager.go +++ b/pkg/storage/stores/tsdb/manager.go @@ -4,7 +4,6 @@ import ( "context" "fmt" "math" - "path" "path/filepath" "sync" "time" @@ -28,35 +27,6 @@ type TSDBManager interface { BuildFromWALs(time.Time, []WALIdentifier) error } -// Identifier can resolve an index to a name (in object storage) -// and a path (on disk) -type Identifier interface { - Name() string - Path() string -} - -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()) -} - /* tsdbManager is responsible for: * Turning WALs into optimized multi-tenant TSDBs when requested @@ -146,14 +116,13 @@ func (m *tsdbManager) BuildFromWALs(t time.Time, ids []WALIdentifier) (err error for p, b := range periods { dstDir := filepath.Join(managerMultitenantDir(m.dir), fmt.Sprint(p)) - dstName := managerMultitenantName() dst := newPrefixedIdentifier( MultitenantTSDBIdentifier{ nodeName: m.nodeName, ts: t, }, dstDir, - dstName, + "", ) level.Debug(m.log).Log("msg", "building tsdb for period", "pd", p, "dst", dst.Path()) diff --git a/pkg/storage/stores/tsdb/single_file_index.go b/pkg/storage/stores/tsdb/single_file_index.go index f3dd30e84cadd..171c23c799d45 100644 --- a/pkg/storage/stores/tsdb/single_file_index.go +++ b/pkg/storage/stores/tsdb/single_file_index.go @@ -10,9 +10,19 @@ import ( "github.com/prometheus/prometheus/model/labels" "github.com/grafana/dskit/multierror" + index_shipper "github.com/grafana/loki/pkg/storage/stores/indexshipper/index" "github.com/grafana/loki/pkg/storage/stores/tsdb/index" ) +func OpenShippableTSDB(p string) (index_shipper.Index, error) { + id, err := identifierFromPath(p) + if err != nil { + return nil, err + } + + return NewShippableTSDBFile(id) +} + // nolint // TSDBFile is backed by an actual file and implements the indexshipper/index.Index interface type TSDBFile struct { From 9fc33857f90e33e667879f5c84df0b1cb35efbd5 Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Tue, 3 May 2022 11:31:53 -0400 Subject: [PATCH 69/85] passses open fn to indexshipper --- pkg/storage/store.go | 1 + pkg/storage/stores/indexshipper/shipper.go | 5 +++-- 2 files changed, 4 insertions(+), 2 deletions(-) diff --git a/pkg/storage/store.go b/pkg/storage/store.go index 2a4a52e6f3ed1..c9252205614ea 100644 --- a/pkg/storage/store.go +++ b/pkg/storage/store.go @@ -218,6 +218,7 @@ func (s *store) storeForPeriod(p config.PeriodConfig, chunkClient client.Client, s.cfg.TSDBShipperConfig, objectClient, s.limits, + tsdb.OpenShippableTSDB, ) if err != nil { return nil, nil, nil, err diff --git a/pkg/storage/stores/indexshipper/shipper.go b/pkg/storage/stores/indexshipper/shipper.go index 9353fa5af55b5..fec4ac12e0e74 100644 --- a/pkg/storage/stores/indexshipper/shipper.go +++ b/pkg/storage/stores/indexshipper/shipper.go @@ -83,9 +83,10 @@ type indexShipper struct { // NewIndexShipper creates a shipper for providing index store functionality using index files and object storage. // It manages the whole life cycle of uploading the index and downloading the index at query time. -func NewIndexShipper(cfg Config, storageClient client.ObjectClient, limits downloads.Limits) (IndexShipper, error) { +func NewIndexShipper(cfg Config, storageClient client.ObjectClient, limits downloads.Limits, open index.OpenIndexFileFunc) (IndexShipper, error) { shipper := indexShipper{ - cfg: cfg, + cfg: cfg, + openIndexFileFunc: open, } err := shipper.init(storageClient, limits) From 3ce911025eb84cf0807b033b0ecc5642060743df Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Tue, 3 May 2022 12:02:59 -0400 Subject: [PATCH 70/85] exposes RealByteSlice --- pkg/storage/stores/tsdb/index/index.go | 22 ++++++++++----------- pkg/storage/stores/tsdb/index/index_test.go | 6 +++--- 2 files changed, 14 insertions(+), 14 deletions(-) diff --git a/pkg/storage/stores/tsdb/index/index.go b/pkg/storage/stores/tsdb/index/index.go index f2059a18584c2..2e5429a2de8bb 100644 --- a/pkg/storage/stores/tsdb/index/index.go +++ b/pkg/storage/stores/tsdb/index/index.go @@ -597,7 +597,7 @@ func (w *Writer) finishSymbols() error { } // Load in the symbol table efficiently for the rest of the index writing. - w.symbols, err = NewSymbols(realByteSlice(w.symbolFile.Bytes()), FormatV2, int(w.toc.Symbols)) + w.symbols, err = NewSymbols(RealByteSlice(w.symbolFile.Bytes()), FormatV2, int(w.toc.Symbols)) if err != nil { return errors.Wrap(err, "read symbols") } @@ -616,7 +616,7 @@ func (w *Writer) writeLabelIndices() error { } defer f.Close() - d := encoding.DecWrap(tsdb_enc.NewDecbufRaw(realByteSlice(f.Bytes()), int(w.fPO.pos))) + d := encoding.DecWrap(tsdb_enc.NewDecbufRaw(RealByteSlice(f.Bytes()), int(w.fPO.pos))) cnt := w.cntPO current := []byte{} values := []uint32{} @@ -786,7 +786,7 @@ func (w *Writer) writePostingsOffsetTable() error { f.Close() } }() - d := encoding.DecWrap(tsdb_enc.NewDecbufRaw(realByteSlice(f.Bytes()), int(w.fPO.pos))) + d := encoding.DecWrap(tsdb_enc.NewDecbufRaw(RealByteSlice(f.Bytes()), int(w.fPO.pos))) cnt := w.cntPO for d.Err() == nil && cnt > 0 { w.buf1.Reset() @@ -904,7 +904,7 @@ func (w *Writer) writePostingsToTmpFiles() error { // Write out the special all posting. offsets := []uint32{} - d := encoding.DecWrap(tsdb_enc.NewDecbufRaw(realByteSlice(f.Bytes()), int(w.toc.LabelIndices))) + d := encoding.DecWrap(tsdb_enc.NewDecbufRaw(RealByteSlice(f.Bytes()), int(w.toc.LabelIndices))) d.Skip(int(w.toc.Series)) for d.Len() > 0 { d.ConsumePadding() @@ -950,7 +950,7 @@ func (w *Writer) writePostingsToTmpFiles() error { // Label name -> label value -> positions. postings := map[uint32]map[uint32][]uint32{} - d := encoding.DecWrap(tsdb_enc.NewDecbufRaw(realByteSlice(f.Bytes()), int(w.toc.LabelIndices))) + d := encoding.DecWrap(tsdb_enc.NewDecbufRaw(RealByteSlice(f.Bytes()), int(w.toc.LabelIndices))) d.Skip(int(w.toc.Series)) for d.Len() > 0 { d.ConsumePadding() @@ -1166,17 +1166,17 @@ type ByteSlice interface { Range(start, end int) []byte } -type realByteSlice []byte +type RealByteSlice []byte -func (b realByteSlice) Len() int { +func (b RealByteSlice) Len() int { return len(b) } -func (b realByteSlice) Range(start, end int) []byte { +func (b RealByteSlice) Range(start, end int) []byte { return b[start:end] } -func (b realByteSlice) Sub(start, end int) ByteSlice { +func (b RealByteSlice) Sub(start, end int) ByteSlice { return b[start:end] } @@ -1188,7 +1188,7 @@ func NewReader(b ByteSlice) (*Reader, error) { type nopCloser struct{} -func (_ nopCloser) Close() error { return nil } +func (nopCloser) Close() error { return nil } // NewFileReader returns a new index reader against the given index file. func NewFileReader(path string) (*Reader, error) { @@ -1196,7 +1196,7 @@ func NewFileReader(path string) (*Reader, error) { if err != nil { return nil, err } - r, err := newReader(realByteSlice(b), nopCloser{}) + r, err := newReader(RealByteSlice(b), nopCloser{}) if err != nil { return r, err } diff --git a/pkg/storage/stores/tsdb/index/index_test.go b/pkg/storage/stores/tsdb/index/index_test.go index 4508827c3c4ad..0f83903081fe2 100644 --- a/pkg/storage/stores/tsdb/index/index_test.go +++ b/pkg/storage/stores/tsdb/index/index_test.go @@ -465,14 +465,14 @@ func TestPersistence_index_e2e(t *testing.T) { } func TestDecbufUvarintWithInvalidBuffer(t *testing.T) { - b := realByteSlice([]byte{0x81, 0x81, 0x81, 0x81, 0x81, 0x81}) + 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}) + b := RealByteSlice([]byte{0x81, 0x81, 0x81, 0x81, 0x81, 0x81}) _, err := NewReader(b) require.Error(t, err) @@ -509,7 +509,7 @@ func TestSymbols(t *testing.T) { checksum := crc32.Checksum(buf.Get()[symbolsStart+4:], castagnoliTable) buf.PutBE32(checksum) // Check sum at the end. - s, err := NewSymbols(realByteSlice(buf.Get()), FormatV2, symbolsStart) + s, err := NewSymbols(RealByteSlice(buf.Get()), FormatV2, symbolsStart) require.NoError(t, err) // We store only 4 offsets to symbols. From 725abf821faf88db5abfcb84e7f7943a23f997eb Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Tue, 3 May 2022 12:28:28 -0400 Subject: [PATCH 71/85] TSDBFile no longer needs a file descriptor, parses gzip extensions --- pkg/storage/stores/tsdb/identifier.go | 17 ++++ pkg/storage/stores/tsdb/single_file_index.go | 86 ++++++++++++-------- 2 files changed, 68 insertions(+), 35 deletions(-) diff --git a/pkg/storage/stores/tsdb/identifier.go b/pkg/storage/stores/tsdb/identifier.go index 98c653a4ebab2..a574475ce8414 100644 --- a/pkg/storage/stores/tsdb/identifier.go +++ b/pkg/storage/stores/tsdb/identifier.go @@ -57,6 +57,23 @@ 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 { diff --git a/pkg/storage/stores/tsdb/single_file_index.go b/pkg/storage/stores/tsdb/single_file_index.go index 171c23c799d45..ea562f34988b1 100644 --- a/pkg/storage/stores/tsdb/single_file_index.go +++ b/pkg/storage/stores/tsdb/single_file_index.go @@ -1,46 +1,58 @@ package tsdb import ( + "bytes" "context" "io" - "os" - "sync" + "io/ioutil" + "strings" "github.com/prometheus/common/model" "github.com/prometheus/prometheus/model/labels" - "github.com/grafana/dskit/multierror" + "github.com/grafana/loki/pkg/chunkenc" index_shipper "github.com/grafana/loki/pkg/storage/stores/indexshipper/index" "github.com/grafana/loki/pkg/storage/stores/tsdb/index" ) +const ( + gzipSuffix = ".gz" +) + func OpenShippableTSDB(p string) (index_shipper.Index, error) { - id, err := identifierFromPath(p) + var gz bool + trimmed := strings.TrimSuffix(p, gzipSuffix) + if trimmed != p { + gz = true + } + + id, err := identifierFromPath(trimmed) if err != nil { return nil, err } - return NewShippableTSDBFile(id) + return NewShippableTSDBFile(id, gz) } // nolint // TSDBFile is backed by an actual file and implements the indexshipper/index.Index interface type TSDBFile struct { - sync.Mutex - // reuse Identifier for resolving locations Identifier // reuse TSDBIndex for reading Index - // open the read only fd // to sastisfy Reader() and Close() methods - f *os.File + r io.ReadSeeker } -func NewShippableTSDBFile(id Identifier) (*TSDBFile, error) { - idx, err := NewTSDBIndexFromFile(id.Path()) +func NewShippableTSDBFile(id Identifier, gzip bool) (*TSDBFile, error) { + if gzip { + id = newSuffixedIdentifier(id, gzipSuffix) + } + + idx, b, err := NewTSDBIndexFromFile(id.Path(), gzip) if err != nil { return nil, err } @@ -48,33 +60,16 @@ func NewShippableTSDBFile(id Identifier) (*TSDBFile, error) { return &TSDBFile{ Identifier: id, Index: idx, + r: bytes.NewReader(b), }, err } func (f *TSDBFile) Close() error { - f.Lock() - defer f.Unlock() - var errs multierror.MultiError - errs.Add(f.Index.Close()) - if f.f != nil { - errs.Add(f.f.Close()) - f.f = nil - } - return errs.Err() + return f.Index.Close() } func (f *TSDBFile) Reader() (io.ReadSeeker, error) { - f.Lock() - defer f.Unlock() - if f.f == nil { - fd, err := os.Open(f.Path()) - if err != nil { - return nil, err - } - f.f = fd - } - - return f.f, nil + return f.r, nil } // nolint @@ -85,12 +80,33 @@ type TSDBIndex struct { reader IndexReader } -func NewTSDBIndexFromFile(location string) (*TSDBIndex, error) { - reader, err := index.NewFileReader(location) +// Return the index as well as the underlying []byte which isn't exposed as an index +// method but is helpful for building an io.reader for the index shipper +func NewTSDBIndexFromFile(location string, gzip bool) (*TSDBIndex, []byte, error) { + raw, err := ioutil.ReadFile(location) if err != nil { - return nil, err + return nil, nil, err + } + + cleaned := raw + + // decompress if needed + if gzip { + r := chunkenc.Gzip.GetReader(bytes.NewReader(raw)) + defer chunkenc.Gzip.PutReader(r) + + var err error + cleaned, err = io.ReadAll(r) + if err != nil { + return nil, nil, err + } + } + + reader, err := index.NewReader(index.RealByteSlice(cleaned)) + if err != nil { + return nil, nil, err } - return NewTSDBIndex(reader), nil + return NewTSDBIndex(reader), cleaned, nil } func NewTSDBIndex(reader IndexReader) *TSDBIndex { From 8da79b8188ef8643a099c75ed0b877572f72897b Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Tue, 3 May 2022 13:35:29 -0400 Subject: [PATCH 72/85] method signature fixing --- clients/cmd/promtail/promtail-local-config.yaml | 5 +++++ cmd/loki/loki-local-config.yaml | 14 +++++++++++--- pkg/storage/stores/tsdb/compact_test.go | 2 +- pkg/storage/stores/tsdb/manager.go | 2 +- pkg/storage/stores/tsdb/util_test.go | 2 +- 5 files changed, 19 insertions(+), 6 deletions(-) diff --git a/clients/cmd/promtail/promtail-local-config.yaml b/clients/cmd/promtail/promtail-local-config.yaml index 3b9256537ec80..c8951e7af29a7 100644 --- a/clients/cmd/promtail/promtail-local-config.yaml +++ b/clients/cmd/promtail/promtail-local-config.yaml @@ -16,3 +16,8 @@ scrape_configs: labels: job: varlogs __path__: /var/log/*log + - targets: + - localhost + labels: + job: tmplogs + __path__: /tmp/*log \ No newline at end of file diff --git a/cmd/loki/loki-local-config.yaml b/cmd/loki/loki-local-config.yaml index 75b3d3968685f..c09d6853606a2 100644 --- a/cmd/loki/loki-local-config.yaml +++ b/cmd/loki/loki-local-config.yaml @@ -8,7 +8,7 @@ common: path_prefix: /tmp/loki storage: filesystem: - chunks_directory: /tmp/loki/chunks + chunks_directory: /tmp/loki/data rules_directory: /tmp/loki/rules replication_factor: 1 ring: @@ -19,16 +19,24 @@ common: schema_config: configs: - from: 2020-10-24 - store: boltdb-shipper + store: tsdb object_store: filesystem - schema: v11 + schema: v12 index: prefix: index_ period: 24h +storage_config: + tsdb_shipper: + query_ready_num_days: 30 + ruler: alertmanager_url: http://localhost:9093 +ingester: + max_chunk_age: 10s + chunk_idle_period: 10s + # By default, Loki will send anonymous, but uniquely-identifiable usage and configuration # analytics to Grafana Labs. These statistics are sent to https://stats.grafana.org/ # diff --git a/pkg/storage/stores/tsdb/compact_test.go b/pkg/storage/stores/tsdb/compact_test.go index 500569aac3b8c..a7423fe34219c 100644 --- a/pkg/storage/stores/tsdb/compact_test.go +++ b/pkg/storage/stores/tsdb/compact_test.go @@ -368,7 +368,7 @@ func TestCompactor(t *testing.T) { } require.Nil(t, err) - idx, err := NewShippableTSDBFile(out) + idx, err := NewShippableTSDBFile(out, false) require.Nil(t, err) defer idx.Close() diff --git a/pkg/storage/stores/tsdb/manager.go b/pkg/storage/stores/tsdb/manager.go index 47f019931777b..5944e6ea31931 100644 --- a/pkg/storage/stores/tsdb/manager.go +++ b/pkg/storage/stores/tsdb/manager.go @@ -141,7 +141,7 @@ func (m *tsdbManager) BuildFromWALs(t time.Time, ids []WALIdentifier) (err error level.Debug(m.log).Log("msg", "finished building tsdb for period", "pd", p, "dst", dst.Path(), "duration", time.Since(start)) - loaded, err := NewShippableTSDBFile(dst) + loaded, err := NewShippableTSDBFile(dst, false) if err != nil { return err } diff --git a/pkg/storage/stores/tsdb/util_test.go b/pkg/storage/stores/tsdb/util_test.go index 4af6ae90c087b..86b9f87e9c74a 100644 --- a/pkg/storage/stores/tsdb/util_test.go +++ b/pkg/storage/stores/tsdb/util_test.go @@ -34,7 +34,7 @@ func BuildIndex(t *testing.T, dir, tenant string, cases []LoadableSeries) *TSDBF }) require.Nil(t, err) - idx, err := NewShippableTSDBFile(dst) + idx, err := NewShippableTSDBFile(dst, false) require.Nil(t, err) return idx } From bce8c52f415258d68078bbea17daaea507e9b4b6 Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Tue, 3 May 2022 17:02:06 -0400 Subject: [PATCH 73/85] stop masquerading as compressed indices post-download in indexshipper --- .../indexshipper/downloads/index_set.go | 76 +++++++++++++++++-- 1 file changed, 68 insertions(+), 8 deletions(-) diff --git a/pkg/storage/stores/indexshipper/downloads/index_set.go b/pkg/storage/stores/indexshipper/downloads/index_set.go index 65363447b067f..b96bd8d6e3809 100644 --- a/pkg/storage/stores/indexshipper/downloads/index_set.go +++ b/pkg/storage/stores/indexshipper/downloads/index_set.go @@ -7,6 +7,7 @@ import ( "io/ioutil" "os" "path/filepath" + "strings" "sync" "time" @@ -14,6 +15,7 @@ import ( "github.com/go-kit/log/level" "github.com/grafana/dskit/concurrency" + "github.com/grafana/loki/pkg/chunkenc" "github.com/grafana/loki/pkg/storage/chunk/client/util" "github.com/grafana/loki/pkg/storage/stores/indexshipper/index" "github.com/grafana/loki/pkg/storage/stores/shipper/storage" @@ -22,6 +24,10 @@ import ( "github.com/grafana/loki/pkg/util/spanlogger" ) +const ( + gzipExtension = ".gz" +) + type IndexSet interface { Init() error Close() @@ -300,11 +306,12 @@ func (t *indexSet) checkStorageForUpdates(ctx context.Context, lock bool) (toDow } for _, file := range files { - listedDBs[file.Name] = struct{}{} + normalized := strings.TrimSuffix(file.Name, gzipExtension) + listedDBs[normalized] = struct{}{} // Checking whether file was already downloaded, if not, download it. // We do not ever upload files in the object store with the same name but different contents so we do not consider downloading modified files again. - _, ok := t.index[file.Name] + _, ok := t.index[normalized] if !ok { toDownload = append(toDownload, file) } @@ -323,11 +330,65 @@ func (t *indexSet) AwaitReady(ctx context.Context) error { return t.indexMtx.awaitReady(ctx) } -func (t *indexSet) downloadFileFromStorage(ctx context.Context, fileName, folderPathForTable string) error { - return shipper_util.DownloadFileFromStorage(filepath.Join(folderPathForTable, fileName), shipper_util.IsCompressedFile(fileName), - true, shipper_util.LoggerWithFilename(t.logger, fileName), func() (io.ReadCloser, error) { +func (t *indexSet) downloadFileFromStorage(ctx context.Context, fileName, folderPathForTable string) (string, error) { + decompress := shipper_util.IsCompressedFile(fileName) + dst := filepath.Join(folderPathForTable, fileName) + if decompress { + dst = strings.Trim(dst, gzipExtension) + } + return filepath.Base(dst), downloadFileFromStorage( + dst, + decompress, + true, + shipper_util.LoggerWithFilename(t.logger, fileName), + func() (io.ReadCloser, error) { return t.baseIndexSet.GetFile(ctx, t.tableName, t.userID, fileName) - }) + }, + ) +} + +// DownloadFileFromStorage downloads a file from storage to given location. +func downloadFileFromStorage(destination string, decompressFile bool, sync bool, logger log.Logger, getFileFunc shipper_util.GetFileFunc) error { + start := time.Now() + readCloser, err := getFileFunc() + if err != nil { + return err + } + + defer func() { + if err := readCloser.Close(); err != nil { + level.Error(logger).Log("msg", "failed to close read closer", "err", err) + } + }() + + f, err := os.Create(destination) + if err != nil { + return err + } + + defer func() { + if err := f.Close(); err != nil { + level.Warn(logger).Log("msg", "failed to close file", "file", destination) + } + }() + var objectReader io.Reader = readCloser + if decompressFile { + decompressedReader := chunkenc.Gzip.GetReader(readCloser) + defer chunkenc.Gzip.PutReader(decompressedReader) + + objectReader = decompressedReader + } + + _, err = io.Copy(f, objectReader) + if err != nil { + return err + } + + level.Info(logger).Log("msg", "downloaded file", "total_time", time.Since(start)) + if sync { + return f.Sync() + } + return nil } // doConcurrentDownload downloads objects(files) concurrently. It ignores only missing file errors caused by removal of file by compaction. @@ -337,8 +398,7 @@ func (t *indexSet) doConcurrentDownload(ctx context.Context, files []storage.Ind downloadedFilesMtx := sync.Mutex{} err := concurrency.ForEachJob(ctx, len(files), maxDownloadConcurrency, func(ctx context.Context, idx int) error { - fileName := files[idx].Name - err := t.downloadFileFromStorage(ctx, fileName, t.cacheLocation) + fileName, err := t.downloadFileFromStorage(ctx, files[idx].Name, t.cacheLocation) if err != nil { if t.baseIndexSet.IsFileNotFoundErr(err) { level.Info(util_log.Logger).Log("msg", fmt.Sprintf("ignoring missing file %s, possibly removed during compaction", fileName)) From e229160078a31a01600b2ed4b0d0f7255d82234f Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Tue, 3 May 2022 17:57:38 -0400 Subject: [PATCH 74/85] variable bucket regexp --- pkg/storage/stores/indexshipper/downloads/table_manager.go | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/pkg/storage/stores/indexshipper/downloads/table_manager.go b/pkg/storage/stores/indexshipper/downloads/table_manager.go index bb9f78e36e7e9..7cc1282accbbf 100644 --- a/pkg/storage/stores/indexshipper/downloads/table_manager.go +++ b/pkg/storage/stores/indexshipper/downloads/table_manager.go @@ -241,18 +241,19 @@ func (tm *tableManager) ensureQueryReadiness(ctx context.Context) error { return err } - // regex for finding daily tables which have a 5 digit number at the end. + // regexp for finding the trailing index bucket number at the end re, err := regexp.Compile(`[0-9]+$`) if err != nil { return err } for _, tableName := range tables { - if !re.MatchString(tableName) { + match := re.Find([]byte(tableName)) + if match == nil { continue } - tableNumber, err := strconv.ParseInt(tableName, 10, 64) + tableNumber, err := strconv.ParseInt(string(match), 10, 64) if err != nil { return err } From 0b2125ede9aea8f1e2975acfc6f4586ea623b34a Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Wed, 4 May 2022 09:27:38 -0400 Subject: [PATCH 75/85] removes accidental configs committed --- clients/cmd/promtail/promtail-local-config.yaml | 5 ----- cmd/loki/loki-local-config.yaml | 14 +++----------- .../stores/tsdb/fake/index-1-15-dc7af36e.tsdb | Bin 496 -> 0 bytes 3 files changed, 3 insertions(+), 16 deletions(-) delete mode 100644 pkg/storage/stores/tsdb/fake/index-1-15-dc7af36e.tsdb diff --git a/clients/cmd/promtail/promtail-local-config.yaml b/clients/cmd/promtail/promtail-local-config.yaml index c8951e7af29a7..3b9256537ec80 100644 --- a/clients/cmd/promtail/promtail-local-config.yaml +++ b/clients/cmd/promtail/promtail-local-config.yaml @@ -16,8 +16,3 @@ scrape_configs: labels: job: varlogs __path__: /var/log/*log - - targets: - - localhost - labels: - job: tmplogs - __path__: /tmp/*log \ No newline at end of file diff --git a/cmd/loki/loki-local-config.yaml b/cmd/loki/loki-local-config.yaml index c09d6853606a2..75b3d3968685f 100644 --- a/cmd/loki/loki-local-config.yaml +++ b/cmd/loki/loki-local-config.yaml @@ -8,7 +8,7 @@ common: path_prefix: /tmp/loki storage: filesystem: - chunks_directory: /tmp/loki/data + chunks_directory: /tmp/loki/chunks rules_directory: /tmp/loki/rules replication_factor: 1 ring: @@ -19,24 +19,16 @@ common: schema_config: configs: - from: 2020-10-24 - store: tsdb + store: boltdb-shipper object_store: filesystem - schema: v12 + schema: v11 index: prefix: index_ period: 24h -storage_config: - tsdb_shipper: - query_ready_num_days: 30 - ruler: alertmanager_url: http://localhost:9093 -ingester: - max_chunk_age: 10s - chunk_idle_period: 10s - # By default, Loki will send anonymous, but uniquely-identifiable usage and configuration # analytics to Grafana Labs. These statistics are sent to https://stats.grafana.org/ # diff --git a/pkg/storage/stores/tsdb/fake/index-1-15-dc7af36e.tsdb b/pkg/storage/stores/tsdb/fake/index-1-15-dc7af36e.tsdb deleted file mode 100644 index 02cb227674daf71fe52281ee3660af534137469d..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 496 zcmdnB>N*1x0|SE;5VJ8RGA1!6B^I$HC013jB$ZZGF{kC{M|^NH0?I;w3Mb!|*!SW; zRTx>Bm>HNMg3J(_5lSpKqh7uRwhO!POvC**Tf_2AQlf0gNy)UQI-B(Xksi1S7$Fn6Jug4sqq7f2>>z3 zS`cOfV$Mf3DIP$Y1BgN9u}zv`bqrT3VSoj5>9ms# kKyeUYg-{IYP Date: Wed, 4 May 2022 11:12:27 -0400 Subject: [PATCH 76/85] label matcher handling for multitenancy and metricname in tsdb --- pkg/storage/stores/tsdb/manager.go | 21 +++++++++++++++++++++ pkg/storage/stores/tsdb/multitenant.go | 5 ++++- 2 files changed, 25 insertions(+), 1 deletion(-) diff --git a/pkg/storage/stores/tsdb/manager.go b/pkg/storage/stores/tsdb/manager.go index 5944e6ea31931..a72d5feea58e9 100644 --- a/pkg/storage/stores/tsdb/manager.go +++ b/pkg/storage/stores/tsdb/manager.go @@ -211,6 +211,7 @@ func (m *tsdbManager) GetChunkRefs(ctx context.Context, userID string, from, thr if err != nil { return nil, err } + matchers = withoutNameLabel(matchers) return idx.GetChunkRefs(ctx, userID, from, through, res, shard, matchers...) } @@ -219,6 +220,7 @@ func (m *tsdbManager) Series(ctx context.Context, userID string, from, through m if err != nil { return nil, err } + matchers = withoutNameLabel(matchers) return idx.Series(ctx, userID, from, through, res, shard, matchers...) } @@ -227,6 +229,7 @@ func (m *tsdbManager) LabelNames(ctx context.Context, userID string, from, throu if err != nil { return nil, err } + matchers = withoutNameLabel(matchers) return idx.LabelNames(ctx, userID, from, through, matchers...) } @@ -235,5 +238,23 @@ func (m *tsdbManager) LabelValues(ctx context.Context, userID string, from, thro 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 +} diff --git a/pkg/storage/stores/tsdb/multitenant.go b/pkg/storage/stores/tsdb/multitenant.go index fafda2d655f31..5b5be5bec5db7 100644 --- a/pkg/storage/stores/tsdb/multitenant.go +++ b/pkg/storage/stores/tsdb/multitenant.go @@ -19,7 +19,10 @@ func NewMultiTenantIndex(idx Index) *MultiTenantIndex { } func withTenantLabel(userID string, matchers []*labels.Matcher) []*labels.Matcher { - return append(matchers, labels.MustNewMatcher(labels.MatchEqual, TenantLabel, userID)) + cpy := make([]*labels.Matcher, len(matchers)) + copy(cpy, matchers) + cpy = append(cpy, labels.MustNewMatcher(labels.MatchEqual, TenantLabel, userID)) + return cpy } func (m *MultiTenantIndex) Bounds() (model.Time, model.Time) { return m.idx.Bounds() } From a850e420ed474078a0426f3936086c93288a08ee Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Wed, 4 May 2022 12:03:12 -0400 Subject: [PATCH 77/85] explicitly require fingerprint when creating tsdb index --- pkg/storage/stores/tsdb/index/index.go | 17 ++++++++++++----- pkg/storage/stores/tsdb/index/index_test.go | 13 +++++++------ 2 files changed, 19 insertions(+), 11 deletions(-) diff --git a/pkg/storage/stores/tsdb/index/index.go b/pkg/storage/stores/tsdb/index/index.go index ceb4fa4359222..64b987991c83f 100644 --- a/pkg/storage/stores/tsdb/index/index.go +++ b/pkg/storage/stores/tsdb/index/index.go @@ -30,6 +30,7 @@ import ( "unsafe" "github.com/pkg/errors" + "github.com/prometheus/common/model" "github.com/prometheus/prometheus/model/labels" "github.com/prometheus/prometheus/storage" tsdb_enc "github.com/prometheus/prometheus/tsdb/encoding" @@ -133,8 +134,9 @@ type Writer struct { fingerprintOffsets FingerprintOffsets // Hold last series to validate that clients insert new series in order. - lastSeries labels.Labels - lastRef storage.SeriesRef + lastSeries labels.Labels + lastSeriesHash uint64 + lastRef storage.SeriesRef crc32 hash.Hash @@ -435,13 +437,17 @@ func (w *Writer) writeMeta() error { } // AddSeries adds the series one at a time along with its chunks. -func (w *Writer) AddSeries(ref storage.SeriesRef, lset labels.Labels, chunks ...ChunkMeta) error { +// Requires a specific fingerprint to be passed in the case where the "desired" +// fingerprint differs from what labels.Hash() produces. For example, +// multitenant TSDBs embed a tenant label, but the actual series has no such +// label and so the derived fingerprint differs. +func (w *Writer) AddSeries(ref storage.SeriesRef, lset labels.Labels, fp model.Fingerprint, chunks ...ChunkMeta) error { if err := w.ensureStage(idxStageSeries); err != nil { return err } - labelHash := lset.Hash() - lastHash := w.lastSeries.Hash() + labelHash := uint64(fp) + lastHash := w.lastSeriesHash // Ensure series are sorted by the priorities: [`hash(labels)`, `labels`] if (labelHash < lastHash && len(w.lastSeries) > 0) || labelHash == lastHash && labels.Compare(lset, w.lastSeries) < 0 { return errors.Errorf("out-of-order series added with label set %q", lset) @@ -529,6 +535,7 @@ func (w *Writer) AddSeries(ref storage.SeriesRef, lset labels.Labels, chunks ... } w.lastSeries = append(w.lastSeries[:0], lset...) + w.lastSeriesHash = labelHash w.lastRef = ref if ref%fingerprintInterval == 0 { diff --git a/pkg/storage/stores/tsdb/index/index_test.go b/pkg/storage/stores/tsdb/index/index_test.go index 0f83903081fe2..962e5dfb5c4fb 100644 --- a/pkg/storage/stores/tsdb/index/index_test.go +++ b/pkg/storage/stores/tsdb/index/index_test.go @@ -28,6 +28,7 @@ import ( "github.com/stretchr/testify/require" "go.uber.org/goleak" + "github.com/prometheus/common/model" "github.com/prometheus/prometheus/model/labels" "github.com/prometheus/prometheus/storage" "github.com/prometheus/prometheus/tsdb/encoding" @@ -168,10 +169,10 @@ func TestIndexRW_Postings(t *testing.T) { // 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.AddSeries(1, series[0], model.Fingerprint(series[0].Hash()))) + require.NoError(t, iw.AddSeries(2, series[1], model.Fingerprint(series[1].Hash()))) + require.NoError(t, iw.AddSeries(3, series[2], model.Fingerprint(series[2].Hash()))) + require.NoError(t, iw.AddSeries(4, series[3], model.Fingerprint(series[3].Hash()))) require.NoError(t, iw.Close()) @@ -257,7 +258,7 @@ func TestPostingsMany(t *testing.T) { }) for i, s := range series { - require.NoError(t, iw.AddSeries(storage.SeriesRef(i), s)) + require.NoError(t, iw.AddSeries(storage.SeriesRef(i), s, model.Fingerprint(s.Hash()))) } require.NoError(t, iw.Close()) @@ -384,7 +385,7 @@ func TestPersistence_index_e2e(t *testing.T) { mi := newMockIndex() for i, s := range input { - err = iw.AddSeries(storage.SeriesRef(i), s.labels, s.chunks...) + err = iw.AddSeries(storage.SeriesRef(i), s.labels, model.Fingerprint(s.labels.Hash()), s.chunks...) require.NoError(t, err) require.NoError(t, mi.AddSeries(storage.SeriesRef(i), s.labels, s.chunks...)) From 14dcb1d6b714f80f16f7d02bd28ab836633bd175 Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Wed, 4 May 2022 12:05:22 -0400 Subject: [PATCH 78/85] only add tenant label when creating multitenant tsdb write fingerprints without synthetic tenant label strip out tenant labels from queries --- pkg/storage/stores/tsdb/builder.go | 6 +++-- pkg/storage/stores/tsdb/compact.go | 2 +- pkg/storage/stores/tsdb/head_manager.go | 7 ------ pkg/storage/stores/tsdb/head_manager_test.go | 9 +++----- pkg/storage/stores/tsdb/manager.go | 11 ++++++++- pkg/storage/stores/tsdb/multitenant.go | 24 +++++++++++++++++++- pkg/storage/stores/tsdb/querier_test.go | 2 +- pkg/storage/stores/tsdb/util_test.go | 2 +- 8 files changed, 43 insertions(+), 20 deletions(-) diff --git a/pkg/storage/stores/tsdb/builder.go b/pkg/storage/stores/tsdb/builder.go index 22bf3a64839db..3bfb2802ceda3 100644 --- a/pkg/storage/stores/tsdb/builder.go +++ b/pkg/storage/stores/tsdb/builder.go @@ -27,6 +27,7 @@ type Builder struct { type stream struct { labels labels.Labels + fp model.Fingerprint chunks index.ChunkMetas } @@ -34,12 +35,13 @@ func NewBuilder() *Builder { return &Builder{streams: make(map[string]*stream)} } -func (b *Builder) AddSeries(ls labels.Labels, chks []index.ChunkMeta) { +func (b *Builder) AddSeries(ls labels.Labels, fp model.Fingerprint, chks []index.ChunkMeta) { id := ls.String() s, ok := b.streams[id] if !ok { s = &stream{ labels: ls, + fp: fp, } b.streams[id] = s } @@ -111,7 +113,7 @@ func (b *Builder) Build( // Add series for i, s := range streams { - if err := writer.AddSeries(storage.SeriesRef(i), s.labels, s.chunks.Finalize()...); err != nil { + if err := writer.AddSeries(storage.SeriesRef(i), s.labels, s.fp, s.chunks.Finalize()...); err != nil { return id, err } } diff --git a/pkg/storage/stores/tsdb/compact.go b/pkg/storage/stores/tsdb/compact.go index f19c29fee29f6..7b4175d6af32d 100644 --- a/pkg/storage/stores/tsdb/compact.go +++ b/pkg/storage/stores/tsdb/compact.go @@ -61,7 +61,7 @@ func (c *Compactor) Compact(ctx context.Context, indices ...*TSDBIndex) (res Ide nil, func(ls labels.Labels, _ model.Fingerprint, chks []index.ChunkMeta) { // AddSeries copies chks into it's own slice - b.AddSeries(ls.Copy(), chks) + b.AddSeries(ls.Copy(), model.Fingerprint(ls.Hash()), chks) }, labels.MustNewMatcher(labels.MatchEqual, "", ""), ); err != nil { diff --git a/pkg/storage/stores/tsdb/head_manager.go b/pkg/storage/stores/tsdb/head_manager.go index 2f69caf2ca2d9..83c7f0b4e25af 100644 --- a/pkg/storage/stores/tsdb/head_manager.go +++ b/pkg/storage/stores/tsdb/head_manager.go @@ -30,11 +30,6 @@ type period time.Duration const defaultRotationPeriod = period(15 * time.Minute) -// 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__" - // Do not specify without bit shifting. This allows us to // do shard index calcuations via bitwise & rather than modulos. const defaultHeadManagerStripeSize = 1 << 7 @@ -132,8 +127,6 @@ func (m *HeadManager) Append(userID string, ls labels.Labels, chks index.ChunkMe labelsBuilder := labels.NewBuilder(ls) // TSDB doesnt need the __name__="log" convention the old chunk store index used. labelsBuilder.Del("__name__") - // userIDs are also included until compaction occurs in tsdb - labelsBuilder.Set(TenantLabel, userID) metric := labelsBuilder.Labels() m.mtx.RLock() diff --git a/pkg/storage/stores/tsdb/head_manager_test.go b/pkg/storage/stores/tsdb/head_manager_test.go index 3756284ce68be..359b0b093b863 100644 --- a/pkg/storage/stores/tsdb/head_manager_test.go +++ b/pkg/storage/stores/tsdb/head_manager_test.go @@ -239,13 +239,11 @@ func Test_HeadManager_Lifecycle(t *testing.T) { // Write old WALs for i, c := range cases { - lbls := labels.NewBuilder(c.Labels) - lbls.Set(TenantLabel, c.User) require.Nil(t, w.Log(&WALRecord{ UserID: c.User, Series: record.RefSeries{ Ref: chunks.HeadSeriesRef(i), - Labels: lbls.Labels(), + Labels: c.Labels, }, Chks: ChunkMetasRecord{ Chks: c.Chunks, @@ -271,7 +269,7 @@ func Test_HeadManager_Lifecycle(t *testing.T) { lbls := labels.NewBuilder(c.Labels) lbls.Set(TenantLabel, c.User) - require.Equal(t, chunkMetasToChunkRefs(c.User, lbls.Labels().Hash(), c.Chunks), refs) + require.Equal(t, chunkMetasToChunkRefs(c.User, c.Labels.Hash(), c.Chunks), refs) } // Add data @@ -306,7 +304,6 @@ func Test_HeadManager_Lifecycle(t *testing.T) { lbls := labels.NewBuilder(c.Labels) lbls.Set(TenantLabel, c.User) - require.Equal(t, chunkMetasToChunkRefs(c.User, lbls.Labels().Hash(), c.Chunks), refs) + require.Equal(t, chunkMetasToChunkRefs(c.User, c.Labels.Hash(), c.Chunks), refs) } - } diff --git a/pkg/storage/stores/tsdb/manager.go b/pkg/storage/stores/tsdb/manager.go index a72d5feea58e9..8a931a1f8a725 100644 --- a/pkg/storage/stores/tsdb/manager.go +++ b/pkg/storage/stores/tsdb/manager.go @@ -94,6 +94,11 @@ func (m *tsdbManager) BuildFromWALs(t time.Time, ids []WALIdentifier) (err error } } + // 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] @@ -103,7 +108,10 @@ func (m *tsdbManager) BuildFromWALs(t time.Time, ids []WALIdentifier) (err error } b.AddSeries( - ls, + withTenant, + // use the fingerprint without the added tenant label + // so queries route to the chunks which actually exist. + model.Fingerprint(ls.Hash()), matchingChks, ) } @@ -256,5 +264,6 @@ func withoutNameLabel(matchers []*labels.Matcher) []*labels.Matcher { } dst = append(dst, m) } + return dst } diff --git a/pkg/storage/stores/tsdb/multitenant.go b/pkg/storage/stores/tsdb/multitenant.go index 5b5be5bec5db7..87d8c1ec9383d 100644 --- a/pkg/storage/stores/tsdb/multitenant.go +++ b/pkg/storage/stores/tsdb/multitenant.go @@ -8,6 +8,11 @@ import ( "github.com/prometheus/prometheus/model/labels" ) +// 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 { @@ -25,6 +30,16 @@ func withTenantLabel(userID string, matchers []*labels.Matcher) []*labels.Matche 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) Close() error { return m.idx.Close() } @@ -34,7 +49,14 @@ func (m *MultiTenantIndex) GetChunkRefs(ctx context.Context, userID string, from } func (m *MultiTenantIndex) Series(ctx context.Context, userID string, from, through model.Time, res []Series, shard *index.ShardAnnotation, matchers ...*labels.Matcher) ([]Series, error) { - return m.idx.Series(ctx, userID, from, through, res, shard, withTenantLabel(userID, matchers)...) + 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) { diff --git a/pkg/storage/stores/tsdb/querier_test.go b/pkg/storage/stores/tsdb/querier_test.go index b2bf685aaa448..3aca71057d3d1 100644 --- a/pkg/storage/stores/tsdb/querier_test.go +++ b/pkg/storage/stores/tsdb/querier_test.go @@ -86,7 +86,7 @@ func TestQueryIndex(t *testing.T) { }, } for _, s := range cases { - b.AddSeries(s.labels, s.chunks) + b.AddSeries(s.labels, model.Fingerprint(s.labels.Hash()), s.chunks) } dst, err := b.Build(context.Background(), dir, func(from, through model.Time, checksum uint32) Identifier { diff --git a/pkg/storage/stores/tsdb/util_test.go b/pkg/storage/stores/tsdb/util_test.go index 86b9f87e9c74a..4b95d0c381a7c 100644 --- a/pkg/storage/stores/tsdb/util_test.go +++ b/pkg/storage/stores/tsdb/util_test.go @@ -20,7 +20,7 @@ func BuildIndex(t *testing.T, dir, tenant string, cases []LoadableSeries) *TSDBF b := NewBuilder() for _, s := range cases { - b.AddSeries(s.Labels, s.Chunks) + b.AddSeries(s.Labels, model.Fingerprint(s.Labels.Hash()), s.Chunks) } dst, err := b.Build(context.Background(), dir, func(from, through model.Time, checksum uint32) Identifier { From a853035941fcf88a9fe4d70f5dbb073e5acd9596 Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Wed, 4 May 2022 12:25:05 -0400 Subject: [PATCH 79/85] linting + unused removal --- pkg/storage/stores/tsdb/chunkwriter.go | 7 ++-- pkg/storage/stores/tsdb/head_wal_test.go | 2 +- pkg/storage/stores/tsdb/identifier.go | 3 +- pkg/storage/stores/tsdb/index.go | 43 ------------------------ pkg/storage/stores/tsdb/index_client.go | 5 +-- tools/tsdb/tsdb-map/main.go | 6 ++-- 6 files changed, 14 insertions(+), 52 deletions(-) diff --git a/pkg/storage/stores/tsdb/chunkwriter.go b/pkg/storage/stores/tsdb/chunkwriter.go index 2c79c095ea714..5b2da8f0db4c5 100644 --- a/pkg/storage/stores/tsdb/chunkwriter.go +++ b/pkg/storage/stores/tsdb/chunkwriter.go @@ -4,15 +4,16 @@ 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" - "github.com/pkg/errors" - "github.com/prometheus/common/model" - "github.com/prometheus/prometheus/model/labels" ) type IndexWriter interface { diff --git a/pkg/storage/stores/tsdb/head_wal_test.go b/pkg/storage/stores/tsdb/head_wal_test.go index 7dae49c9ca850..efb2323582188 100644 --- a/pkg/storage/stores/tsdb/head_wal_test.go +++ b/pkg/storage/stores/tsdb/head_wal_test.go @@ -4,7 +4,7 @@ import ( "testing" "time" - "github.com/go-kit/kit/log" + "github.com/go-kit/log" "github.com/grafana/loki/pkg/storage/stores/tsdb/index" "github.com/prometheus/prometheus/tsdb/chunks" "github.com/prometheus/prometheus/tsdb/record" diff --git a/pkg/storage/stores/tsdb/identifier.go b/pkg/storage/stores/tsdb/identifier.go index a574475ce8414..14f9cf69ec59d 100644 --- a/pkg/storage/stores/tsdb/identifier.go +++ b/pkg/storage/stores/tsdb/identifier.go @@ -8,8 +8,9 @@ import ( "strings" "time" - "github.com/grafana/loki/pkg/storage/stores/tsdb/index" "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) diff --git a/pkg/storage/stores/tsdb/index.go b/pkg/storage/stores/tsdb/index.go index ae471e65a326d..a6cc7cd12ede3 100644 --- a/pkg/storage/stores/tsdb/index.go +++ b/pkg/storage/stores/tsdb/index.go @@ -2,7 +2,6 @@ package tsdb import ( "context" - "sync" "github.com/prometheus/common/model" "github.com/prometheus/prometheus/model/labels" @@ -69,45 +68,3 @@ func (NoopIndex) LabelNames(ctx context.Context, userID string, from, through mo func (NoopIndex) LabelValues(ctx context.Context, userID string, from, through model.Time, name string, matchers ...*labels.Matcher) ([]string, error) { return nil, nil } - -// LockedIndex wraps an index+RWMutex and only calls index methods under rlock -type LockedIndex struct { - mtx *sync.RWMutex - idx Index -} - -func NewLockedMutex(mtx *sync.RWMutex, idx Index) *LockedIndex { - return &LockedIndex{ - mtx: mtx, - idx: idx, - } -} - -func (i *LockedIndex) Close() error { return i.Close() } -func (i *LockedIndex) Bounds() (from, through model.Time) { - i.mtx.RLock() - defer i.mtx.RUnlock() - return i.idx.Bounds() -} -func (i *LockedIndex) GetChunkRefs(ctx context.Context, userID string, from, through model.Time, res []ChunkRef, shard *index.ShardAnnotation, matchers ...*labels.Matcher) ([]ChunkRef, error) { - i.mtx.RLock() - defer i.mtx.RUnlock() - return i.idx.GetChunkRefs(ctx, userID, from, through, res, shard, matchers...) -} - -func (i *LockedIndex) Series(ctx context.Context, userID string, from, through model.Time, res []Series, shard *index.ShardAnnotation, matchers ...*labels.Matcher) ([]Series, error) { - i.mtx.RLock() - defer i.mtx.RUnlock() - return i.idx.Series(ctx, userID, from, through, res, shard, matchers...) -} -func (i *LockedIndex) LabelNames(ctx context.Context, userID string, from, through model.Time, matchers ...*labels.Matcher) ([]string, error) { - i.mtx.RLock() - defer i.mtx.RUnlock() - return i.idx.LabelNames(ctx, userID, from, through, matchers...) -} -func (i *LockedIndex) LabelValues(ctx context.Context, userID string, from, through model.Time, name string, matchers ...*labels.Matcher) ([]string, error) { - i.mtx.RLock() - defer i.mtx.RUnlock() - return i.idx.LabelValues(ctx, userID, from, through, name, matchers...) - -} diff --git a/pkg/storage/stores/tsdb/index_client.go b/pkg/storage/stores/tsdb/index_client.go index 69d0cc8dd44a7..d92050c47a3cf 100644 --- a/pkg/storage/stores/tsdb/index_client.go +++ b/pkg/storage/stores/tsdb/index_client.go @@ -3,13 +3,14 @@ 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" - "github.com/prometheus/common/model" - "github.com/prometheus/prometheus/model/labels" ) // implements stores.Index diff --git a/tools/tsdb/tsdb-map/main.go b/tools/tsdb/tsdb-map/main.go index a648a746ded8d..8dc002d26fa23 100644 --- a/tools/tsdb/tsdb-map/main.go +++ b/tools/tsdb/tsdb-map/main.go @@ -13,7 +13,9 @@ import ( "github.com/grafana/loki/pkg/storage/config" "github.com/grafana/loki/pkg/storage/stores/shipper/compactor/retention" shipper_util "github.com/grafana/loki/pkg/storage/stores/shipper/util" + "github.com/grafana/loki/pkg/storage/stores/tsdb" "github.com/grafana/loki/pkg/storage/stores/tsdb/index" + "github.com/prometheus/common/model" ) var ( @@ -64,7 +66,7 @@ func main() { panic(err) } - builder := index.NewBuilder() + builder := tsdb.NewBuilder() log.Println("Loading index into memory") @@ -80,7 +82,7 @@ func main() { return it.Err() } entry := it.Entry() - builder.AddSeries(entry.Labels, []index.ChunkMeta{{ + builder.AddSeries(entry.Labels, model.Fingerprint(entry.Labels.Hash()), []index.ChunkMeta{{ Checksum: extractChecksumFromChunkID(entry.ChunkID), MinTime: int64(entry.From), MaxTime: int64(entry.Through), From 48712282820c5a6f6de366cec1b9a7bcf4681961 Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Wed, 4 May 2022 13:01:37 -0400 Subject: [PATCH 80/85] more linting :( --- pkg/storage/stores/tsdb/lazy_index.go | 3 ++- pkg/storage/stores/tsdb/manager.go | 7 ++++--- pkg/storage/stores/tsdb/multi_file_index.go | 2 +- tools/tsdb/tsdb-map/main.go | 4 +++- 4 files changed, 10 insertions(+), 6 deletions(-) diff --git a/pkg/storage/stores/tsdb/lazy_index.go b/pkg/storage/stores/tsdb/lazy_index.go index ecc3f60fbc2fa..c8c2515c7f852 100644 --- a/pkg/storage/stores/tsdb/lazy_index.go +++ b/pkg/storage/stores/tsdb/lazy_index.go @@ -3,9 +3,10 @@ package tsdb import ( "context" - "github.com/grafana/loki/pkg/storage/stores/tsdb/index" "github.com/prometheus/common/model" "github.com/prometheus/prometheus/model/labels" + + "github.com/grafana/loki/pkg/storage/stores/tsdb/index" ) // Index adapter for a function which returns an index when queried. diff --git a/pkg/storage/stores/tsdb/manager.go b/pkg/storage/stores/tsdb/manager.go index 8a931a1f8a725..3f0df85976fa4 100644 --- a/pkg/storage/stores/tsdb/manager.go +++ b/pkg/storage/stores/tsdb/manager.go @@ -10,12 +10,13 @@ import ( "github.com/go-kit/log" "github.com/go-kit/log/level" - "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" "github.com/pkg/errors" "github.com/prometheus/common/model" "github.com/prometheus/prometheus/model/labels" + + "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 diff --git a/pkg/storage/stores/tsdb/multi_file_index.go b/pkg/storage/stores/tsdb/multi_file_index.go index 1cf2daa32ccfc..fba710b9b747f 100644 --- a/pkg/storage/stores/tsdb/multi_file_index.go +++ b/pkg/storage/stores/tsdb/multi_file_index.go @@ -4,11 +4,11 @@ import ( "context" "errors" + "github.com/grafana/dskit/multierror" "github.com/prometheus/common/model" "github.com/prometheus/prometheus/model/labels" "golang.org/x/sync/errgroup" - "github.com/grafana/dskit/multierror" "github.com/grafana/loki/pkg/storage/stores/tsdb/index" ) diff --git a/tools/tsdb/tsdb-map/main.go b/tools/tsdb/tsdb-map/main.go index 8dc002d26fa23..385530f9302b3 100644 --- a/tools/tsdb/tsdb-map/main.go +++ b/tools/tsdb/tsdb-map/main.go @@ -97,7 +97,9 @@ func main() { } log.Println("writing index") - if _, err := builder.Build(context.Background(), *dest, "fake"); err != nil { + if _, err := builder.Build(context.Background(), *dest, func(from, through model.Time, checksum uint32) tsdb.Identifier { + panic("todo") + }); err != nil { panic(err) } } From 0a0aed825ff50535db07d2b1fafd3394e82a9b4a Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Wed, 4 May 2022 13:08:49 -0400 Subject: [PATCH 81/85] goimports --- pkg/storage/stores/tsdb/head_manager_test.go | 5 +++-- pkg/storage/stores/tsdb/head_wal_test.go | 3 ++- pkg/storage/stores/tsdb/multitenant.go | 3 ++- tools/tsdb/tsdb-map/main.go | 2 +- 4 files changed, 8 insertions(+), 5 deletions(-) diff --git a/pkg/storage/stores/tsdb/head_manager_test.go b/pkg/storage/stores/tsdb/head_manager_test.go index 359b0b093b863..0b032e9e4fcaf 100644 --- a/pkg/storage/stores/tsdb/head_manager_test.go +++ b/pkg/storage/stores/tsdb/head_manager_test.go @@ -7,13 +7,14 @@ import ( "time" "github.com/go-kit/log" - "github.com/grafana/loki/pkg/storage/chunk/client/util" - "github.com/grafana/loki/pkg/storage/stores/tsdb/index" "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 } diff --git a/pkg/storage/stores/tsdb/head_wal_test.go b/pkg/storage/stores/tsdb/head_wal_test.go index efb2323582188..258bf4472e037 100644 --- a/pkg/storage/stores/tsdb/head_wal_test.go +++ b/pkg/storage/stores/tsdb/head_wal_test.go @@ -5,10 +5,11 @@ import ( "time" "github.com/go-kit/log" - "github.com/grafana/loki/pkg/storage/stores/tsdb/index" "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) { diff --git a/pkg/storage/stores/tsdb/multitenant.go b/pkg/storage/stores/tsdb/multitenant.go index 87d8c1ec9383d..9a55e5c8e35ae 100644 --- a/pkg/storage/stores/tsdb/multitenant.go +++ b/pkg/storage/stores/tsdb/multitenant.go @@ -3,9 +3,10 @@ package tsdb import ( "context" - "github.com/grafana/loki/pkg/storage/stores/tsdb/index" "github.com/prometheus/common/model" "github.com/prometheus/prometheus/model/labels" + + "github.com/grafana/loki/pkg/storage/stores/tsdb/index" ) // TenantLabel is part of the reserved label namespace (__ prefix) diff --git a/tools/tsdb/tsdb-map/main.go b/tools/tsdb/tsdb-map/main.go index 385530f9302b3..308afff92adad 100644 --- a/tools/tsdb/tsdb-map/main.go +++ b/tools/tsdb/tsdb-map/main.go @@ -7,6 +7,7 @@ import ( "log" "strconv" + "github.com/prometheus/common/model" "go.etcd.io/bbolt" "gopkg.in/yaml.v2" @@ -15,7 +16,6 @@ import ( shipper_util "github.com/grafana/loki/pkg/storage/stores/shipper/util" "github.com/grafana/loki/pkg/storage/stores/tsdb" "github.com/grafana/loki/pkg/storage/stores/tsdb/index" - "github.com/prometheus/common/model" ) var ( From 74ccc091a3acbc74bf178f0e0bb5b239edf5338f Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Wed, 4 May 2022 13:27:38 -0400 Subject: [PATCH 82/85] removes uploadername from indexshipper --- pkg/storage/store.go | 1 + pkg/storage/stores/indexshipper/shipper.go | 2 -- pkg/storage/stores/indexshipper/uploads/index_set.go | 1 - pkg/storage/stores/indexshipper/uploads/table.go | 4 +--- pkg/storage/stores/indexshipper/uploads/table_manager.go | 3 +-- pkg/storage/stores/indexshipper/uploads/table_manager_test.go | 1 - pkg/storage/stores/indexshipper/uploads/table_test.go | 3 +-- 7 files changed, 4 insertions(+), 11 deletions(-) diff --git a/pkg/storage/store.go b/pkg/storage/store.go index c9252205614ea..c13f4f9f0a660 100644 --- a/pkg/storage/store.go +++ b/pkg/storage/store.go @@ -214,6 +214,7 @@ func (s *store) storeForPeriod(p config.PeriodConfig, chunkClient client.Client, if err != nil { return nil, nil, nil, err } + shpr, err := indexshipper.NewIndexShipper( s.cfg.TSDBShipperConfig, objectClient, diff --git a/pkg/storage/stores/indexshipper/shipper.go b/pkg/storage/stores/indexshipper/shipper.go index fec4ac12e0e74..fd433c424be29 100644 --- a/pkg/storage/stores/indexshipper/shipper.go +++ b/pkg/storage/stores/indexshipper/shipper.go @@ -55,7 +55,6 @@ type Config struct { ResyncInterval time.Duration `yaml:"resync_interval"` QueryReadyNumDays int `yaml:"query_ready_num_days"` - UploaderName string IngesterName string Mode Mode IngesterDBRetainPeriod time.Duration @@ -104,7 +103,6 @@ func (s *indexShipper) init(storageClient client.ObjectClient, limits downloads. if s.cfg.Mode != ModeReadOnly { cfg := uploads.Config{ - Uploader: s.cfg.UploaderName, UploadInterval: UploadInterval, DBRetainPeriod: s.cfg.IngesterDBRetainPeriod, } diff --git a/pkg/storage/stores/indexshipper/uploads/index_set.go b/pkg/storage/stores/indexshipper/uploads/index_set.go index e636652dfc29b..e99181169751a 100644 --- a/pkg/storage/stores/indexshipper/uploads/index_set.go +++ b/pkg/storage/stores/indexshipper/uploads/index_set.go @@ -31,7 +31,6 @@ type indexSet struct { storageIndexSet storage.IndexSet tableName, userID string logger log.Logger - uploader string index map[string]index.Index indexMtx sync.RWMutex diff --git a/pkg/storage/stores/indexshipper/uploads/table.go b/pkg/storage/stores/indexshipper/uploads/table.go index 50cdc077c460b..f88283581eb9d 100644 --- a/pkg/storage/stores/indexshipper/uploads/table.go +++ b/pkg/storage/stores/indexshipper/uploads/table.go @@ -32,7 +32,6 @@ type Table interface { // All the public methods are concurrency safe and take care of mutexes to avoid any data race. type table struct { name string - uploader string baseUserIndexSet, baseCommonIndexSet storage.IndexSet logger log.Logger @@ -41,10 +40,9 @@ type table struct { } // NewTable create a new table instance. -func NewTable(name, uploader string, storageClient storage.Client) Table { +func NewTable(name string, storageClient storage.Client) Table { return &table{ name: name, - uploader: uploader, baseUserIndexSet: storage.NewIndexSet(storageClient, true), baseCommonIndexSet: storage.NewIndexSet(storageClient, false), logger: log.With(util_log.Logger, "table-name", name), diff --git a/pkg/storage/stores/indexshipper/uploads/table_manager.go b/pkg/storage/stores/indexshipper/uploads/table_manager.go index 285f7e6bd31f8..ec99af79e2858 100644 --- a/pkg/storage/stores/indexshipper/uploads/table_manager.go +++ b/pkg/storage/stores/indexshipper/uploads/table_manager.go @@ -13,7 +13,6 @@ import ( ) type Config struct { - Uploader string UploadInterval time.Duration DBRetainPeriod time.Duration } @@ -101,7 +100,7 @@ func (tm *tableManager) getOrCreateTable(tableName string) Table { table, ok = tm.tables[tableName] if !ok { - table = NewTable(tableName, tm.cfg.Uploader, tm.storageClient) + table = NewTable(tableName, tm.storageClient) tm.tables[tableName] = table } } diff --git a/pkg/storage/stores/indexshipper/uploads/table_manager_test.go b/pkg/storage/stores/indexshipper/uploads/table_manager_test.go index 71267fa3e3501..f1f31263865e0 100644 --- a/pkg/storage/stores/indexshipper/uploads/table_manager_test.go +++ b/pkg/storage/stores/indexshipper/uploads/table_manager_test.go @@ -30,7 +30,6 @@ func buildTestTableManager(t *testing.T, testDir string) (TableManager, stopFunc storageClient := buildTestStorageClient(t, testDir) cfg := Config{ - Uploader: "test-table-manager", UploadInterval: time.Hour, } tm, err := NewTableManager(cfg, storageClient) diff --git a/pkg/storage/stores/indexshipper/uploads/table_test.go b/pkg/storage/stores/indexshipper/uploads/table_test.go index b124a7ac51e65..c518cf29c3c4a 100644 --- a/pkg/storage/stores/indexshipper/uploads/table_test.go +++ b/pkg/storage/stores/indexshipper/uploads/table_test.go @@ -13,13 +13,12 @@ import ( const ( testTableName = "test-table" - uploader = "test-uploader" ) func TestTable(t *testing.T) { tempDir := t.TempDir() storageClient := buildTestStorageClient(t, tempDir) - testTable := NewTable(testTableName, uploader, storageClient) + testTable := NewTable(testTableName, storageClient) defer testTable.Stop() for userIdx := 0; userIdx < 2; userIdx++ { From 65aac1870f2e2440b9f23bc1268835c769fec45b Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Wed, 4 May 2022 15:44:24 -0400 Subject: [PATCH 83/85] maxuint32 for arm32 builds --- pkg/storage/stores/tsdb/index/index.go | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/pkg/storage/stores/tsdb/index/index.go b/pkg/storage/stores/tsdb/index/index.go index 64b987991c83f..9e90461a8bd17 100644 --- a/pkg/storage/stores/tsdb/index/index.go +++ b/pkg/storage/stores/tsdb/index/index.go @@ -854,7 +854,9 @@ func (w *Writer) writeFingerprintOffsetsTable() error { // write length ln := w.buf1.Len() - if ln > math.MaxUint32 { + // TODO(owen-d): can remove the uint32 cast in the future + // Had to uint32 wrap these for arm32 builds, which we'll remove in the future. + if uint32(ln) > uint32(math.MaxUint32) { return errors.Errorf("fingerprint offset size exceeds 4 bytes: %d", ln) } From eafaf1a0b76f13ed31b2f58bce97c7f90edd0871 Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Wed, 4 May 2022 17:09:17 -0400 Subject: [PATCH 84/85] tsdb chunk filterer support --- pkg/storage/stores/tsdb/compact.go | 1 + pkg/storage/stores/tsdb/head_manager.go | 24 ++++++++++++------ pkg/storage/stores/tsdb/index.go | 4 +++ pkg/storage/stores/tsdb/index_client.go | 3 +-- pkg/storage/stores/tsdb/lazy_index.go | 9 +++++++ pkg/storage/stores/tsdb/manager.go | 18 ++++++++++++-- pkg/storage/stores/tsdb/multi_file_index.go | 7 ++++++ pkg/storage/stores/tsdb/multitenant.go | 5 ++++ pkg/storage/stores/tsdb/single_file_index.go | 26 ++++++++++++++++---- 9 files changed, 81 insertions(+), 16 deletions(-) diff --git a/pkg/storage/stores/tsdb/compact.go b/pkg/storage/stores/tsdb/compact.go index 7b4175d6af32d..cb2db193c95fd 100644 --- a/pkg/storage/stores/tsdb/compact.go +++ b/pkg/storage/stores/tsdb/compact.go @@ -58,6 +58,7 @@ func (c *Compactor) Compact(ctx context.Context, indices ...*TSDBIndex) (res Ide return nil, fmt.Errorf("expected tsdb index to compact, found :%T", idx) } if err := casted.forSeries( + ctx, nil, func(ls labels.Labels, _ model.Fingerprint, chks []index.ChunkMeta) { // AddSeries copies chks into it's own slice diff --git a/pkg/storage/stores/tsdb/head_manager.go b/pkg/storage/stores/tsdb/head_manager.go index 83c7f0b4e25af..93bfe015086f2 100644 --- a/pkg/storage/stores/tsdb/head_manager.go +++ b/pkg/storage/stores/tsdb/head_manager.go @@ -21,6 +21,7 @@ import ( "github.com/prometheus/prometheus/tsdb/record" "go.uber.org/atomic" + "github.com/grafana/loki/pkg/storage/chunk" "github.com/grafana/loki/pkg/storage/chunk/client/util" "github.com/grafana/loki/pkg/storage/stores/tsdb/index" "github.com/grafana/loki/pkg/util/wal" @@ -479,12 +480,13 @@ func parseWALPath(p string) (id WALIdentifier, ok bool) { type tenantHeads struct { mint, maxt atomic.Int64 // easy lookup for Bounds() impl - start time.Time - shards int - locks []sync.RWMutex - tenants []map[string]*Head - log log.Logger - metrics *Metrics + start time.Time + shards int + locks []sync.RWMutex + tenants []map[string]*Head + log log.Logger + chunkFilter chunk.RequestChunkFilterer + metrics *Metrics } func newTenantHeads(start time.Time, shards int, metrics *Metrics, logger log.Logger) *tenantHeads { @@ -561,6 +563,10 @@ func (t *tenantHeads) shardForTenant(userID string) uint64 { func (t *tenantHeads) Close() error { return nil } +func (t *tenantHeads) SetChunkFilterer(chunkFilter chunk.RequestChunkFilterer) { + t.chunkFilter = chunkFilter +} + func (t *tenantHeads) Bounds() (model.Time, model.Time) { return model.Time(t.mint.Load()), model.Time(t.maxt.Load()) } @@ -574,7 +580,11 @@ func (t *tenantHeads) tenantIndex(userID string, from, through model.Time) (idx return } - return NewTSDBIndex(tenant.indexRange(int64(from), int64(through))), t.locks[i].RUnlock, true + idx = NewTSDBIndex(tenant.indexRange(int64(from), int64(through))) + if t.chunkFilter != nil { + idx.SetChunkFilterer(t.chunkFilter) + } + return idx, t.locks[i].RUnlock, true } diff --git a/pkg/storage/stores/tsdb/index.go b/pkg/storage/stores/tsdb/index.go index a6cc7cd12ede3..1df7184a50deb 100644 --- a/pkg/storage/stores/tsdb/index.go +++ b/pkg/storage/stores/tsdb/index.go @@ -6,6 +6,7 @@ import ( "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" ) @@ -32,6 +33,7 @@ func (r ChunkRef) Less(x ChunkRef) bool { type Index interface { Bounded + SetChunkFilterer(chunkFilter chunk.RequestChunkFilterer) Close() error // GetChunkRefs accepts an optional []ChunkRef argument. // If not nil, it will use that slice to build the result, @@ -68,3 +70,5 @@ func (NoopIndex) LabelNames(ctx context.Context, userID string, from, through mo func (NoopIndex) LabelValues(ctx context.Context, userID string, from, through model.Time, name string, matchers ...*labels.Matcher) ([]string, error) { return nil, nil } + +func (NoopIndex) SetChunkFilterer(chunkFilter chunk.RequestChunkFilterer) {} diff --git a/pkg/storage/stores/tsdb/index_client.go b/pkg/storage/stores/tsdb/index_client.go index d92050c47a3cf..d683c81fb6963 100644 --- a/pkg/storage/stores/tsdb/index_client.go +++ b/pkg/storage/stores/tsdb/index_client.go @@ -114,6 +114,5 @@ func (c *IndexClient) LabelNamesForMetricName(ctx context.Context, userID string // 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) { - // TODO(owen-d): handle this - panic("unimplemented") + c.idx.SetChunkFilterer(chunkFilter) } diff --git a/pkg/storage/stores/tsdb/lazy_index.go b/pkg/storage/stores/tsdb/lazy_index.go index c8c2515c7f852..9a3da11b565cf 100644 --- a/pkg/storage/stores/tsdb/lazy_index.go +++ b/pkg/storage/stores/tsdb/lazy_index.go @@ -6,6 +6,7 @@ import ( "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" ) @@ -19,6 +20,14 @@ func (f LazyIndex) Bounds() (model.Time, model.Time) { } 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 { diff --git a/pkg/storage/stores/tsdb/manager.go b/pkg/storage/stores/tsdb/manager.go index 3f0df85976fa4..1ee6e04ccfc66 100644 --- a/pkg/storage/stores/tsdb/manager.go +++ b/pkg/storage/stores/tsdb/manager.go @@ -14,6 +14,7 @@ import ( "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" @@ -45,7 +46,8 @@ type tsdbManager struct { sync.RWMutex - shipper indexshipper.IndexShipper + chunkFilter chunk.RequestChunkFilterer + shipper indexshipper.IndexShipper } func NewTSDBManager( @@ -199,7 +201,15 @@ func (m *tsdbManager) indices(ctx context.Context, from, through model.Time, use if len(indices) == 0 { return NoopIndex{}, nil } - return NewMultiIndex(indices...) + 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? @@ -209,6 +219,10 @@ 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 { diff --git a/pkg/storage/stores/tsdb/multi_file_index.go b/pkg/storage/stores/tsdb/multi_file_index.go index fba710b9b747f..9a74455a0fee6 100644 --- a/pkg/storage/stores/tsdb/multi_file_index.go +++ b/pkg/storage/stores/tsdb/multi_file_index.go @@ -9,6 +9,7 @@ import ( "github.com/prometheus/prometheus/model/labels" "golang.org/x/sync/errgroup" + "github.com/grafana/loki/pkg/storage/chunk" "github.com/grafana/loki/pkg/storage/stores/tsdb/index" ) @@ -40,6 +41,12 @@ func (i *MultiIndex) Bounds() (model.Time, model.Time) { return lowest, highest } +func (i *MultiIndex) SetChunkFilterer(chunkFilter chunk.RequestChunkFilterer) { + for _, x := range i.indices { + x.SetChunkFilterer(chunkFilter) + } +} + func (i *MultiIndex) Close() error { var errs multierror.MultiError for _, idx := range i.indices { diff --git a/pkg/storage/stores/tsdb/multitenant.go b/pkg/storage/stores/tsdb/multitenant.go index 9a55e5c8e35ae..f3a26aa7db333 100644 --- a/pkg/storage/stores/tsdb/multitenant.go +++ b/pkg/storage/stores/tsdb/multitenant.go @@ -6,6 +6,7 @@ import ( "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" ) @@ -43,6 +44,10 @@ func withoutTenantLabel(ls labels.Labels) labels.Labels { 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) { diff --git a/pkg/storage/stores/tsdb/single_file_index.go b/pkg/storage/stores/tsdb/single_file_index.go index ea562f34988b1..9937d1fd27085 100644 --- a/pkg/storage/stores/tsdb/single_file_index.go +++ b/pkg/storage/stores/tsdb/single_file_index.go @@ -11,6 +11,7 @@ import ( "github.com/prometheus/prometheus/model/labels" "github.com/grafana/loki/pkg/chunkenc" + "github.com/grafana/loki/pkg/storage/chunk" index_shipper "github.com/grafana/loki/pkg/storage/stores/indexshipper/index" "github.com/grafana/loki/pkg/storage/stores/tsdb/index" ) @@ -77,7 +78,8 @@ func (f *TSDBFile) Reader() (io.ReadSeeker, error) { // and translates the IndexReader to an Index implementation // It loads the file into memory and doesn't keep a file descriptor open type TSDBIndex struct { - reader IndexReader + reader IndexReader + chunkFilter chunk.RequestChunkFilterer } // Return the index as well as the underlying []byte which isn't exposed as an index @@ -124,9 +126,14 @@ func (i *TSDBIndex) Bounds() (model.Time, model.Time) { return model.Time(from), model.Time(through) } +func (i *TSDBIndex) SetChunkFilterer(chunkFilter chunk.RequestChunkFilterer) { + i.chunkFilter = chunkFilter +} + // fn must NOT capture it's arguments. They're reused across series iterations and returned to // a pool after completion. func (i *TSDBIndex) forSeries( + ctx context.Context, shard *index.ShardAnnotation, fn func(labels.Labels, model.Fingerprint, []index.ChunkMeta), matchers ...*labels.Matcher, @@ -140,6 +147,11 @@ func (i *TSDBIndex) forSeries( chks := ChunkMetasPool.Get() defer ChunkMetasPool.Put(chks) + var filterer chunk.Filterer + if i.chunkFilter != nil { + filterer = i.chunkFilter.ForRequest(ctx) + } + for p.Next() { hash, err := i.reader.Series(p.At(), &ls, &chks) if err != nil { @@ -151,19 +163,23 @@ func (i *TSDBIndex) forSeries( continue } + if filterer != nil && filterer.ShouldFilter(ls) { + continue + } + fn(ls, model.Fingerprint(hash), chks) } return p.Err() } -func (i *TSDBIndex) GetChunkRefs(_ context.Context, userID string, from, through model.Time, res []ChunkRef, shard *index.ShardAnnotation, matchers ...*labels.Matcher) ([]ChunkRef, error) { +func (i *TSDBIndex) GetChunkRefs(ctx context.Context, userID string, from, through model.Time, res []ChunkRef, shard *index.ShardAnnotation, matchers ...*labels.Matcher) ([]ChunkRef, error) { queryBounds := newBounds(from, through) if res == nil { res = ChunkRefsPool.Get() } res = res[:0] - if err := i.forSeries(shard, + if err := i.forSeries(ctx, shard, func(ls labels.Labels, fp model.Fingerprint, chks []index.ChunkMeta) { // TODO(owen-d): use logarithmic approach for _, chk := range chks { @@ -189,14 +205,14 @@ func (i *TSDBIndex) GetChunkRefs(_ context.Context, userID string, from, through return res, nil } -func (i *TSDBIndex) Series(_ context.Context, _ string, from, through model.Time, res []Series, shard *index.ShardAnnotation, matchers ...*labels.Matcher) ([]Series, error) { +func (i *TSDBIndex) Series(ctx context.Context, _ string, from, through model.Time, res []Series, shard *index.ShardAnnotation, matchers ...*labels.Matcher) ([]Series, error) { queryBounds := newBounds(from, through) if res == nil { res = SeriesPool.Get() } res = res[:0] - if err := i.forSeries(shard, + if err := i.forSeries(ctx, shard, func(ls labels.Labels, fp model.Fingerprint, chks []index.ChunkMeta) { // TODO(owen-d): use logarithmic approach for _, chk := range chks { From c6e7cf6aab0473f032093b0d83cac93431ae15eb Mon Sep 17 00:00:00 2001 From: Sandeep Sukhani Date: Thu, 5 May 2022 15:51:11 +0530 Subject: [PATCH 85/85] always set ingester name when using object storage index --- pkg/loki/modules.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/pkg/loki/modules.go b/pkg/loki/modules.go index 86d3443ee3d69..62506a8e41fb4 100644 --- a/pkg/loki/modules.go +++ b/pkg/loki/modules.go @@ -397,6 +397,8 @@ func (t *Loki) initStore() (_ services.Service, err error) { // Set configs pertaining to object storage based indices if config.UsingObjectStorageIndex(t.Cfg.SchemaConfig.Configs) { + t.Cfg.StorageConfig.BoltDBShipperConfig.IngesterName = t.Cfg.Ingester.LifecyclerConfig.ID + t.Cfg.StorageConfig.TSDBShipperConfig.IngesterName = t.Cfg.Ingester.LifecyclerConfig.ID switch true { case t.Cfg.isModuleEnabled(Ingester), t.Cfg.isModuleEnabled(Write): @@ -420,11 +422,9 @@ func (t *Loki) initStore() (_ services.Service, err error) { // We do not want ingester to unnecessarily keep downloading files t.Cfg.StorageConfig.BoltDBShipperConfig.Mode = shipper.ModeWriteOnly - t.Cfg.StorageConfig.BoltDBShipperConfig.IngesterName = t.Cfg.Ingester.LifecyclerConfig.ID t.Cfg.StorageConfig.BoltDBShipperConfig.IngesterDBRetainPeriod = shipperQuerierIndexUpdateDelay(t.Cfg.StorageConfig.IndexCacheValidity, t.Cfg.StorageConfig.BoltDBShipperConfig.ResyncInterval) t.Cfg.StorageConfig.TSDBShipperConfig.Mode = indexshipper.ModeWriteOnly - t.Cfg.StorageConfig.TSDBShipperConfig.IngesterName = t.Cfg.Ingester.LifecyclerConfig.ID t.Cfg.StorageConfig.TSDBShipperConfig.IngesterDBRetainPeriod = shipperQuerierIndexUpdateDelay(t.Cfg.StorageConfig.IndexCacheValidity, t.Cfg.StorageConfig.TSDBShipperConfig.ResyncInterval) case t.Cfg.isModuleEnabled(Querier), t.Cfg.isModuleEnabled(Ruler), t.Cfg.isModuleEnabled(Read), t.isModuleActive(IndexGateway):