Skip to content

Update parquet common + fix queryStoreAfter config for parquet querier #6799

New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Merged
merged 4 commits into from
Jun 6, 2025
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension


Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
7 changes: 2 additions & 5 deletions go.mod
Original file line number Diff line number Diff line change
Expand Up @@ -81,8 +81,8 @@ require (
github.com/google/go-cmp v0.7.0
github.com/hashicorp/golang-lru/v2 v2.0.7
github.com/munnerz/goautoneg v0.0.0-20191010083416-a7dc8b61c822
github.com/parquet-go/parquet-go v0.25.0
github.com/prometheus-community/parquet-common v0.0.0-20250528231323-eec9c3c020f0
github.com/parquet-go/parquet-go v0.25.1
github.com/prometheus-community/parquet-common v0.0.0-20250606162055-b81ebb7e1b96
github.com/prometheus/procfs v0.16.1
github.com/sercand/kuberesolver/v5 v5.1.1
github.com/tjhop/slog-gokit v0.1.4
Expand Down Expand Up @@ -189,7 +189,6 @@ require (
github.com/mailru/easyjson v0.9.0 // indirect
github.com/mattn/go-colorable v0.1.14 // indirect
github.com/mattn/go-isatty v0.0.20 // indirect
github.com/mattn/go-runewidth v0.0.16 // indirect
github.com/matttproud/golang_protobuf_extensions v1.0.4 // indirect
github.com/mdlayher/socket v0.5.1 // indirect
github.com/mdlayher/vsock v1.2.1 // indirect
Expand All @@ -206,7 +205,6 @@ require (
github.com/mwitkow/go-conntrack v0.0.0-20190716064945-2f068394615f // indirect
github.com/ncw/swift v1.0.53 // indirect
github.com/oklog/run v1.1.0 // indirect
github.com/olekukonko/tablewriter v0.0.5 // indirect
github.com/open-telemetry/opentelemetry-collector-contrib/internal/exp/metrics v0.121.0 // indirect
github.com/open-telemetry/opentelemetry-collector-contrib/pkg/pdatautil v0.121.0 // indirect
github.com/open-telemetry/opentelemetry-collector-contrib/processor/deltatocumulativeprocessor v0.121.0 // indirect
Expand All @@ -217,7 +215,6 @@ require (
github.com/prometheus/exporter-toolkit v0.14.0 // indirect
github.com/prometheus/sigv4 v0.1.2 // indirect
github.com/redis/rueidis v1.0.60 // indirect
github.com/rivo/uniseg v0.4.7 // indirect
github.com/rs/cors v1.11.1 // indirect
github.com/rs/xid v1.6.0 // indirect
github.com/sean-/seed v0.0.0-20170313163322-e2103e2c3529 // indirect
Expand Down
16 changes: 4 additions & 12 deletions go.sum
Original file line number Diff line number Diff line change
Expand Up @@ -1438,9 +1438,6 @@ github.com/mattn/go-isatty v0.0.16/go.mod h1:kYGgaQfpe5nmfYZH+SKPsOc2e4SrIfOl2e/
github.com/mattn/go-isatty v0.0.17/go.mod h1:kYGgaQfpe5nmfYZH+SKPsOc2e4SrIfOl2e/yFXSvRLM=
github.com/mattn/go-isatty v0.0.20 h1:xfD0iDuEKnDkl03q4limB+vH+GxLEtL/jb4xVJSWWEY=
github.com/mattn/go-isatty v0.0.20/go.mod h1:W+V8PltTTMOvKvAeJH7IuucS94S2C6jfK/D7dTCTo3Y=
github.com/mattn/go-runewidth v0.0.9/go.mod h1:H031xJmbD/WCDINGzjvQ9THkh0rPKHF+m2gUSrubnMI=
github.com/mattn/go-runewidth v0.0.16 h1:E5ScNMtiwvlvB5paMFdw9p4kSQzbXFikJ5SQO6TULQc=
github.com/mattn/go-runewidth v0.0.16/go.mod h1:Jdepj2loyihRzMpdS35Xk/zdY8IAYHsh153qUoGf23w=
github.com/mattn/go-sqlite3 v1.14.14/go.mod h1:NyWgC/yNuGj7Q9rpYnZvas74GogHl5/Z4A/KQRfk6bU=
github.com/mattn/go-sqlite3 v1.14.15/go.mod h1:2eHXhiwb8IkHr+BDWZGa96P6+rkvnG63S2DGjv9HUNg=
github.com/matttproud/golang_protobuf_extensions v1.0.1/go.mod h1:D8He9yQNgCq6Z5Ld7szi9bcBfOoFv/3dc6xSMkL2PC0=
Expand Down Expand Up @@ -1508,8 +1505,6 @@ github.com/oklog/run v1.1.0 h1:GEenZ1cK0+q0+wsJew9qUg/DyD8k3JzYsZAi5gYi2mA=
github.com/oklog/run v1.1.0/go.mod h1:sVPdnTZT1zYwAJeCMu2Th4T21pA3FPOQRfWjQlk7DVU=
github.com/oklog/ulid v1.3.1 h1:EGfNDEx6MqHz8B3uNV6QAib1UR2Lm97sHi3ocA6ESJ4=
github.com/oklog/ulid v1.3.1/go.mod h1:CirwcVhetQ6Lv90oh/F+FBtV6XMibvdAFo93nm5qn4U=
github.com/olekukonko/tablewriter v0.0.5 h1:P2Ga83D34wi1o9J6Wh1mRuqd4mF/x/lgBS7N7AbDhec=
github.com/olekukonko/tablewriter v0.0.5/go.mod h1:hPp6KlRPjbx+hW8ykQs1w3UBbZlj6HuIJcUGPhkA7kY=
github.com/onsi/ginkgo v1.16.5 h1:8xi0RTUf59SOSfEtZMvwTvXYMzG4gV23XVHOZiXNtnE=
github.com/onsi/ginkgo v1.16.5/go.mod h1:+E8gABHa3K6zRBolWtd+ROzc/U5bkGt0FwiG042wbpU=
github.com/onsi/gomega v1.36.2 h1:koNYke6TVk6ZmnyHrCXba/T/MoLBXFjeC1PtvYgw0A8=
Expand Down Expand Up @@ -1538,8 +1533,8 @@ github.com/oracle/oci-go-sdk/v65 v65.91.1/go.mod h1:u6XRPsw9tPziBh76K7GrrRXPa8P8
github.com/orisano/pixelmatch v0.0.0-20220722002657-fb0b55479cde/go.mod h1:nZgzbfBr3hhjoZnS66nKrHmduYNpc34ny7RK4z5/HM0=
github.com/ovh/go-ovh v1.6.0 h1:ixLOwxQdzYDx296sXcgS35TOPEahJkpjMGtzPadCjQI=
github.com/ovh/go-ovh v1.6.0/go.mod h1:cTVDnl94z4tl8pP1uZ/8jlVxntjSIf09bNcQ5TJSC7c=
github.com/parquet-go/parquet-go v0.25.0 h1:GwKy11MuF+al/lV6nUsFw8w8HCiPOSAx1/y8yFxjH5c=
github.com/parquet-go/parquet-go v0.25.0/go.mod h1:OqBBRGBl7+llplCvDMql8dEKaDqjaFA/VAPw+OJiNiw=
github.com/parquet-go/parquet-go v0.25.1 h1:l7jJwNM0xrk0cnIIptWMtnSnuxRkwq53S+Po3KG8Xgo=
github.com/parquet-go/parquet-go v0.25.1/go.mod h1:AXBuotO1XiBtcqJb/FKFyjBG4aqa3aQAAWF3ZPzCanY=
github.com/pascaldekloe/goe v0.0.0-20180627143212-57f6aae5913c/go.mod h1:lzWF7FIEvWOWxwDKqyGYQf6ZUaNfKdP144TG7ZOy1lc=
github.com/pascaldekloe/goe v0.1.0 h1:cBOtyMzM9HTpWjXfbbunk26uA6nG3a8n06Wieeh0MwY=
github.com/pascaldekloe/goe v0.1.0/go.mod h1:lzWF7FIEvWOWxwDKqyGYQf6ZUaNfKdP144TG7ZOy1lc=
Expand Down Expand Up @@ -1567,8 +1562,8 @@ github.com/posener/complete v1.1.1/go.mod h1:em0nMJCgc9GFtwrmVmEMR/ZL6WyhyjMBndr
github.com/posener/complete v1.2.3/go.mod h1:WZIdtGGp+qx0sLrYKtIRAruyNpv6hFCicSgv7Sy7s/s=
github.com/prashantv/gostub v1.1.0 h1:BTyx3RfQjRHnUWaGF9oQos79AlQ5k8WNktv7VGvVH4g=
github.com/prashantv/gostub v1.1.0/go.mod h1:A5zLQHz7ieHGG7is6LLXLz7I8+3LZzsrV0P1IAHhP5U=
github.com/prometheus-community/parquet-common v0.0.0-20250528231323-eec9c3c020f0 h1:XCSo9v3if0v0G+aAO/hSUr/Ck9KJXcUPzDFt1dJnAV8=
github.com/prometheus-community/parquet-common v0.0.0-20250528231323-eec9c3c020f0/go.mod h1:zRW/xXBlELf8v9h9uqWvDkjOr3N5BtQGZ6LsDX9Ea/A=
github.com/prometheus-community/parquet-common v0.0.0-20250606162055-b81ebb7e1b96 h1:5EbDNJOxTWGpe6yzXdgcBCU63BRSrRAh0Q1oB5AVyoA=
github.com/prometheus-community/parquet-common v0.0.0-20250606162055-b81ebb7e1b96/go.mod h1:MwYpD+FKot7LWBMFaPS6FeM8oqo77u5erRlNkSSFPA0=
github.com/prometheus-community/prom-label-proxy v0.11.0 h1:IO02WiiFMfcIqvjhwMbCYnDJiTNcSHBrkCGRQ/7KDd0=
github.com/prometheus-community/prom-label-proxy v0.11.0/go.mod h1:lfvrG70XqsxWDrSh1843QXBG0fSg8EbIXmAo8xGsvw8=
github.com/prometheus/alertmanager v0.28.1 h1:BK5pCoAtaKg01BYRUJhEDV1tqJMEtYBGzPw8QdvnnvA=
Expand Down Expand Up @@ -1614,9 +1609,6 @@ github.com/redis/rueidis v1.0.60 h1:MGZX8uNdw7iyWz22JhjA/9iXzddfCUE/EMK4VxKoKpA=
github.com/redis/rueidis v1.0.60/go.mod h1:Lkhr2QTgcoYBhxARU7kJRO8SyVlgUuEkcJO1Y8MCluA=
github.com/remyoudompheng/bigfft v0.0.0-20200410134404-eec4a21b6bb0/go.mod h1:qqbHyh8v60DhA7CoWK5oRCqLrMHRGoxYCSS9EjAz6Eo=
github.com/remyoudompheng/bigfft v0.0.0-20230129092748-24d4a6f8daec/go.mod h1:qqbHyh8v60DhA7CoWK5oRCqLrMHRGoxYCSS9EjAz6Eo=
github.com/rivo/uniseg v0.2.0/go.mod h1:J6wj4VEh+S6ZtnVlnTBMWIodfgj8LQOQFoIToxlJtxc=
github.com/rivo/uniseg v0.4.7 h1:WUdvkW8uEhrYfLC4ZzdpI2ztxP1I582+49Oc5Mq64VQ=
github.com/rivo/uniseg v0.4.7/go.mod h1:FN3SvrM+Zdj16jyLfmOkMNblXMcoc8DfTHruCPUcx88=
github.com/rogpeppe/fastuuid v1.2.0/go.mod h1:jVj6XXZzXRy/MSR5jhDC/2q6DgLz+nrA6LYCDYWNEvQ=
github.com/rogpeppe/go-internal v1.3.0/go.mod h1:M8bDsm7K2OlrFYOpmOWEs/qY81heoFRclV5y23lUDJ4=
github.com/rogpeppe/go-internal v1.6.1/go.mod h1:xXDCJY+GAPziupqXw64V24skbSoqbTEfhy4qGm1nDQc=
Expand Down
2 changes: 1 addition & 1 deletion pkg/ingester/ingester_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -6447,7 +6447,7 @@ func TestIngester_inflightPushRequests(t *testing.T) {

g, ctx := errgroup.WithContext(ctx)
g.Go(func() error {
count := 3500000
count := 150000
req := generateSamplesForLabel(labels.FromStrings(labels.MetricName, fmt.Sprintf("real-%d", count)), count, 1)
// Signal that we're going to do the real push now.
close(startCh)
Expand Down
102 changes: 69 additions & 33 deletions pkg/querier/parquet_queryable.go
Original file line number Diff line number Diff line change
Expand Up @@ -68,7 +68,7 @@ func getBlockStoreType(ctx context.Context, defaultBlockStoreType blockStoreType

type parquetQueryableFallbackMetrics struct {
blocksQueriedTotal *prometheus.CounterVec
selectCount *prometheus.CounterVec
operationsTotal *prometheus.CounterVec
}

func newParquetQueryableFallbackMetrics(reg prometheus.Registerer) *parquetQueryableFallbackMetrics {
Expand All @@ -77,10 +77,10 @@ func newParquetQueryableFallbackMetrics(reg prometheus.Registerer) *parquetQuery
Name: "cortex_parquet_queryable_blocks_queried_total",
Help: "Total number of blocks found to query.",
}, []string{"type"}),
selectCount: promauto.With(reg).NewCounterVec(prometheus.CounterOpts{
Name: "cortex_parquet_queryable_selects_queried_total",
Help: "Total number of selects.",
}, []string{"type"}),
operationsTotal: promauto.With(reg).NewCounterVec(prometheus.CounterOpts{
Name: "cortex_parquet_queryable_operations_total",
Help: "Total number of Operations.",
}, []string{"type", "method"}),
}
}

Expand Down Expand Up @@ -267,6 +267,7 @@ type parquetQuerierWithFallback struct {

func (q *parquetQuerierWithFallback) LabelValues(ctx context.Context, name string, hints *storage.LabelHints, matchers ...*labels.Matcher) ([]string, annotations.Annotations, error) {
remaining, parquet, err := q.getBlocks(ctx, q.minT, q.maxT)
defer q.incrementOpsMetric("LabelValues", remaining, parquet)
if err != nil {
return nil, nil, err
}
Expand Down Expand Up @@ -312,6 +313,7 @@ func (q *parquetQuerierWithFallback) LabelValues(ctx context.Context, name strin

func (q *parquetQuerierWithFallback) LabelNames(ctx context.Context, hints *storage.LabelHints, matchers ...*labels.Matcher) ([]string, annotations.Annotations, error) {
remaining, parquet, err := q.getBlocks(ctx, q.minT, q.maxT)
defer q.incrementOpsMetric("LabelNames", remaining, parquet)
if err != nil {
return nil, nil, err
}
Expand Down Expand Up @@ -356,7 +358,7 @@ func (q *parquetQuerierWithFallback) LabelNames(ctx context.Context, hints *stor
return result, rAnnotations, nil
}

func (q *parquetQuerierWithFallback) Select(ctx context.Context, sortSeries bool, hints *storage.SelectHints, matchers ...*labels.Matcher) storage.SeriesSet {
func (q *parquetQuerierWithFallback) Select(ctx context.Context, sortSeries bool, h *storage.SelectHints, matchers ...*labels.Matcher) storage.SeriesSet {
userID, err := tenant.TenantID(ctx)
if err != nil {
storage.ErrSeriesSet(err)
Expand All @@ -366,68 +368,101 @@ func (q *parquetQuerierWithFallback) Select(ctx context.Context, sortSeries bool
uLogger := util_log.WithUserID(userID, q.logger)
level.Warn(uLogger).Log("msg", "parquet queryable enabled but vertical sharding > 1. Falling back to the block storage")

return q.blocksStoreQuerier.Select(ctx, sortSeries, hints, matchers...)
return q.blocksStoreQuerier.Select(ctx, sortSeries, h, matchers...)
}

mint, maxt, limit := q.minT, q.maxT, 0
hints := storage.SelectHints{
Start: q.minT,
End: q.maxT,
}

if hints != nil {
mint, maxt, limit := q.minT, q.maxT, 0
if h != nil {
// let copy the hints here as we wanna potentially modify it
hints = *h
mint, maxt, limit = hints.Start, hints.End, hints.Limit
}

maxt = q.adjustMaxT(maxt)
hints.End = maxt

if maxt < mint {
return nil
}

remaining, parquet, err := q.getBlocks(ctx, mint, maxt)
defer q.incrementOpsMetric("Select", remaining, parquet)

if err != nil {
return storage.ErrSeriesSet(err)
}

serieSets := []storage.SeriesSet{}

// Lets sort the series to merge
if len(parquet) > 0 && len(remaining) > 0 {
sortSeries = true
}

promises := make([]chan storage.SeriesSet, 0, 2)

if len(parquet) > 0 {
serieSets = append(serieSets, q.parquetQuerier.Select(InjectBlocksIntoContext(ctx, parquet...), sortSeries, hints, matchers...))
p := make(chan storage.SeriesSet, 1)
promises = append(promises, p)
go func() {
p <- q.parquetQuerier.Select(InjectBlocksIntoContext(ctx, parquet...), sortSeries, &hints, matchers...)
}()
}

if len(remaining) > 0 {
serieSets = append(serieSets, q.blocksStoreQuerier.Select(InjectBlocksIntoContext(ctx, remaining...), sortSeries, hints, matchers...))
p := make(chan storage.SeriesSet, 1)
promises = append(promises, p)
go func() {
p <- q.blocksStoreQuerier.Select(InjectBlocksIntoContext(ctx, remaining...), sortSeries, &hints, matchers...)
}()
}

if len(serieSets) == 1 {
return serieSets[0]
if len(promises) == 1 {
return <-promises[0]
}

return storage.NewMergeSeriesSet(serieSets, limit, storage.ChainedSeriesMerge)
}
seriesSets := make([]storage.SeriesSet, len(promises))
for i, promise := range promises {
seriesSets[i] = <-promise
}

func (q *parquetQuerierWithFallback) Close() error {
mErr := multierror.MultiError{}
mErr.Add(q.parquetQuerier.Close())
mErr.Add(q.blocksStoreQuerier.Close())
return mErr.Err()
return storage.NewMergeSeriesSet(seriesSets, limit, storage.ChainedSeriesMerge)
}

func (q *parquetQuerierWithFallback) getBlocks(ctx context.Context, minT, maxT int64) ([]*bucketindex.Block, []*bucketindex.Block, error) {
func (q *parquetQuerierWithFallback) adjustMaxT(maxt int64) int64 {
// If queryStoreAfter is enabled, we do manipulate the query maxt to query samples up until
// now - queryStoreAfter, because the most recent time range is covered by ingesters. This
// optimization is particularly important for the blocks storage because can be used to skip
// querying most recent not-compacted-yet blocks from the storage.
if q.queryStoreAfter > 0 {
now := time.Now()
maxT = min(maxT, util.TimeToMillis(now.Add(-q.queryStoreAfter)))

if maxT < minT {
return nil, nil, nil
}
maxt = min(maxt, util.TimeToMillis(now.Add(-q.queryStoreAfter)))
}
return maxt
}

func (q *parquetQuerierWithFallback) Close() error {
mErr := multierror.MultiError{}
mErr.Add(q.parquetQuerier.Close())
mErr.Add(q.blocksStoreQuerier.Close())
return mErr.Err()
}

func (q *parquetQuerierWithFallback) getBlocks(ctx context.Context, minT, maxT int64) ([]*bucketindex.Block, []*bucketindex.Block, error) {
userID, err := tenant.TenantID(ctx)
if err != nil {
return nil, nil, err
}

maxT = q.adjustMaxT(maxT)

if maxT < minT {
return nil, nil, nil
}

blocks, _, err := q.finder.GetBlocks(ctx, userID, minT, maxT)
if err != nil {
return nil, nil, err
Expand All @@ -446,17 +481,18 @@ func (q *parquetQuerierWithFallback) getBlocks(ctx context.Context, minT, maxT i

q.metrics.blocksQueriedTotal.WithLabelValues("parquet").Add(float64(len(parquetBlocks)))
q.metrics.blocksQueriedTotal.WithLabelValues("tsdb").Add(float64(len(remaining)))
return remaining, parquetBlocks, nil
}

func (q *parquetQuerierWithFallback) incrementOpsMetric(method string, remaining []*bucketindex.Block, parquetBlocks []*bucketindex.Block) {
switch {
case len(remaining) > 0 && len(parquetBlocks) > 0:
q.metrics.selectCount.WithLabelValues("mixed").Inc()
q.metrics.operationsTotal.WithLabelValues("mixed", method).Inc()
case len(remaining) > 0 && len(parquetBlocks) == 0:
q.metrics.selectCount.WithLabelValues("tsdb").Inc()
q.metrics.operationsTotal.WithLabelValues("tsdb", method).Inc()
case len(remaining) == 0 && len(parquetBlocks) > 0:
q.metrics.selectCount.WithLabelValues("parquet").Inc()
q.metrics.operationsTotal.WithLabelValues("parquet", method).Inc()
}

return remaining, parquetBlocks, nil
}

type cacheInterface[T any] interface {
Expand Down
23 changes: 18 additions & 5 deletions pkg/querier/parquet_queryable_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -3,6 +3,7 @@ package querier
import (
"context"
"testing"
"time"

"github.com/go-kit/log"
"github.com/oklog/ulid"
Expand All @@ -19,6 +20,7 @@ import (
"github.com/cortexproject/cortex/pkg/querier/series"
"github.com/cortexproject/cortex/pkg/storage/parquet"
"github.com/cortexproject/cortex/pkg/storage/tsdb/bucketindex"
"github.com/cortexproject/cortex/pkg/util"
"github.com/cortexproject/cortex/pkg/util/flagext"
"github.com/cortexproject/cortex/pkg/util/validation"
)
Expand All @@ -27,7 +29,7 @@ func TestParquetQueryableFallbackLogic(t *testing.T) {
block1 := ulid.MustNew(1, nil)
block2 := ulid.MustNew(2, nil)
minT := int64(10)
maxT := int64(20)
maxT := util.TimeToMillis(time.Now())

createStore := func() *blocksStoreSetMock {
return &blocksStoreSetMock{mockedResponses: []interface{}{
Expand Down Expand Up @@ -124,13 +126,14 @@ func TestParquetQueryableFallbackLogic(t *testing.T) {
finder: finder,
blocksStoreQuerier: q,
parquetQuerier: mParquetQuerier,
queryStoreAfter: time.Hour,
metrics: newParquetQueryableFallbackMetrics(prometheus.NewRegistry()),
limits: defaultOverrides(t, 0),
logger: log.NewNopLogger(),
defaultBlockStoreType: parquetBlockStore,
}

finder.On("GetBlocks", mock.Anything, "user-1", minT, maxT).Return(bucketindex.Blocks{
finder.On("GetBlocks", mock.Anything, "user-1", minT, mock.Anything).Return(bucketindex.Blocks{
&bucketindex.Block{ID: block1},
&bucketindex.Block{ID: block2},
}, map[ulid.ULID]*bucketindex.BlockDeletionMark(nil), nil)
Expand Down Expand Up @@ -246,30 +249,39 @@ func TestParquetQueryableFallbackLogic(t *testing.T) {
}

mParquetQuerier := &mockParquetQuerier{}
queryStoreAfter := time.Hour
pq := &parquetQuerierWithFallback{
minT: minT,
maxT: maxT,
finder: finder,
blocksStoreQuerier: q,
parquetQuerier: mParquetQuerier,
queryStoreAfter: queryStoreAfter,
metrics: newParquetQueryableFallbackMetrics(prometheus.NewRegistry()),
limits: defaultOverrides(t, 0),
logger: log.NewNopLogger(),
defaultBlockStoreType: parquetBlockStore,
}

finder.On("GetBlocks", mock.Anything, "user-1", minT, maxT).Return(bucketindex.Blocks{
finder.On("GetBlocks", mock.Anything, "user-1", minT, mock.Anything).Return(bucketindex.Blocks{
&bucketindex.Block{ID: block1, Parquet: &parquet.ConverterMarkMeta{Version: 1}},
&bucketindex.Block{ID: block2, Parquet: &parquet.ConverterMarkMeta{Version: 1}},
}, map[ulid.ULID]*bucketindex.BlockDeletionMark(nil), nil)

t.Run("select", func(t *testing.T) {
stores.Reset()
mParquetQuerier.Reset()
ss := pq.Select(ctx, true, nil, matchers...)
hints := storage.SelectHints{
Start: minT,
End: maxT,
}
ss := pq.Select(ctx, true, &hints, matchers...)
require.NoError(t, ss.Err())
require.Len(t, stores.queriedBlocks, 0)
require.Len(t, mParquetQuerier.queriedBlocks, 2)
require.Equal(t, mParquetQuerier.queriedHints.Start, minT)
queriedDelta := time.Duration(maxT-mParquetQuerier.queriedHints.End) * time.Millisecond
require.InDeltaf(t, queriedDelta.Minutes(), queryStoreAfter.Minutes(), 0.1, "query after not set")
})

t.Run("labelNames", func(t *testing.T) {
Expand Down Expand Up @@ -409,13 +421,14 @@ func defaultOverrides(t *testing.T, queryVerticalShardSize int) *validation.Over

type mockParquetQuerier struct {
queriedBlocks []*bucketindex.Block
queriedHints *storage.SelectHints
}

func (m *mockParquetQuerier) Select(ctx context.Context, sortSeries bool, sp *storage.SelectHints, matchers ...*labels.Matcher) storage.SeriesSet {
if blocks, ok := ExtractBlocksFromContext(ctx); ok {
m.queriedBlocks = append(m.queriedBlocks, blocks...)
}

m.queriedHints = sp
return series.NewConcreteSeriesSet(sortSeries, nil)
}

Expand Down
Loading
Loading