Add first step for creating a single bloom from series (#11115)

**What this PR does / why we need it**:
This is the first step to create a bloom from a series. The logic to get tenants and tables is a temporary hack and will be replaced by the ring logic in https://github.com/grafana/loki/pull/11154

**Which issue(s) this PR fixes**:
Fixes #<issue number>

**Special notes for your reviewer**:
It does not handle appending multiple blooms into blocks. 
It does not handle creating and uploading of meta files. 

**Checklist**
- [ ] Reviewed the
[`CONTRIBUTING.md`](https://github.com/grafana/loki/blob/main/CONTRIBUTING.md)
guide (**required**)
- [ ] Documentation added
- [ ] Tests updated
- [ ] `CHANGELOG.md` updated
- [ ] If the change is worth mentioning in the release notes, add
`add-to-release-notes` label
- [ ] Changes that require user attention or interaction to upgrade are
documented in `docs/sources/setup/upgrade/_index.md`
- [ ] For Helm chart changes bump the Helm chart version in
`production/helm/loki/Chart.yaml` and update
`production/helm/loki/CHANGELOG.md` and
`production/helm/loki/README.md`. [Example
PR](d10549e3ec)
- [ ] If the change is deprecating or removing a configuration option,
update the `deprecated-config.yaml` and `deleted-config.yaml` files
respectively in the `tools/deprecated-config-checker` directory.
[Example
PR](0d4416a4b0)
pull/11175/head
Poyzan 2 years ago committed by GitHub
parent 30fbe38341
commit fa378ac0ec
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
  1. 2
      docs/sources/configure/_index.md
  2. 5
      pkg/bloomcompactor/TODO.md
  3. 422
      pkg/bloomcompactor/bloomcompactor.go
  4. 6
      pkg/bloomcompactor/config.go
  5. 3
      pkg/loki/modules.go
  6. 2
      pkg/storage/bloom/v1/block.go
  7. 2
      pkg/storage/bloom/v1/bloom_tokenizer.go
  8. 1
      pkg/storage/stores/shipper/bloomshipper/client.go

@ -2532,8 +2532,6 @@ ring:
[enabled: <boolean> | default = false]
[working_directory: <string> | default = ""]
[max_look_back_period: <duration>]
```
### limits_config

@ -0,0 +1,5 @@
* Should we consider configuring falsePosRate of sbf at runtime?
* Use tarGz, untarGz before uploding blocks to storage
* Return checksum from `BuildFrom`
* Move meta creation to an outer layer, ensure one meta.json per compaction cycle.
* Introduce back `maxLookBackPeriod` as `RejectOldSamplesMaxAge` limit in distributors

@ -26,21 +26,46 @@ package bloomcompactor
import (
"context"
"encoding/binary"
"fmt"
"math"
"os"
"path/filepath"
"strconv"
"strings"
"time"
"github.com/go-kit/log/level"
"github.com/pkg/errors"
"github.com/prometheus/common/model"
"github.com/grafana/loki/pkg/logproto"
"github.com/grafana/loki/pkg/storage/chunk"
chunk_client "github.com/grafana/loki/pkg/storage/chunk/client"
"github.com/grafana/loki/pkg/storage/stores/shipper/indexshipper/downloads"
shipperindex "github.com/grafana/loki/pkg/storage/stores/shipper/indexshipper/index"
index_storage "github.com/grafana/loki/pkg/storage/stores/shipper/indexshipper/storage"
"github.com/grafana/loki/pkg/storage/stores/shipper/indexshipper/tsdb"
tsdbindex "github.com/grafana/loki/pkg/storage/stores/shipper/indexshipper/tsdb/index"
util_log "github.com/grafana/loki/pkg/util/log"
"github.com/go-kit/log"
"github.com/grafana/dskit/ring"
"github.com/grafana/dskit/services"
"github.com/prometheus/client_golang/prometheus"
"github.com/prometheus/prometheus/model/labels"
"github.com/grafana/loki/pkg/storage"
v1 "github.com/grafana/loki/pkg/storage/bloom/v1"
"github.com/grafana/loki/pkg/storage/bloom/v1/filter"
"github.com/grafana/loki/pkg/storage/config"
"github.com/grafana/loki/pkg/storage/stores/shipper/bloomshipper"
"github.com/grafana/loki/pkg/storage/stores/shipper/indexshipper"
)
const (
fpRate = 0.01
bloomFileName = "bloom"
)
type Compactor struct {
@ -49,17 +74,26 @@ type Compactor struct {
cfg Config
logger log.Logger
bloomCompactorRing ring.ReadRing
periodConfigs []config.PeriodConfig
// Client used to run operations on the bucket storing bloom blocks.
storeClients map[config.DayTime]storeClient
// temporary workaround until store has implemented read/write shipper interface
bloomShipperClient bloomshipper.Client
bloomStore bloomshipper.Store
}
type storeClient struct {
object chunk_client.ObjectClient
index index_storage.Client
chunk chunk_client.Client
indexShipper indexshipper.IndexShipper
}
func New(cfg Config,
readRing ring.ReadRing,
storageCfg storage.Config,
periodConfigs []config.PeriodConfig,
schemaConfig config.SchemaConfig,
limits downloads.Limits,
logger log.Logger,
clientMetrics storage.ClientMetrics,
_ prometheus.Registerer) (*Compactor, error) {
@ -67,31 +101,69 @@ func New(cfg Config,
cfg: cfg,
logger: logger,
bloomCompactorRing: readRing,
periodConfigs: periodConfigs,
}
client, err := bloomshipper.NewBloomClient(periodConfigs, storageCfg, clientMetrics)
//Configure BloomClient for meta.json management
bloomClient, err := bloomshipper.NewBloomClient(schemaConfig.Configs, storageCfg, clientMetrics)
if err != nil {
return nil, err
}
shipper, err := bloomshipper.NewShipper(
client,
storageCfg.BloomShipperConfig,
logger,
)
if err != nil {
return nil, err
}
c.storeClients = make(map[config.DayTime]storeClient)
for i, periodicConfig := range schemaConfig.Configs {
var indexStorageCfg indexshipper.Config
switch periodicConfig.IndexType {
case config.TSDBType:
indexStorageCfg = storageCfg.TSDBShipperConfig
case config.BoltDBShipperType:
indexStorageCfg = storageCfg.BoltDBShipperConfig.Config
default:
level.Warn(util_log.Logger).Log("msg", "skipping period because index type is unsupported")
continue
}
//Configure ObjectClient and IndexShipper for series and chunk management
objectClient, err := storage.NewObjectClient(periodicConfig.ObjectType, storageCfg, clientMetrics)
if err != nil {
return nil, fmt.Errorf("error creating object client '%s': %w", periodicConfig.ObjectType, err)
}
periodEndTime := config.DayTime{Time: math.MaxInt64}
if i < len(schemaConfig.Configs)-1 {
periodEndTime = config.DayTime{Time: schemaConfig.Configs[i+1].From.Time.Add(-time.Millisecond)}
}
indexShipper, err := indexshipper.NewIndexShipper(
periodicConfig.IndexTables.PathPrefix,
indexStorageCfg,
objectClient,
limits,
nil,
func(p string) (shipperindex.Index, error) {
return tsdb.OpenShippableTSDB(p)
},
periodicConfig.GetIndexTableNumberRange(periodEndTime),
prometheus.WrapRegistererWithPrefix("loki_tsdb_shipper_", prometheus.DefaultRegisterer),
logger,
)
if err != nil {
return nil, errors.Wrap(err, "create index shipper")
}
c.storeClients[periodicConfig.From] = storeClient{
object: objectClient,
index: index_storage.NewIndexStorageClient(objectClient, periodicConfig.IndexTables.PathPrefix),
chunk: chunk_client.NewClient(objectClient, nil, schemaConfig),
indexShipper: indexShipper,
}
store, err := bloomshipper.NewBloomStore(shipper)
if err != nil {
return nil, err
}
// temporary workaround until store has implemented read/write shipper interface
c.bloomShipperClient = client
c.bloomStore = store
c.bloomShipperClient = bloomClient
// TODO use a new service with a loop
c.Service = services.NewIdleService(c.starting, c.stopping)
@ -106,141 +178,263 @@ func (c *Compactor) stopping(_ error) error {
return nil
}
// TODO Get fpRange owned by the compactor instance
func NoopGetFingerprintRange() (uint64, uint64) { return 0, 0 }
type Series struct { // TODO this can be replaced with Job struct based on Salva's ring work.
tableName, tenant string
labels labels.Labels
fingerPrint model.Fingerprint
chunks []chunk.Chunk
from, through model.Time
indexPath string
}
// TODO List Users from TSDB and add logic to owned user via ring
func NoopGetUserID() string { return "" }
func makeChunkRefs(chksMetas []tsdbindex.ChunkMeta, tenant string, fp model.Fingerprint) []chunk.Chunk {
chunkRefs := make([]chunk.Chunk, 0, len(chksMetas))
for _, chk := range chksMetas {
chunkRefs = append(chunkRefs, chunk.Chunk{
ChunkRef: logproto.ChunkRef{
Fingerprint: uint64(fp),
UserID: tenant,
From: chk.From(),
Through: chk.Through(),
Checksum: chk.Checksum,
},
})
}
// TODO get series from objectClient (TSDB) instead of params
func NoopGetSeries() *v1.Series { return nil }
return chunkRefs
}
// TODO Then get chunk data from series
func NoopGetChunks() []byte { return nil }
// TODO Revisit this step once v1/bloom lib updated to combine blooms in the same series
func buildBloomBlock(bloomForChks v1.SeriesWithBloom, series Series, workingDir string) (bloomshipper.Block, error) {
localDst := createLocalDirName(workingDir, series)
// part1: Create a compact method that assumes no block/meta files exists (eg first compaction)
// part2: Write logic to check first for existing block/meta files and does above.
func (c *Compactor) compactNewChunks(ctx context.Context, dst string) (err error) {
// part1
series := NoopGetSeries()
data := NoopGetChunks()
//write bloom to a local dir
builder, err := v1.NewBlockBuilder(v1.NewBlockOptions(), v1.NewDirectoryBlockWriter(localDst))
if err != nil {
level.Info(util_log.Logger).Log("creating builder", err)
return bloomshipper.Block{}, err
}
bloom := v1.Bloom{ScalableBloomFilter: *filter.NewDefaultScalableBloomFilter(0.01)}
// create bloom filters from that.
bloom.Add([]byte(fmt.Sprint(data)))
err = builder.BuildFrom(v1.NewSliceIter([]v1.SeriesWithBloom{bloomForChks}))
if err != nil {
level.Info(util_log.Logger).Log("writing bloom", err)
return bloomshipper.Block{}, err
}
// block and seriesList
seriesList := []v1.SeriesWithBloom{
{
Series: series,
Bloom: &bloom,
},
blockFile, err := os.Open(filepath.Join(localDst, bloomFileName))
if err != nil {
level.Info(util_log.Logger).Log("reading bloomBlock", err)
}
writer := v1.NewDirectoryBlockWriter(dst)
// read the checksum
if _, err := blockFile.Seek(-4, 2); err != nil {
return bloomshipper.Block{}, errors.Wrap(err, "seeking to bloom checksum")
}
checksum := make([]byte, 4)
if _, err := blockFile.Read(checksum); err != nil {
return bloomshipper.Block{}, errors.Wrap(err, "reading bloom checksum")
}
builder, err := v1.NewBlockBuilder(
v1.BlockOptions{
SeriesPageSize: 100,
BloomPageSize: 10 << 10,
}, writer)
if err != nil {
return err
// Reset back to beginning
if _, err := blockFile.Seek(0, 0); err != nil {
return bloomshipper.Block{}, errors.Wrap(err, "seeking to back to beginning of the file")
}
blocks := bloomshipper.Block{
BlockRef: bloomshipper.BlockRef{
Ref: bloomshipper.Ref{
TenantID: series.tenant,
TableName: series.tableName,
MinFingerprint: uint64(series.fingerPrint), //TODO will change once we compact multiple blooms into a block
MaxFingerprint: uint64(series.fingerPrint),
StartTimestamp: series.from.Unix(),
EndTimestamp: series.through.Unix(),
Checksum: binary.BigEndian.Uint32(checksum),
},
IndexPath: series.indexPath,
},
Data: blockFile,
}
// BuildFrom closes itself
err = builder.BuildFrom(v1.NewSliceIter[v1.SeriesWithBloom](seriesList))
return blocks, nil
}
// TODO Will be replaced with ring implementation in https://github.com/grafana/loki/pull/11154/
func listSeriesForBlooms(ctx context.Context, objectClient storeClient) ([]Series, error) {
// Returns all the TSDB files, including subdirectories
prefix := "index/"
indices, _, err := objectClient.object.List(ctx, prefix, "")
if err != nil {
return err
return nil, err
}
// TODO Ask Owen, shall we expose a method to expose these paths on BlockWriter?
indexPath := filepath.Join(dst, "series")
bloomPath := filepath.Join(dst, "bloom")
var result []Series
for _, index := range indices {
s := strings.Split(index.Key, "/")
if len(s) > 3 {
tableName := s[1]
blockRef := bloomshipper.BlockRef{
IndexPath: indexPath,
BlockPath: bloomPath,
if !strings.HasPrefix(tableName, "loki_") || strings.Contains(tableName, "backup") {
continue
}
userID := s[2]
_, err := strconv.Atoi(userID)
if err != nil {
continue
}
result = append(result, Series{tableName: tableName, tenant: userID, indexPath: index.Key})
}
}
return result, nil
}
blocks := []bloomshipper.Block{
{
BlockRef: blockRef,
func createLocalDirName(workingDir string, series Series) string {
dir := fmt.Sprintf("bloomBlock-%s-%s-%s-%s-%s-%s", series.tableName, series.tenant, series.fingerPrint, series.fingerPrint, series.from, series.through)
return filepath.Join(workingDir, dir)
}
// TODO point to the data to be read
Data: nil,
func CompactNewChunks(ctx context.Context, series Series, bt *v1.BloomTokenizer, bloomShipperClient bloomshipper.Client, dst string) (err error) {
// Create a bloom for this series
bloomForChks := v1.SeriesWithBloom{
Series: &v1.Series{
Fingerprint: series.fingerPrint,
},
Bloom: &v1.Bloom{
ScalableBloomFilter: *filter.NewDefaultScalableBloomFilter(fpRate),
},
}
// Tokenize data into n-grams
bt.PopulateSeriesWithBloom(&bloomForChks, series.chunks)
// Build and upload bloomBlock to storage
blocks, err := buildBloomBlock(bloomForChks, series, dst)
if err != nil {
level.Info(util_log.Logger).Log("building bloomBlocks", err)
return
}
storedBlocks, err := bloomShipperClient.PutBlocks(ctx, []bloomshipper.Block{blocks})
if err != nil {
level.Info(util_log.Logger).Log("putting blocks to storage", err)
return
}
storedBlockRefs := make([]bloomshipper.BlockRef, len(storedBlocks))
// Build and upload meta.json to storage
meta := bloomshipper.Meta{
// After successful compaction there should be no tombstones
Tombstones: make([]bloomshipper.BlockRef, 0),
Blocks: []bloomshipper.BlockRef{blockRef},
Blocks: storedBlockRefs,
}
err = c.bloomShipperClient.PutMeta(ctx, meta)
//TODO move this to an outer layer, otherwise creates a meta per block
err = bloomShipperClient.PutMeta(ctx, meta)
if err != nil {
return err
level.Info(util_log.Logger).Log("putting meta.json to storage", err)
return
}
_, err = c.bloomShipperClient.PutBlocks(ctx, blocks)
if err != nil {
return err
}
// TODO may need to change return value of this func
return nil
}
func (c *Compactor) runCompact(ctx context.Context) error {
// TODO set MaxLookBackPeriod to Max ingester accepts
maxLookBackPeriod := c.cfg.MaxLookBackPeriod
func (c *Compactor) runCompact(ctx context.Context, bloomShipperClient bloomshipper.Client, storeClient storeClient) error {
stFp, endFp := NoopGetFingerprintRange()
tenantID := NoopGetUserID()
series, err := listSeriesForBlooms(ctx, storeClient)
end := time.Now().UTC().UnixMilli()
start := end - maxLookBackPeriod.Milliseconds()
// TODO tokenizer is not thread-safe
// consider moving to Job/worker level with https://github.com/grafana/loki/pull/11154/
// create a tokenizer
bt, _ := v1.NewBloomTokenizer(prometheus.DefaultRegisterer)
metaSearchParams := bloomshipper.MetaSearchParams{
TenantID: tenantID,
MinFingerprint: stFp,
MaxFingerprint: endFp,
StartTimestamp: start,
EndTimestamp: end,
}
metas, err := c.bloomShipperClient.GetMetas(ctx, metaSearchParams)
if err != nil {
return err
}
if len(metas) == 0 {
// run compaction from scratch
tempDst := os.TempDir()
err = c.compactNewChunks(ctx, tempDst)
if err != nil {
return err
}
} else {
// part 2
// When already compacted metas exists
// Deduplicate index paths
uniqueIndexPaths := make(map[string]struct{})
for _, meta := range metas {
for _, blockRef := range meta.Blocks {
uniqueIndexPaths[blockRef.IndexPath] = struct{}{}
for _, s := range series {
err := storeClient.indexShipper.ForEach(ctx, s.tableName, s.tenant, func(isMultiTenantIndex bool, idx shipperindex.Index) error {
if isMultiTenantIndex {
return nil
}
}
// TODO complete part 2 - discuss with Owen - add part to compare chunks and blocks.
// 1. for each period at hand, get TSDB table indexes for given fp range
// 2. Check blocks for given uniqueIndexPaths and TSDBindexes
// if bloomBlock refs are a superset (covers TSDBIndexes plus more outside of range)
// create a new meta.json file, tombstone unused index/block paths.
// else if: there are TSDBindexes that are not covered in bloomBlocks (a subset)
// then call compactNewChunks on them and create a new meta.json
// else: all good, no compaction
// TODO make this casting safe
_ = idx.(*tsdb.TSDBFile).Index.(*tsdb.TSDBIndex).ForSeries(
ctx,
nil, // Process all shards
0, math.MaxInt64, // Replace with MaxLookBackPeriod
// Get chunks for a series label and a fp
func(ls labels.Labels, fp model.Fingerprint, chksMetas []tsdbindex.ChunkMeta) {
// TODO call bloomShipperClient.GetMetas to get existing meta.json
var metas []bloomshipper.Meta
if len(metas) == 0 {
// Get chunks data from list of chunkRefs
chks, err := storeClient.chunk.GetChunks(
ctx,
makeChunkRefs(chksMetas, s.tenant, fp),
)
if err != nil {
level.Info(util_log.Logger).Log("getting chunks", err)
return
}
// effectively get min and max of timestamps of the list of chunks in a series
// There must be a better way to get this, ordering chunkRefs by timestamp doesn't fully solve it
// chunk files name have this info in ObjectStore, but it's not really exposed
minFrom := model.Latest
maxThrough := model.Earliest
for _, c := range chks {
if minFrom > c.From {
minFrom = c.From
}
if maxThrough < c.From {
maxThrough = c.Through
}
}
series := Series{
tableName: s.tableName,
tenant: s.tenant,
labels: ls,
fingerPrint: fp,
chunks: chks,
from: minFrom,
through: maxThrough,
indexPath: s.indexPath,
}
err = CompactNewChunks(ctx, series, bt, bloomShipperClient, c.cfg.WorkingDirectory)
if err != nil {
return
}
} else {
// TODO complete part 2 - periodic compaction for delta from previous period
// When already compacted metas exists
// Deduplicate index paths
uniqueIndexPaths := make(map[string]struct{})
for _, meta := range metas {
for _, blockRef := range meta.Blocks {
uniqueIndexPaths[blockRef.IndexPath] = struct{}{}
//...
}
}
}
})
return nil
})
if err != nil {
return errors.Wrap(err, "getting each series")
}
}
return nil
}

@ -2,7 +2,6 @@ package bloomcompactor
import (
"flag"
"time"
"github.com/grafana/loki/pkg/util/ring"
)
@ -14,9 +13,8 @@ type Config struct {
// section and the ingester configuration by default).
Ring ring.RingConfig `yaml:"ring,omitempty" doc:"description=Defines the ring to be used by the bloom-compactor servers. In case this isn't configured, this block supports inheriting configuration from the common ring section."`
// Enabled configures whether bloom-compactors should be used to compact index values into bloomfilters
Enabled bool `yaml:"enabled"`
WorkingDirectory string `yaml:"working_directory"`
MaxLookBackPeriod time.Duration `yaml:"max_look_back_period"`
Enabled bool `yaml:"enabled"`
WorkingDirectory string `yaml:"working_directory"`
}
// RegisterFlags registers flags for the Bloom-Compactor configuration.

@ -1395,7 +1395,8 @@ func (t *Loki) initBloomCompactor() (services.Service, error) {
compactor, err := bloomcompactor.New(t.Cfg.BloomCompactor,
t.ring,
t.Cfg.StorageConfig,
t.Cfg.SchemaConfig.Configs,
t.Cfg.SchemaConfig,
t.Overrides,
logger,
t.clientMetrics,
prometheus.DefaultRegisterer)

@ -124,7 +124,7 @@ func (bq *BlockQuerier) Err() error {
// CheckChunksForSeries checks if the given chunks pass a set of searches in the given bloom block.
// It returns the list of chunks which will need to be downloaded for a query based on the initial list
// passed as the `chks` argument. Chunks will be removed from the result set if they they are indexed in the bloom
// passed as the `chks` argument. Chunks will be removed from the result set if they are indexed in the bloom
// and fail to pass all the searches.
func (bq *BlockQuerier) CheckChunksForSeries(fp model.Fingerprint, chks ChunkRefs, searches [][]byte) (ChunkRefs, error) {
if err := bq.Seek(fp); err != nil {

@ -11,9 +11,9 @@ import (
"github.com/grafana/loki/pkg/chunkenc"
"github.com/grafana/loki/pkg/logproto"
"github.com/grafana/loki/pkg/logql/log"
"github.com/grafana/loki/pkg/storage/chunk"
util_log "github.com/grafana/loki/pkg/util/log"
//"github.com/grafana/loki/tools/tsdb/helpers"
)
type metrics struct{}

@ -225,6 +225,7 @@ func (b *BloomClient) GetBlocks(ctx context.Context, references []BlockRef) (cha
return blocksChannel, errChannel
}
// TODO zip (archive) blocks before uploading to storage
func (b *BloomClient) PutBlocks(ctx context.Context, blocks []Block) ([]Block, error) {
results := make([]Block, len(blocks))
//todo move concurrency to the config

Loading…
Cancel
Save