From 8ee10a192cf5034b8285a0a5a52b9120fdf6a45d Mon Sep 17 00:00:00 2001 From: Cyril Tovena Date: Wed, 24 Mar 2021 22:06:00 +0100 Subject: [PATCH 01/42] Playing around with a POC based on the design doc. Signed-off-by: Cyril Tovena --- pkg/storage/store.go | 2 - pkg/storage/store_test.go | 10 +- .../shipper/compactor/retention_test.go | 544 ++++++++++++++++++ pkg/storage/stores/shipper/compactor/table.go | 6 +- 4 files changed, 549 insertions(+), 13 deletions(-) create mode 100644 pkg/storage/stores/shipper/compactor/retention_test.go diff --git a/pkg/storage/store.go b/pkg/storage/store.go index 312c530b0ab35..e8b0a7b81b434 100644 --- a/pkg/storage/store.go +++ b/pkg/storage/store.go @@ -261,7 +261,6 @@ func (s *store) GetSeries(ctx context.Context, req logql.SelectLogParams) ([]log } sort.Sort(results) return results, nil - } // SelectLogs returns an iterator that will query the store for more chunks while iterating instead of fetching all chunks upfront @@ -292,7 +291,6 @@ func (s *store) SelectLogs(ctx context.Context, req logql.SelectLogParams) (iter } return newLogBatchIterator(ctx, s.chunkMetrics, lazyChunks, s.cfg.MaxChunkBatchSize, matchers, pipeline, req.Direction, req.Start, req.End) - } func (s *store) SelectSamples(ctx context.Context, req logql.SelectSampleParams) (iter.SampleIterator, error) { diff --git a/pkg/storage/store_test.go b/pkg/storage/store_test.go index 5bd9ff441525c..0dd4636f46adf 100644 --- a/pkg/storage/store_test.go +++ b/pkg/storage/store_test.go @@ -43,7 +43,7 @@ var ( chunkStore = getLocalStore() ) -//go test -bench=. -benchmem -memprofile memprofile.out -cpuprofile profile.out +// go test -bench=. -benchmem -memprofile memprofile.out -cpuprofile profile.out func Benchmark_store_SelectLogsRegexBackward(b *testing.B) { benchmarkStoreQuery(b, &logproto.QueryRequest{ Selector: `{foo="bar"} |~ "fuzz"`, @@ -122,13 +122,12 @@ func Benchmark_store_SelectSample(b *testing.B) { }) } log.Print("sample processed ", len(sampleRes)) - } func benchmarkStoreQuery(b *testing.B, query *logproto.QueryRequest) { b.ReportAllocs() // force to run gc 10x more often this can be useful to detect fast allocation vs leak. - //debug.SetGCPercent(10) + // debug.SetGCPercent(10) stop := make(chan struct{}) go func() { _ = http.ListenAndServe(":6060", http.DefaultServeMux) @@ -220,7 +219,6 @@ func getLocalStore() Store { storeConfig.Config, chunk.StoreConfig{}, schemaConfig.SchemaConfig, limits, nil, nil, util_log.Logger) - if err != nil { panic(err) } @@ -233,7 +231,6 @@ func getLocalStore() Store { } func Test_store_SelectLogs(t *testing.T) { - tests := []struct { name string req *logproto.QueryRequest @@ -421,7 +418,6 @@ func Test_store_SelectLogs(t *testing.T) { } func Test_store_SelectSample(t *testing.T) { - tests := []struct { name string req *logproto.SampleQueryRequest @@ -631,7 +627,6 @@ func Test_store_SelectSample(t *testing.T) { } func Test_store_GetSeries(t *testing.T) { - tests := []struct { name string req *logproto.QueryRequest @@ -879,7 +874,6 @@ func TestStore_MultipleBoltDBShippersInConfig(t *testing.T) { func mustParseLabels(s string) map[string]string { l, err := marshal.NewLabelSet(s) - if err != nil { log.Fatalf("Failed to parse %s", s) } diff --git a/pkg/storage/stores/shipper/compactor/retention_test.go b/pkg/storage/stores/shipper/compactor/retention_test.go new file mode 100644 index 0000000000000..f6423c091f187 --- /dev/null +++ b/pkg/storage/stores/shipper/compactor/retention_test.go @@ -0,0 +1,544 @@ +package compactor + +import ( + "bytes" + "context" + "errors" + "fmt" + "io/ioutil" + "os" + "path/filepath" + "sort" + "strconv" + "strings" + "testing" + "time" + "unsafe" + + "github.com/cortexproject/cortex/pkg/chunk" + "github.com/cortexproject/cortex/pkg/chunk/local" + cortex_storage "github.com/cortexproject/cortex/pkg/chunk/storage" + "github.com/cortexproject/cortex/pkg/ingester/client" + util_log "github.com/cortexproject/cortex/pkg/util/log" + "github.com/cortexproject/cortex/pkg/util/math" + "github.com/grafana/loki/pkg/chunkenc" + "github.com/grafana/loki/pkg/logproto" + "github.com/grafana/loki/pkg/storage" + shipper_util "github.com/grafana/loki/pkg/storage/stores/shipper/util" + "github.com/grafana/loki/pkg/util/validation" + "github.com/prometheus/common/model" + "github.com/prometheus/prometheus/pkg/labels" + "github.com/stretchr/testify/require" + "go.etcd.io/bbolt" +) + +const ( + chunkTimeRangeKeyV3 = '3' +) + +func Test_Retention(t *testing.T) { + indexDir, err := ioutil.TempDir("", "boltdb_test") + require.Nil(t, err) + + chunkDir, err := ioutil.TempDir("", "chunk_test") + require.Nil(t, err) + + defer func() { + require.NoError(t, os.RemoveAll(indexDir)) + require.NoError(t, os.RemoveAll(chunkDir)) + }() + limits, err := validation.NewOverrides(validation.Limits{}, nil) + require.NoError(t, err) + + schemaCfg := storage.SchemaConfig{ + SchemaConfig: chunk.SchemaConfig{ + Configs: []chunk.PeriodConfig{ + { + From: chunk.DayTime{Time: model.Earliest}, + IndexType: "boltdb", + ObjectType: "filesystem", + Schema: "v9", + IndexTables: chunk.PeriodicTableConfig{ + Prefix: "index_", + Period: time.Hour * 24, + }, + RowShards: 16, + }, + { + From: chunk.DayTime{Time: model.Earliest.Add(25 * time.Hour)}, + IndexType: "boltdb", + ObjectType: "filesystem", + Schema: "v11", + IndexTables: chunk.PeriodicTableConfig{ + Prefix: "index_", + Period: time.Hour * 24, + }, + RowShards: 16, + }, + }, + }, + } + require.NoError(t, schemaCfg.SchemaConfig.Validate()) + + config := storage.Config{ + Config: cortex_storage.Config{ + BoltDBConfig: local.BoltDBConfig{ + Directory: indexDir, + }, + FSConfig: local.FSConfig{ + Directory: chunkDir, + }, + }, + } + chunkStore, err := cortex_storage.NewStore( + config.Config, + chunk.StoreConfig{}, + schemaCfg.SchemaConfig, + limits, + nil, + nil, + util_log.Logger, + ) + require.NoError(t, err) + + store, err := storage.NewStore(config, schemaCfg, chunkStore, nil) + require.NoError(t, err) + + require.NoError(t, store.Put(context.TODO(), []chunk.Chunk{ + createChunk(t, "1", labels.Labels{labels.Label{Name: "foo", Value: "bar"}}, model.Earliest, model.Earliest.Add(1*time.Hour)), + createChunk(t, "2", labels.Labels{labels.Label{Name: "foo", Value: "buzz"}}, model.Earliest.Add(26*time.Hour), model.Earliest.Add(27*time.Hour)), + })) + + store.Stop() + + indexFilesInfo, err := ioutil.ReadDir(indexDir) + require.NoError(t, err) + + retentionRules := fakeRule{} + + // 1- Get all series ID for given retention per stream.... + // 2 - Delete from index and Mark for delete all chunk based on retention with seriesID/tenantID. + // 3 - Seek chunk entries via series id for each series and verify if we still have chunk. + // 4 - Delete Label entries for empty series with empty chunk entries. + + // For 1. only equality matcher are OK so we can use GetReadMetricLabelValueQueries + for _, indexFileInfo := range indexFilesInfo { + db, err := shipper_util.SafeOpenBoltdbFile(filepath.Join(indexDir, indexFileInfo.Name())) + fmt.Fprintf(os.Stdout, "Opening Table %s\n", indexFileInfo.Name()) + require.NoError(t, err) + + // 1 - Get all series ID for given retention per stream.... + + // 1.1 find the schema for this table + + currentSchema, ok := schemaPeriodForTable(schemaCfg, indexFileInfo.Name()) + if !ok { + fmt.Fprintf(os.Stdout, "Could not find Schema for Table %s\n", indexFileInfo.Name()) + continue + } + fmt.Fprintf(os.Stdout, "Found Schema for Table %s => %+v\n", indexFileInfo.Name(), currentSchema) + findSeriesIDsForRules(db, currentSchema, retentionRules.PerStream()) + require.NoError(t, + db.Update(func(tx *bbolt.Tx) error { + return tx.Bucket(bucketName).ForEach(func(k, v []byte) error { + ref, ok, err := parseChunkRef(decodeKey(k)) + if err != nil { + return err + } + if ok { + fmt.Fprintf(os.Stdout, "%+v\n", ref) + } + return nil + }) + })) + } +} + +func findSeriesIDsForRules(db *bbolt.DB, config chunk.PeriodConfig, rules []StreamRule) ([][]string, error) { + schema, err := config.CreateSchema() + if err != nil { + return nil, err + } + // cover the whole table. + from, through := config.From.Time, config.From.Time.Add(config.IndexTables.Period) + result := make([][]string, len(rules)) + + for ruleIndex, rule := range rules { + incomingIDs := make(chan []string) + incomingErrors := make(chan error) + + for _, matcher := range rule.Matchers { + go func(matcher *labels.Matcher) { + ids, err := lookupSeriesByMatcher(db, schema, from, through, rule.UserID, matcher) + if err != nil { + incomingErrors <- err + return + } + incomingIDs <- ids + }(&matcher) + } + // intersect. and add to result. + var ids []string + var lastErr error + var initialized bool + for i := 0; i < len(rule.Matchers); i++ { + select { + case incoming := <-incomingIDs: + if !initialized { + ids = incoming + initialized = true + } else { + ids = intersectStrings(ids, incoming) + } + case err := <-incomingErrors: + lastErr = err + } + } + if lastErr != nil { + return nil, err + } + result[ruleIndex] = ids + } + + return result, nil +} + +var QueryParallelism = 100 + +func lookupSeriesByMatcher( + db *bbolt.DB, + schema chunk.BaseSchema, + from, through model.Time, + userID string, + matcher *labels.Matcher) ([]string, error) { + queries, err := schema.GetReadQueriesForMetricLabelValue( + from, through, userID, "logs", matcher.Name, matcher.Value) + if err != nil { + return nil, err + } + if len(queries) == 0 { + return nil, nil + } + if len(queries) == 1 { + return lookupSeriesByQuery(db, queries[0]) + } + queue := make(chan chunk.IndexQuery) + incomingResult := make(chan struct { + ids []string + err error + }) + n := math.Min(len(queries), QueryParallelism) + for i := 0; i < n; i++ { + go func() { + for { + query, ok := <-queue + if !ok { + return + } + res, err := lookupSeriesByQuery(db, query) + incomingResult <- struct { + ids []string + err error + }{res, err} + } + }() + } + go func() { + for _, query := range queries { + queue <- query + } + close(queue) + }() + + // Now receive all the results. + var ids []string + var lastErr error + for i := 0; i < len(queries); i++ { + res := <-incomingResult + if res.err != nil { + lastErr = res.err + continue + } + ids = append(ids, res.ids...) + } + sort.Strings(ids) + ids = uniqueStrings(ids) + return ids, lastErr +} + +func uniqueStrings(cs []string) []string { + if len(cs) == 0 { + return []string{} + } + + result := make([]string, 1, len(cs)) + result[0] = cs[0] + i, j := 0, 1 + for j < len(cs) { + if result[i] == cs[j] { + j++ + continue + } + result = append(result, cs[j]) + i++ + j++ + } + return result +} + +func intersectStrings(left, right []string) []string { + var ( + i, j = 0, 0 + result = []string{} + ) + for i < len(left) && j < len(right) { + if left[i] == right[j] { + result = append(result, left[i]) + } + + if left[i] < right[j] { + i++ + } else { + j++ + } + } + return result +} + +const separator = "\000" + +func lookupSeriesByQuery(db *bbolt.DB, query chunk.IndexQuery) ([]string, error) { + start = []byte(query.HashValue + separator + string(query.RangeValueStart)) + + db.View(func(t *bbolt.Tx) error { + return nil + }) + + return nil, nil +} + +// func (b *BoltIndexClient) QueryWithCursor(_ context.Context, c *bbolt.Cursor, query chunk.IndexQuery, callback func(chunk.IndexQuery, chunk.ReadBatch) (shouldContinue bool)) error { +// var start []byte +// if len(query.RangeValuePrefix) > 0 { +// start = []byte(query.HashValue + separator + string(query.RangeValuePrefix)) +// } else if len(query.RangeValueStart) > 0 { +// start = []byte(query.HashValue + separator + string(query.RangeValueStart)) +// } else { +// start = []byte(query.HashValue + separator) +// } + +// rowPrefix := []byte(query.HashValue + separator) + +// var batch boltReadBatch + +// for k, v := c.Seek(start); k != nil; k, v = c.Next() { +// if len(query.ValueEqual) > 0 && !bytes.Equal(v, query.ValueEqual) { +// continue +// } + +// if len(query.RangeValuePrefix) > 0 && !bytes.HasPrefix(k, start) { +// break +// } + +// if !bytes.HasPrefix(k, rowPrefix) { +// break +// } + +// // make a copy since k, v are only valid for the life of the transaction. +// // See: https://godoc.org/github.com/boltdb/bolt#Cursor.Seek +// batch.rangeValue = make([]byte, len(k)-len(rowPrefix)) +// copy(batch.rangeValue, k[len(rowPrefix):]) + +// batch.value = make([]byte, len(v)) +// copy(batch.value, v) + +// if !callback(query, &batch) { +// break +// } +// } + +// return nil +// } + +func schemaPeriodForTable(config storage.SchemaConfig, tableName string) (chunk.PeriodConfig, bool) { + for _, schema := range config.Configs { + periodIndex, err := strconv.ParseInt(strings.TrimPrefix(tableName, schema.IndexTables.Prefix), 10, 64) + if err != nil { + continue + } + periodSecs := int64((schema.IndexTables.Period) / time.Second) + if periodIndex == schema.From.Time.Unix()/periodSecs { + return schema, true + } + } + return chunk.PeriodConfig{}, false +} + +type ChunkRef struct { + UserID []byte + SeriesID []byte + // Fingerprint model.Fingerprint + From model.Time + Through model.Time +} + +func (c ChunkRef) String() string { + return fmt.Sprintf("UserID: %s , SeriesID: %s , Time: [%s,%s]", c.UserID, c.SeriesID, c.From, c.Through) +} + +var ErrInvalidIndexKey = errors.New("invalid index key") + +type InvalidIndexKeyError struct { + HashKey string + RangeKey string +} + +func newInvalidIndexKeyError(h, r []byte) InvalidIndexKeyError { + return InvalidIndexKeyError{ + HashKey: string(h), + RangeKey: string(r), + } +} + +func (e InvalidIndexKeyError) Error() string { + return fmt.Sprintf("%s: hash_key:%s range_key:%s", ErrInvalidIndexKey, e.HashKey, e.RangeKey) +} + +func (e InvalidIndexKeyError) Is(target error) bool { + return target == ErrInvalidIndexKey +} + +func parseChunkRef(hashKey, rangeKey []byte) (ChunkRef, bool, error) { + // todo reuse memory + var components [][]byte + components = decodeRangeKey(rangeKey, components) + if len(components) == 0 { + return ChunkRef{}, false, newInvalidIndexKeyError(hashKey, rangeKey) + } + + keyType := components[len(components)-1] + if len(keyType) == 0 || keyType[0] != chunkTimeRangeKeyV3 { + return ChunkRef{}, false, nil + } + chunkID := components[len(components)-2] + + // todo split manually + parts := bytes.Split(chunkID, []byte("/")) + if len(parts) != 2 { + return ChunkRef{}, false, newInvalidIndexKeyError(hashKey, rangeKey) + } + userID := parts[0] + // todo split manually + hexParts := bytes.Split(parts[1], []byte(":")) + if len(hexParts) != 4 { + return ChunkRef{}, false, newInvalidIndexKeyError(hashKey, rangeKey) + } + + from, err := strconv.ParseInt(unsafeGetString(hexParts[1]), 16, 64) + if err != nil { + return ChunkRef{}, false, err + } + through, err := strconv.ParseInt(unsafeGetString(hexParts[2]), 16, 64) + if err != nil { + return ChunkRef{}, false, err + } + + return ChunkRef{ + UserID: userID, + SeriesID: seriesFromHash(hashKey), + From: model.Time(from), + Through: model.Time(through), + }, true, nil +} + +func unsafeGetString(buf []byte) string { + return *((*string)(unsafe.Pointer(&buf))) +} + +func seriesFromHash(h []byte) (seriesID []byte) { + var index int + for i := range h { + if h[i] == ':' { + index++ + } + if index == 2 { + seriesID = h[i+1:] + return + } + } + return +} + +func decodeKey(k []byte) (hashValue, rangeValue []byte) { + // hashValue + 0 + string(rangeValue) + for i := range k { + if k[i] == 0 { + hashValue = k[:i] + rangeValue = k[i+1:] + return + } + } + return +} + +func decodeRangeKey(value []byte, components [][]byte) [][]byte { + components = components[:0] + i, j := 0, 0 + for j < len(value) { + if value[j] != 0 { + j++ + continue + } + components = append(components, value[i:j]) + j++ + i = j + } + return components +} + +func createChunk(t testing.TB, userID string, lbs labels.Labels, from model.Time, through model.Time) chunk.Chunk { + t.Helper() + const ( + targetSize = 1500 * 1024 + blockSize = 256 * 1024 + ) + labelsBuilder := labels.NewBuilder(lbs) + labelsBuilder.Set(labels.MetricName, "logs") + metric := labelsBuilder.Labels() + fp := client.Fingerprint(lbs) + chunkEnc := chunkenc.NewMemChunk(chunkenc.EncSnappy, blockSize, targetSize) + + for ts := from; ts.Before(through); ts = ts.Add(1 * time.Minute) { + require.NoError(t, chunkEnc.Append(&logproto.Entry{ + Timestamp: ts.Time(), + Line: ts.String(), + })) + } + c := chunk.NewChunk(userID, fp, metric, chunkenc.NewFacade(chunkEnc, blockSize, targetSize), from, through) + require.NoError(t, c.Encode()) + return c +} + +type StreamRule struct { + Matchers []labels.Matcher + Duration time.Duration + UserID string +} + +type Rules interface { + PerTenant(userID string) time.Duration + PerStream() []StreamRule +} + +type fakeRule struct { + streams []StreamRule + tenants map[string]time.Duration +} + +func (f fakeRule) PerTenant(userID string) time.Duration { + return f.tenants[userID] +} + +func (f fakeRule) PerStream() []StreamRule { + return f.streams +} diff --git a/pkg/storage/stores/shipper/compactor/table.go b/pkg/storage/stores/shipper/compactor/table.go index ca2b845e71700..695e2a472bf99 100644 --- a/pkg/storage/stores/shipper/compactor/table.go +++ b/pkg/storage/stores/shipper/compactor/table.go @@ -82,6 +82,7 @@ func (t *table) compact() error { } }() + // create a new compacted db t.compactedDB, err = shipper_util.SafeOpenBoltdbFile(filepath.Join(t.workingDirectory, fmt.Sprint(time.Now().Unix()))) if err != nil { return err @@ -93,7 +94,7 @@ func (t *table) compact() error { readObjectChan := make(chan string) n := util_math.Min(len(objects), readDBsParallelism) - // read files parallelly + // read files in parallel for i := 0; i < n; i++ { go func() { var err error @@ -137,7 +138,6 @@ func (t *table) compact() error { return } } - }() } @@ -268,7 +268,7 @@ func (t *table) readFile(path string) error { if err != nil { return err } - + // todo(cyriltovena) we should just re-slice to avoid allocations writeBatch = make([]indexEntry, 0, batchSize) } From 934d04d2fc2617a59a9a2d4b14d14391e67a027c Mon Sep 17 00:00:00 2001 From: Cyril Tovena Date: Thu, 25 Mar 2021 16:57:13 +0100 Subject: [PATCH 02/42] Getting series ID for label matchers is now working. Signed-off-by: Cyril Tovena --- .../shipper/compactor/retention_test.go | 149 ++++++++++++------ 1 file changed, 97 insertions(+), 52 deletions(-) diff --git a/pkg/storage/stores/shipper/compactor/retention_test.go b/pkg/storage/stores/shipper/compactor/retention_test.go index f6423c091f187..ef20b72361bc2 100644 --- a/pkg/storage/stores/shipper/compactor/retention_test.go +++ b/pkg/storage/stores/shipper/compactor/retention_test.go @@ -3,6 +3,8 @@ package compactor import ( "bytes" "context" + "crypto/sha256" + "encoding/base64" "errors" "fmt" "io/ioutil" @@ -21,15 +23,16 @@ import ( "github.com/cortexproject/cortex/pkg/ingester/client" util_log "github.com/cortexproject/cortex/pkg/util/log" "github.com/cortexproject/cortex/pkg/util/math" - "github.com/grafana/loki/pkg/chunkenc" - "github.com/grafana/loki/pkg/logproto" - "github.com/grafana/loki/pkg/storage" - shipper_util "github.com/grafana/loki/pkg/storage/stores/shipper/util" "github.com/grafana/loki/pkg/util/validation" "github.com/prometheus/common/model" "github.com/prometheus/prometheus/pkg/labels" "github.com/stretchr/testify/require" "go.etcd.io/bbolt" + + "github.com/grafana/loki/pkg/chunkenc" + "github.com/grafana/loki/pkg/logproto" + "github.com/grafana/loki/pkg/storage" + shipper_util "github.com/grafana/loki/pkg/storage/stores/shipper/util" ) const ( @@ -114,7 +117,20 @@ func Test_Retention(t *testing.T) { indexFilesInfo, err := ioutil.ReadDir(indexDir) require.NoError(t, err) - retentionRules := fakeRule{} + retentionRules := fakeRule{ + streams: []StreamRule{ + { + UserID: "1", + Matchers: []labels.Matcher{ + { + Type: labels.MatchEqual, + Name: "foo", + Value: "bar", + }, + }, + }, + }, + } // 1- Get all series ID for given retention per stream.... // 2 - Delete from index and Mark for delete all chunk based on retention with seriesID/tenantID. @@ -137,7 +153,10 @@ func Test_Retention(t *testing.T) { continue } fmt.Fprintf(os.Stdout, "Found Schema for Table %s => %+v\n", indexFileInfo.Name(), currentSchema) - findSeriesIDsForRules(db, currentSchema, retentionRules.PerStream()) + ids, err := findSeriesIDsForRules(db, currentSchema, retentionRules.PerStream()) + + require.NoError(t, err) + fmt.Fprintf(os.Stdout, "Found IDS for rules %+v\n", ids) require.NoError(t, db.Update(func(tx *bbolt.Tx) error { return tx.Bucket(bucketName).ForEach(func(k, v []byte) error { @@ -308,58 +327,41 @@ func intersectStrings(left, right []string) []string { const separator = "\000" func lookupSeriesByQuery(db *bbolt.DB, query chunk.IndexQuery) ([]string, error) { - start = []byte(query.HashValue + separator + string(query.RangeValueStart)) + start := []byte(query.HashValue + separator + string(query.RangeValueStart)) + rowPrefix := []byte(query.HashValue + separator) + var res []string + var components [][]byte + err := db.View(func(tx *bbolt.Tx) error { + bucket := tx.Bucket(bucketName) + if bucket == nil { + return nil + } + c := bucket.Cursor() + for k, v := c.Seek(start); k != nil; k, v = c.Next() { + // technically we can run regex that are not matching empty. + if len(query.ValueEqual) > 0 && !bytes.Equal(v, query.ValueEqual) { + continue + } + if !bytes.HasPrefix(k, rowPrefix) { + break + } + // parse series ID and add to res + _, r := decodeKey(k) - db.View(func(t *bbolt.Tx) error { + components = decodeRangeKey(r, components) + if len(components) != 4 { + continue + } + // we store in label entries range keys: label hash value | seriesID | empty | type. + // and we want the seriesID + res = append(res, string(components[len(components)-3])) + } return nil }) - return nil, nil + return res, err } -// func (b *BoltIndexClient) QueryWithCursor(_ context.Context, c *bbolt.Cursor, query chunk.IndexQuery, callback func(chunk.IndexQuery, chunk.ReadBatch) (shouldContinue bool)) error { -// var start []byte -// if len(query.RangeValuePrefix) > 0 { -// start = []byte(query.HashValue + separator + string(query.RangeValuePrefix)) -// } else if len(query.RangeValueStart) > 0 { -// start = []byte(query.HashValue + separator + string(query.RangeValueStart)) -// } else { -// start = []byte(query.HashValue + separator) -// } - -// rowPrefix := []byte(query.HashValue + separator) - -// var batch boltReadBatch - -// for k, v := c.Seek(start); k != nil; k, v = c.Next() { -// if len(query.ValueEqual) > 0 && !bytes.Equal(v, query.ValueEqual) { -// continue -// } - -// if len(query.RangeValuePrefix) > 0 && !bytes.HasPrefix(k, start) { -// break -// } - -// if !bytes.HasPrefix(k, rowPrefix) { -// break -// } - -// // make a copy since k, v are only valid for the life of the transaction. -// // See: https://godoc.org/github.com/boltdb/bolt#Cursor.Seek -// batch.rangeValue = make([]byte, len(k)-len(rowPrefix)) -// copy(batch.rangeValue, k[len(rowPrefix):]) - -// batch.value = make([]byte, len(v)) -// copy(batch.value, v) - -// if !callback(query, &batch) { -// break -// } -// } - -// return nil -// } - func schemaPeriodForTable(config storage.SchemaConfig, tableName string) (chunk.PeriodConfig, bool) { for _, schema := range config.Configs { periodIndex, err := strconv.ParseInt(strings.TrimPrefix(tableName, schema.IndexTables.Prefix), 10, 64) @@ -519,6 +521,49 @@ func createChunk(t testing.TB, userID string, lbs labels.Labels, from model.Time return c } +func labelsSeriesID(ls labels.Labels) []byte { + h := sha256.Sum256([]byte(labelsString(ls))) + return encodeBase64Bytes(h[:]) +} + +func encodeBase64Bytes(bytes []byte) []byte { + encodedLen := base64.RawStdEncoding.EncodedLen(len(bytes)) + encoded := make([]byte, encodedLen) + base64.RawStdEncoding.Encode(encoded, bytes) + return encoded +} + +// Backwards-compatible with model.Metric.String() +func labelsString(ls labels.Labels) string { + metricName := ls.Get(labels.MetricName) + if metricName != "" && len(ls) == 1 { + return metricName + } + var b strings.Builder + b.Grow(1000) + + b.WriteString(metricName) + b.WriteByte('{') + i := 0 + for _, l := range ls { + if l.Name == labels.MetricName { + continue + } + if i > 0 { + b.WriteByte(',') + b.WriteByte(' ') + } + b.WriteString(l.Name) + b.WriteByte('=') + var buf [1000]byte + b.Write(strconv.AppendQuote(buf[:0], l.Value)) + i++ + } + b.WriteByte('}') + + return b.String() +} + type StreamRule struct { Matchers []labels.Matcher Duration time.Duration From e87510f2f815c43ccaa6e2d08961f38911c3a571 Mon Sep 17 00:00:00 2001 From: Cyril Tovena Date: Fri, 26 Mar 2021 23:05:39 +0100 Subject: [PATCH 03/42] Fixes tests. Signed-off-by: Cyril Tovena --- pkg/logql/sharding_test.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pkg/logql/sharding_test.go b/pkg/logql/sharding_test.go index 38c6f27037b41..e7b374eaadc71 100644 --- a/pkg/logql/sharding_test.go +++ b/pkg/logql/sharding_test.go @@ -20,7 +20,7 @@ func TestMappingEquivalence(t *testing.T) { shards = 3 nStreams = 60 rounds = 20 - streams = randomStreams(nStreams, rounds, shards, []string{"a", "b", "c", "d"}) + streams = randomStreams(nStreams, rounds+1, shards, []string{"a", "b", "c", "d"}) start = time.Unix(0, 0) end = time.Unix(0, int64(time.Second*time.Duration(rounds))) step = time.Second From 337c10789354eaa8b0726a2d2f5dbd556415010a Mon Sep 17 00:00:00 2001 From: Cyril Tovena Date: Wed, 31 Mar 2021 11:36:35 +0200 Subject: [PATCH 04/42] wip/ Signed-off-by: Cyril Tovena --- .../shipper/compactor/retention/expiration.go | 31 + .../compactor/retention/expiration_test.go | 45 ++ .../shipper/compactor/retention/index.go | 305 +++++++++ .../shipper/compactor/retention/retention.go | 108 ++++ .../compactor/retention/retention_test.go | 256 ++++++++ .../shipper/compactor/retention/rules.go | 48 ++ .../shipper/compactor/retention/util.go | 49 ++ .../shipper/compactor/retention/util_test.go | 34 + .../shipper/compactor/retention_test.go | 589 ------------------ 9 files changed, 876 insertions(+), 589 deletions(-) create mode 100644 pkg/storage/stores/shipper/compactor/retention/expiration.go create mode 100644 pkg/storage/stores/shipper/compactor/retention/expiration_test.go create mode 100644 pkg/storage/stores/shipper/compactor/retention/index.go create mode 100644 pkg/storage/stores/shipper/compactor/retention/retention.go create mode 100644 pkg/storage/stores/shipper/compactor/retention/retention_test.go create mode 100644 pkg/storage/stores/shipper/compactor/retention/rules.go create mode 100644 pkg/storage/stores/shipper/compactor/retention/util.go create mode 100644 pkg/storage/stores/shipper/compactor/retention/util_test.go delete mode 100644 pkg/storage/stores/shipper/compactor/retention_test.go diff --git a/pkg/storage/stores/shipper/compactor/retention/expiration.go b/pkg/storage/stores/shipper/compactor/retention/expiration.go new file mode 100644 index 0000000000000..2a2f71dae1daa --- /dev/null +++ b/pkg/storage/stores/shipper/compactor/retention/expiration.go @@ -0,0 +1,31 @@ +package retention + +import ( + "github.com/prometheus/common/model" +) + +type ExpirationChecker interface { + Expired(ref *ChunkRef) bool +} + +type expirationChecker struct { + series map[string]StreamRule + TenantRules +} + +func NewExpirationChecker(seriesPerRule map[string]StreamRule, rules TenantRules) ExpirationChecker { + return &expirationChecker{ + series: seriesPerRule, + TenantRules: rules, + } +} + +// Expired tells if a ref chunk is expired based on retention rules. +func (e *expirationChecker) Expired(ref *ChunkRef) bool { + r, ok := e.series[string(ref.SeriesID)] + // if the series matches a stream rules we'll use that. + if ok && r.UserID == string(ref.UserID) { + return ref.From.After(model.Now().Add(r.Duration)) + } + return ref.From.After(model.Now().Add(e.TenantRules.PerTenant(unsafeGetString(ref.UserID)))) +} diff --git a/pkg/storage/stores/shipper/compactor/retention/expiration_test.go b/pkg/storage/stores/shipper/compactor/retention/expiration_test.go new file mode 100644 index 0000000000000..3eb1bf269146f --- /dev/null +++ b/pkg/storage/stores/shipper/compactor/retention/expiration_test.go @@ -0,0 +1,45 @@ +package retention + +import ( + "testing" + "time" + + "github.com/prometheus/common/model" + "github.com/prometheus/prometheus/pkg/labels" + "github.com/stretchr/testify/require" +) + +func Test_expirationChecker_Expired(t *testing.T) { + e := NewExpirationChecker(map[string]StreamRule{ + string(labelsSeriesID(labels.Labels{labels.Label{Name: "foo", Value: "bar"}})): { + UserID: "1", + Duration: 2 * time.Hour, + }, + string(labelsSeriesID(labels.Labels{labels.Label{Name: "foo", Value: "bar"}})): { + UserID: "2", + Duration: 1 * time.Hour, + }, + }, fakeRule{ + tenants: map[string]time.Duration{ + "1": time.Hour, + "2": 24 * time.Hour, + }, + }) + tests := []struct { + name string + ref ChunkRef + want bool + }{ + {"expired tenant", newChunkRef("1", `{foo="buzz"}`, model.Now().Add(2*time.Hour), model.Now().Add(3*time.Hour)), true}, + {"just expired tenant", newChunkRef("1", `{foo="buzz"}`, model.Now().Add(1*time.Hour), model.Now().Add(3*time.Hour)), false}, + {"not expired tenant", newChunkRef("1", `{foo="buzz"}`, model.Now().Add(30*time.Minute), model.Now().Add(3*time.Hour)), false}, + {"not expired tenant by far", newChunkRef("2", `{foo="buzz"}`, model.Now().Add(30*time.Minute), model.Now().Add(3*time.Hour)), false}, + {"expired stream override", newChunkRef("2", `{foo="bar"}`, model.Now().Add(3*time.Hour), model.Now().Add(4*time.Hour)), true}, + {"non expired stream override", newChunkRef("1", `{foo="bar"}`, model.Now().Add(1*time.Hour), model.Now().Add(4*time.Hour)), false}, + } + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + require.Equal(t, tt.want, e.Expired(&tt.ref)) + }) + } +} diff --git a/pkg/storage/stores/shipper/compactor/retention/index.go b/pkg/storage/stores/shipper/compactor/retention/index.go new file mode 100644 index 0000000000000..d0bfb07491d90 --- /dev/null +++ b/pkg/storage/stores/shipper/compactor/retention/index.go @@ -0,0 +1,305 @@ +package retention + +import ( + "bytes" + "errors" + "fmt" + "sort" + "strconv" + "strings" + "time" + + "github.com/cortexproject/cortex/pkg/chunk" + "github.com/cortexproject/cortex/pkg/util/math" + "github.com/prometheus/common/model" + "github.com/prometheus/prometheus/pkg/labels" + "go.etcd.io/bbolt" + + "github.com/grafana/loki/pkg/storage" +) + +const ( + chunkTimeRangeKeyV3 = '3' + separator = "\000" +) + +var QueryParallelism = 100 + +type ChunkRef struct { + UserID []byte + SeriesID []byte + ChunkID []byte + From model.Time + Through model.Time +} + +func (c ChunkRef) String() string { + return fmt.Sprintf("UserID: %s , SeriesID: %s , Time: [%s,%s]", c.UserID, c.SeriesID, c.From, c.Through) +} + +var ErrInvalidIndexKey = errors.New("invalid index key") + +type InvalidIndexKeyError struct { + HashKey string + RangeKey string +} + +func newInvalidIndexKeyError(h, r []byte) InvalidIndexKeyError { + return InvalidIndexKeyError{ + HashKey: string(h), + RangeKey: string(r), + } +} + +func (e InvalidIndexKeyError) Error() string { + return fmt.Sprintf("%s: hash_key:%s range_key:%s", ErrInvalidIndexKey, e.HashKey, e.RangeKey) +} + +func (e InvalidIndexKeyError) Is(target error) bool { + return target == ErrInvalidIndexKey +} + +func parseChunkRef(hashKey, rangeKey []byte) (ChunkRef, bool, error) { + // todo reuse memory + var components [][]byte + components = decodeRangeKey(rangeKey, components) + if len(components) == 0 { + return ChunkRef{}, false, newInvalidIndexKeyError(hashKey, rangeKey) + } + + keyType := components[len(components)-1] + if len(keyType) == 0 || keyType[0] != chunkTimeRangeKeyV3 { + return ChunkRef{}, false, nil + } + chunkID := components[len(components)-2] + + // todo split manually + parts := bytes.Split(chunkID, []byte("/")) + if len(parts) != 2 { + return ChunkRef{}, false, newInvalidIndexKeyError(hashKey, rangeKey) + } + userID := parts[0] + // todo split manually + hexParts := bytes.Split(parts[1], []byte(":")) + if len(hexParts) != 4 { + return ChunkRef{}, false, newInvalidIndexKeyError(hashKey, rangeKey) + } + + from, err := strconv.ParseInt(unsafeGetString(hexParts[1]), 16, 64) + if err != nil { + return ChunkRef{}, false, err + } + through, err := strconv.ParseInt(unsafeGetString(hexParts[2]), 16, 64) + if err != nil { + return ChunkRef{}, false, err + } + + return ChunkRef{ + UserID: userID, + SeriesID: seriesFromHash(hashKey), + From: model.Time(from), + Through: model.Time(through), + ChunkID: chunkID, + }, true, nil +} + +func findSeriesIDsForRules(db *bbolt.DB, config chunk.PeriodConfig, rules []StreamRule) ([][]string, error) { + schema, err := config.CreateSchema() + if err != nil { + return nil, err + } + // cover the whole table. + from, through := config.From.Time, config.From.Time.Add(config.IndexTables.Period) + result := make([][]string, len(rules)) + + for ruleIndex, rule := range rules { + incomingIDs := make(chan []string) + incomingErrors := make(chan error) + + for _, matcher := range rule.Matchers { + go func(matcher *labels.Matcher) { + ids, err := lookupSeriesByMatcher(db, schema, from, through, rule.UserID, matcher) + if err != nil { + incomingErrors <- err + return + } + incomingIDs <- ids + }(&matcher) + } + // intersect. and add to result. + var ids []string + var lastErr error + var initialized bool + for i := 0; i < len(rule.Matchers); i++ { + select { + case incoming := <-incomingIDs: + if !initialized { + ids = incoming + initialized = true + } else { + ids = intersectStrings(ids, incoming) + } + case err := <-incomingErrors: + lastErr = err + } + } + if lastErr != nil { + return nil, err + } + result[ruleIndex] = ids + } + + return result, nil +} + +func lookupSeriesByMatcher( + db *bbolt.DB, + schema chunk.BaseSchema, + from, through model.Time, + userID string, + matcher *labels.Matcher) ([]string, error) { + queries, err := schema.GetReadQueriesForMetricLabelValue( + from, through, userID, "logs", matcher.Name, matcher.Value) + if err != nil { + return nil, err + } + if len(queries) == 0 { + return nil, nil + } + if len(queries) == 1 { + return lookupSeriesByQuery(db, queries[0]) + } + queue := make(chan chunk.IndexQuery) + incomingResult := make(chan struct { + ids []string + err error + }) + n := math.Min(len(queries), QueryParallelism) + for i := 0; i < n; i++ { + go func() { + for { + query, ok := <-queue + if !ok { + return + } + res, err := lookupSeriesByQuery(db, query) + incomingResult <- struct { + ids []string + err error + }{res, err} + } + }() + } + go func() { + for _, query := range queries { + queue <- query + } + close(queue) + }() + + // Now receive all the results. + var ids []string + var lastErr error + for i := 0; i < len(queries); i++ { + res := <-incomingResult + if res.err != nil { + lastErr = res.err + continue + } + ids = append(ids, res.ids...) + } + sort.Strings(ids) + ids = uniqueStrings(ids) + return ids, lastErr +} + +func lookupSeriesByQuery(db *bbolt.DB, query chunk.IndexQuery) ([]string, error) { + start := []byte(query.HashValue + separator + string(query.RangeValueStart)) + rowPrefix := []byte(query.HashValue + separator) + var res []string + var components [][]byte + err := db.View(func(tx *bbolt.Tx) error { + bucket := tx.Bucket(bucketName) + if bucket == nil { + return nil + } + c := bucket.Cursor() + for k, v := c.Seek(start); k != nil; k, v = c.Next() { + // technically we can run regex that are not matching empty. + if len(query.ValueEqual) > 0 && !bytes.Equal(v, query.ValueEqual) { + continue + } + if !bytes.HasPrefix(k, rowPrefix) { + break + } + // parse series ID and add to res + _, r := decodeKey(k) + + components = decodeRangeKey(r, components) + if len(components) != 4 { + continue + } + // we store in label entries range keys: label hash value | seriesID | empty | type. + // and we want the seriesID + res = append(res, string(components[len(components)-3])) + } + return nil + }) + + return res, err +} + +func schemaPeriodForTable(config storage.SchemaConfig, tableName string) (chunk.PeriodConfig, bool) { + for _, schema := range config.Configs { + periodIndex, err := strconv.ParseInt(strings.TrimPrefix(tableName, schema.IndexTables.Prefix), 10, 64) + if err != nil { + continue + } + periodSecs := int64((schema.IndexTables.Period) / time.Second) + if periodIndex == schema.From.Time.Unix()/periodSecs { + return schema, true + } + } + return chunk.PeriodConfig{}, false +} + +func seriesFromHash(h []byte) (seriesID []byte) { + var index int + for i := range h { + if h[i] == ':' { + index++ + } + if index == 2 { + seriesID = h[i+1:] + return + } + } + return +} + +func decodeKey(k []byte) (hashValue, rangeValue []byte) { + // hashValue + 0 + string(rangeValue) + for i := range k { + if k[i] == 0 { + hashValue = k[:i] + rangeValue = k[i+1:] + return + } + } + return +} + +func decodeRangeKey(value []byte, components [][]byte) [][]byte { + components = components[:0] + i, j := 0, 0 + for j < len(value) { + if value[j] != 0 { + j++ + continue + } + components = append(components, value[i:j]) + j++ + i = j + } + return components +} diff --git a/pkg/storage/stores/shipper/compactor/retention/retention.go b/pkg/storage/stores/shipper/compactor/retention/retention.go new file mode 100644 index 0000000000000..6f2fcd965f8a8 --- /dev/null +++ b/pkg/storage/stores/shipper/compactor/retention/retention.go @@ -0,0 +1,108 @@ +package retention + +import ( + "fmt" + "time" + + "github.com/cortexproject/cortex/pkg/chunk" + "github.com/prometheus/common/model" + "go.etcd.io/bbolt" +) + +var bucketName = []byte("index") + +// todo test double open db file. +// markForDelete delete index entries for expired chunk in `in` and add chunkid to delete in `marker`. +// All of this inside a single transaction. +func markForDelete(in, marker *bbolt.DB, expiration ExpirationChecker, buckets []string) error { + return in.Update(func(inTx *bbolt.Tx) error { + return marker.Update(func(outTx *bbolt.Tx) error { + bucket := inTx.Bucket(bucketName) + if bucket == nil { + return nil + } + bucketOut, err := outTx.CreateBucket(bucketName) + if err != nil { + return err + } + // Phase 1 we mark chunkID that needs to be deleted in marker DB + c := bucket.Cursor() + for k, _ := c.First(); k != nil; k, _ = c.Next() { + ref, ok, err := parseChunkRef(decodeKey(k)) + if err != nil { + return err + } + // skiping anything else than chunk index entries. + if !ok { + continue + } + if expiration.Expired(&ref) { + if err := bucketOut.Put(ref.ChunkID, ref.SeriesID); err != nil { + return err + } + if err := c.Delete(); err != nil { + return err + } + } + } + // Phase 2 verify series that have marked chunks have still other chunks in the index. + // If not this means we can delete labels index entries for the given series. + cOut := bucketOut.Cursor() + for chunkID, seriesID := cOut.First(); chunkID != nil; chunkID, seriesID = cOut.Next() { + // for all buckets if seek to bucket.hashKey + ":" + string(seriesID) is not nil we still have chunks. + var found bool + for _, bucket := range buckets { + if key, _ := c.Seek([]byte(bucket + ":" + string(seriesID))); key != nil { + found = true + break + } + } + // we need to delete all index label entry for this series. + if !found { + // entries := []IndexEntry{ + // // Entry for metricName -> seriesID + // { + // TableName: bucket.tableName, + // HashValue: bucket.hashKey + ":" + metricName, + // RangeValue: encodeRangeKey(seriesRangeKeyV1, seriesID, nil, nil), + // Value: empty, + // }, + // } + + // // Entries for metricName:labelName -> hash(value):seriesID + // // We use a hash of the value to limit its length. + // for _, v := range labels { + // if v.Name == model.MetricNameLabel { + // continue + // } + // valueHash := sha256bytes(v.Value) + // entries = append(entries, IndexEntry{ + // TableName: bucket.tableName, + // HashValue: fmt.Sprintf("%s:%s:%s", bucket.hashKey, metricName, v.Name), + // RangeValue: encodeRangeKey(labelSeriesRangeKeyV1, valueHash, seriesID, nil), + // Value: []byte(v.Value), + // }) + // } + } + + } + return nil + }) + }) +} + +func allBucketsHashes(config chunk.PeriodConfig, userID string) []string { + return bucketsHashes(config.From.Time, config.From.Add(config.IndexTables.Period), config, userID) +} + +func bucketsHashes(from, through model.Time, config chunk.PeriodConfig, userID string) []string { + var ( + fromDay = from.Unix() / int64(config.IndexTables.Period/time.Second) + throughDay = through.Unix() / int64(config.IndexTables.Period/time.Second) + result = []string{} + ) + for i := fromDay; i <= throughDay; i++ { + result = append(result, fmt.Sprintf("%s:d%d", userID, i)) + } + return result +} diff --git a/pkg/storage/stores/shipper/compactor/retention/retention_test.go b/pkg/storage/stores/shipper/compactor/retention/retention_test.go new file mode 100644 index 0000000000000..f7ad4f2975efe --- /dev/null +++ b/pkg/storage/stores/shipper/compactor/retention/retention_test.go @@ -0,0 +1,256 @@ +package retention + +import ( + "context" + "crypto/sha256" + "encoding/base64" + "fmt" + "io/ioutil" + "os" + "path/filepath" + "strconv" + "strings" + "testing" + "time" + + "github.com/cortexproject/cortex/pkg/chunk" + "github.com/cortexproject/cortex/pkg/chunk/local" + cortex_storage "github.com/cortexproject/cortex/pkg/chunk/storage" + "github.com/cortexproject/cortex/pkg/ingester/client" + util_log "github.com/cortexproject/cortex/pkg/util/log" + "github.com/prometheus/common/model" + "github.com/prometheus/prometheus/pkg/labels" + "github.com/stretchr/testify/require" + "go.etcd.io/bbolt" + + "github.com/grafana/loki/pkg/chunkenc" + "github.com/grafana/loki/pkg/logproto" + "github.com/grafana/loki/pkg/storage" + shipper_util "github.com/grafana/loki/pkg/storage/stores/shipper/util" + "github.com/grafana/loki/pkg/util/validation" +) + +func Test_Retention(t *testing.T) { + indexDir, err := ioutil.TempDir("", "boltdb_test") + require.Nil(t, err) + + chunkDir, err := ioutil.TempDir("", "chunk_test") + require.Nil(t, err) + + defer func() { + require.NoError(t, os.RemoveAll(indexDir)) + require.NoError(t, os.RemoveAll(chunkDir)) + }() + limits, err := validation.NewOverrides(validation.Limits{}, nil) + require.NoError(t, err) + + schemaCfg := storage.SchemaConfig{ + SchemaConfig: chunk.SchemaConfig{ + Configs: []chunk.PeriodConfig{ + { + From: chunk.DayTime{Time: model.Earliest}, + IndexType: "boltdb", + ObjectType: "filesystem", + Schema: "v9", + IndexTables: chunk.PeriodicTableConfig{ + Prefix: "index_", + Period: time.Hour * 24, + }, + RowShards: 16, + }, + { + From: chunk.DayTime{Time: model.Earliest.Add(25 * time.Hour)}, + IndexType: "boltdb", + ObjectType: "filesystem", + Schema: "v11", + IndexTables: chunk.PeriodicTableConfig{ + Prefix: "index_", + Period: time.Hour * 24, + }, + RowShards: 16, + }, + }, + }, + } + require.NoError(t, schemaCfg.SchemaConfig.Validate()) + + config := storage.Config{ + Config: cortex_storage.Config{ + BoltDBConfig: local.BoltDBConfig{ + Directory: indexDir, + }, + FSConfig: local.FSConfig{ + Directory: chunkDir, + }, + }, + } + chunkStore, err := cortex_storage.NewStore( + config.Config, + chunk.StoreConfig{}, + schemaCfg.SchemaConfig, + limits, + nil, + nil, + util_log.Logger, + ) + require.NoError(t, err) + + store, err := storage.NewStore(config, schemaCfg, chunkStore, nil) + require.NoError(t, err) + + require.NoError(t, store.Put(context.TODO(), []chunk.Chunk{ + createChunk(t, "1", labels.Labels{labels.Label{Name: "foo", Value: "bar"}}, model.Earliest, model.Earliest.Add(1*time.Hour)), + createChunk(t, "2", labels.Labels{labels.Label{Name: "foo", Value: "buzz"}}, model.Earliest.Add(26*time.Hour), model.Earliest.Add(27*time.Hour)), + })) + + store.Stop() + + indexFilesInfo, err := ioutil.ReadDir(indexDir) + require.NoError(t, err) + + retentionRules := fakeRule{ + streams: []StreamRule{ + { + UserID: "1", + Matchers: []labels.Matcher{ + { + Type: labels.MatchEqual, + Name: "foo", + Value: "bar", + }, + }, + }, + }, + } + + // 1- Get all series ID for given retention per stream.... + // 2 - Delete from index and Mark for delete all chunk based on retention with seriesID/tenantID. + // 3 - Seek chunk entries via series id for each series and verify if we still have chunk. + // 4 - Delete Label entries for empty series with empty chunk entries. + + // For 1. only equality matcher are OK so we can use GetReadMetricLabelValueQueries + for _, indexFileInfo := range indexFilesInfo { + db, err := shipper_util.SafeOpenBoltdbFile(filepath.Join(indexDir, indexFileInfo.Name())) + fmt.Fprintf(os.Stdout, "Opening Table %s\n", indexFileInfo.Name()) + require.NoError(t, err) + + // 1 - Get all series ID for given retention per stream.... + + // 1.1 find the schema for this table + + currentSchema, ok := schemaPeriodForTable(schemaCfg, indexFileInfo.Name()) + if !ok { + fmt.Fprintf(os.Stdout, "Could not find Schema for Table %s\n", indexFileInfo.Name()) + continue + } + fmt.Fprintf(os.Stdout, "Found Schema for Table %s => %+v\n", indexFileInfo.Name(), currentSchema) + ids, err := findSeriesIDsForRules(db, currentSchema, retentionRules.PerStream()) + require.NoError(t, err) + fmt.Fprintf(os.Stdout, "Found IDS for rules %+v\n", ids) + require.NoError(t, + db.Update(func(tx *bbolt.Tx) error { + return tx.Bucket(bucketName).ForEach(func(k, v []byte) error { + ref, ok, err := parseChunkRef(decodeKey(k)) + if err != nil { + return err + } + if ok { + fmt.Fprintf(os.Stdout, "%+v\n", ref) + } + return nil + }) + })) + } +} + +func createChunk(t testing.TB, userID string, lbs labels.Labels, from model.Time, through model.Time) chunk.Chunk { + t.Helper() + const ( + targetSize = 1500 * 1024 + blockSize = 256 * 1024 + ) + labelsBuilder := labels.NewBuilder(lbs) + labelsBuilder.Set(labels.MetricName, "logs") + metric := labelsBuilder.Labels() + fp := client.Fingerprint(lbs) + chunkEnc := chunkenc.NewMemChunk(chunkenc.EncSnappy, blockSize, targetSize) + + for ts := from; ts.Before(through); ts = ts.Add(1 * time.Minute) { + require.NoError(t, chunkEnc.Append(&logproto.Entry{ + Timestamp: ts.Time(), + Line: ts.String(), + })) + } + c := chunk.NewChunk(userID, fp, metric, chunkenc.NewFacade(chunkEnc, blockSize, targetSize), from, through) + require.NoError(t, c.Encode()) + return c +} + +func labelsSeriesID(ls labels.Labels) []byte { + h := sha256.Sum256([]byte(labelsString(ls))) + return encodeBase64Bytes(h[:]) +} + +func encodeBase64Bytes(bytes []byte) []byte { + encodedLen := base64.RawStdEncoding.EncodedLen(len(bytes)) + encoded := make([]byte, encodedLen) + base64.RawStdEncoding.Encode(encoded, bytes) + return encoded +} + +// Backwards-compatible with model.Metric.String() +func labelsString(ls labels.Labels) string { + metricName := ls.Get(labels.MetricName) + if metricName != "" && len(ls) == 1 { + return metricName + } + var b strings.Builder + b.Grow(1000) + + b.WriteString(metricName) + b.WriteByte('{') + i := 0 + for _, l := range ls { + if l.Name == labels.MetricName { + continue + } + if i > 0 { + b.WriteByte(',') + b.WriteByte(' ') + } + b.WriteString(l.Name) + b.WriteByte('=') + var buf [1000]byte + b.Write(strconv.AppendQuote(buf[:0], l.Value)) + i++ + } + b.WriteByte('}') + + return b.String() +} + +func Test_AllBucketHash(t *testing.T) { + fmt.Fprintf(os.Stdout, "BucketHash %+v", allBucketsHashes(chunk.PeriodConfig{ + From: chunk.DayTime{Time: model.Now()}, + IndexType: "boltdb", + ObjectType: "filesystem", + Schema: "v9", + IndexTables: chunk.PeriodicTableConfig{ + Prefix: "index_", + Period: time.Hour * 24, + }, + RowShards: 16, + }, "foo")) + + fmt.Fprintf(os.Stdout, "BucketHash %+v", bucketsHashes(model.Now().Add(-2*time.Hour), model.Now().Add(48*time.Hour), chunk.PeriodConfig{ + From: chunk.DayTime{Time: model.Now()}, + IndexType: "boltdb", + ObjectType: "filesystem", + Schema: "v9", + IndexTables: chunk.PeriodicTableConfig{ + Prefix: "index_", + Period: time.Hour * 24, + }, + RowShards: 16, + }, "foo")) +} diff --git a/pkg/storage/stores/shipper/compactor/retention/rules.go b/pkg/storage/stores/shipper/compactor/retention/rules.go new file mode 100644 index 0000000000000..8462204569b72 --- /dev/null +++ b/pkg/storage/stores/shipper/compactor/retention/rules.go @@ -0,0 +1,48 @@ +package retention + +import ( + "time" + + "github.com/prometheus/prometheus/pkg/labels" +) + +// getSeriesPerRule merges rules per seriesID, if multiple series matches a rules, the strongest weight wins. +// Otherwise the shorter retention wins. +func getSeriesPerRule(series [][]string, rules []StreamRule) map[string]*StreamRule { + res := map[string]*StreamRule{} + for i, seriesPerRules := range series { + for _, series := range seriesPerRules { + r, ok := res[series] + newRule := rules[i] + if ok { + // we already have a rules for this series. + if newRule.Weight > r.Weight { + res[series] = &newRule + } + if newRule.Weight == r.Weight && newRule.Duration < r.Duration { + res[series] = &newRule + } + continue + } + res[series] = &newRule + } + } + return res +} + +type StreamRule struct { + Matchers []labels.Matcher + Duration time.Duration + // in case a series matches multiple Rules takes the one with higher weight or the first + Weight int + UserID string +} + +type Rules interface { + TenantRules + PerStream() []StreamRule +} + +type TenantRules interface { + PerTenant(userID string) time.Duration +} diff --git a/pkg/storage/stores/shipper/compactor/retention/util.go b/pkg/storage/stores/shipper/compactor/retention/util.go new file mode 100644 index 0000000000000..97904fa61b8a5 --- /dev/null +++ b/pkg/storage/stores/shipper/compactor/retention/util.go @@ -0,0 +1,49 @@ +package retention + +import ( + "unsafe" +) + +// unsafeGetString is like yolostring but with a meaningful name +func unsafeGetString(buf []byte) string { + return *((*string)(unsafe.Pointer(&buf))) +} + +func uniqueStrings(cs []string) []string { + if len(cs) == 0 { + return []string{} + } + + result := make([]string, 1, len(cs)) + result[0] = cs[0] + i, j := 0, 1 + for j < len(cs) { + if result[i] == cs[j] { + j++ + continue + } + result = append(result, cs[j]) + i++ + j++ + } + return result +} + +func intersectStrings(left, right []string) []string { + var ( + i, j = 0, 0 + result = []string{} + ) + for i < len(left) && j < len(right) { + if left[i] == right[j] { + result = append(result, left[i]) + } + + if left[i] < right[j] { + i++ + } else { + j++ + } + } + return result +} diff --git a/pkg/storage/stores/shipper/compactor/retention/util_test.go b/pkg/storage/stores/shipper/compactor/retention/util_test.go new file mode 100644 index 0000000000000..dc74624366f26 --- /dev/null +++ b/pkg/storage/stores/shipper/compactor/retention/util_test.go @@ -0,0 +1,34 @@ +package retention + +import ( + "time" + + "github.com/grafana/loki/pkg/logql" + "github.com/prometheus/common/model" +) + +type fakeRule struct { + streams []StreamRule + tenants map[string]time.Duration +} + +func (f fakeRule) PerTenant(userID string) time.Duration { + return f.tenants[userID] +} + +func (f fakeRule) PerStream() []StreamRule { + return f.streams +} + +func newChunkRef(userID, labels string, from, through model.Time) ChunkRef { + lbs, err := logql.ParseLabels(labels) + if err != nil { + panic(err) + } + return ChunkRef{ + UserID: []byte(userID), + SeriesID: labelsSeriesID(lbs), + From: from, + Through: through, + } +} diff --git a/pkg/storage/stores/shipper/compactor/retention_test.go b/pkg/storage/stores/shipper/compactor/retention_test.go deleted file mode 100644 index ef20b72361bc2..0000000000000 --- a/pkg/storage/stores/shipper/compactor/retention_test.go +++ /dev/null @@ -1,589 +0,0 @@ -package compactor - -import ( - "bytes" - "context" - "crypto/sha256" - "encoding/base64" - "errors" - "fmt" - "io/ioutil" - "os" - "path/filepath" - "sort" - "strconv" - "strings" - "testing" - "time" - "unsafe" - - "github.com/cortexproject/cortex/pkg/chunk" - "github.com/cortexproject/cortex/pkg/chunk/local" - cortex_storage "github.com/cortexproject/cortex/pkg/chunk/storage" - "github.com/cortexproject/cortex/pkg/ingester/client" - util_log "github.com/cortexproject/cortex/pkg/util/log" - "github.com/cortexproject/cortex/pkg/util/math" - "github.com/grafana/loki/pkg/util/validation" - "github.com/prometheus/common/model" - "github.com/prometheus/prometheus/pkg/labels" - "github.com/stretchr/testify/require" - "go.etcd.io/bbolt" - - "github.com/grafana/loki/pkg/chunkenc" - "github.com/grafana/loki/pkg/logproto" - "github.com/grafana/loki/pkg/storage" - shipper_util "github.com/grafana/loki/pkg/storage/stores/shipper/util" -) - -const ( - chunkTimeRangeKeyV3 = '3' -) - -func Test_Retention(t *testing.T) { - indexDir, err := ioutil.TempDir("", "boltdb_test") - require.Nil(t, err) - - chunkDir, err := ioutil.TempDir("", "chunk_test") - require.Nil(t, err) - - defer func() { - require.NoError(t, os.RemoveAll(indexDir)) - require.NoError(t, os.RemoveAll(chunkDir)) - }() - limits, err := validation.NewOverrides(validation.Limits{}, nil) - require.NoError(t, err) - - schemaCfg := storage.SchemaConfig{ - SchemaConfig: chunk.SchemaConfig{ - Configs: []chunk.PeriodConfig{ - { - From: chunk.DayTime{Time: model.Earliest}, - IndexType: "boltdb", - ObjectType: "filesystem", - Schema: "v9", - IndexTables: chunk.PeriodicTableConfig{ - Prefix: "index_", - Period: time.Hour * 24, - }, - RowShards: 16, - }, - { - From: chunk.DayTime{Time: model.Earliest.Add(25 * time.Hour)}, - IndexType: "boltdb", - ObjectType: "filesystem", - Schema: "v11", - IndexTables: chunk.PeriodicTableConfig{ - Prefix: "index_", - Period: time.Hour * 24, - }, - RowShards: 16, - }, - }, - }, - } - require.NoError(t, schemaCfg.SchemaConfig.Validate()) - - config := storage.Config{ - Config: cortex_storage.Config{ - BoltDBConfig: local.BoltDBConfig{ - Directory: indexDir, - }, - FSConfig: local.FSConfig{ - Directory: chunkDir, - }, - }, - } - chunkStore, err := cortex_storage.NewStore( - config.Config, - chunk.StoreConfig{}, - schemaCfg.SchemaConfig, - limits, - nil, - nil, - util_log.Logger, - ) - require.NoError(t, err) - - store, err := storage.NewStore(config, schemaCfg, chunkStore, nil) - require.NoError(t, err) - - require.NoError(t, store.Put(context.TODO(), []chunk.Chunk{ - createChunk(t, "1", labels.Labels{labels.Label{Name: "foo", Value: "bar"}}, model.Earliest, model.Earliest.Add(1*time.Hour)), - createChunk(t, "2", labels.Labels{labels.Label{Name: "foo", Value: "buzz"}}, model.Earliest.Add(26*time.Hour), model.Earliest.Add(27*time.Hour)), - })) - - store.Stop() - - indexFilesInfo, err := ioutil.ReadDir(indexDir) - require.NoError(t, err) - - retentionRules := fakeRule{ - streams: []StreamRule{ - { - UserID: "1", - Matchers: []labels.Matcher{ - { - Type: labels.MatchEqual, - Name: "foo", - Value: "bar", - }, - }, - }, - }, - } - - // 1- Get all series ID for given retention per stream.... - // 2 - Delete from index and Mark for delete all chunk based on retention with seriesID/tenantID. - // 3 - Seek chunk entries via series id for each series and verify if we still have chunk. - // 4 - Delete Label entries for empty series with empty chunk entries. - - // For 1. only equality matcher are OK so we can use GetReadMetricLabelValueQueries - for _, indexFileInfo := range indexFilesInfo { - db, err := shipper_util.SafeOpenBoltdbFile(filepath.Join(indexDir, indexFileInfo.Name())) - fmt.Fprintf(os.Stdout, "Opening Table %s\n", indexFileInfo.Name()) - require.NoError(t, err) - - // 1 - Get all series ID for given retention per stream.... - - // 1.1 find the schema for this table - - currentSchema, ok := schemaPeriodForTable(schemaCfg, indexFileInfo.Name()) - if !ok { - fmt.Fprintf(os.Stdout, "Could not find Schema for Table %s\n", indexFileInfo.Name()) - continue - } - fmt.Fprintf(os.Stdout, "Found Schema for Table %s => %+v\n", indexFileInfo.Name(), currentSchema) - ids, err := findSeriesIDsForRules(db, currentSchema, retentionRules.PerStream()) - - require.NoError(t, err) - fmt.Fprintf(os.Stdout, "Found IDS for rules %+v\n", ids) - require.NoError(t, - db.Update(func(tx *bbolt.Tx) error { - return tx.Bucket(bucketName).ForEach(func(k, v []byte) error { - ref, ok, err := parseChunkRef(decodeKey(k)) - if err != nil { - return err - } - if ok { - fmt.Fprintf(os.Stdout, "%+v\n", ref) - } - return nil - }) - })) - } -} - -func findSeriesIDsForRules(db *bbolt.DB, config chunk.PeriodConfig, rules []StreamRule) ([][]string, error) { - schema, err := config.CreateSchema() - if err != nil { - return nil, err - } - // cover the whole table. - from, through := config.From.Time, config.From.Time.Add(config.IndexTables.Period) - result := make([][]string, len(rules)) - - for ruleIndex, rule := range rules { - incomingIDs := make(chan []string) - incomingErrors := make(chan error) - - for _, matcher := range rule.Matchers { - go func(matcher *labels.Matcher) { - ids, err := lookupSeriesByMatcher(db, schema, from, through, rule.UserID, matcher) - if err != nil { - incomingErrors <- err - return - } - incomingIDs <- ids - }(&matcher) - } - // intersect. and add to result. - var ids []string - var lastErr error - var initialized bool - for i := 0; i < len(rule.Matchers); i++ { - select { - case incoming := <-incomingIDs: - if !initialized { - ids = incoming - initialized = true - } else { - ids = intersectStrings(ids, incoming) - } - case err := <-incomingErrors: - lastErr = err - } - } - if lastErr != nil { - return nil, err - } - result[ruleIndex] = ids - } - - return result, nil -} - -var QueryParallelism = 100 - -func lookupSeriesByMatcher( - db *bbolt.DB, - schema chunk.BaseSchema, - from, through model.Time, - userID string, - matcher *labels.Matcher) ([]string, error) { - queries, err := schema.GetReadQueriesForMetricLabelValue( - from, through, userID, "logs", matcher.Name, matcher.Value) - if err != nil { - return nil, err - } - if len(queries) == 0 { - return nil, nil - } - if len(queries) == 1 { - return lookupSeriesByQuery(db, queries[0]) - } - queue := make(chan chunk.IndexQuery) - incomingResult := make(chan struct { - ids []string - err error - }) - n := math.Min(len(queries), QueryParallelism) - for i := 0; i < n; i++ { - go func() { - for { - query, ok := <-queue - if !ok { - return - } - res, err := lookupSeriesByQuery(db, query) - incomingResult <- struct { - ids []string - err error - }{res, err} - } - }() - } - go func() { - for _, query := range queries { - queue <- query - } - close(queue) - }() - - // Now receive all the results. - var ids []string - var lastErr error - for i := 0; i < len(queries); i++ { - res := <-incomingResult - if res.err != nil { - lastErr = res.err - continue - } - ids = append(ids, res.ids...) - } - sort.Strings(ids) - ids = uniqueStrings(ids) - return ids, lastErr -} - -func uniqueStrings(cs []string) []string { - if len(cs) == 0 { - return []string{} - } - - result := make([]string, 1, len(cs)) - result[0] = cs[0] - i, j := 0, 1 - for j < len(cs) { - if result[i] == cs[j] { - j++ - continue - } - result = append(result, cs[j]) - i++ - j++ - } - return result -} - -func intersectStrings(left, right []string) []string { - var ( - i, j = 0, 0 - result = []string{} - ) - for i < len(left) && j < len(right) { - if left[i] == right[j] { - result = append(result, left[i]) - } - - if left[i] < right[j] { - i++ - } else { - j++ - } - } - return result -} - -const separator = "\000" - -func lookupSeriesByQuery(db *bbolt.DB, query chunk.IndexQuery) ([]string, error) { - start := []byte(query.HashValue + separator + string(query.RangeValueStart)) - rowPrefix := []byte(query.HashValue + separator) - var res []string - var components [][]byte - err := db.View(func(tx *bbolt.Tx) error { - bucket := tx.Bucket(bucketName) - if bucket == nil { - return nil - } - c := bucket.Cursor() - for k, v := c.Seek(start); k != nil; k, v = c.Next() { - // technically we can run regex that are not matching empty. - if len(query.ValueEqual) > 0 && !bytes.Equal(v, query.ValueEqual) { - continue - } - if !bytes.HasPrefix(k, rowPrefix) { - break - } - // parse series ID and add to res - _, r := decodeKey(k) - - components = decodeRangeKey(r, components) - if len(components) != 4 { - continue - } - // we store in label entries range keys: label hash value | seriesID | empty | type. - // and we want the seriesID - res = append(res, string(components[len(components)-3])) - } - return nil - }) - - return res, err -} - -func schemaPeriodForTable(config storage.SchemaConfig, tableName string) (chunk.PeriodConfig, bool) { - for _, schema := range config.Configs { - periodIndex, err := strconv.ParseInt(strings.TrimPrefix(tableName, schema.IndexTables.Prefix), 10, 64) - if err != nil { - continue - } - periodSecs := int64((schema.IndexTables.Period) / time.Second) - if periodIndex == schema.From.Time.Unix()/periodSecs { - return schema, true - } - } - return chunk.PeriodConfig{}, false -} - -type ChunkRef struct { - UserID []byte - SeriesID []byte - // Fingerprint model.Fingerprint - From model.Time - Through model.Time -} - -func (c ChunkRef) String() string { - return fmt.Sprintf("UserID: %s , SeriesID: %s , Time: [%s,%s]", c.UserID, c.SeriesID, c.From, c.Through) -} - -var ErrInvalidIndexKey = errors.New("invalid index key") - -type InvalidIndexKeyError struct { - HashKey string - RangeKey string -} - -func newInvalidIndexKeyError(h, r []byte) InvalidIndexKeyError { - return InvalidIndexKeyError{ - HashKey: string(h), - RangeKey: string(r), - } -} - -func (e InvalidIndexKeyError) Error() string { - return fmt.Sprintf("%s: hash_key:%s range_key:%s", ErrInvalidIndexKey, e.HashKey, e.RangeKey) -} - -func (e InvalidIndexKeyError) Is(target error) bool { - return target == ErrInvalidIndexKey -} - -func parseChunkRef(hashKey, rangeKey []byte) (ChunkRef, bool, error) { - // todo reuse memory - var components [][]byte - components = decodeRangeKey(rangeKey, components) - if len(components) == 0 { - return ChunkRef{}, false, newInvalidIndexKeyError(hashKey, rangeKey) - } - - keyType := components[len(components)-1] - if len(keyType) == 0 || keyType[0] != chunkTimeRangeKeyV3 { - return ChunkRef{}, false, nil - } - chunkID := components[len(components)-2] - - // todo split manually - parts := bytes.Split(chunkID, []byte("/")) - if len(parts) != 2 { - return ChunkRef{}, false, newInvalidIndexKeyError(hashKey, rangeKey) - } - userID := parts[0] - // todo split manually - hexParts := bytes.Split(parts[1], []byte(":")) - if len(hexParts) != 4 { - return ChunkRef{}, false, newInvalidIndexKeyError(hashKey, rangeKey) - } - - from, err := strconv.ParseInt(unsafeGetString(hexParts[1]), 16, 64) - if err != nil { - return ChunkRef{}, false, err - } - through, err := strconv.ParseInt(unsafeGetString(hexParts[2]), 16, 64) - if err != nil { - return ChunkRef{}, false, err - } - - return ChunkRef{ - UserID: userID, - SeriesID: seriesFromHash(hashKey), - From: model.Time(from), - Through: model.Time(through), - }, true, nil -} - -func unsafeGetString(buf []byte) string { - return *((*string)(unsafe.Pointer(&buf))) -} - -func seriesFromHash(h []byte) (seriesID []byte) { - var index int - for i := range h { - if h[i] == ':' { - index++ - } - if index == 2 { - seriesID = h[i+1:] - return - } - } - return -} - -func decodeKey(k []byte) (hashValue, rangeValue []byte) { - // hashValue + 0 + string(rangeValue) - for i := range k { - if k[i] == 0 { - hashValue = k[:i] - rangeValue = k[i+1:] - return - } - } - return -} - -func decodeRangeKey(value []byte, components [][]byte) [][]byte { - components = components[:0] - i, j := 0, 0 - for j < len(value) { - if value[j] != 0 { - j++ - continue - } - components = append(components, value[i:j]) - j++ - i = j - } - return components -} - -func createChunk(t testing.TB, userID string, lbs labels.Labels, from model.Time, through model.Time) chunk.Chunk { - t.Helper() - const ( - targetSize = 1500 * 1024 - blockSize = 256 * 1024 - ) - labelsBuilder := labels.NewBuilder(lbs) - labelsBuilder.Set(labels.MetricName, "logs") - metric := labelsBuilder.Labels() - fp := client.Fingerprint(lbs) - chunkEnc := chunkenc.NewMemChunk(chunkenc.EncSnappy, blockSize, targetSize) - - for ts := from; ts.Before(through); ts = ts.Add(1 * time.Minute) { - require.NoError(t, chunkEnc.Append(&logproto.Entry{ - Timestamp: ts.Time(), - Line: ts.String(), - })) - } - c := chunk.NewChunk(userID, fp, metric, chunkenc.NewFacade(chunkEnc, blockSize, targetSize), from, through) - require.NoError(t, c.Encode()) - return c -} - -func labelsSeriesID(ls labels.Labels) []byte { - h := sha256.Sum256([]byte(labelsString(ls))) - return encodeBase64Bytes(h[:]) -} - -func encodeBase64Bytes(bytes []byte) []byte { - encodedLen := base64.RawStdEncoding.EncodedLen(len(bytes)) - encoded := make([]byte, encodedLen) - base64.RawStdEncoding.Encode(encoded, bytes) - return encoded -} - -// Backwards-compatible with model.Metric.String() -func labelsString(ls labels.Labels) string { - metricName := ls.Get(labels.MetricName) - if metricName != "" && len(ls) == 1 { - return metricName - } - var b strings.Builder - b.Grow(1000) - - b.WriteString(metricName) - b.WriteByte('{') - i := 0 - for _, l := range ls { - if l.Name == labels.MetricName { - continue - } - if i > 0 { - b.WriteByte(',') - b.WriteByte(' ') - } - b.WriteString(l.Name) - b.WriteByte('=') - var buf [1000]byte - b.Write(strconv.AppendQuote(buf[:0], l.Value)) - i++ - } - b.WriteByte('}') - - return b.String() -} - -type StreamRule struct { - Matchers []labels.Matcher - Duration time.Duration - UserID string -} - -type Rules interface { - PerTenant(userID string) time.Duration - PerStream() []StreamRule -} - -type fakeRule struct { - streams []StreamRule - tenants map[string]time.Duration -} - -func (f fakeRule) PerTenant(userID string) time.Duration { - return f.tenants[userID] -} - -func (f fakeRule) PerStream() []StreamRule { - return f.streams -} From a68cb4c557137dd5bb9db85038d8415612762deb Mon Sep 17 00:00:00 2001 From: Cyril Tovena Date: Wed, 7 Apr 2021 00:17:14 +0200 Subject: [PATCH 05/42] We're parsing the index label now. Signed-off-by: Cyril Tovena --- .../shipper/compactor/retention/index.go | 24 +-- .../shipper/compactor/retention/retention.go | 167 ++++++++++++------ .../compactor/retention/retention_test.go | 6 + .../shipper/compactor/retention/util.go | 20 +++ 4 files changed, 156 insertions(+), 61 deletions(-) diff --git a/pkg/storage/stores/shipper/compactor/retention/index.go b/pkg/storage/stores/shipper/compactor/retention/index.go index d0bfb07491d90..2c07784982cc4 100644 --- a/pkg/storage/stores/shipper/compactor/retention/index.go +++ b/pkg/storage/stores/shipper/compactor/retention/index.go @@ -19,8 +19,12 @@ import ( ) const ( - chunkTimeRangeKeyV3 = '3' - separator = "\000" + chunkTimeRangeKeyV3 = '3' + metricNameRangeKeyV1 = '6' + seriesRangeKeyV1 = '7' + labelSeriesRangeKeyV1 = '8' + labelNamesRangeKeyV1 = '9' + separator = "\000" ) var QueryParallelism = 100 @@ -59,42 +63,42 @@ func (e InvalidIndexKeyError) Is(target error) bool { return target == ErrInvalidIndexKey } -func parseChunkRef(hashKey, rangeKey []byte) (ChunkRef, bool, error) { +func parseChunkRef(hashKey, rangeKey []byte) (*ChunkRef, bool, error) { // todo reuse memory var components [][]byte components = decodeRangeKey(rangeKey, components) if len(components) == 0 { - return ChunkRef{}, false, newInvalidIndexKeyError(hashKey, rangeKey) + return nil, false, newInvalidIndexKeyError(hashKey, rangeKey) } keyType := components[len(components)-1] if len(keyType) == 0 || keyType[0] != chunkTimeRangeKeyV3 { - return ChunkRef{}, false, nil + return nil, false, nil } chunkID := components[len(components)-2] // todo split manually parts := bytes.Split(chunkID, []byte("/")) if len(parts) != 2 { - return ChunkRef{}, false, newInvalidIndexKeyError(hashKey, rangeKey) + return nil, false, newInvalidIndexKeyError(hashKey, rangeKey) } userID := parts[0] // todo split manually hexParts := bytes.Split(parts[1], []byte(":")) if len(hexParts) != 4 { - return ChunkRef{}, false, newInvalidIndexKeyError(hashKey, rangeKey) + return nil, false, newInvalidIndexKeyError(hashKey, rangeKey) } from, err := strconv.ParseInt(unsafeGetString(hexParts[1]), 16, 64) if err != nil { - return ChunkRef{}, false, err + return nil, false, err } through, err := strconv.ParseInt(unsafeGetString(hexParts[2]), 16, 64) if err != nil { - return ChunkRef{}, false, err + return nil, false, err } - return ChunkRef{ + return &ChunkRef{ UserID: userID, SeriesID: seriesFromHash(hashKey), From: model.Time(from), diff --git a/pkg/storage/stores/shipper/compactor/retention/retention.go b/pkg/storage/stores/shipper/compactor/retention/retention.go index 6f2fcd965f8a8..3ab2b118f90aa 100644 --- a/pkg/storage/stores/shipper/compactor/retention/retention.go +++ b/pkg/storage/stores/shipper/compactor/retention/retention.go @@ -1,6 +1,8 @@ package retention import ( + "bytes" + "encoding/binary" "fmt" "time" @@ -9,88 +11,151 @@ import ( "go.etcd.io/bbolt" ) -var bucketName = []byte("index") +var ( + bucketName = []byte("index") + chunkBucket = []byte("chunks") + seriesBucket = []byte("series") + empty = []byte("-") + logMetricName = "logs" +) // todo test double open db file. // markForDelete delete index entries for expired chunk in `in` and add chunkid to delete in `marker`. // All of this inside a single transaction. -func markForDelete(in, marker *bbolt.DB, expiration ExpirationChecker, buckets []string) error { +func markForDelete(in, marker *bbolt.DB, expiration ExpirationChecker, config chunk.PeriodConfig) error { return in.Update(func(inTx *bbolt.Tx) error { return marker.Update(func(outTx *bbolt.Tx) error { bucket := inTx.Bucket(bucketName) if bucket == nil { return nil } - bucketOut, err := outTx.CreateBucket(bucketName) + deleteChunkBucket, err := outTx.CreateBucket(chunkBucket) + if err != nil { + return err + } + deleteSeriesBucket, err := outTx.CreateBucket(seriesBucket) if err != nil { return err } // Phase 1 we mark chunkID that needs to be deleted in marker DB c := bucket.Cursor() - for k, _ := c.First(); k != nil; k, _ = c.Next() { - ref, ok, err := parseChunkRef(decodeKey(k)) - if err != nil { - return err - } - // skiping anything else than chunk index entries. - if !ok { - continue - } - if expiration.Expired(&ref) { - if err := bucketOut.Put(ref.ChunkID, ref.SeriesID); err != nil { + var aliveChunk bool + if err := forAllChunkRef(c, func(ref *ChunkRef) error { + if expiration.Expired(ref) { + if err := deleteChunkBucket.Put(ref.ChunkID, empty); err != nil { + return err + } + // todo: we should not overrides series for different userid. just merge both + if err := deleteSeriesBucket.Put(ref.SeriesID, ref.UserID); err != nil { return err } if err := c.Delete(); err != nil { return err } + return nil } + // we found a key that will stay. + aliveChunk = true + return nil + }); err != nil { + return err } - // Phase 2 verify series that have marked chunks have still other chunks in the index. + // shortcircuit: no chunks remaining we can delete everything. + if !aliveChunk { + if err := inTx.DeleteBucket(bucketName); err != nil { + return err + } + return outTx.DeleteBucket(seriesBucket) + } + // Phase 2 verify series that have marked chunks have still other chunks in the index per buckets. // If not this means we can delete labels index entries for the given series. - cOut := bucketOut.Cursor() - for chunkID, seriesID := cOut.First(); chunkID != nil; chunkID, seriesID = cOut.Next() { + seriesCursor := deleteSeriesBucket.Cursor() + Outer: + for seriesID, userID := seriesCursor.First(); seriesID != nil; seriesID, userID = seriesCursor.Next() { // for all buckets if seek to bucket.hashKey + ":" + string(seriesID) is not nil we still have chunks. - var found bool - for _, bucket := range buckets { - if key, _ := c.Seek([]byte(bucket + ":" + string(seriesID))); key != nil { - found = true - break + bucketHashes := allBucketsHashes(config, unsafeGetString(userID)) + for _, bucketHash := range bucketHashes { + if key, _ := c.Seek([]byte(bucketHash + ":" + string(seriesID))); key != nil { + continue Outer + } + // this bucketHash doesn't contains the given series. Let's remove it. + if err := forAllLabelRef(c, bucketHash, seriesID, config, func(keyType rune) error { + if err := c.Delete(); err != nil { + return err + } + return nil + }); err != nil { + return err } } - // we need to delete all index label entry for this series. - if !found { - // entries := []IndexEntry{ - // // Entry for metricName -> seriesID - // { - // TableName: bucket.tableName, - // HashValue: bucket.hashKey + ":" + metricName, - // RangeValue: encodeRangeKey(seriesRangeKeyV1, seriesID, nil, nil), - // Value: empty, - // }, - // } - - // // Entries for metricName:labelName -> hash(value):seriesID - // // We use a hash of the value to limit its length. - // for _, v := range labels { - // if v.Name == model.MetricNameLabel { - // continue - // } - // valueHash := sha256bytes(v.Value) - // entries = append(entries, IndexEntry{ - // TableName: bucket.tableName, - // HashValue: fmt.Sprintf("%s:%s:%s", bucket.hashKey, metricName, v.Name), - // RangeValue: encodeRangeKey(labelSeriesRangeKeyV1, valueHash, seriesID, nil), - // Value: []byte(v.Value), - // }) - // } - } - } - return nil + // we don't need the series bucket anymore. + return outTx.DeleteBucket(seriesBucket) }) }) } +func forAllChunkRef(c *bbolt.Cursor, callback func(ref *ChunkRef) error) error { + for k, _ := c.First(); k != nil; k, _ = c.Next() { + ref, ok, err := parseChunkRef(decodeKey(k)) + if err != nil { + return err + } + // skiping anything else than chunk index entries. + if !ok { + continue + } + if err := callback(ref); err != nil { + return err + } + } + return nil +} + +func forAllLabelRef(c *bbolt.Cursor, bucketHash string, seriesID []byte, config chunk.PeriodConfig, callback func(keyType rune) error) error { + // todo reuse memory and refactor + var ( + prefix string + components [][]byte + seriesIDRead []byte + ) + // todo refactor ParseLabelRef. => keyType,SeriesID + switch config.Schema { + case "v11": + shard := binary.BigEndian.Uint32(seriesID) % config.RowShards + prefix = fmt.Sprintf("%02d:%s:%s", shard, bucketHash, logMetricName) + default: + prefix = fmt.Sprintf("%s:%s", bucketHash, logMetricName) + } + for k, _ := c.Seek([]byte(prefix)); k != nil; k, _ = c.Next() { + _, rv := decodeKey(k) + components = decodeRangeKey(rv, components) + if len(components) > 4 { + continue + } + keyType := components[len(components)-1] + if len(keyType) == 0 { + continue + } + switch keyType[0] { + case labelSeriesRangeKeyV1: + seriesIDRead = components[1] + case seriesRangeKeyV1: + seriesIDRead = components[0] + default: + continue + } + if !bytes.Equal(seriesID, seriesIDRead) { + continue + } + if err := callback(rune(keyType[0])); err != nil { + return err + } + } + + return nil +} + func allBucketsHashes(config chunk.PeriodConfig, userID string) []string { return bucketsHashes(config.From.Time, config.From.Add(config.IndexTables.Period), config, userID) } diff --git a/pkg/storage/stores/shipper/compactor/retention/retention_test.go b/pkg/storage/stores/shipper/compactor/retention/retention_test.go index f7ad4f2975efe..b3008d46b067b 100644 --- a/pkg/storage/stores/shipper/compactor/retention/retention_test.go +++ b/pkg/storage/stores/shipper/compactor/retention/retention_test.go @@ -156,7 +156,13 @@ func Test_Retention(t *testing.T) { } if ok { fmt.Fprintf(os.Stdout, "%+v\n", ref) + return nil } + _, r := decodeKey(k) + components := decodeRangeKey(r, nil) + keyType := components[len(components)-1] + + fmt.Fprintf(os.Stdout, "type:%s \n", keyType) return nil }) })) diff --git a/pkg/storage/stores/shipper/compactor/retention/util.go b/pkg/storage/stores/shipper/compactor/retention/util.go index 97904fa61b8a5..6a12e241cd207 100644 --- a/pkg/storage/stores/shipper/compactor/retention/util.go +++ b/pkg/storage/stores/shipper/compactor/retention/util.go @@ -47,3 +47,23 @@ func intersectStrings(left, right []string) []string { } return result } + +// Build an index key, encoded as multiple parts separated by a 0 byte, with extra space at the end. +func buildRangeValue(extra int, ss ...[]byte) []byte { + length := extra + for _, s := range ss { + length += len(s) + 1 + } + output, i := make([]byte, length), 0 + for _, s := range ss { + i += copy(output[i:], s) + 1 + } + return output +} + +// Encode a complete key including type marker (which goes at the end) +func encodeRangeKey(keyType byte, ss ...[]byte) []byte { + output := buildRangeValue(2, ss...) + output[len(output)-2] = keyType + return output +} From 2ae282d14a03b6a72eedd3d2042c556d4c312f9b Mon Sep 17 00:00:00 2001 From: Cyril Tovena Date: Thu, 8 Apr 2021 13:36:03 +0200 Subject: [PATCH 06/42] Starting to extract interfaces to make the code testable. Signed-off-by: Cyril Tovena --- .../shipper/compactor/retention/index.go | 60 +++++++-- .../compactor/retention/iterato_test.go | 71 +++++++++++ .../shipper/compactor/retention/iterator.go | 65 ++++++++++ .../shipper/compactor/retention/retention.go | 80 +++++------- .../compactor/retention/retention_test.go | 99 ++------------- .../shipper/compactor/retention/rules.go | 10 +- .../compactor/retention/user_series.go | 48 +++++++ .../compactor/retention/user_series_test.go | 35 ++++++ .../shipper/compactor/retention/util_test.go | 119 ++++++++++++++++++ 9 files changed, 435 insertions(+), 152 deletions(-) create mode 100644 pkg/storage/stores/shipper/compactor/retention/iterato_test.go create mode 100644 pkg/storage/stores/shipper/compactor/retention/iterator.go create mode 100644 pkg/storage/stores/shipper/compactor/retention/user_series.go create mode 100644 pkg/storage/stores/shipper/compactor/retention/user_series_test.go diff --git a/pkg/storage/stores/shipper/compactor/retention/index.go b/pkg/storage/stores/shipper/compactor/retention/index.go index 2c07784982cc4..863da1b3931a5 100644 --- a/pkg/storage/stores/shipper/compactor/retention/index.go +++ b/pkg/storage/stores/shipper/compactor/retention/index.go @@ -29,18 +29,6 @@ const ( var QueryParallelism = 100 -type ChunkRef struct { - UserID []byte - SeriesID []byte - ChunkID []byte - From model.Time - Through model.Time -} - -func (c ChunkRef) String() string { - return fmt.Sprintf("UserID: %s , SeriesID: %s , Time: [%s,%s]", c.UserID, c.SeriesID, c.From, c.Through) -} - var ErrInvalidIndexKey = errors.New("invalid index key") type InvalidIndexKeyError struct { @@ -63,8 +51,20 @@ func (e InvalidIndexKeyError) Is(target error) bool { return target == ErrInvalidIndexKey } +type ChunkRef struct { + UserID []byte + SeriesID []byte + ChunkID []byte + From model.Time + Through model.Time +} + +func (c ChunkRef) String() string { + return fmt.Sprintf("UserID: %s , SeriesID: %s , Time: [%s,%s]", c.UserID, c.SeriesID, c.From, c.Through) +} + func parseChunkRef(hashKey, rangeKey []byte) (*ChunkRef, bool, error) { - // todo reuse memory + // todo reuse memory via pool var components [][]byte components = decodeRangeKey(rangeKey, components) if len(components) == 0 { @@ -107,6 +107,39 @@ func parseChunkRef(hashKey, rangeKey []byte) (*ChunkRef, bool, error) { }, true, nil } +type LabelIndexRef struct { + KeyType byte + SeriesID []byte +} + +func parseLabelIndexRef(hashKey, rangeKey []byte) (*LabelIndexRef, bool, error) { + // todo reuse memory via pool + var ( + components [][]byte + seriesID []byte + ) + components = decodeRangeKey(rangeKey, components) + if len(components) < 4 { + return nil, false, newInvalidIndexKeyError(hashKey, rangeKey) + } + keyType := components[len(components)-1] + if len(keyType) == 0 { + return nil, false, nil + } + switch keyType[0] { + case labelSeriesRangeKeyV1: + seriesID = components[1] + case seriesRangeKeyV1: + seriesID = components[0] + default: + return nil, false, nil + } + return &LabelIndexRef{ + KeyType: keyType[0], + SeriesID: seriesID, + }, true, nil +} + func findSeriesIDsForRules(db *bbolt.DB, config chunk.PeriodConfig, rules []StreamRule) ([][]string, error) { schema, err := config.CreateSchema() if err != nil { @@ -281,6 +314,7 @@ func seriesFromHash(h []byte) (seriesID []byte) { return } +// decodeKey decodes hash and range value from a boltdb key. func decodeKey(k []byte) (hashValue, rangeValue []byte) { // hashValue + 0 + string(rangeValue) for i := range k { diff --git a/pkg/storage/stores/shipper/compactor/retention/iterato_test.go b/pkg/storage/stores/shipper/compactor/retention/iterato_test.go new file mode 100644 index 0000000000000..f6c52cc41d86d --- /dev/null +++ b/pkg/storage/stores/shipper/compactor/retention/iterato_test.go @@ -0,0 +1,71 @@ +package retention + +import ( + "context" + "testing" + "time" + + "github.com/cortexproject/cortex/pkg/chunk" + "github.com/prometheus/common/model" + "github.com/prometheus/prometheus/pkg/labels" + "github.com/stretchr/testify/require" + "go.etcd.io/bbolt" +) + +func Test_ChunkIterator(t *testing.T) { + store := newTestStore(t) + defer store.cleanup() + c1 := createChunk(t, "1", labels.Labels{labels.Label{Name: "foo", Value: "bar"}}, model.Earliest, model.Earliest.Add(1*time.Hour)) + c2 := createChunk(t, "2", labels.Labels{labels.Label{Name: "foo", Value: "buzz"}, labels.Label{Name: "bar", Value: "foo"}}, model.Earliest, model.Earliest.Add(1*time.Hour)) + + require.NoError(t, store.Put(context.TODO(), []chunk.Chunk{ + c1, c2, + })) + + store.Stop() + + tables := store.indexTables() + require.Len(t, tables, 1) + var actual []*ChunkRef + err := tables[0].DB.Update(func(tx *bbolt.Tx) error { + it := newBoltdbChunkIndexIterator(tx.Bucket(bucketName)) + for it.Next() { + require.NoError(t, it.Err()) + actual = append(actual, it.Entry()) + // delete the last entry + if len(actual) == 2 { + require.NoError(t, it.Delete()) + } + } + return nil + }) + require.NoError(t, err) + require.Equal(t, []*ChunkRef{ + refFromChunk(c1), + refFromChunk(c2), + }, actual) + + // second pass we delete c2 + actual = actual[:0] + err = tables[0].DB.Update(func(tx *bbolt.Tx) error { + it := newBoltdbChunkIndexIterator(tx.Bucket(bucketName)) + for it.Next() { + actual = append(actual, it.Entry()) + } + return it.Err() + }) + require.NoError(t, err) + require.Equal(t, []*ChunkRef{ + refFromChunk(c1), + }, actual) +} + +func refFromChunk(c chunk.Chunk) *ChunkRef { + return &ChunkRef{ + UserID: []byte(c.UserID), + SeriesID: labelsSeriesID(c.Metric), + ChunkID: []byte(c.ExternalKey()), + From: c.From, + Through: c.Through, + } +} diff --git a/pkg/storage/stores/shipper/compactor/retention/iterator.go b/pkg/storage/stores/shipper/compactor/retention/iterator.go new file mode 100644 index 0000000000000..6452a644b82b2 --- /dev/null +++ b/pkg/storage/stores/shipper/compactor/retention/iterator.go @@ -0,0 +1,65 @@ +package retention + +import ( + "go.etcd.io/bbolt" +) + +var _ ChunkEntryIterator = &boltdbChunkIndexIterator{} + +type ChunkEntryIterator interface { + Next() bool + Entry() *ChunkRef + Delete() error + Err() error +} + +type boltdbChunkIndexIterator struct { + cursor *bbolt.Cursor + current *ChunkRef + first bool + err error +} + +func newBoltdbChunkIndexIterator(bucket *bbolt.Bucket) *boltdbChunkIndexIterator { + return &boltdbChunkIndexIterator{ + cursor: bucket.Cursor(), + first: true, + } +} + +func (b boltdbChunkIndexIterator) Err() error { + return b.err +} + +func (b boltdbChunkIndexIterator) Entry() *ChunkRef { + return b.current +} + +func (b boltdbChunkIndexIterator) Delete() error { + return b.cursor.Delete() +} + +func (b *boltdbChunkIndexIterator) Next() bool { + var key []byte + if b.first { + key, _ = b.cursor.First() + b.first = false + } else { + key, _ = b.cursor.Next() + } + for key != nil { + ref, ok, err := parseChunkRef(decodeKey(key)) + if err != nil { + b.err = err + return false + } + // skips anything else than chunk index entries. + if !ok { + key, _ = b.cursor.Next() + continue + } + b.current = ref + return true + } + return false +} diff --git a/pkg/storage/stores/shipper/compactor/retention/retention.go b/pkg/storage/stores/shipper/compactor/retention/retention.go index 3ab2b118f90aa..c7f620cd8b74a 100644 --- a/pkg/storage/stores/shipper/compactor/retention/retention.go +++ b/pkg/storage/stores/shipper/compactor/retention/retention.go @@ -14,29 +14,37 @@ import ( var ( bucketName = []byte("index") chunkBucket = []byte("chunks") - seriesBucket = []byte("series") empty = []byte("-") logMetricName = "logs" ) +// todo we want to extract interfaces for series iterator and marker + +type MarkerTx interface { + Mark(id []byte) error +} + +// type Marker interface { +// Begin() MarkerTx +// Commit() error +// Rollback() error +// } + // todo test double open db file. // markForDelete delete index entries for expired chunk in `in` and add chunkid to delete in `marker`. // All of this inside a single transaction. func markForDelete(in, marker *bbolt.DB, expiration ExpirationChecker, config chunk.PeriodConfig) error { return in.Update(func(inTx *bbolt.Tx) error { + bucket := inTx.Bucket(bucketName) + if bucket == nil { + return nil + } return marker.Update(func(outTx *bbolt.Tx) error { - bucket := inTx.Bucket(bucketName) - if bucket == nil { - return nil - } deleteChunkBucket, err := outTx.CreateBucket(chunkBucket) if err != nil { return err } - deleteSeriesBucket, err := outTx.CreateBucket(seriesBucket) - if err != nil { - return err - } + seriesMap := newUserSeriesMap() // Phase 1 we mark chunkID that needs to be deleted in marker DB c := bucket.Cursor() var aliveChunk bool @@ -45,10 +53,7 @@ func markForDelete(in, marker *bbolt.DB, expiration ExpirationChecker, config ch if err := deleteChunkBucket.Put(ref.ChunkID, empty); err != nil { return err } - // todo: we should not overrides series for different userid. just merge both - if err := deleteSeriesBucket.Put(ref.SeriesID, ref.UserID); err != nil { - return err - } + seriesMap.Add(ref.SeriesID, ref.UserID) if err := c.Delete(); err != nil { return err } @@ -62,24 +67,19 @@ func markForDelete(in, marker *bbolt.DB, expiration ExpirationChecker, config ch } // shortcircuit: no chunks remaining we can delete everything. if !aliveChunk { - if err := inTx.DeleteBucket(bucketName); err != nil { - return err - } - return outTx.DeleteBucket(seriesBucket) + return inTx.DeleteBucket(bucketName) } // Phase 2 verify series that have marked chunks have still other chunks in the index per buckets. // If not this means we can delete labels index entries for the given series. - seriesCursor := deleteSeriesBucket.Cursor() - Outer: - for seriesID, userID := seriesCursor.First(); seriesID != nil; seriesID, userID = seriesCursor.Next() { + return seriesMap.ForEach(func(seriesID, userID []byte) error { // for all buckets if seek to bucket.hashKey + ":" + string(seriesID) is not nil we still have chunks. bucketHashes := allBucketsHashes(config, unsafeGetString(userID)) for _, bucketHash := range bucketHashes { if key, _ := c.Seek([]byte(bucketHash + ":" + string(seriesID))); key != nil { - continue Outer + return nil } // this bucketHash doesn't contains the given series. Let's remove it. - if err := forAllLabelRef(c, bucketHash, seriesID, config, func(keyType rune) error { + if err := forAllLabelRef(c, bucketHash, seriesID, config, func(_ *LabelIndexRef) error { if err := c.Delete(); err != nil { return err } @@ -88,9 +88,8 @@ func markForDelete(in, marker *bbolt.DB, expiration ExpirationChecker, config ch return err } } - } - // we don't need the series bucket anymore. - return outTx.DeleteBucket(seriesBucket) + return nil + }) }) }) } @@ -101,7 +100,7 @@ func forAllChunkRef(c *bbolt.Cursor, callback func(ref *ChunkRef) error) error { if err != nil { return err } - // skiping anything else than chunk index entries. + // skips anything else than chunk index entries. if !ok { continue } @@ -112,12 +111,10 @@ func forAllChunkRef(c *bbolt.Cursor, callback func(ref *ChunkRef) error) error { return nil } -func forAllLabelRef(c *bbolt.Cursor, bucketHash string, seriesID []byte, config chunk.PeriodConfig, callback func(keyType rune) error) error { +func forAllLabelRef(c *bbolt.Cursor, bucketHash string, seriesID []byte, config chunk.PeriodConfig, callback func(ref *LabelIndexRef) error) error { // todo reuse memory and refactor var ( - prefix string - components [][]byte - seriesIDRead []byte + prefix string ) // todo refactor ParseLabelRef. => keyType,SeriesID switch config.Schema { @@ -128,27 +125,14 @@ func forAllLabelRef(c *bbolt.Cursor, bucketHash string, seriesID []byte, config prefix = fmt.Sprintf("%s:%s", bucketHash, logMetricName) } for k, _ := c.Seek([]byte(prefix)); k != nil; k, _ = c.Next() { - _, rv := decodeKey(k) - components = decodeRangeKey(rv, components) - if len(components) > 4 { - continue - } - keyType := components[len(components)-1] - if len(keyType) == 0 { - continue - } - switch keyType[0] { - case labelSeriesRangeKeyV1: - seriesIDRead = components[1] - case seriesRangeKeyV1: - seriesIDRead = components[0] - default: - continue + ref, ok, err := parseLabelIndexRef(decodeKey(k)) + if err != nil { + return err } - if !bytes.Equal(seriesID, seriesIDRead) { + if !ok || !bytes.Equal(seriesID, ref.SeriesID) { continue } - if err := callback(rune(keyType[0])); err != nil { + if err := callback(ref); err != nil { return err } } diff --git a/pkg/storage/stores/shipper/compactor/retention/retention_test.go b/pkg/storage/stores/shipper/compactor/retention/retention_test.go index b3008d46b067b..cdfb98e06ca7f 100644 --- a/pkg/storage/stores/shipper/compactor/retention/retention_test.go +++ b/pkg/storage/stores/shipper/compactor/retention/retention_test.go @@ -5,19 +5,14 @@ import ( "crypto/sha256" "encoding/base64" "fmt" - "io/ioutil" "os" - "path/filepath" "strconv" "strings" "testing" "time" "github.com/cortexproject/cortex/pkg/chunk" - "github.com/cortexproject/cortex/pkg/chunk/local" - cortex_storage "github.com/cortexproject/cortex/pkg/chunk/storage" "github.com/cortexproject/cortex/pkg/ingester/client" - util_log "github.com/cortexproject/cortex/pkg/util/log" "github.com/prometheus/common/model" "github.com/prometheus/prometheus/pkg/labels" "github.com/stretchr/testify/require" @@ -25,78 +20,11 @@ import ( "github.com/grafana/loki/pkg/chunkenc" "github.com/grafana/loki/pkg/logproto" - "github.com/grafana/loki/pkg/storage" - shipper_util "github.com/grafana/loki/pkg/storage/stores/shipper/util" - "github.com/grafana/loki/pkg/util/validation" ) func Test_Retention(t *testing.T) { - indexDir, err := ioutil.TempDir("", "boltdb_test") - require.Nil(t, err) - - chunkDir, err := ioutil.TempDir("", "chunk_test") - require.Nil(t, err) - - defer func() { - require.NoError(t, os.RemoveAll(indexDir)) - require.NoError(t, os.RemoveAll(chunkDir)) - }() - limits, err := validation.NewOverrides(validation.Limits{}, nil) - require.NoError(t, err) - - schemaCfg := storage.SchemaConfig{ - SchemaConfig: chunk.SchemaConfig{ - Configs: []chunk.PeriodConfig{ - { - From: chunk.DayTime{Time: model.Earliest}, - IndexType: "boltdb", - ObjectType: "filesystem", - Schema: "v9", - IndexTables: chunk.PeriodicTableConfig{ - Prefix: "index_", - Period: time.Hour * 24, - }, - RowShards: 16, - }, - { - From: chunk.DayTime{Time: model.Earliest.Add(25 * time.Hour)}, - IndexType: "boltdb", - ObjectType: "filesystem", - Schema: "v11", - IndexTables: chunk.PeriodicTableConfig{ - Prefix: "index_", - Period: time.Hour * 24, - }, - RowShards: 16, - }, - }, - }, - } - require.NoError(t, schemaCfg.SchemaConfig.Validate()) - - config := storage.Config{ - Config: cortex_storage.Config{ - BoltDBConfig: local.BoltDBConfig{ - Directory: indexDir, - }, - FSConfig: local.FSConfig{ - Directory: chunkDir, - }, - }, - } - chunkStore, err := cortex_storage.NewStore( - config.Config, - chunk.StoreConfig{}, - schemaCfg.SchemaConfig, - limits, - nil, - nil, - util_log.Logger, - ) - require.NoError(t, err) - - store, err := storage.NewStore(config, schemaCfg, chunkStore, nil) - require.NoError(t, err) + store := newTestStore(t) + defer store.cleanup() require.NoError(t, store.Put(context.TODO(), []chunk.Chunk{ createChunk(t, "1", labels.Labels{labels.Label{Name: "foo", Value: "bar"}}, model.Earliest, model.Earliest.Add(1*time.Hour)), @@ -105,9 +33,6 @@ func Test_Retention(t *testing.T) { store.Stop() - indexFilesInfo, err := ioutil.ReadDir(indexDir) - require.NoError(t, err) - retentionRules := fakeRule{ streams: []StreamRule{ { @@ -129,26 +54,28 @@ func Test_Retention(t *testing.T) { // 4 - Delete Label entries for empty series with empty chunk entries. // For 1. only equality matcher are OK so we can use GetReadMetricLabelValueQueries - for _, indexFileInfo := range indexFilesInfo { - db, err := shipper_util.SafeOpenBoltdbFile(filepath.Join(indexDir, indexFileInfo.Name())) - fmt.Fprintf(os.Stdout, "Opening Table %s\n", indexFileInfo.Name()) - require.NoError(t, err) + for _, table := range store.indexTables() { + fmt.Fprintf(os.Stdout, "Opening Table %s\n", table.name) // 1 - Get all series ID for given retention per stream.... // 1.1 find the schema for this table - currentSchema, ok := schemaPeriodForTable(schemaCfg, indexFileInfo.Name()) + currentSchema, ok := schemaPeriodForTable(store.schemaCfg, table.name) if !ok { - fmt.Fprintf(os.Stdout, "Could not find Schema for Table %s\n", indexFileInfo.Name()) + fmt.Fprintf(os.Stdout, "Could not find Schema for Table %s\n", table.name) continue } - fmt.Fprintf(os.Stdout, "Found Schema for Table %s => %+v\n", indexFileInfo.Name(), currentSchema) - ids, err := findSeriesIDsForRules(db, currentSchema, retentionRules.PerStream()) + fmt.Fprintf(os.Stdout, "Found Schema for Table %s => %+v\n", table.name, currentSchema) + ids, err := findSeriesIDsForRules(table.DB, currentSchema, retentionRules.PerStream()) require.NoError(t, err) + _ = NewExpirationChecker(getSeriesPerRule(ids, retentionRules.PerStream()), retentionRules) + + // markForDelete(db, marker*bbolt.DB, checker, currentSchema) + fmt.Fprintf(os.Stdout, "Found IDS for rules %+v\n", ids) require.NoError(t, - db.Update(func(tx *bbolt.Tx) error { + table.DB.Update(func(tx *bbolt.Tx) error { return tx.Bucket(bucketName).ForEach(func(k, v []byte) error { ref, ok, err := parseChunkRef(decodeKey(k)) if err != nil { diff --git a/pkg/storage/stores/shipper/compactor/retention/rules.go b/pkg/storage/stores/shipper/compactor/retention/rules.go index 8462204569b72..9d4679d53a7f2 100644 --- a/pkg/storage/stores/shipper/compactor/retention/rules.go +++ b/pkg/storage/stores/shipper/compactor/retention/rules.go @@ -8,8 +8,8 @@ import ( // getSeriesPerRule merges rules per seriesID, if multiple series matches a rules, the strongest weight wins. // Otherwise the shorter retention wins. -func getSeriesPerRule(series [][]string, rules []StreamRule) map[string]*StreamRule { - res := map[string]*StreamRule{} +func getSeriesPerRule(series [][]string, rules []StreamRule) map[string]StreamRule { + res := map[string]StreamRule{} for i, seriesPerRules := range series { for _, series := range seriesPerRules { r, ok := res[series] @@ -17,14 +17,14 @@ func getSeriesPerRule(series [][]string, rules []StreamRule) map[string]*StreamR if ok { // we already have a rules for this series. if newRule.Weight > r.Weight { - res[series] = &newRule + res[series] = newRule } if newRule.Weight == r.Weight && newRule.Duration < r.Duration { - res[series] = &newRule + res[series] = newRule } continue } - res[series] = &newRule + res[series] = newRule } } return res diff --git a/pkg/storage/stores/shipper/compactor/retention/user_series.go b/pkg/storage/stores/shipper/compactor/retention/user_series.go new file mode 100644 index 0000000000000..200646574817c --- /dev/null +++ b/pkg/storage/stores/shipper/compactor/retention/user_series.go @@ -0,0 +1,48 @@ +package retention + +type userSeries struct { + key []byte + seriesIDLen int +} + +func newUserSeries(seriesID []byte, userID []byte) userSeries { + key := make([]byte, 0, len(seriesID)+len(userID)) + key = append(key, seriesID...) + key = append(key, userID...) + return userSeries{ + key: key, + seriesIDLen: len(seriesID), + } +} + +func (us userSeries) Key() string { + return unsafeGetString(us.key) +} + +func (us userSeries) SeriesID() []byte { + return us.key[:us.seriesIDLen] +} + +func (us userSeries) UserID() []byte { + return us.key[us.seriesIDLen:] +} + +type userSeriesMap map[string]userSeries + +func newUserSeriesMap() userSeriesMap { + return make(userSeriesMap) +} + +func (u userSeriesMap) Add(seriesID []byte, userID []byte) { + us := newUserSeries(seriesID, userID) + u[us.Key()] = us +} + +func (u userSeriesMap) ForEach(callback func(seriesID []byte, userID []byte) error) error { + for _, v := range u { + if err := callback(v.SeriesID(), v.UserID()); err != nil { + return err + } + } + return nil +} diff --git a/pkg/storage/stores/shipper/compactor/retention/user_series_test.go b/pkg/storage/stores/shipper/compactor/retention/user_series_test.go new file mode 100644 index 0000000000000..dc978055c4f3d --- /dev/null +++ b/pkg/storage/stores/shipper/compactor/retention/user_series_test.go @@ -0,0 +1,35 @@ +package retention + +import ( + "sort" + "testing" + + "github.com/stretchr/testify/require" +) + +func Test_UserSeries(t *testing.T) { + m := newUserSeriesMap() + + m.Add([]byte(`series1`), []byte(`user1`)) + m.Add([]byte(`series1`), []byte(`user1`)) + m.Add([]byte(`series1`), []byte(`user2`)) + m.Add([]byte(`series2`), []byte(`user1`)) + m.Add([]byte(`series2`), []byte(`user1`)) + m.Add([]byte(`series2`), []byte(`user2`)) + + keys := []string{} + + err := m.ForEach(func(seriesID, userID []byte) error { + keys = append(keys, string(seriesID)+":"+string(userID)) + return nil + }) + require.NoError(t, err) + require.Len(t, keys, 4) + sort.Strings(keys) + require.Equal(t, []string{ + "series1:user1", + "series1:user2", + "series2:user1", + "series2:user2", + }, keys) +} diff --git a/pkg/storage/stores/shipper/compactor/retention/util_test.go b/pkg/storage/stores/shipper/compactor/retention/util_test.go index dc74624366f26..20604560d1cf6 100644 --- a/pkg/storage/stores/shipper/compactor/retention/util_test.go +++ b/pkg/storage/stores/shipper/compactor/retention/util_test.go @@ -1,10 +1,23 @@ package retention import ( + "io/ioutil" + "os" + "path/filepath" + "testing" "time" + "github.com/cortexproject/cortex/pkg/chunk" + "github.com/cortexproject/cortex/pkg/chunk/local" + cortex_storage "github.com/cortexproject/cortex/pkg/chunk/storage" + util_log "github.com/cortexproject/cortex/pkg/util/log" "github.com/grafana/loki/pkg/logql" + "github.com/grafana/loki/pkg/storage" + shipper_util "github.com/grafana/loki/pkg/storage/stores/shipper/util" + "github.com/grafana/loki/pkg/util/validation" "github.com/prometheus/common/model" + "github.com/stretchr/testify/require" + "go.etcd.io/bbolt" ) type fakeRule struct { @@ -32,3 +45,109 @@ func newChunkRef(userID, labels string, from, through model.Time) ChunkRef { Through: through, } } + +type testStore struct { + storage.Store + indexDir, chunkDir string + schemaCfg storage.SchemaConfig + t *testing.T +} + +func (t *testStore) cleanup() { + t.t.Helper() + require.NoError(t.t, os.RemoveAll(t.indexDir)) + require.NoError(t.t, os.RemoveAll(t.indexDir)) +} + +type table struct { + name string + *bbolt.DB +} + +func (t *testStore) indexTables() []table { + t.t.Helper() + res := []table{} + indexFilesInfo, err := ioutil.ReadDir(t.indexDir) + require.NoError(t.t, err) + for _, indexFileInfo := range indexFilesInfo { + db, err := shipper_util.SafeOpenBoltdbFile(filepath.Join(t.indexDir, indexFileInfo.Name())) + require.NoError(t.t, err) + res = append(res, table{name: indexFileInfo.Name(), DB: db}) + } + return res +} + +func newTestStore(t *testing.T) *testStore { + t.Helper() + indexDir, err := ioutil.TempDir("", "boltdb_test") + require.Nil(t, err) + + chunkDir, err := ioutil.TempDir("", "chunk_test") + require.Nil(t, err) + + defer func() { + }() + limits, err := validation.NewOverrides(validation.Limits{}, nil) + require.NoError(t, err) + + schemaCfg := storage.SchemaConfig{ + SchemaConfig: chunk.SchemaConfig{ + Configs: []chunk.PeriodConfig{ + { + From: chunk.DayTime{Time: model.Earliest}, + IndexType: "boltdb", + ObjectType: "filesystem", + Schema: "v9", + IndexTables: chunk.PeriodicTableConfig{ + Prefix: "index_", + Period: time.Hour * 24, + }, + RowShards: 16, + }, + { + From: chunk.DayTime{Time: model.Earliest.Add(25 * time.Hour)}, + IndexType: "boltdb", + ObjectType: "filesystem", + Schema: "v11", + IndexTables: chunk.PeriodicTableConfig{ + Prefix: "index_", + Period: time.Hour * 24, + }, + RowShards: 16, + }, + }, + }, + } + require.NoError(t, schemaCfg.SchemaConfig.Validate()) + + config := storage.Config{ + Config: cortex_storage.Config{ + BoltDBConfig: local.BoltDBConfig{ + Directory: indexDir, + }, + FSConfig: local.FSConfig{ + Directory: chunkDir, + }, + }, + } + chunkStore, err := cortex_storage.NewStore( + config.Config, + chunk.StoreConfig{}, + schemaCfg.SchemaConfig, + limits, + nil, + nil, + util_log.Logger, + ) + require.NoError(t, err) + + store, err := storage.NewStore(config, schemaCfg, chunkStore, nil) + require.NoError(t, err) + return &testStore{ + indexDir: indexDir, + chunkDir: chunkDir, + t: t, + Store: store, + schemaCfg: schemaCfg, + } +} From 7ca79a8a919bfdf95126f055d88be5ccc92a7959 Mon Sep 17 00:00:00 2001 From: Cyril Tovena Date: Mon, 12 Apr 2021 16:30:57 +0200 Subject: [PATCH 07/42] Work in progress will try to add labels to chunk ref iterator Signed-off-by: Cyril Tovena --- .../shipper/compactor/retention/expiration.go | 12 ++ .../compactor/retention/iterato_test.go | 71 -------- .../shipper/compactor/retention/iterator.go | 153 +++++++++++++++++- .../compactor/retention/iterator_test.go | 75 +++++++++ .../shipper/compactor/retention/retention.go | 69 ++++---- .../shipper/compactor/retention/util.go | 9 ++ .../shipper/compactor/retention/util_test.go | 78 +++++---- 7 files changed, 321 insertions(+), 146 deletions(-) delete mode 100644 pkg/storage/stores/shipper/compactor/retention/iterato_test.go create mode 100644 pkg/storage/stores/shipper/compactor/retention/iterator_test.go diff --git a/pkg/storage/stores/shipper/compactor/retention/expiration.go b/pkg/storage/stores/shipper/compactor/retention/expiration.go index 2a2f71dae1daa..a9590c02f0039 100644 --- a/pkg/storage/stores/shipper/compactor/retention/expiration.go +++ b/pkg/storage/stores/shipper/compactor/retention/expiration.go @@ -8,6 +8,10 @@ type ExpirationChecker interface { Expired(ref *ChunkRef) bool } +// func MultiExpirationCheck(checker ...ExpirationChecker) ExpirationChecker { + +// } + type expirationChecker struct { series map[string]StreamRule TenantRules @@ -29,3 +33,11 @@ func (e *expirationChecker) Expired(ref *ChunkRef) bool { } return ref.From.After(model.Now().Add(e.TenantRules.PerTenant(unsafeGetString(ref.UserID)))) } + +// all chunk ref.... chunkid,seriesid, from,to, userid (labels) (n + 1) +// +// HashValue: fmt.Sprintf("%02d:%s:%s:%s", shard, bucket.hashKey, metricName, v.Name), +// RangeValue: encodeRangeKey(labelSeriesRangeKeyV1, valueHash, seriesID, nil), +// Value: []byte(v.Value), + +// custom stream only {app="foo"} diff --git a/pkg/storage/stores/shipper/compactor/retention/iterato_test.go b/pkg/storage/stores/shipper/compactor/retention/iterato_test.go deleted file mode 100644 index f6c52cc41d86d..0000000000000 --- a/pkg/storage/stores/shipper/compactor/retention/iterato_test.go +++ /dev/null @@ -1,71 +0,0 @@ -package retention - -import ( - "context" - "testing" - "time" - - "github.com/cortexproject/cortex/pkg/chunk" - "github.com/prometheus/common/model" - "github.com/prometheus/prometheus/pkg/labels" - "github.com/stretchr/testify/require" - "go.etcd.io/bbolt" -) - -func Test_ChunkIterator(t *testing.T) { - store := newTestStore(t) - defer store.cleanup() - c1 := createChunk(t, "1", labels.Labels{labels.Label{Name: "foo", Value: "bar"}}, model.Earliest, model.Earliest.Add(1*time.Hour)) - c2 := createChunk(t, "2", labels.Labels{labels.Label{Name: "foo", Value: "buzz"}, labels.Label{Name: "bar", Value: "foo"}}, model.Earliest, model.Earliest.Add(1*time.Hour)) - - require.NoError(t, store.Put(context.TODO(), []chunk.Chunk{ - c1, c2, - })) - - store.Stop() - - tables := store.indexTables() - require.Len(t, tables, 1) - var actual []*ChunkRef - err := tables[0].DB.Update(func(tx *bbolt.Tx) error { - it := newBoltdbChunkIndexIterator(tx.Bucket(bucketName)) - for it.Next() { - require.NoError(t, it.Err()) - actual = append(actual, it.Entry()) - // delete the last entry - if len(actual) == 2 { - require.NoError(t, it.Delete()) - } - } - return nil - }) - require.NoError(t, err) - require.Equal(t, []*ChunkRef{ - refFromChunk(c1), - refFromChunk(c2), - }, actual) - - // second pass we delete c2 - actual = actual[:0] - err = tables[0].DB.Update(func(tx *bbolt.Tx) error { - it := newBoltdbChunkIndexIterator(tx.Bucket(bucketName)) - for it.Next() { - actual = append(actual, it.Entry()) - } - return it.Err() - }) - require.NoError(t, err) - require.Equal(t, []*ChunkRef{ - refFromChunk(c1), - }, actual) -} - -func refFromChunk(c chunk.Chunk) *ChunkRef { - return &ChunkRef{ - UserID: []byte(c.UserID), - SeriesID: labelsSeriesID(c.Metric), - ChunkID: []byte(c.ExternalKey()), - From: c.From, - Through: c.Through, - } -} diff --git a/pkg/storage/stores/shipper/compactor/retention/iterator.go b/pkg/storage/stores/shipper/compactor/retention/iterator.go index 6452a644b82b2..828616282ed8e 100644 --- a/pkg/storage/stores/shipper/compactor/retention/iterator.go +++ b/pkg/storage/stores/shipper/compactor/retention/iterator.go @@ -1,45 +1,54 @@ package retention import ( + "bytes" + "encoding/binary" + "fmt" + + "github.com/cortexproject/cortex/pkg/chunk" "go.etcd.io/bbolt" ) -var _ ChunkEntryIterator = &boltdbChunkIndexIterator{} +var ( + _ ChunkEntryIterator = &chunkIndexIterator{} + _ Series = &series{} +) type ChunkEntryIterator interface { Next() bool Entry() *ChunkRef + // Delete deletes the current entry. Delete() error Err() error } -type boltdbChunkIndexIterator struct { +type chunkIndexIterator struct { cursor *bbolt.Cursor current *ChunkRef first bool err error } -func newBoltdbChunkIndexIterator(bucket *bbolt.Bucket) *boltdbChunkIndexIterator { - return &boltdbChunkIndexIterator{ +func newChunkIndexIterator(bucket *bbolt.Bucket) *chunkIndexIterator { + return &chunkIndexIterator{ cursor: bucket.Cursor(), first: true, } } -func (b boltdbChunkIndexIterator) Err() error { +func (b *chunkIndexIterator) Err() error { return b.err } -func (b boltdbChunkIndexIterator) Entry() *ChunkRef { +func (b *chunkIndexIterator) Entry() *ChunkRef { return b.current } -func (b boltdbChunkIndexIterator) Delete() error { +func (b *chunkIndexIterator) Delete() error { return b.cursor.Delete() } -func (b *boltdbChunkIndexIterator) Next() bool { +func (b *chunkIndexIterator) Next() bool { var key []byte if b.first { key, _ = b.cursor.First() @@ -63,3 +72,131 @@ func (b *boltdbChunkIndexIterator) Next() bool { } return false } + +type Series interface { + Buckets(userID []byte) []Bucket +} + +type Bucket interface { + ContainsChunkFor(seriesID []byte) bool + LabelEntries(seriesID []byte) LabelEntryIterator +} + +type LabelEntryIterator interface { + Next() bool + Entry() *LabelIndexRef + // Delete deletes the current entry. + Delete() error + Err() error +} + +type series struct { + bucket *bbolt.Bucket + config chunk.PeriodConfig +} + +func newSeries(bucket *bbolt.Bucket, config chunk.PeriodConfig) *series { + return &series{ + bucket: bucket, + config: config, + } +} + +func (s *series) Buckets(userID []byte) []Bucket { + bucketHashes := allBucketsHashes(s.config, unsafeGetString(userID)) + res := make([]Bucket, 0, len(bucketHashes)) + for _, h := range bucketHashes { + res = append(res, newBucket(s.bucket.Cursor(), h, s.config)) + } + return res +} + +type bucket struct { + cursor *bbolt.Cursor + hash string + config chunk.PeriodConfig +} + +func newBucket(cursor *bbolt.Cursor, hash string, config chunk.PeriodConfig) *bucket { + return &bucket{ + cursor: cursor, + hash: hash, + config: config, + } +} + +func (b *bucket) ContainsChunkFor(seriesID []byte) bool { + if key, _ := b.cursor.Seek([]byte(b.hash + ":" + string(seriesID))); key != nil { + return true + } + return false +} + +func (b *bucket) LabelEntries(seriesID []byte) LabelEntryIterator { + return newLabelEntryIterator(b, b.keyPrefix(seriesID)) +} + +func (b *bucket) keyPrefix(series []byte) (prefix []byte) { + switch b.config.Schema { + case "v11", "v10": + shard := binary.BigEndian.Uint32(series) % b.config.RowShards + prefix = unsafeGetBytes(fmt.Sprintf("%02d:%s:%s", shard, b.hash, logMetricName)) + default: + prefix = unsafeGetBytes(fmt.Sprintf("%s:%s", b.hash, logMetricName)) + } + return +} + +type labelEntryIterator struct { + *bucket + + current *LabelIndexRef + first bool + err error + prefix []byte +} + +func newLabelEntryIterator(b *bucket, prefix []byte) *labelEntryIterator { + return &labelEntryIterator{ + bucket: b, + first: true, + prefix: prefix, + } +} + +func (it *labelEntryIterator) Err() error { + return it.err +} + +func (it *labelEntryIterator) Entry() *LabelIndexRef { + return it.current +} + +func (it *labelEntryIterator) Delete() error { + return it.cursor.Delete() +} + +func (it *labelEntryIterator) Next() bool { + var key []byte + if it.first { + key, _ = it.cursor.Seek(it.prefix) + it.first = false + } else { + key, _ = it.cursor.Next() + } + for key != nil && bytes.HasPrefix(key, it.prefix) { + ref, ok, err := parseLabelIndexRef(decodeKey(key)) + if err != nil { + it.err = err + return false + } + // skips anything else than labels index entries. + if !ok { + key, _ = it.cursor.Next() + continue + } + it.current = ref + return true + } + return false +} diff --git a/pkg/storage/stores/shipper/compactor/retention/iterator_test.go b/pkg/storage/stores/shipper/compactor/retention/iterator_test.go new file mode 100644 index 0000000000000..8619eecedf7f7 --- /dev/null +++ b/pkg/storage/stores/shipper/compactor/retention/iterator_test.go @@ -0,0 +1,75 @@ +package retention + +import ( + "context" + "testing" + "time" + + "github.com/cortexproject/cortex/pkg/chunk" + "github.com/prometheus/prometheus/pkg/labels" + "github.com/stretchr/testify/require" + "go.etcd.io/bbolt" +) + +func Test_ChunkIterator(t *testing.T) { + for _, tt := range allSchemas { + tt := tt + t.Run(tt.schema, func(t *testing.T) { + store := newTestStore(t) + defer store.cleanup() + c1 := createChunk(t, "1", labels.Labels{labels.Label{Name: "foo", Value: "bar"}}, tt.from, tt.from.Add(1*time.Hour)) + c2 := createChunk(t, "2", labels.Labels{labels.Label{Name: "foo", Value: "buzz"}, labels.Label{Name: "bar", Value: "foo"}}, tt.from, tt.from.Add(1*time.Hour)) + + require.NoError(t, store.Put(context.TODO(), []chunk.Chunk{ + c1, c2, + })) + + store.Stop() + + tables := store.indexTables() + require.Len(t, tables, 1) + var actual []*ChunkRef + err := tables[0].DB.Update(func(tx *bbolt.Tx) error { + it := newChunkIndexIterator(tx.Bucket(bucketName)) + for it.Next() { + require.NoError(t, it.Err()) + actual = append(actual, it.Entry()) + // delete the last entry + if len(actual) == 2 { + require.NoError(t, it.Delete()) + } + } + return nil + }) + require.NoError(t, err) + require.Equal(t, []*ChunkRef{ + refFromChunk(c1), + refFromChunk(c2), + }, actual) + + // second pass we delete c2 + actual = actual[:0] + err = tables[0].DB.Update(func(tx *bbolt.Tx) error { + it := newChunkIndexIterator(tx.Bucket(bucketName)) + for it.Next() { + actual = append(actual, it.Entry()) + } + return it.Err() + }) + require.NoError(t, err) + require.Equal(t, []*ChunkRef{ + refFromChunk(c1), + }, actual) + }) + } +} + +func refFromChunk(c chunk.Chunk) *ChunkRef { + return &ChunkRef{ + UserID: []byte(c.UserID), + SeriesID: labelsSeriesID(c.Metric), + ChunkID: []byte(c.ExternalKey()), + From: c.From, + Through: c.Through, + } +} diff --git a/pkg/storage/stores/shipper/compactor/retention/retention.go b/pkg/storage/stores/shipper/compactor/retention/retention.go index c7f620cd8b74a..69e9961af9339 100644 --- a/pkg/storage/stores/shipper/compactor/retention/retention.go +++ b/pkg/storage/stores/shipper/compactor/retention/retention.go @@ -30,7 +30,7 @@ type MarkerTx interface { // Rollback() error // } -// todo test double open db file. +// todo clean up with interfaces. // markForDelete delete index entries for expired chunk in `in` and add chunkid to delete in `marker`. // All of this inside a single transaction. func markForDelete(in, marker *bbolt.DB, expiration ExpirationChecker, config chunk.PeriodConfig) error { @@ -40,31 +40,39 @@ func markForDelete(in, marker *bbolt.DB, expiration ExpirationChecker, config ch return nil } return marker.Update(func(outTx *bbolt.Tx) error { - deleteChunkBucket, err := outTx.CreateBucket(chunkBucket) - if err != nil { - return err - } + // deleteChunkBucket, err := outTx.CreateBucket(chunkBucket) + // if err != nil { + // return err + // } seriesMap := newUserSeriesMap() // Phase 1 we mark chunkID that needs to be deleted in marker DB c := bucket.Cursor() var aliveChunk bool - if err := forAllChunkRef(c, func(ref *ChunkRef) error { - if expiration.Expired(ref) { - if err := deleteChunkBucket.Put(ref.ChunkID, empty); err != nil { - return err - } - seriesMap.Add(ref.SeriesID, ref.UserID) - if err := c.Delete(); err != nil { - return err - } - return nil - } - // we found a key that will stay. - aliveChunk = true - return nil - }); err != nil { - return err - } + + // it := newBoltdbChunkIndexIterator(bucket) + // for it.Next() { + // if it.Err() != nil { + // return it.Err() + // } + // ref := it.Entry() + // } + // if err := forAllChunkRef(c, func(ref *ChunkRef) error { + // if expiration.Expired(ref) { + // if err := deleteChunkBucket.Put(ref.ChunkID, empty); err != nil { + // return err + // } + // seriesMap.Add(ref.SeriesID, ref.UserID) + // if err := c.Delete(); err != nil { + // return err + // } + // return nil + // } + // // we found a key that will stay. + // aliveChunk = true + // return nil + // }); err != nil { + // return err + // } // shortcircuit: no chunks remaining we can delete everything. if !aliveChunk { return inTx.DeleteBucket(bucketName) @@ -94,23 +102,6 @@ func markForDelete(in, marker *bbolt.DB, expiration ExpirationChecker, config ch }) } -func forAllChunkRef(c *bbolt.Cursor, callback func(ref *ChunkRef) error) error { - for k, _ := c.First(); k != nil; k, _ = c.Next() { - ref, ok, err := parseChunkRef(decodeKey(k)) - if err != nil { - return err - } - // skips anything else than chunk index entries. - if !ok { - continue - } - if err := callback(ref); err != nil { - return err - } - } - return nil -} - func forAllLabelRef(c *bbolt.Cursor, bucketHash string, seriesID []byte, config chunk.PeriodConfig, callback func(ref *LabelIndexRef) error) error { // todo reuse memory and refactor var ( diff --git a/pkg/storage/stores/shipper/compactor/retention/util.go b/pkg/storage/stores/shipper/compactor/retention/util.go index 6a12e241cd207..674fe10141eed 100644 --- a/pkg/storage/stores/shipper/compactor/retention/util.go +++ b/pkg/storage/stores/shipper/compactor/retention/util.go @@ -1,6 +1,7 @@ package retention import ( + "reflect" "unsafe" ) @@ -9,6 +10,14 @@ func unsafeGetString(buf []byte) string { return *((*string)(unsafe.Pointer(&buf))) } +func unsafeGetBytes(s string) []byte { + var buf []byte + p := unsafe.Pointer(&buf) + *(*string)(p) = s + (*reflect.SliceHeader)(p).Cap = len(s) + return buf +} + func uniqueStrings(cs []string) []string { if len(cs) == 0 { return []string{} diff --git a/pkg/storage/stores/shipper/compactor/retention/util_test.go b/pkg/storage/stores/shipper/compactor/retention/util_test.go index 20604560d1cf6..3919a73736c06 100644 --- a/pkg/storage/stores/shipper/compactor/retention/util_test.go +++ b/pkg/storage/stores/shipper/compactor/retention/util_test.go @@ -20,6 +20,56 @@ import ( "go.etcd.io/bbolt" ) +var schemaCfg = storage.SchemaConfig{ + SchemaConfig: chunk.SchemaConfig{ + // we want to test over all supported schema. + Configs: []chunk.PeriodConfig{ + { + From: chunk.DayTime{Time: model.Earliest}, + IndexType: "boltdb", + ObjectType: "filesystem", + Schema: "v9", + IndexTables: chunk.PeriodicTableConfig{ + Prefix: "index_", + Period: time.Hour * 24, + }, + RowShards: 16, + }, + { + From: chunk.DayTime{Time: model.Earliest.Add(25 * time.Hour)}, + IndexType: "boltdb", + ObjectType: "filesystem", + Schema: "v10", + IndexTables: chunk.PeriodicTableConfig{ + Prefix: "index_", + Period: time.Hour * 24, + }, + RowShards: 16, + }, + { + From: chunk.DayTime{Time: model.Earliest.Add(49 * time.Hour)}, + IndexType: "boltdb", + ObjectType: "filesystem", + Schema: "v11", + IndexTables: chunk.PeriodicTableConfig{ + Prefix: "index_", + Period: time.Hour * 24, + }, + RowShards: 16, + }, + }, + }, +} + +var allSchemas = []struct { + schema string + from model.Time +}{ + {"v9", model.Earliest}, + {"v10", model.Earliest.Add(25 * time.Hour)}, + {"v11", model.Earliest.Add(49 * time.Hour)}, +} + type fakeRule struct { streams []StreamRule tenants map[string]time.Duration @@ -90,34 +140,6 @@ func newTestStore(t *testing.T) *testStore { limits, err := validation.NewOverrides(validation.Limits{}, nil) require.NoError(t, err) - schemaCfg := storage.SchemaConfig{ - SchemaConfig: chunk.SchemaConfig{ - Configs: []chunk.PeriodConfig{ - { - From: chunk.DayTime{Time: model.Earliest}, - IndexType: "boltdb", - ObjectType: "filesystem", - Schema: "v9", - IndexTables: chunk.PeriodicTableConfig{ - Prefix: "index_", - Period: time.Hour * 24, - }, - RowShards: 16, - }, - { - From: chunk.DayTime{Time: model.Earliest.Add(25 * time.Hour)}, - IndexType: "boltdb", - ObjectType: "filesystem", - Schema: "v11", - IndexTables: chunk.PeriodicTableConfig{ - Prefix: "index_", - Period: time.Hour * 24, - }, - RowShards: 16, - }, - }, - }, - } require.NoError(t, schemaCfg.SchemaConfig.Validate()) config := storage.Config{ From 22af4684ecfbaba166a4f2f05e0fecc243f61c58 Mon Sep 17 00:00:00 2001 From: Cyril Tovena Date: Mon, 12 Apr 2021 22:35:18 +0200 Subject: [PATCH 08/42] Iterator for chunks ref with all labels !!!. Signed-off-by: Cyril Tovena --- .../shipper/compactor/retention/index.go | 54 ++++++++ .../shipper/compactor/retention/iterator.go | 31 +++-- .../compactor/retention/iterator_test.go | 66 +++++++-- .../shipper/compactor/retention/series.go | 130 ++++++++++++++++++ .../{user_series_test.go => series_test.go} | 0 .../compactor/retention/user_series.go | 48 ------- .../shipper/compactor/retention/util_test.go | 7 +- 7 files changed, 262 insertions(+), 74 deletions(-) create mode 100644 pkg/storage/stores/shipper/compactor/retention/series.go rename pkg/storage/stores/shipper/compactor/retention/{user_series_test.go => series_test.go} (100%) delete mode 100644 pkg/storage/stores/shipper/compactor/retention/user_series.go diff --git a/pkg/storage/stores/shipper/compactor/retention/index.go b/pkg/storage/stores/shipper/compactor/retention/index.go index 863da1b3931a5..9e9b6abe1dca5 100644 --- a/pkg/storage/stores/shipper/compactor/retention/index.go +++ b/pkg/storage/stores/shipper/compactor/retention/index.go @@ -140,6 +140,44 @@ func parseLabelIndexRef(hashKey, rangeKey []byte) (*LabelIndexRef, bool, error) }, true, nil } +type LabelSeriesRangeKey struct { + SeriesID []byte + UserID []byte + Name []byte +} + +func (l LabelSeriesRangeKey) String() string { + return fmt.Sprintf("%s:%s:%s", l.SeriesID, l.UserID, l.Name) +} + +func parseLabelSeriesRangeKey(hashKey, rangeKey []byte) (*LabelSeriesRangeKey, bool, error) { + // todo reuse memory via pool + var ( + rangeComponents [][]byte + hashComponents [][]byte + ) + rangeComponents = decodeRangeKey(rangeKey, rangeComponents) + if len(rangeComponents) < 4 { + return nil, false, newInvalidIndexKeyError(hashKey, rangeKey) + } + keyType := rangeComponents[len(rangeComponents)-1] + if len(keyType) == 0 || keyType[0] != labelSeriesRangeKeyV1 { + return nil, false, nil + } + hashComponents = splitBytesBy(hashKey, ':', hashComponents) + // > v10 HashValue: fmt.Sprintf("%02d:%s:%s:%s", shard, bucket.hashKey , metricName, v.Name), + // < v10 HashValue: fmt.Sprintf("%s:%s:%s", bucket.hashKey, metricName, v.Name), + + if len(hashComponents) < 4 { + return nil, false, newInvalidIndexKeyError(hashKey, rangeKey) + } + return &LabelSeriesRangeKey{ + SeriesID: rangeComponents[1], + Name: hashComponents[len(hashComponents)-1], + UserID: hashComponents[len(hashComponents)-4], + }, true, nil +} + func findSeriesIDsForRules(db *bbolt.DB, config chunk.PeriodConfig, rules []StreamRule) ([][]string, error) { schema, err := config.CreateSchema() if err != nil { @@ -327,6 +365,22 @@ func decodeKey(k []byte) (hashValue, rangeValue []byte) { return } +func splitBytesBy(value []byte, by byte, components [][]byte) [][]byte { + components = components[:0] + i, j := 0, 0 + for j < len(value) { + if value[j] != by { + j++ + continue + } + components = append(components, value[i:j]) + j++ + i = j + } + components = append(components, value[i:]) + return components +} + func decodeRangeKey(value []byte, components [][]byte) [][]byte { components = components[:0] i, j := 0, 0 diff --git a/pkg/storage/stores/shipper/compactor/retention/iterator.go b/pkg/storage/stores/shipper/compactor/retention/iterator.go index 828616282ed8e..b96b81b997af3 100644 --- a/pkg/storage/stores/shipper/compactor/retention/iterator.go +++ b/pkg/storage/stores/shipper/compactor/retention/iterator.go @@ -6,6 +6,7 @@ import ( "fmt" "github.com/cortexproject/cortex/pkg/chunk" + "github.com/prometheus/prometheus/pkg/labels" "go.etcd.io/bbolt" ) @@ -14,9 +15,14 @@ var ( _ Series = &series{} ) +type ChunkEntry struct { + ChunkRef + Labels labels.Labels +} + type ChunkEntryIterator interface { Next() bool - Entry() *ChunkRef + Entry() ChunkEntry // Delete deletes the current entry. Delete() error Err() error @@ -24,23 +30,31 @@ type ChunkEntryIterator interface { type chunkIndexIterator struct { cursor *bbolt.Cursor - current *ChunkRef + current ChunkEntry first bool err error + + labelsMapper *seriesLabelsMapper } -func newChunkIndexIterator(bucket *bbolt.Bucket) *chunkIndexIterator { - return &chunkIndexIterator{ - cursor: bucket.Cursor(), - first: true, +func newChunkIndexIterator(bucket *bbolt.Bucket, config chunk.PeriodConfig) (*chunkIndexIterator, error) { + labelsMapper, err := newSeriesLabelsMapper(bucket, config) + if err != nil { + return nil, err } + return &chunkIndexIterator{ + cursor: bucket.Cursor(), + first: true, + labelsMapper: labelsMapper, + current: ChunkEntry{}, + }, nil } func (b *chunkIndexIterator) Err() error { return b.err } -func (b *chunkIndexIterator) Entry() *ChunkRef { +func (b *chunkIndexIterator) Entry() ChunkEntry { return b.current } @@ -67,7 +81,8 @@ func (b *chunkIndexIterator) Next() bool { key, _ = b.cursor.Next() continue } - b.current = ref + b.current.ChunkRef = *ref + b.current.Labels = b.labelsMapper.Get(ref.SeriesID, ref.UserID) return true } return false diff --git a/pkg/storage/stores/shipper/compactor/retention/iterator_test.go b/pkg/storage/stores/shipper/compactor/retention/iterator_test.go index 8619eecedf7f7..dd8a7e233f91c 100644 --- a/pkg/storage/stores/shipper/compactor/retention/iterator_test.go +++ b/pkg/storage/stores/shipper/compactor/retention/iterator_test.go @@ -6,6 +6,8 @@ import ( "time" "github.com/cortexproject/cortex/pkg/chunk" + shipper_util "github.com/grafana/loki/pkg/storage/stores/shipper/util" + "github.com/prometheus/common/model" "github.com/prometheus/prometheus/pkg/labels" "github.com/stretchr/testify/require" "go.etcd.io/bbolt" @@ -28,9 +30,10 @@ func Test_ChunkIterator(t *testing.T) { tables := store.indexTables() require.Len(t, tables, 1) - var actual []*ChunkRef + var actual []ChunkEntry err := tables[0].DB.Update(func(tx *bbolt.Tx) error { - it := newChunkIndexIterator(tx.Bucket(bucketName)) + it, err := newChunkIndexIterator(tx.Bucket(bucketName), tt.config) + require.NoError(t, err) for it.Next() { require.NoError(t, it.Err()) actual = append(actual, it.Entry()) @@ -42,34 +45,67 @@ func Test_ChunkIterator(t *testing.T) { return nil }) require.NoError(t, err) - require.Equal(t, []*ChunkRef{ - refFromChunk(c1), - refFromChunk(c2), + require.Equal(t, []ChunkEntry{ + entryFromChunk(c1), + entryFromChunk(c2), }, actual) // second pass we delete c2 actual = actual[:0] err = tables[0].DB.Update(func(tx *bbolt.Tx) error { - it := newChunkIndexIterator(tx.Bucket(bucketName)) + it, err := newChunkIndexIterator(tx.Bucket(bucketName), tt.config) + require.NoError(t, err) for it.Next() { actual = append(actual, it.Entry()) } return it.Err() }) require.NoError(t, err) - require.Equal(t, []*ChunkRef{ - refFromChunk(c1), + require.Equal(t, []ChunkEntry{ + entryFromChunk(c1), }, actual) }) } } -func refFromChunk(c chunk.Chunk) *ChunkRef { - return &ChunkRef{ - UserID: []byte(c.UserID), - SeriesID: labelsSeriesID(c.Metric), - ChunkID: []byte(c.ExternalKey()), - From: c.From, - Through: c.Through, +func entryFromChunk(c chunk.Chunk) ChunkEntry { + return ChunkEntry{ + ChunkRef: ChunkRef{ + UserID: []byte(c.UserID), + SeriesID: labelsSeriesID(c.Metric), + ChunkID: []byte(c.ExternalKey()), + From: c.From, + Through: c.Through, + }, + Labels: c.Metric.WithoutLabels("__name__"), } } + +var chunkEntry ChunkEntry + +func Benchmark_ChunkIterator(b *testing.B) { + b.ReportAllocs() + + db, err := shipper_util.SafeOpenBoltdbFile("/Users/ctovena/Downloads/index_loki_ops_index_18669_compactor-1617841099") + require.NoError(b, err) + t, err := time.Parse("2006-01-02", "2020-07-31") + require.NoError(b, err) + var total int64 + db.View(func(tx *bbolt.Tx) error { + bucket := tx.Bucket(bucketName) + for n := 0; n < b.N; n++ { + it, err := newChunkIndexIterator(bucket, chunk.PeriodConfig{ + From: chunk.DayTime{Time: model.TimeFromUnix(t.Unix())}, + Schema: "v11", + RowShards: 16, + }) + require.NoError(b, err) + for it.Next() { + chunkEntry = it.Entry() + total++ + } + } + return nil + }) + b.Logf("Total chunk ref:%d", total) +} diff --git a/pkg/storage/stores/shipper/compactor/retention/series.go b/pkg/storage/stores/shipper/compactor/retention/series.go new file mode 100644 index 0000000000000..5a2893a5eb6b8 --- /dev/null +++ b/pkg/storage/stores/shipper/compactor/retention/series.go @@ -0,0 +1,130 @@ +package retention + +import ( + "github.com/cortexproject/cortex/pkg/chunk" + "github.com/prometheus/prometheus/pkg/labels" + "go.etcd.io/bbolt" +) + +type userSeries struct { + key []byte + seriesIDLen int +} + +func newUserSeries(seriesID []byte, userID []byte) userSeries { + key := make([]byte, 0, len(seriesID)+len(userID)) + key = append(key, seriesID...) + key = append(key, userID...) + return userSeries{ + key: key, + seriesIDLen: len(seriesID), + } +} + +func (us userSeries) Key() string { + return unsafeGetString(us.key) +} + +func (us userSeries) SeriesID() []byte { + return us.key[:us.seriesIDLen] +} + +func (us userSeries) UserID() []byte { + return us.key[us.seriesIDLen:] +} + +func (us *userSeries) Reset(seriesID []byte, userID []byte) { + if us.key == nil { + us.key = make([]byte, 0, len(seriesID)+len(userID)) + } + us.key = us.key[:0] + us.key = append(us.key, seriesID...) + us.key = append(us.key, userID...) + us.seriesIDLen = len(seriesID) +} + +type userSeriesMap map[string]userSeries + +func newUserSeriesMap() userSeriesMap { + return make(userSeriesMap) +} + +func (u userSeriesMap) Add(seriesID []byte, userID []byte) { + us := newUserSeries(seriesID, userID) + u[us.Key()] = us +} + +func (u userSeriesMap) ForEach(callback func(seriesID []byte, userID []byte) error) error { + for _, v := range u { + if err := callback(v.SeriesID(), v.UserID()); err != nil { + return err + } + } + return nil +} + +type seriesLabels struct { + userSeries + lbs labels.Labels +} + +type seriesLabelsMapper struct { + cursor *bbolt.Cursor + config chunk.PeriodConfig + + bufKey userSeries + mapping map[string]*seriesLabels +} + +func newSeriesLabelsMapper(bucket *bbolt.Bucket, config chunk.PeriodConfig) (*seriesLabelsMapper, error) { + sm := &seriesLabelsMapper{ + cursor: bucket.Cursor(), + mapping: map[string]*seriesLabels{}, + config: config, + bufKey: newUserSeries(nil, nil), + } + if err := sm.build(); err != nil { + return nil, err + } + return sm, nil +} + +func (sm *seriesLabelsMapper) Get(seriesID []byte, userID []byte) labels.Labels { + sm.bufKey.Reset(seriesID, userID) + lbs, ok := sm.mapping[sm.bufKey.Key()] + if ok { + return lbs.lbs + } + return labels.Labels{} +} + +func (sm *seriesLabelsMapper) build() error { +Outer: + for k, v := sm.cursor.First(); k != nil; k, v = sm.cursor.Next() { + ref, ok, err := parseLabelSeriesRangeKey(decodeKey(k)) + if err != nil { + return err + } + if !ok { + continue + } + sm.bufKey.Reset(ref.SeriesID, ref.UserID) + lbs, ok := sm.mapping[sm.bufKey.Key()] + if !ok { + k := newUserSeries(ref.SeriesID, ref.UserID) + lbs = &seriesLabels{ + userSeries: k, + lbs: make(labels.Labels, 0, 5), + } + sm.mapping[k.Key()] = lbs + } + // add the labels if it doesn't exist. + for _, l := range lbs.lbs { + if l.Name == unsafeGetString(ref.Name) { + continue Outer + } + } + lbs.lbs = append(lbs.lbs, labels.Label{Name: string(ref.Name), Value: string(v)}) + } + return nil +} diff --git a/pkg/storage/stores/shipper/compactor/retention/user_series_test.go b/pkg/storage/stores/shipper/compactor/retention/series_test.go similarity index 100% rename from pkg/storage/stores/shipper/compactor/retention/user_series_test.go rename to pkg/storage/stores/shipper/compactor/retention/series_test.go diff --git a/pkg/storage/stores/shipper/compactor/retention/user_series.go b/pkg/storage/stores/shipper/compactor/retention/user_series.go deleted file mode 100644 index 200646574817c..0000000000000 --- a/pkg/storage/stores/shipper/compactor/retention/user_series.go +++ /dev/null @@ -1,48 +0,0 @@ -package retention - -type userSeries struct { - key []byte - seriesIDLen int -} - -func newUserSeries(seriesID []byte, userID []byte) userSeries { - key := make([]byte, 0, len(seriesID)+len(userID)) - key = append(key, seriesID...) - key = append(key, userID...) - return userSeries{ - key: key, - seriesIDLen: len(seriesID), - } -} - -func (us userSeries) Key() string { - return unsafeGetString(us.key) -} - -func (us userSeries) SeriesID() []byte { - return us.key[:us.seriesIDLen] -} - -func (us userSeries) UserID() []byte { - return us.key[us.seriesIDLen:] -} - -type userSeriesMap map[string]userSeries - -func newUserSeriesMap() userSeriesMap { - return make(userSeriesMap) -} - -func (u userSeriesMap) Add(seriesID []byte, userID []byte) { - us := newUserSeries(seriesID, userID) - u[us.Key()] = us -} - -func (u userSeriesMap) ForEach(callback func(seriesID []byte, userID []byte) error) error { - for _, v := range u { - if err := callback(v.SeriesID(), v.UserID()); err != nil { - return err - } - } - return nil -} diff --git a/pkg/storage/stores/shipper/compactor/retention/util_test.go b/pkg/storage/stores/shipper/compactor/retention/util_test.go index 3919a73736c06..8494796f77c8f 100644 --- a/pkg/storage/stores/shipper/compactor/retention/util_test.go +++ b/pkg/storage/stores/shipper/compactor/retention/util_test.go @@ -64,10 +64,11 @@ var schemaCfg = storage.SchemaConfig{ var allSchemas = []struct { schema string from model.Time + config chunk.PeriodConfig }{ - {"v9", model.Earliest}, - {"v10", model.Earliest.Add(25 * time.Hour)}, - {"v11", model.Earliest.Add(49 * time.Hour)}, + {"v9", model.Earliest, schemaCfg.Configs[0]}, + {"v10", model.Earliest.Add(25 * time.Hour), schemaCfg.Configs[1]}, + {"v11", model.Earliest.Add(49 * time.Hour), schemaCfg.Configs[2]}, } type fakeRule struct { From 0af888378db5eea4ae7d5a8dddd1b88fecb2f9b9 Mon Sep 17 00:00:00 2001 From: Cyril Tovena Date: Tue, 13 Apr 2021 00:12:49 +0200 Subject: [PATCH 09/42] Optimize code away. Signed-off-by: Cyril Tovena --- .../shipper/compactor/retention/index.go | 104 ++++++++++++------ .../shipper/compactor/retention/iterator.go | 2 +- .../shipper/compactor/retention/pool.go | 27 +++++ .../shipper/compactor/retention/series.go | 2 +- 4 files changed, 102 insertions(+), 33 deletions(-) create mode 100644 pkg/storage/stores/shipper/compactor/retention/pool.go diff --git a/pkg/storage/stores/shipper/compactor/retention/index.go b/pkg/storage/stores/shipper/compactor/retention/index.go index 9e9b6abe1dca5..ccdd6f57ee0b9 100644 --- a/pkg/storage/stores/shipper/compactor/retention/index.go +++ b/pkg/storage/stores/shipper/compactor/retention/index.go @@ -63,42 +63,36 @@ func (c ChunkRef) String() string { return fmt.Sprintf("UserID: %s , SeriesID: %s , Time: [%s,%s]", c.UserID, c.SeriesID, c.From, c.Through) } -func parseChunkRef(hashKey, rangeKey []byte) (*ChunkRef, bool, error) { - // todo reuse memory via pool - var components [][]byte +func parseChunkRef(hashKey, rangeKey []byte) (ChunkRef, bool, error) { + componentsRef := getComponents() + defer putComponents(componentsRef) + components := componentsRef.components + components = decodeRangeKey(rangeKey, components) if len(components) == 0 { - return nil, false, newInvalidIndexKeyError(hashKey, rangeKey) + return ChunkRef{}, false, newInvalidIndexKeyError(hashKey, rangeKey) } keyType := components[len(components)-1] if len(keyType) == 0 || keyType[0] != chunkTimeRangeKeyV3 { - return nil, false, nil + return ChunkRef{}, false, nil } chunkID := components[len(components)-2] - // todo split manually - parts := bytes.Split(chunkID, []byte("/")) - if len(parts) != 2 { - return nil, false, newInvalidIndexKeyError(hashKey, rangeKey) - } - userID := parts[0] - // todo split manually - hexParts := bytes.Split(parts[1], []byte(":")) - if len(hexParts) != 4 { - return nil, false, newInvalidIndexKeyError(hashKey, rangeKey) + userID, hexFrom, hexThrough, ok := parseChunkID(chunkID) + if !ok { + return ChunkRef{}, false, newInvalidIndexKeyError(hashKey, rangeKey) } - - from, err := strconv.ParseInt(unsafeGetString(hexParts[1]), 16, 64) + from, err := strconv.ParseInt(unsafeGetString(hexFrom), 16, 64) if err != nil { - return nil, false, err + return ChunkRef{}, false, err } - through, err := strconv.ParseInt(unsafeGetString(hexParts[2]), 16, 64) + through, err := strconv.ParseInt(unsafeGetString(hexThrough), 16, 64) if err != nil { - return nil, false, err + return ChunkRef{}, false, err } - return &ChunkRef{ + return ChunkRef{ UserID: userID, SeriesID: seriesFromHash(hashKey), From: model.Time(from), @@ -107,6 +101,52 @@ func parseChunkRef(hashKey, rangeKey []byte) (*ChunkRef, bool, error) { }, true, nil } +func parseChunkID(chunkID []byte) (userID []byte, hexFrom, hexThrough []byte, valid bool) { + var ( + j, i int + hex []byte + ) + + for j < len(chunkID) { + if chunkID[j] != '/' { + j++ + continue + } + userID = chunkID[:j] + hex = chunkID[j+1:] + break + } + if len(userID) == 0 { + return nil, nil, nil, false + } + _, i = readOneHexPart(hex) + if i == 0 { + return nil, nil, nil, false + } + hex = hex[i+1:] + hexFrom, i = readOneHexPart(hex) + if i == 0 { + return nil, nil, nil, false + } + hex = hex[i+1:] + hexThrough, i = readOneHexPart(hex) + if i == 0 { + return nil, nil, nil, false + } + return userID, hexFrom, hexThrough, true +} + +func readOneHexPart(hex []byte) (part []byte, i int) { + for i < len(hex) { + if hex[i] != ':' { + i++ + continue + } + return hex[:i], i + } + return nil, 0 +} + type LabelIndexRef struct { KeyType byte SeriesID []byte @@ -150,28 +190,30 @@ func (l LabelSeriesRangeKey) String() string { return fmt.Sprintf("%s:%s:%s", l.SeriesID, l.UserID, l.Name) } -func parseLabelSeriesRangeKey(hashKey, rangeKey []byte) (*LabelSeriesRangeKey, bool, error) { - // todo reuse memory via pool - var ( - rangeComponents [][]byte - hashComponents [][]byte - ) +func parseLabelSeriesRangeKey(hashKey, rangeKey []byte) (LabelSeriesRangeKey, bool, error) { + rangeComponentsRef := getComponents() + defer putComponents(rangeComponentsRef) + rangeComponents := rangeComponentsRef.components + hashComponentsRef := getComponents() + defer putComponents(hashComponentsRef) + hashComponents := hashComponentsRef.components + rangeComponents = decodeRangeKey(rangeKey, rangeComponents) if len(rangeComponents) < 4 { - return nil, false, newInvalidIndexKeyError(hashKey, rangeKey) + return LabelSeriesRangeKey{}, false, newInvalidIndexKeyError(hashKey, rangeKey) } keyType := rangeComponents[len(rangeComponents)-1] if len(keyType) == 0 || keyType[0] != labelSeriesRangeKeyV1 { - return nil, false, nil + return LabelSeriesRangeKey{}, false, nil } hashComponents = splitBytesBy(hashKey, ':', hashComponents) // > v10 HashValue: fmt.Sprintf("%02d:%s:%s:%s", shard, bucket.hashKey , metricName, v.Name), // < v10 HashValue: fmt.Sprintf("%s:%s:%s", bucket.hashKey, metricName, v.Name), if len(hashComponents) < 4 { - return nil, false, newInvalidIndexKeyError(hashKey, rangeKey) + return LabelSeriesRangeKey{}, false, newInvalidIndexKeyError(hashKey, rangeKey) } - return &LabelSeriesRangeKey{ + return LabelSeriesRangeKey{ SeriesID: rangeComponents[1], Name: hashComponents[len(hashComponents)-1], UserID: hashComponents[len(hashComponents)-4], diff --git a/pkg/storage/stores/shipper/compactor/retention/iterator.go b/pkg/storage/stores/shipper/compactor/retention/iterator.go index b96b81b997af3..8c4cde53c2a75 100644 --- a/pkg/storage/stores/shipper/compactor/retention/iterator.go +++ b/pkg/storage/stores/shipper/compactor/retention/iterator.go @@ -81,7 +81,7 @@ func (b *chunkIndexIterator) Next() bool { key, _ = b.cursor.Next() continue } - b.current.ChunkRef = *ref + b.current.ChunkRef = ref b.current.Labels = b.labelsMapper.Get(ref.SeriesID, ref.UserID) return true } diff --git a/pkg/storage/stores/shipper/compactor/retention/pool.go b/pkg/storage/stores/shipper/compactor/retention/pool.go new file mode 100644 index 0000000000000..446d0de98a89a --- /dev/null +++ b/pkg/storage/stores/shipper/compactor/retention/pool.go @@ -0,0 +1,27 @@ +package retention + +import ( + "sync" +) + +type componentRef struct { + components [][]byte +} + +var componentPools = sync.Pool{ + New: func() interface{} { + return &componentRef{ + components: make([][]byte, 0, 5), + } + }, +} + +func getComponents() *componentRef { + ref := componentPools.Get().(*componentRef) + ref.components = ref.components[:0] + return ref +} + +func putComponents(ref *componentRef) { + componentPools.Put(ref) +} diff --git a/pkg/storage/stores/shipper/compactor/retention/series.go b/pkg/storage/stores/shipper/compactor/retention/series.go index 5a2893a5eb6b8..f33dbb7975985 100644 --- a/pkg/storage/stores/shipper/compactor/retention/series.go +++ b/pkg/storage/stores/shipper/compactor/retention/series.go @@ -114,7 +114,7 @@ Outer: k := newUserSeries(ref.SeriesID, ref.UserID) lbs = &seriesLabels{ userSeries: k, - lbs: make(labels.Labels, 0, 5), + lbs: make(labels.Labels, 0, 15), } sm.mapping[k.Key()] = lbs } From ce3b995ecbcb523eacb98765146d9ffa17bac3f2 Mon Sep 17 00:00:00 2001 From: Cyril Tovena Date: Tue, 13 Apr 2021 00:33:47 +0200 Subject: [PATCH 10/42] with delete into the mix Signed-off-by: Cyril Tovena --- .../stores/shipper/compactor/retention/iterator_test.go | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/pkg/storage/stores/shipper/compactor/retention/iterator_test.go b/pkg/storage/stores/shipper/compactor/retention/iterator_test.go index dd8a7e233f91c..35a353de8a841 100644 --- a/pkg/storage/stores/shipper/compactor/retention/iterator_test.go +++ b/pkg/storage/stores/shipper/compactor/retention/iterator_test.go @@ -2,6 +2,7 @@ package retention import ( "context" + "errors" "testing" "time" @@ -91,7 +92,7 @@ func Benchmark_ChunkIterator(b *testing.B) { t, err := time.Parse("2006-01-02", "2020-07-31") require.NoError(b, err) var total int64 - db.View(func(tx *bbolt.Tx) error { + db.Update(func(tx *bbolt.Tx) error { bucket := tx.Bucket(bucketName) for n := 0; n < b.N; n++ { it, err := newChunkIndexIterator(bucket, chunk.PeriodConfig{ @@ -102,10 +103,11 @@ func Benchmark_ChunkIterator(b *testing.B) { require.NoError(b, err) for it.Next() { chunkEntry = it.Entry() + require.NoError(b, it.Delete()) total++ } } - return nil + return errors.New("don't commit") }) b.Logf("Total chunk ref:%d", total) } From edb035c0b1c5d0313a731f2cb46a6926a74dc92f Mon Sep 17 00:00:00 2001 From: Cyril Tovena Date: Tue, 13 Apr 2021 14:27:26 +0200 Subject: [PATCH 11/42] Cleaner but not yet working for v10 and v11. Signed-off-by: Cyril Tovena --- .../shipper/compactor/retention/index.go | 21 +- .../shipper/compactor/retention/iterator.go | 185 +++++++----------- .../compactor/retention/iterator_test.go | 64 ++++++ .../shipper/compactor/retention/retention.go | 22 +-- 4 files changed, 153 insertions(+), 139 deletions(-) diff --git a/pkg/storage/stores/shipper/compactor/retention/index.go b/pkg/storage/stores/shipper/compactor/retention/index.go index ccdd6f57ee0b9..f2bfbeb9058c5 100644 --- a/pkg/storage/stores/shipper/compactor/retention/index.go +++ b/pkg/storage/stores/shipper/compactor/retention/index.go @@ -147,17 +147,11 @@ func readOneHexPart(hex []byte) (part []byte, i int) { return nil, 0 } -type LabelIndexRef struct { - KeyType byte - SeriesID []byte -} - -func parseLabelIndexRef(hashKey, rangeKey []byte) (*LabelIndexRef, bool, error) { - // todo reuse memory via pool - var ( - components [][]byte - seriesID []byte - ) +func parseLabelIndexSeriesID(hashKey, rangeKey []byte) ([]byte, bool, error) { + componentsRef := getComponents() + defer putComponents(componentsRef) + components := componentsRef.components + var seriesID []byte components = decodeRangeKey(rangeKey, components) if len(components) < 4 { return nil, false, newInvalidIndexKeyError(hashKey, rangeKey) @@ -174,10 +168,7 @@ func parseLabelIndexRef(hashKey, rangeKey []byte) (*LabelIndexRef, bool, error) default: return nil, false, nil } - return &LabelIndexRef{ - KeyType: keyType[0], - SeriesID: seriesID, - }, true, nil + return seriesID, true, nil } type LabelSeriesRangeKey struct { diff --git a/pkg/storage/stores/shipper/compactor/retention/iterator.go b/pkg/storage/stores/shipper/compactor/retention/iterator.go index 8c4cde53c2a75..814cf2b1dfaab 100644 --- a/pkg/storage/stores/shipper/compactor/retention/iterator.go +++ b/pkg/storage/stores/shipper/compactor/retention/iterator.go @@ -4,6 +4,7 @@ import ( "bytes" "encoding/binary" "fmt" + "time" "github.com/cortexproject/cortex/pkg/chunk" "github.com/prometheus/prometheus/pkg/labels" @@ -12,7 +13,7 @@ import ( var ( _ ChunkEntryIterator = &chunkIndexIterator{} - _ Series = &series{} + _ SeriesCleaner = &seriesCleaner{} ) type ChunkEntry struct { @@ -88,130 +89,88 @@ func (b *chunkIndexIterator) Next() bool { return false } -type Series interface { - Buckets(userID []byte) []Bucket +type SeriesCleaner interface { + Cleanup(seriesID []byte, userID []byte) error } -type Bucket interface { - ContainsChunkFor(seriesID []byte) bool - LabelEntries(seriesID []byte) LabelEntryIterator -} - -type LabelEntryIterator interface { - Next() bool - Entry() *LabelIndexRef - // Delete deletes the current entry. - Delete() error - Err() error -} - -type series struct { - bucket *bbolt.Bucket - config chunk.PeriodConfig -} - -func newSeries(bucket *bbolt.Bucket, config chunk.PeriodConfig) *series { - return &series{ - bucket: bucket, - config: config, - } -} +type seriesCleaner struct { + bucketTimestamps []string + shards map[uint32]string + cursor *bbolt.Cursor + config chunk.PeriodConfig -func (s *series) Buckets(userID []byte) []Bucket { - bucketHashes := allBucketsHashes(s.config, unsafeGetString(userID)) - res := make([]Bucket, 0, len(bucketHashes)) - for _, h := range bucketHashes { - res = append(res, newBucket(s.bucket.Cursor(), h, s.config)) - } - return res -} - -type bucket struct { - cursor *bbolt.Cursor - hash string - config chunk.PeriodConfig -} - -func newBucket(cursor *bbolt.Cursor, hash string, config chunk.PeriodConfig) *bucket { - return &bucket{ - cursor: cursor, - hash: hash, - config: config, - } + buf []byte } -func (b *bucket) ContainsChunkFor(seriesID []byte) bool { - if key, _ := b.cursor.Seek([]byte(b.hash + ":" + string(seriesID))); key != nil { - return true +func newSeriesCleaner(bucket *bbolt.Bucket, config chunk.PeriodConfig) *seriesCleaner { + var ( + fromDay = config.From.Time.Unix() / int64(config.IndexTables.Period/time.Second) + throughDay = config.From.Add(config.IndexTables.Period).Unix() / int64(config.IndexTables.Period/time.Second) + bucketTimestamps = []string{} + ) + for i := fromDay; i <= throughDay; i++ { + bucketTimestamps = append(bucketTimestamps, fmt.Sprintf("d%d", i)) } - return false -} - -func (b *bucket) LabelEntries(seriesID []byte) LabelEntryIterator { - return newLabelEntryIterator(b, b.keyPrefix(seriesID)) -} - -func (b *bucket) keyPrefix(series []byte) (prefix []byte) { - switch b.config.Schema { - case "v11", "v10": - shard := binary.BigEndian.Uint32(series) % b.config.RowShards - prefix = unsafeGetBytes(fmt.Sprintf("%02d:%s:%s", shard, b.hash, logMetricName)) - default: - prefix = unsafeGetBytes(fmt.Sprintf("%s:%s", b.hash, logMetricName)) + var shards map[uint32]string + if config.RowShards != 0 { + shards = map[uint32]string{} + for s := uint32(0); s <= config.RowShards; s++ { + shards[s] = fmt.Sprintf("%02d", s) + } } - return -} - -type labelEntryIterator struct { - *bucket - - current *LabelIndexRef - first bool - err error - prefix []byte -} - -func newLabelEntryIterator(b *bucket, prefix []byte) *labelEntryIterator { - return &labelEntryIterator{ - bucket: b, - first: true, - prefix: prefix, + return &seriesCleaner{ + bucketTimestamps: bucketTimestamps, + cursor: bucket.Cursor(), + buf: make([]byte, 0, 1024), + config: config, + shards: shards, } } -func (it *labelEntryIterator) Err() error { - return it.err -} - -func (it *labelEntryIterator) Entry() *LabelIndexRef { - return it.current -} - -func (it *labelEntryIterator) Delete() error { - return it.cursor.Delete() -} +func (s *seriesCleaner) Cleanup(seriesID []byte, userID []byte) error { + for _, timestamp := range s.bucketTimestamps { + // build the chunk ref prefix + s.buf = s.buf[:0] + s.buf = append(s.buf, userID...) + s.buf = append(s.buf, ':') + s.buf = append(s.buf, unsafeGetBytes(timestamp)...) + s.buf = append(s.buf, ':') + s.buf = append(s.buf, seriesID...) -func (it *labelEntryIterator) Next() bool { - var key []byte - if it.first { - key, _ = it.cursor.Seek(it.prefix) - it.first = false - } else { - key, _ = it.cursor.Next() - } - for key != nil && bytes.HasPrefix(key, it.prefix) { - ref, ok, err := parseLabelIndexRef(decodeKey(key)) - if err != nil { - it.err = err - return false - } - // skips anything else than labels index entries. - if !ok { - key, _ = it.cursor.Next() + if key, _ := s.cursor.Seek(s.buf); key != nil && bytes.HasPrefix(key, s.buf) { + // this series still have chunk entries we can't cleanup continue } - it.current = ref - return true + // we don't have any chunk ref for that series let's delete all label index entries + s.buf = s.buf[:0] + if s.config.Schema != "v9" { + shard := binary.BigEndian.Uint32(seriesID) % s.config.RowShards + s.buf = append(s.buf, unsafeGetBytes(s.shards[shard])...) + } + s.buf = append(s.buf, userID...) + s.buf = append(s.buf, ':') + s.buf = append(s.buf, unsafeGetBytes(timestamp)...) + s.buf = append(s.buf, ':') + s.buf = append(s.buf, unsafeGetBytes(logMetricName)...) + + // delete all seriesRangeKeyV1 and labelSeriesRangeKeyV1 via prefix + // todo(cyriltovena) we might be able to encode index key instead of parsing all label entries for faster delete. + for key, _ := s.cursor.Seek(s.buf); key != nil && bytes.HasPrefix(key, s.buf); key, _ = s.cursor.Next() { + + parsedSeriesID, ok, err := parseLabelIndexSeriesID(decodeKey(key)) + if err != nil { + return err + } + if !ok { + continue + } + if !bytes.Equal(seriesID, parsedSeriesID) { + continue + } + if err := s.cursor.Delete(); err != nil { + return err + } + } } - return false + return nil } diff --git a/pkg/storage/stores/shipper/compactor/retention/iterator_test.go b/pkg/storage/stores/shipper/compactor/retention/iterator_test.go index 35a353de8a841..8e83e9c03f377 100644 --- a/pkg/storage/stores/shipper/compactor/retention/iterator_test.go +++ b/pkg/storage/stores/shipper/compactor/retention/iterator_test.go @@ -69,6 +69,70 @@ func Test_ChunkIterator(t *testing.T) { } } +func Test_SeriesCleaner(t *testing.T) { + for _, tt := range allSchemas { + tt := tt + t.Run(tt.schema, func(t *testing.T) { + store := newTestStore(t) + defer store.cleanup() + c1 := createChunk(t, "1", labels.Labels{labels.Label{Name: "foo", Value: "bar"}}, tt.from, tt.from.Add(1*time.Hour)) + c2 := createChunk(t, "2", labels.Labels{labels.Label{Name: "foo", Value: "buzz"}, labels.Label{Name: "bar", Value: "foo"}}, tt.from, tt.from.Add(1*time.Hour)) + c3 := createChunk(t, "2", labels.Labels{labels.Label{Name: "foo", Value: "buzz"}, labels.Label{Name: "bar", Value: "buzz"}}, tt.from, tt.from.Add(1*time.Hour)) + + require.NoError(t, store.Put(context.TODO(), []chunk.Chunk{ + c1, c2, c3, + })) + + store.Stop() + + tables := store.indexTables() + require.Len(t, tables, 1) + // remove c2 chunk + err := tables[0].DB.Update(func(tx *bbolt.Tx) error { + it, err := newChunkIndexIterator(tx.Bucket(bucketName), tt.config) + require.NoError(t, err) + for it.Next() { + require.NoError(t, it.Err()) + if it.Entry().Labels.Get("bar") == "foo" { + require.NoError(t, it.Delete()) + } + } + return nil + }) + require.NoError(t, err) + + err = tables[0].DB.Update(func(tx *bbolt.Tx) error { + cleaner := newSeriesCleaner(tx.Bucket(bucketName), tt.config) + if err := cleaner.Cleanup(entryFromChunk(c2).SeriesID, entryFromChunk(c2).UserID); err != nil { + return err + } + if err := cleaner.Cleanup(entryFromChunk(c1).SeriesID, entryFromChunk(c1).UserID); err != nil { + return err + } + return nil + }) + require.NoError(t, err) + + err = tables[0].DB.View(func(tx *bbolt.Tx) error { + return tx.Bucket(bucketName).ForEach(func(k, _ []byte) error { + expectedDeleteSeries := entryFromChunk(c2).SeriesID + series, ok, err := parseLabelIndexSeriesID(decodeKey(k)) + if !ok { + return nil + } + if err != nil { + return err + } + require.NotEqual(t, string(expectedDeleteSeries), string(series), "series %s should be deleted", expectedDeleteSeries) + + return nil + }) + }) + require.NoError(t, err) + }) + } +} + func entryFromChunk(c chunk.Chunk) ChunkEntry { return ChunkEntry{ ChunkRef: ChunkRef{ diff --git a/pkg/storage/stores/shipper/compactor/retention/retention.go b/pkg/storage/stores/shipper/compactor/retention/retention.go index 69e9961af9339..f729ac8acf508 100644 --- a/pkg/storage/stores/shipper/compactor/retention/retention.go +++ b/pkg/storage/stores/shipper/compactor/retention/retention.go @@ -87,14 +87,14 @@ func markForDelete(in, marker *bbolt.DB, expiration ExpirationChecker, config ch return nil } // this bucketHash doesn't contains the given series. Let's remove it. - if err := forAllLabelRef(c, bucketHash, seriesID, config, func(_ *LabelIndexRef) error { - if err := c.Delete(); err != nil { - return err - } - return nil - }); err != nil { - return err - } + // if err := forAllLabelRef(c, bucketHash, seriesID, config, func(_ *LabelIndexRef) error { + // if err := c.Delete(); err != nil { + // return err + // } + // return nil + // }); err != nil { + // return err + // } } return nil }) @@ -102,7 +102,7 @@ func markForDelete(in, marker *bbolt.DB, expiration ExpirationChecker, config ch }) } -func forAllLabelRef(c *bbolt.Cursor, bucketHash string, seriesID []byte, config chunk.PeriodConfig, callback func(ref *LabelIndexRef) error) error { +func forAllLabelRef(c *bbolt.Cursor, bucketHash string, seriesID []byte, config chunk.PeriodConfig, callback func(ref []byte) error) error { // todo reuse memory and refactor var ( prefix string @@ -116,11 +116,11 @@ func forAllLabelRef(c *bbolt.Cursor, bucketHash string, seriesID []byte, config prefix = fmt.Sprintf("%s:%s", bucketHash, logMetricName) } for k, _ := c.Seek([]byte(prefix)); k != nil; k, _ = c.Next() { - ref, ok, err := parseLabelIndexRef(decodeKey(k)) + ref, ok, err := parseLabelIndexSeriesID(decodeKey(k)) if err != nil { return err } - if !ok || !bytes.Equal(seriesID, ref.SeriesID) { + if !ok || !bytes.Equal(seriesID, ref) { continue } if err := callback(ref); err != nil { From 0c0032f6dbb50fd053891a7242f25591ea3170d3 Mon Sep 17 00:00:00 2001 From: Cyril Tovena Date: Tue, 13 Apr 2021 16:25:05 +0200 Subject: [PATCH 12/42] Fixes series cleaner. Signed-off-by: Cyril Tovena --- pkg/storage/stores/shipper/compactor/retention/iterator.go | 6 ++++++ .../stores/shipper/compactor/retention/iterator_test.go | 4 +++- 2 files changed, 9 insertions(+), 1 deletion(-) diff --git a/pkg/storage/stores/shipper/compactor/retention/iterator.go b/pkg/storage/stores/shipper/compactor/retention/iterator.go index 814cf2b1dfaab..b95ff5e2b68a0 100644 --- a/pkg/storage/stores/shipper/compactor/retention/iterator.go +++ b/pkg/storage/stores/shipper/compactor/retention/iterator.go @@ -131,6 +131,11 @@ func (s *seriesCleaner) Cleanup(seriesID []byte, userID []byte) error { for _, timestamp := range s.bucketTimestamps { // build the chunk ref prefix s.buf = s.buf[:0] + if s.config.Schema != "v9" { + shard := binary.BigEndian.Uint32(seriesID) % s.config.RowShards + s.buf = append(s.buf, unsafeGetBytes(s.shards[shard])...) + s.buf = append(s.buf, ':') + } s.buf = append(s.buf, userID...) s.buf = append(s.buf, ':') s.buf = append(s.buf, unsafeGetBytes(timestamp)...) @@ -146,6 +151,7 @@ func (s *seriesCleaner) Cleanup(seriesID []byte, userID []byte) error { if s.config.Schema != "v9" { shard := binary.BigEndian.Uint32(seriesID) % s.config.RowShards s.buf = append(s.buf, unsafeGetBytes(s.shards[shard])...) + s.buf = append(s.buf, ':') } s.buf = append(s.buf, userID...) s.buf = append(s.buf, ':') diff --git a/pkg/storage/stores/shipper/compactor/retention/iterator_test.go b/pkg/storage/stores/shipper/compactor/retention/iterator_test.go index 8e83e9c03f377..ed0a86fc2ed0a 100644 --- a/pkg/storage/stores/shipper/compactor/retention/iterator_test.go +++ b/pkg/storage/stores/shipper/compactor/retention/iterator_test.go @@ -123,7 +123,9 @@ func Test_SeriesCleaner(t *testing.T) { if err != nil { return err } - require.NotEqual(t, string(expectedDeleteSeries), string(series), "series %s should be deleted", expectedDeleteSeries) + if string(expectedDeleteSeries) == string(series) { + require.Fail(t, "series should be deleted", expectedDeleteSeries) + } return nil }) From 773d40372bf095399184b8854e05db593e85b24a Mon Sep 17 00:00:00 2001 From: Cyril Tovena Date: Wed, 14 Apr 2021 17:41:21 +0200 Subject: [PATCH 13/42] hooking into the compactor. Signed-off-by: Cyril Tovena --- pkg/loki/modules.go | 7 +- .../stores/shipper/compactor/compactor.go | 101 ++++++- .../stores/shipper/compactor/metrics.go | 20 ++ .../shipper/compactor/retention/expiration.go | 25 +- .../shipper/compactor/retention/index.go | 153 ---------- .../shipper/compactor/retention/marker.go | 80 +++++ .../shipper/compactor/retention/retention.go | 281 ++++++++++-------- .../compactor/retention/retention_test.go | 26 -- .../shipper/compactor/retention/rules.go | 7 +- .../shipper/compactor/retention/util.go | 70 ++--- pkg/storage/stores/shipper/compactor/table.go | 5 +- pkg/storage/stores/shipper/util/util.go | 10 + 12 files changed, 409 insertions(+), 376 deletions(-) create mode 100644 pkg/storage/stores/shipper/compactor/retention/marker.go diff --git a/pkg/loki/modules.go b/pkg/loki/modules.go index 7c8e57a548ec6..bdaaf934750ec 100644 --- a/pkg/loki/modules.go +++ b/pkg/loki/modules.go @@ -576,8 +576,11 @@ func (t *Loki) initMemberlistKV() (services.Service, error) { } func (t *Loki) initCompactor() (services.Service, error) { - var err error - t.compactor, err = compactor.NewCompactor(t.cfg.CompactorConfig, t.cfg.StorageConfig.Config, prometheus.DefaultRegisterer) + err := t.cfg.SchemaConfig.Load() + if err != nil { + return nil, err + } + t.compactor, err = compactor.NewCompactor(t.cfg.CompactorConfig, t.cfg.StorageConfig.Config, t.cfg.SchemaConfig, prometheus.DefaultRegisterer) if err != nil { return nil, err } diff --git a/pkg/storage/stores/shipper/compactor/compactor.go b/pkg/storage/stores/shipper/compactor/compactor.go index 94828f4ed2994..e2de9783b12bc 100644 --- a/pkg/storage/stores/shipper/compactor/compactor.go +++ b/pkg/storage/stores/shipper/compactor/compactor.go @@ -7,6 +7,7 @@ import ( "path/filepath" "reflect" "strings" + "sync" "time" "github.com/cortexproject/cortex/pkg/chunk" @@ -17,8 +18,11 @@ import ( "github.com/go-kit/kit/log/level" "github.com/prometheus/client_golang/prometheus" + loki_storage "github.com/grafana/loki/pkg/storage" + "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/util" + errUtil "github.com/grafana/loki/pkg/util" ) const delimiter = "/" @@ -28,6 +32,7 @@ type Config struct { SharedStoreType string `yaml:"shared_store"` SharedStoreKeyPrefix string `yaml:"shared_store_key_prefix"` CompactionInterval time.Duration `yaml:"compaction_interval"` + RetentionInterval time.Duration `yaml:"retention_interval"` } // RegisterFlags registers flags. @@ -36,6 +41,7 @@ func (cfg *Config) RegisterFlags(f *flag.FlagSet) { f.StringVar(&cfg.SharedStoreType, "boltdb.shipper.compactor.shared-store", "", "Shared store used for storing boltdb files. Supported types: gcs, s3, azure, swift, filesystem") f.StringVar(&cfg.SharedStoreKeyPrefix, "boltdb.shipper.compactor.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.DurationVar(&cfg.CompactionInterval, "boltdb.shipper.compactor.compaction-interval", 2*time.Hour, "Interval at which to re-run the compaction operation.") + f.DurationVar(&cfg.CompactionInterval, "boltdb.shipper.compactor.retention-interval", 2*time.Hour, "Interval at which to re-run the retention operation.") } func (cfg *Config) IsDefaults() bool { @@ -53,11 +59,12 @@ type Compactor struct { cfg Config objectClient chunk.ObjectClient + tableMarker *retention.Marker metrics *metrics } -func NewCompactor(cfg Config, storageConfig storage.Config, r prometheus.Registerer) (*Compactor, error) { +func NewCompactor(cfg Config, storageConfig storage.Config, schemaConfig loki_storage.SchemaConfig, r prometheus.Registerer) (*Compactor, error) { if cfg.IsDefaults() { return nil, errors.New("Must specify compactor config") } @@ -71,11 +78,14 @@ func NewCompactor(cfg Config, storageConfig storage.Config, r prometheus.Registe if err != nil { return nil, err } + prefixedClient := util.NewPrefixedObjectClient(objectClient, cfg.SharedStoreKeyPrefix) + // todo configuration and expiration checker. compactor := Compactor{ cfg: cfg, - objectClient: util.NewPrefixedObjectClient(objectClient, cfg.SharedStoreKeyPrefix), + objectClient: prefixedClient, metrics: newMetrics(r), + tableMarker: retention.NewMarker(filepath.Join(cfg.WorkingDirectory, "retention"), schemaConfig, prefixedClient, nil), } compactor.Service = services.NewBasicService(nil, compactor.loop, nil) @@ -84,28 +94,56 @@ func NewCompactor(cfg Config, storageConfig storage.Config, r prometheus.Registe func (c *Compactor) loop(ctx context.Context) error { runCompaction := func() { - err := c.Run(ctx) + err := c.RunCompaction(ctx) if err != nil { level.Error(util_log.Logger).Log("msg", "failed to run compaction", "err", err) } } + runRetention := func() { + err := c.RunRetention(ctx) + if err != nil { + level.Error(util_log.Logger).Log("msg", "failed to run retention", "err", err) + } + } + var wg sync.WaitGroup + wg.Add(2) + go func() { + defer wg.Done() + runCompaction() + + ticker := time.NewTicker(c.cfg.CompactionInterval) + defer ticker.Stop() + + for { + select { + case <-ticker.C: + runCompaction() + case <-ctx.Done(): + return + } + } + }() - runCompaction() + go func() { + defer wg.Done() - ticker := time.NewTicker(c.cfg.CompactionInterval) - defer ticker.Stop() + ticker := time.NewTicker(c.cfg.RetentionInterval) + defer ticker.Stop() - for { - select { - case <-ticker.C: - runCompaction() - case <-ctx.Done(): - return nil + for { + select { + case <-ticker.C: + runRetention() + case <-ctx.Done(): + return + } } - } + }() + wg.Wait() + return nil } -func (c *Compactor) Run(ctx context.Context) error { +func (c *Compactor) RunCompaction(ctx context.Context) error { status := statusSuccess start := time.Now() @@ -152,3 +190,38 @@ func (c *Compactor) Run(ctx context.Context) error { return nil } + +func (c *Compactor) RunRetention(ctx context.Context) error { + status := statusSuccess + start := time.Now() + + defer func() { + c.metrics.retentionOperationTotal.WithLabelValues(status).Inc() + if status == statusSuccess { + c.metrics.retentionOperationDurationSeconds.Set(time.Since(start).Seconds()) + c.metrics.retentionOperationLastSuccess.SetToCurrentTime() + } + }() + + _, dirs, err := c.objectClient.List(ctx, "", delimiter) + if err != nil { + status = statusFailure + return err + } + + tables := make([]string, len(dirs)) + for i, dir := range dirs { + tables[i] = strings.TrimSuffix(string(dir), delimiter) + } + + var errs errUtil.MultiError + + for _, tableName := range tables { + if err := c.tableMarker.MarkTableForDelete(ctx, tableName); err != nil { + level.Error(util_log.Logger).Log("msg", "failed to mark table for deletes", "table", tableName, "err", err) + errs.Add(err) + status = statusFailure + } + } + return errs.Err() +} diff --git a/pkg/storage/stores/shipper/compactor/metrics.go b/pkg/storage/stores/shipper/compactor/metrics.go index fdb304b7897b9..96b4a2aff7cc3 100644 --- a/pkg/storage/stores/shipper/compactor/metrics.go +++ b/pkg/storage/stores/shipper/compactor/metrics.go @@ -14,6 +14,10 @@ type metrics struct { compactTablesOperationTotal *prometheus.CounterVec compactTablesOperationDurationSeconds prometheus.Gauge compactTablesOperationLastSuccess prometheus.Gauge + + retentionOperationTotal *prometheus.CounterVec + retentionOperationDurationSeconds prometheus.Gauge + retentionOperationLastSuccess prometheus.Gauge } func newMetrics(r prometheus.Registerer) *metrics { @@ -33,6 +37,22 @@ func newMetrics(r prometheus.Registerer) *metrics { Name: "compact_tables_operation_last_successful_run_timestamp_seconds", Help: "Unix timestamp of the last successful compaction run", }), + + retentionOperationTotal: promauto.With(r).NewCounterVec(prometheus.CounterOpts{ + Namespace: "loki_boltdb_shipper", + Name: "retention_operation_total", + Help: "Total number of retention applied by status", + }, []string{"status"}), + retentionOperationDurationSeconds: promauto.With(r).NewGauge(prometheus.GaugeOpts{ + Namespace: "loki_boltdb_shipper", + Name: "retention_operation_duration_seconds", + Help: "Time (in seconds) spent in applying retention for all the tables", + }), + retentionOperationLastSuccess: promauto.With(r).NewGauge(prometheus.GaugeOpts{ + Namespace: "loki_boltdb_shipper", + Name: "retention_operation_last_successful_run_timestamp_seconds", + Help: "Unix timestamp of the last successful retention run", + }), } return &m diff --git a/pkg/storage/stores/shipper/compactor/retention/expiration.go b/pkg/storage/stores/shipper/compactor/retention/expiration.go index a9590c02f0039..fd058ffd44e9e 100644 --- a/pkg/storage/stores/shipper/compactor/retention/expiration.go +++ b/pkg/storage/stores/shipper/compactor/retention/expiration.go @@ -5,39 +5,24 @@ import ( ) type ExpirationChecker interface { - Expired(ref *ChunkRef) bool + Expired(ref ChunkEntry) bool } -// func MultiExpirationCheck(checker ...ExpirationChecker) ExpirationChecker { - -// } - type expirationChecker struct { - series map[string]StreamRule - TenantRules + rules Rules } -func NewExpirationChecker(seriesPerRule map[string]StreamRule, rules TenantRules) ExpirationChecker { +func NewExpirationChecker(rules Rules) ExpirationChecker { return &expirationChecker{ - series: seriesPerRule, - TenantRules: rules, + rules: rules, } } // Expired tells if a ref chunk is expired based on retention rules. -func (e *expirationChecker) Expired(ref *ChunkRef) bool { - r, ok := e.series[string(ref.SeriesID)] +func (e *expirationChecker) Expired(ref ChunkEntry) bool { // if the series matches a stream rules we'll use that. if ok && r.UserID == string(ref.UserID) { return ref.From.After(model.Now().Add(r.Duration)) } return ref.From.After(model.Now().Add(e.TenantRules.PerTenant(unsafeGetString(ref.UserID)))) } - -// all chunk ref.... chunkid,seriesid, from,to, userid (labels) (n + 1) -// -// HashValue: fmt.Sprintf("%02d:%s:%s:%s", shard, bucket.hashKey, metricName, v.Name), -// RangeValue: encodeRangeKey(labelSeriesRangeKeyV1, valueHash, seriesID, nil), -// Value: []byte(v.Value), - -// custom stream only {app="foo"} diff --git a/pkg/storage/stores/shipper/compactor/retention/index.go b/pkg/storage/stores/shipper/compactor/retention/index.go index f2bfbeb9058c5..1597e3f776f84 100644 --- a/pkg/storage/stores/shipper/compactor/retention/index.go +++ b/pkg/storage/stores/shipper/compactor/retention/index.go @@ -1,19 +1,14 @@ package retention import ( - "bytes" "errors" "fmt" - "sort" "strconv" "strings" "time" "github.com/cortexproject/cortex/pkg/chunk" - "github.com/cortexproject/cortex/pkg/util/math" "github.com/prometheus/common/model" - "github.com/prometheus/prometheus/pkg/labels" - "go.etcd.io/bbolt" "github.com/grafana/loki/pkg/storage" ) @@ -27,8 +22,6 @@ const ( separator = "\000" ) -var QueryParallelism = 100 - var ErrInvalidIndexKey = errors.New("invalid index key") type InvalidIndexKeyError struct { @@ -211,152 +204,6 @@ func parseLabelSeriesRangeKey(hashKey, rangeKey []byte) (LabelSeriesRangeKey, bo }, true, nil } -func findSeriesIDsForRules(db *bbolt.DB, config chunk.PeriodConfig, rules []StreamRule) ([][]string, error) { - schema, err := config.CreateSchema() - if err != nil { - return nil, err - } - // cover the whole table. - from, through := config.From.Time, config.From.Time.Add(config.IndexTables.Period) - result := make([][]string, len(rules)) - - for ruleIndex, rule := range rules { - incomingIDs := make(chan []string) - incomingErrors := make(chan error) - - for _, matcher := range rule.Matchers { - go func(matcher *labels.Matcher) { - ids, err := lookupSeriesByMatcher(db, schema, from, through, rule.UserID, matcher) - if err != nil { - incomingErrors <- err - return - } - incomingIDs <- ids - }(&matcher) - } - // intersect. and add to result. - var ids []string - var lastErr error - var initialized bool - for i := 0; i < len(rule.Matchers); i++ { - select { - case incoming := <-incomingIDs: - if !initialized { - ids = incoming - initialized = true - } else { - ids = intersectStrings(ids, incoming) - } - case err := <-incomingErrors: - lastErr = err - } - } - if lastErr != nil { - return nil, err - } - result[ruleIndex] = ids - } - - return result, nil -} - -func lookupSeriesByMatcher( - db *bbolt.DB, - schema chunk.BaseSchema, - from, through model.Time, - userID string, - matcher *labels.Matcher) ([]string, error) { - queries, err := schema.GetReadQueriesForMetricLabelValue( - from, through, userID, "logs", matcher.Name, matcher.Value) - if err != nil { - return nil, err - } - if len(queries) == 0 { - return nil, nil - } - if len(queries) == 1 { - return lookupSeriesByQuery(db, queries[0]) - } - queue := make(chan chunk.IndexQuery) - incomingResult := make(chan struct { - ids []string - err error - }) - n := math.Min(len(queries), QueryParallelism) - for i := 0; i < n; i++ { - go func() { - for { - query, ok := <-queue - if !ok { - return - } - res, err := lookupSeriesByQuery(db, query) - incomingResult <- struct { - ids []string - err error - }{res, err} - } - }() - } - go func() { - for _, query := range queries { - queue <- query - } - close(queue) - }() - - // Now receive all the results. - var ids []string - var lastErr error - for i := 0; i < len(queries); i++ { - res := <-incomingResult - if res.err != nil { - lastErr = res.err - continue - } - ids = append(ids, res.ids...) - } - sort.Strings(ids) - ids = uniqueStrings(ids) - return ids, lastErr -} - -func lookupSeriesByQuery(db *bbolt.DB, query chunk.IndexQuery) ([]string, error) { - start := []byte(query.HashValue + separator + string(query.RangeValueStart)) - rowPrefix := []byte(query.HashValue + separator) - var res []string - var components [][]byte - err := db.View(func(tx *bbolt.Tx) error { - bucket := tx.Bucket(bucketName) - if bucket == nil { - return nil - } - c := bucket.Cursor() - for k, v := c.Seek(start); k != nil; k, v = c.Next() { - // technically we can run regex that are not matching empty. - if len(query.ValueEqual) > 0 && !bytes.Equal(v, query.ValueEqual) { - continue - } - if !bytes.HasPrefix(k, rowPrefix) { - break - } - // parse series ID and add to res - _, r := decodeKey(k) - - components = decodeRangeKey(r, components) - if len(components) != 4 { - continue - } - // we store in label entries range keys: label hash value | seriesID | empty | type. - // and we want the seriesID - res = append(res, string(components[len(components)-3])) - } - return nil - }) - - return res, err -} - func schemaPeriodForTable(config storage.SchemaConfig, tableName string) (chunk.PeriodConfig, bool) { for _, schema := range config.Configs { periodIndex, err := strconv.ParseInt(strings.TrimPrefix(tableName, schema.IndexTables.Prefix), 10, 64) diff --git a/pkg/storage/stores/shipper/compactor/retention/marker.go b/pkg/storage/stores/shipper/compactor/retention/marker.go new file mode 100644 index 0000000000000..c90da911642e0 --- /dev/null +++ b/pkg/storage/stores/shipper/compactor/retention/marker.go @@ -0,0 +1,80 @@ +package retention + +import ( + "fmt" + "os" + "path/filepath" + "time" + + chunk_util "github.com/cortexproject/cortex/pkg/chunk/util" + shipper_util "github.com/grafana/loki/pkg/storage/stores/shipper/util" + "go.etcd.io/bbolt" +) + +type MarkerStorageWriter interface { + Put(chunkID []byte) error + Count() int64 + Close() error +} + +type markerStorageWriter struct { + db *bbolt.DB + tx *bbolt.Tx + bucket *bbolt.Bucket + + count int64 + fileName string +} + +func NewMarkerStorageWriter(workingDir string) (*markerStorageWriter, error) { + err := chunk_util.EnsureDirectory(filepath.Join(workingDir, markersFolder)) + if err != nil { + return nil, err + } + fileName := filepath.Join(workingDir, markersFolder, fmt.Sprint(time.Now().UnixNano())) + db, err := shipper_util.SafeOpenBoltdbFile(fileName) + if err != nil { + return nil, err + } + tx, err := db.Begin(true) + if err != nil { + return nil, err + } + bucket, err := tx.CreateBucketIfNotExists(chunkBucket) + if err != nil { + return nil, err + } + return &markerStorageWriter{ + db: db, + tx: tx, + bucket: bucket, + count: 0, + fileName: fileName, + }, err +} + +func (m *markerStorageWriter) Put(chunkID []byte) error { + if err := m.bucket.Put(chunkID, empty); err != nil { + return err + } + m.count++ + return nil +} + +func (m *markerStorageWriter) Count() int64 { + return m.count +} + +func (m *markerStorageWriter) Close() error { + if err := m.tx.Commit(); err != nil { + return err + } + if err := m.db.Close(); err != nil { + return err + } + // The marker file is empty we can remove. + if m.count > 0 { + return os.Remove(m.fileName) + } + return nil +} diff --git a/pkg/storage/stores/shipper/compactor/retention/retention.go b/pkg/storage/stores/shipper/compactor/retention/retention.go index f729ac8acf508..b0249ae77ea54 100644 --- a/pkg/storage/stores/shipper/compactor/retention/retention.go +++ b/pkg/storage/stores/shipper/compactor/retention/retention.go @@ -1,148 +1,195 @@ package retention import ( - "bytes" - "encoding/binary" + "context" "fmt" - "time" + "os" + "path" + "path/filepath" + "strings" "github.com/cortexproject/cortex/pkg/chunk" - "github.com/prometheus/common/model" + chunk_util "github.com/cortexproject/cortex/pkg/chunk/util" + util_log "github.com/cortexproject/cortex/pkg/util/log" + "github.com/go-kit/kit/log/level" "go.etcd.io/bbolt" + + "github.com/grafana/loki/pkg/storage" + "github.com/grafana/loki/pkg/storage/stores/shipper/util" + shipper_util "github.com/grafana/loki/pkg/storage/stores/shipper/util" ) var ( - bucketName = []byte("index") - chunkBucket = []byte("chunks") - empty = []byte("-") + bucketName = []byte("index") + chunkBucket = []byte("chunks") + empty = []byte("-") +) + +const ( logMetricName = "logs" + delimiter = "/" + markersFolder = "markers" ) -// todo we want to extract interfaces for series iterator and marker +type Marker struct { + workingDirectory string + config storage.SchemaConfig + objectClient chunk.ObjectClient + expiration ExpirationChecker +} -type MarkerTx interface { - Mark(id []byte) error +func NewMarker(workingDirectory string, config storage.SchemaConfig, objectClient chunk.ObjectClient, expiration ExpirationChecker) *Marker { + return &Marker{ + workingDirectory: workingDirectory, + config: config, + objectClient: objectClient, + } } -// type Marker interface { -// Begin() MarkerTx -// Commit() error -// Rollback() error -// } - -// todo clean up with interfaces. -// markForDelete delete index entries for expired chunk in `in` and add chunkid to delete in `marker`. -// All of this inside a single transaction. -func markForDelete(in, marker *bbolt.DB, expiration ExpirationChecker, config chunk.PeriodConfig) error { - return in.Update(func(inTx *bbolt.Tx) error { - bucket := inTx.Bucket(bucketName) +func (t *Marker) MarkTableForDelete(ctx context.Context, tableName string) error { + objects, err := util.ListDirectory(ctx, tableName, t.objectClient) + if err != nil { + return err + } + + if len(objects) != 1 { + // todo(1): in the future we would want to support more tables so that we can apply retention below 1d. + // for simplicity and to avoid conflict with compactor we'll support only compacted db file. + // Possibly we should apply retention right before the compactor upload compacted db. + + // todo(2): Depending on the retention rules we should be able to skip tables. + // For instance if there isn't a retention rules below 1 week, then we can skip the first 7 tables. + level.Debug(util_log.Logger).Log("msg", "skipping retention for non-compacted table", "name", tableName) + return nil + } + tableKey := objects[0].Key + + if shipper_util.IsDirectory(tableKey) { + level.Debug(util_log.Logger).Log("msg", "skipping retention no table file found", "key", tableKey) + return nil + } + + tableDirectory := path.Join(t.workingDirectory, tableName) + err = chunk_util.EnsureDirectory(tableDirectory) + if err != nil { + return err + } + + downloadAt := filepath.Join(t.workingDirectory, tableName) + + err = shipper_util.GetFileFromStorage(ctx, t.objectClient, tableKey, downloadAt) + if err != nil { + return err + } + + db, err := shipper_util.SafeOpenBoltdbFile(downloadAt) + if err != nil { + return err + } + + defer func() { + if err := os.Remove(db.Path()); err != nil { + level.Warn(util_log.Logger).Log("msg", "failed to removed downloaded db", "err", err) + } + }() + + schemaCfg, ok := schemaPeriodForTable(t.config, tableName) + if !ok { + return fmt.Errorf("could not find schema for table: %s", tableName) + } + + markerWriter, err := NewMarkerStorageWriter(t.workingDirectory) + if err != nil { + return fmt.Errorf("failed to create marker writer: %w", err) + } + + var empty bool + err = db.Update(func(tx *bbolt.Tx) error { + bucket := tx.Bucket(bucketName) if bucket == nil { return nil } - return marker.Update(func(outTx *bbolt.Tx) error { - // deleteChunkBucket, err := outTx.CreateBucket(chunkBucket) - // if err != nil { - // return err - // } - seriesMap := newUserSeriesMap() - // Phase 1 we mark chunkID that needs to be deleted in marker DB - c := bucket.Cursor() - var aliveChunk bool - - // it := newBoltdbChunkIndexIterator(bucket) - // for it.Next() { - // if it.Err() != nil { - // return it.Err() - // } - // ref := it.Entry() - // } - // if err := forAllChunkRef(c, func(ref *ChunkRef) error { - // if expiration.Expired(ref) { - // if err := deleteChunkBucket.Put(ref.ChunkID, empty); err != nil { - // return err - // } - // seriesMap.Add(ref.SeriesID, ref.UserID) - // if err := c.Delete(); err != nil { - // return err - // } - // return nil - // } - // // we found a key that will stay. - // aliveChunk = true - // return nil - // }); err != nil { - // return err - // } - // shortcircuit: no chunks remaining we can delete everything. - if !aliveChunk { - return inTx.DeleteBucket(bucketName) - } - // Phase 2 verify series that have marked chunks have still other chunks in the index per buckets. - // If not this means we can delete labels index entries for the given series. - return seriesMap.ForEach(func(seriesID, userID []byte) error { - // for all buckets if seek to bucket.hashKey + ":" + string(seriesID) is not nil we still have chunks. - bucketHashes := allBucketsHashes(config, unsafeGetString(userID)) - for _, bucketHash := range bucketHashes { - if key, _ := c.Seek([]byte(bucketHash + ":" + string(seriesID))); key != nil { - return nil - } - // this bucketHash doesn't contains the given series. Let's remove it. - // if err := forAllLabelRef(c, bucketHash, seriesID, config, func(_ *LabelIndexRef) error { - // if err := c.Delete(); err != nil { - // return err - // } - // return nil - // }); err != nil { - // return err - // } - } - return nil - }) - }) - }) -} -func forAllLabelRef(c *bbolt.Cursor, bucketHash string, seriesID []byte, config chunk.PeriodConfig, callback func(ref []byte) error) error { - // todo reuse memory and refactor - var ( - prefix string - ) - // todo refactor ParseLabelRef. => keyType,SeriesID - switch config.Schema { - case "v11": - shard := binary.BigEndian.Uint32(seriesID) % config.RowShards - prefix = fmt.Sprintf("%02d:%s:%s", shard, bucketHash, logMetricName) - default: - prefix = fmt.Sprintf("%s:%s", bucketHash, logMetricName) - } - for k, _ := c.Seek([]byte(prefix)); k != nil; k, _ = c.Next() { - ref, ok, err := parseLabelIndexSeriesID(decodeKey(k)) + chunkIt, err := newChunkIndexIterator(bucket, schemaCfg) if err != nil { - return err + return fmt.Errorf("failed to create chunk index iterator: %w", err) } - if !ok || !bytes.Equal(seriesID, ref) { - continue - } - if err := callback(ref); err != nil { + + seriesCleaner := newSeriesCleaner(bucket, schemaCfg) + empty, err = markforDelete(markerWriter, chunkIt, seriesCleaner, nil) + if err != nil { return err } + if err := markerWriter.Close(); err != nil { + return fmt.Errorf("failed to close marker writer: %w", err) + } + return nil + }) + if err != nil { + return err } - - return nil + if err := db.Close(); err != nil { + return err + } + // either delete if all entries are removed or upload the new file. + if empty { + return t.objectClient.DeleteObject(ctx, tableName+delimiter) + } + // No chunks to delete means no changes to the remote index, we don't need to upload it. + if markerWriter.Count() == 0 { + return nil + } + return t.uploadDB(ctx, db, tableKey) } -func allBucketsHashes(config chunk.PeriodConfig, userID string) []string { - return bucketsHashes(config.From.Time, config.From.Add(config.IndexTables.Period), config, userID) +func (t *Marker) uploadDB(ctx context.Context, db *bbolt.DB, objectKey string) error { + sourcePath := db.Path() + if strings.HasSuffix(objectKey, ".gz") { + compressedPath := fmt.Sprintf("%s.gz", sourcePath) + err := shipper_util.CompressFile(sourcePath, compressedPath) + if err != nil { + return err + } + defer func() { + os.Remove(compressedPath) + }() + sourcePath = compressedPath + } + sourceFile, err := os.Open(sourcePath) + if err != nil { + return err + } + defer func() { + if err := sourceFile.Close(); err != nil { + level.Error(util_log.Logger).Log("msg", "failed to close file", "path", sourceFile, "err", err) + } + }() + return t.objectClient.PutObject(ctx, objectKey, sourceFile) } -func bucketsHashes(from, through model.Time, config chunk.PeriodConfig, userID string) []string { - var ( - fromDay = from.Unix() / int64(config.IndexTables.Period/time.Second) - throughDay = through.Unix() / int64(config.IndexTables.Period/time.Second) - result = []string{} - ) - for i := fromDay; i <= throughDay; i++ { - result = append(result, fmt.Sprintf("%s:d%d", userID, i)) +func markforDelete(marker MarkerStorageWriter, chunkIt ChunkEntryIterator, seriesCleaner SeriesCleaner, expiration ExpirationChecker) (bool, error) { + seriesMap := newUserSeriesMap() + var empty bool + for chunkIt.Next() { + if chunkIt.Err() != nil { + return false, chunkIt.Err() + } + c := chunkIt.Entry() + if expiration.Expired(c) { + seriesMap.Add(c.SeriesID, c.UserID) + if err := chunkIt.Delete(); err != nil { + return false, err + } + if err := marker.Put(c.ChunkID); err != nil { + return false, err + } + } + empty = false + } + if empty { + return true, nil } - return result + return false, seriesMap.ForEach(func(seriesID, userID []byte) error { + return seriesCleaner.Cleanup(seriesID, userID) + }) } diff --git a/pkg/storage/stores/shipper/compactor/retention/retention_test.go b/pkg/storage/stores/shipper/compactor/retention/retention_test.go index cdfb98e06ca7f..e2502ff3a47cb 100644 --- a/pkg/storage/stores/shipper/compactor/retention/retention_test.go +++ b/pkg/storage/stores/shipper/compactor/retention/retention_test.go @@ -161,29 +161,3 @@ func labelsString(ls labels.Labels) string { return b.String() } - -func Test_AllBucketHash(t *testing.T) { - fmt.Fprintf(os.Stdout, "BucketHash %+v", allBucketsHashes(chunk.PeriodConfig{ - From: chunk.DayTime{Time: model.Now()}, - IndexType: "boltdb", - ObjectType: "filesystem", - Schema: "v9", - IndexTables: chunk.PeriodicTableConfig{ - Prefix: "index_", - Period: time.Hour * 24, - }, - RowShards: 16, - }, "foo")) - - fmt.Fprintf(os.Stdout, "BucketHash %+v", bucketsHashes(model.Now().Add(-2*time.Hour), model.Now().Add(48*time.Hour), chunk.PeriodConfig{ - From: chunk.DayTime{Time: model.Now()}, - IndexType: "boltdb", - ObjectType: "filesystem", - Schema: "v9", - IndexTables: chunk.PeriodicTableConfig{ - Prefix: "index_", - Period: time.Hour * 24, - }, - RowShards: 16, - }, "foo")) -} diff --git a/pkg/storage/stores/shipper/compactor/retention/rules.go b/pkg/storage/stores/shipper/compactor/retention/rules.go index 9d4679d53a7f2..297836fdc99f3 100644 --- a/pkg/storage/stores/shipper/compactor/retention/rules.go +++ b/pkg/storage/stores/shipper/compactor/retention/rules.go @@ -35,14 +35,9 @@ type StreamRule struct { Duration time.Duration // in case a series matches multiple Rules takes the one with higher weight or the first Weight int - UserID string } type Rules interface { - TenantRules - PerStream() []StreamRule -} - -type TenantRules interface { PerTenant(userID string) time.Duration + PerStream(userID string) []StreamRule } diff --git a/pkg/storage/stores/shipper/compactor/retention/util.go b/pkg/storage/stores/shipper/compactor/retention/util.go index 674fe10141eed..b88d9d84e1e3f 100644 --- a/pkg/storage/stores/shipper/compactor/retention/util.go +++ b/pkg/storage/stores/shipper/compactor/retention/util.go @@ -18,44 +18,44 @@ func unsafeGetBytes(s string) []byte { return buf } -func uniqueStrings(cs []string) []string { - if len(cs) == 0 { - return []string{} - } +// func uniqueStrings(cs []string) []string { +// if len(cs) == 0 { +// return []string{} +// } - result := make([]string, 1, len(cs)) - result[0] = cs[0] - i, j := 0, 1 - for j < len(cs) { - if result[i] == cs[j] { - j++ - continue - } - result = append(result, cs[j]) - i++ - j++ - } - return result -} +// result := make([]string, 1, len(cs)) +// result[0] = cs[0] +// i, j := 0, 1 +// for j < len(cs) { +// if result[i] == cs[j] { +// j++ +// continue +// } +// result = append(result, cs[j]) +// i++ +// j++ +// } +// return result +// } -func intersectStrings(left, right []string) []string { - var ( - i, j = 0, 0 - result = []string{} - ) - for i < len(left) && j < len(right) { - if left[i] == right[j] { - result = append(result, left[i]) - } +// func intersectStrings(left, right []string) []string { +// var ( +// i, j = 0, 0 +// result = []string{} +// ) +// for i < len(left) && j < len(right) { +// if left[i] == right[j] { +// result = append(result, left[i]) +// } - if left[i] < right[j] { - i++ - } else { - j++ - } - } - return result -} +// if left[i] < right[j] { +// i++ +// } else { +// j++ +// } +// } +// return result +// } // Build an index key, encoded as multiple parts separated by a 0 byte, with extra space at the end. func buildRangeValue(extra int, ss ...[]byte) []byte { diff --git a/pkg/storage/stores/shipper/compactor/table.go b/pkg/storage/stores/shipper/compactor/table.go index 695e2a472bf99..a1752c12b7dce 100644 --- a/pkg/storage/stores/shipper/compactor/table.go +++ b/pkg/storage/stores/shipper/compactor/table.go @@ -15,6 +15,7 @@ import ( "github.com/go-kit/kit/log/level" "go.etcd.io/bbolt" + "github.com/grafana/loki/pkg/storage/stores/shipper/util" shipper_util "github.com/grafana/loki/pkg/storage/stores/shipper/util" ) @@ -61,9 +62,7 @@ func newTable(ctx context.Context, workingDirectory string, objectClient chunk.O } func (t *table) compact() error { - // The forward slash here needs to stay because we are trying to list contents of a directory without it we will get the name of the same directory back with hosted object stores. - // This is due to the object stores not having a concept of directories. - objects, _, err := t.storageClient.List(t.ctx, t.name+delimiter, delimiter) + objects, err := util.ListDirectory(t.ctx, t.name, t.storageClient) if err != nil { return err } diff --git a/pkg/storage/stores/shipper/util/util.go b/pkg/storage/stores/shipper/util/util.go index 48bedcdf75a0d..e030522d29de9 100644 --- a/pkg/storage/stores/shipper/util/util.go +++ b/pkg/storage/stores/shipper/util/util.go @@ -191,3 +191,13 @@ func ValidateSharedStoreKeyPrefix(prefix string) error { return nil } + +func ListDirectory(ctx context.Context, dirName string, objectClient chunk.ObjectClient) ([]chunk.StorageObject, error) { + // The forward slash here needs to stay because we are trying to list contents of a directory without it we will get the name of the same directory back with hosted object stores. + // This is due to the object stores not having a concept of directories. + objects, _, err := objectClient.List(ctx, dirName+delimiter, delimiter) + if err != nil { + return nil, err + } + return objects, nil +} From cdc752dd438393ab0e0cfce56e75e9f9b496db35 Mon Sep 17 00:00:00 2001 From: Cyril Tovena Date: Fri, 16 Apr 2021 11:28:11 +0200 Subject: [PATCH 14/42] Hooking limit retention config. Signed-off-by: Cyril Tovena --- pkg/loki/loki.go | 2 +- pkg/loki/modules.go | 2 +- pkg/loki/runtime_config.go | 16 ++- pkg/loki/runtime_config_test.go | 116 ++++++++++++++++++ .../stores/shipper/compactor/compactor.go | 43 ++++--- .../shipper/compactor/retention/expiration.go | 49 ++++++-- .../compactor/retention/expiration_test.go | 53 +++++--- .../shipper/compactor/retention/retention.go | 2 +- .../compactor/retention/retention_test.go | 21 +--- .../shipper/compactor/retention/rules.go | 43 ------- .../shipper/compactor/retention/util.go | 39 ------ .../shipper/compactor/retention/util_test.go | 28 ++--- pkg/util/validation/limits.go | 50 ++++++++ 13 files changed, 294 insertions(+), 170 deletions(-) create mode 100644 pkg/loki/runtime_config_test.go delete mode 100644 pkg/storage/stores/shipper/compactor/retention/rules.go diff --git a/pkg/loki/loki.go b/pkg/loki/loki.go index 3ae9d20627bd4..4d3d4dac06689 100644 --- a/pkg/loki/loki.go +++ b/pkg/loki/loki.go @@ -376,7 +376,7 @@ func (t *Loki) setupModuleManager() error { QueryFrontend: {Server, Overrides, TenantConfigs}, Ruler: {Ring, Server, Store, RulerStorage, IngesterQuerier, Overrides, TenantConfigs}, TableManager: {Server}, - Compactor: {Server}, + Compactor: {Server, Overrides}, IngesterQuerier: {Ring}, All: {Querier, Ingester, Distributor, TableManager, Ruler}, } diff --git a/pkg/loki/modules.go b/pkg/loki/modules.go index bdaaf934750ec..b09b409453921 100644 --- a/pkg/loki/modules.go +++ b/pkg/loki/modules.go @@ -580,7 +580,7 @@ func (t *Loki) initCompactor() (services.Service, error) { if err != nil { return nil, err } - t.compactor, err = compactor.NewCompactor(t.cfg.CompactorConfig, t.cfg.StorageConfig.Config, t.cfg.SchemaConfig, prometheus.DefaultRegisterer) + t.compactor, err = compactor.NewCompactor(t.cfg.CompactorConfig, t.cfg.StorageConfig.Config, t.cfg.SchemaConfig, t.overrides, prometheus.DefaultRegisterer) if err != nil { return nil, err } diff --git a/pkg/loki/runtime_config.go b/pkg/loki/runtime_config.go index 2f753df774af2..76ec77e1bc418 100644 --- a/pkg/loki/runtime_config.go +++ b/pkg/loki/runtime_config.go @@ -1,6 +1,7 @@ package loki import ( + "fmt" "io" "github.com/cortexproject/cortex/pkg/ring/kv" @@ -21,15 +22,26 @@ type runtimeConfigValues struct { Multi kv.MultiRuntimeConfig `yaml:"multi_kv_config"` } +func (r runtimeConfigValues) validate() error { + for t, c := range r.TenantLimits { + if err := c.Validate(); err != nil { + return fmt.Errorf("invalid override for tenant %s: %w", t, err) + } + } + return nil +} + func loadRuntimeConfig(r io.Reader) (interface{}, error) { - var overrides = &runtimeConfigValues{} + overrides := &runtimeConfigValues{} decoder := yaml.NewDecoder(r) decoder.SetStrict(true) if err := decoder.Decode(&overrides); err != nil { return nil, err } - + if err := overrides.validate(); err != nil { + return nil, err + } return overrides, nil } diff --git a/pkg/loki/runtime_config_test.go b/pkg/loki/runtime_config_test.go new file mode 100644 index 0000000000000..b5b5cf7ef22de --- /dev/null +++ b/pkg/loki/runtime_config_test.go @@ -0,0 +1,116 @@ +package loki + +import ( + "context" + "flag" + "io" + "io/ioutil" + "strings" + "testing" + "time" + + "github.com/cortexproject/cortex/pkg/util/runtimeconfig" + "github.com/grafana/loki/pkg/util/validation" + "github.com/prometheus/client_golang/prometheus" + "github.com/prometheus/prometheus/pkg/labels" + "github.com/stretchr/testify/require" +) + +func Test_LoadRetentionRules(t *testing.T) { + overrides := newTestOverrides(t, + ` +overrides: + "1": + creation_grace_period: 48h + "29": + creation_grace_period: 48h + ingestion_burst_size_mb: 140 + ingestion_rate_mb: 120 + max_concurrent_tail_requests: 1000 + max_global_streams_per_user: 100000 + max_label_names_per_series: 30 + max_query_parallelism: 256 + split_queries_by_interval: 15m + retention_period: 1440h + retention_stream: + - selector: '{app="foo"}' + period: 48h + priority: 10 + - selector: '{namespace="bar", cluster=~"fo.*|b.+|[1-2]"}' + period: 24h + priority: 5 +`) + require.Equal(t, 31*24*time.Hour, overrides.RetentionPeriod("1")) // default + require.Equal(t, 2*30*24*time.Hour, overrides.RetentionPeriod("29")) // overrides + require.Equal(t, []validation.StreamRetention(nil), overrides.StreamRetention("1")) + require.Equal(t, []validation.StreamRetention{ + {Period: 48 * time.Hour, Priority: 10, Selector: `{app="foo"}`, Matchers: []*labels.Matcher{ + labels.MustNewMatcher(labels.MatchEqual, "app", "foo"), + }}, + {Period: 24 * time.Hour, Priority: 5, Selector: `{namespace="bar", cluster=~"fo.*|b.+|[1-2]"}`, Matchers: []*labels.Matcher{ + labels.MustNewMatcher(labels.MatchEqual, "namespace", "bar"), + labels.MustNewMatcher(labels.MatchRegexp, "cluster", "fo.*|b.+|[1-2]"), + }}, + }, overrides.StreamRetention("29")) +} + +func Test_ValidateRules(t *testing.T) { + _, err := loadRuntimeConfig(strings.NewReader( + ` +overrides: + "29": + retention_stream: + - selector: '{app=foo"}' + period: 48h + priority: 10 + - selector: '{namespace="bar", cluster=~"fo.*|b.+|[1-2]"}' + period: 24h + priority: 10 +`)) + require.Equal(t, "invalid override for tenant 29: invalid labels matchers: parse error at line 1, col 6: syntax error: unexpected IDENTIFIER, expecting STRING", err.Error()) + _, err = loadRuntimeConfig(strings.NewReader( + ` +overrides: + "29": + retention_stream: + - selector: '{app="foo"}' + period: 5h + priority: 10 +`)) + require.Equal(t, "invalid override for tenant 29: retention period must be >= 24h was 5h0m0s", err.Error()) +} + +func newTestOverrides(t *testing.T, yaml string) *validation.Overrides { + t.Helper() + f, err := ioutil.TempFile(t.TempDir(), "bar") + require.NoError(t, err) + path := f.Name() + // fake loader to load from string instead of file. + loader := func(_ io.Reader) (interface{}, error) { + return loadRuntimeConfig(strings.NewReader(yaml)) + } + cfg := runtimeconfig.ManagerConfig{ + ReloadPeriod: 1 * time.Second, + Loader: loader, + LoadPath: path, + } + flagset := flag.NewFlagSet("", flag.PanicOnError) + var defaults validation.Limits + defaults.RegisterFlags(flagset) + require.NoError(t, flagset.Parse(nil)) + validation.SetDefaultLimitsForYAMLUnmarshalling(defaults) + + runtimeConfig, err := runtimeconfig.NewRuntimeConfigManager(cfg, prometheus.DefaultRegisterer) + require.NoError(t, err) + + require.NoError(t, runtimeConfig.StartAsync(context.Background())) + require.NoError(t, runtimeConfig.AwaitRunning(context.Background())) + defer func() { + runtimeConfig.StopAsync() + require.NoError(t, runtimeConfig.AwaitTerminated(context.Background())) + }() + + overrides, err := validation.NewOverrides(defaults, tenantLimitsFromRuntimeConfig(runtimeConfig)) + require.NoError(t, err) + return overrides +} diff --git a/pkg/storage/stores/shipper/compactor/compactor.go b/pkg/storage/stores/shipper/compactor/compactor.go index e2de9783b12bc..3a2a744d88404 100644 --- a/pkg/storage/stores/shipper/compactor/compactor.go +++ b/pkg/storage/stores/shipper/compactor/compactor.go @@ -32,6 +32,7 @@ type Config struct { SharedStoreType string `yaml:"shared_store"` SharedStoreKeyPrefix string `yaml:"shared_store_key_prefix"` CompactionInterval time.Duration `yaml:"compaction_interval"` + RetentionEnabled bool `yaml:"retention_enabled"` RetentionInterval time.Duration `yaml:"retention_interval"` } @@ -41,7 +42,8 @@ func (cfg *Config) RegisterFlags(f *flag.FlagSet) { f.StringVar(&cfg.SharedStoreType, "boltdb.shipper.compactor.shared-store", "", "Shared store used for storing boltdb files. Supported types: gcs, s3, azure, swift, filesystem") f.StringVar(&cfg.SharedStoreKeyPrefix, "boltdb.shipper.compactor.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.DurationVar(&cfg.CompactionInterval, "boltdb.shipper.compactor.compaction-interval", 2*time.Hour, "Interval at which to re-run the compaction operation.") - f.DurationVar(&cfg.CompactionInterval, "boltdb.shipper.compactor.retention-interval", 2*time.Hour, "Interval at which to re-run the retention operation.") + f.DurationVar(&cfg.RetentionInterval, "boltdb.shipper.compactor.retention-interval", 2*time.Hour, "Interval at which to re-run the retention operation.") + f.BoolVar(&cfg.RetentionEnabled, "boltdb.shipper.compactor.retention-enabled", false, "Activate custom (per-stream,per-tenant) retention.") } func (cfg *Config) IsDefaults() bool { @@ -64,7 +66,7 @@ type Compactor struct { metrics *metrics } -func NewCompactor(cfg Config, storageConfig storage.Config, schemaConfig loki_storage.SchemaConfig, r prometheus.Registerer) (*Compactor, error) { +func NewCompactor(cfg Config, storageConfig storage.Config, schemaConfig loki_storage.SchemaConfig, limits retention.Limits, r prometheus.Registerer) (*Compactor, error) { if cfg.IsDefaults() { return nil, errors.New("Must specify compactor config") } @@ -85,7 +87,7 @@ func NewCompactor(cfg Config, storageConfig storage.Config, schemaConfig loki_st cfg: cfg, objectClient: prefixedClient, metrics: newMetrics(r), - tableMarker: retention.NewMarker(filepath.Join(cfg.WorkingDirectory, "retention"), schemaConfig, prefixedClient, nil), + tableMarker: retention.NewMarker(filepath.Join(cfg.WorkingDirectory, "retention"), schemaConfig, prefixedClient, retention.NewExpirationChecker(limits)), } compactor.Service = services.NewBasicService(nil, compactor.loop, nil) @@ -106,7 +108,7 @@ func (c *Compactor) loop(ctx context.Context) error { } } var wg sync.WaitGroup - wg.Add(2) + wg.Add(1) go func() { defer wg.Done() runCompaction() @@ -123,22 +125,25 @@ func (c *Compactor) loop(ctx context.Context) error { } } }() - - go func() { - defer wg.Done() - - ticker := time.NewTicker(c.cfg.RetentionInterval) - defer ticker.Stop() - - for { - select { - case <-ticker.C: - runRetention() - case <-ctx.Done(): - return + if c.cfg.RetentionEnabled { + wg.Add(1) + go func() { + defer wg.Done() + + ticker := time.NewTicker(c.cfg.RetentionInterval) + defer ticker.Stop() + + for { + select { + case <-ticker.C: + runRetention() + case <-ctx.Done(): + return + } } - } - }() + }() + } + wg.Wait() return nil } diff --git a/pkg/storage/stores/shipper/compactor/retention/expiration.go b/pkg/storage/stores/shipper/compactor/retention/expiration.go index fd058ffd44e9e..394ff5398c27b 100644 --- a/pkg/storage/stores/shipper/compactor/retention/expiration.go +++ b/pkg/storage/stores/shipper/compactor/retention/expiration.go @@ -1,6 +1,9 @@ package retention import ( + "time" + + "github.com/grafana/loki/pkg/util/validation" "github.com/prometheus/common/model" ) @@ -9,20 +12,52 @@ type ExpirationChecker interface { } type expirationChecker struct { - rules Rules + limits Limits +} + +type Limits interface { + RetentionPeriod(userID string) time.Duration + StreamRetention(userID string) []validation.StreamRetention } -func NewExpirationChecker(rules Rules) ExpirationChecker { +func NewExpirationChecker(limits Limits) ExpirationChecker { return &expirationChecker{ - rules: rules, + limits: limits, } } // Expired tells if a ref chunk is expired based on retention rules. func (e *expirationChecker) Expired(ref ChunkEntry) bool { - // if the series matches a stream rules we'll use that. - if ok && r.UserID == string(ref.UserID) { - return ref.From.After(model.Now().Add(r.Duration)) + userID := unsafeGetString(ref.UserID) + streamRetentions := e.limits.StreamRetention(userID) + globalRetention := e.limits.RetentionPeriod(userID) + var ( + matchedRule validation.StreamRetention + found bool + ) +Outer: + for _, streamRetention := range streamRetentions { + for _, m := range streamRetention.Matchers { + if !m.Matches(ref.Labels.Get(m.Name)) { + continue Outer + } + } + // the rule is matched. + if found { + // if the current matched rule has a higher priority we keep it. + if matchedRule.Priority > streamRetention.Priority { + continue + } + // if priority is equal we keep the lowest retention. + if matchedRule.Priority == streamRetention.Priority && matchedRule.Period <= streamRetention.Period { + continue + } + } + found = true + matchedRule = streamRetention + } + if found { + return ref.From.After(model.Now().Add(matchedRule.Period)) } - return ref.From.After(model.Now().Add(e.TenantRules.PerTenant(unsafeGetString(ref.UserID)))) + return ref.From.After(model.Now().Add(globalRetention)) } diff --git a/pkg/storage/stores/shipper/compactor/retention/expiration_test.go b/pkg/storage/stores/shipper/compactor/retention/expiration_test.go index 3eb1bf269146f..b98e7d863518a 100644 --- a/pkg/storage/stores/shipper/compactor/retention/expiration_test.go +++ b/pkg/storage/stores/shipper/compactor/retention/expiration_test.go @@ -4,42 +4,57 @@ import ( "testing" "time" + "github.com/grafana/loki/pkg/util/validation" "github.com/prometheus/common/model" "github.com/prometheus/prometheus/pkg/labels" "github.com/stretchr/testify/require" ) +type fakeLimits struct { + perTenant map[string]time.Duration + perStream map[string][]validation.StreamRetention +} + +func (f fakeLimits) RetentionPeriod(userID string) time.Duration { + return f.perTenant[userID] +} + +func (f fakeLimits) StreamRetention(userID string) []validation.StreamRetention { + return f.perStream[userID] +} + func Test_expirationChecker_Expired(t *testing.T) { - e := NewExpirationChecker(map[string]StreamRule{ - string(labelsSeriesID(labels.Labels{labels.Label{Name: "foo", Value: "bar"}})): { - UserID: "1", - Duration: 2 * time.Hour, - }, - string(labelsSeriesID(labels.Labels{labels.Label{Name: "foo", Value: "bar"}})): { - UserID: "2", - Duration: 1 * time.Hour, - }, - }, fakeRule{ - tenants: map[string]time.Duration{ + e := NewExpirationChecker(&fakeLimits{ + perTenant: map[string]time.Duration{ "1": time.Hour, "2": 24 * time.Hour, }, + perStream: map[string][]validation.StreamRetention{ + "1": { + {Period: 2 * time.Hour, Priority: 10, Matchers: []*labels.Matcher{labels.MustNewMatcher(labels.MatchEqual, "foo", "bar")}}, + {Period: 2 * time.Hour, Priority: 1, Matchers: []*labels.Matcher{labels.MustNewMatcher(labels.MatchRegexp, "foo", "ba.+")}}, + }, + "2": { + {Period: 1 * time.Hour, Matchers: []*labels.Matcher{labels.MustNewMatcher(labels.MatchEqual, "foo", "bar")}}, + {Period: 2 * time.Hour, Matchers: []*labels.Matcher{labels.MustNewMatcher(labels.MatchRegexp, "foo", "ba.")}}, + }, + }, }) tests := []struct { name string - ref ChunkRef + ref ChunkEntry want bool }{ - {"expired tenant", newChunkRef("1", `{foo="buzz"}`, model.Now().Add(2*time.Hour), model.Now().Add(3*time.Hour)), true}, - {"just expired tenant", newChunkRef("1", `{foo="buzz"}`, model.Now().Add(1*time.Hour), model.Now().Add(3*time.Hour)), false}, - {"not expired tenant", newChunkRef("1", `{foo="buzz"}`, model.Now().Add(30*time.Minute), model.Now().Add(3*time.Hour)), false}, - {"not expired tenant by far", newChunkRef("2", `{foo="buzz"}`, model.Now().Add(30*time.Minute), model.Now().Add(3*time.Hour)), false}, - {"expired stream override", newChunkRef("2", `{foo="bar"}`, model.Now().Add(3*time.Hour), model.Now().Add(4*time.Hour)), true}, - {"non expired stream override", newChunkRef("1", `{foo="bar"}`, model.Now().Add(1*time.Hour), model.Now().Add(4*time.Hour)), false}, + {"expired tenant", newChunkEntry("1", `{foo="buzz"}`, model.Now().Add(2*time.Hour), model.Now().Add(3*time.Hour)), true}, + {"just expired tenant", newChunkEntry("1", `{foo="buzz"}`, model.Now().Add(1*time.Hour), model.Now().Add(3*time.Hour)), false}, + {"not expired tenant", newChunkEntry("1", `{foo="buzz"}`, model.Now().Add(30*time.Minute), model.Now().Add(3*time.Hour)), false}, + {"not expired tenant by far", newChunkEntry("2", `{foo="buzz"}`, model.Now().Add(30*time.Minute), model.Now().Add(3*time.Hour)), false}, + {"expired stream override", newChunkEntry("2", `{foo="bar"}`, model.Now().Add(3*time.Hour), model.Now().Add(4*time.Hour)), true}, + {"non expired stream override", newChunkEntry("1", `{foo="bar"}`, model.Now().Add(1*time.Hour), model.Now().Add(4*time.Hour)), false}, } for _, tt := range tests { t.Run(tt.name, func(t *testing.T) { - require.Equal(t, tt.want, e.Expired(&tt.ref)) + require.Equal(t, tt.want, e.Expired(tt.ref)) }) } } diff --git a/pkg/storage/stores/shipper/compactor/retention/retention.go b/pkg/storage/stores/shipper/compactor/retention/retention.go index b0249ae77ea54..a0efc0aaafcb9 100644 --- a/pkg/storage/stores/shipper/compactor/retention/retention.go +++ b/pkg/storage/stores/shipper/compactor/retention/retention.go @@ -116,7 +116,7 @@ func (t *Marker) MarkTableForDelete(ctx context.Context, tableName string) error } seriesCleaner := newSeriesCleaner(bucket, schemaCfg) - empty, err = markforDelete(markerWriter, chunkIt, seriesCleaner, nil) + empty, err = markforDelete(markerWriter, chunkIt, seriesCleaner, t.expiration) if err != nil { return err } diff --git a/pkg/storage/stores/shipper/compactor/retention/retention_test.go b/pkg/storage/stores/shipper/compactor/retention/retention_test.go index e2502ff3a47cb..1afdba4083c8b 100644 --- a/pkg/storage/stores/shipper/compactor/retention/retention_test.go +++ b/pkg/storage/stores/shipper/compactor/retention/retention_test.go @@ -33,20 +33,7 @@ func Test_Retention(t *testing.T) { store.Stop() - retentionRules := fakeRule{ - streams: []StreamRule{ - { - UserID: "1", - Matchers: []labels.Matcher{ - { - Type: labels.MatchEqual, - Name: "foo", - Value: "bar", - }, - }, - }, - }, - } + // retentionRules := fakeRule{} // 1- Get all series ID for given retention per stream.... // 2 - Delete from index and Mark for delete all chunk based on retention with seriesID/tenantID. @@ -67,13 +54,9 @@ func Test_Retention(t *testing.T) { continue } fmt.Fprintf(os.Stdout, "Found Schema for Table %s => %+v\n", table.name, currentSchema) - ids, err := findSeriesIDsForRules(table.DB, currentSchema, retentionRules.PerStream()) - require.NoError(t, err) - _ = NewExpirationChecker(getSeriesPerRule(ids, retentionRules.PerStream()), retentionRules) - // markForDelete(db, marker*bbolt.DB, checker, currentSchema) + _ = NewExpirationChecker(nil) - fmt.Fprintf(os.Stdout, "Found IDS for rules %+v\n", ids) require.NoError(t, table.DB.Update(func(tx *bbolt.Tx) error { return tx.Bucket(bucketName).ForEach(func(k, v []byte) error { diff --git a/pkg/storage/stores/shipper/compactor/retention/rules.go b/pkg/storage/stores/shipper/compactor/retention/rules.go deleted file mode 100644 index 297836fdc99f3..0000000000000 --- a/pkg/storage/stores/shipper/compactor/retention/rules.go +++ /dev/null @@ -1,43 +0,0 @@ -package retention - -import ( - "time" - - "github.com/prometheus/prometheus/pkg/labels" -) - -// getSeriesPerRule merges rules per seriesID, if multiple series matches a rules, the strongest weight wins. -// Otherwise the shorter retention wins. -func getSeriesPerRule(series [][]string, rules []StreamRule) map[string]StreamRule { - res := map[string]StreamRule{} - for i, seriesPerRules := range series { - for _, series := range seriesPerRules { - r, ok := res[series] - newRule := rules[i] - if ok { - // we already have a rules for this series. - if newRule.Weight > r.Weight { - res[series] = newRule - } - if newRule.Weight == r.Weight && newRule.Duration < r.Duration { - res[series] = newRule - } - continue - } - res[series] = newRule - } - } - return res -} - -type StreamRule struct { - Matchers []labels.Matcher - Duration time.Duration - // in case a series matches multiple Rules takes the one with higher weight or the first - Weight int -} - -type Rules interface { - PerTenant(userID string) time.Duration - PerStream(userID string) []StreamRule -} diff --git a/pkg/storage/stores/shipper/compactor/retention/util.go b/pkg/storage/stores/shipper/compactor/retention/util.go index b88d9d84e1e3f..991cc3e0225c9 100644 --- a/pkg/storage/stores/shipper/compactor/retention/util.go +++ b/pkg/storage/stores/shipper/compactor/retention/util.go @@ -18,45 +18,6 @@ func unsafeGetBytes(s string) []byte { return buf } -// func uniqueStrings(cs []string) []string { -// if len(cs) == 0 { -// return []string{} -// } - -// result := make([]string, 1, len(cs)) -// result[0] = cs[0] -// i, j := 0, 1 -// for j < len(cs) { -// if result[i] == cs[j] { -// j++ -// continue -// } -// result = append(result, cs[j]) -// i++ -// j++ -// } -// return result -// } - -// func intersectStrings(left, right []string) []string { -// var ( -// i, j = 0, 0 -// result = []string{} -// ) -// for i < len(left) && j < len(right) { -// if left[i] == right[j] { -// result = append(result, left[i]) -// } - -// if left[i] < right[j] { -// i++ -// } else { -// j++ -// } -// } -// return result -// } - // Build an index key, encoded as multiple parts separated by a 0 byte, with extra space at the end. func buildRangeValue(extra int, ss ...[]byte) []byte { length := extra diff --git a/pkg/storage/stores/shipper/compactor/retention/util_test.go b/pkg/storage/stores/shipper/compactor/retention/util_test.go index 8494796f77c8f..06e9594801228 100644 --- a/pkg/storage/stores/shipper/compactor/retention/util_test.go +++ b/pkg/storage/stores/shipper/compactor/retention/util_test.go @@ -71,29 +71,19 @@ var allSchemas = []struct { {"v11", model.Earliest.Add(49 * time.Hour), schemaCfg.Configs[2]}, } -type fakeRule struct { - streams []StreamRule - tenants map[string]time.Duration -} - -func (f fakeRule) PerTenant(userID string) time.Duration { - return f.tenants[userID] -} - -func (f fakeRule) PerStream() []StreamRule { - return f.streams -} - -func newChunkRef(userID, labels string, from, through model.Time) ChunkRef { +func newChunkEntry(userID, labels string, from, through model.Time) ChunkEntry { lbs, err := logql.ParseLabels(labels) if err != nil { panic(err) } - return ChunkRef{ - UserID: []byte(userID), - SeriesID: labelsSeriesID(lbs), - From: from, - Through: through, + return ChunkEntry{ + ChunkRef: ChunkRef{ + UserID: []byte(userID), + SeriesID: labelsSeriesID(lbs), + From: from, + Through: through, + }, + Labels: lbs, } } diff --git a/pkg/util/validation/limits.go b/pkg/util/validation/limits.go index 84a5ff846bdd1..414448ecdd768 100644 --- a/pkg/util/validation/limits.go +++ b/pkg/util/validation/limits.go @@ -2,9 +2,12 @@ package validation import ( "flag" + "fmt" "time" + "github.com/grafana/loki/pkg/logql" "github.com/grafana/loki/pkg/util/flagext" + "github.com/prometheus/prometheus/pkg/labels" ) const ( @@ -60,6 +63,17 @@ type Limits struct { // Config for overrides, convenient if it goes here. PerTenantOverrideConfig string `yaml:"per_tenant_override_config"` PerTenantOverridePeriod time.Duration `yaml:"per_tenant_override_period"` + + // Global and per tenant retention + RetentionPeriod time.Duration `yaml:"retention_period"` + StreamRetention []StreamRetention `yaml:"retention_stream"` +} + +type StreamRetention struct { + Period time.Duration `yaml:"period"` + Priority int `yaml:"priority"` + Selector string `yaml:"selector"` + Matchers []*labels.Matcher `yaml:"-"` // populated during validation. } // RegisterFlags adds the flags required to config this to the given FlagSet @@ -96,6 +110,8 @@ func (l *Limits) RegisterFlags(f *flag.FlagSet) { f.StringVar(&l.PerTenantOverrideConfig, "limits.per-user-override-config", "", "File name of per-user overrides.") f.DurationVar(&l.PerTenantOverridePeriod, "limits.per-user-override-period", 10*time.Second, "Period with this to reload the overrides.") + + f.DurationVar(&l.RetentionPeriod, "store.retention", 31*24*time.Hour, "How long before chunks will be deleted from the store. (requires compactor retention enabled).") } // UnmarshalYAML implements the yaml.Unmarshaler interface. @@ -112,6 +128,30 @@ func (l *Limits) UnmarshalYAML(unmarshal func(interface{}) error) error { return unmarshal((*plain)(l)) } +// Validate validates that this limits config is valid. +func (l *Limits) Validate() error { + if l.StreamRetention != nil { + for i, rule := range l.StreamRetention { + matchers, err := logql.ParseMatchers(rule.Selector) + if err != nil { + return fmt.Errorf("invalid labels matchers: %w", err) + } + if rule.Period < 24*time.Hour { + return fmt.Errorf("retention period must be >= 24h was %s", rule.Period) + } + // populate matchers during validation + l.StreamRetention[i] = StreamRetention{ + Period: rule.Period, + Priority: rule.Priority, + Selector: rule.Selector, + Matchers: matchers, + } + + } + } + return nil +} + // When we load YAML from disk, we want the various per-customer limits // to default to any values specified on the command line, not default // command line values. This global contains those values. I (Tom) cannot @@ -293,6 +333,16 @@ func (o *Overrides) RulerMaxRuleGroupsPerTenant(userID string) int { return o.getOverridesForUser(userID).RulerMaxRuleGroupsPerTenant } +// RetentionPeriod returns the retention period for a given user. +func (o *Overrides) RetentionPeriod(userID string) time.Duration { + return o.getOverridesForUser(userID).RetentionPeriod +} + +// RetentionPeriod returns the retention period for a given user. +func (o *Overrides) StreamRetention(userID string) []StreamRetention { + return o.getOverridesForUser(userID).StreamRetention +} + func (o *Overrides) getOverridesForUser(userID string) *Limits { if o.tenantLimits != nil { l := o.tenantLimits(userID) From a93bb91f30b5c2a51d91e4279805c0101dcf2589 Mon Sep 17 00:00:00 2001 From: Cyril Tovena Date: Mon, 19 Apr 2021 20:25:48 +0200 Subject: [PATCH 15/42] Finishing off the marker processor. Signed-off-by: Cyril Tovena --- .../shipper/compactor/retention/marker.go | 237 +++++++++++++++++- .../compactor/retention/marker_test.go | 135 ++++++++++ .../shipper/compactor/retention/pool.go | 36 ++- .../shipper/compactor/retention/retention.go | 12 + 4 files changed, 410 insertions(+), 10 deletions(-) create mode 100644 pkg/storage/stores/shipper/compactor/retention/marker_test.go diff --git a/pkg/storage/stores/shipper/compactor/retention/marker.go b/pkg/storage/stores/shipper/compactor/retention/marker.go index c90da911642e0..af1523bef45cc 100644 --- a/pkg/storage/stores/shipper/compactor/retention/marker.go +++ b/pkg/storage/stores/shipper/compactor/retention/marker.go @@ -1,16 +1,27 @@ package retention import ( + "bytes" + "context" "fmt" + "io/fs" "os" "path/filepath" + "sort" + "strconv" + "sync" "time" chunk_util "github.com/cortexproject/cortex/pkg/chunk/util" - shipper_util "github.com/grafana/loki/pkg/storage/stores/shipper/util" + util_log "github.com/cortexproject/cortex/pkg/util/log" + "github.com/go-kit/kit/log/level" "go.etcd.io/bbolt" + + shipper_util "github.com/grafana/loki/pkg/storage/stores/shipper/util" ) +var minListMarkDelay = time.Minute + type MarkerStorageWriter interface { Put(chunkID []byte) error Count() int64 @@ -26,7 +37,7 @@ type markerStorageWriter struct { fileName string } -func NewMarkerStorageWriter(workingDir string) (*markerStorageWriter, error) { +func NewMarkerStorageWriter(workingDir string) (MarkerStorageWriter, error) { err := chunk_util.EnsureDirectory(filepath.Join(workingDir, markersFolder)) if err != nil { return nil, err @@ -73,8 +84,228 @@ func (m *markerStorageWriter) Close() error { return err } // The marker file is empty we can remove. - if m.count > 0 { + if m.count == 0 { return os.Remove(m.fileName) } return nil } + +type MarkerProcessor interface { + // Start starts parsing marks and calling deleteFunc for each. + // If deleteFunc returns no error the mark is deleted from the storage. + // Otherwise the mark will reappears in future iteration. + Start(deleteFunc func(ctx context.Context, chunkId []byte) error) + Stop() +} + +type markerProcessor struct { + folder string // folder where to find markers file. + maxParallelism int + minAgeFile time.Duration + + ctx context.Context + cancel context.CancelFunc + wg sync.WaitGroup +} + +func newMarkerStorageReader(workingDir string, maxParallelism int, minAgeFile time.Duration) (*markerProcessor, error) { + folder := filepath.Join(workingDir, markersFolder) + err := chunk_util.EnsureDirectory(folder) + if err != nil { + return nil, err + } + ctx, cancel := context.WithCancel(context.Background()) + return &markerProcessor{ + folder: folder, + ctx: ctx, + cancel: cancel, + maxParallelism: maxParallelism, + minAgeFile: minAgeFile, + }, nil +} + +func (r *markerProcessor) Start(deleteFunc func(ctx context.Context, chunkId []byte) error) { + r.wg.Wait() // only one start at a time. + r.wg.Add(1) + go func() { + defer r.wg.Done() + ticker := time.NewTicker(minListMarkDelay) + defer ticker.Stop() + tick := func() { + select { + case <-r.ctx.Done(): + case <-ticker.C: + } + } + // instant first tick + for ; true; tick() { + if r.ctx.Err() != nil { + // cancelled + return + } + paths, err := r.availablePath() + if err != nil { + level.Error(util_log.Logger).Log("msg", "failed to list marks path", "path", r.folder, "err", err) + continue + } + if len(paths) == 0 { + level.Info(util_log.Logger).Log("msg", "No marks file found") + } + for _, path := range paths { + if r.ctx.Err() != nil { + return + } + if err := r.processPath(path, deleteFunc); err != nil { + level.Warn(util_log.Logger).Log("msg", "failed to process marks", "path", path, "err", err) + continue + } + // delete if empty. + if err := r.deleteEmptyMarks(path); err != nil { + level.Warn(util_log.Logger).Log("msg", "failed to delete marks", "path", path, "err", err) + } + } + + } + }() +} + +func (r *markerProcessor) processPath(path string, deleteFunc func(ctx context.Context, chunkId []byte) error) error { + var ( + wg sync.WaitGroup + queue = make(chan *bytes.Buffer) + ) + db, err := shipper_util.SafeOpenBoltdbFile(path) + if err != nil { + return err + } + defer func() { + close(queue) + wg.Wait() + db.Close() + }() + for i := 0; i < r.maxParallelism; i++ { + wg.Add(1) + go func() { + defer wg.Done() + for key := range queue { + if err := processKey(r.ctx, key, db, deleteFunc); err != nil { + level.Warn(util_log.Logger).Log("msg", "failed to delete key", "key", key.String(), "err", err) + } + putKeyBuffer(key) + } + }() + } + if err := db.View(func(tx *bbolt.Tx) error { + b := tx.Bucket(chunkBucket) + if b == nil { + return nil + } + + c := b.Cursor() + for k, _ := c.First(); k != nil; k, _ = c.Next() { + key, err := getKeyBuffer(k) + if err != nil { + return err + } + select { + case queue <- key: + case <-r.ctx.Done(): + return r.ctx.Err() + } + + } + return nil + }); err != nil { + return err + } + return nil +} + +func processKey(ctx context.Context, key *bytes.Buffer, db *bbolt.DB, deleteFunc func(ctx context.Context, chunkId []byte) error) error { + keyData := key.Bytes() + if err := deleteFunc(ctx, keyData); err != nil { + return err + } + // no error we can delete the key + return db.Update(func(tx *bbolt.Tx) error { + b := tx.Bucket(chunkBucket) + if b == nil { + return nil + } + return b.Delete(keyData) + }) +} + +func (r *markerProcessor) deleteEmptyMarks(path string) error { + db, err := shipper_util.SafeOpenBoltdbFile(path) + if err != nil { + return err + } + var empty bool + err = db.View(func(tx *bbolt.Tx) error { + b := tx.Bucket(chunkBucket) + if b == nil { + empty = true + return nil + } + if k, _ := b.Cursor().First(); k == nil { + empty = true + return nil + } + + return nil + }) + db.Close() + if err != nil { + return err + } + if empty { + fmt.Fprintln(os.Stdout, path) + return os.Remove(path) + } + return nil +} + +// availablePath returns markers path in chronological order, skipping file that are not old enough. +func (r *markerProcessor) availablePath() ([]string, error) { + found := []int64{} + if err := filepath.WalkDir(r.folder, func(path string, d fs.DirEntry, err error) error { + if d == nil || err != nil { + return err + } + + if d.IsDir() && d.Name() != markersFolder { + return filepath.SkipDir + } + if d.IsDir() { + return nil + } + base := filepath.Base(path) + i, err := strconv.ParseInt(base, 10, 64) + if err != nil { + level.Warn(util_log.Logger).Log("msg", "wrong file name", "path", path, "base", base, "err", err) + return nil + } + + if time.Since(time.Unix(0, i)) > r.minAgeFile { + found = append(found, i) + } + return nil + }); err != nil { + return nil, err + } + if len(found) == 0 { + return nil, nil + } + sort.Slice(found, func(i, j int) bool { return found[i] < found[j] }) + res := make([]string, len(found)) + for i, f := range found { + res[i] = filepath.Join(r.folder, fmt.Sprintf("%d", f)) + } + return res, nil +} + +func (r *markerProcessor) Stop() { + r.cancel() + r.wg.Wait() +} diff --git a/pkg/storage/stores/shipper/compactor/retention/marker_test.go b/pkg/storage/stores/shipper/compactor/retention/marker_test.go new file mode 100644 index 0000000000000..4f850bd60cbdd --- /dev/null +++ b/pkg/storage/stores/shipper/compactor/retention/marker_test.go @@ -0,0 +1,135 @@ +package retention + +import ( + "context" + "errors" + "fmt" + "os" + "path/filepath" + "sort" + "sync" + "testing" + "time" + + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" +) + +func initAndFeedMarkerProcessor(t *testing.T) *markerProcessor { + t.Helper() + minListMarkDelay = time.Second + dir := t.TempDir() + p, err := newMarkerStorageReader(dir, 5, time.Second) + require.NoError(t, err) + defer p.Stop() + go func() { + w, err := NewMarkerStorageWriter(dir) + require.NoError(t, err) + + require.NoError(t, w.Put([]byte("1"))) + require.NoError(t, w.Put([]byte("2"))) + require.NoError(t, w.Close()) + w, err = NewMarkerStorageWriter(dir) + require.NoError(t, err) + require.NoError(t, w.Put([]byte("3"))) + require.NoError(t, w.Put([]byte("4"))) + require.NoError(t, w.Close()) + }() + return p +} + +func Test_markerProcessor_StartRetryKey(t *testing.T) { + p := initAndFeedMarkerProcessor(t) + counts := map[string]int{} + l := sync.Mutex{} + + p.Start(func(ctx context.Context, id []byte) error { + l.Lock() + defer l.Unlock() + counts[string(id)]++ + return errors.New("don't delete") + }) + + require.Eventually(t, func() bool { + l.Lock() + defer l.Unlock() + actual := []string{} + expected := []string{"1", "2", "3", "4"} + for k, v := range counts { + if v <= 1 { // we expects value to come back more than once since we don't delete them. + return false + } + actual = append(actual, k) + } + sort.Strings(actual) + return assert.ObjectsAreEqual(expected, actual) + }, 10*time.Second, 100*time.Microsecond) +} + +func Test_markerProcessor_StartDeleteOnSuccess(t *testing.T) { + p := initAndFeedMarkerProcessor(t) + counts := map[string]int{} + l := sync.Mutex{} + + p.Start(func(ctx context.Context, id []byte) error { + l.Lock() + defer l.Unlock() + counts[string(id)]++ + return nil + }) + + require.Eventually(t, func() bool { + l.Lock() + defer l.Unlock() + actual := []string{} + expected := []string{"1", "2", "3", "4"} + for k, v := range counts { + if v > 1 { // we should see keys only once ! + return false + } + actual = append(actual, k) + } + sort.Strings(actual) + return assert.ObjectsAreEqual(expected, actual) + }, 10*time.Second, 100*time.Microsecond) +} + +func Test_markerProcessor_availablePath(t *testing.T) { + now := time.Now() + for _, tt := range []struct { + name string + expectedPath func(dir string) []string + }{ + {"empty", func(_ string) []string { return nil }}, + {"skips bad files", func(dir string) []string { + _, _ = os.Create(filepath.Join(dir, "foo")) + return nil + }}, + {"happy path", func(dir string) []string { + _, _ = os.Create(filepath.Join(dir, fmt.Sprintf("%d", now.UnixNano()))) + _, _ = os.Create(filepath.Join(dir, "foo")) + _, _ = os.Create(filepath.Join(dir, fmt.Sprintf("%d", now.Add(-30*time.Minute).UnixNano()))) + _, _ = os.Create(filepath.Join(dir, fmt.Sprintf("%d", now.Add(-1*time.Hour).UnixNano()))) + _, _ = os.Create(filepath.Join(dir, fmt.Sprintf("%d", now.Add(-3*time.Hour).UnixNano()))) + _, _ = os.Create(filepath.Join(dir, fmt.Sprintf("%d", now.Add(-2*time.Hour).UnixNano()))) + _, _ = os.Create(filepath.Join(dir, fmt.Sprintf("%d", now.Add(-48*time.Hour).UnixNano()))) + return []string{ + filepath.Join(dir, fmt.Sprintf("%d", now.Add(-48*time.Hour).UnixNano())), // oldest should be first + filepath.Join(dir, fmt.Sprintf("%d", now.Add(-3*time.Hour).UnixNano())), + filepath.Join(dir, fmt.Sprintf("%d", now.Add(-2*time.Hour).UnixNano())), + } + }}, + } { + t.Run("", func(t *testing.T) { + dir := t.TempDir() + p, err := newMarkerStorageReader(dir, 5, 2*time.Hour) + + expected := tt.expectedPath(p.folder) + + require.NoError(t, err) + paths, err := p.availablePath() + require.Nil(t, err) + require.Equal(t, expected, paths) + }) + } +} diff --git a/pkg/storage/stores/shipper/compactor/retention/pool.go b/pkg/storage/stores/shipper/compactor/retention/pool.go index 446d0de98a89a..0dd00b5e43354 100644 --- a/pkg/storage/stores/shipper/compactor/retention/pool.go +++ b/pkg/storage/stores/shipper/compactor/retention/pool.go @@ -1,6 +1,7 @@ package retention import ( + "bytes" "sync" ) @@ -8,13 +9,20 @@ type componentRef struct { components [][]byte } -var componentPools = sync.Pool{ - New: func() interface{} { - return &componentRef{ - components: make([][]byte, 0, 5), - } - }, -} +var ( + componentPools = sync.Pool{ + New: func() interface{} { + return &componentRef{ + components: make([][]byte, 0, 5), + } + }, + } + keyPool = sync.Pool{ + New: func() interface{} { + return bytes.NewBuffer(make([]byte, 0, 512)) + }, + } +) func getComponents() *componentRef { ref := componentPools.Get().(*componentRef) @@ -25,3 +33,17 @@ func getComponents() *componentRef { func putComponents(ref *componentRef) { componentPools.Put(ref) } + +func getKeyBuffer(key []byte) (*bytes.Buffer, error) { + buf := keyPool.Get().(*bytes.Buffer) + if _, err := buf.Write(key); err != nil { + putKeyBuffer(buf) + return nil, err + } + return buf, nil +} + +func putKeyBuffer(buf *bytes.Buffer) { + buf.Reset() + keyPool.Put(buf) +} diff --git a/pkg/storage/stores/shipper/compactor/retention/retention.go b/pkg/storage/stores/shipper/compactor/retention/retention.go index a0efc0aaafcb9..ae24ddd01bfb3 100644 --- a/pkg/storage/stores/shipper/compactor/retention/retention.go +++ b/pkg/storage/stores/shipper/compactor/retention/retention.go @@ -193,3 +193,15 @@ func markforDelete(marker MarkerStorageWriter, chunkIt ChunkEntryIterator, serie return seriesCleaner.Cleanup(seriesID, userID) }) } + +type Sweeper struct { + workingDirectory string + objectClient chunk.ObjectClient +} + +func NewSweeper(workingDir string, objectClient chunk.ObjectClient) *Sweeper { + return &Sweeper{ + workingDirectory: workingDir, + objectClient: objectClient, + } +} From 87822c755510eb4033f5c7f1fb916040630dad8d Mon Sep 17 00:00:00 2001 From: Cyril Tovena Date: Tue, 20 Apr 2021 09:44:36 +0200 Subject: [PATCH 16/42] Working on sweeper and fixing tests. Signed-off-by: Cyril Tovena --- .../shipper/compactor/retention/marker.go | 4 +- .../compactor/retention/marker_test.go | 3 +- .../shipper/compactor/retention/retention.go | 42 ++++++++++++++----- 3 files changed, 36 insertions(+), 13 deletions(-) diff --git a/pkg/storage/stores/shipper/compactor/retention/marker.go b/pkg/storage/stores/shipper/compactor/retention/marker.go index af1523bef45cc..6bf680d3fb75d 100644 --- a/pkg/storage/stores/shipper/compactor/retention/marker.go +++ b/pkg/storage/stores/shipper/compactor/retention/marker.go @@ -95,6 +95,7 @@ type MarkerProcessor interface { // If deleteFunc returns no error the mark is deleted from the storage. // Otherwise the mark will reappears in future iteration. Start(deleteFunc func(ctx context.Context, chunkId []byte) error) + // Stop stops processing marks. Stop() } @@ -227,7 +228,7 @@ func processKey(ctx context.Context, key *bytes.Buffer, db *bbolt.DB, deleteFunc return err } // no error we can delete the key - return db.Update(func(tx *bbolt.Tx) error { + return db.Batch(func(tx *bbolt.Tx) error { b := tx.Bucket(chunkBucket) if b == nil { return nil @@ -260,7 +261,6 @@ func (r *markerProcessor) deleteEmptyMarks(path string) error { return err } if empty { - fmt.Fprintln(os.Stdout, path) return os.Remove(path) } return nil diff --git a/pkg/storage/stores/shipper/compactor/retention/marker_test.go b/pkg/storage/stores/shipper/compactor/retention/marker_test.go index 4f850bd60cbdd..4651d48d5981d 100644 --- a/pkg/storage/stores/shipper/compactor/retention/marker_test.go +++ b/pkg/storage/stores/shipper/compactor/retention/marker_test.go @@ -21,7 +21,6 @@ func initAndFeedMarkerProcessor(t *testing.T) *markerProcessor { dir := t.TempDir() p, err := newMarkerStorageReader(dir, 5, time.Second) require.NoError(t, err) - defer p.Stop() go func() { w, err := NewMarkerStorageWriter(dir) require.NoError(t, err) @@ -40,6 +39,7 @@ func initAndFeedMarkerProcessor(t *testing.T) *markerProcessor { func Test_markerProcessor_StartRetryKey(t *testing.T) { p := initAndFeedMarkerProcessor(t) + defer p.Stop() counts := map[string]int{} l := sync.Mutex{} @@ -68,6 +68,7 @@ func Test_markerProcessor_StartRetryKey(t *testing.T) { func Test_markerProcessor_StartDeleteOnSuccess(t *testing.T) { p := initAndFeedMarkerProcessor(t) + defer p.Stop() counts := map[string]int{} l := sync.Mutex{} diff --git a/pkg/storage/stores/shipper/compactor/retention/retention.go b/pkg/storage/stores/shipper/compactor/retention/retention.go index ae24ddd01bfb3..856ee5d5f7287 100644 --- a/pkg/storage/stores/shipper/compactor/retention/retention.go +++ b/pkg/storage/stores/shipper/compactor/retention/retention.go @@ -7,6 +7,7 @@ import ( "path" "path/filepath" "strings" + "time" "github.com/cortexproject/cortex/pkg/chunk" chunk_util "github.com/cortexproject/cortex/pkg/chunk/util" @@ -26,9 +27,10 @@ var ( ) const ( - logMetricName = "logs" - delimiter = "/" - markersFolder = "markers" + logMetricName = "logs" + delimiter = "/" + markersFolder = "markers" + deletionWorkerCount = 10 ) type Marker struct { @@ -131,7 +133,7 @@ func (t *Marker) MarkTableForDelete(ctx context.Context, tableName string) error if err := db.Close(); err != nil { return err } - // either delete if all entries are removed or upload the new file. + // if the index is empty we can delete the index table. if empty { return t.objectClient.DeleteObject(ctx, tableName+delimiter) } @@ -195,13 +197,33 @@ func markforDelete(marker MarkerStorageWriter, chunkIt ChunkEntryIterator, serie } type Sweeper struct { - workingDirectory string - objectClient chunk.ObjectClient + markerProcessor MarkerProcessor + objectClient chunk.ObjectClient } -func NewSweeper(workingDir string, objectClient chunk.ObjectClient) *Sweeper { - return &Sweeper{ - workingDirectory: workingDir, - objectClient: objectClient, +func NewSweeper(workingDir string, objectClient chunk.ObjectClient) (*Sweeper, error) { + p, err := newMarkerStorageReader(workingDir, deletionWorkerCount, 2*time.Hour) + if err != nil { + return nil, err } + return &Sweeper{ + markerProcessor: p, + objectClient: objectClient, + }, nil +} + +func (s *Sweeper) Start() { + s.markerProcessor.Start(func(ctx context.Context, chunkId []byte) error { + chunkIDString := unsafeGetString(chunkId) + err := s.objectClient.DeleteObject(ctx, chunkIDString) + if err == chunk.ErrStorageObjectNotFound { + level.Debug(util_log.Logger).Log("msg", "delete on not found chunk", "chunkID", chunkIDString) + return nil + } + return err + }) +} + +func (s *Sweeper) Stop() { + s.markerProcessor.Stop() } From 42dd29e219b61b5730464332c62ffc288f7bc0ae Mon Sep 17 00:00:00 2001 From: Cyril Tovena Date: Tue, 20 Apr 2021 21:35:23 +0200 Subject: [PATCH 17/42] Adding more tests and founding more bugs along the way. Signed-off-by: Cyril Tovena --- .../stores/shipper/compactor/compactor.go | 28 ++- .../compactor/retention/iterator_test.go | 2 - .../shipper/compactor/retention/retention.go | 41 ++++- .../compactor/retention/retention_test.go | 69 ++----- .../shipper/compactor/retention/util_test.go | 174 ++++++++++++------ 5 files changed, 190 insertions(+), 124 deletions(-) diff --git a/pkg/storage/stores/shipper/compactor/compactor.go b/pkg/storage/stores/shipper/compactor/compactor.go index 3a2a744d88404..7ed663e7276a3 100644 --- a/pkg/storage/stores/shipper/compactor/compactor.go +++ b/pkg/storage/stores/shipper/compactor/compactor.go @@ -34,6 +34,7 @@ type Config struct { CompactionInterval time.Duration `yaml:"compaction_interval"` RetentionEnabled bool `yaml:"retention_enabled"` RetentionInterval time.Duration `yaml:"retention_interval"` + RetentionDeleteDelay time.Duration `yaml:"retention_delete_delay"` } // RegisterFlags registers flags. @@ -42,7 +43,8 @@ func (cfg *Config) RegisterFlags(f *flag.FlagSet) { f.StringVar(&cfg.SharedStoreType, "boltdb.shipper.compactor.shared-store", "", "Shared store used for storing boltdb files. Supported types: gcs, s3, azure, swift, filesystem") f.StringVar(&cfg.SharedStoreKeyPrefix, "boltdb.shipper.compactor.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.DurationVar(&cfg.CompactionInterval, "boltdb.shipper.compactor.compaction-interval", 2*time.Hour, "Interval at which to re-run the compaction operation.") - f.DurationVar(&cfg.RetentionInterval, "boltdb.shipper.compactor.retention-interval", 2*time.Hour, "Interval at which to re-run the retention operation.") + f.DurationVar(&cfg.RetentionInterval, "boltdb.shipper.compactor.retention-interval", 10*time.Minute, "Interval at which to re-run the retention operation.") + f.DurationVar(&cfg.RetentionDeleteDelay, "boltdb.shipper.compactor.retention-delete-delay", 2*time.Hour, "Delay after which chunks will be fully deleted during retention.") f.BoolVar(&cfg.RetentionEnabled, "boltdb.shipper.compactor.retention-enabled", false, "Activate custom (per-stream,per-tenant) retention.") } @@ -62,6 +64,7 @@ type Compactor struct { cfg Config objectClient chunk.ObjectClient tableMarker *retention.Marker + sweeper *retention.Sweeper metrics *metrics } @@ -82,12 +85,18 @@ func NewCompactor(cfg Config, storageConfig storage.Config, schemaConfig loki_st } prefixedClient := util.NewPrefixedObjectClient(objectClient, cfg.SharedStoreKeyPrefix) - // todo configuration and expiration checker. + retentionWorkDir := filepath.Join(cfg.WorkingDirectory, "retention") + + sweeper, err := retention.NewSweeper(retentionWorkDir, retention.NewDeleteClient(objectClient), cfg.RetentionDeleteDelay) + if err != nil { + return nil, err + } compactor := Compactor{ cfg: cfg, objectClient: prefixedClient, metrics: newMetrics(r), - tableMarker: retention.NewMarker(filepath.Join(cfg.WorkingDirectory, "retention"), schemaConfig, prefixedClient, retention.NewExpirationChecker(limits)), + tableMarker: retention.NewMarker(retentionWorkDir, schemaConfig, prefixedClient, retention.NewExpirationChecker(limits)), + sweeper: sweeper, } compactor.Service = services.NewBasicService(nil, compactor.loop, nil) @@ -126,10 +135,19 @@ func (c *Compactor) loop(ctx context.Context) error { } }() if c.cfg.RetentionEnabled { - wg.Add(1) + wg.Add(2) go func() { + // starts the chunk sweeper + defer func() { + c.sweeper.Stop() + wg.Done() + }() + c.sweeper.Start() + <-ctx.Done() + }() + go func() { + // start the index marker defer wg.Done() - ticker := time.NewTicker(c.cfg.RetentionInterval) defer ticker.Stop() diff --git a/pkg/storage/stores/shipper/compactor/retention/iterator_test.go b/pkg/storage/stores/shipper/compactor/retention/iterator_test.go index ed0a86fc2ed0a..4227a933d894d 100644 --- a/pkg/storage/stores/shipper/compactor/retention/iterator_test.go +++ b/pkg/storage/stores/shipper/compactor/retention/iterator_test.go @@ -19,7 +19,6 @@ func Test_ChunkIterator(t *testing.T) { tt := tt t.Run(tt.schema, func(t *testing.T) { store := newTestStore(t) - defer store.cleanup() c1 := createChunk(t, "1", labels.Labels{labels.Label{Name: "foo", Value: "bar"}}, tt.from, tt.from.Add(1*time.Hour)) c2 := createChunk(t, "2", labels.Labels{labels.Label{Name: "foo", Value: "buzz"}, labels.Label{Name: "bar", Value: "foo"}}, tt.from, tt.from.Add(1*time.Hour)) @@ -74,7 +73,6 @@ func Test_SeriesCleaner(t *testing.T) { tt := tt t.Run(tt.schema, func(t *testing.T) { store := newTestStore(t) - defer store.cleanup() c1 := createChunk(t, "1", labels.Labels{labels.Label{Name: "foo", Value: "bar"}}, tt.from, tt.from.Add(1*time.Hour)) c2 := createChunk(t, "2", labels.Labels{labels.Label{Name: "foo", Value: "buzz"}, labels.Label{Name: "bar", Value: "foo"}}, tt.from, tt.from.Add(1*time.Hour)) c3 := createChunk(t, "2", labels.Labels{labels.Label{Name: "foo", Value: "buzz"}, labels.Label{Name: "bar", Value: "buzz"}}, tt.from, tt.from.Add(1*time.Hour)) diff --git a/pkg/storage/stores/shipper/compactor/retention/retention.go b/pkg/storage/stores/shipper/compactor/retention/retention.go index 856ee5d5f7287..8308036f98100 100644 --- a/pkg/storage/stores/shipper/compactor/retention/retention.go +++ b/pkg/storage/stores/shipper/compactor/retention/retention.go @@ -2,6 +2,7 @@ package retention import ( "context" + "encoding/base64" "fmt" "os" "path" @@ -10,6 +11,7 @@ import ( "time" "github.com/cortexproject/cortex/pkg/chunk" + "github.com/cortexproject/cortex/pkg/chunk/local" chunk_util "github.com/cortexproject/cortex/pkg/chunk/util" util_log "github.com/cortexproject/cortex/pkg/util/log" "github.com/go-kit/kit/log/level" @@ -45,6 +47,7 @@ func NewMarker(workingDirectory string, config storage.SchemaConfig, objectClien workingDirectory: workingDirectory, config: config, objectClient: objectClient, + expiration: expiration, } } @@ -77,7 +80,7 @@ func (t *Marker) MarkTableForDelete(ctx context.Context, tableName string) error return err } - downloadAt := filepath.Join(t.workingDirectory, tableName) + downloadAt := filepath.Join(tableDirectory, tableKey) err = shipper_util.GetFileFromStorage(ctx, t.objectClient, tableKey, downloadAt) if err != nil { @@ -90,6 +93,9 @@ func (t *Marker) MarkTableForDelete(ctx context.Context, tableName string) error } defer func() { + if err := db.Close(); err != nil { + level.Warn(util_log.Logger).Log("msg", "failed to close local db", "err", err) + } if err := os.Remove(db.Path()); err != nil { level.Warn(util_log.Logger).Log("msg", "failed to removed downloaded db", "err", err) } @@ -130,9 +136,6 @@ func (t *Marker) MarkTableForDelete(ctx context.Context, tableName string) error if err != nil { return err } - if err := db.Close(); err != nil { - return err - } // if the index is empty we can delete the index table. if empty { return t.objectClient.DeleteObject(ctx, tableName+delimiter) @@ -196,26 +199,46 @@ func markforDelete(marker MarkerStorageWriter, chunkIt ChunkEntryIterator, serie }) } +type DeleteClient interface { + DeleteObject(ctx context.Context, objectKey string) error +} + +type DeleteClientFunc func(ctx context.Context, objectKey string) error + +func (d DeleteClientFunc) DeleteObject(ctx context.Context, objectKey string) error { + return d(ctx, objectKey) +} + +func NewDeleteClient(objectClient chunk.ObjectClient) DeleteClient { + // filesystem encode64 keys on disk. useful for testing. + if fs, ok := objectClient.(*local.FSObjectClient); ok { + return DeleteClientFunc(func(ctx context.Context, objectKey string) error { + return fs.DeleteObject(ctx, base64.StdEncoding.EncodeToString([]byte(objectKey))) + }) + } + return objectClient +} + type Sweeper struct { markerProcessor MarkerProcessor - objectClient chunk.ObjectClient + deleteClient DeleteClient } -func NewSweeper(workingDir string, objectClient chunk.ObjectClient) (*Sweeper, error) { - p, err := newMarkerStorageReader(workingDir, deletionWorkerCount, 2*time.Hour) +func NewSweeper(workingDir string, deleteClient DeleteClient, minAgeDelete time.Duration) (*Sweeper, error) { + p, err := newMarkerStorageReader(workingDir, deletionWorkerCount, minAgeDelete) if err != nil { return nil, err } return &Sweeper{ markerProcessor: p, - objectClient: objectClient, + deleteClient: deleteClient, }, nil } func (s *Sweeper) Start() { s.markerProcessor.Start(func(ctx context.Context, chunkId []byte) error { chunkIDString := unsafeGetString(chunkId) - err := s.objectClient.DeleteObject(ctx, chunkIDString) + err := s.deleteClient.DeleteObject(ctx, chunkIDString) if err == chunk.ErrStorageObjectNotFound { level.Debug(util_log.Logger).Log("msg", "delete on not found chunk", "chunkID", chunkIDString) return nil diff --git a/pkg/storage/stores/shipper/compactor/retention/retention_test.go b/pkg/storage/stores/shipper/compactor/retention/retention_test.go index 1afdba4083c8b..3aef09cfb1f0d 100644 --- a/pkg/storage/stores/shipper/compactor/retention/retention_test.go +++ b/pkg/storage/stores/shipper/compactor/retention/retention_test.go @@ -4,8 +4,7 @@ import ( "context" "crypto/sha256" "encoding/base64" - "fmt" - "os" + "path/filepath" "strconv" "strings" "testing" @@ -16,67 +15,39 @@ import ( "github.com/prometheus/common/model" "github.com/prometheus/prometheus/pkg/labels" "github.com/stretchr/testify/require" - "go.etcd.io/bbolt" "github.com/grafana/loki/pkg/chunkenc" "github.com/grafana/loki/pkg/logproto" + "github.com/grafana/loki/pkg/storage/stores/util" + "github.com/grafana/loki/pkg/util/validation" ) func Test_Retention(t *testing.T) { store := newTestStore(t) - defer store.cleanup() - require.NoError(t, store.Put(context.TODO(), []chunk.Chunk{ - createChunk(t, "1", labels.Labels{labels.Label{Name: "foo", Value: "bar"}}, model.Earliest, model.Earliest.Add(1*time.Hour)), - createChunk(t, "2", labels.Labels{labels.Label{Name: "foo", Value: "buzz"}}, model.Earliest.Add(26*time.Hour), model.Earliest.Add(27*time.Hour)), - })) + c1 := createChunk(t, "1", labels.Labels{labels.Label{Name: "foo", Value: "bar"}}, start, start.Add(1*time.Hour)) + c2 := createChunk(t, "2", labels.Labels{labels.Label{Name: "foo", Value: "buzz"}}, start.Add(26*time.Hour), start.Add(27*time.Hour)) + require.NoError(t, store.Put(context.TODO(), []chunk.Chunk{c1, c2})) store.Stop() - // retentionRules := fakeRule{} + expiration := NewExpirationChecker(fakeLimits{ + perTenant: map[string]time.Duration{ + "1": 1000 * time.Hour, + "2": 1000 * time.Hour, + }, + perStream: map[string][]validation.StreamRetention{}, + }) + workDir := filepath.Join(t.TempDir(), "retention") + marker := NewMarker(workDir, store.schemaCfg, util.NewPrefixedObjectClient(store.objectClient, "index/"), expiration) - // 1- Get all series ID for given retention per stream.... - // 2 - Delete from index and Mark for delete all chunk based on retention with seriesID/tenantID. - // 3 - Seek chunk entries via series id for each series and verify if we still have chunk. - // 4 - Delete Label entries for empty series with empty chunk entries. - - // For 1. only equality matcher are OK so we can use GetReadMetricLabelValueQueries for _, table := range store.indexTables() { - fmt.Fprintf(os.Stdout, "Opening Table %s\n", table.name) - - // 1 - Get all series ID for given retention per stream.... - - // 1.1 find the schema for this table - - currentSchema, ok := schemaPeriodForTable(store.schemaCfg, table.name) - if !ok { - fmt.Fprintf(os.Stdout, "Could not find Schema for Table %s\n", table.name) - continue - } - fmt.Fprintf(os.Stdout, "Found Schema for Table %s => %+v\n", table.name, currentSchema) - - _ = NewExpirationChecker(nil) - - require.NoError(t, - table.DB.Update(func(tx *bbolt.Tx) error { - return tx.Bucket(bucketName).ForEach(func(k, v []byte) error { - ref, ok, err := parseChunkRef(decodeKey(k)) - if err != nil { - return err - } - if ok { - fmt.Fprintf(os.Stdout, "%+v\n", ref) - return nil - } - _, r := decodeKey(k) - components := decodeRangeKey(r, nil) - keyType := components[len(components)-1] - - fmt.Fprintf(os.Stdout, "type:%s \n", keyType) - return nil - }) - })) + require.NoError(t, marker.MarkTableForDelete(context.Background(), table.name)) } + + store.open() + store.requiresHasChunk(c1) + store.requiresHasChunk(c2) } func createChunk(t testing.TB, userID string, lbs labels.Labels, from model.Time, through model.Time) chunk.Chunk { diff --git a/pkg/storage/stores/shipper/compactor/retention/util_test.go b/pkg/storage/stores/shipper/compactor/retention/util_test.go index 06e9594801228..a13f0dbcac106 100644 --- a/pkg/storage/stores/shipper/compactor/retention/util_test.go +++ b/pkg/storage/stores/shipper/compactor/retention/util_test.go @@ -1,8 +1,8 @@ package retention import ( + "context" "io/ioutil" - "os" "path/filepath" "testing" "time" @@ -10,66 +10,72 @@ import ( "github.com/cortexproject/cortex/pkg/chunk" "github.com/cortexproject/cortex/pkg/chunk/local" cortex_storage "github.com/cortexproject/cortex/pkg/chunk/storage" + chunk_util "github.com/cortexproject/cortex/pkg/chunk/util" util_log "github.com/cortexproject/cortex/pkg/util/log" "github.com/grafana/loki/pkg/logql" "github.com/grafana/loki/pkg/storage" + "github.com/grafana/loki/pkg/storage/stores/shipper" shipper_util "github.com/grafana/loki/pkg/storage/stores/shipper/util" "github.com/grafana/loki/pkg/util/validation" "github.com/prometheus/common/model" + "github.com/prometheus/prometheus/pkg/labels" "github.com/stretchr/testify/require" + "github.com/weaveworks/common/user" "go.etcd.io/bbolt" ) -var schemaCfg = storage.SchemaConfig{ - SchemaConfig: chunk.SchemaConfig{ - // we want to test over all supported schema. - Configs: []chunk.PeriodConfig{ - { - From: chunk.DayTime{Time: model.Earliest}, - IndexType: "boltdb", - ObjectType: "filesystem", - Schema: "v9", - IndexTables: chunk.PeriodicTableConfig{ - Prefix: "index_", - Period: time.Hour * 24, +var ( + schemaCfg = storage.SchemaConfig{ + SchemaConfig: chunk.SchemaConfig{ + // we want to test over all supported schema. + Configs: []chunk.PeriodConfig{ + { + From: chunk.DayTime{Time: start}, + IndexType: "boltdb", + ObjectType: "filesystem", + Schema: "v9", + IndexTables: chunk.PeriodicTableConfig{ + Prefix: "index_", + Period: time.Hour * 24, + }, + RowShards: 16, }, - RowShards: 16, - }, - { - From: chunk.DayTime{Time: model.Earliest.Add(25 * time.Hour)}, - IndexType: "boltdb", - ObjectType: "filesystem", - Schema: "v10", - IndexTables: chunk.PeriodicTableConfig{ - Prefix: "index_", - Period: time.Hour * 24, + { + From: chunk.DayTime{Time: start.Add(25 * time.Hour)}, + IndexType: "boltdb", + ObjectType: "filesystem", + Schema: "v10", + IndexTables: chunk.PeriodicTableConfig{ + Prefix: "index_", + Period: time.Hour * 24, + }, + RowShards: 16, }, - RowShards: 16, - }, - { - From: chunk.DayTime{Time: model.Earliest.Add(49 * time.Hour)}, - IndexType: "boltdb", - ObjectType: "filesystem", - Schema: "v11", - IndexTables: chunk.PeriodicTableConfig{ - Prefix: "index_", - Period: time.Hour * 24, + { + From: chunk.DayTime{Time: start.Add(49 * time.Hour)}, + IndexType: "boltdb", + ObjectType: "filesystem", + Schema: "v11", + IndexTables: chunk.PeriodicTableConfig{ + Prefix: "index_", + Period: time.Hour * 24, + }, + RowShards: 16, }, - RowShards: 16, }, }, - }, -} - -var allSchemas = []struct { - schema string - from model.Time - config chunk.PeriodConfig -}{ - {"v9", model.Earliest, schemaCfg.Configs[0]}, - {"v10", model.Earliest.Add(25 * time.Hour), schemaCfg.Configs[1]}, - {"v11", model.Earliest.Add(49 * time.Hour), schemaCfg.Configs[2]}, -} + } + allSchemas = []struct { + schema string + from model.Time + config chunk.PeriodConfig + }{ + {"v9", start, schemaCfg.Configs[0]}, + {"v10", start.Add(25 * time.Hour), schemaCfg.Configs[1]}, + {"v11", start.Add(49 * time.Hour), schemaCfg.Configs[2]}, + } + start = model.Now().Add(-30 * 24 * time.Hour) +) func newChunkEntry(userID, labels string, from, through model.Time) ChunkEntry { lbs, err := logql.ParseLabels(labels) @@ -89,15 +95,12 @@ func newChunkEntry(userID, labels string, from, through model.Time) ChunkEntry { type testStore struct { storage.Store + cfg storage.Config + objectClient chunk.ObjectClient indexDir, chunkDir string schemaCfg storage.SchemaConfig t *testing.T -} - -func (t *testStore) cleanup() { - t.t.Helper() - require.NoError(t.t, os.RemoveAll(t.indexDir)) - require.NoError(t.t, os.RemoveAll(t.indexDir)) + limits cortex_storage.StoreLimits } type table struct { @@ -118,12 +121,47 @@ func (t *testStore) indexTables() []table { return res } +func (t *testStore) requiresHasChunk(c chunk.Chunk) { + t.t.Helper() + var matchers []*labels.Matcher + for _, l := range c.Metric { + matchers = append(matchers, labels.MustNewMatcher(labels.MatchEqual, l.Name, l.Value)) + } + chunks, err := t.Store.Get(user.InjectOrgID(context.Background(), c.UserID), + c.UserID, c.From, c.Through, matchers...) + require.NoError(t.t, err) + require.Len(t.t, chunks, 1) + require.Equal(t.t, c.ExternalKey(), chunks[0].ExternalKey()) +} + +func (t *testStore) open() { + chunkStore, err := cortex_storage.NewStore( + t.cfg.Config, + chunk.StoreConfig{}, + schemaCfg.SchemaConfig, + t.limits, + nil, + nil, + util_log.Logger, + ) + require.NoError(t.t, err) + + store, err := storage.NewStore(t.cfg, schemaCfg, chunkStore, nil) + require.NoError(t.t, err) + t.Store = store +} + func newTestStore(t *testing.T) *testStore { t.Helper() - indexDir, err := ioutil.TempDir("", "boltdb_test") + workdir := t.TempDir() + filepath.Join(workdir, "index") + indexDir := filepath.Join(workdir, "index") + err := chunk_util.EnsureDirectory(indexDir) require.Nil(t, err) - chunkDir, err := ioutil.TempDir("", "chunk_test") + chunkDir := filepath.Join(workdir, "chunk_test") + err = chunk_util.EnsureDirectory(indexDir) + require.Nil(t, err) require.Nil(t, err) defer func() { @@ -142,7 +180,22 @@ func newTestStore(t *testing.T) *testStore { Directory: chunkDir, }, }, + BoltDBShipperConfig: shipper.Config{ + ActiveIndexDirectory: indexDir, + SharedStoreType: "filesystem", + SharedStoreKeyPrefix: "index", + ResyncInterval: 1 * time.Millisecond, + IngesterName: "foo", + Mode: shipper.ModeReadWrite, + }, } + objectClient, err := cortex_storage.NewObjectClient("filesystem", cortex_storage.Config{ + FSConfig: local.FSConfig{ + Directory: workdir, + }, + }) + require.NoError(t, err) + chunkStore, err := cortex_storage.NewStore( config.Config, chunk.StoreConfig{}, @@ -157,10 +210,13 @@ func newTestStore(t *testing.T) *testStore { store, err := storage.NewStore(config, schemaCfg, chunkStore, nil) require.NoError(t, err) return &testStore{ - indexDir: indexDir, - chunkDir: chunkDir, - t: t, - Store: store, - schemaCfg: schemaCfg, + indexDir: indexDir, + chunkDir: chunkDir, + t: t, + Store: store, + schemaCfg: schemaCfg, + objectClient: objectClient, + cfg: config, + limits: limits, } } From 8468cfb1363d0cdc2284932cd48fdc58894b3a6e Mon Sep 17 00:00:00 2001 From: Cyril Tovena Date: Tue, 20 Apr 2021 21:48:56 +0200 Subject: [PATCH 18/42] Bug with path once boltdb is closed. Signed-off-by: Cyril Tovena --- .../shipper/compactor/retention/retention.go | 6 ++++-- .../shipper/compactor/retention/util_test.go | 15 ++++++++++----- 2 files changed, 14 insertions(+), 7 deletions(-) diff --git a/pkg/storage/stores/shipper/compactor/retention/retention.go b/pkg/storage/stores/shipper/compactor/retention/retention.go index 8308036f98100..3d7ff665e0583 100644 --- a/pkg/storage/stores/shipper/compactor/retention/retention.go +++ b/pkg/storage/stores/shipper/compactor/retention/retention.go @@ -93,11 +93,13 @@ func (t *Marker) MarkTableForDelete(ctx context.Context, tableName string) error } defer func() { + path := db.Path() if err := db.Close(); err != nil { level.Warn(util_log.Logger).Log("msg", "failed to close local db", "err", err) } - if err := os.Remove(db.Path()); err != nil { - level.Warn(util_log.Logger).Log("msg", "failed to removed downloaded db", "err", err) + + if err := os.Remove(path); err != nil { + level.Warn(util_log.Logger).Log("msg", "failed to removed downloaded db", "err", err, "path", path) } }() diff --git a/pkg/storage/stores/shipper/compactor/retention/util_test.go b/pkg/storage/stores/shipper/compactor/retention/util_test.go index a13f0dbcac106..4ac91bcb93e56 100644 --- a/pkg/storage/stores/shipper/compactor/retention/util_test.go +++ b/pkg/storage/stores/shipper/compactor/retention/util_test.go @@ -12,16 +12,18 @@ import ( cortex_storage "github.com/cortexproject/cortex/pkg/chunk/storage" chunk_util "github.com/cortexproject/cortex/pkg/chunk/util" util_log "github.com/cortexproject/cortex/pkg/util/log" - "github.com/grafana/loki/pkg/logql" - "github.com/grafana/loki/pkg/storage" - "github.com/grafana/loki/pkg/storage/stores/shipper" - shipper_util "github.com/grafana/loki/pkg/storage/stores/shipper/util" - "github.com/grafana/loki/pkg/util/validation" "github.com/prometheus/common/model" "github.com/prometheus/prometheus/pkg/labels" "github.com/stretchr/testify/require" + ww "github.com/weaveworks/common/server" "github.com/weaveworks/common/user" "go.etcd.io/bbolt" + + "github.com/grafana/loki/pkg/logql" + "github.com/grafana/loki/pkg/storage" + "github.com/grafana/loki/pkg/storage/stores/shipper" + shipper_util "github.com/grafana/loki/pkg/storage/stores/shipper/util" + "github.com/grafana/loki/pkg/util/validation" ) var ( @@ -153,6 +155,9 @@ func (t *testStore) open() { func newTestStore(t *testing.T) *testStore { t.Helper() + cfg := &ww.Config{} + require.Nil(t, cfg.LogLevel.Set("debug")) + util_log.InitLogger(cfg) workdir := t.TempDir() filepath.Join(workdir, "index") indexDir := filepath.Join(workdir, "index") From b560196ad120aa05302d4c361d37445658c41c76 Mon Sep 17 00:00:00 2001 From: Cyril Tovena Date: Tue, 20 Apr 2021 22:34:07 +0200 Subject: [PATCH 19/42] Fixing more bug and more robust test. Signed-off-by: Cyril Tovena --- .../shipper/compactor/retention/expiration.go | 4 +- .../compactor/retention/retention_test.go | 89 ++++++++++++++----- .../shipper/compactor/retention/util_test.go | 5 +- 3 files changed, 70 insertions(+), 28 deletions(-) diff --git a/pkg/storage/stores/shipper/compactor/retention/expiration.go b/pkg/storage/stores/shipper/compactor/retention/expiration.go index 394ff5398c27b..cbf11ce12fdee 100644 --- a/pkg/storage/stores/shipper/compactor/retention/expiration.go +++ b/pkg/storage/stores/shipper/compactor/retention/expiration.go @@ -57,7 +57,7 @@ Outer: matchedRule = streamRetention } if found { - return ref.From.After(model.Now().Add(matchedRule.Period)) + return model.Now().Sub(ref.Through) > matchedRule.Period } - return ref.From.After(model.Now().Add(globalRetention)) + return model.Now().Sub(ref.Through) > globalRetention } diff --git a/pkg/storage/stores/shipper/compactor/retention/retention_test.go b/pkg/storage/stores/shipper/compactor/retention/retention_test.go index 3aef09cfb1f0d..38878ae04a3b0 100644 --- a/pkg/storage/stores/shipper/compactor/retention/retention_test.go +++ b/pkg/storage/stores/shipper/compactor/retention/retention_test.go @@ -23,31 +23,74 @@ import ( ) func Test_Retention(t *testing.T) { - store := newTestStore(t) - - c1 := createChunk(t, "1", labels.Labels{labels.Label{Name: "foo", Value: "bar"}}, start, start.Add(1*time.Hour)) - c2 := createChunk(t, "2", labels.Labels{labels.Label{Name: "foo", Value: "buzz"}}, start.Add(26*time.Hour), start.Add(27*time.Hour)) - require.NoError(t, store.Put(context.TODO(), []chunk.Chunk{c1, c2})) - - store.Stop() - - expiration := NewExpirationChecker(fakeLimits{ - perTenant: map[string]time.Duration{ - "1": 1000 * time.Hour, - "2": 1000 * time.Hour, + for _, tt := range []struct { + name string + limits Limits + chunks []chunk.Chunk + alive []bool + }{ + { + "nothing is expiring", + fakeLimits{ + perTenant: map[string]time.Duration{ + "1": 1000 * time.Hour, + "2": 1000 * time.Hour, + }, + perStream: map[string][]validation.StreamRetention{}, + }, + []chunk.Chunk{ + createChunk(t, "1", labels.Labels{labels.Label{Name: "foo", Value: "bar"}}, start, start.Add(1*time.Hour)), + createChunk(t, "2", labels.Labels{labels.Label{Name: "foo", Value: "buzz"}}, start.Add(26*time.Hour), start.Add(27*time.Hour)), + }, + []bool{ + true, + true, + }, }, - perStream: map[string][]validation.StreamRetention{}, - }) - workDir := filepath.Join(t.TempDir(), "retention") - marker := NewMarker(workDir, store.schemaCfg, util.NewPrefixedObjectClient(store.objectClient, "index/"), expiration) - - for _, table := range store.indexTables() { - require.NoError(t, marker.MarkTableForDelete(context.Background(), table.name)) + { + "one global expiration", + fakeLimits{ + perTenant: map[string]time.Duration{ + "1": 10 * time.Hour, + "2": 1000 * time.Hour, + }, + perStream: map[string][]validation.StreamRetention{}, + }, + []chunk.Chunk{ + createChunk(t, "1", labels.Labels{labels.Label{Name: "foo", Value: "bar"}}, start, start.Add(1*time.Hour)), + createChunk(t, "2", labels.Labels{labels.Label{Name: "foo", Value: "buzz"}}, start.Add(26*time.Hour), start.Add(27*time.Hour)), + }, + []bool{ + false, + true, + }, + }, + } { + tt := tt + t.Run(tt.name, func(t *testing.T) { + // insert in the store. + store := newTestStore(t) + for _, c := range tt.chunks { + require.NoError(t, store.Put(context.TODO(), []chunk.Chunk{c})) + } + store.Stop() + + // marks + expiration := NewExpirationChecker(tt.limits) + workDir := filepath.Join(t.TempDir(), "retention") + marker := NewMarker(workDir, store.schemaCfg, util.NewPrefixedObjectClient(store.objectClient, "index/"), expiration) + for _, table := range store.indexTables() { + require.NoError(t, marker.MarkTableForDelete(context.Background(), table.name)) + } + + // assert using the store again. + store.open() + for i, e := range tt.alive { + require.Equal(t, e, store.HasChunk(tt.chunks[i])) + } + store.Stop() + }) } - - store.open() - store.requiresHasChunk(c1) - store.requiresHasChunk(c2) } func createChunk(t testing.TB, userID string, lbs labels.Labels, from model.Time, through model.Time) chunk.Chunk { diff --git a/pkg/storage/stores/shipper/compactor/retention/util_test.go b/pkg/storage/stores/shipper/compactor/retention/util_test.go index 4ac91bcb93e56..9d49fcdff1f08 100644 --- a/pkg/storage/stores/shipper/compactor/retention/util_test.go +++ b/pkg/storage/stores/shipper/compactor/retention/util_test.go @@ -123,7 +123,7 @@ func (t *testStore) indexTables() []table { return res } -func (t *testStore) requiresHasChunk(c chunk.Chunk) { +func (t *testStore) HasChunk(c chunk.Chunk) bool { t.t.Helper() var matchers []*labels.Matcher for _, l := range c.Metric { @@ -132,8 +132,7 @@ func (t *testStore) requiresHasChunk(c chunk.Chunk) { chunks, err := t.Store.Get(user.InjectOrgID(context.Background(), c.UserID), c.UserID, c.From, c.Through, matchers...) require.NoError(t.t, err) - require.Len(t.t, chunks, 1) - require.Equal(t.t, c.ExternalKey(), chunks[0].ExternalKey()) + return len(chunks) == 1 && c.ExternalKey() == chunks[0].ExternalKey() } func (t *testStore) open() { From 8f85309c6e8bfdb0fc47d368c9047a2089be0d00 Mon Sep 17 00:00:00 2001 From: Cyril Tovena Date: Wed, 21 Apr 2021 15:22:37 +0200 Subject: [PATCH 20/42] More test and cleanup getting close. Signed-off-by: Cyril Tovena --- .../shipper/compactor/retention/expiration.go | 3 +- .../compactor/retention/expiration_test.go | 15 +++++----- .../shipper/compactor/retention/index.go | 3 -- .../compactor/retention/iterator_test.go | 3 +- .../compactor/retention/retention_test.go | 30 +++++++++++++++++-- .../shipper/compactor/retention/util.go | 20 ------------- 6 files changed, 40 insertions(+), 34 deletions(-) diff --git a/pkg/storage/stores/shipper/compactor/retention/expiration.go b/pkg/storage/stores/shipper/compactor/retention/expiration.go index cbf11ce12fdee..6850faf0e1faa 100644 --- a/pkg/storage/stores/shipper/compactor/retention/expiration.go +++ b/pkg/storage/stores/shipper/compactor/retention/expiration.go @@ -3,8 +3,9 @@ package retention import ( "time" - "github.com/grafana/loki/pkg/util/validation" "github.com/prometheus/common/model" + + "github.com/grafana/loki/pkg/util/validation" ) type ExpirationChecker interface { diff --git a/pkg/storage/stores/shipper/compactor/retention/expiration_test.go b/pkg/storage/stores/shipper/compactor/retention/expiration_test.go index b98e7d863518a..66fb95297915d 100644 --- a/pkg/storage/stores/shipper/compactor/retention/expiration_test.go +++ b/pkg/storage/stores/shipper/compactor/retention/expiration_test.go @@ -4,10 +4,11 @@ import ( "testing" "time" - "github.com/grafana/loki/pkg/util/validation" "github.com/prometheus/common/model" "github.com/prometheus/prometheus/pkg/labels" "github.com/stretchr/testify/require" + + "github.com/grafana/loki/pkg/util/validation" ) type fakeLimits struct { @@ -45,12 +46,12 @@ func Test_expirationChecker_Expired(t *testing.T) { ref ChunkEntry want bool }{ - {"expired tenant", newChunkEntry("1", `{foo="buzz"}`, model.Now().Add(2*time.Hour), model.Now().Add(3*time.Hour)), true}, - {"just expired tenant", newChunkEntry("1", `{foo="buzz"}`, model.Now().Add(1*time.Hour), model.Now().Add(3*time.Hour)), false}, - {"not expired tenant", newChunkEntry("1", `{foo="buzz"}`, model.Now().Add(30*time.Minute), model.Now().Add(3*time.Hour)), false}, - {"not expired tenant by far", newChunkEntry("2", `{foo="buzz"}`, model.Now().Add(30*time.Minute), model.Now().Add(3*time.Hour)), false}, - {"expired stream override", newChunkEntry("2", `{foo="bar"}`, model.Now().Add(3*time.Hour), model.Now().Add(4*time.Hour)), true}, - {"non expired stream override", newChunkEntry("1", `{foo="bar"}`, model.Now().Add(1*time.Hour), model.Now().Add(4*time.Hour)), false}, + {"expired tenant", newChunkEntry("1", `{foo="buzz"}`, model.Now().Add(-3*time.Hour), model.Now().Add(-2*time.Hour)), true}, + {"just expired tenant", newChunkEntry("1", `{foo="buzz"}`, model.Now().Add(-3*time.Hour), model.Now().Add(-1*time.Hour+(10*time.Microsecond))), false}, + {"not expired tenant", newChunkEntry("1", `{foo="buzz"}`, model.Now().Add(-3*time.Hour), model.Now().Add(-30*time.Minute)), false}, + {"not expired tenant by far", newChunkEntry("2", `{foo="buzz"}`, model.Now().Add(-72*time.Hour), model.Now().Add(-3*time.Hour)), false}, + {"expired stream override", newChunkEntry("2", `{foo="bar"}`, model.Now().Add(-12*time.Hour), model.Now().Add(-10*time.Hour)), true}, + {"non expired stream override", newChunkEntry("1", `{foo="bar"}`, model.Now().Add(-3*time.Hour), model.Now().Add(-90*time.Minute)), false}, } for _, tt := range tests { t.Run(tt.name, func(t *testing.T) { diff --git a/pkg/storage/stores/shipper/compactor/retention/index.go b/pkg/storage/stores/shipper/compactor/retention/index.go index 1597e3f776f84..a986271493a5e 100644 --- a/pkg/storage/stores/shipper/compactor/retention/index.go +++ b/pkg/storage/stores/shipper/compactor/retention/index.go @@ -15,11 +15,8 @@ import ( const ( chunkTimeRangeKeyV3 = '3' - metricNameRangeKeyV1 = '6' seriesRangeKeyV1 = '7' labelSeriesRangeKeyV1 = '8' - labelNamesRangeKeyV1 = '9' - separator = "\000" ) var ErrInvalidIndexKey = errors.New("invalid index key") diff --git a/pkg/storage/stores/shipper/compactor/retention/iterator_test.go b/pkg/storage/stores/shipper/compactor/retention/iterator_test.go index 4227a933d894d..89de74739d4ca 100644 --- a/pkg/storage/stores/shipper/compactor/retention/iterator_test.go +++ b/pkg/storage/stores/shipper/compactor/retention/iterator_test.go @@ -7,11 +7,12 @@ import ( "time" "github.com/cortexproject/cortex/pkg/chunk" - shipper_util "github.com/grafana/loki/pkg/storage/stores/shipper/util" "github.com/prometheus/common/model" "github.com/prometheus/prometheus/pkg/labels" "github.com/stretchr/testify/require" "go.etcd.io/bbolt" + + shipper_util "github.com/grafana/loki/pkg/storage/stores/shipper/util" ) func Test_ChunkIterator(t *testing.T) { diff --git a/pkg/storage/stores/shipper/compactor/retention/retention_test.go b/pkg/storage/stores/shipper/compactor/retention/retention_test.go index 38878ae04a3b0..a17ca549de5bc 100644 --- a/pkg/storage/stores/shipper/compactor/retention/retention_test.go +++ b/pkg/storage/stores/shipper/compactor/retention/retention_test.go @@ -7,6 +7,7 @@ import ( "path/filepath" "strconv" "strings" + "sync" "testing" "time" @@ -14,6 +15,7 @@ import ( "github.com/cortexproject/cortex/pkg/ingester/client" "github.com/prometheus/common/model" "github.com/prometheus/prometheus/pkg/labels" + "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" "github.com/grafana/loki/pkg/chunkenc" @@ -69,26 +71,50 @@ func Test_Retention(t *testing.T) { tt := tt t.Run(tt.name, func(t *testing.T) { // insert in the store. - store := newTestStore(t) + var ( + store = newTestStore(t) + expectDeleted = []string{} + actualDeleted = []string{} + lock sync.Mutex + ) for _, c := range tt.chunks { require.NoError(t, store.Put(context.TODO(), []chunk.Chunk{c})) } store.Stop() - // marks + // marks and sweep expiration := NewExpirationChecker(tt.limits) workDir := filepath.Join(t.TempDir(), "retention") marker := NewMarker(workDir, store.schemaCfg, util.NewPrefixedObjectClient(store.objectClient, "index/"), expiration) for _, table := range store.indexTables() { require.NoError(t, marker.MarkTableForDelete(context.Background(), table.name)) } + sweep, err := NewSweeper(workDir, DeleteClientFunc(func(ctx context.Context, objectKey string) error { + lock.Lock() + defer lock.Unlock() + actualDeleted = append(actualDeleted, objectKey) + return nil + }), 0) + sweep.Start() + defer sweep.Stop() + require.NoError(t, err) // assert using the store again. store.open() for i, e := range tt.alive { require.Equal(t, e, store.HasChunk(tt.chunks[i])) + if !e { + expectDeleted = append(expectDeleted, tt.chunks[i].ExternalKey()) + } } store.Stop() + if len(expectDeleted) != 0 { + require.Eventually(t, func() bool { + lock.Lock() + defer lock.Unlock() + return assert.Equal(t, expectDeleted, actualDeleted) + }, 10*time.Second, 1*time.Second) + } }) } } diff --git a/pkg/storage/stores/shipper/compactor/retention/util.go b/pkg/storage/stores/shipper/compactor/retention/util.go index 991cc3e0225c9..9caf47e6342e0 100644 --- a/pkg/storage/stores/shipper/compactor/retention/util.go +++ b/pkg/storage/stores/shipper/compactor/retention/util.go @@ -17,23 +17,3 @@ func unsafeGetBytes(s string) []byte { (*reflect.SliceHeader)(p).Cap = len(s) return buf } - -// Build an index key, encoded as multiple parts separated by a 0 byte, with extra space at the end. -func buildRangeValue(extra int, ss ...[]byte) []byte { - length := extra - for _, s := range ss { - length += len(s) + 1 - } - output, i := make([]byte, length), 0 - for _, s := range ss { - i += copy(output[i:], s) + 1 - } - return output -} - -// Encode a complete key including type marker (which goes at the end) -func encodeRangeKey(keyType byte, ss ...[]byte) []byte { - output := buildRangeValue(2, ss...) - output[len(output)-2] = keyType - return output -} From b0aa3dd165cd4fa3b28a907d2c0d07f29a8d59d4 Mon Sep 17 00:00:00 2001 From: Cyril Tovena Date: Thu, 22 Apr 2021 11:10:06 +0200 Subject: [PATCH 21/42] Fixes moar bugs with regards to period schema Signed-off-by: Cyril Tovena --- .../stores/shipper/compactor/compactor.go | 6 ++- .../shipper/compactor/retention/index.go | 32 ++++++++++++++-- .../shipper/compactor/retention/index_test.go | 38 +++++++++++++++++++ .../shipper/compactor/retention/retention.go | 7 +++- .../compactor/retention/retention_test.go | 38 +++++++++++++++++-- .../shipper/compactor/retention/util_test.go | 24 ++++++++---- 6 files changed, 127 insertions(+), 18 deletions(-) create mode 100644 pkg/storage/stores/shipper/compactor/retention/index_test.go diff --git a/pkg/storage/stores/shipper/compactor/compactor.go b/pkg/storage/stores/shipper/compactor/compactor.go index 7ed663e7276a3..f9f152fc193f3 100644 --- a/pkg/storage/stores/shipper/compactor/compactor.go +++ b/pkg/storage/stores/shipper/compactor/compactor.go @@ -91,11 +91,15 @@ func NewCompactor(cfg Config, storageConfig storage.Config, schemaConfig loki_st if err != nil { return nil, err } + marker, err := retention.NewMarker(retentionWorkDir, schemaConfig, prefixedClient, retention.NewExpirationChecker(limits)) + if err != nil { + return nil, err + } compactor := Compactor{ cfg: cfg, objectClient: prefixedClient, metrics: newMetrics(r), - tableMarker: retention.NewMarker(retentionWorkDir, schemaConfig, prefixedClient, retention.NewExpirationChecker(limits)), + tableMarker: marker, sweeper: sweeper, } diff --git a/pkg/storage/stores/shipper/compactor/retention/index.go b/pkg/storage/stores/shipper/compactor/retention/index.go index a986271493a5e..9ff9e3cd7175e 100644 --- a/pkg/storage/stores/shipper/compactor/retention/index.go +++ b/pkg/storage/stores/shipper/compactor/retention/index.go @@ -201,18 +201,42 @@ func parseLabelSeriesRangeKey(hashKey, rangeKey []byte) (LabelSeriesRangeKey, bo }, true, nil } +func validatePeriods(config storage.SchemaConfig) error { + for _, schema := range config.Configs { + if schema.IndexTables.Period != 24*time.Hour { + return fmt.Errorf("schema period must be daily, was: %s", schema.IndexTables.Period) + } + } + return nil +} + func schemaPeriodForTable(config storage.SchemaConfig, tableName string) (chunk.PeriodConfig, bool) { + // first round removes configs that does not have the prefix. + candidates := []chunk.PeriodConfig{} for _, schema := range config.Configs { + if strings.HasPrefix(tableName, schema.IndexTables.Prefix) { + candidates = append(candidates, schema) + } + } + // WARN we assume period is always daily. This is only true for boltdb-shipper. + var ( + matched chunk.PeriodConfig + found bool + ) + for _, schema := range candidates { periodIndex, err := strconv.ParseInt(strings.TrimPrefix(tableName, schema.IndexTables.Prefix), 10, 64) if err != nil { continue } - periodSecs := int64((schema.IndexTables.Period) / time.Second) - if periodIndex == schema.From.Time.Unix()/periodSecs { - return schema, true + periodSec := int64(schema.IndexTables.Period / time.Second) + tableTs := model.TimeFromUnix(periodIndex * periodSec) + if tableTs.After(schema.From.Time) || tableTs == schema.From.Time { + matched = schema + found = true } } - return chunk.PeriodConfig{}, false + + return matched, found } func seriesFromHash(h []byte) (seriesID []byte) { diff --git a/pkg/storage/stores/shipper/compactor/retention/index_test.go b/pkg/storage/stores/shipper/compactor/retention/index_test.go new file mode 100644 index 0000000000000..81c39abc36e0c --- /dev/null +++ b/pkg/storage/stores/shipper/compactor/retention/index_test.go @@ -0,0 +1,38 @@ +package retention + +import ( + "fmt" + "testing" + "time" + + "github.com/cortexproject/cortex/pkg/chunk" + "github.com/stretchr/testify/require" + + "github.com/grafana/loki/pkg/storage" +) + +func Test_schemaPeriodForTable(t *testing.T) { + indexFromTime := func(t time.Time) string { + return fmt.Sprintf("%d", t.Unix()/int64(24*time.Hour/time.Second)) + } + tests := []struct { + name string + config storage.SchemaConfig + tableName string + expected chunk.PeriodConfig + expectedFound bool + }{ + {"out of scope", schemaCfg, "index_" + indexFromTime(start.Time().Add(-24*time.Hour)), chunk.PeriodConfig{}, false}, + {"first table", schemaCfg, "index_" + indexFromTime(dayFromTime(start).Time.Time()), schemaCfg.Configs[0], true}, + {"4 hour after first table", schemaCfg, "index_" + indexFromTime(dayFromTime(start).Time.Time().Add(4*time.Hour)), schemaCfg.Configs[0], true}, + {"second schema", schemaCfg, "index_" + indexFromTime(dayFromTime(start.Add(28*time.Hour)).Time.Time()), schemaCfg.Configs[1], true}, + {"now", schemaCfg, "index_" + indexFromTime(time.Now()), schemaCfg.Configs[2], true}, + } + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + actual, actualFound := schemaPeriodForTable(tt.config, tt.tableName) + require.Equal(t, tt.expected, actual) + require.Equal(t, tt.expectedFound, actualFound) + }) + } +} diff --git a/pkg/storage/stores/shipper/compactor/retention/retention.go b/pkg/storage/stores/shipper/compactor/retention/retention.go index 3d7ff665e0583..a2d2f770813e1 100644 --- a/pkg/storage/stores/shipper/compactor/retention/retention.go +++ b/pkg/storage/stores/shipper/compactor/retention/retention.go @@ -42,13 +42,16 @@ type Marker struct { expiration ExpirationChecker } -func NewMarker(workingDirectory string, config storage.SchemaConfig, objectClient chunk.ObjectClient, expiration ExpirationChecker) *Marker { +func NewMarker(workingDirectory string, config storage.SchemaConfig, objectClient chunk.ObjectClient, expiration ExpirationChecker) (*Marker, error) { + if err := validatePeriods(config); err != nil { + return nil, err + } return &Marker{ workingDirectory: workingDirectory, config: config, objectClient: objectClient, expiration: expiration, - } + }, nil } func (t *Marker) MarkTableForDelete(ctx context.Context, tableName string) error { diff --git a/pkg/storage/stores/shipper/compactor/retention/retention_test.go b/pkg/storage/stores/shipper/compactor/retention/retention_test.go index a17ca549de5bc..b4c7bfcca23fa 100644 --- a/pkg/storage/stores/shipper/compactor/retention/retention_test.go +++ b/pkg/storage/stores/shipper/compactor/retention/retention_test.go @@ -67,6 +67,32 @@ func Test_Retention(t *testing.T) { true, }, }, + { + "one global expiration and stream", + fakeLimits{ + perTenant: map[string]time.Duration{ + "1": 10 * time.Hour, + "2": 1000 * time.Hour, + }, + perStream: map[string][]validation.StreamRetention{ + "1": { + {Period: 5 * time.Hour, Matchers: []*labels.Matcher{labels.MustNewMatcher(labels.MatchEqual, "foo", "buzz")}}, + }, + }, + }, + []chunk.Chunk{ + createChunk(t, "1", labels.Labels{labels.Label{Name: "foo", Value: "bar"}}, start, start.Add(1*time.Hour)), + createChunk(t, "2", labels.Labels{labels.Label{Name: "foo", Value: "fuzz"}}, start.Add(26*time.Hour), start.Add(27*time.Hour)), + createChunk(t, "1", labels.Labels{labels.Label{Name: "foo", Value: "buzz"}}, model.Now().Add(-2*time.Hour), model.Now().Add(-1*time.Hour)), + createChunk(t, "1", labels.Labels{labels.Label{Name: "foo", Value: "buzz"}}, model.Now().Add(-7*time.Hour), model.Now().Add(-6*time.Hour)), + }, + []bool{ + false, + true, + true, + false, + }, + }, } { tt := tt t.Run(tt.name, func(t *testing.T) { @@ -85,14 +111,16 @@ func Test_Retention(t *testing.T) { // marks and sweep expiration := NewExpirationChecker(tt.limits) workDir := filepath.Join(t.TempDir(), "retention") - marker := NewMarker(workDir, store.schemaCfg, util.NewPrefixedObjectClient(store.objectClient, "index/"), expiration) + marker, err := NewMarker(workDir, store.schemaCfg, util.NewPrefixedObjectClient(store.objectClient, "index/"), expiration) + require.NoError(t, err) for _, table := range store.indexTables() { require.NoError(t, marker.MarkTableForDelete(context.Background(), table.name)) } sweep, err := NewSweeper(workDir, DeleteClientFunc(func(ctx context.Context, objectKey string) error { lock.Lock() defer lock.Unlock() - actualDeleted = append(actualDeleted, objectKey) + key := string([]byte(objectKey)) // forces a copy, because this string is only valid within the delete fn. + actualDeleted = append(actualDeleted, key) return nil }), 0) sweep.Start() @@ -101,13 +129,15 @@ func Test_Retention(t *testing.T) { // assert using the store again. store.open() + defer store.Stop() + for i, e := range tt.alive { - require.Equal(t, e, store.HasChunk(tt.chunks[i])) + require.Equal(t, e, store.HasChunk(tt.chunks[i]), "chunk %d should be %t", i, e) if !e { expectDeleted = append(expectDeleted, tt.chunks[i].ExternalKey()) } } - store.Stop() + if len(expectDeleted) != 0 { require.Eventually(t, func() bool { lock.Lock() diff --git a/pkg/storage/stores/shipper/compactor/retention/util_test.go b/pkg/storage/stores/shipper/compactor/retention/util_test.go index 9d49fcdff1f08..ebefbfb3d0c5d 100644 --- a/pkg/storage/stores/shipper/compactor/retention/util_test.go +++ b/pkg/storage/stores/shipper/compactor/retention/util_test.go @@ -26,13 +26,24 @@ import ( "github.com/grafana/loki/pkg/util/validation" ) +func dayFromTime(t model.Time) chunk.DayTime { + parsed, err := time.Parse("2006-01-02", t.Time().Format("2006-01-02")) + if err != nil { + panic(err) + } + return chunk.DayTime{ + Time: model.TimeFromUnix(parsed.Unix()), + } +} + var ( + start = model.Now().Add(-30 * 24 * time.Hour) schemaCfg = storage.SchemaConfig{ SchemaConfig: chunk.SchemaConfig{ // we want to test over all supported schema. Configs: []chunk.PeriodConfig{ { - From: chunk.DayTime{Time: start}, + From: dayFromTime(start), IndexType: "boltdb", ObjectType: "filesystem", Schema: "v9", @@ -43,7 +54,7 @@ var ( RowShards: 16, }, { - From: chunk.DayTime{Time: start.Add(25 * time.Hour)}, + From: dayFromTime(start.Add(25 * time.Hour)), IndexType: "boltdb", ObjectType: "filesystem", Schema: "v10", @@ -54,7 +65,7 @@ var ( RowShards: 16, }, { - From: chunk.DayTime{Time: start.Add(49 * time.Hour)}, + From: dayFromTime(start.Add(49 * time.Hour)), IndexType: "boltdb", ObjectType: "filesystem", Schema: "v11", @@ -72,11 +83,10 @@ var ( from model.Time config chunk.PeriodConfig }{ - {"v9", start, schemaCfg.Configs[0]}, - {"v10", start.Add(25 * time.Hour), schemaCfg.Configs[1]}, - {"v11", start.Add(49 * time.Hour), schemaCfg.Configs[2]}, + {"v9", schemaCfg.Configs[0].From.Time, schemaCfg.Configs[0]}, + {"v10", schemaCfg.Configs[1].From.Time, schemaCfg.Configs[1]}, + {"v11", schemaCfg.Configs[2].From.Time, schemaCfg.Configs[2]}, } - start = model.Now().Add(-30 * 24 * time.Hour) ) func newChunkEntry(userID, labels string, from, through model.Time) ChunkEntry { From bb6c010fa5f25473899a841e405db7ae95f24e94 Mon Sep 17 00:00:00 2001 From: Cyril Tovena Date: Thu, 22 Apr 2021 11:57:53 +0200 Subject: [PATCH 22/42] Fix a flaky tests because of boltdb still open. Signed-off-by: Cyril Tovena --- .../shipper/compactor/retention/marker.go | 4 +- .../shipper/compactor/retention/metrics.go | 59 +++++++++++++++++++ .../compactor/retention/retention_test.go | 16 ++--- 3 files changed, 70 insertions(+), 9 deletions(-) create mode 100644 pkg/storage/stores/shipper/compactor/retention/metrics.go diff --git a/pkg/storage/stores/shipper/compactor/retention/marker.go b/pkg/storage/stores/shipper/compactor/retention/marker.go index 6bf680d3fb75d..75b379a51e319 100644 --- a/pkg/storage/stores/shipper/compactor/retention/marker.go +++ b/pkg/storage/stores/shipper/compactor/retention/marker.go @@ -227,8 +227,8 @@ func processKey(ctx context.Context, key *bytes.Buffer, db *bbolt.DB, deleteFunc if err := deleteFunc(ctx, keyData); err != nil { return err } - // no error we can delete the key - return db.Batch(func(tx *bbolt.Tx) error { + // we don't use a batch because it would force us to copy the key. + return db.Update(func(tx *bbolt.Tx) error { b := tx.Bucket(chunkBucket) if b == nil { return nil diff --git a/pkg/storage/stores/shipper/compactor/retention/metrics.go b/pkg/storage/stores/shipper/compactor/retention/metrics.go new file mode 100644 index 0000000000000..281e891564c34 --- /dev/null +++ b/pkg/storage/stores/shipper/compactor/retention/metrics.go @@ -0,0 +1,59 @@ +package retention + +import ( + "github.com/prometheus/client_golang/prometheus" + "github.com/prometheus/client_golang/prometheus/promauto" +) + +const ( + statusFailure = "failure" + statusSuccess = "success" +) + +type metrics struct { + deletedChunkTotal *prometheus.CounterVec + // compactTablesOperationDurationSeconds prometheus.Gauge + // compactTablesOperationLastSuccess prometheus.Gauge + + // retentionOperationTotal *prometheus.CounterVec + // retentionOperationDurationSeconds prometheus.Gauge + // retentionOperationLastSuccess prometheus.Gauge +} + +func newMetrics(r prometheus.Registerer) *metrics { + m := metrics{ + deletedChunkTotal: promauto.With(r).NewCounterVec(prometheus.CounterOpts{ + Namespace: "loki_boltdb_shipper", + Name: "retention_chunk_deleted_total", + Help: "Total number of chunks deleted by retention", + }, []string{"status"}), + // compactTablesOperationDurationSeconds: promauto.With(r).NewGauge(prometheus.GaugeOpts{ + // Namespace: "loki_boltdb_shipper", + // Name: "compact_tables_operation_duration_seconds", + // Help: "Time (in seconds) spent in compacting all the tables", + // }), + // compactTablesOperationLastSuccess: promauto.With(r).NewGauge(prometheus.GaugeOpts{ + // Namespace: "loki_boltdb_shipper", + // Name: "compact_tables_operation_last_successful_run_timestamp_seconds", + // Help: "Unix timestamp of the last successful compaction run", + // }), + + // retentionOperationTotal: promauto.With(r).NewCounterVec(prometheus.CounterOpts{ + // Namespace: "loki_boltdb_shipper", + // Name: "retention_operation_total", + // Help: "Total number of retention applied by status", + // }, []string{"status"}), + // retentionOperationDurationSeconds: promauto.With(r).NewGauge(prometheus.GaugeOpts{ + // Namespace: "loki_boltdb_shipper", + // Name: "retention_operation_duration_seconds", + // Help: "Time (in seconds) spent in applying retention for all the tables", + // }), + // retentionOperationLastSuccess: promauto.With(r).NewGauge(prometheus.GaugeOpts{ + // Namespace: "loki_boltdb_shipper", + // Name: "retention_operation_last_successful_run_timestamp_seconds", + // Help: "Unix timestamp of the last successful retention run", + // }), + } + + return &m +} diff --git a/pkg/storage/stores/shipper/compactor/retention/retention_test.go b/pkg/storage/stores/shipper/compactor/retention/retention_test.go index b4c7bfcca23fa..74521fdcb65cf 100644 --- a/pkg/storage/stores/shipper/compactor/retention/retention_test.go +++ b/pkg/storage/stores/shipper/compactor/retention/retention_test.go @@ -25,6 +25,7 @@ import ( ) func Test_Retention(t *testing.T) { + minListMarkDelay = 1 * time.Second for _, tt := range []struct { name string limits Limits @@ -111,11 +112,6 @@ func Test_Retention(t *testing.T) { // marks and sweep expiration := NewExpirationChecker(tt.limits) workDir := filepath.Join(t.TempDir(), "retention") - marker, err := NewMarker(workDir, store.schemaCfg, util.NewPrefixedObjectClient(store.objectClient, "index/"), expiration) - require.NoError(t, err) - for _, table := range store.indexTables() { - require.NoError(t, marker.MarkTableForDelete(context.Background(), table.name)) - } sweep, err := NewSweeper(workDir, DeleteClientFunc(func(ctx context.Context, objectKey string) error { lock.Lock() defer lock.Unlock() @@ -123,13 +119,19 @@ func Test_Retention(t *testing.T) { actualDeleted = append(actualDeleted, key) return nil }), 0) + require.NoError(t, err) sweep.Start() defer sweep.Stop() + + marker, err := NewMarker(workDir, store.schemaCfg, util.NewPrefixedObjectClient(store.objectClient, "index/"), expiration) require.NoError(t, err) + for _, table := range store.indexTables() { + table.Close() + require.NoError(t, marker.MarkTableForDelete(context.Background(), table.name)) + } // assert using the store again. store.open() - defer store.Stop() for i, e := range tt.alive { require.Equal(t, e, store.HasChunk(tt.chunks[i]), "chunk %d should be %t", i, e) @@ -137,7 +139,7 @@ func Test_Retention(t *testing.T) { expectDeleted = append(expectDeleted, tt.chunks[i].ExternalKey()) } } - + store.Stop() if len(expectDeleted) != 0 { require.Eventually(t, func() bool { lock.Lock() From a10c8982e2b02967b7483dba3ce926d694b5b0a6 Mon Sep 17 00:00:00 2001 From: Cyril Tovena Date: Thu, 22 Apr 2021 14:30:11 +0200 Subject: [PATCH 23/42] Add more metrics. Signed-off-by: Cyril Tovena --- .../stores/shipper/compactor/compactor.go | 2 +- .../shipper/compactor/retention/marker.go | 22 ++++-- .../compactor/retention/marker_test.go | 35 +++++---- .../shipper/compactor/retention/metrics.go | 76 +++++++++---------- .../shipper/compactor/retention/retention.go | 21 ++++- .../compactor/retention/retention_test.go | 2 +- .../shipper/compactor/retention/util_test.go | 3 + 7 files changed, 91 insertions(+), 70 deletions(-) diff --git a/pkg/storage/stores/shipper/compactor/compactor.go b/pkg/storage/stores/shipper/compactor/compactor.go index f9f152fc193f3..bcb475bd75424 100644 --- a/pkg/storage/stores/shipper/compactor/compactor.go +++ b/pkg/storage/stores/shipper/compactor/compactor.go @@ -87,7 +87,7 @@ func NewCompactor(cfg Config, storageConfig storage.Config, schemaConfig loki_st retentionWorkDir := filepath.Join(cfg.WorkingDirectory, "retention") - sweeper, err := retention.NewSweeper(retentionWorkDir, retention.NewDeleteClient(objectClient), cfg.RetentionDeleteDelay) + sweeper, err := retention.NewSweeper(retentionWorkDir, retention.NewDeleteClient(objectClient), cfg.RetentionDeleteDelay, r) if err != nil { return nil, err } diff --git a/pkg/storage/stores/shipper/compactor/retention/marker.go b/pkg/storage/stores/shipper/compactor/retention/marker.go index 75b379a51e319..bfeb879d2af18 100644 --- a/pkg/storage/stores/shipper/compactor/retention/marker.go +++ b/pkg/storage/stores/shipper/compactor/retention/marker.go @@ -107,9 +107,11 @@ type markerProcessor struct { ctx context.Context cancel context.CancelFunc wg sync.WaitGroup + + sweeperMetrics *sweeperMetrics } -func newMarkerStorageReader(workingDir string, maxParallelism int, minAgeFile time.Duration) (*markerProcessor, error) { +func newMarkerStorageReader(workingDir string, maxParallelism int, minAgeFile time.Duration, sweeperMetrics *sweeperMetrics) (*markerProcessor, error) { folder := filepath.Join(workingDir, markersFolder) err := chunk_util.EnsureDirectory(folder) if err != nil { @@ -122,6 +124,7 @@ func newMarkerStorageReader(workingDir string, maxParallelism int, minAgeFile ti cancel: cancel, maxParallelism: maxParallelism, minAgeFile: minAgeFile, + sweeperMetrics: sweeperMetrics, }, nil } @@ -144,18 +147,20 @@ func (r *markerProcessor) Start(deleteFunc func(ctx context.Context, chunkId []b // cancelled return } - paths, err := r.availablePath() + paths, times, err := r.availablePath() if err != nil { level.Error(util_log.Logger).Log("msg", "failed to list marks path", "path", r.folder, "err", err) continue } + r.sweeperMetrics.markerFilesCurrent.Set(float64(len(paths))) if len(paths) == 0 { level.Info(util_log.Logger).Log("msg", "No marks file found") } - for _, path := range paths { + for i, path := range paths { if r.ctx.Err() != nil { return } + r.sweeperMetrics.markerFileCurrentTime.Set(float64(times[i].UnixNano()) / 1e9) if err := r.processPath(path, deleteFunc); err != nil { level.Warn(util_log.Logger).Log("msg", "failed to process marks", "path", path, "err", err) continue @@ -261,13 +266,14 @@ func (r *markerProcessor) deleteEmptyMarks(path string) error { return err } if empty { + r.sweeperMetrics.markerFilesDeletedTotal.Inc() return os.Remove(path) } return nil } // availablePath returns markers path in chronological order, skipping file that are not old enough. -func (r *markerProcessor) availablePath() ([]string, error) { +func (r *markerProcessor) availablePath() ([]string, []time.Time, error) { found := []int64{} if err := filepath.WalkDir(r.folder, func(path string, d fs.DirEntry, err error) error { if d == nil || err != nil { @@ -292,17 +298,19 @@ func (r *markerProcessor) availablePath() ([]string, error) { } return nil }); err != nil { - return nil, err + return nil, nil, err } if len(found) == 0 { - return nil, nil + return nil, nil, nil } sort.Slice(found, func(i, j int) bool { return found[i] < found[j] }) res := make([]string, len(found)) + resTime := make([]time.Time, len(found)) for i, f := range found { res[i] = filepath.Join(r.folder, fmt.Sprintf("%d", f)) + resTime[i] = time.Unix(0, f) } - return res, nil + return res, resTime, nil } func (r *markerProcessor) Stop() { diff --git a/pkg/storage/stores/shipper/compactor/retention/marker_test.go b/pkg/storage/stores/shipper/compactor/retention/marker_test.go index 4651d48d5981d..9fab3534928ac 100644 --- a/pkg/storage/stores/shipper/compactor/retention/marker_test.go +++ b/pkg/storage/stores/shipper/compactor/retention/marker_test.go @@ -19,7 +19,7 @@ func initAndFeedMarkerProcessor(t *testing.T) *markerProcessor { t.Helper() minListMarkDelay = time.Second dir := t.TempDir() - p, err := newMarkerStorageReader(dir, 5, time.Second) + p, err := newMarkerStorageReader(dir, 5, time.Second, sweepMetrics) require.NoError(t, err) go func() { w, err := NewMarkerStorageWriter(dir) @@ -98,15 +98,15 @@ func Test_markerProcessor_StartDeleteOnSuccess(t *testing.T) { func Test_markerProcessor_availablePath(t *testing.T) { now := time.Now() for _, tt := range []struct { - name string - expectedPath func(dir string) []string + name string + expected func(dir string) ([]string, []time.Time) }{ - {"empty", func(_ string) []string { return nil }}, - {"skips bad files", func(dir string) []string { + {"empty", func(_ string) ([]string, []time.Time) { return nil, nil }}, + {"skips bad files", func(dir string) ([]string, []time.Time) { _, _ = os.Create(filepath.Join(dir, "foo")) - return nil + return nil, nil }}, - {"happy path", func(dir string) []string { + {"happy path", func(dir string) ([]string, []time.Time) { _, _ = os.Create(filepath.Join(dir, fmt.Sprintf("%d", now.UnixNano()))) _, _ = os.Create(filepath.Join(dir, "foo")) _, _ = os.Create(filepath.Join(dir, fmt.Sprintf("%d", now.Add(-30*time.Minute).UnixNano()))) @@ -115,22 +115,27 @@ func Test_markerProcessor_availablePath(t *testing.T) { _, _ = os.Create(filepath.Join(dir, fmt.Sprintf("%d", now.Add(-2*time.Hour).UnixNano()))) _, _ = os.Create(filepath.Join(dir, fmt.Sprintf("%d", now.Add(-48*time.Hour).UnixNano()))) return []string{ - filepath.Join(dir, fmt.Sprintf("%d", now.Add(-48*time.Hour).UnixNano())), // oldest should be first - filepath.Join(dir, fmt.Sprintf("%d", now.Add(-3*time.Hour).UnixNano())), - filepath.Join(dir, fmt.Sprintf("%d", now.Add(-2*time.Hour).UnixNano())), - } + filepath.Join(dir, fmt.Sprintf("%d", now.Add(-48*time.Hour).UnixNano())), // oldest should be first + filepath.Join(dir, fmt.Sprintf("%d", now.Add(-3*time.Hour).UnixNano())), + filepath.Join(dir, fmt.Sprintf("%d", now.Add(-2*time.Hour).UnixNano())), + }, []time.Time{ + time.Unix(0, now.Add(-48*time.Hour).UnixNano()), + time.Unix(0, now.Add(-3*time.Hour).UnixNano()), + time.Unix(0, now.Add(-2*time.Hour).UnixNano()), + } }}, } { t.Run("", func(t *testing.T) { dir := t.TempDir() - p, err := newMarkerStorageReader(dir, 5, 2*time.Hour) + p, err := newMarkerStorageReader(dir, 5, 2*time.Hour, sweepMetrics) - expected := tt.expectedPath(p.folder) + expectedPath, expectedTimes := tt.expected(p.folder) require.NoError(t, err) - paths, err := p.availablePath() + paths, times, err := p.availablePath() require.Nil(t, err) - require.Equal(t, expected, paths) + require.Equal(t, expectedPath, paths) + require.Equal(t, expectedTimes, times) }) } } diff --git a/pkg/storage/stores/shipper/compactor/retention/metrics.go b/pkg/storage/stores/shipper/compactor/retention/metrics.go index 281e891564c34..417d8e50fd8fb 100644 --- a/pkg/storage/stores/shipper/compactor/retention/metrics.go +++ b/pkg/storage/stores/shipper/compactor/retention/metrics.go @@ -6,54 +6,46 @@ import ( ) const ( - statusFailure = "failure" - statusSuccess = "success" + statusFailure = "failure" + statusSuccess = "success" + statusNotFound = "notfound" ) -type metrics struct { - deletedChunkTotal *prometheus.CounterVec - // compactTablesOperationDurationSeconds prometheus.Gauge - // compactTablesOperationLastSuccess prometheus.Gauge - - // retentionOperationTotal *prometheus.CounterVec - // retentionOperationDurationSeconds prometheus.Gauge - // retentionOperationLastSuccess prometheus.Gauge +type sweeperMetrics struct { + deleteChunkTotal *prometheus.CounterVec + deleteChunkDurationSeconds *prometheus.HistogramVec + markerFileCurrentTime prometheus.Gauge + markerFilesCurrent prometheus.Gauge + markerFilesDeletedTotal prometheus.Counter } -func newMetrics(r prometheus.Registerer) *metrics { - m := metrics{ - deletedChunkTotal: promauto.With(r).NewCounterVec(prometheus.CounterOpts{ +func newSweeperMetrics(r prometheus.Registerer) *sweeperMetrics { + return &sweeperMetrics{ + deleteChunkTotal: promauto.With(r).NewCounterVec(prometheus.CounterOpts{ Namespace: "loki_boltdb_shipper", - Name: "retention_chunk_deleted_total", + Name: "retention_sweeper_chunk_deleted_total", Help: "Total number of chunks deleted by retention", }, []string{"status"}), - // compactTablesOperationDurationSeconds: promauto.With(r).NewGauge(prometheus.GaugeOpts{ - // Namespace: "loki_boltdb_shipper", - // Name: "compact_tables_operation_duration_seconds", - // Help: "Time (in seconds) spent in compacting all the tables", - // }), - // compactTablesOperationLastSuccess: promauto.With(r).NewGauge(prometheus.GaugeOpts{ - // Namespace: "loki_boltdb_shipper", - // Name: "compact_tables_operation_last_successful_run_timestamp_seconds", - // Help: "Unix timestamp of the last successful compaction run", - // }), - - // retentionOperationTotal: promauto.With(r).NewCounterVec(prometheus.CounterOpts{ - // Namespace: "loki_boltdb_shipper", - // Name: "retention_operation_total", - // Help: "Total number of retention applied by status", - // }, []string{"status"}), - // retentionOperationDurationSeconds: promauto.With(r).NewGauge(prometheus.GaugeOpts{ - // Namespace: "loki_boltdb_shipper", - // Name: "retention_operation_duration_seconds", - // Help: "Time (in seconds) spent in applying retention for all the tables", - // }), - // retentionOperationLastSuccess: promauto.With(r).NewGauge(prometheus.GaugeOpts{ - // Namespace: "loki_boltdb_shipper", - // Name: "retention_operation_last_successful_run_timestamp_seconds", - // Help: "Unix timestamp of the last successful retention run", - // }), + deleteChunkDurationSeconds: promauto.With(r).NewHistogramVec(prometheus.HistogramOpts{ + Namespace: "loki_boltdb_shipper", + Name: "retention_sweeper_chunk_deleted_duration_seconds", + Help: "Time (in seconds) spent in deleting chunk", + Buckets: prometheus.ExponentialBuckets(0.1, 2, 8), + }, []string{"status"}), + markerFilesCurrent: promauto.With(r).NewGauge(prometheus.GaugeOpts{ + Namespace: "loki_boltdb_shipper", + Name: "retention_sweeper_marker_files_current", + Help: "The current total of marker files valid for deletion.", + }), + markerFileCurrentTime: promauto.With(r).NewGauge(prometheus.GaugeOpts{ + Namespace: "loki_boltdb_shipper", + Name: "retention_sweeper_marker_file_processing_current_time", + Help: "The current time of creation of the marker file being processed.", + }), + markerFilesDeletedTotal: promauto.With(r).NewCounter(prometheus.CounterOpts{ + Namespace: "loki_boltdb_shipper", + Name: "retention_sweeper_marker_files_deleted_total", + Help: "The total of marker files deleted after being fully processed.", + }), } - - return &m } diff --git a/pkg/storage/stores/shipper/compactor/retention/retention.go b/pkg/storage/stores/shipper/compactor/retention/retention.go index a2d2f770813e1..9b82b056e6e8f 100644 --- a/pkg/storage/stores/shipper/compactor/retention/retention.go +++ b/pkg/storage/stores/shipper/compactor/retention/retention.go @@ -15,6 +15,7 @@ import ( chunk_util "github.com/cortexproject/cortex/pkg/chunk/util" util_log "github.com/cortexproject/cortex/pkg/util/log" "github.com/go-kit/kit/log/level" + "github.com/prometheus/client_golang/prometheus" "go.etcd.io/bbolt" "github.com/grafana/loki/pkg/storage" @@ -128,8 +129,7 @@ func (t *Marker) MarkTableForDelete(ctx context.Context, tableName string) error return fmt.Errorf("failed to create chunk index iterator: %w", err) } - seriesCleaner := newSeriesCleaner(bucket, schemaCfg) - empty, err = markforDelete(markerWriter, chunkIt, seriesCleaner, t.expiration) + empty, err = markforDelete(markerWriter, chunkIt, newSeriesCleaner(bucket, schemaCfg), t.expiration) if err != nil { return err } @@ -227,27 +227,40 @@ func NewDeleteClient(objectClient chunk.ObjectClient) DeleteClient { type Sweeper struct { markerProcessor MarkerProcessor deleteClient DeleteClient + sweeperMetrics *sweeperMetrics } -func NewSweeper(workingDir string, deleteClient DeleteClient, minAgeDelete time.Duration) (*Sweeper, error) { - p, err := newMarkerStorageReader(workingDir, deletionWorkerCount, minAgeDelete) +func NewSweeper(workingDir string, deleteClient DeleteClient, minAgeDelete time.Duration, r prometheus.Registerer) (*Sweeper, error) { + m := newSweeperMetrics(r) + p, err := newMarkerStorageReader(workingDir, deletionWorkerCount, minAgeDelete, m) if err != nil { return nil, err } return &Sweeper{ markerProcessor: p, deleteClient: deleteClient, + sweeperMetrics: m, }, nil } func (s *Sweeper) Start() { s.markerProcessor.Start(func(ctx context.Context, chunkId []byte) error { + status := statusSuccess + start := time.Now() + defer func() { + s.sweeperMetrics.deleteChunkTotal.WithLabelValues(status).Inc() + s.sweeperMetrics.deleteChunkDurationSeconds.WithLabelValues(status).Observe(time.Since(start).Seconds()) + }() chunkIDString := unsafeGetString(chunkId) err := s.deleteClient.DeleteObject(ctx, chunkIDString) if err == chunk.ErrStorageObjectNotFound { + status = statusNotFound level.Debug(util_log.Logger).Log("msg", "delete on not found chunk", "chunkID", chunkIDString) return nil } + if err != nil { + status = statusFailure + } return err }) } diff --git a/pkg/storage/stores/shipper/compactor/retention/retention_test.go b/pkg/storage/stores/shipper/compactor/retention/retention_test.go index 74521fdcb65cf..29f5dd2f55954 100644 --- a/pkg/storage/stores/shipper/compactor/retention/retention_test.go +++ b/pkg/storage/stores/shipper/compactor/retention/retention_test.go @@ -118,7 +118,7 @@ func Test_Retention(t *testing.T) { key := string([]byte(objectKey)) // forces a copy, because this string is only valid within the delete fn. actualDeleted = append(actualDeleted, key) return nil - }), 0) + }), 0, nil) require.NoError(t, err) sweep.Start() defer sweep.Stop() diff --git a/pkg/storage/stores/shipper/compactor/retention/util_test.go b/pkg/storage/stores/shipper/compactor/retention/util_test.go index ebefbfb3d0c5d..a1aa182eb574b 100644 --- a/pkg/storage/stores/shipper/compactor/retention/util_test.go +++ b/pkg/storage/stores/shipper/compactor/retention/util_test.go @@ -12,6 +12,7 @@ import ( cortex_storage "github.com/cortexproject/cortex/pkg/chunk/storage" chunk_util "github.com/cortexproject/cortex/pkg/chunk/util" util_log "github.com/cortexproject/cortex/pkg/util/log" + "github.com/prometheus/client_golang/prometheus" "github.com/prometheus/common/model" "github.com/prometheus/prometheus/pkg/labels" "github.com/stretchr/testify/require" @@ -87,6 +88,8 @@ var ( {"v10", schemaCfg.Configs[1].From.Time, schemaCfg.Configs[1]}, {"v11", schemaCfg.Configs[2].From.Time, schemaCfg.Configs[2]}, } + + sweepMetrics = newSweeperMetrics(prometheus.DefaultRegisterer) ) func newChunkEntry(userID, labels string, from, through model.Time) ChunkEntry { From 3e21016bc30af331ec79d81630fe4196b1c98118 Mon Sep 17 00:00:00 2001 From: Cyril Tovena Date: Thu, 22 Apr 2021 16:04:24 +0200 Subject: [PATCH 24/42] Adding metrics. Signed-off-by: Cyril Tovena --- .../stores/shipper/compactor/compactor.go | 2 +- .../shipper/compactor/retention/metrics.go | 31 +++++++++++++++++++ .../shipper/compactor/retention/retention.go | 22 ++++++++++++- .../compactor/retention/retention_test.go | 3 +- .../shipper/compactor/retention/util_test.go | 1 + 5 files changed, 56 insertions(+), 3 deletions(-) diff --git a/pkg/storage/stores/shipper/compactor/compactor.go b/pkg/storage/stores/shipper/compactor/compactor.go index bcb475bd75424..1581fe886f259 100644 --- a/pkg/storage/stores/shipper/compactor/compactor.go +++ b/pkg/storage/stores/shipper/compactor/compactor.go @@ -91,7 +91,7 @@ func NewCompactor(cfg Config, storageConfig storage.Config, schemaConfig loki_st if err != nil { return nil, err } - marker, err := retention.NewMarker(retentionWorkDir, schemaConfig, prefixedClient, retention.NewExpirationChecker(limits)) + marker, err := retention.NewMarker(retentionWorkDir, schemaConfig, prefixedClient, retention.NewExpirationChecker(limits), r) if err != nil { return nil, err } diff --git a/pkg/storage/stores/shipper/compactor/retention/metrics.go b/pkg/storage/stores/shipper/compactor/retention/metrics.go index 417d8e50fd8fb..a929c80c0189d 100644 --- a/pkg/storage/stores/shipper/compactor/retention/metrics.go +++ b/pkg/storage/stores/shipper/compactor/retention/metrics.go @@ -9,6 +9,10 @@ const ( statusFailure = "failure" statusSuccess = "success" statusNotFound = "notfound" + + tableActionModified = "modified" + tableActionDeleted = "deleted" + tableActionNone = "none" ) type sweeperMetrics struct { @@ -49,3 +53,30 @@ func newSweeperMetrics(r prometheus.Registerer) *sweeperMetrics { }), } } + +type markerMetrics struct { + tableProcessedTotal *prometheus.CounterVec + tableMarksCreatedTotal *prometheus.CounterVec + tableProcessedDurationSeconds *prometheus.HistogramVec +} + +func newMarkerMetrics(r prometheus.Registerer) *markerMetrics { + return &markerMetrics{ + tableProcessedTotal: promauto.With(r).NewCounterVec(prometheus.CounterOpts{ + Namespace: "loki_boltdb_shipper", + Name: "retention_marker_table_processed_total", + Help: "Total amount of table processed per action.", + }, []string{"table", "action"}), + tableMarksCreatedTotal: promauto.With(r).NewCounterVec(prometheus.CounterOpts{ + Namespace: "loki_boltdb_shipper", + Name: "retention_marker_count_total", + Help: "Total count of markers created per table.", + }, []string{"table"}), + tableProcessedDurationSeconds: promauto.With(r).NewHistogramVec(prometheus.HistogramOpts{ + Namespace: "loki_boltdb_shipper", + Name: "retention_marker_table_processed_duration_seconds", + Help: "Time (in seconds) spent in marking table for chunks to delete", + Buckets: []float64{1, 2.5, 5, 10, 20, 40, 90, 360, 600, 1800}, + }, []string{"table", "status"}), + } +} diff --git a/pkg/storage/stores/shipper/compactor/retention/retention.go b/pkg/storage/stores/shipper/compactor/retention/retention.go index 9b82b056e6e8f..472be6658f412 100644 --- a/pkg/storage/stores/shipper/compactor/retention/retention.go +++ b/pkg/storage/stores/shipper/compactor/retention/retention.go @@ -41,21 +41,37 @@ type Marker struct { config storage.SchemaConfig objectClient chunk.ObjectClient expiration ExpirationChecker + markerMetrics *markerMetrics } -func NewMarker(workingDirectory string, config storage.SchemaConfig, objectClient chunk.ObjectClient, expiration ExpirationChecker) (*Marker, error) { +func NewMarker(workingDirectory string, config storage.SchemaConfig, objectClient chunk.ObjectClient, expiration ExpirationChecker, r prometheus.Registerer) (*Marker, error) { if err := validatePeriods(config); err != nil { return nil, err } + metrics := newMarkerMetrics(r) return &Marker{ workingDirectory: workingDirectory, config: config, objectClient: objectClient, expiration: expiration, + markerMetrics: metrics, }, nil } func (t *Marker) MarkTableForDelete(ctx context.Context, tableName string) error { + start := time.Now() + status := statusSuccess + defer func() { + t.markerMetrics.tableProcessedDurationSeconds.WithLabelValues(tableName, status).Observe(time.Since(start).Seconds()) + }() + if err := t.markTable(ctx, tableName); err != nil { + status = statusFailure + return err + } + return nil +} + +func (t *Marker) markTable(ctx context.Context, tableName string) error { objects, err := util.ListDirectory(ctx, tableName, t.objectClient) if err != nil { return err @@ -133,6 +149,7 @@ func (t *Marker) MarkTableForDelete(ctx context.Context, tableName string) error if err != nil { return err } + t.markerMetrics.tableMarksCreatedTotal.WithLabelValues(tableName).Add(float64(markerWriter.Count())) if err := markerWriter.Close(); err != nil { return fmt.Errorf("failed to close marker writer: %w", err) } @@ -143,12 +160,15 @@ func (t *Marker) MarkTableForDelete(ctx context.Context, tableName string) error } // if the index is empty we can delete the index table. if empty { + t.markerMetrics.tableProcessedTotal.WithLabelValues(tableName, tableActionDeleted).Inc() return t.objectClient.DeleteObject(ctx, tableName+delimiter) } // No chunks to delete means no changes to the remote index, we don't need to upload it. if markerWriter.Count() == 0 { + t.markerMetrics.tableProcessedTotal.WithLabelValues(tableName, tableActionNone).Inc() return nil } + t.markerMetrics.tableProcessedTotal.WithLabelValues(tableName, tableActionModified).Inc() return t.uploadDB(ctx, db, tableKey) } diff --git a/pkg/storage/stores/shipper/compactor/retention/retention_test.go b/pkg/storage/stores/shipper/compactor/retention/retention_test.go index 29f5dd2f55954..503b3478135e5 100644 --- a/pkg/storage/stores/shipper/compactor/retention/retention_test.go +++ b/pkg/storage/stores/shipper/compactor/retention/retention_test.go @@ -13,6 +13,7 @@ import ( "github.com/cortexproject/cortex/pkg/chunk" "github.com/cortexproject/cortex/pkg/ingester/client" + "github.com/prometheus/client_golang/prometheus" "github.com/prometheus/common/model" "github.com/prometheus/prometheus/pkg/labels" "github.com/stretchr/testify/assert" @@ -123,7 +124,7 @@ func Test_Retention(t *testing.T) { sweep.Start() defer sweep.Stop() - marker, err := NewMarker(workDir, store.schemaCfg, util.NewPrefixedObjectClient(store.objectClient, "index/"), expiration) + marker, err := NewMarker(workDir, store.schemaCfg, util.NewPrefixedObjectClient(store.objectClient, "index/"), expiration, prometheus.NewRegistry()) require.NoError(t, err) for _, table := range store.indexTables() { table.Close() diff --git a/pkg/storage/stores/shipper/compactor/retention/util_test.go b/pkg/storage/stores/shipper/compactor/retention/util_test.go index a1aa182eb574b..5684726c2fa98 100644 --- a/pkg/storage/stores/shipper/compactor/retention/util_test.go +++ b/pkg/storage/stores/shipper/compactor/retention/util_test.go @@ -90,6 +90,7 @@ var ( } sweepMetrics = newSweeperMetrics(prometheus.DefaultRegisterer) + markMetrics = newMarkerMetrics(prometheus.DefaultRegisterer) ) func newChunkEntry(userID, labels string, from, through model.Time) ChunkEntry { From f66e6be73658515efbc4c114ced93d9eb649fb33 Mon Sep 17 00:00:00 2001 From: Cyril Tovena Date: Thu, 22 Apr 2021 16:18:46 +0200 Subject: [PATCH 25/42] Improve benchmark. Signed-off-by: Cyril Tovena --- .../compactor/retention/iterator_test.go | 28 ++++++++++--------- .../shipper/compactor/retention/util_test.go | 4 +-- 2 files changed, 17 insertions(+), 15 deletions(-) diff --git a/pkg/storage/stores/shipper/compactor/retention/iterator_test.go b/pkg/storage/stores/shipper/compactor/retention/iterator_test.go index 89de74739d4ca..972c7c7151657 100644 --- a/pkg/storage/stores/shipper/compactor/retention/iterator_test.go +++ b/pkg/storage/stores/shipper/compactor/retention/iterator_test.go @@ -3,16 +3,14 @@ package retention import ( "context" "errors" + "fmt" "testing" "time" "github.com/cortexproject/cortex/pkg/chunk" - "github.com/prometheus/common/model" "github.com/prometheus/prometheus/pkg/labels" "github.com/stretchr/testify/require" "go.etcd.io/bbolt" - - shipper_util "github.com/grafana/loki/pkg/storage/stores/shipper/util" ) func Test_ChunkIterator(t *testing.T) { @@ -150,21 +148,25 @@ func entryFromChunk(c chunk.Chunk) ChunkEntry { var chunkEntry ChunkEntry func Benchmark_ChunkIterator(b *testing.B) { + store := newTestStore(b) + for i := 0; i < 100; i++ { + require.NoError(b, store.Put(context.TODO(), + []chunk.Chunk{ + createChunk(b, "1", + labels.Labels{labels.Label{Name: "foo", Value: "bar"}, labels.Label{Name: "i", Value: fmt.Sprintf("%d", i)}}, + allSchemas[0].from, allSchemas[0].from.Add(1*time.Hour)), + }, + )) + } + store.Stop() b.ReportAllocs() + b.ResetTimer() - db, err := shipper_util.SafeOpenBoltdbFile("/Users/ctovena/Downloads/index_loki_ops_index_18669_compactor-1617841099") - require.NoError(b, err) - t, err := time.Parse("2006-01-02", "2020-07-31") - require.NoError(b, err) var total int64 - db.Update(func(tx *bbolt.Tx) error { + _ = store.indexTables()[0].Update(func(tx *bbolt.Tx) error { bucket := tx.Bucket(bucketName) for n := 0; n < b.N; n++ { - it, err := newChunkIndexIterator(bucket, chunk.PeriodConfig{ - From: chunk.DayTime{Time: model.TimeFromUnix(t.Unix())}, - Schema: "v11", - RowShards: 16, - }) + it, err := newChunkIndexIterator(bucket, allSchemas[0].config) require.NoError(b, err) for it.Next() { chunkEntry = it.Entry() diff --git a/pkg/storage/stores/shipper/compactor/retention/util_test.go b/pkg/storage/stores/shipper/compactor/retention/util_test.go index 5684726c2fa98..98f165e9a12ae 100644 --- a/pkg/storage/stores/shipper/compactor/retention/util_test.go +++ b/pkg/storage/stores/shipper/compactor/retention/util_test.go @@ -115,7 +115,7 @@ type testStore struct { objectClient chunk.ObjectClient indexDir, chunkDir string schemaCfg storage.SchemaConfig - t *testing.T + t testing.TB limits cortex_storage.StoreLimits } @@ -166,7 +166,7 @@ func (t *testStore) open() { t.Store = store } -func newTestStore(t *testing.T) *testStore { +func newTestStore(t testing.TB) *testStore { t.Helper() cfg := &ww.Config{} require.Nil(t, cfg.LogLevel.Set("debug")) From 520fad5d63c841bcee0bc5e51c6e8a5b19e1eae2 Mon Sep 17 00:00:00 2001 From: Cyril Tovena Date: Thu, 22 Apr 2021 16:36:46 +0200 Subject: [PATCH 26/42] Fixes issue. Signed-off-by: Cyril Tovena --- pkg/loki/runtime_config_test.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pkg/loki/runtime_config_test.go b/pkg/loki/runtime_config_test.go index b5b5cf7ef22de..e8c9fdee0ae37 100644 --- a/pkg/loki/runtime_config_test.go +++ b/pkg/loki/runtime_config_test.go @@ -10,7 +10,7 @@ import ( "time" "github.com/cortexproject/cortex/pkg/util/runtimeconfig" - "github.com/grafana/loki/pkg/util/validation" + "github.com/grafana/loki/pkg/validation" "github.com/prometheus/client_golang/prometheus" "github.com/prometheus/prometheus/pkg/labels" "github.com/stretchr/testify/require" From f644e9bfb1b8a0a046f691d9b3fde473bf73ca22 Mon Sep 17 00:00:00 2001 From: Cyril Tovena Date: Thu, 22 Apr 2021 16:41:41 +0200 Subject: [PATCH 27/42] lint code. Signed-off-by: Cyril Tovena --- pkg/loki/runtime_config_test.go | 3 ++- pkg/storage/stores/shipper/compactor/compactor_test.go | 2 ++ pkg/storage/stores/shipper/compactor/retention/util_test.go | 1 - 3 files changed, 4 insertions(+), 2 deletions(-) diff --git a/pkg/loki/runtime_config_test.go b/pkg/loki/runtime_config_test.go index e8c9fdee0ae37..0ed35de32ffea 100644 --- a/pkg/loki/runtime_config_test.go +++ b/pkg/loki/runtime_config_test.go @@ -10,10 +10,11 @@ import ( "time" "github.com/cortexproject/cortex/pkg/util/runtimeconfig" - "github.com/grafana/loki/pkg/validation" "github.com/prometheus/client_golang/prometheus" "github.com/prometheus/prometheus/pkg/labels" "github.com/stretchr/testify/require" + + "github.com/grafana/loki/pkg/validation" ) func Test_LoadRetentionRules(t *testing.T) { diff --git a/pkg/storage/stores/shipper/compactor/compactor_test.go b/pkg/storage/stores/shipper/compactor/compactor_test.go index f291a0fff7d21..7805240096b6a 100644 --- a/pkg/storage/stores/shipper/compactor/compactor_test.go +++ b/pkg/storage/stores/shipper/compactor/compactor_test.go @@ -20,6 +20,8 @@ func TestIsDefaults(t *testing.T) { {&Config{ SharedStoreKeyPrefix: "index/", CompactionInterval: 2 * time.Hour, + RetentionInterval: 10 * time.Minute, + RetentionDeleteDelay: 2 * time.Hour, }, true}, } { t.Run(fmt.Sprint(i), func(t *testing.T) { diff --git a/pkg/storage/stores/shipper/compactor/retention/util_test.go b/pkg/storage/stores/shipper/compactor/retention/util_test.go index 089545beb8cf0..3ca7d33901dc8 100644 --- a/pkg/storage/stores/shipper/compactor/retention/util_test.go +++ b/pkg/storage/stores/shipper/compactor/retention/util_test.go @@ -90,7 +90,6 @@ var ( } sweepMetrics = newSweeperMetrics(prometheus.DefaultRegisterer) - markMetrics = newMarkerMetrics(prometheus.DefaultRegisterer) ) func newChunkEntry(userID, labels string, from, through model.Time) ChunkEntry { From a6cceaf5108635f1bf3f1ac1611bc748cc6bef0a Mon Sep 17 00:00:00 2001 From: Cyril Tovena Date: Fri, 23 Apr 2021 10:20:46 +0200 Subject: [PATCH 28/42] more logs. Signed-off-by: Cyril Tovena --- pkg/storage/stores/shipper/compactor/retention/retention.go | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/pkg/storage/stores/shipper/compactor/retention/retention.go b/pkg/storage/stores/shipper/compactor/retention/retention.go index 472be6658f412..5efad894ab902 100644 --- a/pkg/storage/stores/shipper/compactor/retention/retention.go +++ b/pkg/storage/stores/shipper/compactor/retention/retention.go @@ -99,16 +99,20 @@ func (t *Marker) markTable(ctx context.Context, tableName string) error { if err != nil { return err } + level.Debug(util_log.Logger).Log("msg", "table dir", "dir", tableDirectory) downloadAt := filepath.Join(tableDirectory, tableKey) + level.Debug(util_log.Logger).Log("msg", "Downloading", "key", tableKey, "at", downloadAt) err = shipper_util.GetFileFromStorage(ctx, t.objectClient, tableKey, downloadAt) if err != nil { + level.Warn(util_log.Logger).Log("msg", "failed to download table", "err", err, "path", downloadAt, "tableKey", tableKey) return err } db, err := shipper_util.SafeOpenBoltdbFile(downloadAt) if err != nil { + level.Warn(util_log.Logger).Log("msg", "failed to open db", "err", err, "path", downloadAt) return err } From 9a8f43c193c90c5a362b0faeb93079039efd97a8 Mon Sep 17 00:00:00 2001 From: Cyril Tovena Date: Fri, 23 Apr 2021 11:07:41 +0200 Subject: [PATCH 29/42] Save files without using table key Signed-off-by: Cyril Tovena --- pkg/storage/stores/shipper/compactor/retention/retention.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pkg/storage/stores/shipper/compactor/retention/retention.go b/pkg/storage/stores/shipper/compactor/retention/retention.go index 5efad894ab902..c997b2accc54c 100644 --- a/pkg/storage/stores/shipper/compactor/retention/retention.go +++ b/pkg/storage/stores/shipper/compactor/retention/retention.go @@ -101,7 +101,7 @@ func (t *Marker) markTable(ctx context.Context, tableName string) error { } level.Debug(util_log.Logger).Log("msg", "table dir", "dir", tableDirectory) - downloadAt := filepath.Join(tableDirectory, tableKey) + downloadAt := filepath.Join(tableDirectory, fmt.Sprintf("retention-%d", time.Now().UnixNano())) level.Debug(util_log.Logger).Log("msg", "Downloading", "key", tableKey, "at", downloadAt) err = shipper_util.GetFileFromStorage(ctx, t.objectClient, tableKey, downloadAt) From 74cb42bb56477f09c7d34b7ec4fdf8159bf7a9f9 Mon Sep 17 00:00:00 2001 From: Cyril Tovena Date: Fri, 23 Apr 2021 17:52:28 +0200 Subject: [PATCH 30/42] Improve logging and ability to use more goroutines. Signed-off-by: Cyril Tovena --- .../stores/shipper/compactor/compactor.go | 20 +++++++++++-------- .../shipper/compactor/retention/marker.go | 2 ++ .../shipper/compactor/retention/retention.go | 17 ++++++++-------- .../compactor/retention/retention_test.go | 2 +- 4 files changed, 24 insertions(+), 17 deletions(-) diff --git a/pkg/storage/stores/shipper/compactor/compactor.go b/pkg/storage/stores/shipper/compactor/compactor.go index 1581fe886f259..17c02a93411f6 100644 --- a/pkg/storage/stores/shipper/compactor/compactor.go +++ b/pkg/storage/stores/shipper/compactor/compactor.go @@ -28,13 +28,14 @@ import ( const delimiter = "/" type Config struct { - WorkingDirectory string `yaml:"working_directory"` - SharedStoreType string `yaml:"shared_store"` - SharedStoreKeyPrefix string `yaml:"shared_store_key_prefix"` - CompactionInterval time.Duration `yaml:"compaction_interval"` - RetentionEnabled bool `yaml:"retention_enabled"` - RetentionInterval time.Duration `yaml:"retention_interval"` - RetentionDeleteDelay time.Duration `yaml:"retention_delete_delay"` + WorkingDirectory string `yaml:"working_directory"` + SharedStoreType string `yaml:"shared_store"` + SharedStoreKeyPrefix string `yaml:"shared_store_key_prefix"` + CompactionInterval time.Duration `yaml:"compaction_interval"` + RetentionEnabled bool `yaml:"retention_enabled"` + RetentionInterval time.Duration `yaml:"retention_interval"` + RetentionDeleteDelay time.Duration `yaml:"retention_delete_delay"` + RetentionDeleteWorkCount int `yaml:"retention_delete_worker_count"` } // RegisterFlags registers flags. @@ -46,6 +47,7 @@ func (cfg *Config) RegisterFlags(f *flag.FlagSet) { f.DurationVar(&cfg.RetentionInterval, "boltdb.shipper.compactor.retention-interval", 10*time.Minute, "Interval at which to re-run the retention operation.") f.DurationVar(&cfg.RetentionDeleteDelay, "boltdb.shipper.compactor.retention-delete-delay", 2*time.Hour, "Delay after which chunks will be fully deleted during retention.") f.BoolVar(&cfg.RetentionEnabled, "boltdb.shipper.compactor.retention-enabled", false, "Activate custom (per-stream,per-tenant) retention.") + f.IntVar(&cfg.RetentionDeleteWorkCount, "boltdb.shipper.compactor.retention-delete-worker-count", 150, "The total amount of worker to use to delete chunks.") } func (cfg *Config) IsDefaults() bool { @@ -87,7 +89,7 @@ func NewCompactor(cfg Config, storageConfig storage.Config, schemaConfig loki_st retentionWorkDir := filepath.Join(cfg.WorkingDirectory, "retention") - sweeper, err := retention.NewSweeper(retentionWorkDir, retention.NewDeleteClient(objectClient), cfg.RetentionDeleteDelay, r) + sweeper, err := retention.NewSweeper(retentionWorkDir, retention.NewDeleteClient(objectClient), cfg.RetentionDeleteDelay, cfg.RetentionDeleteWorkCount, r) if err != nil { return nil, err } @@ -223,12 +225,14 @@ func (c *Compactor) RunRetention(ctx context.Context) error { start := time.Now() defer func() { + level.Debug(util_log.Logger).Log("msg", "finished to processing retention on all all tables", "status", status, "duration", time.Since(start)) c.metrics.retentionOperationTotal.WithLabelValues(status).Inc() if status == statusSuccess { c.metrics.retentionOperationDurationSeconds.Set(time.Since(start).Seconds()) c.metrics.retentionOperationLastSuccess.SetToCurrentTime() } }() + level.Debug(util_log.Logger).Log("msg", "starting to processing retention on all all tables") _, dirs, err := c.objectClient.List(ctx, "", delimiter) if err != nil { diff --git a/pkg/storage/stores/shipper/compactor/retention/marker.go b/pkg/storage/stores/shipper/compactor/retention/marker.go index bfeb879d2af18..384a94da70b5f 100644 --- a/pkg/storage/stores/shipper/compactor/retention/marker.go +++ b/pkg/storage/stores/shipper/compactor/retention/marker.go @@ -157,6 +157,7 @@ func (r *markerProcessor) Start(deleteFunc func(ctx context.Context, chunkId []b level.Info(util_log.Logger).Log("msg", "No marks file found") } for i, path := range paths { + level.Debug(util_log.Logger).Log("msg", "processing mark file:") if r.ctx.Err() != nil { return } @@ -233,6 +234,7 @@ func processKey(ctx context.Context, key *bytes.Buffer, db *bbolt.DB, deleteFunc return err } // we don't use a batch because it would force us to copy the key. + // but we most likely want to do batch in the future. return db.Update(func(tx *bbolt.Tx) error { b := tx.Bucket(chunkBucket) if b == nil { diff --git a/pkg/storage/stores/shipper/compactor/retention/retention.go b/pkg/storage/stores/shipper/compactor/retention/retention.go index c997b2accc54c..6e897a9dc74ec 100644 --- a/pkg/storage/stores/shipper/compactor/retention/retention.go +++ b/pkg/storage/stores/shipper/compactor/retention/retention.go @@ -30,10 +30,9 @@ var ( ) const ( - logMetricName = "logs" - delimiter = "/" - markersFolder = "markers" - deletionWorkerCount = 10 + logMetricName = "logs" + delimiter = "/" + markersFolder = "markers" ) type Marker struct { @@ -63,7 +62,10 @@ func (t *Marker) MarkTableForDelete(ctx context.Context, tableName string) error status := statusSuccess defer func() { t.markerMetrics.tableProcessedDurationSeconds.WithLabelValues(tableName, status).Observe(time.Since(start).Seconds()) + level.Debug(util_log.Logger).Log("msg", "finished to process table", "table", tableName, "duration", time.Since(start)) }() + level.Debug(util_log.Logger).Log("msg", "starting to process table", "table", tableName) + if err := t.markTable(ctx, tableName); err != nil { status = statusFailure return err @@ -99,10 +101,8 @@ func (t *Marker) markTable(ctx context.Context, tableName string) error { if err != nil { return err } - level.Debug(util_log.Logger).Log("msg", "table dir", "dir", tableDirectory) downloadAt := filepath.Join(tableDirectory, fmt.Sprintf("retention-%d", time.Now().UnixNano())) - level.Debug(util_log.Logger).Log("msg", "Downloading", "key", tableKey, "at", downloadAt) err = shipper_util.GetFileFromStorage(ctx, t.objectClient, tableKey, downloadAt) if err != nil { @@ -254,9 +254,9 @@ type Sweeper struct { sweeperMetrics *sweeperMetrics } -func NewSweeper(workingDir string, deleteClient DeleteClient, minAgeDelete time.Duration, r prometheus.Registerer) (*Sweeper, error) { +func NewSweeper(workingDir string, deleteClient DeleteClient, minAgeDelete time.Duration, deleteWorkerCount int, r prometheus.Registerer) (*Sweeper, error) { m := newSweeperMetrics(r) - p, err := newMarkerStorageReader(workingDir, deletionWorkerCount, minAgeDelete, m) + p, err := newMarkerStorageReader(workingDir, deleteWorkerCount, minAgeDelete, m) if err != nil { return nil, err } @@ -283,6 +283,7 @@ func (s *Sweeper) Start() { return nil } if err != nil { + level.Error(util_log.Logger).Log("msg", "error deleting chunk", "chunkID", chunkIDString, "err", err) status = statusFailure } return err diff --git a/pkg/storage/stores/shipper/compactor/retention/retention_test.go b/pkg/storage/stores/shipper/compactor/retention/retention_test.go index 9e199ab361ebb..41fe28f70189d 100644 --- a/pkg/storage/stores/shipper/compactor/retention/retention_test.go +++ b/pkg/storage/stores/shipper/compactor/retention/retention_test.go @@ -119,7 +119,7 @@ func Test_Retention(t *testing.T) { key := string([]byte(objectKey)) // forces a copy, because this string is only valid within the delete fn. actualDeleted = append(actualDeleted, key) return nil - }), 0, nil) + }), 10, 0, nil) require.NoError(t, err) sweep.Start() defer sweep.Stop() From 78a4673c995da81334490ed6c5a402bf60a687b0 Mon Sep 17 00:00:00 2001 From: Cyril Tovena Date: Mon, 26 Apr 2021 09:09:53 +0200 Subject: [PATCH 31/42] Removes duplicate metrics since histogram contains total too. Signed-off-by: Cyril Tovena --- pkg/storage/stores/shipper/compactor/retention/retention.go | 1 - 1 file changed, 1 deletion(-) diff --git a/pkg/storage/stores/shipper/compactor/retention/retention.go b/pkg/storage/stores/shipper/compactor/retention/retention.go index 6e897a9dc74ec..4cf7dfde6db9f 100644 --- a/pkg/storage/stores/shipper/compactor/retention/retention.go +++ b/pkg/storage/stores/shipper/compactor/retention/retention.go @@ -272,7 +272,6 @@ func (s *Sweeper) Start() { status := statusSuccess start := time.Now() defer func() { - s.sweeperMetrics.deleteChunkTotal.WithLabelValues(status).Inc() s.sweeperMetrics.deleteChunkDurationSeconds.WithLabelValues(status).Observe(time.Since(start).Seconds()) }() chunkIDString := unsafeGetString(chunkId) From 6091368caee56b807af998babb7bcbebef078047 Mon Sep 17 00:00:00 2001 From: Cyril Tovena Date: Mon, 26 Apr 2021 09:21:54 +0200 Subject: [PATCH 32/42] Add more logs. Signed-off-by: Cyril Tovena --- pkg/storage/stores/shipper/compactor/retention/marker.go | 5 +++-- pkg/storage/stores/shipper/compactor/retention/metrics.go | 6 ------ 2 files changed, 3 insertions(+), 8 deletions(-) diff --git a/pkg/storage/stores/shipper/compactor/retention/marker.go b/pkg/storage/stores/shipper/compactor/retention/marker.go index 384a94da70b5f..7e9f35168047f 100644 --- a/pkg/storage/stores/shipper/compactor/retention/marker.go +++ b/pkg/storage/stores/shipper/compactor/retention/marker.go @@ -129,6 +129,7 @@ func newMarkerStorageReader(workingDir string, maxParallelism int, minAgeFile ti } func (r *markerProcessor) Start(deleteFunc func(ctx context.Context, chunkId []byte) error) { + level.Info(util_log.Logger).Log("msg", "mark processor started", "workers", r.maxParallelism, "delay", r.minAgeFile) r.wg.Wait() // only one start at a time. r.wg.Add(1) go func() { @@ -154,10 +155,10 @@ func (r *markerProcessor) Start(deleteFunc func(ctx context.Context, chunkId []b } r.sweeperMetrics.markerFilesCurrent.Set(float64(len(paths))) if len(paths) == 0 { - level.Info(util_log.Logger).Log("msg", "No marks file found") + level.Info(util_log.Logger).Log("msg", "no marks file found") } for i, path := range paths { - level.Debug(util_log.Logger).Log("msg", "processing mark file:") + level.Debug(util_log.Logger).Log("msg", "processing mark file", "path", path) if r.ctx.Err() != nil { return } diff --git a/pkg/storage/stores/shipper/compactor/retention/metrics.go b/pkg/storage/stores/shipper/compactor/retention/metrics.go index a929c80c0189d..6734b9b24a79a 100644 --- a/pkg/storage/stores/shipper/compactor/retention/metrics.go +++ b/pkg/storage/stores/shipper/compactor/retention/metrics.go @@ -16,7 +16,6 @@ const ( ) type sweeperMetrics struct { - deleteChunkTotal *prometheus.CounterVec deleteChunkDurationSeconds *prometheus.HistogramVec markerFileCurrentTime prometheus.Gauge markerFilesCurrent prometheus.Gauge @@ -25,11 +24,6 @@ type sweeperMetrics struct { func newSweeperMetrics(r prometheus.Registerer) *sweeperMetrics { return &sweeperMetrics{ - deleteChunkTotal: promauto.With(r).NewCounterVec(prometheus.CounterOpts{ - Namespace: "loki_boltdb_shipper", - Name: "retention_sweeper_chunk_deleted_total", - Help: "Total number of chunks deleted by retention", - }, []string{"status"}), deleteChunkDurationSeconds: promauto.With(r).NewHistogramVec(prometheus.HistogramOpts{ Namespace: "loki_boltdb_shipper", Name: "retention_sweeper_chunk_deleted_duration_seconds", From 7086c12e85b3d86ae00698e8a06a38cd2e3d65ff Mon Sep 17 00:00:00 2001 From: Cyril Tovena Date: Mon, 26 Apr 2021 10:22:36 +0200 Subject: [PATCH 33/42] Fixes a deadlock bug when too many workers are trying to update the mark file. Signed-off-by: Cyril Tovena --- .../shipper/compactor/retention/marker.go | 2 +- .../compactor/retention/marker_test.go | 31 ++++++++++++++++--- 2 files changed, 28 insertions(+), 5 deletions(-) diff --git a/pkg/storage/stores/shipper/compactor/retention/marker.go b/pkg/storage/stores/shipper/compactor/retention/marker.go index 7e9f35168047f..2a6db09533bc3 100644 --- a/pkg/storage/stores/shipper/compactor/retention/marker.go +++ b/pkg/storage/stores/shipper/compactor/retention/marker.go @@ -236,7 +236,7 @@ func processKey(ctx context.Context, key *bytes.Buffer, db *bbolt.DB, deleteFunc } // we don't use a batch because it would force us to copy the key. // but we most likely want to do batch in the future. - return db.Update(func(tx *bbolt.Tx) error { + return db.Batch(func(tx *bbolt.Tx) error { b := tx.Bucket(chunkBucket) if b == nil { return nil diff --git a/pkg/storage/stores/shipper/compactor/retention/marker_test.go b/pkg/storage/stores/shipper/compactor/retention/marker_test.go index 9fab3534928ac..1b2f9c861f48e 100644 --- a/pkg/storage/stores/shipper/compactor/retention/marker_test.go +++ b/pkg/storage/stores/shipper/compactor/retention/marker_test.go @@ -15,11 +15,11 @@ import ( "github.com/stretchr/testify/require" ) -func initAndFeedMarkerProcessor(t *testing.T) *markerProcessor { +func initAndFeedMarkerProcessor(t *testing.T, deleteWorkerCount int) *markerProcessor { t.Helper() minListMarkDelay = time.Second dir := t.TempDir() - p, err := newMarkerStorageReader(dir, 5, time.Second, sweepMetrics) + p, err := newMarkerStorageReader(dir, deleteWorkerCount, time.Second, sweepMetrics) require.NoError(t, err) go func() { w, err := NewMarkerStorageWriter(dir) @@ -37,8 +37,31 @@ func initAndFeedMarkerProcessor(t *testing.T) *markerProcessor { return p } +func Test_marlkerProcessor_Deadlock(t *testing.T) { + minListMarkDelay = time.Second + dir := t.TempDir() + p, err := newMarkerStorageReader(dir, 150, 0, sweepMetrics) + require.NoError(t, err) + w, err := NewMarkerStorageWriter(dir) + require.NoError(t, err) + for i := 0; i <= 200; i++ { + require.NoError(t, w.Put([]byte(fmt.Sprintf("%d", i)))) + } + require.NoError(t, w.Close()) + defer p.Stop() + p.Start(func(ctx context.Context, id []byte) error { + return nil + }) + + require.Eventually(t, func() bool { + path, _, err := p.availablePath() + require.NoError(t, err) + return len(path) == 0 + }, 20*time.Second, 100*time.Millisecond) +} + func Test_markerProcessor_StartRetryKey(t *testing.T) { - p := initAndFeedMarkerProcessor(t) + p := initAndFeedMarkerProcessor(t, 5) defer p.Stop() counts := map[string]int{} l := sync.Mutex{} @@ -67,7 +90,7 @@ func Test_markerProcessor_StartRetryKey(t *testing.T) { } func Test_markerProcessor_StartDeleteOnSuccess(t *testing.T) { - p := initAndFeedMarkerProcessor(t) + p := initAndFeedMarkerProcessor(t, 5) defer p.Stop() counts := map[string]int{} l := sync.Mutex{} From ceb3b8f307691e32aa8605e5b2c43e2ca4ac5928 Mon Sep 17 00:00:00 2001 From: Cyril Tovena Date: Mon, 26 Apr 2021 12:09:21 +0200 Subject: [PATCH 34/42] Fixes a deadlock when reading and updating db at the same time. Signed-off-by: Cyril Tovena --- .../stores/shipper/compactor/compactor.go | 2 +- .../shipper/compactor/retention/marker.go | 38 ++++++++++++++++--- .../compactor/retention/marker_test.go | 22 +++++------ .../shipper/compactor/retention/retention.go | 2 +- .../shipper/compactor/retention/util.go | 28 ++++++++++++++ 5 files changed, 72 insertions(+), 20 deletions(-) diff --git a/pkg/storage/stores/shipper/compactor/compactor.go b/pkg/storage/stores/shipper/compactor/compactor.go index 17c02a93411f6..fdd8ead7f889b 100644 --- a/pkg/storage/stores/shipper/compactor/compactor.go +++ b/pkg/storage/stores/shipper/compactor/compactor.go @@ -89,7 +89,7 @@ func NewCompactor(cfg Config, storageConfig storage.Config, schemaConfig loki_st retentionWorkDir := filepath.Join(cfg.WorkingDirectory, "retention") - sweeper, err := retention.NewSweeper(retentionWorkDir, retention.NewDeleteClient(objectClient), cfg.RetentionDeleteDelay, cfg.RetentionDeleteWorkCount, r) + sweeper, err := retention.NewSweeper(retentionWorkDir, retention.NewDeleteClient(objectClient), cfg.RetentionDeleteWorkCount, cfg.RetentionDeleteDelay, r) if err != nil { return nil, err } diff --git a/pkg/storage/stores/shipper/compactor/retention/marker.go b/pkg/storage/stores/shipper/compactor/retention/marker.go index 2a6db09533bc3..2eb708b5e3715 100644 --- a/pkg/storage/stores/shipper/compactor/retention/marker.go +++ b/pkg/storage/stores/shipper/compactor/retention/marker.go @@ -5,6 +5,7 @@ import ( "context" "fmt" "io/fs" + "io/ioutil" "os" "path/filepath" "sort" @@ -182,28 +183,55 @@ func (r *markerProcessor) processPath(path string, deleteFunc func(ctx context.C wg sync.WaitGroup queue = make(chan *bytes.Buffer) ) - db, err := shipper_util.SafeOpenBoltdbFile(path) + // we use a copy to view the file so that we can read and update at the same time. + viewFile, err := ioutil.TempFile("/tmp/", "marker-view-") + if err != nil { + return err + } + if err := viewFile.Close(); err != nil { + return fmt.Errorf("failed to close view file: %w", err) + } + defer func() { + if err := os.Remove(viewFile.Name()); err != nil { + level.Warn(util_log.Logger).Log("msg", "failed to delete view file", "file", viewFile.Name(), "err", err) + } + }() + if _, err := copyFile(path, viewFile.Name()); err != nil { + return fmt.Errorf("failed to copy view file: %w", err) + } + dbView, err := shipper_util.SafeOpenBoltdbFile(viewFile.Name()) + if err != nil { + return err + } + defer func() { + if err := dbView.Close(); err != nil { + level.Warn(util_log.Logger).Log("msg", "failed to close db view", "err", err) + } + }() + dbUpdate, err := shipper_util.SafeOpenBoltdbFile(path) if err != nil { return err } defer func() { close(queue) wg.Wait() - db.Close() + if err := dbUpdate.Close(); err != nil { + level.Warn(util_log.Logger).Log("msg", "failed to close db", "err", err) + } }() for i := 0; i < r.maxParallelism; i++ { wg.Add(1) go func() { defer wg.Done() for key := range queue { - if err := processKey(r.ctx, key, db, deleteFunc); err != nil { + if err := processKey(r.ctx, key, dbUpdate, deleteFunc); err != nil { level.Warn(util_log.Logger).Log("msg", "failed to delete key", "key", key.String(), "err", err) } putKeyBuffer(key) } }() } - if err := db.View(func(tx *bbolt.Tx) error { + if err := dbView.View(func(tx *bbolt.Tx) error { b := tx.Bucket(chunkBucket) if b == nil { return nil @@ -234,8 +262,6 @@ func processKey(ctx context.Context, key *bytes.Buffer, db *bbolt.DB, deleteFunc if err := deleteFunc(ctx, keyData); err != nil { return err } - // we don't use a batch because it would force us to copy the key. - // but we most likely want to do batch in the future. return db.Batch(func(tx *bbolt.Tx) error { b := tx.Bucket(chunkBucket) if b == nil { diff --git a/pkg/storage/stores/shipper/compactor/retention/marker_test.go b/pkg/storage/stores/shipper/compactor/retention/marker_test.go index 1b2f9c861f48e..8cfeacdfd36d8 100644 --- a/pkg/storage/stores/shipper/compactor/retention/marker_test.go +++ b/pkg/storage/stores/shipper/compactor/retention/marker_test.go @@ -38,26 +38,24 @@ func initAndFeedMarkerProcessor(t *testing.T, deleteWorkerCount int) *markerProc } func Test_marlkerProcessor_Deadlock(t *testing.T) { - minListMarkDelay = time.Second dir := t.TempDir() p, err := newMarkerStorageReader(dir, 150, 0, sweepMetrics) require.NoError(t, err) w, err := NewMarkerStorageWriter(dir) require.NoError(t, err) - for i := 0; i <= 200; i++ { + for i := 0; i <= 2000; i++ { require.NoError(t, w.Put([]byte(fmt.Sprintf("%d", i)))) } require.NoError(t, w.Close()) - defer p.Stop() - p.Start(func(ctx context.Context, id []byte) error { - return nil - }) - - require.Eventually(t, func() bool { - path, _, err := p.availablePath() - require.NoError(t, err) - return len(path) == 0 - }, 20*time.Second, 100*time.Millisecond) + paths, _, err := p.availablePath() + require.NoError(t, err) + for _, path := range paths { + require.NoError(t, p.processPath(path, func(ctx context.Context, chunkId []byte) error { return nil })) + require.NoError(t, p.deleteEmptyMarks(path)) + } + paths, _, err = p.availablePath() + require.NoError(t, err) + require.Len(t, paths, 0) } func Test_markerProcessor_StartRetryKey(t *testing.T) { diff --git a/pkg/storage/stores/shipper/compactor/retention/retention.go b/pkg/storage/stores/shipper/compactor/retention/retention.go index 4cf7dfde6db9f..bb3caeb478df8 100644 --- a/pkg/storage/stores/shipper/compactor/retention/retention.go +++ b/pkg/storage/stores/shipper/compactor/retention/retention.go @@ -254,7 +254,7 @@ type Sweeper struct { sweeperMetrics *sweeperMetrics } -func NewSweeper(workingDir string, deleteClient DeleteClient, minAgeDelete time.Duration, deleteWorkerCount int, r prometheus.Registerer) (*Sweeper, error) { +func NewSweeper(workingDir string, deleteClient DeleteClient, deleteWorkerCount int, minAgeDelete time.Duration, r prometheus.Registerer) (*Sweeper, error) { m := newSweeperMetrics(r) p, err := newMarkerStorageReader(workingDir, deleteWorkerCount, minAgeDelete, m) if err != nil { diff --git a/pkg/storage/stores/shipper/compactor/retention/util.go b/pkg/storage/stores/shipper/compactor/retention/util.go index 9caf47e6342e0..4802cbe9534f2 100644 --- a/pkg/storage/stores/shipper/compactor/retention/util.go +++ b/pkg/storage/stores/shipper/compactor/retention/util.go @@ -1,6 +1,9 @@ package retention import ( + "fmt" + "io" + "os" "reflect" "unsafe" ) @@ -17,3 +20,28 @@ func unsafeGetBytes(s string) []byte { (*reflect.SliceHeader)(p).Cap = len(s) return buf } + +func copyFile(src, dst string) (int64, error) { + sourceFileStat, err := os.Stat(src) + if err != nil { + return 0, err + } + + if !sourceFileStat.Mode().IsRegular() { + return 0, fmt.Errorf("%s is not a regular file", src) + } + + source, err := os.Open(src) + if err != nil { + return 0, err + } + defer source.Close() + + destination, err := os.Create(dst) + if err != nil { + return 0, err + } + defer destination.Close() + nBytes, err := io.Copy(destination, source) + return nBytes, err +} From 0e9ad6105fa7b0a27cf42e444128188e880df72c Mon Sep 17 00:00:00 2001 From: Cyril Tovena Date: Mon, 26 Apr 2021 14:48:32 +0200 Subject: [PATCH 35/42] Fixes default config test of boltdb. Signed-off-by: Cyril Tovena --- pkg/storage/stores/shipper/compactor/compactor_test.go | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/pkg/storage/stores/shipper/compactor/compactor_test.go b/pkg/storage/stores/shipper/compactor/compactor_test.go index 7805240096b6a..77d79fdc2232a 100644 --- a/pkg/storage/stores/shipper/compactor/compactor_test.go +++ b/pkg/storage/stores/shipper/compactor/compactor_test.go @@ -18,10 +18,11 @@ func TestIsDefaults(t *testing.T) { }, false}, {&Config{}, false}, {&Config{ - SharedStoreKeyPrefix: "index/", - CompactionInterval: 2 * time.Hour, - RetentionInterval: 10 * time.Minute, - RetentionDeleteDelay: 2 * time.Hour, + SharedStoreKeyPrefix: "index/", + CompactionInterval: 2 * time.Hour, + RetentionInterval: 10 * time.Minute, + RetentionDeleteDelay: 2 * time.Hour, + RetentionDeleteWorkCount: 150, }, true}, } { t.Run(fmt.Sprint(i), func(t *testing.T) { From 80f8cc469af809c10a8233795560f61f5289cd4a Mon Sep 17 00:00:00 2001 From: Cyril Tovena Date: Mon, 26 Apr 2021 16:49:23 +0200 Subject: [PATCH 36/42] PR Review feedbacks. Signed-off-by: Cyril Tovena --- pkg/storage/stores/shipper/compactor/compactor.go | 4 ++-- .../shipper/compactor/retention/retention.go | 15 ++++++++------- .../shipper/compactor/retention/retention_test.go | 2 +- 3 files changed, 11 insertions(+), 10 deletions(-) diff --git a/pkg/storage/stores/shipper/compactor/compactor.go b/pkg/storage/stores/shipper/compactor/compactor.go index fdd8ead7f889b..977393191431c 100644 --- a/pkg/storage/stores/shipper/compactor/compactor.go +++ b/pkg/storage/stores/shipper/compactor/compactor.go @@ -225,7 +225,7 @@ func (c *Compactor) RunRetention(ctx context.Context) error { start := time.Now() defer func() { - level.Debug(util_log.Logger).Log("msg", "finished to processing retention on all all tables", "status", status, "duration", time.Since(start)) + level.Debug(util_log.Logger).Log("msg", "finished to processing retention on all tables", "status", status, "duration", time.Since(start)) c.metrics.retentionOperationTotal.WithLabelValues(status).Inc() if status == statusSuccess { c.metrics.retentionOperationDurationSeconds.Set(time.Since(start).Seconds()) @@ -248,7 +248,7 @@ func (c *Compactor) RunRetention(ctx context.Context) error { var errs errUtil.MultiError for _, tableName := range tables { - if err := c.tableMarker.MarkTableForDelete(ctx, tableName); err != nil { + if err := c.tableMarker.MarkForDelete(ctx, tableName); err != nil { level.Error(util_log.Logger).Log("msg", "failed to mark table for deletes", "table", tableName, "err", err) errs.Add(err) status = statusFailure diff --git a/pkg/storage/stores/shipper/compactor/retention/retention.go b/pkg/storage/stores/shipper/compactor/retention/retention.go index bb3caeb478df8..95e1a6a188630 100644 --- a/pkg/storage/stores/shipper/compactor/retention/retention.go +++ b/pkg/storage/stores/shipper/compactor/retention/retention.go @@ -57,7 +57,8 @@ func NewMarker(workingDirectory string, config storage.SchemaConfig, objectClien }, nil } -func (t *Marker) MarkTableForDelete(ctx context.Context, tableName string) error { +// MarkForDelete marks all chunks expired for a given table. +func (t *Marker) MarkForDelete(ctx context.Context, tableName string) error { start := time.Now() status := statusSuccess defer func() { @@ -89,10 +90,10 @@ func (t *Marker) markTable(ctx context.Context, tableName string) error { level.Debug(util_log.Logger).Log("msg", "skipping retention for non-compacted table", "name", tableName) return nil } - tableKey := objects[0].Key + objectKey := objects[0].Key - if shipper_util.IsDirectory(tableKey) { - level.Debug(util_log.Logger).Log("msg", "skipping retention no table file found", "key", tableKey) + if shipper_util.IsDirectory(objectKey) { + level.Debug(util_log.Logger).Log("msg", "skipping retention no table file found", "objectKey", objectKey) return nil } @@ -104,9 +105,9 @@ func (t *Marker) markTable(ctx context.Context, tableName string) error { downloadAt := filepath.Join(tableDirectory, fmt.Sprintf("retention-%d", time.Now().UnixNano())) - err = shipper_util.GetFileFromStorage(ctx, t.objectClient, tableKey, downloadAt) + err = shipper_util.GetFileFromStorage(ctx, t.objectClient, objectKey, downloadAt) if err != nil { - level.Warn(util_log.Logger).Log("msg", "failed to download table", "err", err, "path", downloadAt, "tableKey", tableKey) + level.Warn(util_log.Logger).Log("msg", "failed to download table", "err", err, "path", downloadAt, "objectKey", objectKey) return err } @@ -173,7 +174,7 @@ func (t *Marker) markTable(ctx context.Context, tableName string) error { return nil } t.markerMetrics.tableProcessedTotal.WithLabelValues(tableName, tableActionModified).Inc() - return t.uploadDB(ctx, db, tableKey) + return t.uploadDB(ctx, db, objectKey) } func (t *Marker) uploadDB(ctx context.Context, db *bbolt.DB, objectKey string) error { diff --git a/pkg/storage/stores/shipper/compactor/retention/retention_test.go b/pkg/storage/stores/shipper/compactor/retention/retention_test.go index 41fe28f70189d..4566ee0525d53 100644 --- a/pkg/storage/stores/shipper/compactor/retention/retention_test.go +++ b/pkg/storage/stores/shipper/compactor/retention/retention_test.go @@ -128,7 +128,7 @@ func Test_Retention(t *testing.T) { require.NoError(t, err) for _, table := range store.indexTables() { table.Close() - require.NoError(t, marker.MarkTableForDelete(context.Background(), table.name)) + require.NoError(t, marker.MarkForDelete(context.Background(), table.name)) } // assert using the store again. From 456bb5227d36b3dca3356d99bc837582c3892e23 Mon Sep 17 00:00:00 2001 From: Cyril Tovena Date: Mon, 26 Apr 2021 17:01:34 +0200 Subject: [PATCH 37/42] Remove the user of assert to not fail a test if it's incorrect on the first shot. Signed-off-by: Cyril Tovena --- .../stores/shipper/compactor/retention/retention_test.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pkg/storage/stores/shipper/compactor/retention/retention_test.go b/pkg/storage/stores/shipper/compactor/retention/retention_test.go index 4566ee0525d53..e2670665a3904 100644 --- a/pkg/storage/stores/shipper/compactor/retention/retention_test.go +++ b/pkg/storage/stores/shipper/compactor/retention/retention_test.go @@ -145,7 +145,7 @@ func Test_Retention(t *testing.T) { require.Eventually(t, func() bool { lock.Lock() defer lock.Unlock() - return assert.Equal(t, expectDeleted, actualDeleted) + return assert.ObjectsAreEqual(expectDeleted, actualDeleted) }, 10*time.Second, 1*time.Second) } }) From 520d3933fc4eaea9f6d180ec136a9383066311b5 Mon Sep 17 00:00:00 2001 From: Cyril Tovena Date: Tue, 27 Apr 2021 15:38:26 +0200 Subject: [PATCH 38/42] Add experimental notice to the flag documentation Signed-off-by: Cyril Tovena --- pkg/storage/stores/shipper/compactor/compactor.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pkg/storage/stores/shipper/compactor/compactor.go b/pkg/storage/stores/shipper/compactor/compactor.go index 977393191431c..f7a7b5dd94515 100644 --- a/pkg/storage/stores/shipper/compactor/compactor.go +++ b/pkg/storage/stores/shipper/compactor/compactor.go @@ -46,7 +46,7 @@ func (cfg *Config) RegisterFlags(f *flag.FlagSet) { f.DurationVar(&cfg.CompactionInterval, "boltdb.shipper.compactor.compaction-interval", 2*time.Hour, "Interval at which to re-run the compaction operation.") f.DurationVar(&cfg.RetentionInterval, "boltdb.shipper.compactor.retention-interval", 10*time.Minute, "Interval at which to re-run the retention operation.") f.DurationVar(&cfg.RetentionDeleteDelay, "boltdb.shipper.compactor.retention-delete-delay", 2*time.Hour, "Delay after which chunks will be fully deleted during retention.") - f.BoolVar(&cfg.RetentionEnabled, "boltdb.shipper.compactor.retention-enabled", false, "Activate custom (per-stream,per-tenant) retention.") + f.BoolVar(&cfg.RetentionEnabled, "boltdb.shipper.compactor.retention-enabled", false, "(Experimental) Activate custom (per-stream,per-tenant) retention.") f.IntVar(&cfg.RetentionDeleteWorkCount, "boltdb.shipper.compactor.retention-delete-worker-count", 150, "The total amount of worker to use to delete chunks.") } From 9616c44fa76242eceb36226b18c88a3b35ae9079 Mon Sep 17 00:00:00 2001 From: Cyril Tovena Date: Wed, 28 Apr 2021 11:18:02 +0200 Subject: [PATCH 39/42] Fixes empty index detection and table deletion. Signed-off-by: Cyril Tovena --- .../shipper/compactor/retention/retention.go | 11 +++++- .../compactor/retention/retention_test.go | 37 +++++++++++++++++++ .../shipper/compactor/retention/util_test.go | 35 ++++++++++++++---- 3 files changed, 74 insertions(+), 9 deletions(-) diff --git a/pkg/storage/stores/shipper/compactor/retention/retention.go b/pkg/storage/stores/shipper/compactor/retention/retention.go index 95e1a6a188630..b21382b920e89 100644 --- a/pkg/storage/stores/shipper/compactor/retention/retention.go +++ b/pkg/storage/stores/shipper/compactor/retention/retention.go @@ -102,6 +102,11 @@ func (t *Marker) markTable(ctx context.Context, tableName string) error { if err != nil { return err } + defer func() { + if err := os.Remove(tableDirectory); err != nil { + level.Warn(util_log.Logger).Log("msg", "failed to remove temporary table directory", "err", err, "path", tableDirectory) + } + }() downloadAt := filepath.Join(tableDirectory, fmt.Sprintf("retention-%d", time.Now().UnixNano())) @@ -166,6 +171,9 @@ func (t *Marker) markTable(ctx context.Context, tableName string) error { // if the index is empty we can delete the index table. if empty { t.markerMetrics.tableProcessedTotal.WithLabelValues(tableName, tableActionDeleted).Inc() + if err := t.objectClient.DeleteObject(ctx, objectKey); err != nil { + level.Warn(util_log.Logger).Log("msg", "failed to delete empty index table", "err", err, "objectKey", objectKey) + } return t.objectClient.DeleteObject(ctx, tableName+delimiter) } // No chunks to delete means no changes to the remote index, we don't need to upload it. @@ -204,7 +212,7 @@ func (t *Marker) uploadDB(ctx context.Context, db *bbolt.DB, objectKey string) e func markforDelete(marker MarkerStorageWriter, chunkIt ChunkEntryIterator, seriesCleaner SeriesCleaner, expiration ExpirationChecker) (bool, error) { seriesMap := newUserSeriesMap() - var empty bool + empty := true for chunkIt.Next() { if chunkIt.Err() != nil { return false, chunkIt.Err() @@ -218,6 +226,7 @@ func markforDelete(marker MarkerStorageWriter, chunkIt ChunkEntryIterator, serie if err := marker.Put(c.ChunkID); err != nil { return false, err } + continue } empty = false } diff --git a/pkg/storage/stores/shipper/compactor/retention/retention_test.go b/pkg/storage/stores/shipper/compactor/retention/retention_test.go index e2670665a3904..f13f4f9d639e9 100644 --- a/pkg/storage/stores/shipper/compactor/retention/retention_test.go +++ b/pkg/storage/stores/shipper/compactor/retention/retention_test.go @@ -18,6 +18,7 @@ import ( "github.com/prometheus/prometheus/pkg/labels" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" + "go.etcd.io/bbolt" "github.com/grafana/loki/pkg/chunkenc" "github.com/grafana/loki/pkg/logproto" @@ -152,6 +153,42 @@ func Test_Retention(t *testing.T) { } } +type noopWriter struct{} + +func (noopWriter) Put(chunkID []byte) error { return nil } +func (noopWriter) Count() int64 { return 0 } +func (noopWriter) Close() error { return nil } + +type noopCleaner struct{} + +func (noopCleaner) Cleanup(seriesID []byte, userID []byte) error { return nil } + +func Test_EmptyTable(t *testing.T) { + schema := allSchemas[0] + store := newTestStore(t) + c1 := createChunk(t, "1", labels.Labels{labels.Label{Name: "foo", Value: "bar"}}, schema.from, schema.from.Add(1*time.Hour)) + c2 := createChunk(t, "2", labels.Labels{labels.Label{Name: "foo", Value: "buzz"}, labels.Label{Name: "bar", Value: "foo"}}, schema.from, schema.from.Add(1*time.Hour)) + c3 := createChunk(t, "2", labels.Labels{labels.Label{Name: "foo", Value: "buzz"}, labels.Label{Name: "bar", Value: "buzz"}}, schema.from, schema.from.Add(1*time.Hour)) + + require.NoError(t, store.Put(context.TODO(), []chunk.Chunk{ + c1, c2, c3, + })) + + store.Stop() + + tables := store.indexTables() + require.Len(t, tables, 1) + err := tables[0].DB.Update(func(tx *bbolt.Tx) error { + it, err := newChunkIndexIterator(tx.Bucket(bucketName), schema.config) + require.NoError(t, err) + empty, err := markforDelete(noopWriter{}, it, noopCleaner{}, NewExpirationChecker(&fakeLimits{perTenant: map[string]time.Duration{"1": 0, "2": 0}})) + require.NoError(t, err) + require.True(t, empty) + return nil + }) + require.NoError(t, err) +} + func createChunk(t testing.TB, userID string, lbs labels.Labels, from model.Time, through model.Time) chunk.Chunk { t.Helper() const ( diff --git a/pkg/storage/stores/shipper/compactor/retention/util_test.go b/pkg/storage/stores/shipper/compactor/retention/util_test.go index 3ca7d33901dc8..551e2f1ff3670 100644 --- a/pkg/storage/stores/shipper/compactor/retention/util_test.go +++ b/pkg/storage/stores/shipper/compactor/retention/util_test.go @@ -118,6 +118,32 @@ type testStore struct { limits cortex_storage.StoreLimits } +// testObjectClient is a testing object client +type testObjectClient struct { + chunk.ObjectClient + path string +} + +func (t testObjectClient) DeleteObject(ctx context.Context, objectKey string) error { + _ = t.ObjectClient.DeleteObject(ctx, objectKey) // locally we don't have a table folder so we swallow the error. + return nil +} + +func newTestObjectClient(path string) chunk.ObjectClient { + c, err := cortex_storage.NewObjectClient("filesystem", cortex_storage.Config{ + FSConfig: local.FSConfig{ + Directory: path, + }, + }) + if err != nil { + panic(err) + } + return &testObjectClient{ + ObjectClient: c, + path: path, + } +} + type table struct { name string *bbolt.DB @@ -206,13 +232,6 @@ func newTestStore(t testing.TB) *testStore { Mode: shipper.ModeReadWrite, }, } - objectClient, err := cortex_storage.NewObjectClient("filesystem", cortex_storage.Config{ - FSConfig: local.FSConfig{ - Directory: workdir, - }, - }) - require.NoError(t, err) - chunkStore, err := cortex_storage.NewStore( config.Config, chunk.StoreConfig{}, @@ -232,7 +251,7 @@ func newTestStore(t testing.TB) *testStore { t: t, Store: store, schemaCfg: schemaCfg, - objectClient: objectClient, + objectClient: newTestObjectClient(workdir), cfg: config, limits: limits, } From a7dc298fc859747e763ef1836fd80a0c7b3a3a66 Mon Sep 17 00:00:00 2001 From: Cyril Tovena Date: Wed, 28 Apr 2021 11:46:40 +0200 Subject: [PATCH 40/42] Do not delete folder it's not necessary with object store. Signed-off-by: Cyril Tovena --- pkg/storage/stores/shipper/compactor/retention/retention.go | 5 +---- pkg/storage/stores/shipper/compactor/retention/util_test.go | 5 ----- 2 files changed, 1 insertion(+), 9 deletions(-) diff --git a/pkg/storage/stores/shipper/compactor/retention/retention.go b/pkg/storage/stores/shipper/compactor/retention/retention.go index b21382b920e89..0d917924fb5e4 100644 --- a/pkg/storage/stores/shipper/compactor/retention/retention.go +++ b/pkg/storage/stores/shipper/compactor/retention/retention.go @@ -171,10 +171,7 @@ func (t *Marker) markTable(ctx context.Context, tableName string) error { // if the index is empty we can delete the index table. if empty { t.markerMetrics.tableProcessedTotal.WithLabelValues(tableName, tableActionDeleted).Inc() - if err := t.objectClient.DeleteObject(ctx, objectKey); err != nil { - level.Warn(util_log.Logger).Log("msg", "failed to delete empty index table", "err", err, "objectKey", objectKey) - } - return t.objectClient.DeleteObject(ctx, tableName+delimiter) + return t.objectClient.DeleteObject(ctx, objectKey) } // No chunks to delete means no changes to the remote index, we don't need to upload it. if markerWriter.Count() == 0 { diff --git a/pkg/storage/stores/shipper/compactor/retention/util_test.go b/pkg/storage/stores/shipper/compactor/retention/util_test.go index 551e2f1ff3670..2f01dc610a469 100644 --- a/pkg/storage/stores/shipper/compactor/retention/util_test.go +++ b/pkg/storage/stores/shipper/compactor/retention/util_test.go @@ -124,11 +124,6 @@ type testObjectClient struct { path string } -func (t testObjectClient) DeleteObject(ctx context.Context, objectKey string) error { - _ = t.ObjectClient.DeleteObject(ctx, objectKey) // locally we don't have a table folder so we swallow the error. - return nil -} - func newTestObjectClient(path string) chunk.ObjectClient { c, err := cortex_storage.NewObjectClient("filesystem", cortex_storage.Config{ FSConfig: local.FSConfig{ From af0edc924ed96feb4f606812369711bfbb6f15d1 Mon Sep 17 00:00:00 2001 From: Cyril Tovena Date: Wed, 28 Apr 2021 11:54:09 +0200 Subject: [PATCH 41/42] Better working path cleanup Signed-off-by: Cyril Tovena --- .../stores/shipper/compactor/retention/retention.go | 7 +------ 1 file changed, 1 insertion(+), 6 deletions(-) diff --git a/pkg/storage/stores/shipper/compactor/retention/retention.go b/pkg/storage/stores/shipper/compactor/retention/retention.go index 0d917924fb5e4..6be5a258d68ff 100644 --- a/pkg/storage/stores/shipper/compactor/retention/retention.go +++ b/pkg/storage/stores/shipper/compactor/retention/retention.go @@ -103,7 +103,7 @@ func (t *Marker) markTable(ctx context.Context, tableName string) error { return err } defer func() { - if err := os.Remove(tableDirectory); err != nil { + if err := os.RemoveAll(tableDirectory); err != nil { level.Warn(util_log.Logger).Log("msg", "failed to remove temporary table directory", "err", err, "path", tableDirectory) } }() @@ -123,14 +123,9 @@ func (t *Marker) markTable(ctx context.Context, tableName string) error { } defer func() { - path := db.Path() if err := db.Close(); err != nil { level.Warn(util_log.Logger).Log("msg", "failed to close local db", "err", err) } - - if err := os.Remove(path); err != nil { - level.Warn(util_log.Logger).Log("msg", "failed to removed downloaded db", "err", err, "path", path) - } }() schemaCfg, ok := schemaPeriodForTable(t.config, tableName) From b451e75e45ce3c0b59072c56a16cc7b67837e069 Mon Sep 17 00:00:00 2001 From: Cyril Tovena Date: Wed, 28 Apr 2021 13:17:33 +0200 Subject: [PATCH 42/42] got linted. Signed-off-by: Cyril Tovena --- pkg/storage/stores/shipper/compactor/retention/retention.go | 1 - 1 file changed, 1 deletion(-) diff --git a/pkg/storage/stores/shipper/compactor/retention/retention.go b/pkg/storage/stores/shipper/compactor/retention/retention.go index 6be5a258d68ff..47d42e38edd34 100644 --- a/pkg/storage/stores/shipper/compactor/retention/retention.go +++ b/pkg/storage/stores/shipper/compactor/retention/retention.go @@ -31,7 +31,6 @@ var ( const ( logMetricName = "logs" - delimiter = "/" markersFolder = "markers" )