chore(blooms): Various bloom gateway fixes/improvements (#12120)

* Response order of `FetchBlocks` is deterministic
    The response keeps the order of the refs from the request.
* Update reference documentation
* Skip not found blocks
* Unify error messages with object key in bloom client
* Change default value for bloom gateway shard size to 0
* Use only 16 tokens per ring instance
* Add debug log line to each request in the bloom gateway client

Signed-off-by: Christian Haudum <christian.haudum@gmail.com>
pull/12064/head^2
Christian Haudum 1 year ago committed by GitHub
parent 97ae790f36
commit 38e2d7be4a
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
  1. 7
      docs/sources/configure/_index.md
  2. 9
      pkg/bloomgateway/bloomgateway.go
  3. 30
      pkg/bloomgateway/client.go
  4. 40
      pkg/bloomgateway/processor.go
  5. 5
      pkg/bloomgateway/processor_test.go
  6. 2
      pkg/loki/modules.go
  7. 18
      pkg/storage/stores/shipper/bloomshipper/client.go
  8. 2
      pkg/storage/stores/shipper/bloomshipper/config/config.go
  9. 13
      pkg/storage/stores/shipper/bloomshipper/fetcher.go
  10. 56
      pkg/storage/stores/shipper/bloomshipper/store.go
  11. 42
      pkg/storage/stores/shipper/bloomshipper/store_test.go
  12. 2
      pkg/validation/limits.go

@ -2342,6 +2342,11 @@ bloom_shipper:
# CLI flag: -bloom.shipper.working-directory
[working_directory: <string> | default = "bloom-shipper"]
# In an eventually consistent system like the bloom components, we usually
# want to ignore blocks that are missing in storage.
# CLI flag: -bloom.shipper.ignore-missing-blocks
[ignore_missing_blocks: <boolean> | default = true]
blocks_downloading_queue:
# The count of parallel workers that download Bloom Blocks.
# CLI flag: -bloom.shipper.blocks-downloading-queue.workers-count
@ -3154,7 +3159,7 @@ shard_streams:
# The shard size defines how many bloom gateways should be used by a tenant for
# querying.
# CLI flag: -bloom-gateway.shard-size
[bloom_gateway_shard_size: <int> | default = 1]
[bloom_gateway_shard_size: <int> | default = 0]
# Whether to use the bloom gateway component in the read path to filter chunks.
# CLI flag: -bloom-gateway.enable-filtering

@ -274,7 +274,14 @@ func (g *Gateway) FilterChunkRefs(ctx context.Context, req *logproto.FilterChunk
if err != nil {
return nil, err
}
level.Debug(g.logger).Log("msg", "creating task for day", "day", seriesForDay.day, "interval", seriesForDay.interval.String(), "task", task.ID)
level.Debug(g.logger).Log(
"msg", "created task for day",
"task", task.ID,
"day", seriesForDay.day,
"interval", seriesForDay.interval.String(),
"nSeries", len(seriesForDay.series),
"filters", JoinFunc(filters, ";", func(e syntax.LineFilterExpr) string { return e.String() }),
)
tasks = append(tasks, task)
numSeries += len(seriesForDay.series)
}

@ -7,6 +7,7 @@ import (
"io"
"math"
"math/rand"
"strings"
"sync"
"github.com/go-kit/log"
@ -216,6 +217,14 @@ func NewClient(
}, nil
}
func JoinFunc[S ~[]E, E any](elems S, sep string, f func(e E) string) string {
res := make([]string, len(elems))
for i := range elems {
res[i] = f(elems[i])
}
return strings.Join(res, sep)
}
func shuffleAddrs(addrs []string) []string {
rand.Shuffle(len(addrs), func(i, j int) {
addrs[i], addrs[j] = addrs[j], addrs[i]
@ -236,6 +245,7 @@ func (c *GatewayClient) FilterChunks(ctx context.Context, tenant string, from, t
}
servers, err := replicationSetsWithBounds(subRing, rs.Instances)
if err != nil {
return nil, errors.Wrap(err, "bloom gateway get replication sets")
}
@ -244,9 +254,23 @@ func (c *GatewayClient) FilterChunks(ctx context.Context, tenant string, from, t
filteredChunkRefs := groupedChunksRefPool.Get(len(groups))
defer groupedChunksRefPool.Put(filteredChunkRefs)
for _, rs := range servers {
for i, rs := range servers {
// randomize order of addresses so we don't hotspot the first server in the list
addrs := shuffleAddrs(rs.rs.GetAddresses())
level.Info(c.logger).Log(
"msg", "do FilterChunkRefs for addresses",
"progress", fmt.Sprintf("%d/%d", i+1, len(servers)),
"bounds", JoinFunc(rs.ranges, ",", func(e v1.FingerprintBounds) string { return e.String() }),
"addrs", strings.Join(addrs, ","),
"from", from.Time(),
"through", through.Time(),
"num_refs", len(rs.groups),
"refs", JoinFunc(rs.groups, ",", func(e *logproto.GroupedChunkRefs) string {
return model.Fingerprint(e.Fingerprint).String()
}),
"plan", plan.String(),
"plan_hash", plan.Hash(),
)
err := c.doForAddrs(addrs, func(client logproto.BloomGatewayClient) error {
req := &logproto.FilterChunkRefRequest{
From: from,
@ -278,12 +302,12 @@ func (c *GatewayClient) doForAddrs(addrs []string, fn func(logproto.BloomGateway
for _, addr := range addrs {
poolClient, err = c.pool.GetClientFor(addr)
if err != nil {
level.Error(c.logger).Log("msg", fmt.Sprintf("failed to get client for instance %s", addr), "err", err)
level.Error(c.logger).Log("msg", "failed to get client for instance", "addr", addr, "err", err)
continue
}
err = fn(poolClient.(logproto.BloomGatewayClient))
if err != nil {
level.Error(c.logger).Log("msg", fmt.Sprintf("client do failed for instance %s", addr), "err", err)
level.Error(c.logger).Log("msg", "client do failed for instance", "addr", addr, "err", err)
continue
}
return nil

@ -7,6 +7,7 @@ import (
"github.com/go-kit/log"
"github.com/go-kit/log/level"
"github.com/pkg/errors"
v1 "github.com/grafana/loki/pkg/storage/bloom/v1"
"github.com/grafana/loki/pkg/storage/config"
@ -68,11 +69,12 @@ func (p *processor) processTasks(ctx context.Context, tenant string, day config.
}
blocksRefs := bloomshipper.BlocksForMetas(metas, interval, keyspaces)
level.Info(p.logger).Log("msg", "blocks for metas", "num_metas", len(metas), "num_blocks", len(blocksRefs))
return p.processBlocks(ctx, partitionTasks(tasks, blocksRefs))
}
func (p *processor) processBlocks(ctx context.Context, data []blockWithTasks) error {
refs := make([]bloomshipper.BlockRef, len(data))
refs := make([]bloomshipper.BlockRef, 0, len(data))
for _, block := range data {
refs = append(refs, block.ref)
}
@ -82,24 +84,28 @@ func (p *processor) processBlocks(ctx context.Context, data []blockWithTasks) er
return err
}
blockIter := v1.NewSliceIter(bqs)
outer:
for blockIter.Next() {
bq := blockIter.At()
for i, block := range data {
if block.ref.Bounds.Equal(bq.Bounds) {
err := p.processBlock(ctx, bq.BlockQuerier, block.tasks)
bq.Close()
if err != nil {
return err
}
data = append(data[:i], data[i+1:]...)
continue outer
}
for i, bq := range bqs {
block := data[i]
if bq == nil {
level.Warn(p.logger).Log("msg", "skipping not found block", "block", block.ref)
continue
}
level.Debug(p.logger).Log(
"msg", "process block with tasks",
"block", block.ref,
"block_bounds", block.ref.Bounds,
"querier_bounds", bq.Bounds,
"num_tasks", len(block.tasks),
)
if !block.ref.Bounds.Equal(bq.Bounds) {
bq.Close()
return errors.Errorf("block and querier bounds differ: %s vs %s", block.ref.Bounds, bq.Bounds)
}
// should not happen, but close anyway
err := p.processBlock(ctx, bq.BlockQuerier, block.tasks)
bq.Close()
if err != nil {
return errors.Wrap(err, "processing block")
}
}
return nil
}

@ -2,7 +2,6 @@ package bloomgateway
import (
"context"
"math/rand"
"sync"
"testing"
"time"
@ -82,10 +81,6 @@ func (s *dummyStore) FetchBlocks(_ context.Context, refs []bloomshipper.BlockRef
}
}
rand.Shuffle(len(result), func(i, j int) {
result[i], result[j] = result[j], result[i]
})
time.Sleep(s.delay)
return result, nil

@ -1333,7 +1333,7 @@ func (t *Loki) initBloomGatewayRing() (services.Service, error) {
if t.Cfg.isModuleEnabled(BloomGateway) || t.Cfg.isModuleEnabled(Backend) || legacyReadMode {
mode = lokiring.ServerMode
}
manager, err := lokiring.NewRingManager(bloomGatewayRingKey, mode, t.Cfg.BloomGateway.Ring.RingConfig, t.Cfg.BloomGateway.Ring.ReplicationFactor, 128, util_log.Logger, prometheus.DefaultRegisterer)
manager, err := lokiring.NewRingManager(bloomGatewayRingKey, mode, t.Cfg.BloomGateway.Ring.RingConfig, t.Cfg.BloomGateway.Ring.ReplicationFactor, 16, util_log.Logger, prometheus.DefaultRegisterer)
if err != nil {
return nil, gerrors.Wrap(err, "error initializing bloom gateway ring manager")

@ -266,7 +266,7 @@ func (b *BloomClient) IsObjectNotFoundErr(err error) bool {
func (b *BloomClient) PutMeta(ctx context.Context, meta Meta) error {
data, err := json.Marshal(meta)
if err != nil {
return fmt.Errorf("can not marshal the meta to json: %w", err)
return fmt.Errorf("failed to encode meta file %s: %w", meta.String(), err)
}
key := b.Meta(meta.MetaRef).Addr()
return b.client.PutObject(ctx, key, bytes.NewReader(data))
@ -288,7 +288,7 @@ func (b *BloomClient) GetBlock(ctx context.Context, ref BlockRef) (BlockDirector
rc, _, err := b.client.GetObject(ctx, key)
if err != nil {
return BlockDirectory{}, fmt.Errorf("failed to get block from storage: %w", err)
return BlockDirectory{}, fmt.Errorf("failed to get block file %s: %w", key, err)
}
defer rc.Close()
@ -297,12 +297,12 @@ func (b *BloomClient) GetBlock(ctx context.Context, ref BlockRef) (BlockDirector
path = strings.TrimSuffix(path, ".tar.gz")
err = util.EnsureDirectory(path)
if err != nil {
return BlockDirectory{}, fmt.Errorf("failed to create block directory: %w", err)
return BlockDirectory{}, fmt.Errorf("failed to create block directory %s: %w", path, err)
}
err = v1.UnTarGz(path, rc)
if err != nil {
return BlockDirectory{}, fmt.Errorf("failed to extract block: %w", err)
return BlockDirectory{}, fmt.Errorf("failed to extract block file %s: %w", key, err)
}
return NewBlockDirectory(ref, path, b.logger), nil
@ -334,12 +334,12 @@ func (b *BloomClient) PutBlock(ctx context.Context, block Block) error {
key := b.Block(block.BlockRef).Addr()
_, err := block.Data.Seek(0, 0)
if err != nil {
return fmt.Errorf("error uploading block file %s : %w", key, err)
return fmt.Errorf("failed to seek block file %s: %w", key, err)
}
err = b.client.PutObject(ctx, key, block.Data)
if err != nil {
return fmt.Errorf("error uploading block file: %w", err)
return fmt.Errorf("failed to put block file %s: %w", key, err)
}
return nil
}
@ -351,7 +351,7 @@ func (b *BloomClient) DeleteBlocks(ctx context.Context, references []BlockRef) e
err := b.client.DeleteObject(ctx, key)
if err != nil {
return fmt.Errorf("error deleting block file: %w", err)
return fmt.Errorf("failed to delete block file %s: %w", key, err)
}
return nil
})
@ -381,13 +381,13 @@ func (b *BloomClient) GetMeta(ctx context.Context, ref MetaRef) (Meta, error) {
key := b.KeyResolver.Meta(ref).Addr()
reader, _, err := b.client.GetObject(ctx, key)
if err != nil {
return Meta{}, fmt.Errorf("error downloading meta file %s : %w", key, err)
return Meta{}, fmt.Errorf("failed to get meta file%s: %w", key, err)
}
defer reader.Close()
err = json.NewDecoder(reader).Decode(&meta)
if err != nil {
return Meta{}, fmt.Errorf("error unmarshalling content of meta file %s: %w", key, err)
return Meta{}, fmt.Errorf("failed to decode meta file %s: %w", key, err)
}
return meta, nil
}

@ -12,6 +12,7 @@ import (
type Config struct {
WorkingDirectory string `yaml:"working_directory"`
IgnoreMissingBlocks bool `yaml:"ignore_missing_blocks"`
BlocksDownloadingQueue DownloadingQueueConfig `yaml:"blocks_downloading_queue"`
BlocksCache cache.EmbeddedCacheConfig `yaml:"blocks_cache"`
MetasCache cache.Config `yaml:"metas_cache"`
@ -29,6 +30,7 @@ func (cfg *DownloadingQueueConfig) RegisterFlagsWithPrefix(prefix string, f *fla
func (c *Config) RegisterFlagsWithPrefix(prefix string, f *flag.FlagSet) {
f.StringVar(&c.WorkingDirectory, prefix+"shipper.working-directory", "bloom-shipper", "Working directory to store downloaded Bloom Blocks.")
f.BoolVar(&c.IgnoreMissingBlocks, prefix+"shipper.ignore-missing-blocks", true, "In an eventually consistent system like the bloom components, we usually want to ignore blocks that are missing in storage.")
c.BlocksDownloadingQueue.RegisterFlagsWithPrefix(prefix+"shipper.blocks-downloading-queue.", f)
c.BlocksCache.RegisterFlagsWithPrefixAndDefaults(prefix+"blocks-cache.", "Cache for bloom blocks. ", f, 24*time.Hour)
c.MetasCache.RegisterFlagsWithPrefix(prefix+"metas-cache.", "Cache for bloom metas. ", f)

@ -37,12 +37,14 @@ type Fetcher struct {
q *downloadQueue[BlockRef, BlockDirectory]
cfg bloomStoreConfig
metrics *fetcherMetrics
logger log.Logger
}
func NewFetcher(cfg bloomStoreConfig, client Client, metasCache cache.Cache, blocksCache cache.TypedCache[string, BlockDirectory], reg prometheus.Registerer, logger log.Logger) (*Fetcher, error) {
fetcher := &Fetcher{
cfg: cfg,
client: client,
metasCache: metasCache,
blocksCache: blocksCache,
@ -153,18 +155,23 @@ func (f *Fetcher) FetchBlocks(ctx context.Context, refs []BlockRef) ([]*Closeabl
})
}
count := 0
results := make([]*CloseableBlockQuerier, n)
for i := 0; i < n; i++ {
select {
case err := <-errors:
f.metrics.blocksFetched.Observe(float64(i + 1))
return results, err
if !f.cfg.ignoreMissingBlocks && !f.client.IsObjectNotFoundErr(err) {
f.metrics.blocksFetched.Observe(float64(count))
return results, err
}
level.Warn(f.logger).Log("msg", "ignore missing block", "err", err)
case res := <-responses:
count++
results[res.idx] = res.item.BlockQuerier()
}
}
f.metrics.blocksFetched.Observe(float64(n))
f.metrics.blocksFetched.Observe(float64(count))
return results, nil
}

@ -7,6 +7,7 @@ import (
"sort"
"github.com/go-kit/log"
"github.com/go-kit/log/level"
"github.com/pkg/errors"
"github.com/prometheus/client_golang/prometheus"
"github.com/prometheus/common/model"
@ -35,8 +36,9 @@ type Store interface {
}
type bloomStoreConfig struct {
workingDir string
numWorkers int
workingDir string
numWorkers int
ignoreMissingBlocks bool
}
// Compiler check to ensure bloomStoreEntry implements the Store interface
@ -57,6 +59,13 @@ func (b *bloomStoreEntry) ResolveMetas(ctx context.Context, params MetaSearchPar
tables := tablesForRange(b.cfg, params.Interval)
for _, table := range tables {
prefix := filepath.Join(rootFolder, table, params.TenantID, metasFolder)
level.Debug(b.fetcher.logger).Log(
"msg", "listing metas",
"store", b.cfg.From,
"table", table,
"tenant", params.TenantID,
"prefix", prefix,
)
list, _, err := b.objectClient.List(ctx, prefix, "")
if err != nil {
return nil, nil, fmt.Errorf("error listing metas under prefix [%s]: %w", prefix, err)
@ -142,6 +151,7 @@ type BloomStore struct {
stores []*bloomStoreEntry
storageConfig storage.Config
metrics *storeMetrics
logger log.Logger
defaultKeyResolver // TODO(owen-d): impl schema aware resolvers
}
@ -157,6 +167,7 @@ func NewBloomStore(
store := &BloomStore{
storageConfig: storageConfig,
metrics: newStoreMetrics(reg, constants.Loki, "bloom_store"),
logger: logger,
}
if metasCache == nil {
@ -174,8 +185,9 @@ func NewBloomStore(
// TODO(chaudum): Remove wrapper
cfg := bloomStoreConfig{
workingDir: storageConfig.BloomShipperConfig.WorkingDirectory,
numWorkers: storageConfig.BloomShipperConfig.BlocksDownloadingQueue.WorkersCount,
workingDir: storageConfig.BloomShipperConfig.WorkingDirectory,
numWorkers: storageConfig.BloomShipperConfig.BlocksDownloadingQueue.WorkersCount,
ignoreMissingBlocks: storageConfig.BloomShipperConfig.IgnoreMissingBlocks,
}
if err := util.EnsureDirectory(cfg.workingDir); err != nil {
@ -319,9 +331,9 @@ func (b *BloomStore) FetchBlocks(ctx context.Context, blocks []BlockRef) ([]*Clo
}
var res []BlockRef
for _, meta := range blocks {
if meta.StartTimestamp >= from && meta.StartTimestamp < through {
res = append(res, meta)
for _, ref := range blocks {
if ref.StartTimestamp >= from && ref.StartTimestamp < through {
res = append(res, ref)
}
}
@ -340,20 +352,32 @@ func (b *BloomStore) FetchBlocks(ctx context.Context, blocks []BlockRef) ([]*Clo
results = append(results, res...)
}
level.Debug(b.logger).Log("msg", "fetch blocks", "num_req", len(blocks), "num_resp", len(results))
// sort responses (results []*CloseableBlockQuerier) based on requests (blocks []BlockRef)
slices.SortFunc(results, func(a, b *CloseableBlockQuerier) int {
ia, ib := slices.Index(blocks, a.BlockRef), slices.Index(blocks, b.BlockRef)
if ia < ib {
return -1
} else if ia > ib {
return +1
}
return 0
})
sortBlocks(results, blocks)
return results, nil
}
func sortBlocks(bqs []*CloseableBlockQuerier, refs []BlockRef) {
tmp := make([]*CloseableBlockQuerier, len(refs))
for _, bq := range bqs {
if bq == nil {
// ignore responses with nil values
continue
}
idx := slices.Index(refs, bq.BlockRef)
if idx < 0 {
// not found
// should not happen in the context of sorting responses based on requests
continue
}
tmp[idx] = bq
}
copy(bqs, tmp)
}
// Stop implements Store.
func (b *BloomStore) Stop() {
for _, s := range b.stores {

@ -4,6 +4,7 @@ import (
"bytes"
"context"
"encoding/json"
"math/rand"
"os"
"path/filepath"
"testing"
@ -341,3 +342,44 @@ func TestTablesForRange(t *testing.T) {
})
}
}
func TestBloomStore_SortBlocks(t *testing.T) {
now := parseTime("2024-02-01 00:00")
refs := make([]BlockRef, 10)
bqs := make([]*CloseableBlockQuerier, 10)
for i := 0; i < 10; i++ {
refs[i] = BlockRef{
Ref: Ref{
TenantID: "fake",
Bounds: v1.NewBounds(
model.Fingerprint(i*1000),
model.Fingerprint((i+1)*1000-1),
),
StartTimestamp: now,
EndTimestamp: now.Add(12 * time.Hour),
},
}
if i%2 == 0 {
bqs[i] = &CloseableBlockQuerier{
BlockRef: refs[i],
}
}
}
// shuffle the slice of block queriers
rand.Shuffle(len(bqs), func(i, j int) { bqs[i], bqs[j] = bqs[j], bqs[i] })
// sort the block queriers based on the refs
sortBlocks(bqs, refs)
// assert order of block queriers
for i := 0; i < 10; i++ {
if i%2 == 0 {
require.Equal(t, refs[i], bqs[i].BlockRef)
} else {
require.Nil(t, nil, bqs[i])
}
}
}

@ -329,7 +329,7 @@ func (l *Limits) RegisterFlags(f *flag.FlagSet) {
f.IntVar(&l.IndexGatewayShardSize, "index-gateway.shard-size", 0, "The shard size defines how many index gateways should be used by a tenant for querying. If the global shard factor is 0, the global shard factor is set to the deprecated -replication-factor for backwards compatibility reasons.")
f.IntVar(&l.BloomGatewayShardSize, "bloom-gateway.shard-size", 1, "The shard size defines how many bloom gateways should be used by a tenant for querying.")
f.IntVar(&l.BloomGatewayShardSize, "bloom-gateway.shard-size", 0, "The shard size defines how many bloom gateways should be used by a tenant for querying.")
f.BoolVar(&l.BloomGatewayEnabled, "bloom-gateway.enable-filtering", false, "Whether to use the bloom gateway component in the read path to filter chunks.")
f.IntVar(&l.BloomCompactorShardSize, "bloom-compactor.shard-size", 1, "The shard size defines how many bloom compactors should be used by a tenant when computing blooms. If it's set to 0, shuffle sharding is disabled.")

Loading…
Cancel
Save