diff --git a/integration/parquet_querier_test.go b/integration/parquet_querier_test.go index ca31a019c9a..69095d386fc 100644 --- a/integration/parquet_querier_test.go +++ b/integration/parquet_querier_test.go @@ -63,8 +63,9 @@ func TestParquetFuzz(t *testing.T) { "-store-gateway.sharding-enabled": "false", "--querier.store-gateway-addresses": "nonExistent", // Make sure we do not call Store gateways // alert manager - "-alertmanager.web.external-url": "http://localhost/alertmanager", - "-frontend.query-vertical-shard-size": "1", + "-alertmanager.web.external-url": "http://localhost/alertmanager", + // Enable vertical sharding. + "-frontend.query-vertical-shard-size": "3", "-frontend.max-cache-freshness": "1m", // enable experimental promQL funcs "-querier.enable-promql-experimental-functions": "true", diff --git a/pkg/querier/parquet_queryable.go b/pkg/querier/parquet_queryable.go index 8d7fe7152ed..520438c5414 100644 --- a/pkg/querier/parquet_queryable.go +++ b/pkg/querier/parquet_queryable.go @@ -6,13 +6,13 @@ import ( "time" "github.com/go-kit/log" - "github.com/go-kit/log/level" lru "github.com/hashicorp/golang-lru/v2" "github.com/opentracing/opentracing-go" "github.com/parquet-go/parquet-go" "github.com/pkg/errors" "github.com/prometheus-community/parquet-common/queryable" "github.com/prometheus-community/parquet-common/schema" + "github.com/prometheus-community/parquet-common/search" parquet_storage "github.com/prometheus-community/parquet-common/storage" "github.com/prometheus/client_golang/prometheus" "github.com/prometheus/client_golang/prometheus/promauto" @@ -20,17 +20,18 @@ import ( "github.com/prometheus/prometheus/storage" "github.com/prometheus/prometheus/tsdb/chunkenc" "github.com/prometheus/prometheus/util/annotations" + "github.com/thanos-io/thanos/pkg/store/storepb" "github.com/thanos-io/thanos/pkg/strutil" "golang.org/x/sync/errgroup" "github.com/cortexproject/cortex/pkg/cortexpb" + "github.com/cortexproject/cortex/pkg/querysharding" "github.com/cortexproject/cortex/pkg/storage/bucket" cortex_tsdb "github.com/cortexproject/cortex/pkg/storage/tsdb" "github.com/cortexproject/cortex/pkg/storage/tsdb/bucketindex" "github.com/cortexproject/cortex/pkg/tenant" "github.com/cortexproject/cortex/pkg/util" "github.com/cortexproject/cortex/pkg/util/limiter" - util_log "github.com/cortexproject/cortex/pkg/util/log" "github.com/cortexproject/cortex/pkg/util/multierror" "github.com/cortexproject/cortex/pkg/util/services" "github.com/cortexproject/cortex/pkg/util/validation" @@ -153,6 +154,7 @@ func NewParquetQueryable( userID, _ := tenant.TenantID(ctx) return int64(limits.ParquetMaxFetchedDataBytes(userID)) }), + queryable.WithMaterializedLabelsFilterCallback(materializedLabelsFilterCallback), queryable.WithMaterializedSeriesCallback(func(ctx context.Context, cs []storage.ChunkSeries) error { queryLimiter := limiter.QueryLimiterFromContextWithFallback(ctx) lbls := make([][]cortexpb.LabelAdapter, 0, len(cs)) @@ -432,17 +434,11 @@ func (q *parquetQuerierWithFallback) Select(ctx context.Context, sortSeries bool span, ctx := opentracing.StartSpanFromContext(ctx, "parquetQuerierWithFallback.Select") defer span.Finish() - userID, err := tenant.TenantID(ctx) + newMatchers, shardMatcher, err := querysharding.ExtractShardingMatchers(matchers) if err != nil { return storage.ErrSeriesSet(err) } - - if q.limits.QueryVerticalShardSize(userID) > 1 { - 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, h, matchers...) - } + defer shardMatcher.Close() hints := storage.SelectHints{ Start: q.minT, @@ -483,7 +479,11 @@ func (q *parquetQuerierWithFallback) Select(ctx context.Context, sortSeries bool go func() { span, _ := opentracing.StartSpanFromContext(ctx, "parquetQuerier.Select") defer span.Finish() - p <- q.parquetQuerier.Select(InjectBlocksIntoContext(ctx, parquet...), sortSeries, &hints, matchers...) + parquetCtx := InjectBlocksIntoContext(ctx, parquet...) + if shardMatcher != nil { + parquetCtx = injectShardMatcherIntoContext(parquetCtx, shardMatcher) + } + p <- q.parquetQuerier.Select(parquetCtx, sortSeries, &hints, newMatchers...) }() } @@ -570,6 +570,26 @@ func (q *parquetQuerierWithFallback) incrementOpsMetric(method string, remaining } } +type shardMatcherLabelsFilter struct { + shardMatcher *storepb.ShardMatcher +} + +func (f *shardMatcherLabelsFilter) Filter(lbls labels.Labels) bool { + return f.shardMatcher.MatchesLabels(lbls) +} + +func (f *shardMatcherLabelsFilter) Close() { + f.shardMatcher.Close() +} + +func materializedLabelsFilterCallback(ctx context.Context, _ *storage.SelectHints) (search.MaterializedLabelsFilter, bool) { + shardMatcher, exists := extractShardMatcherFromContext(ctx) + if !exists || !shardMatcher.IsSharded() { + return nil, false + } + return &shardMatcherLabelsFilter{shardMatcher: shardMatcher}, true +} + type cacheInterface[T any] interface { Get(path string) T Set(path string, reader T) @@ -655,3 +675,19 @@ func (n noopCache[T]) Get(_ string) (r T) { func (n noopCache[T]) Set(_ string, _ T) { } + +var ( + shardMatcherCtxKey contextKey = 1 +) + +func injectShardMatcherIntoContext(ctx context.Context, sm *storepb.ShardMatcher) context.Context { + return context.WithValue(ctx, shardMatcherCtxKey, sm) +} + +func extractShardMatcherFromContext(ctx context.Context) (*storepb.ShardMatcher, bool) { + if sm := ctx.Value(shardMatcherCtxKey); sm != nil { + return sm.(*storepb.ShardMatcher), true + } + + return nil, false +} diff --git a/pkg/querier/parquet_queryable_test.go b/pkg/querier/parquet_queryable_test.go index 13cdde6cd57..73f7c50af21 100644 --- a/pkg/querier/parquet_queryable_test.go +++ b/pkg/querier/parquet_queryable_test.go @@ -5,6 +5,7 @@ import ( "fmt" "math/rand" "path/filepath" + "sync" "testing" "time" @@ -75,49 +76,6 @@ func TestParquetQueryableFallbackLogic(t *testing.T) { } ctx := user.InjectOrgID(context.Background(), "user-1") - t.Run("should fallback when vertical sharding is enabled", func(t *testing.T) { - finder := &blocksFinderMock{} - stores := createStore() - - q := &blocksStoreQuerier{ - minT: minT, - maxT: maxT, - finder: finder, - stores: stores, - consistency: NewBlocksConsistencyChecker(0, 0, log.NewNopLogger(), nil), - logger: log.NewNopLogger(), - metrics: newBlocksStoreQueryableMetrics(prometheus.NewPedanticRegistry()), - limits: &blocksStoreLimitsMock{}, - - storeGatewayConsistencyCheckMaxAttempts: 3, - } - - mParquetQuerier := &mockParquetQuerier{} - pq := &parquetQuerierWithFallback{ - minT: minT, - maxT: maxT, - finder: finder, - blocksStoreQuerier: q, - parquetQuerier: mParquetQuerier, - metrics: newParquetQueryableFallbackMetrics(prometheus.NewRegistry()), - limits: defaultOverrides(t, 4), - logger: log.NewNopLogger(), - defaultBlockStoreType: parquetBlockStore, - } - - finder.On("GetBlocks", mock.Anything, "user-1", minT, maxT).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) { - ss := pq.Select(ctx, true, nil, matchers...) - require.NoError(t, ss.Err()) - require.Len(t, stores.queriedBlocks, 2) - require.Len(t, mParquetQuerier.queriedBlocks, 0) - }) - }) - t.Run("should fallback all blocks", func(t *testing.T) { finder := &blocksFinderMock{} stores := createStore() @@ -671,3 +629,90 @@ func (m *mockParquetQuerier) Reset() { func (mockParquetQuerier) Close() error { return nil } + +func TestMaterializedLabelsFilterCallback(t *testing.T) { + tests := []struct { + name string + setupContext func() context.Context + expectedFilterReturned bool + expectedCallbackReturned bool + }{ + { + name: "no shard matcher in context", + setupContext: func() context.Context { + return context.Background() + }, + expectedFilterReturned: false, + expectedCallbackReturned: false, + }, + { + name: "shard matcher exists but is not sharded", + setupContext: func() context.Context { + // Create a ShardInfo with TotalShards = 0 (not sharded) + shardInfo := &storepb.ShardInfo{ + ShardIndex: 0, + TotalShards: 0, // Not sharded + By: true, + Labels: []string{"__name__"}, + } + + buffers := &sync.Pool{New: func() interface{} { + b := make([]byte, 0, 100) + return &b + }} + shardMatcher := shardInfo.Matcher(buffers) + + return injectShardMatcherIntoContext(context.Background(), shardMatcher) + }, + expectedFilterReturned: false, + expectedCallbackReturned: false, + }, + { + name: "shard matcher exists and is sharded", + setupContext: func() context.Context { + // Create a ShardInfo with TotalShards > 0 (sharded) + shardInfo := &storepb.ShardInfo{ + ShardIndex: 0, + TotalShards: 2, // Sharded + By: true, + Labels: []string{"__name__"}, + } + + buffers := &sync.Pool{New: func() interface{} { + b := make([]byte, 0, 100) + return &b + }} + shardMatcher := shardInfo.Matcher(buffers) + + return injectShardMatcherIntoContext(context.Background(), shardMatcher) + }, + expectedFilterReturned: true, + expectedCallbackReturned: true, + }, + } + + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + ctx := tt.setupContext() + + filter, exists := materializedLabelsFilterCallback(ctx, nil) + + require.Equal(t, tt.expectedCallbackReturned, exists) + + if tt.expectedFilterReturned { + require.NotNil(t, filter) + + // Test that the filter can be used + testLabels := labels.FromStrings("__name__", "test_metric", "label1", "value1") + // We can't easily test the actual filtering logic without knowing the internal + // shard matching implementation, but we can at least verify the filter interface works + _ = filter.Filter(testLabels) + + // Cleanup + filter.Close() + } else { + require.Nil(t, filter) + } + }) + } +}