Bloom gateway: Integrate processor into worker (#11895)

This PR replaces the iteration of the bloom gateway worker with the code
encapsulated in the previously established processor.

Since the processor only relies on the BloomStore, rather than on the
BloomShipper, the PR also removes unused code from the shipper.


Signed-off-by: Christian Haudum <christian.haudum@gmail.com>
pull/11902/head
Christian Haudum 1 year ago committed by GitHub
parent 483ee56139
commit 9128036145
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
  1. 27
      pkg/bloomgateway/bloomgateway.go
  2. 27
      pkg/bloomgateway/bloomgateway_test.go
  3. 7
      pkg/bloomgateway/processor.go
  4. 82
      pkg/bloomgateway/processor_test.go
  5. 54
      pkg/bloomgateway/util_test.go
  6. 160
      pkg/bloomgateway/worker.go
  7. 62
      pkg/storage/stores/shipper/bloomshipper/shipper.go
  8. 34
      pkg/storage/stores/shipper/bloomshipper/shipper_test.go

@ -23,13 +23,15 @@ of line filter expressions.
|
bloomgateway.Gateway
|
queue.RequestQueue
queue.RequestQueue
|
bloomgateway.Worker
bloomgateway.Worker
|
bloomshipper.Shipper
bloomgateway.Processor
|
bloomshipper.BloomFileClient
bloomshipper.Store
|
bloomshipper.Client
|
ObjectClient
|
@ -170,9 +172,9 @@ type Gateway struct {
workerMetrics *workerMetrics
queueMetrics *queue.Metrics
queue *queue.RequestQueue
activeUsers *util.ActiveUsersCleanupService
bloomShipper bloomshipper.Interface
queue *queue.RequestQueue
activeUsers *util.ActiveUsersCleanupService
bloomStore bloomshipper.Store
sharding ShardingStrategy
@ -218,13 +220,8 @@ func New(cfg Config, schemaCfg config.SchemaConfig, storageCfg storage.Config, o
return nil, err
}
bloomShipper, err := bloomshipper.NewShipper(store, storageCfg.BloomShipperConfig, overrides, logger, reg)
if err != nil {
return nil, err
}
// We need to keep a reference to be able to call Stop() on shutdown of the gateway.
g.bloomShipper = bloomShipper
g.bloomStore = store
if err := g.initServices(); err != nil {
return nil, err
@ -239,7 +236,7 @@ func (g *Gateway) initServices() error {
svcs := []services.Service{g.queue, g.activeUsers}
for i := 0; i < g.cfg.WorkerConcurrency; i++ {
id := fmt.Sprintf("bloom-query-worker-%d", i)
w := newWorker(id, g.workerConfig, g.queue, g.bloomShipper, g.pendingTasks, g.logger, g.workerMetrics)
w := newWorker(id, g.workerConfig, g.queue, g.bloomStore, g.pendingTasks, g.logger, g.workerMetrics)
svcs = append(svcs, w)
}
g.serviceMngr, err = services.NewManager(svcs...)
@ -291,7 +288,7 @@ func (g *Gateway) running(ctx context.Context) error {
}
func (g *Gateway) stopping(_ error) error {
g.bloomShipper.Stop()
g.bloomStore.Stop()
return services.StopManagerAndAwaitStopped(context.Background(), g.serviceMngr)
}

@ -170,10 +170,12 @@ func TestBloomGateway_FilterChunkRefs(t *testing.T) {
now := mktime("2023-10-03 10:00")
bqs, data := createBlockQueriers(t, 10, now.Add(-24*time.Hour), now, 0, 1000)
mockStore := newMockBloomStore(bqs)
mockStore.err = errors.New("failed to fetch block")
gw.bloomShipper = mockStore
// replace store implementation and re-initialize workers and sub-services
_, metas, queriers, data := createBlocks(t, tenantID, 10, now.Add(-1*time.Hour), now, 0x0000, 0x0fff)
mockStore := newMockBloomStore(queriers, metas)
mockStore.err = errors.New("request failed")
gw.bloomStore = mockStore
err = gw.initServices()
require.NoError(t, err)
@ -204,7 +206,7 @@ func TestBloomGateway_FilterChunkRefs(t *testing.T) {
t.Cleanup(cancelFn)
res, err := gw.FilterChunkRefs(ctx, req)
require.ErrorContainsf(t, err, "request failed: failed to fetch block", "%+v", res)
require.ErrorContainsf(t, err, "request failed", "%+v", res)
}
})
@ -215,10 +217,12 @@ func TestBloomGateway_FilterChunkRefs(t *testing.T) {
now := mktime("2024-01-25 10:00")
bqs, data := createBlockQueriers(t, 50, now.Add(-24*time.Hour), now, 0, 1024)
mockStore := newMockBloomStore(bqs)
mockStore.delay = 50 * time.Millisecond // delay for each block - 50x50=2500ms
gw.bloomShipper = mockStore
// replace store implementation and re-initialize workers and sub-services
_, metas, queriers, data := createBlocks(t, tenantID, 10, now.Add(-1*time.Hour), now, 0x0000, 0x0fff)
mockStore := newMockBloomStore(queriers, metas)
mockStore.delay = 2000 * time.Millisecond
gw.bloomStore = mockStore
err = gw.initServices()
require.NoError(t, err)
@ -346,8 +350,9 @@ func TestBloomGateway_FilterChunkRefs(t *testing.T) {
now := mktime("2023-10-03 10:00")
// replace store implementation and re-initialize workers and sub-services
bqs, data := createBlockQueriers(t, 5, now.Add(-8*time.Hour), now, 0, 1024)
gw.bloomShipper = newMockBloomStore(bqs)
_, metas, queriers, data := createBlocks(t, tenantID, 10, now.Add(-1*time.Hour), now, 0x0000, 0x0fff)
gw.bloomStore = newMockBloomStore(queriers, metas)
err = gw.initServices()
require.NoError(t, err)

@ -17,6 +17,13 @@ type tasksForBlock struct {
tasks []Task
}
func newProcessor(store bloomshipper.Store, logger log.Logger) *processor {
return &processor{
store: store,
logger: logger,
}
}
type processor struct {
store bloomshipper.Store
logger log.Logger

@ -7,6 +7,8 @@ import (
"testing"
"time"
"github.com/go-kit/log"
"github.com/pkg/errors"
"github.com/prometheus/common/model"
"github.com/stretchr/testify/require"
"go.uber.org/atomic"
@ -17,13 +19,26 @@ import (
var _ bloomshipper.Store = &dummyStore{}
func newMockBloomStore(bqs []*bloomshipper.CloseableBlockQuerier, metas []bloomshipper.Meta) *dummyStore {
return &dummyStore{
querieres: bqs,
metas: metas,
}
}
type dummyStore struct {
metas []bloomshipper.Meta
blocks []bloomshipper.BlockRef
querieres []*bloomshipper.CloseableBlockQuerier
// mock how long it takes to serve block queriers
delay time.Duration
// mock response error when serving block queriers in ForEach
err error
}
func (s *dummyStore) ResolveMetas(_ context.Context, _ bloomshipper.MetaSearchParams) ([][]bloomshipper.MetaRef, []*bloomshipper.Fetcher, error) {
time.Sleep(s.delay)
//TODO(chaudum) Filter metas based on search params
refs := make([]bloomshipper.MetaRef, 0, len(s.metas))
for _, meta := range s.metas {
@ -51,6 +66,11 @@ func (s *dummyStore) Stop() {
func (s *dummyStore) FetchBlocks(_ context.Context, refs []bloomshipper.BlockRef) ([]*bloomshipper.CloseableBlockQuerier, error) {
result := make([]*bloomshipper.CloseableBlockQuerier, 0, len(s.querieres))
if s.err != nil {
time.Sleep(s.delay)
return result, s.err
}
for _, ref := range refs {
for _, bq := range s.querieres {
if ref.Bounds.Equal(bq.Bounds) {
@ -63,6 +83,8 @@ func (s *dummyStore) FetchBlocks(_ context.Context, refs []bloomshipper.BlockRef
result[i], result[j] = result[j], result[i]
})
time.Sleep(s.delay)
return result, nil
}
@ -71,14 +93,11 @@ func TestProcessor(t *testing.T) {
tenant := "fake"
now := mktime("2024-01-27 12:00")
t.Run("dummy", func(t *testing.T) {
blocks, metas, queriers, data := createBlocks(t, tenant, 10, now.Add(-1*time.Hour), now, 0x0000, 0x1000)
t.Run("success case", func(t *testing.T) {
_, metas, queriers, data := createBlocks(t, tenant, 10, now.Add(-1*time.Hour), now, 0x0000, 0x0fff)
p := &processor{
store: &dummyStore{
querieres: queriers,
metas: metas,
blocks: blocks,
},
store: newMockBloomStore(queriers, metas),
logger: log.NewNopLogger(),
}
chunkRefs := createQueryInputFromBlockData(t, tenant, data, 10)
@ -116,4 +135,51 @@ func TestProcessor(t *testing.T) {
require.NoError(t, err)
require.Equal(t, int64(len(swb.series)), results.Load())
})
t.Run("failure case", func(t *testing.T) {
_, metas, queriers, data := createBlocks(t, tenant, 10, now.Add(-1*time.Hour), now, 0x0000, 0x0fff)
mockStore := newMockBloomStore(queriers, metas)
mockStore.err = errors.New("store failed")
p := &processor{
store: mockStore,
logger: log.NewNopLogger(),
}
chunkRefs := createQueryInputFromBlockData(t, tenant, data, 10)
swb := seriesWithBounds{
series: groupRefs(t, chunkRefs),
bounds: model.Interval{
Start: now.Add(-1 * time.Hour),
End: now,
},
day: truncateDay(now),
}
filters := []syntax.LineFilter{
{Ty: 0, Match: "no match"},
}
t.Log("series", len(swb.series))
task, _ := NewTask(ctx, "fake", swb, filters)
tasks := []Task{task}
results := atomic.NewInt64(0)
var wg sync.WaitGroup
for i := range tasks {
wg.Add(1)
go func(ta Task) {
defer wg.Done()
for range ta.resCh {
results.Inc()
}
t.Log("done", results.Load())
}(tasks[i])
}
err := p.run(ctx, tasks)
wg.Wait()
require.Errorf(t, err, "store failed")
require.Equal(t, int64(0), results.Load())
})
}

@ -1,8 +1,6 @@
package bloomgateway
import (
"context"
"math/rand"
"testing"
"time"
@ -370,58 +368,6 @@ func createBlocks(t *testing.T, tenant string, n int, from, through model.Time,
return blocks, metas, queriers, series
}
func newMockBloomStore(bqs []*bloomshipper.CloseableBlockQuerier) *mockBloomStore {
return &mockBloomStore{bqs: bqs}
}
type mockBloomStore struct {
bqs []*bloomshipper.CloseableBlockQuerier
// mock how long it takes to serve block queriers
delay time.Duration
// mock response error when serving block queriers in ForEach
err error
}
var _ bloomshipper.Interface = &mockBloomStore{}
// GetBlockRefs implements bloomshipper.Interface
func (s *mockBloomStore) GetBlockRefs(_ context.Context, _ string, _ bloomshipper.Interval) ([]bloomshipper.BlockRef, error) {
time.Sleep(s.delay)
blocks := make([]bloomshipper.BlockRef, 0, len(s.bqs))
for i := range s.bqs {
blocks = append(blocks, s.bqs[i].BlockRef)
}
return blocks, nil
}
// Stop implements bloomshipper.Interface
func (s *mockBloomStore) Stop() {}
// ForEach implements bloomshipper.Interface
func (s *mockBloomStore) ForEach(_ context.Context, _ string, _ []bloomshipper.BlockRef, callback bloomshipper.ForEachBlockCallback) error {
if s.err != nil {
time.Sleep(s.delay)
return s.err
}
shuffled := make([]*bloomshipper.CloseableBlockQuerier, len(s.bqs))
_ = copy(shuffled, s.bqs)
rand.Shuffle(len(shuffled), func(i, j int) {
shuffled[i], shuffled[j] = shuffled[j], shuffled[i]
})
for _, bq := range shuffled {
// ignore errors in the mock
time.Sleep(s.delay)
err := callback(bq.BlockQuerier, bq.Bounds)
if err != nil {
return err
}
}
return nil
}
func createQueryInputFromBlockData(t *testing.T, tenant string, data [][]v1.SeriesWithBloom, nthSeries int) []*logproto.ChunkRef {
t.Helper()
n := 0

@ -10,11 +10,8 @@ import (
"github.com/pkg/errors"
"github.com/prometheus/client_golang/prometheus"
"github.com/prometheus/client_golang/prometheus/promauto"
"github.com/prometheus/common/model"
"golang.org/x/exp/slices"
"github.com/grafana/loki/pkg/queue"
v1 "github.com/grafana/loki/pkg/storage/bloom/v1"
"github.com/grafana/loki/pkg/storage/stores/shipper/bloomshipper"
)
@ -23,11 +20,9 @@ type workerConfig struct {
}
type workerMetrics struct {
dequeuedTasks *prometheus.CounterVec
dequeueErrors *prometheus.CounterVec
dequeueWaitTime *prometheus.SummaryVec
storeAccessLatency *prometheus.HistogramVec
bloomQueryLatency *prometheus.HistogramVec
dequeuedTasks *prometheus.CounterVec
dequeueErrors *prometheus.CounterVec
dequeueWaitTime *prometheus.SummaryVec
}
func newWorkerMetrics(registerer prometheus.Registerer, namespace, subsystem string) *workerMetrics {
@ -51,19 +46,6 @@ func newWorkerMetrics(registerer prometheus.Registerer, namespace, subsystem str
Name: "dequeue_wait_time",
Help: "Time spent waiting for dequeuing tasks from queue",
}, labels),
bloomQueryLatency: promauto.With(registerer).NewHistogramVec(prometheus.HistogramOpts{
Namespace: namespace,
Subsystem: subsystem,
Name: "bloom_query_latency",
Help: "Latency in seconds of processing bloom blocks",
}, append(labels, "status")),
// TODO(chaudum): Move this metric into the bloomshipper
storeAccessLatency: promauto.With(registerer).NewHistogramVec(prometheus.HistogramOpts{
Namespace: namespace,
Subsystem: subsystem,
Name: "store_latency",
Help: "Latency in seconds of accessing the bloom store component",
}, append(labels, "operation")),
}
}
@ -78,18 +60,18 @@ type worker struct {
id string
cfg workerConfig
queue *queue.RequestQueue
shipper bloomshipper.Interface
store bloomshipper.Store
pending *pendingTasks
logger log.Logger
metrics *workerMetrics
}
func newWorker(id string, cfg workerConfig, queue *queue.RequestQueue, shipper bloomshipper.Interface, pending *pendingTasks, logger log.Logger, metrics *workerMetrics) *worker {
func newWorker(id string, cfg workerConfig, queue *queue.RequestQueue, store bloomshipper.Store, pending *pendingTasks, logger log.Logger, metrics *workerMetrics) *worker {
w := &worker{
id: id,
cfg: cfg,
queue: queue,
shipper: shipper,
store: store,
pending: pending,
logger: log.With(logger, "worker", id),
metrics: metrics,
@ -107,6 +89,8 @@ func (w *worker) starting(_ context.Context) error {
func (w *worker) running(_ context.Context) error {
idx := queue.StartIndexWithLocalQueue
p := processor{store: w.store, logger: w.logger}
for st := w.State(); st == services.Running || st == services.Stopping; {
taskCtx := context.Background()
dequeueStart := time.Now()
@ -128,8 +112,7 @@ func (w *worker) running(_ context.Context) error {
}
w.metrics.dequeuedTasks.WithLabelValues(w.id).Add(float64(len(items)))
tasksPerDay := make(map[model.Time][]Task)
tasks := make([]Task, 0, len(items))
for _, item := range items {
task, ok := item.(Task)
if !ok {
@ -139,91 +122,12 @@ func (w *worker) running(_ context.Context) error {
}
level.Debug(w.logger).Log("msg", "dequeued task", "task", task.ID)
w.pending.Delete(task.ID)
tasksPerDay[task.day] = append(tasksPerDay[task.day], task)
tasks = append(tasks, task)
}
for day, tasks := range tasksPerDay {
// Remove tasks that are already cancelled
tasks = slices.DeleteFunc(tasks, func(t Task) bool {
if res := t.ctx.Err(); res != nil {
t.CloseWithError(res)
return true
}
return false
})
// no tasks to process
// continue with tasks of next day
if len(tasks) == 0 {
continue
}
// interval is [Start, End)
interval := bloomshipper.NewInterval(day, day.Add(Day))
logger := log.With(w.logger, "day", day.Time(), "tenant", tasks[0].Tenant)
level.Debug(logger).Log("msg", "process tasks", "tasks", len(tasks))
storeFetchStart := time.Now()
blockRefs, err := w.shipper.GetBlockRefs(taskCtx, tasks[0].Tenant, interval)
w.metrics.storeAccessLatency.WithLabelValues(w.id, "GetBlockRefs").Observe(time.Since(storeFetchStart).Seconds())
if err != nil {
for _, t := range tasks {
t.CloseWithError(err)
}
// continue with tasks of next day
continue
}
if len(tasks) == 0 {
continue
}
// No blocks found.
// Since there are no blocks for the given tasks, we need to return the
// unfiltered list of chunk refs.
if len(blockRefs) == 0 {
level.Warn(logger).Log("msg", "no blocks found")
for _, t := range tasks {
t.Close()
}
// continue with tasks of next day
continue
}
// Remove tasks that are already cancelled
tasks = slices.DeleteFunc(tasks, func(t Task) bool {
if res := t.ctx.Err(); res != nil {
t.CloseWithError(res)
return true
}
return false
})
// no tasks to process
// continue with tasks of next day
if len(tasks) == 0 {
continue
}
tasksForBlocks := partitionFingerprintRange(tasks, blockRefs)
blockRefs = blockRefs[:0]
for _, b := range tasksForBlocks {
blockRefs = append(blockRefs, b.blockRef)
}
err = w.processBlocksWithCallback(taskCtx, tasks[0].Tenant, blockRefs, tasksForBlocks)
if err != nil {
for _, t := range tasks {
t.CloseWithError(err)
}
// continue with tasks of next day
continue
}
// all tasks for this day are done.
// close them to notify the request handler
for _, task := range tasks {
task.Close()
}
err = p.run(taskCtx, tasks)
if err != nil {
level.Error(w.logger).Log("msg", "failed to process tasks", "err", err)
}
// return dequeued items back to the pool
@ -238,41 +142,3 @@ func (w *worker) stopping(err error) error {
w.queue.UnregisterConsumerConnection(w.id)
return nil
}
func (w *worker) processBlocksWithCallback(taskCtx context.Context, tenant string, blockRefs []bloomshipper.BlockRef, boundedRefs []boundedTasks) error {
return w.shipper.ForEach(taskCtx, tenant, blockRefs, func(bq *v1.BlockQuerier, bounds v1.FingerprintBounds) error {
for _, b := range boundedRefs {
if b.blockRef.Bounds.Equal(bounds) {
return w.processBlock(bq, b.tasks)
}
}
return nil
})
}
func (w *worker) processBlock(blockQuerier *v1.BlockQuerier, tasks []Task) error {
schema, err := blockQuerier.Schema()
if err != nil {
return err
}
tokenizer := v1.NewNGramTokenizer(schema.NGramLen(), 0)
iters := make([]v1.PeekingIterator[v1.Request], 0, len(tasks))
for _, task := range tasks {
it := v1.NewPeekingIter(task.RequestIter(tokenizer))
iters = append(iters, it)
}
fq := blockQuerier.Fuse(iters)
start := time.Now()
err = fq.Run()
duration := time.Since(start).Seconds()
if err != nil {
w.metrics.bloomQueryLatency.WithLabelValues(w.id, "failure").Observe(duration)
return err
}
w.metrics.bloomQueryLatency.WithLabelValues(w.id, "success").Observe(duration)
return nil
}

@ -3,59 +3,34 @@ package bloomshipper
import (
"context"
"fmt"
"math"
"sort"
"github.com/go-kit/log"
"github.com/go-kit/log/level"
"github.com/prometheus/client_golang/prometheus"
v1 "github.com/grafana/loki/pkg/storage/bloom/v1"
"github.com/grafana/loki/pkg/storage/stores/shipper/bloomshipper/config"
)
type ForEachBlockCallback func(bq *v1.BlockQuerier, bounds v1.FingerprintBounds) error
type Interface interface {
GetBlockRefs(ctx context.Context, tenant string, interval Interval) ([]BlockRef, error)
ForEach(ctx context.Context, tenant string, blocks []BlockRef, callback ForEachBlockCallback) error
Stop()
}
type Shipper struct {
store Store
config config.Config
logger log.Logger
store Store
}
type Limits interface {
BloomGatewayBlocksDownloadingParallelism(tenantID string) int
}
func NewShipper(client Store, config config.Config, _ Limits, logger log.Logger, _ prometheus.Registerer) (*Shipper, error) {
logger = log.With(logger, "component", "bloom-shipper")
return &Shipper{
store: client,
config: config,
logger: logger,
}, nil
}
func (s *Shipper) GetBlockRefs(ctx context.Context, tenantID string, interval Interval) ([]BlockRef, error) {
level.Debug(s.logger).Log("msg", "GetBlockRefs", "tenant", tenantID, "[", interval.Start, "", interval.End)
// TODO(chaudum): The bloom gateway should not fetch blocks for the complete key space
bounds := []v1.FingerprintBounds{v1.NewBounds(0, math.MaxUint64)}
blockRefs, err := s.getActiveBlockRefs(ctx, tenantID, interval, bounds)
if err != nil {
return nil, fmt.Errorf("error fetching active block references : %w", err)
}
return blockRefs, nil
func NewShipper(client Store) *Shipper {
return &Shipper{store: client}
}
func (s *Shipper) ForEach(ctx context.Context, _ string, refs []BlockRef, callback ForEachBlockCallback) error {
// ForEach is a convenience function that wraps the store's FetchBlocks function
// and automatically closes the block querier once the callback was run.
func (s *Shipper) ForEach(ctx context.Context, refs []BlockRef, callback ForEachBlockCallback) error {
bqs, err := s.store.FetchBlocks(ctx, refs)
if err != nil {
return err
}
@ -79,31 +54,6 @@ func (s *Shipper) Stop() {
s.store.Stop()
}
// getFirstLast returns the first and last item of a fingerprint slice
// It assumes an ascending sorted list of fingerprints.
func getFirstLast[T any](s []T) (T, T) {
var zero T
if len(s) == 0 {
return zero, zero
}
return s[0], s[len(s)-1]
}
func (s *Shipper) getActiveBlockRefs(ctx context.Context, tenantID string, interval Interval, bounds []v1.FingerprintBounds) ([]BlockRef, error) {
minFpRange, maxFpRange := getFirstLast(bounds)
metas, err := s.store.FetchMetas(ctx, MetaSearchParams{
TenantID: tenantID,
Keyspace: v1.NewBounds(minFpRange.Min, maxFpRange.Max),
Interval: interval,
})
if err != nil {
return []BlockRef{}, fmt.Errorf("error fetching meta.json files: %w", err)
}
level.Debug(s.logger).Log("msg", "dowloaded metas", "count", len(metas))
return BlocksForMetas(metas, interval, bounds), nil
}
// BlocksForMetas returns all the blocks from all the metas listed that are within the requested bounds
// and not tombstoned in any of the metas
func BlocksForMetas(metas []Meta, interval Interval, keyspaces []v1.FingerprintBounds) []BlockRef {

@ -1,6 +1,7 @@
package bloomshipper
import (
"context"
"fmt"
"math"
"testing"
@ -12,7 +13,7 @@ import (
v1 "github.com/grafana/loki/pkg/storage/bloom/v1"
)
func Test_Shipper_findBlocks(t *testing.T) {
func TestBloomShipper_findBlocks(t *testing.T) {
t.Run("expected block that are specified in tombstones to be filtered out", func(t *testing.T) {
metas := []Meta{
{
@ -110,7 +111,7 @@ func Test_Shipper_findBlocks(t *testing.T) {
}
}
func TestIsOutsideRange(t *testing.T) {
func TestBloomShipper_IsOutsideRange(t *testing.T) {
startTs := model.Time(1000)
endTs := model.Time(2000)
@ -181,6 +182,35 @@ func TestIsOutsideRange(t *testing.T) {
})
}
func TestBloomShipper_ForEach(t *testing.T) {
blockRefs := make([]BlockRef, 0, 3)
store, _ := newMockBloomStore(t)
for i := 0; i < len(blockRefs); i++ {
block, err := createBlockInStorage(t, store, "tenant", model.Time(i*24*int(time.Hour)), 0x0000, 0x00ff)
require.NoError(t, err)
blockRefs = append(blockRefs, block.BlockRef)
}
shipper := NewShipper(store)
var count int
err := shipper.ForEach(context.Background(), blockRefs, func(_ *v1.BlockQuerier, _ v1.FingerprintBounds) error {
count++
return nil
})
require.NoError(t, err)
require.Equal(t, len(blockRefs), count)
// check that the BlockDirectory ref counter is 0
for i := 0; i < len(blockRefs); i++ {
s := store.stores[0]
key := s.Block(blockRefs[i]).Addr()
dir, found := s.fetcher.blocksCache.Get(context.Background(), key)
require.True(t, found)
require.Equal(t, int32(0), dir.refCount.Load())
}
}
func createMatchingBlockRef(checksum uint32) BlockRef {
block := createBlockRef(0, math.MaxUint64, model.Time(0), model.Time(math.MaxInt64))
block.Checksum = checksum

Loading…
Cancel
Save