From 4a25bc336a9ef64cf5df3dcae1249d0388198d09 Mon Sep 17 00:00:00 2001 From: alanprot Date: Tue, 13 May 2025 23:01:08 -0700 Subject: [PATCH 01/12] Implementing Parquet Queryable with fallback Signed-off-by: alanprot --- go.mod | 2 +- go.sum | 4 +- pkg/querier/parquet_queryable.go | 349 ++++++++++ .../parquet-common/convert/convert.go | 17 +- .../parquet-common/convert/merge.go | 76 +++ .../parquet-common/convert/writer.go | 7 +- .../parquet-common/schema/encoder.go | 207 ++++-- .../parquet-common/schema/schema.go | 3 +- .../parquet-common/schema/schema_builder.go | 40 ++ .../parquet-common/search/constraint.go | 610 ++++++++++++++++++ .../parquet-common/search/materialize.go | 511 +++++++++++++++ .../search/parquet_queriable.go | 281 ++++++++ .../parquet-common/search/rowrange.go | 166 +++++ .../parquet-common/storage/bucket_read_at.go | 61 ++ .../parquet-common/storage/parquet_shard.go | 96 +++ .../parquet-common/util/bucket_read_at.go | 35 - .../parquet-common/util/strutil.go | 84 +++ .../parquet-common/util/util.go | 82 ++- vendor/modules.txt | 4 +- 19 files changed, 2528 insertions(+), 107 deletions(-) create mode 100644 pkg/querier/parquet_queryable.go create mode 100644 vendor/github.com/prometheus-community/parquet-common/search/constraint.go create mode 100644 vendor/github.com/prometheus-community/parquet-common/search/materialize.go create mode 100644 vendor/github.com/prometheus-community/parquet-common/search/parquet_queriable.go create mode 100644 vendor/github.com/prometheus-community/parquet-common/search/rowrange.go create mode 100644 vendor/github.com/prometheus-community/parquet-common/storage/bucket_read_at.go create mode 100644 vendor/github.com/prometheus-community/parquet-common/storage/parquet_shard.go delete mode 100644 vendor/github.com/prometheus-community/parquet-common/util/bucket_read_at.go create mode 100644 vendor/github.com/prometheus-community/parquet-common/util/strutil.go diff --git a/go.mod b/go.mod index 0001b75b255..0c282404820 100644 --- a/go.mod +++ b/go.mod @@ -82,7 +82,7 @@ require ( 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-20250428074311-306c8486441d + github.com/prometheus-community/parquet-common v0.0.0-20250514003255-382b6ec8ae40 github.com/prometheus/procfs v0.15.1 github.com/sercand/kuberesolver/v5 v5.1.1 github.com/tjhop/slog-gokit v0.1.3 diff --git a/go.sum b/go.sum index 5e8f2f5afb8..67e199c927b 100644 --- a/go.sum +++ b/go.sum @@ -1573,8 +1573,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-20250428074311-306c8486441d h1:j7d62fP5x6yUFNgNDth5JCLOoj6ZclXkBneSATbPZig= -github.com/prometheus-community/parquet-common v0.0.0-20250428074311-306c8486441d/go.mod h1:Eo3B53ZLcfCEV06clM4UIFTgwxRXm0BHdiaRslKe3Y8= +github.com/prometheus-community/parquet-common v0.0.0-20250514003255-382b6ec8ae40 h1:45NOJV7a7QGKg7rITB8wCs/f5O4bpe3mPAiTFsqcX8s= +github.com/prometheus-community/parquet-common v0.0.0-20250514003255-382b6ec8ae40/go.mod h1:zRW/xXBlELf8v9h9uqWvDkjOr3N5BtQGZ6LsDX9Ea/A= github.com/prometheus-community/prom-label-proxy v0.8.1-0.20240127162815-c1195f9aabc0 h1:owfYHh79h8Y5HvNMGyww+DaVwo10CKiRW1RQrrZzIwg= github.com/prometheus-community/prom-label-proxy v0.8.1-0.20240127162815-c1195f9aabc0/go.mod h1:rT989D4UtOcfd9tVqIZRVIM8rkg+9XbreBjFNEKXvVI= github.com/prometheus/alertmanager v0.28.1 h1:BK5pCoAtaKg01BYRUJhEDV1tqJMEtYBGzPw8QdvnnvA= diff --git a/pkg/querier/parquet_queryable.go b/pkg/querier/parquet_queryable.go new file mode 100644 index 00000000000..0ae35f364d0 --- /dev/null +++ b/pkg/querier/parquet_queryable.go @@ -0,0 +1,349 @@ +package querier + +import ( + "context" + "fmt" + "time" + + "github.com/go-kit/log" + "github.com/pkg/errors" + "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/prometheus/model/labels" + "github.com/prometheus/prometheus/storage" + "github.com/prometheus/prometheus/util/annotations" + "github.com/thanos-io/thanos/pkg/strutil" + + "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/multierror" + "github.com/cortexproject/cortex/pkg/util/services" +) + +type contextKey int + +var ( + blockIdsCtxKey contextKey = 0 +) + +type parquetQueryableWithFallback struct { + services.Service + + queryStoreAfter time.Duration + parquetQueryable storage.Queryable + blockStorageQueryable BlocksStoreQueryable + + finder BlocksFinder + + // Subservices manager. + subservices *services.Manager + subservicesWatcher *services.FailureWatcher +} + +func newParquetQueryable( + storageCfg cortex_tsdb.BlocksStorageConfig, + limits BlocksStoreLimits, + config Config, + blockStorageQueryable BlocksStoreQueryable, + logger log.Logger, + reg prometheus.Registerer, +) (storage.Queryable, error) { + bucketClient, err := bucket.NewClient(context.Background(), storageCfg.Bucket, nil, "parquet-querier", logger, reg) + + if err != nil { + return nil, err + } + + // Create the blocks finder. + var finder BlocksFinder + if storageCfg.BucketStore.BucketIndex.Enabled { + finder = NewBucketIndexBlocksFinder(BucketIndexBlocksFinderConfig{ + IndexLoader: bucketindex.LoaderConfig{ + CheckInterval: time.Minute, + UpdateOnStaleInterval: storageCfg.BucketStore.SyncInterval, + UpdateOnErrorInterval: storageCfg.BucketStore.BucketIndex.UpdateOnErrorInterval, + IdleTimeout: storageCfg.BucketStore.BucketIndex.IdleTimeout, + }, + MaxStalePeriod: storageCfg.BucketStore.BucketIndex.MaxStalePeriod, + IgnoreDeletionMarksDelay: storageCfg.BucketStore.IgnoreDeletionMarksDelay, + IgnoreBlocksWithin: storageCfg.BucketStore.IgnoreBlocksWithin, + }, bucketClient, limits, logger, reg) + } else { + finder = NewBucketScanBlocksFinder(BucketScanBlocksFinderConfig{ + ScanInterval: storageCfg.BucketStore.SyncInterval, + TenantsConcurrency: storageCfg.BucketStore.TenantSyncConcurrency, + MetasConcurrency: storageCfg.BucketStore.MetaSyncConcurrency, + CacheDir: storageCfg.BucketStore.SyncDir, + IgnoreDeletionMarksDelay: storageCfg.BucketStore.IgnoreDeletionMarksDelay, + IgnoreBlocksWithin: storageCfg.BucketStore.IgnoreBlocksWithin, + BlockDiscoveryStrategy: storageCfg.BucketStore.BlockDiscoveryStrategy, + }, bucketClient, limits, logger, reg) + } + + manager, err := services.NewManager(finder) + if err != nil { + return nil, err + } + + pq, err := search.NewParquetQueryable(nil, func(ctx context.Context, mint, maxt int64) ([]*parquet_storage.ParquetShard, error) { + userID, err := tenant.TenantID(ctx) + if err != nil { + return nil, err + } + + blocks := ctx.Value(blockIdsCtxKey).([]*bucketindex.Block) + userBkt := bucket.NewUserBucketClient(userID, bucketClient, limits) + + shards := make([]*parquet_storage.ParquetShard, 0, len(blocks)) + + for _, block := range blocks { + blockName := fmt.Sprintf("%v/block", block.ID.String()) + shard, err := parquet_storage.OpenParquetShard(ctx, userBkt, blockName, 0) + if err != nil { + return nil, err + } + shards = append(shards, shard) + } + + return shards, nil + }) + + q := &parquetQueryableWithFallback{ + subservices: manager, + blockStorageQueryable: blockStorageQueryable, + parquetQueryable: pq, + queryStoreAfter: config.QueryStoreAfter, + subservicesWatcher: services.NewFailureWatcher(), + finder: finder, + } + + q.Service = services.NewBasicService(q.starting, q.running, q.stopping) + + return pq, nil +} + +func (p *parquetQueryableWithFallback) starting(ctx context.Context) error { + p.subservicesWatcher.WatchManager(p.subservices) + if err := services.StartManagerAndAwaitHealthy(ctx, p.subservices); err != nil { + return errors.Wrap(err, "unable to start blocks storage queryable subservices") + } + return nil +} + +func (p *parquetQueryableWithFallback) running(ctx context.Context) error { + for { + select { + case <-ctx.Done(): + return nil + case err := <-p.subservicesWatcher.Chan(): + return errors.Wrap(err, "block storage queryable subservice failed") + } + } +} + +func (p *parquetQueryableWithFallback) stopping(_ error) error { + return services.StopManagerAndAwaitStopped(context.Background(), p.subservices) +} + +func (p *parquetQueryableWithFallback) Querier(mint, maxt int64) (storage.Querier, error) { + pq, err := p.parquetQueryable.Querier(mint, maxt) + if err != nil { + return nil, err + } + + bsq, err := p.blockStorageQueryable.Querier(mint, maxt) + if err != nil { + return nil, err + } + + return &parquetQuerier{ + minT: mint, + maxT: maxt, + parquetQuerier: pq, + queryStoreAfter: p.queryStoreAfter, + blocksStoreQuerier: bsq, + finder: p.finder, + }, nil +} + +type parquetQuerier struct { + minT, maxT int64 + + finder BlocksFinder + + parquetQuerier storage.Querier + blocksStoreQuerier storage.Querier + + // If set, the querier manipulates the max time to not be greater than + // "now - queryStoreAfter" so that most recent blocks are not queried. + queryStoreAfter time.Duration +} + +func (q *parquetQuerier) 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) + if err != nil { + return nil, nil, err + } + + limit := 0 + + if hints != nil { + limit = hints.Limit + } + + var ( + result []string + rAnnotations annotations.Annotations + ) + + if len(parquet) > 0 { + res, ann, qErr := q.parquetQuerier.LabelValues(context.WithValue(ctx, blockIdsCtxKey, parquet), name, hints, matchers...) + if qErr != nil { + return nil, nil, err + } + result = res + rAnnotations = ann + } + + if len(remaining) > 0 { + res, ann, qErr := q.blocksStoreQuerier.LabelValues(context.WithValue(ctx, blockIdsCtxKey, remaining), name, hints, matchers...) + if qErr != nil { + return nil, nil, err + } + + if len(result) == 0 { + result = res + } else { + result = strutil.MergeSlices(limit, result, res) + } + + if rAnnotations != nil { + rAnnotations = rAnnotations.Merge(ann) + } + } + + return result, rAnnotations, nil +} + +func (q *parquetQuerier) LabelNames(ctx context.Context, hints *storage.LabelHints, matchers ...*labels.Matcher) ([]string, annotations.Annotations, error) { + remaining, parquet, err := q.getBlocks(ctx, q.minT, q.maxT) + if err != nil { + return nil, nil, err + } + + limit := 0 + + if hints != nil { + limit = hints.Limit + } + + var ( + result []string + rAnnotations annotations.Annotations + ) + + if len(parquet) > 0 { + res, ann, qErr := q.parquetQuerier.LabelNames(context.WithValue(ctx, blockIdsCtxKey, parquet), hints, matchers...) + if qErr != nil { + return nil, nil, err + } + result = res + rAnnotations = ann + } + + if len(remaining) > 0 { + res, ann, qErr := q.blocksStoreQuerier.LabelNames(context.WithValue(ctx, blockIdsCtxKey, remaining), hints, matchers...) + if qErr != nil { + return nil, nil, err + } + + if len(result) == 0 { + result = res + } else { + result = strutil.MergeSlices(limit, result, res) + } + + if rAnnotations != nil { + rAnnotations = rAnnotations.Merge(ann) + } + } + + return result, rAnnotations, nil +} + +func (q *parquetQuerier) Select(ctx context.Context, sortSeries bool, hints *storage.SelectHints, matchers ...*labels.Matcher) storage.SeriesSet { + mint, maxt, limit := q.minT, q.maxT, 0 + + if hints != nil { + mint, maxt, limit = hints.Start, hints.End, hints.Limit + } + + remaining, parquet, err := q.getBlocks(ctx, mint, maxt) + if err != nil { + return storage.ErrSeriesSet(err) + } + + serieSets := []storage.SeriesSet{} + + if len(parquet) > 0 { + serieSets = append(serieSets, q.parquetQuerier.Select(context.WithValue(ctx, blockIdsCtxKey, parquet), sortSeries, hints, matchers...)) + } + + if len(remaining) > 0 { + serieSets = append(serieSets, q.blocksStoreQuerier.Select(context.WithValue(ctx, blockIdsCtxKey, remaining), sortSeries, hints, matchers...)) + } + + if len(serieSets) == 1 { + return serieSets[0] + } + + return storage.NewMergeSeriesSet(serieSets, limit, storage.ChainedSeriesMerge) +} + +func (q *parquetQuerier) Close() error { + mErr := multierror.MultiError{} + mErr.Add(q.parquetQuerier.Close()) + mErr.Add(q.blocksStoreQuerier.Close()) + return mErr.Err() +} + +func (q *parquetQuerier) getBlocks(ctx context.Context, minT, maxT int64) ([]*bucketindex.Block, []*bucketindex.Block, error) { + // 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 + } + } + + userID, err := tenant.TenantID(ctx) + if err != nil { + return nil, nil, err + } + + blocks, _, err := q.finder.GetBlocks(ctx, userID, minT, maxT) + if err != nil { + return nil, nil, err + } + + parquetBlocks := make([]*bucketindex.Block, 0, len(blocks)) + remaining := make([]*bucketindex.Block, 0, len(blocks)) + for _, b := range blocks { + if b.Parquet != nil { + parquetBlocks = append(parquetBlocks, b) + continue + } + remaining = append(remaining, b) + } + + return remaining, parquetBlocks, nil +} diff --git a/vendor/github.com/prometheus-community/parquet-common/convert/convert.go b/vendor/github.com/prometheus-community/parquet-common/convert/convert.go index 10e426a241c..d4862646a69 100644 --- a/vendor/github.com/prometheus-community/parquet-common/convert/convert.go +++ b/vendor/github.com/prometheus-community/parquet-common/convert/convert.go @@ -23,16 +23,17 @@ import ( "strings" "time" + "github.com/efficientgo/core/errors" "github.com/hashicorp/go-multierror" "github.com/parquet-go/parquet-go" - "github.com/pkg/errors" - "github.com/prometheus-community/parquet-common/schema" "github.com/prometheus/prometheus/model/labels" "github.com/prometheus/prometheus/storage" "github.com/prometheus/prometheus/tsdb" "github.com/prometheus/prometheus/tsdb/index" "github.com/prometheus/prometheus/tsdb/tombstones" "github.com/thanos-io/objstore" + + "github.com/prometheus-community/parquet-common/schema" ) var DefaultConvertOpts = convertOpts{ @@ -119,6 +120,18 @@ func WithName(name string) ConvertOption { } } +func WithNumRowGroups(n int) ConvertOption { + return func(opts *convertOpts) { + opts.numRowGroups = n + } +} + +func WithRowGroupSize(size int) ConvertOption { + return func(opts *convertOpts) { + opts.rowGroupSize = size + } +} + func WithConcurrency(concurrency int) ConvertOption { return func(opts *convertOpts) { opts.concurrency = concurrency diff --git a/vendor/github.com/prometheus-community/parquet-common/convert/merge.go b/vendor/github.com/prometheus-community/parquet-common/convert/merge.go index d6f40ed0f74..90db2ec54eb 100644 --- a/vendor/github.com/prometheus-community/parquet-common/convert/merge.go +++ b/vendor/github.com/prometheus-community/parquet-common/convert/merge.go @@ -16,6 +16,8 @@ package convert import ( "container/heap" + "github.com/prometheus/prometheus/tsdb/chunkenc" + "github.com/prometheus/prometheus/model/labels" "github.com/prometheus/prometheus/storage" "github.com/prometheus/prometheus/util/annotations" @@ -124,3 +126,77 @@ func (h *heapChunkSeries) Pop() any { h.heap = old[0 : n-1] return x } + +func NewChunksSeriesSet(series []storage.ChunkSeries) storage.ChunkSeriesSet { + return &concreteChunkSeries{ + series: series, + curr: -1, + } +} + +type concreteChunkSeries struct { + series []storage.ChunkSeries + curr int +} + +func (c *concreteChunkSeries) Next() bool { + c.curr++ + return c.curr < len(c.series) +} + +func (c *concreteChunkSeries) At() storage.ChunkSeries { + return c.series[c.curr] +} + +func (c *concreteChunkSeries) Err() error { + return nil +} + +func (c *concreteChunkSeries) Warnings() annotations.Annotations { + return nil +} + +// NewSeriesSetFromChunkSeriesSet is an adaptor to transform ChunkSeriesSet to SeriesSet when the +func NewSeriesSetFromChunkSeriesSet(ss storage.ChunkSeriesSet, skipChunks bool) storage.SeriesSet { + if skipChunks { + return seriesSetFromChunksSeriesSet{chunkSeriesSet: ss} + } + return storage.NewSeriesSetFromChunkSeriesSet(ss) +} + +var ( + _ storage.SeriesSet = (*seriesSetFromChunksSeriesSet)(nil) + _ storage.Series = (*seriesSetFromChunksSeries)(nil) +) + +type seriesSetFromChunksSeries struct { + chunkSeries storage.ChunkSeries +} + +func (s seriesSetFromChunksSeries) Labels() labels.Labels { + return s.chunkSeries.Labels() +} + +func (s seriesSetFromChunksSeries) Iterator(_ chunkenc.Iterator) chunkenc.Iterator { + return storage.NewListSeriesIterator(nil) +} + +type seriesSetFromChunksSeriesSet struct { + chunkSeriesSet storage.ChunkSeriesSet +} + +func (s seriesSetFromChunksSeriesSet) Next() bool { + return s.chunkSeriesSet.Next() +} + +func (s seriesSetFromChunksSeriesSet) At() storage.Series { + return &seriesSetFromChunksSeries{chunkSeries: s.chunkSeriesSet.At()} +} + +func (s seriesSetFromChunksSeriesSet) Err() error { + return nil +} + +func (s seriesSetFromChunksSeriesSet) Warnings() annotations.Annotations { + return nil +} diff --git a/vendor/github.com/prometheus-community/parquet-common/convert/writer.go b/vendor/github.com/prometheus-community/parquet-common/convert/writer.go index 2f20a32d076..e182bf3c980 100644 --- a/vendor/github.com/prometheus-community/parquet-common/convert/writer.go +++ b/vendor/github.com/prometheus-community/parquet-common/convert/writer.go @@ -18,14 +18,15 @@ import ( "fmt" "io" + "github.com/efficientgo/core/errors" "github.com/hashicorp/go-multierror" "github.com/parquet-go/parquet-go" - "github.com/pkg/errors" - "github.com/prometheus-community/parquet-common/schema" - "github.com/prometheus-community/parquet-common/util" "github.com/prometheus/prometheus/util/zeropool" "github.com/thanos-io/objstore" "golang.org/x/sync/errgroup" + + "github.com/prometheus-community/parquet-common/schema" + "github.com/prometheus-community/parquet-common/util" ) type ShardedWriter struct { diff --git a/vendor/github.com/prometheus-community/parquet-common/schema/encoder.go b/vendor/github.com/prometheus-community/parquet-common/schema/encoder.go index a858a223097..6b6556a4c3f 100644 --- a/vendor/github.com/prometheus-community/parquet-common/schema/encoder.go +++ b/vendor/github.com/prometheus-community/parquet-common/schema/encoder.go @@ -17,6 +17,7 @@ import ( "bytes" "encoding/binary" "fmt" + "io" "math" "slices" "sort" @@ -49,19 +50,36 @@ func (e *PrometheusParquetChunksEncoder) Encode(it chunks.Iterator) ([][]byte, e dataColSize := len(e.schema.DataColsIndexes) - reEncodedChunks := make([]chunks.Meta, dataColSize) - reEncodedChunksAppenders := make([]chunkenc.Appender, dataColSize) + reEncodedChunks := make([]map[chunkenc.Encoding][]*chunks.Meta, dataColSize) + reEncodedChunksAppenders := make([]map[chunkenc.Encoding]chunkenc.Appender, dataColSize) for i := 0; i < dataColSize; i++ { - reEncodedChunks[i] = chunks.Meta{ - Chunk: chunkenc.NewXORChunk(), - MinTime: math.MaxInt64, - } - app, err := reEncodedChunks[i].Chunk.Appender() - if err != nil { - return nil, err + reEncodedChunks[i] = make(map[chunkenc.Encoding][]*chunks.Meta) + reEncodedChunksAppenders[i] = make(map[chunkenc.Encoding]chunkenc.Appender) + + for _, enc := range []chunkenc.Encoding{chunkenc.EncXOR, chunkenc.EncHistogram, chunkenc.EncFloatHistogram} { + var chunk chunkenc.Chunk + switch enc { + case chunkenc.EncXOR: + chunk = chunkenc.NewXORChunk() + case chunkenc.EncHistogram: + chunk = chunkenc.NewHistogramChunk() + case chunkenc.EncFloatHistogram: + chunk = chunkenc.NewFloatHistogramChunk() + default: + return nil, fmt.Errorf("unknown encoding %v", enc) + } + + reEncodedChunks[i][enc] = append(reEncodedChunks[i][enc], &chunks.Meta{ + Chunk: chunk, + MinTime: math.MaxInt64, + }) + app, err := reEncodedChunks[i][enc][0].Chunk.Appender() + if err != nil { + return nil, err + } + reEncodedChunksAppenders[i][enc] = app } - reEncodedChunksAppenders[i] = app } var sampleIt chunkenc.Iterator @@ -70,42 +88,103 @@ func (e *PrometheusParquetChunksEncoder) Encode(it chunks.Iterator) ([][]byte, e switch chk.Chunk.Encoding() { case chunkenc.EncXOR: for vt := sampleIt.Next(); vt != chunkenc.ValNone; vt = sampleIt.Next() { - // TODO: Native histograms support if vt != chunkenc.ValFloat { return nil, fmt.Errorf("found value type %v in float chunk", vt) } t, v := sampleIt.At() chkIdx := e.schema.DataColumIdx(t) - reEncodedChunksAppenders[chkIdx].Append(t, v) - if t < reEncodedChunks[chkIdx].MinTime { - reEncodedChunks[chkIdx].MinTime = t + reEncodedChunksAppenders[chkIdx][chunkenc.EncXOR].Append(t, v) + if t < reEncodedChunks[chkIdx][chunkenc.EncXOR][len(reEncodedChunks[chkIdx][chunkenc.EncXOR])-1].MinTime { + reEncodedChunks[chkIdx][chunkenc.EncXOR][len(reEncodedChunks[chkIdx][chunkenc.EncXOR])-1].MinTime = t + } + if t > reEncodedChunks[chkIdx][chunkenc.EncXOR][len(reEncodedChunks[chkIdx][chunkenc.EncXOR])-1].MaxTime { + reEncodedChunks[chkIdx][chunkenc.EncXOR][len(reEncodedChunks[chkIdx][chunkenc.EncXOR])-1].MaxTime = t + } + } + case chunkenc.EncFloatHistogram: + for vt := sampleIt.Next(); vt != chunkenc.ValNone; vt = sampleIt.Next() { + if vt != chunkenc.ValFloatHistogram { + return nil, fmt.Errorf("found value type %v in float histogram chunk", vt) + } + t, v := sampleIt.AtFloatHistogram(nil) + + chkIdx := e.schema.DataColumIdx(t) + newC, recoded, app, err := reEncodedChunksAppenders[chkIdx][chunkenc.EncFloatHistogram].AppendFloatHistogram(nil, t, v, false) + if err != nil { + return nil, err + } + reEncodedChunksAppenders[chkIdx][chunkenc.EncFloatHistogram] = app + if newC != nil { + if !recoded { + reEncodedChunks[chkIdx][chunkenc.EncFloatHistogram] = append(reEncodedChunks[chkIdx][chunkenc.EncFloatHistogram], &chunks.Meta{ + MinTime: math.MaxInt64, + }) + } + reEncodedChunks[chkIdx][chunkenc.EncFloatHistogram][len(reEncodedChunks[chkIdx][chunkenc.EncFloatHistogram])-1].Chunk = newC + } + + if t < reEncodedChunks[chkIdx][chunkenc.EncFloatHistogram][len(reEncodedChunks[chkIdx][chunkenc.EncFloatHistogram])-1].MinTime { + reEncodedChunks[chkIdx][chunkenc.EncFloatHistogram][len(reEncodedChunks[chkIdx][chunkenc.EncFloatHistogram])-1].MinTime = t + } + if t > reEncodedChunks[chkIdx][chunkenc.EncFloatHistogram][len(reEncodedChunks[chkIdx][chunkenc.EncFloatHistogram])-1].MaxTime { + reEncodedChunks[chkIdx][chunkenc.EncFloatHistogram][len(reEncodedChunks[chkIdx][chunkenc.EncFloatHistogram])-1].MaxTime = t + } + } + case chunkenc.EncHistogram: + for vt := sampleIt.Next(); vt != chunkenc.ValNone; vt = sampleIt.Next() { + if vt != chunkenc.ValHistogram { + return nil, fmt.Errorf("found value type %v in histogram chunk", vt) + } + t, v := sampleIt.AtHistogram(nil) + + chkIdx := e.schema.DataColumIdx(t) + newC, recoded, app, err := reEncodedChunksAppenders[chkIdx][chunkenc.EncHistogram].AppendHistogram(nil, t, v, false) + if err != nil { + return nil, err + } + reEncodedChunksAppenders[chkIdx][chunkenc.EncHistogram] = app + if newC != nil { + if !recoded { + reEncodedChunks[chkIdx][chunkenc.EncHistogram] = append(reEncodedChunks[chkIdx][chunkenc.EncHistogram], &chunks.Meta{ + MinTime: math.MaxInt64, + }) + } + reEncodedChunks[chkIdx][chunkenc.EncHistogram][len(reEncodedChunks[chkIdx][chunkenc.EncHistogram])-1].Chunk = newC + } + + if t < reEncodedChunks[chkIdx][chunkenc.EncHistogram][len(reEncodedChunks[chkIdx][chunkenc.EncHistogram])-1].MinTime { + reEncodedChunks[chkIdx][chunkenc.EncHistogram][len(reEncodedChunks[chkIdx][chunkenc.EncHistogram])-1].MinTime = t } - if t > reEncodedChunks[chkIdx].MaxTime { - reEncodedChunks[chkIdx].MaxTime = t + if t > reEncodedChunks[chkIdx][chunkenc.EncHistogram][len(reEncodedChunks[chkIdx][chunkenc.EncHistogram])-1].MaxTime { + reEncodedChunks[chkIdx][chunkenc.EncHistogram][len(reEncodedChunks[chkIdx][chunkenc.EncHistogram])-1].MaxTime = t } } default: - continue + return nil, fmt.Errorf("unknown encoding %v", chk.Chunk.Encoding()) } } result := make([][]byte, dataColSize) - for i, chk := range reEncodedChunks { - if chk.Chunk.NumSamples() == 0 { - continue + for i, chunks := range reEncodedChunks { + for _, enc := range []chunkenc.Encoding{chunkenc.EncXOR, chunkenc.EncHistogram, chunkenc.EncFloatHistogram} { + for _, chk := range chunks[enc] { + if chk.Chunk.NumSamples() == 0 { + continue + } + var b [varint.MaxLen64]byte + n := binary.PutUvarint(b[:], uint64(chk.Chunk.Encoding())) + result[i] = append(result[i], b[:n]...) + n = binary.PutUvarint(b[:], uint64(chk.MinTime)) + result[i] = append(result[i], b[:n]...) + n = binary.PutUvarint(b[:], uint64(chk.MaxTime)) + result[i] = append(result[i], b[:n]...) + n = binary.PutUvarint(b[:], uint64(len(chk.Chunk.Bytes()))) + result[i] = append(result[i], b[:n]...) + result[i] = append(result[i], chk.Chunk.Bytes()...) + } } - var b [varint.MaxLen64]byte - n := binary.PutUvarint(b[:], uint64(chk.Chunk.Encoding())) - result[i] = append(result[i], b[:n]...) - n = binary.PutUvarint(b[:], uint64(chk.MinTime)) - result[i] = append(result[i], b[:n]...) - n = binary.PutUvarint(b[:], uint64(chk.MaxTime)) - result[i] = append(result[i], b[:n]...) - n = binary.PutUvarint(b[:], uint64(len(chk.Chunk.Bytes()))) - result[i] = append(result[i], b[:n]...) - result[i] = append(result[i], chk.Chunk.Bytes()...) } return result, nil } @@ -125,39 +204,47 @@ func (e *PrometheusParquetChunksDecoder) Decode(data []byte, mint, maxt int64) ( b := bytes.NewBuffer(data) - chkEnc, err := binary.ReadUvarint(b) - if err != nil { - return nil, err - } + for { + chkEnc, err := binary.ReadUvarint(b) + if err == io.EOF { + break + } - minTime, err := binary.ReadUvarint(b) - if err != nil { - return nil, err - } - if int64(minTime) > maxt { - return nil, nil - } + if err != nil { + return nil, err + } - maxTime, err := binary.ReadUvarint(b) - if err != nil { - return nil, err - } - size, err := binary.ReadUvarint(b) - if err != nil { - return nil, err - } - cData := b.Bytes()[:size] - chk, err := e.Pool.Get(chunkenc.Encoding(chkEnc), cData) - if err != nil { - return nil, err - } + minTime, err := binary.ReadUvarint(b) + if err != nil { + return nil, err + } + + maxTime, err := binary.ReadUvarint(b) + if err != nil { + return nil, err + } + size, err := binary.ReadUvarint(b) + if err != nil { + return nil, err + } + cData := b.Bytes()[:size] + chk, err := e.Pool.Get(chunkenc.Encoding(chkEnc), cData) + if err != nil { + return nil, err + } + b.Next(int(size)) - if int64(maxTime) >= mint { - result = append(result, chunks.Meta{ - MinTime: int64(minTime), - MaxTime: int64(maxTime), - Chunk: chk, - }) + if int64(minTime) > maxt { + continue + } + + if int64(maxTime) >= mint { + result = append(result, chunks.Meta{ + MinTime: int64(minTime), + MaxTime: int64(maxTime), + Chunk: chk, + }) + } } return result, nil diff --git a/vendor/github.com/prometheus-community/parquet-common/schema/schema.go b/vendor/github.com/prometheus-community/parquet-common/schema/schema.go index 5fac886fc7f..6c86484190d 100644 --- a/vendor/github.com/prometheus-community/parquet-common/schema/schema.go +++ b/vendor/github.com/prometheus-community/parquet-common/schema/schema.go @@ -40,8 +40,7 @@ func ExtractLabelFromColumn(col string) (string, bool) { if !strings.HasPrefix(col, LabelColumnPrefix) { return "", false } - - return strings.TrimPrefix(col, LabelColumnPrefix), true + return col[len(LabelColumnPrefix):], true } func IsDataColumn(col string) bool { diff --git a/vendor/github.com/prometheus-community/parquet-common/schema/schema_builder.go b/vendor/github.com/prometheus-community/parquet-common/schema/schema_builder.go index 8585ade6c8f..2ed8b423e89 100644 --- a/vendor/github.com/prometheus-community/parquet-common/schema/schema_builder.go +++ b/vendor/github.com/prometheus-community/parquet-common/schema/schema_builder.go @@ -17,6 +17,8 @@ import ( "fmt" "strconv" + "github.com/efficientgo/core/errors" + "github.com/parquet-go/parquet-go" ) @@ -44,6 +46,44 @@ func NewBuilder(mint, maxt, colDuration int64) *Builder { return b } +func FromLabelsFile(lf *parquet.File) (*TSDBSchema, error) { + md := MetadataToMap(lf.Metadata().KeyValueMetadata) + mint, err := strconv.ParseInt(md[MinTMd], 0, 64) + if err != nil { + return nil, errors.Wrap(err, "failed to convert mint to int") + } + + maxt, err := strconv.ParseInt(md[MaxTMd], 10, 64) + if err != nil { + return nil, errors.Wrap(err, "failed to convert max to int") + } + + dataColDurationMs, err := strconv.ParseInt(md[DataColSizeMd], 10, 64) + if err != nil { + return nil, errors.Wrap(err, "failed to convert dataColDurationMs to int") + } + g := make(parquet.Group) + + b := &Builder{ + g: g, + metadata: md, + mint: mint, + maxt: maxt, + dataColDurationMs: dataColDurationMs, + } + + for _, c := range lf.Schema().Columns() { + lbl, ok := ExtractLabelFromColumn(c[0]) + if !ok { + continue + } + + b.AddLabelNameColumn(lbl) + } + + return b.Build() +} + func (b *Builder) AddLabelNameColumn(lbls ...string) { for _, lbl := range lbls { b.g[LabelToColumn(lbl)] = parquet.Optional(parquet.Encoded(parquet.String(), &parquet.RLEDictionary)) diff --git a/vendor/github.com/prometheus-community/parquet-common/search/constraint.go b/vendor/github.com/prometheus-community/parquet-common/search/constraint.go new file mode 100644 index 00000000000..4c60c568491 --- /dev/null +++ b/vendor/github.com/prometheus-community/parquet-common/search/constraint.go @@ -0,0 +1,610 @@ +// Copyright The Prometheus Authors +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package search + +import ( + "bytes" + "context" + "fmt" + "slices" + "sort" + + "github.com/parquet-go/parquet-go" + "github.com/prometheus/prometheus/model/labels" + + "github.com/prometheus-community/parquet-common/schema" + "github.com/prometheus-community/parquet-common/storage" + "github.com/prometheus-community/parquet-common/util" +) + +type Constraint interface { + fmt.Stringer + + // filter returns a set of non-overlapping increasing row indexes that may satisfy the constraint. + filter(ctx context.Context, rg parquet.RowGroup, primary bool, rr []RowRange) ([]RowRange, error) + // init initializes the constraint with respect to the file schema and projections. + init(f *storage.ParquetFile) error + // path is the path for the column that is constrained + path() string +} + +func MatchersToConstraint(matchers ...*labels.Matcher) ([]Constraint, error) { + r := make([]Constraint, 0, len(matchers)) + for _, matcher := range matchers { + switch matcher.Type { + case labels.MatchEqual: + if matcher.Value == "" { + r = append(r, Null(schema.LabelToColumn(matcher.Name))) + continue + } + r = append(r, Equal(schema.LabelToColumn(matcher.Name), parquet.ValueOf(matcher.Value))) + case labels.MatchNotEqual: + if matcher.Value == "" { + r = append(r, Not(Null(schema.LabelToColumn(matcher.Name)))) + continue + } + r = append(r, Not(Equal(schema.LabelToColumn(matcher.Name), parquet.ValueOf(matcher.Value)))) + case labels.MatchRegexp: + if matcher.Value == "" { + r = append(r, Null(schema.LabelToColumn(matcher.Name))) + continue + } + if matcher.Value == ".+" { + r = append(r, Not(Null(schema.LabelToColumn(matcher.Name)))) + continue + } + res, err := labels.NewFastRegexMatcher(matcher.Value) + if err != nil { + return nil, err + } + r = append(r, Regex(schema.LabelToColumn(matcher.Name), res)) + case labels.MatchNotRegexp: + if matcher.Value == "" { + r = append(r, Not(Null(schema.LabelToColumn(matcher.Name)))) + continue + } + if matcher.Value == ".+" { + r = append(r, Null(schema.LabelToColumn(matcher.Name))) + continue + } + res, err := labels.NewFastRegexMatcher(matcher.Value) + if err != nil { + return nil, err + } + r = append(r, Not(Regex(schema.LabelToColumn(matcher.Name), res))) + default: + return nil, fmt.Errorf("unsupported matcher type %s", matcher.Type) + } + } + return r, nil +} + +func Initialize(f *storage.ParquetFile, cs ...Constraint) error { + for i := range cs { + if err := cs[i].init(f); err != nil { + return fmt.Errorf("unable to initialize constraint %d: %w", i, err) + } + } + return nil +} + +func Filter(ctx context.Context, rg parquet.RowGroup, cs ...Constraint) ([]RowRange, error) { + // Constraints for sorting columns are cheaper to evaluate, so we sort them first. + sc := rg.SortingColumns() + + var n int + for i := range sc { + if n == len(cs) { + break + } + for j := range cs { + if cs[j].path() == sc[i].Path()[0] { + cs[n], cs[j] = cs[j], cs[n] + n++ + } + } + } + var err error + rr := []RowRange{{from: int64(0), count: rg.NumRows()}} + for i := range cs { + isPrimary := len(sc) > 0 && cs[i].path() == sc[0].Path()[0] + rr, err = cs[i].filter(ctx, rg, isPrimary, rr) + if err != nil { + return nil, fmt.Errorf("unable to filter with constraint %d: %w", i, err) + } + } + return rr, nil +} + +// symbolTable is a helper that can decode the i-th value of a page. +// Using it we only need to allocate an int32 slice and not a slice of +// string values. +// It only works for optional dictionary encoded columns. All of our label +// columns are that though. +type symbolTable struct { + dict parquet.Dictionary + syms []int32 +} + +func (s *symbolTable) Get(i int) parquet.Value { + switch s.syms[i] { + case -1: + return parquet.NullValue() + default: + return s.dict.Index(s.syms[i]) + } +} + +func (s *symbolTable) GetIndex(i int) int32 { + return s.syms[i] +} + +func (s *symbolTable) Reset(pg parquet.Page) { + dict := pg.Dictionary() + data := pg.Data() + syms := data.Int32() + defs := pg.DefinitionLevels() + + if s.syms == nil { + s.syms = make([]int32, len(defs)) + } else { + s.syms = slices.Grow(s.syms, len(defs))[:len(defs)] + } + + sidx := 0 + for i := range defs { + if defs[i] == 1 { + s.syms[i] = syms[sidx] + sidx++ + } else { + s.syms[i] = -1 + } + } + s.dict = dict +} + +type equalConstraint struct { + pth string + + val parquet.Value + f *storage.ParquetFile + + comp func(l, r parquet.Value) int +} + +func (ec *equalConstraint) String() string { + return fmt.Sprintf("equal(%q,%q)", ec.pth, ec.val) +} + +func Equal(path string, value parquet.Value) Constraint { + return &equalConstraint{pth: path, val: value} +} + +func (ec *equalConstraint) filter(ctx context.Context, rg parquet.RowGroup, primary bool, rr []RowRange) ([]RowRange, error) { + if len(rr) == 0 { + return nil, nil + } + from, to := rr[0].from, rr[len(rr)-1].from+rr[len(rr)-1].count + + col, ok := rg.Schema().Lookup(ec.path()) + if !ok { + // If match empty, return rr (filter nothing) + // otherwise return empty + if ec.matches(parquet.ValueOf("")) { + return rr, nil + } + return []RowRange{}, nil + } + cc := rg.ColumnChunks()[col.ColumnIndex] + + if skip, err := ec.skipByBloomfilter(cc); err != nil { + return nil, fmt.Errorf("unable to skip by bloomfilter: %w", err) + } else if skip { + return nil, nil + } + + pgs := ec.f.GetPages(ctx, cc) + defer func() { _ = pgs.Close() }() + + oidx, err := cc.OffsetIndex() + if err != nil { + return nil, fmt.Errorf("unable to read offset index: %w", err) + } + cidx, err := cc.ColumnIndex() + if err != nil { + return nil, fmt.Errorf("unable to read column index: %w", err) + } + var ( + symbols = new(symbolTable) + res = make([]RowRange, 0) + ) + for i := 0; i < cidx.NumPages(); i++ { + // If page does not intersect from, to; we can immediately discard it + pfrom := oidx.FirstRowIndex(i) + pcount := rg.NumRows() - pfrom + if i < oidx.NumPages()-1 { + pcount = oidx.FirstRowIndex(i+1) - pfrom + } + pto := pfrom + pcount + if pfrom > to { + break + } + if pto < from { + continue + } + // Page intersects [from, to] but we might be able to discard it with statistics + if cidx.NullPage(i) { + continue + } + minv, maxv := cidx.MinValue(i), cidx.MaxValue(i) + + if !ec.val.IsNull() && !maxv.IsNull() && ec.comp(ec.val, maxv) > 0 { + if cidx.IsDescending() { + break + } + continue + } + if !ec.val.IsNull() && !minv.IsNull() && ec.comp(ec.val, minv) < 0 { + if cidx.IsAscending() { + break + } + continue + } + // We cannot discard the page through statistics but we might need to read it to see if it has the value + if err := pgs.SeekToRow(pfrom); err != nil { + return nil, fmt.Errorf("unable to seek to row: %w", err) + } + pg, err := pgs.ReadPage() + if err != nil { + return nil, fmt.Errorf("unable to read page: %w", err) + } + + symbols.Reset(pg) + + // The page has the value, we need to find the matching row ranges + n := int(pg.NumRows()) + bl := int(max(pfrom, from) - pfrom) + br := n - int(pto-min(pto, to)) + var l, r int + switch { + case cidx.IsAscending() && primary: + l = sort.Search(n, func(i int) bool { return ec.comp(ec.val, symbols.Get(i)) <= 0 }) + r = sort.Search(n, func(i int) bool { return ec.comp(ec.val, symbols.Get(i)) < 0 }) + + if lv, rv := max(bl, l), min(br, r); rv > lv { + res = append(res, RowRange{pfrom + int64(lv), int64(rv - lv)}) + } + default: + off, count := bl, 0 + for j := bl; j < br; j++ { + if ec.comp(ec.val, symbols.Get(j)) != 0 { + if count != 0 { + res = append(res, RowRange{pfrom + int64(off), int64(count)}) + } + off, count = j, 0 + } else { + if count == 0 { + off = j + } + count++ + } + } + if count != 0 { + res = append(res, RowRange{pfrom + int64(off), int64(count)}) + } + } + } + if len(res) == 0 { + return nil, nil + } + return intersectRowRanges(simplify(res), rr), nil +} + +func (ec *equalConstraint) init(f *storage.ParquetFile) error { + c, ok := f.Schema().Lookup(ec.path()) + ec.f = f + if !ok { + return nil + } + if c.Node.Type().Kind() != ec.val.Kind() { + return fmt.Errorf("schema: cannot search value of kind %s in column of kind %s", ec.val.Kind(), c.Node.Type().Kind()) + } + ec.comp = c.Node.Type().Compare + return nil +} + +func (ec *equalConstraint) path() string { + return ec.pth +} + +func (ec *equalConstraint) matches(v parquet.Value) bool { + return bytes.Equal(v.ByteArray(), ec.val.ByteArray()) +} + +func (ec *equalConstraint) skipByBloomfilter(cc parquet.ColumnChunk) (bool, error) { + bf := cc.BloomFilter() + if bf == nil { + return false, nil + } + ok, err := bf.Check(ec.val) + if err != nil { + return false, fmt.Errorf("unable to check bloomfilter: %w", err) + } + return !ok, nil +} + +func Regex(path string, r *labels.FastRegexMatcher) Constraint { + return ®exConstraint{pth: path, cache: make(map[parquet.Value]bool), r: r} +} + +type regexConstraint struct { + pth string + cache map[parquet.Value]bool + f *storage.ParquetFile + r *labels.FastRegexMatcher +} + +func (rc *regexConstraint) String() string { + return fmt.Sprintf("regex(%v,%v)", rc.pth, rc.r.GetRegexString()) +} + +func (rc *regexConstraint) filter(ctx context.Context, rg parquet.RowGroup, primary bool, rr []RowRange) ([]RowRange, error) { + if len(rr) == 0 { + return nil, nil + } + from, to := rr[0].from, rr[len(rr)-1].from+rr[len(rr)-1].count + + col, ok := rg.Schema().Lookup(rc.path()) + if !ok { + // If match empty, return rr (filter nothing) + // otherwise return empty + if rc.matches(parquet.ValueOf("")) { + return rr, nil + } + return []RowRange{}, nil + } + cc := rg.ColumnChunks()[col.ColumnIndex] + + pgs := rc.f.GetPages(ctx, cc) + defer func() { _ = pgs.Close() }() + + oidx, err := cc.OffsetIndex() + if err != nil { + return nil, fmt.Errorf("unable to read offset index: %w", err) + } + cidx, err := cc.ColumnIndex() + if err != nil { + return nil, fmt.Errorf("unable to read column index: %w", err) + } + var ( + symbols = new(symbolTable) + res = make([]RowRange, 0) + ) + for i := 0; i < cidx.NumPages(); i++ { + // If page does not intersect from, to; we can immediately discard it + pfrom := oidx.FirstRowIndex(i) + pcount := rg.NumRows() - pfrom + if i < oidx.NumPages()-1 { + pcount = oidx.FirstRowIndex(i+1) - pfrom + } + pto := pfrom + pcount + if pfrom > to { + break + } + if pto < from { + continue + } + // Page intersects [from, to] but we might be able to discard it with statistics + if cidx.NullPage(i) { + continue + } + // TODO: use setmatches / prefix for statistics + + // We cannot discard the page through statistics but we might need to read it to see if it has the value + if err := pgs.SeekToRow(pfrom); err != nil { + return nil, fmt.Errorf("unable to seek to row: %w", err) + } + pg, err := pgs.ReadPage() + if err != nil { + return nil, fmt.Errorf("unable to read page: %w", err) + } + + symbols.Reset(pg) + + // The page has the value, we need to find the matching row ranges + n := int(pg.NumRows()) + bl := int(max(pfrom, from) - pfrom) + br := n - int(pto-min(pto, to)) + off, count := bl, 0 + for j := bl; j < br; j++ { + if !rc.matches(symbols.Get(j)) { + if count != 0 { + res = append(res, RowRange{pfrom + int64(off), int64(count)}) + } + off, count = j, 0 + } else { + if count == 0 { + off = j + } + count++ + } + } + if count != 0 { + res = append(res, RowRange{pfrom + int64(off), int64(count)}) + } + } + if len(res) == 0 { + return nil, nil + } + return intersectRowRanges(simplify(res), rr), nil +} + +func (rc *regexConstraint) init(f *storage.ParquetFile) error { + c, ok := f.Schema().Lookup(rc.path()) + rc.f = f + if !ok { + return nil + } + if stringKind := parquet.String().Type().Kind(); c.Node.Type().Kind() != stringKind { + return fmt.Errorf("schema: cannot search value of kind %s in column of kind %s", stringKind, c.Node.Type().Kind()) + } + rc.cache = make(map[parquet.Value]bool) + return nil +} + +func (rc *regexConstraint) path() string { + return rc.pth +} + +func (rc *regexConstraint) matches(v parquet.Value) bool { + accept, seen := rc.cache[v] + if !seen { + accept = rc.r.MatchString(util.YoloString(v.ByteArray())) + rc.cache[v] = accept + } + return accept +} + +func Not(c Constraint) Constraint { + return ¬Constraint{c: c} +} + +type notConstraint struct { + c Constraint +} + +func (nc *notConstraint) String() string { + return fmt.Sprintf("not(%v)", nc.c.String()) +} + +func (nc *notConstraint) filter(ctx context.Context, rg parquet.RowGroup, primary bool, rr []RowRange) ([]RowRange, error) { + base, err := nc.c.filter(ctx, rg, primary, rr) + if err != nil { + return nil, fmt.Errorf("unable to compute child constraint: %w", err) + } + // no need to intersect since its already subset of rr + return complementRowRanges(base, rr), nil +} + +func (nc *notConstraint) init(f *storage.ParquetFile) error { + return nc.c.init(f) +} + +func (nc *notConstraint) path() string { + return nc.c.path() +} + +type nullConstraint struct { + pth string +} + +func (null *nullConstraint) String() string { + return fmt.Sprintf("null(%q)", null.pth) +} + +func Null(path string) Constraint { + return &nullConstraint{pth: path} +} + +func (null *nullConstraint) filter(ctx context.Context, rg parquet.RowGroup, _ bool, rr []RowRange) ([]RowRange, error) { + if len(rr) == 0 { + return nil, nil + } + from, to := rr[0].from, rr[len(rr)-1].from+rr[len(rr)-1].count + + col, ok := rg.Schema().Lookup(null.path()) + if !ok { + // filter nothing + return rr, nil + } + cc := rg.ColumnChunks()[col.ColumnIndex] + + pgs := cc.Pages() + defer func() { _ = pgs.Close() }() + + oidx, err := cc.OffsetIndex() + if err != nil { + return nil, fmt.Errorf("unable to read offset index: %w", err) + } + cidx, err := cc.ColumnIndex() + if err != nil { + return nil, fmt.Errorf("unable to read column index: %w", err) + } + res := make([]RowRange, 0) + for i := 0; i < cidx.NumPages(); i++ { + // If page does not intersect from, to; we can immediately discard it + pfrom := oidx.FirstRowIndex(i) + pcount := rg.NumRows() - pfrom + if i < oidx.NumPages()-1 { + pcount = oidx.FirstRowIndex(i+1) - pfrom + } + pto := pfrom + pcount + if pfrom > to { + break + } + if pto < from { + continue + } + + if cidx.NullPage(i) { + res = append(res, RowRange{from: pfrom, count: pcount}) + continue + } + + if cidx.NullCount(i) == 0 { + continue + } + + // We cannot discard the page through statistics but we might need to read it to see if it has the value + if err := pgs.SeekToRow(pfrom); err != nil { + return nil, fmt.Errorf("unable to seek to row: %w", err) + } + pg, err := pgs.ReadPage() + if err != nil { + return nil, fmt.Errorf("unable to read page: %w", err) + } + // The page has null value, we need to find the matching row ranges + bl := int(max(pfrom, from) - pfrom) + off, count := bl, 0 + for j, def := range pg.DefinitionLevels() { + if def != 1 { + if count == 0 { + off = j + } + count++ + } else { + if count != 0 { + res = append(res, RowRange{pfrom + int64(off), int64(count)}) + } + off, count = j, 0 + } + } + + if count != 0 { + res = append(res, RowRange{pfrom + int64(off), int64(count)}) + } + } + if len(res) == 0 { + return nil, nil + } + return intersectRowRanges(simplify(res), rr), nil +} + +func (null *nullConstraint) init(*storage.ParquetFile) error { + return nil +} + +func (null *nullConstraint) path() string { + return null.pth +} diff --git a/vendor/github.com/prometheus-community/parquet-common/search/materialize.go b/vendor/github.com/prometheus-community/parquet-common/search/materialize.go new file mode 100644 index 00000000000..ffa3f862984 --- /dev/null +++ b/vendor/github.com/prometheus-community/parquet-common/search/materialize.go @@ -0,0 +1,511 @@ +// Copyright The Prometheus Authors +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package search + +import ( + "context" + "fmt" + "io" + "runtime" + "slices" + "sort" + + "github.com/efficientgo/core/errors" + "github.com/parquet-go/parquet-go" + "github.com/prometheus/prometheus/model/labels" + prom_storage "github.com/prometheus/prometheus/storage" + "github.com/prometheus/prometheus/tsdb/chunks" + "golang.org/x/sync/errgroup" + + "github.com/prometheus-community/parquet-common/schema" + "github.com/prometheus-community/parquet-common/storage" + "github.com/prometheus-community/parquet-common/util" +) + +type Materializer struct { + b *storage.ParquetShard + s *schema.TSDBSchema + d *schema.PrometheusParquetChunksDecoder + + colIdx int + concurrency int + + dataColToIndex []int +} + +func NewMaterializer(s *schema.TSDBSchema, d *schema.PrometheusParquetChunksDecoder, block *storage.ParquetShard) (*Materializer, error) { + colIdx, ok := block.LabelsFile().Schema().Lookup(schema.ColIndexes) + if !ok { + return nil, errors.New(fmt.Sprintf("schema index %s not found", schema.ColIndexes)) + } + + dataColToIndex := make([]int, len(block.ChunksFile().Schema().Columns())) + for i := 0; i < len(s.DataColsIndexes); i++ { + c, ok := block.ChunksFile().Schema().Lookup(schema.DataColumn(i)) + if !ok { + return nil, errors.New(fmt.Sprintf("schema column %s not found", schema.DataColumn(i))) + } + + dataColToIndex[i] = c.ColumnIndex + } + + return &Materializer{ + s: s, + d: d, + b: block, + colIdx: colIdx.ColumnIndex, + concurrency: runtime.GOMAXPROCS(0), + dataColToIndex: dataColToIndex, + }, nil +} + +// Materialize reconstructs the ChunkSeries that belong to the specified row ranges (rr). +// It uses the row group index (rgi) and time bounds (mint, maxt) to filter and decode the series. +func (m *Materializer) Materialize(ctx context.Context, rgi int, mint, maxt int64, skipChunks bool, rr []RowRange) ([]prom_storage.ChunkSeries, error) { + sLbls, err := m.materializeAllLabels(ctx, rgi, rr) + if err != nil { + return nil, errors.Wrapf(err, "error materializing labels") + } + + results := make([]prom_storage.ChunkSeries, len(sLbls)) + for i, s := range sLbls { + sort.Sort(s) + results[i] = &concreteChunksSeries{ + lbls: s, + } + } + + if !skipChunks { + chks, err := m.materializeChunks(ctx, rgi, mint, maxt, rr) + if err != nil { + return nil, errors.Wrap(err, "materializer failed to materialize chunks") + } + + for i, result := range results { + result.(*concreteChunksSeries).chks = chks[i] + } + + // If we are not skipping chunks and there is no chunks for the time range queried, lets remove the series + results = slices.DeleteFunc(results, func(cs prom_storage.ChunkSeries) bool { + return len(cs.(*concreteChunksSeries).chks) == 0 + }) + } + return results, err +} + +func (m *Materializer) MaterializeAllLabelNames() []string { + r := make([]string, 0, len(m.b.LabelsFile().Schema().Columns())) + for _, c := range m.b.LabelsFile().Schema().Columns() { + lbl, ok := schema.ExtractLabelFromColumn(c[0]) + if !ok { + continue + } + + r = append(r, lbl) + } + return r +} + +func (m *Materializer) MaterializeLabelNames(ctx context.Context, rgi int, rr []RowRange) ([]string, error) { + labelsRg := m.b.LabelsFile().RowGroups()[rgi] + cc := labelsRg.ColumnChunks()[m.colIdx] + colsIdxs, err := m.materializeColumn(ctx, m.b.LabelsFile(), labelsRg, cc, rr) + if err != nil { + return nil, errors.Wrap(err, "materializer failed to materialize columns") + } + + seen := make(map[string]struct{}) + colsMap := make(map[string]struct{}, 10) + for _, colsIdx := range colsIdxs { + key := util.YoloString(colsIdx.ByteArray()) + if _, ok := seen[key]; !ok { + idxs, err := schema.DecodeUintSlice(colsIdx.ByteArray()) + if err != nil { + return nil, errors.Wrap(err, "failed to decode column index") + } + for _, idx := range idxs { + if _, ok := colsMap[m.b.LabelsFile().Schema().Columns()[idx][0]]; !ok { + colsMap[m.b.LabelsFile().Schema().Columns()[idx][0]] = struct{}{} + } + } + } + } + lbls := make([]string, 0, len(colsMap)) + for col := range colsMap { + l, ok := schema.ExtractLabelFromColumn(col) + if !ok { + return nil, errors.New(fmt.Sprintf("error extracting label name from col %v", col)) + } + lbls = append(lbls, l) + } + return lbls, nil +} + +func (m *Materializer) MaterializeLabelValues(ctx context.Context, name string, rgi int, rr []RowRange) ([]string, error) { + labelsRg := m.b.LabelsFile().RowGroups()[rgi] + cIdx, ok := m.b.LabelsFile().Schema().Lookup(schema.LabelToColumn(name)) + if !ok { + return []string{}, nil + } + cc := labelsRg.ColumnChunks()[cIdx.ColumnIndex] + values, err := m.materializeColumn(ctx, m.b.LabelsFile(), labelsRg, cc, rr) + if err != nil { + return nil, errors.Wrap(err, "materializer failed to materialize columns") + } + + r := make([]string, 0, len(values)) + vMap := make(map[string]struct{}, 10) + for _, v := range values { + strValue := util.YoloString(v.ByteArray()) + if _, ok := vMap[strValue]; !ok { + r = append(r, strValue) + vMap[strValue] = struct{}{} + } + } + return r, nil +} + +func (m *Materializer) MaterializeAllLabelValues(ctx context.Context, name string, rgi int) ([]string, error) { + labelsRg := m.b.LabelsFile().RowGroups()[rgi] + cIdx, ok := m.b.LabelsFile().Schema().Lookup(schema.LabelToColumn(name)) + if !ok { + return []string{}, nil + } + cc := labelsRg.ColumnChunks()[cIdx.ColumnIndex] + pages := m.b.LabelsFile().GetPages(ctx, cc) + p, err := pages.ReadPage() + if err != nil { + return []string{}, errors.Wrap(err, "failed to read page") + } + defer parquet.Release(p) + + r := make([]string, 0, p.Dictionary().Len()) + for i := 0; i < p.Dictionary().Len(); i++ { + r = append(r, p.Dictionary().Index(int32(i)).String()) + } + return r, nil +} + +func (m *Materializer) materializeAllLabels(ctx context.Context, rgi int, rr []RowRange) ([]labels.Labels, error) { + labelsRg := m.b.LabelsFile().RowGroups()[rgi] + cc := labelsRg.ColumnChunks()[m.colIdx] + colsIdxs, err := m.materializeColumn(ctx, m.b.LabelsFile(), labelsRg, cc, rr) + if err != nil { + return nil, errors.Wrap(err, "materializer failed to materialize columns") + } + + colsMap := make(map[int]*[]parquet.Value, 10) + results := make([]labels.Labels, len(colsIdxs)) + + for _, colsIdx := range colsIdxs { + idxs, err := schema.DecodeUintSlice(colsIdx.ByteArray()) + if err != nil { + return nil, errors.Wrap(err, "materializer failed to decode column index") + } + for _, idx := range idxs { + v := make([]parquet.Value, 0, len(colsIdxs)) + colsMap[idx] = &v + } + } + + errGroup, ctx := errgroup.WithContext(ctx) + errGroup.SetLimit(m.concurrency) + + for cIdx, v := range colsMap { + errGroup.Go(func() error { + cc := labelsRg.ColumnChunks()[cIdx] + values, err := m.materializeColumn(ctx, m.b.LabelsFile(), labelsRg, cc, rr) + if err != nil { + return errors.Wrap(err, "failed to materialize labels values") + } + *v = append(*v, values...) + return nil + }) + } + + if err := errGroup.Wait(); err != nil { + return nil, err + } + + for cIdx, values := range colsMap { + labelName, ok := schema.ExtractLabelFromColumn(m.b.LabelsFile().Schema().Columns()[cIdx][0]) + if !ok { + return nil, fmt.Errorf("column %d not found in schema", cIdx) + } + for i, value := range *values { + if value.IsNull() { + continue + } + results[i] = append(results[i], labels.Label{ + Name: labelName, + Value: util.YoloString(value.ByteArray()), + }) + } + } + + return results, nil +} + +func (m *Materializer) materializeChunks(ctx context.Context, rgi int, mint, maxt int64, rr []RowRange) ([][]chunks.Meta, error) { + minDataCol := m.s.DataColumIdx(mint) + maxDataCol := m.s.DataColumIdx(maxt) + rg := m.b.ChunksFile().RowGroups()[rgi] + totalRows := int64(0) + for _, r := range rr { + totalRows += r.count + } + r := make([][]chunks.Meta, totalRows) + + for i := minDataCol; i <= min(maxDataCol, len(m.dataColToIndex)-1); i++ { + values, err := m.materializeColumn(ctx, m.b.ChunksFile(), rg, rg.ColumnChunks()[m.dataColToIndex[i]], rr) + if err != nil { + return r, err + } + + for vi, value := range values { + chks, err := m.d.Decode(value.ByteArray(), mint, maxt) + if err != nil { + return r, errors.Wrap(err, "failed to decode chunks") + } + r[vi] = append(r[vi], chks...) + } + } + + return r, nil +} + +func (m *Materializer) materializeColumn(ctx context.Context, file *storage.ParquetFile, group parquet.RowGroup, cc parquet.ColumnChunk, rr []RowRange) ([]parquet.Value, error) { + if len(rr) == 0 { + return nil, nil + } + + oidx, err := cc.OffsetIndex() + if err != nil { + return nil, errors.Wrap(err, "could not get offset index") + } + + cidx, err := cc.ColumnIndex() + if err != nil { + return nil, errors.Wrap(err, "could not get column index") + } + + pagesToRowsMap := make(map[int][]RowRange, len(rr)) + + for i := 0; i < cidx.NumPages(); i++ { + pageRowRange := RowRange{ + from: oidx.FirstRowIndex(i), + } + pageRowRange.count = group.NumRows() + + if i < oidx.NumPages()-1 { + pageRowRange.count = oidx.FirstRowIndex(i+1) - pageRowRange.from + } + + for _, r := range rr { + if pageRowRange.Overlaps(r) { + pagesToRowsMap[i] = append(pagesToRowsMap[i], r) + } + } + } + + r := make(map[RowRange][]parquet.Value, len(rr)) + for _, v := range rr { + r[v] = []parquet.Value{} + } + + errGroup := &errgroup.Group{} + errGroup.SetLimit(m.concurrency) + + for _, p := range coalescePageRanges(pagesToRowsMap, oidx) { + errGroup.Go(func() error { + pgs := file.GetPages(ctx, cc) + defer func() { _ = pgs.Close() }() + err := pgs.SeekToRow(p.rows[0].from) + if err != nil { + return errors.Wrap(err, "could not seek to row") + } + + vi := new(valuesIterator) + remainingRr := p.rows + currentRr := remainingRr[0] + next := currentRr.from + remaining := currentRr.count + currentRow := currentRr.from + + remainingRr = remainingRr[1:] + for len(remainingRr) > 0 || remaining > 0 { + page, err := pgs.ReadPage() + if err != nil { + return errors.Wrap(err, "could not read page") + } + vi.Reset(page) + for vi.Next() { + if currentRow == next { + r[currentRr] = append(r[currentRr], vi.At()) + remaining-- + if remaining > 0 { + next = next + 1 + } else if len(remainingRr) > 0 { + currentRr = remainingRr[0] + next = currentRr.from + remaining = currentRr.count + remainingRr = remainingRr[1:] + } + } + currentRow++ + } + parquet.Release(page) + + if vi.Error() != nil { + return vi.Error() + } + } + return nil + }) + } + err = errGroup.Wait() + if err != nil { + return nil, errors.Wrap(err, "failed to materialize columns") + } + + values := make([]parquet.Value, 0, len(rr)) + for _, v := range rr { + values = append(values, r[v]...) + } + return values, err +} + +type pageEntryRead struct { + pages []int + rows []RowRange +} + +// Merge nearby pages to enable efficient sequential reads. +// Pages that are not close to each other will be scheduled for concurrent reads. +func coalescePageRanges(pagedIdx map[int][]RowRange, offset parquet.OffsetIndex) []pageEntryRead { + // TODO: Add the max gap size as parameter + partitioner := util.NewGapBasedPartitioner(10 * 1024) + if len(pagedIdx) == 0 { + return []pageEntryRead{} + } + idxs := make([]int, 0, len(pagedIdx)) + for idx := range pagedIdx { + idxs = append(idxs, idx) + } + + slices.Sort(idxs) + + parts := partitioner.Partition(len(idxs), func(i int) (uint64, uint64) { + return uint64(offset.Offset(idxs[i])), uint64(offset.Offset(idxs[i]) + offset.CompressedPageSize(idxs[i])) + }) + + r := make([]pageEntryRead, 0, len(parts)) + for _, part := range parts { + pagesToRead := pageEntryRead{} + for i := part.ElemRng[0]; i < part.ElemRng[1]; i++ { + pagesToRead.pages = append(pagesToRead.pages, idxs[i]) + pagesToRead.rows = append(pagesToRead.rows, pagedIdx[idxs[i]]...) + } + pagesToRead.rows = simplify(pagesToRead.rows) + r = append(r, pagesToRead) + } + + return r +} + +type valuesIterator struct { + p parquet.Page + + // TODO: consider using unique.Handle + cachedSymbols map[int32]parquet.Value + st symbolTable + + vr parquet.ValueReader + + current int + buffer []parquet.Value + currentBufferIndex int + err error +} + +func (vi *valuesIterator) Reset(p parquet.Page) { + vi.p = p + vi.vr = nil + if p.Dictionary() != nil { + vi.st.Reset(p) + vi.cachedSymbols = make(map[int32]parquet.Value, p.Dictionary().Len()) + } else { + vi.vr = p.Values() + vi.buffer = make([]parquet.Value, 0, 128) + vi.currentBufferIndex = -1 + } + vi.current = -1 +} + +func (vi *valuesIterator) Next() bool { + if vi.err != nil { + return false + } + + vi.current++ + if vi.current >= int(vi.p.NumRows()) { + return false + } + + vi.currentBufferIndex++ + + if vi.currentBufferIndex == len(vi.buffer) { + n, err := vi.vr.ReadValues(vi.buffer[:cap(vi.buffer)]) + if err != nil && err != io.EOF { + vi.err = err + } + vi.buffer = vi.buffer[:n] + vi.currentBufferIndex = 0 + } + + return true +} + +func (vi *valuesIterator) Error() error { + return vi.err +} + +func (vi *valuesIterator) At() parquet.Value { + if vi.vr == nil { + dicIndex := vi.st.GetIndex(vi.current) + // Cache a clone of the current symbol table entry. + // This allows us to release the original page while avoiding unnecessary future clones. + if _, ok := vi.cachedSymbols[dicIndex]; !ok { + vi.cachedSymbols[dicIndex] = vi.st.Get(vi.current).Clone() + } + return vi.cachedSymbols[dicIndex] + } + + return vi.buffer[vi.currentBufferIndex].Clone() +} + +var _ prom_storage.ChunkSeries = &concreteChunksSeries{} + +type concreteChunksSeries struct { + lbls labels.Labels + chks []chunks.Meta +} + +func (c concreteChunksSeries) Labels() labels.Labels { + return c.lbls +} + +func (c concreteChunksSeries) Iterator(_ chunks.Iterator) chunks.Iterator { + return prom_storage.NewListChunkSeriesIterator(c.chks...) +} diff --git a/vendor/github.com/prometheus-community/parquet-common/search/parquet_queriable.go b/vendor/github.com/prometheus-community/parquet-common/search/parquet_queriable.go new file mode 100644 index 00000000000..cf609b31169 --- /dev/null +++ b/vendor/github.com/prometheus-community/parquet-common/search/parquet_queriable.go @@ -0,0 +1,281 @@ +// Copyright The Prometheus Authors +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package search + +import ( + "context" + "sort" + + "github.com/prometheus/prometheus/model/labels" + prom_storage "github.com/prometheus/prometheus/storage" + "github.com/prometheus/prometheus/util/annotations" + + "github.com/prometheus-community/parquet-common/convert" + "github.com/prometheus-community/parquet-common/schema" + "github.com/prometheus-community/parquet-common/storage" + "github.com/prometheus-community/parquet-common/util" +) + +type ShardsFinderFunction func(ctx context.Context, mint, maxt int64) ([]*storage.ParquetShard, error) + +type parquetQueryable struct { + shardsFinder ShardsFinderFunction + d *schema.PrometheusParquetChunksDecoder +} + +func NewParquetQueryable(d *schema.PrometheusParquetChunksDecoder, shardFinder ShardsFinderFunction) (prom_storage.Queryable, error) { + return &parquetQueryable{ + shardsFinder: shardFinder, + d: d, + }, nil +} + +func (p parquetQueryable) Querier(mint, maxt int64) (prom_storage.Querier, error) { + return &parquetQuerier{ + mint: mint, + maxt: maxt, + shardsFinder: p.shardsFinder, + d: p.d, + }, nil +} + +type parquetQuerier struct { + mint, maxt int64 + shardsFinder ShardsFinderFunction + d *schema.PrometheusParquetChunksDecoder +} + +func (p parquetQuerier) LabelValues(ctx context.Context, name string, hints *prom_storage.LabelHints, matchers ...*labels.Matcher) ([]string, annotations.Annotations, error) { + shards, err := p.queryableShards(ctx, p.mint, p.maxt) + if err != nil { + return nil, nil, err + } + + limit := int64(0) + + if hints != nil { + limit = int64(hints.Limit) + } + + resNameValues := [][]string{} + + for _, s := range shards { + r, err := s.LabelValues(ctx, name, matchers) + if err != nil { + return nil, nil, err + } + + resNameValues = append(resNameValues, r...) + } + + return util.MergeUnsortedSlices(int(limit), resNameValues...), nil, nil +} + +func (p parquetQuerier) LabelNames(ctx context.Context, hints *prom_storage.LabelHints, matchers ...*labels.Matcher) ([]string, annotations.Annotations, error) { + shards, err := p.queryableShards(ctx, p.mint, p.maxt) + if err != nil { + return nil, nil, err + } + + limit := int64(0) + + if hints != nil { + limit = int64(hints.Limit) + } + + resNameSets := [][]string{} + + for _, s := range shards { + r, err := s.LabelNames(ctx, matchers) + if err != nil { + return nil, nil, err + } + + resNameSets = append(resNameSets, r...) + } + + return util.MergeUnsortedSlices(int(limit), resNameSets...), nil, nil +} + +func (p parquetQuerier) Close() error { + return nil +} + +func (p parquetQuerier) Select(ctx context.Context, sorted bool, sp *prom_storage.SelectHints, matchers ...*labels.Matcher) prom_storage.SeriesSet { + shards, err := p.queryableShards(ctx, p.mint, p.maxt) + if err != nil { + return prom_storage.ErrSeriesSet(err) + } + seriesSet := make([]prom_storage.ChunkSeriesSet, len(shards)) + + minT, maxT := p.mint, p.maxt + if sp != nil { + minT, maxT = sp.Start, sp.End + } + skipChunks := sp != nil && sp.Func == "series" + + for i, shard := range shards { + ss, err := shard.Query(ctx, sorted, minT, maxT, skipChunks, matchers) + if err != nil { + return prom_storage.ErrSeriesSet(err) + } + seriesSet[i] = ss + } + ss := convert.NewMergeChunkSeriesSet(seriesSet, labels.Compare, prom_storage.NewConcatenatingChunkSeriesMerger()) + + return convert.NewSeriesSetFromChunkSeriesSet(ss, skipChunks) +} + +func (p parquetQuerier) queryableShards(ctx context.Context, mint, maxt int64) ([]*queryableShard, error) { + shards, err := p.shardsFinder(ctx, mint, maxt) + if err != nil { + return nil, err + } + qBlocks := make([]*queryableShard, len(shards)) + for i, shard := range shards { + qb, err := newQueryableShard(shard, p.d) + if err != nil { + return nil, err + } + qBlocks[i] = qb + } + return qBlocks, nil +} + +type queryableShard struct { + shard *storage.ParquetShard + m *Materializer +} + +func newQueryableShard(block *storage.ParquetShard, d *schema.PrometheusParquetChunksDecoder) (*queryableShard, error) { + s, err := block.TSDBSchema() + if err != nil { + return nil, err + } + m, err := NewMaterializer(s, d, block) + if err != nil { + return nil, err + } + + return &queryableShard{ + shard: block, + m: m, + }, nil +} + +func (b queryableShard) Query(ctx context.Context, sorted bool, mint, maxt int64, skipChunks bool, matchers []*labels.Matcher) (prom_storage.ChunkSeriesSet, error) { + cs, err := MatchersToConstraint(matchers...) + if err != nil { + return nil, err + } + err = Initialize(b.shard.LabelsFile(), cs...) + if err != nil { + return nil, err + } + + results := make([]prom_storage.ChunkSeries, 0, 1024) + for i, group := range b.shard.LabelsFile().RowGroups() { + rr, err := Filter(ctx, group, cs...) + if err != nil { + return nil, err + } + series, err := b.m.Materialize(ctx, i, mint, maxt, skipChunks, rr) + if err != nil { + return nil, err + } + results = append(results, series...) + } + + if sorted { + sort.Sort(byLabels(results)) + } + return convert.NewChunksSeriesSet(results), nil +} + +func (b queryableShard) LabelNames(ctx context.Context, matchers []*labels.Matcher) ([][]string, error) { + if len(matchers) == 0 { + return [][]string{b.m.MaterializeAllLabelNames()}, nil + } + cs, err := MatchersToConstraint(matchers...) + if err != nil { + return nil, err + } + err = Initialize(b.shard.LabelsFile(), cs...) + if err != nil { + return nil, err + } + + results := make([][]string, len(b.shard.LabelsFile().RowGroups())) + for i, group := range b.shard.LabelsFile().RowGroups() { + rr, err := Filter(ctx, group, cs...) + if err != nil { + return nil, err + } + series, err := b.m.MaterializeLabelNames(ctx, i, rr) + if err != nil { + return nil, err + } + results[i] = series + } + + return results, nil +} + +func (b queryableShard) LabelValues(ctx context.Context, name string, matchers []*labels.Matcher) ([][]string, error) { + if len(matchers) == 0 { + return b.allLabelValues(ctx, name) + } + cs, err := MatchersToConstraint(matchers...) + if err != nil { + return nil, err + } + err = Initialize(b.shard.LabelsFile(), cs...) + if err != nil { + return nil, err + } + + results := make([][]string, len(b.shard.LabelsFile().RowGroups())) + for i, group := range b.shard.LabelsFile().RowGroups() { + rr, err := Filter(ctx, group, cs...) + if err != nil { + return nil, err + } + series, err := b.m.MaterializeLabelValues(ctx, name, i, rr) + if err != nil { + return nil, err + } + results[i] = series + } + + return results, nil +} + +func (b queryableShard) allLabelValues(ctx context.Context, name string) ([][]string, error) { + results := make([][]string, len(b.shard.LabelsFile().RowGroups())) + for i := range b.shard.LabelsFile().RowGroups() { + series, err := b.m.MaterializeAllLabelValues(ctx, name, i) + if err != nil { + return nil, err + } + results[i] = series + } + + return results, nil +} + +type byLabels []prom_storage.ChunkSeries + +func (b byLabels) Len() int { return len(b) } +func (b byLabels) Swap(i, j int) { b[i], b[j] = b[j], b[i] } +func (b byLabels) Less(i, j int) bool { return labels.Compare(b[i].Labels(), b[j].Labels()) < 0 } diff --git a/vendor/github.com/prometheus-community/parquet-common/search/rowrange.go b/vendor/github.com/prometheus-community/parquet-common/search/rowrange.go new file mode 100644 index 00000000000..10cf136136a --- /dev/null +++ b/vendor/github.com/prometheus-community/parquet-common/search/rowrange.go @@ -0,0 +1,166 @@ +// Copyright The Prometheus Authors +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package search + +import ( + "sort" +) + +type RowRange struct { + from int64 + count int64 +} + +func NewRowRange(from, count int64) *RowRange { + return &RowRange{ + from: from, + count: count, + } +} + +// Overlaps returns true if the receiver and the given RowRange share any overlapping rows. +// Both ranges are treated as half-open intervals: [from, from+count). +func (rr RowRange) Overlaps(o RowRange) bool { + endA := rr.from + rr.count + endB := o.from + o.count + return rr.from < endB && o.from < endA +} + +// intersect intersects the row ranges from left hand sight with the row ranges from rhs +// it assumes that lhs and rhs are simplified and returns a simplified result. +// it operates in o(l+r) time by cursoring through ranges with a two pointer approach. +func intersectRowRanges(lhs, rhs []RowRange) []RowRange { + res := make([]RowRange, 0) + for l, r := 0, 0; l < len(lhs) && r < len(rhs); { + al, bl := lhs[l].from, lhs[l].from+lhs[l].count + ar, br := rhs[r].from, rhs[r].from+rhs[r].count + + // check if rows intersect + if al <= br && ar <= bl { + os, oe := max(al, ar), min(bl, br) + res = append(res, RowRange{from: os, count: oe - os}) + } + + // advance the cursor of the range that ends first + if bl <= br { + l++ + } else { + r++ + } + } + return simplify(res) +} + +// complementRowRanges returns the ranges that are in rhs but not in lhs. +// For example, if you have: +// lhs: [{from: 1, count: 3}] // represents rows 1,2,3 +// rhs: [{from: 0, count: 5}] // represents rows 0,1,2,3,4 +// The complement would be [{from: 0, count: 1}, {from: 4, count: 1}] // represents rows 0,4 +// because these are the rows in rhs that are not in lhs. +// +// The function assumes that lhs and rhs are simplified (no overlapping ranges) +// and returns a simplified result. It operates in O(l+r) time by using a two-pointer approach +// to efficiently process both ranges. +func complementRowRanges(lhs, rhs []RowRange) []RowRange { + res := make([]RowRange, 0) + + l, r := 0, 0 + for l < len(lhs) && r < len(rhs) { + al, bl := lhs[l].from, lhs[l].from+lhs[l].count + ar, br := rhs[r].from, rhs[r].from+rhs[r].count + + // check if rows intersect + switch { + case al > br || ar > bl: + // no intersection, advance cursor that ends first + if bl <= br { + l++ + } else { + res = append(res, RowRange{from: ar, count: br - ar}) + r++ + } + case al < ar && bl > br: + // l contains r, complement of l in r is empty, advance r + r++ + case al < ar && bl <= br: + // l covers r from left but has room on top + oe := min(bl, br) + rhs[r].from += oe - ar + rhs[r].count -= oe - ar + l++ + case al >= ar && bl > br: + // l covers r from right but has room on bottom + os := max(al, ar) + res = append(res, RowRange{from: ar, count: os - ar}) + r++ + case al >= ar && bl <= br: + // l is included r + os, oe := max(al, ar), min(bl, br) + res = append(res, RowRange{from: rhs[r].from, count: os - rhs[r].from}) + rhs[r].from = oe + rhs[r].count = br - oe + l++ + } + } + + for ; r < len(rhs); r++ { + res = append(res, rhs[r]) + } + + return simplify(res) +} + +func simplify(rr []RowRange) []RowRange { + if len(rr) == 0 { + return nil + } + + sort.Slice(rr, func(i, j int) bool { + return rr[i].from < rr[j].from + }) + + tmp := make([]RowRange, 0) + l := rr[0] + for i := 1; i < len(rr); i++ { + r := rr[i] + al, bl := l.from, l.from+l.count + ar, br := r.from, r.from+r.count + if bl < ar { + tmp = append(tmp, l) + l = r + continue + } + + from := min(al, ar) + count := max(bl, br) - from + if count == 0 { + continue + } + + l = RowRange{ + from: from, + count: count, + } + } + + tmp = append(tmp, l) + res := make([]RowRange, 0, len(tmp)) + for i := range tmp { + if tmp[i].count != 0 { + res = append(res, tmp[i]) + } + } + + return res +} diff --git a/vendor/github.com/prometheus-community/parquet-common/storage/bucket_read_at.go b/vendor/github.com/prometheus-community/parquet-common/storage/bucket_read_at.go new file mode 100644 index 00000000000..0f022ca8007 --- /dev/null +++ b/vendor/github.com/prometheus-community/parquet-common/storage/bucket_read_at.go @@ -0,0 +1,61 @@ +// Copyright The Prometheus Authors +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package storage + +import ( + "context" + "io" + + "github.com/thanos-io/objstore" +) + +type ReadAtWithContext interface { + io.ReaderAt + WithContext(ctx context.Context) io.ReaderAt +} + +type bReadAt struct { + path string + obj objstore.Bucket + ctx context.Context +} + +func NewBucketReadAt(ctx context.Context, path string, obj objstore.Bucket) ReadAtWithContext { + return &bReadAt{ + path: path, + obj: obj, + ctx: ctx, + } +} + +func (b *bReadAt) WithContext(ctx context.Context) io.ReaderAt { + return &bReadAt{ + path: b.path, + obj: b.obj, + ctx: ctx, + } +} + +func (b *bReadAt) ReadAt(p []byte, off int64) (n int, err error) { + rc, err := b.obj.GetRange(b.ctx, b.path, off, int64(len(p))) + if err != nil { + return 0, err + } + defer func() { _ = rc.Close() }() + n, err = rc.Read(p) + if err == io.EOF { + err = nil + } + return +} diff --git a/vendor/github.com/prometheus-community/parquet-common/storage/parquet_shard.go b/vendor/github.com/prometheus-community/parquet-common/storage/parquet_shard.go new file mode 100644 index 00000000000..628ebc7579d --- /dev/null +++ b/vendor/github.com/prometheus-community/parquet-common/storage/parquet_shard.go @@ -0,0 +1,96 @@ +// Copyright The Prometheus Authors +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package storage + +import ( + "context" + "sync" + + "github.com/parquet-go/parquet-go" + "github.com/thanos-io/objstore" + + "github.com/prometheus-community/parquet-common/schema" +) + +type ParquetFile struct { + *parquet.File + ReadAtWithContext +} + +func (f *ParquetFile) GetPages(ctx context.Context, cc parquet.ColumnChunk) *parquet.FilePages { + colChunk := cc.(*parquet.FileColumnChunk) + pages := colChunk.PagesFrom(f.WithContext(ctx)) + return pages +} + +func OpenFile(r ReadAtWithContext, size int64, options ...parquet.FileOption) (*ParquetFile, error) { + file, err := parquet.OpenFile(r, size, options...) + if err != nil { + return nil, err + } + return &ParquetFile{ + File: file, + ReadAtWithContext: r, + }, nil +} + +type ParquetShard struct { + labelsFile, chunksFile *ParquetFile + schema *schema.TSDBSchema + o sync.Once +} + +// OpenParquetShard opens the sharded parquet block, +// using the options param. +func OpenParquetShard(ctx context.Context, bkt objstore.Bucket, name string, shard int, options ...parquet.FileOption) (*ParquetShard, error) { + labelsFileName := schema.LabelsPfileNameForShard(name, shard) + chunksFileName := schema.ChunksPfileNameForShard(name, shard) + labelsAttr, err := bkt.Attributes(ctx, labelsFileName) + if err != nil { + return nil, err + } + labelsFile, err := OpenFile(NewBucketReadAt(ctx, labelsFileName, bkt), labelsAttr.Size, options...) + if err != nil { + return nil, err + } + + chunksFileAttr, err := bkt.Attributes(ctx, chunksFileName) + if err != nil { + return nil, err + } + chunksFile, err := OpenFile(NewBucketReadAt(ctx, chunksFileName, bkt), chunksFileAttr.Size, options...) + if err != nil { + return nil, err + } + return &ParquetShard{ + labelsFile: labelsFile, + chunksFile: chunksFile, + }, nil +} + +func (b *ParquetShard) LabelsFile() *ParquetFile { + return b.labelsFile +} + +func (b *ParquetShard) ChunksFile() *ParquetFile { + return b.chunksFile +} + +func (b *ParquetShard) TSDBSchema() (*schema.TSDBSchema, error) { + var err error + b.o.Do(func() { + b.schema, err = schema.FromLabelsFile(b.labelsFile.File) + }) + return b.schema, err +} diff --git a/vendor/github.com/prometheus-community/parquet-common/util/bucket_read_at.go b/vendor/github.com/prometheus-community/parquet-common/util/bucket_read_at.go deleted file mode 100644 index d65c46ed505..00000000000 --- a/vendor/github.com/prometheus-community/parquet-common/util/bucket_read_at.go +++ /dev/null @@ -1,35 +0,0 @@ -package util - -import ( - "context" - "io" - - "github.com/thanos-io/objstore" -) - -type bReadAt struct { - path string - obj objstore.Bucket - ctx context.Context -} - -func NewBucketReadAt(ctx context.Context, path string, obj objstore.Bucket) io.ReaderAt { - return &bReadAt{ - path: path, - obj: obj, - ctx: ctx, - } -} - -func (b *bReadAt) ReadAt(p []byte, off int64) (n int, err error) { - rc, err := b.obj.GetRange(b.ctx, b.path, off, int64(len(p))) - if err != nil { - return 0, err - } - defer func() { _ = rc.Close() }() - n, err = rc.Read(p) - if err == io.EOF { - err = nil - } - return -} diff --git a/vendor/github.com/prometheus-community/parquet-common/util/strutil.go b/vendor/github.com/prometheus-community/parquet-common/util/strutil.go new file mode 100644 index 00000000000..f4e777ac83e --- /dev/null +++ b/vendor/github.com/prometheus-community/parquet-common/util/strutil.go @@ -0,0 +1,84 @@ +// Copyright The Prometheus Authors +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package util + +import ( + "sort" + "strings" +) + +// MergeSlices merges a set of sorted string slices into a single ones +// while removing all duplicates. +// If limit is set, only the first limit results will be returned. 0 to disable. +func MergeSlices(limit int, a ...[]string) []string { + if len(a) == 0 { + return nil + } + if len(a) == 1 { + return truncateToLimit(limit, a[0]) + } + l := len(a) / 2 + return mergeTwoStringSlices(limit, MergeSlices(limit, a[:l]...), MergeSlices(limit, a[l:]...)) +} + +// MergeUnsortedSlices behaves like StringSlices but input slices are validated +// for sortedness and are sorted if they are not ordered yet. +// If limit is set, only the first limit results will be returned. 0 to disable. +func MergeUnsortedSlices(limit int, a ...[]string) []string { + for _, s := range a { + if !sort.StringsAreSorted(s) { + sort.Strings(s) + } + } + return MergeSlices(limit, a...) +} + +func mergeTwoStringSlices(limit int, a, b []string) []string { + a = truncateToLimit(limit, a) + b = truncateToLimit(limit, b) + + maxl := len(a) + if len(b) > len(a) { + maxl = len(b) + } + + res := make([]string, 0, maxl*10/9) + + for len(a) > 0 && len(b) > 0 { + d := strings.Compare(a[0], b[0]) + + if d == 0 { + res = append(res, a[0]) + a, b = a[1:], b[1:] + } else if d < 0 { + res = append(res, a[0]) + a = a[1:] + } else if d > 0 { + res = append(res, b[0]) + b = b[1:] + } + } + // Append all remaining elements. + res = append(res, a...) + res = append(res, b...) + res = truncateToLimit(limit, res) + return res +} + +func truncateToLimit(limit int, a []string) []string { + if limit > 0 && len(a) > limit { + return a[:limit] + } + return a +} diff --git a/vendor/github.com/prometheus-community/parquet-common/util/util.go b/vendor/github.com/prometheus-community/parquet-common/util/util.go index 16bcdc0fc1e..4851a51a7a9 100644 --- a/vendor/github.com/prometheus-community/parquet-common/util/util.go +++ b/vendor/github.com/prometheus-community/parquet-common/util/util.go @@ -1,6 +1,27 @@ +// Copyright The Prometheus Authors +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + package util -import "github.com/parquet-go/parquet-go" +import ( + "unsafe" + + "github.com/parquet-go/parquet-go" +) + +func YoloString(buf []byte) string { + return *((*string)(unsafe.Pointer(&buf))) +} func CloneRows(rows []parquet.Row) []parquet.Row { rr := make([]parquet.Row, len(rows)) @@ -9,3 +30,62 @@ func CloneRows(rows []parquet.Row) []parquet.Row { } return rr } + +// Copied from thanos repository: +// https://github.com/thanos-io/thanos/blob/2a5a856e34adb2653dda700c4d87637236afb2dd/pkg/store/bucket.go#L3466 + +type Part struct { + Start uint64 + End uint64 + + ElemRng [2]int +} + +type Partitioner interface { + // Partition partitions length entries into n <= length ranges that cover all + // input ranges + // It supports overlapping ranges. + // NOTE: It expects range to be sorted by start time. + Partition(length int, rng func(int) (uint64, uint64)) []Part +} + +type gapBasedPartitioner struct { + maxGapSize uint64 +} + +func NewGapBasedPartitioner(maxGapSize uint64) Partitioner { + return gapBasedPartitioner{ + maxGapSize: maxGapSize, + } +} + +// Partition partitions length entries into n <= length ranges that cover all +// input ranges by combining entries that are separated by reasonably small gaps. +// It is used to combine multiple small ranges from object storage into bigger, more efficient/cheaper ones. +func (g gapBasedPartitioner) Partition(length int, rng func(int) (uint64, uint64)) (parts []Part) { + j := 0 + k := 0 + for k < length { + j = k + k++ + + p := Part{} + p.Start, p.End = rng(j) + + // Keep growing the range until the end or we encounter a large gap. + for ; k < length; k++ { + s, e := rng(k) + + if p.End+g.maxGapSize < s { + break + } + + if p.End <= e { + p.End = e + } + } + p.ElemRng = [2]int{j, k} + parts = append(parts, p) + } + return parts +} diff --git a/vendor/modules.txt b/vendor/modules.txt index c1c3a285ff3..6dde2be2516 100644 --- a/vendor/modules.txt +++ b/vendor/modules.txt @@ -837,10 +837,12 @@ github.com/pkg/errors # github.com/pmezard/go-difflib v1.0.1-0.20181226105442-5d4384ee4fb2 ## explicit github.com/pmezard/go-difflib/difflib -# github.com/prometheus-community/parquet-common v0.0.0-20250428074311-306c8486441d +# github.com/prometheus-community/parquet-common v0.0.0-20250514003255-382b6ec8ae40 ## explicit; go 1.23.4 github.com/prometheus-community/parquet-common/convert github.com/prometheus-community/parquet-common/schema +github.com/prometheus-community/parquet-common/search +github.com/prometheus-community/parquet-common/storage github.com/prometheus-community/parquet-common/util # github.com/prometheus-community/prom-label-proxy v0.8.1-0.20240127162815-c1195f9aabc0 ## explicit; go 1.20 From dc8fd37acd929037372435abba2f45304b66ae29 Mon Sep 17 00:00:00 2001 From: alanprot Date: Thu, 15 May 2025 08:53:23 -0700 Subject: [PATCH 02/12] changing modules to create the parquet queryable Signed-off-by: alanprot --- pkg/cortex/modules.go | 13 +++++++++++-- pkg/querier/parquet_queryable.go | 12 ++++++------ pkg/querier/querier.go | 4 ++++ 3 files changed, 21 insertions(+), 8 deletions(-) diff --git a/pkg/cortex/modules.go b/pkg/cortex/modules.go index 3002e64667f..a390b989d6d 100644 --- a/pkg/cortex/modules.go +++ b/pkg/cortex/modules.go @@ -400,11 +400,20 @@ func (t *Cortex) initStoreQueryables() (services.Service, error) { var servs []services.Service //nolint:revive // I prefer this form over removing 'else', because it allows q to have smaller scope. + var queriable prom_storage.Queryable if q, err := initQueryableForEngine(t.Cfg, t.Overrides, prometheus.DefaultRegisterer); err != nil { return nil, fmt.Errorf("failed to initialize querier: %v", err) } else { + queriable = q + if t.Cfg.Querier.QueryParquetFiles { + pq, err := querier.NewParquetQueryable(t.Cfg.Querier, t.Cfg.BlocksStorage, t.Overrides, q, util_log.Logger, prometheus.DefaultRegisterer) + if err != nil { + return nil, fmt.Errorf("failed to initialize parquet querier: %v", err) + } + queriable = pq + } t.StoreQueryables = append(t.StoreQueryables, querier.UseAlwaysQueryable(q)) - if s, ok := q.(services.Service); ok { + if s, ok := queriable.(services.Service); ok { servs = append(servs, s) } } @@ -424,7 +433,7 @@ func (t *Cortex) initStoreQueryables() (services.Service, error) { } } -func initQueryableForEngine(cfg Config, limits *validation.Overrides, reg prometheus.Registerer) (prom_storage.Queryable, error) { +func initQueryableForEngine(cfg Config, limits *validation.Overrides, reg prometheus.Registerer) (*querier.BlocksStoreQueryable, error) { // When running in single binary, if the blocks sharding is disabled and no custom // store-gateway address has been configured, we can set it to the running process. if cfg.isModuleEnabled(All) && !cfg.StoreGateway.ShardingEnabled && cfg.Querier.StoreGatewayAddresses == "" { diff --git a/pkg/querier/parquet_queryable.go b/pkg/querier/parquet_queryable.go index 0ae35f364d0..ef885b1857c 100644 --- a/pkg/querier/parquet_queryable.go +++ b/pkg/querier/parquet_queryable.go @@ -35,7 +35,7 @@ type parquetQueryableWithFallback struct { queryStoreAfter time.Duration parquetQueryable storage.Queryable - blockStorageQueryable BlocksStoreQueryable + blockStorageQueryable *BlocksStoreQueryable finder BlocksFinder @@ -44,11 +44,11 @@ type parquetQueryableWithFallback struct { subservicesWatcher *services.FailureWatcher } -func newParquetQueryable( +func NewParquetQueryable( + config Config, storageCfg cortex_tsdb.BlocksStorageConfig, limits BlocksStoreLimits, - config Config, - blockStorageQueryable BlocksStoreQueryable, + blockStorageQueryable *BlocksStoreQueryable, logger log.Logger, reg prometheus.Registerer, ) (storage.Queryable, error) { @@ -112,7 +112,7 @@ func newParquetQueryable( return shards, nil }) - q := &parquetQueryableWithFallback{ + p := &parquetQueryableWithFallback{ subservices: manager, blockStorageQueryable: blockStorageQueryable, parquetQueryable: pq, @@ -121,7 +121,7 @@ func newParquetQueryable( finder: finder, } - q.Service = services.NewBasicService(q.starting, q.running, q.stopping) + p.Service = services.NewBasicService(p.starting, p.running, p.stopping) return pq, nil } diff --git a/pkg/querier/querier.go b/pkg/querier/querier.go index 3bf8be517cf..0aab0c5f472 100644 --- a/pkg/querier/querier.go +++ b/pkg/querier/querier.go @@ -90,6 +90,9 @@ type Config struct { // Ignore max query length check at Querier. IgnoreMaxQueryLength bool `yaml:"ignore_max_query_length"` EnablePromQLExperimentalFunctions bool `yaml:"enable_promql_experimental_functions"` + + // Query Parquet files if available + QueryParquetFiles bool `yaml:"query_parquet_files" doc:"hidden"` } var ( @@ -135,6 +138,7 @@ func (cfg *Config) RegisterFlags(f *flag.FlagSet) { f.Int64Var(&cfg.MaxSubQuerySteps, "querier.max-subquery-steps", 0, "Max number of steps allowed for every subquery expression in query. Number of steps is calculated using subquery range / step. A value > 0 enables it.") f.BoolVar(&cfg.IgnoreMaxQueryLength, "querier.ignore-max-query-length", false, "If enabled, ignore max query length check at Querier select method. Users can choose to ignore it since the validation can be done before Querier evaluation like at Query Frontend or Ruler.") f.BoolVar(&cfg.EnablePromQLExperimentalFunctions, "querier.enable-promql-experimental-functions", false, "[Experimental] If true, experimental promQL functions are enabled.") + f.BoolVar(&cfg.QueryParquetFiles, "querier.query-parquet-files", false, "[Experimental] If true, querier will try to query the parquet files if available.") } // Validate the config From 931efd047b9500feff7944919e40ac324786bc4e Mon Sep 17 00:00:00 2001 From: alanprot Date: Tue, 20 May 2025 17:05:14 -0700 Subject: [PATCH 03/12] adding ParquetConverterEnabled config Signed-off-by: alanprot --- pkg/compactor/blocks_cleaner.go | 5 +++++ pkg/compactor/blocks_cleaner_test.go | 4 ++++ pkg/compactor/compactor.go | 1 + pkg/parquetconverter/converter.go | 4 ++++ pkg/parquetconverter/converter_test.go | 5 ++++- pkg/querier/parquet_queryable.go | 2 +- pkg/util/validation/limits.go | 9 ++++++++- 7 files changed, 27 insertions(+), 3 deletions(-) diff --git a/pkg/compactor/blocks_cleaner.go b/pkg/compactor/blocks_cleaner.go index 4c2e4887b08..03e7564b0a6 100644 --- a/pkg/compactor/blocks_cleaner.go +++ b/pkg/compactor/blocks_cleaner.go @@ -597,6 +597,11 @@ func (c *BlocksCleaner) cleanUser(ctx context.Context, userLogger log.Logger, us // Generate an updated in-memory version of the bucket index. begin = time.Now() w := bucketindex.NewUpdater(c.bucketClient, userID, c.cfgProvider, c.logger) + + if c.cfgProvider.ParquetConverterEnabled(userID) { + w.EnableParquet() + } + idx, partials, totalBlocksBlocksMarkedForNoCompaction, err := w.UpdateIndex(ctx, idx) if err != nil { idxs.Status = bucketindex.GenericError diff --git a/pkg/compactor/blocks_cleaner_test.go b/pkg/compactor/blocks_cleaner_test.go index 82f477cea65..17583f7d80f 100644 --- a/pkg/compactor/blocks_cleaner_test.go +++ b/pkg/compactor/blocks_cleaner_test.go @@ -984,6 +984,10 @@ type mockConfigProvider struct { userRetentionPeriods map[string]time.Duration } +func (m *mockConfigProvider) ParquetConverterEnabled(userID string) bool { + return false +} + func newMockConfigProvider() *mockConfigProvider { return &mockConfigProvider{ userRetentionPeriods: make(map[string]time.Duration), diff --git a/pkg/compactor/compactor.go b/pkg/compactor/compactor.go index c18a874b5df..602fd0bddbe 100644 --- a/pkg/compactor/compactor.go +++ b/pkg/compactor/compactor.go @@ -388,6 +388,7 @@ func (cfg *Config) Validate(limits validation.Limits) error { // ConfigProvider defines the per-tenant config provider for the Compactor. type ConfigProvider interface { bucket.TenantConfigProvider + ParquetConverterEnabled(userID string) bool CompactorBlocksRetentionPeriod(user string) time.Duration } diff --git a/pkg/parquetconverter/converter.go b/pkg/parquetconverter/converter.go index 2b6cc8ee238..4e1005f7bbd 100644 --- a/pkg/parquetconverter/converter.go +++ b/pkg/parquetconverter/converter.go @@ -171,6 +171,10 @@ func (c *Converter) running(ctx context.Context) error { } ownedUsers := map[string]struct{}{} for _, userID := range users { + if !c.limits.ParquetConverterEnabled(userID) { + continue + } + var ring ring.ReadRing ring = c.ring if c.limits.ParquetConverterTenantShardSize(userID) > 0 { diff --git a/pkg/parquetconverter/converter_test.go b/pkg/parquetconverter/converter_test.go index c3ebb2bf706..3e6c9705122 100644 --- a/pkg/parquetconverter/converter_test.go +++ b/pkg/parquetconverter/converter_test.go @@ -45,8 +45,11 @@ func TestConverter(t *testing.T) { bucketClient, err := filesystem.NewBucket(t.TempDir()) require.NoError(t, err) userBucket := bucket.NewPrefixedBucketClient(bucketClient, user) + limits := &validation.Limits{} + flagext.DefaultValues(limits) + limits.ParquetConverterEnabled = true - c, logger, _ := prepare(t, cfg, objstore.WithNoopInstr(bucketClient), nil) + c, logger, _ := prepare(t, cfg, objstore.WithNoopInstr(bucketClient), limits) ctx := context.Background() diff --git a/pkg/querier/parquet_queryable.go b/pkg/querier/parquet_queryable.go index ef885b1857c..507019fe05b 100644 --- a/pkg/querier/parquet_queryable.go +++ b/pkg/querier/parquet_queryable.go @@ -84,7 +84,7 @@ func NewParquetQueryable( }, bucketClient, limits, logger, reg) } - manager, err := services.NewManager(finder) + manager, err := services.NewManager(finder, blockStorageQueryable) if err != nil { return nil, err } diff --git a/pkg/util/validation/limits.go b/pkg/util/validation/limits.go index e598422f9f1..d64eb9486c6 100644 --- a/pkg/util/validation/limits.go +++ b/pkg/util/validation/limits.go @@ -202,7 +202,8 @@ type Limits struct { CompactorPartitionSeriesCount int64 `yaml:"compactor_partition_series_count" json:"compactor_partition_series_count"` // Parquet converter - ParquetConverterTenantShardSize int `yaml:"parquet_converter_tenant_shard_size" json:"parquet_converter_tenant_shard_size" doc:"hidden"` + ParquetConverterEnabled bool `yaml:"parquet_converter_enabled" doc:"hidden"` + ParquetConverterTenantShardSize int `yaml:"parquet_converter_tenant_shard_size" json:"parquet_converter_tenant_shard_size" doc:"hidden"` // This config doesn't have a CLI flag registered here because they're registered in // their own original config struct. @@ -300,6 +301,7 @@ func (l *Limits) RegisterFlags(f *flag.FlagSet) { f.Int64Var(&l.CompactorPartitionSeriesCount, "compactor.partition-series-count", 0, "Time series count limit for each compaction partition. 0 means no limit") f.IntVar(&l.ParquetConverterTenantShardSize, "parquet-converter.tenant-shard-size", 0, "The default tenant's shard size when the shuffle-sharding strategy is used by the parquet converter. When this setting is specified in the per-tenant overrides, a value of 0 disables shuffle sharding for the tenant.") + f.BoolVar(&l.ParquetConverterEnabled, "parquet-converter.enabled", false, "If set, enables the Parquet converter to create the parquet files.") // Store-gateway. f.Float64Var(&l.StoreGatewayTenantShardSize, "store-gateway.tenant-shard-size", 0, "The default tenant's shard size when the shuffle-sharding strategy is used. Must be set when the store-gateway sharding is enabled with the shuffle-sharding strategy. When this setting is specified in the per-tenant overrides, a value of 0 disables shuffle sharding for the tenant. If the value is < 1 the shard size will be a percentage of the total store-gateways.") @@ -839,6 +841,11 @@ func (o *Overrides) ParquetConverterTenantShardSize(userID string) int { return o.GetOverridesForUser(userID).ParquetConverterTenantShardSize } +// ParquetConverterEnabled returns true is parquet is enabled. +func (o *Overrides) ParquetConverterEnabled(userID string) bool { + return o.GetOverridesForUser(userID).ParquetConverterEnabled +} + // CompactorPartitionIndexSizeBytes returns shard size (number of rulers) used by this tenant when using shuffle-sharding strategy. func (o *Overrides) CompactorPartitionIndexSizeBytes(userID string) int64 { return o.GetOverridesForUser(userID).CompactorPartitionIndexSizeBytes From 013fa9312195313e82e24d5fdcc7eb10d7b67584 Mon Sep 17 00:00:00 2001 From: alanprot Date: Tue, 20 May 2025 17:39:02 -0700 Subject: [PATCH 04/12] BlockStoreQueryable: querying only blocks in the context when present Signed-off-by: alanprot --- pkg/querier/block.go | 20 ++++++++++ pkg/querier/blocks_store_queryable.go | 5 +++ pkg/querier/blocks_store_queryable_test.go | 43 ++++++++++++++++++++++ pkg/querier/parquet_queryable.go | 25 ++++++------- pkg/util/validation/limits.go | 2 +- 5 files changed, 80 insertions(+), 15 deletions(-) diff --git a/pkg/querier/block.go b/pkg/querier/block.go index d4d53d09243..b431ea1afa4 100644 --- a/pkg/querier/block.go +++ b/pkg/querier/block.go @@ -1,10 +1,30 @@ package querier import ( + "context" + "github.com/cortexproject/cortex/pkg/storage/tsdb/bucketindex" "github.com/prometheus/prometheus/model/labels" "github.com/thanos-io/thanos/pkg/store/storepb" ) +type contextKey int + +var ( + blockCtxKey contextKey = 0 +) + +func InjectBlocksIntoContext(ctx context.Context, blocks ...*bucketindex.Block) context.Context { + return context.WithValue(ctx, blockCtxKey, blocks) +} + +func ExtractBlocksFromContext(ctx context.Context) ([]*bucketindex.Block, bool) { + if blocks := ctx.Value(blockCtxKey); blocks != nil { + return blocks.([]*bucketindex.Block), true + } + + return nil, false +} + func convertMatchersToLabelMatcher(matchers []*labels.Matcher) []storepb.LabelMatcher { var converted []storepb.LabelMatcher for _, m := range matchers { diff --git a/pkg/querier/blocks_store_queryable.go b/pkg/querier/blocks_store_queryable.go index a5647e55451..83de51b28fc 100644 --- a/pkg/querier/blocks_store_queryable.go +++ b/pkg/querier/blocks_store_queryable.go @@ -513,6 +513,11 @@ func (q *blocksStoreQuerier) queryWithConsistencyCheck(ctx context.Context, logg // Find the list of blocks we need to query given the time range. knownBlocks, knownDeletionMarks, err := q.finder.GetBlocks(ctx, userID, minT, maxT) + + // if blocks were already discovered, we should use then + if b, ok := ExtractBlocksFromContext(ctx); ok { + knownBlocks = b + } if err != nil { return err } diff --git a/pkg/querier/blocks_store_queryable_test.go b/pkg/querier/blocks_store_queryable_test.go index 3ca5c03fee2..5bd99dbb0be 100644 --- a/pkg/querier/blocks_store_queryable_test.go +++ b/pkg/querier/blocks_store_queryable_test.go @@ -1648,6 +1648,49 @@ func TestBlocksStoreQuerier_Select(t *testing.T) { } } +func TestOverrideBlockDiscovery(t *testing.T) { + block1 := ulid.MustNew(1, nil) + block2 := ulid.MustNew(2, nil) + minT := int64(10) + maxT := int64(20) + + stores := &blocksStoreSetMock{mockedResponses: []interface{}{ + map[BlocksStoreClient][]ulid.ULID{ + &storeGatewayClientMock{remoteAddr: "1.1.1.1", mockedSeriesResponses: []*storepb.SeriesResponse{ + mockHintsResponse(block1), + }}: {block1}, + }, + }, + } + finder := &blocksFinderMock{} + // return block 1 and 2 on finder but only query block 1 + finder.On("GetBlocks", mock.Anything, "user-1", minT, maxT).Return(bucketindex.Blocks{ + &bucketindex.Block{ID: block1}, + &bucketindex.Block{ID: block2}, + }, map[ulid.ULID]*bucketindex.BlockDeletionMark(nil), nil) + + 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, + } + + matchers := []*labels.Matcher{ + labels.MustNewMatcher(labels.MatchEqual, labels.MetricName, "name"), + } + ctx := user.InjectOrgID(context.Background(), "user-1") + ctx = InjectBlocksIntoContext(ctx, &bucketindex.Block{ID: block1}) + ss := q.Select(ctx, true, nil, matchers...) + require.NoError(t, ss.Err()) +} + func TestBlocksStoreQuerier_Labels(t *testing.T) { t.Parallel() diff --git a/pkg/querier/parquet_queryable.go b/pkg/querier/parquet_queryable.go index 507019fe05b..439fd4806ed 100644 --- a/pkg/querier/parquet_queryable.go +++ b/pkg/querier/parquet_queryable.go @@ -24,12 +24,6 @@ import ( "github.com/cortexproject/cortex/pkg/util/services" ) -type contextKey int - -var ( - blockIdsCtxKey contextKey = 0 -) - type parquetQueryableWithFallback struct { services.Service @@ -95,7 +89,10 @@ func NewParquetQueryable( return nil, err } - blocks := ctx.Value(blockIdsCtxKey).([]*bucketindex.Block) + blocks, ok := ExtractBlocksFromContext(ctx) + if !ok { + return nil, errors.Errorf("failed to extract blocks from context") + } userBkt := bucket.NewUserBucketClient(userID, bucketClient, limits) shards := make([]*parquet_storage.ParquetShard, 0, len(blocks)) @@ -123,7 +120,7 @@ func NewParquetQueryable( p.Service = services.NewBasicService(p.starting, p.running, p.stopping) - return pq, nil + return pq, err } func (p *parquetQueryableWithFallback) starting(ctx context.Context) error { @@ -201,7 +198,7 @@ func (q *parquetQuerier) LabelValues(ctx context.Context, name string, hints *st ) if len(parquet) > 0 { - res, ann, qErr := q.parquetQuerier.LabelValues(context.WithValue(ctx, blockIdsCtxKey, parquet), name, hints, matchers...) + res, ann, qErr := q.parquetQuerier.LabelValues(InjectBlocksIntoContext(ctx, parquet...), name, hints, matchers...) if qErr != nil { return nil, nil, err } @@ -210,7 +207,7 @@ func (q *parquetQuerier) LabelValues(ctx context.Context, name string, hints *st } if len(remaining) > 0 { - res, ann, qErr := q.blocksStoreQuerier.LabelValues(context.WithValue(ctx, blockIdsCtxKey, remaining), name, hints, matchers...) + res, ann, qErr := q.blocksStoreQuerier.LabelValues(InjectBlocksIntoContext(ctx, remaining...), name, hints, matchers...) if qErr != nil { return nil, nil, err } @@ -247,7 +244,7 @@ func (q *parquetQuerier) LabelNames(ctx context.Context, hints *storage.LabelHin ) if len(parquet) > 0 { - res, ann, qErr := q.parquetQuerier.LabelNames(context.WithValue(ctx, blockIdsCtxKey, parquet), hints, matchers...) + res, ann, qErr := q.parquetQuerier.LabelNames(InjectBlocksIntoContext(ctx, parquet...), hints, matchers...) if qErr != nil { return nil, nil, err } @@ -256,7 +253,7 @@ func (q *parquetQuerier) LabelNames(ctx context.Context, hints *storage.LabelHin } if len(remaining) > 0 { - res, ann, qErr := q.blocksStoreQuerier.LabelNames(context.WithValue(ctx, blockIdsCtxKey, remaining), hints, matchers...) + res, ann, qErr := q.blocksStoreQuerier.LabelNames(InjectBlocksIntoContext(ctx, remaining...), hints, matchers...) if qErr != nil { return nil, nil, err } @@ -290,11 +287,11 @@ func (q *parquetQuerier) Select(ctx context.Context, sortSeries bool, hints *sto serieSets := []storage.SeriesSet{} if len(parquet) > 0 { - serieSets = append(serieSets, q.parquetQuerier.Select(context.WithValue(ctx, blockIdsCtxKey, parquet), sortSeries, hints, matchers...)) + serieSets = append(serieSets, q.parquetQuerier.Select(InjectBlocksIntoContext(ctx, parquet...), sortSeries, hints, matchers...)) } if len(remaining) > 0 { - serieSets = append(serieSets, q.blocksStoreQuerier.Select(context.WithValue(ctx, blockIdsCtxKey, remaining), sortSeries, hints, matchers...)) + serieSets = append(serieSets, q.blocksStoreQuerier.Select(InjectBlocksIntoContext(ctx, remaining...), sortSeries, hints, matchers...)) } if len(serieSets) == 1 { diff --git a/pkg/util/validation/limits.go b/pkg/util/validation/limits.go index d64eb9486c6..8d46235ebe6 100644 --- a/pkg/util/validation/limits.go +++ b/pkg/util/validation/limits.go @@ -202,7 +202,7 @@ type Limits struct { CompactorPartitionSeriesCount int64 `yaml:"compactor_partition_series_count" json:"compactor_partition_series_count"` // Parquet converter - ParquetConverterEnabled bool `yaml:"parquet_converter_enabled" doc:"hidden"` + ParquetConverterEnabled bool `yaml:"parquet_converter_enabled" json:"parquet_converter_enabled" doc:"hidden"` ParquetConverterTenantShardSize int `yaml:"parquet_converter_tenant_shard_size" json:"parquet_converter_tenant_shard_size" doc:"hidden"` // This config doesn't have a CLI flag registered here because they're registered in From 7c60669b9eab808bf74c052bd84195cc34cf5efe Mon Sep 17 00:00:00 2001 From: alanprot Date: Wed, 21 May 2025 09:45:57 -0700 Subject: [PATCH 05/12] test parquet querier fallback logic Signed-off-by: alanprot --- pkg/querier/block.go | 6 +- pkg/querier/blocks_store_queryable_test.go | 9 +- pkg/querier/parquet_queryable_test.go | 269 +++++++++++++++++++++ 3 files changed, 281 insertions(+), 3 deletions(-) create mode 100644 pkg/querier/parquet_queryable_test.go diff --git a/pkg/querier/block.go b/pkg/querier/block.go index b431ea1afa4..bdd93a75b92 100644 --- a/pkg/querier/block.go +++ b/pkg/querier/block.go @@ -2,9 +2,11 @@ package querier import ( "context" - "github.com/cortexproject/cortex/pkg/storage/tsdb/bucketindex" + "github.com/prometheus/prometheus/model/labels" "github.com/thanos-io/thanos/pkg/store/storepb" + + "github.com/cortexproject/cortex/pkg/storage/tsdb/bucketindex" ) type contextKey int @@ -21,7 +23,7 @@ func ExtractBlocksFromContext(ctx context.Context) ([]*bucketindex.Block, bool) if blocks := ctx.Value(blockCtxKey); blocks != nil { return blocks.([]*bucketindex.Block), true } - + return nil, false } diff --git a/pkg/querier/blocks_store_queryable_test.go b/pkg/querier/blocks_store_queryable_test.go index 5bd99dbb0be..738f26fb93a 100644 --- a/pkg/querier/blocks_store_queryable_test.go +++ b/pkg/querier/blocks_store_queryable_test.go @@ -2502,12 +2502,14 @@ type blocksStoreSetMock struct { mockedResponses []interface{} nextResult int + queriedBlocks []ulid.ULID } -func (m *blocksStoreSetMock) GetClientsFor(_ string, _ []ulid.ULID, _ map[ulid.ULID][]string, _ map[ulid.ULID]map[string]int) (map[BlocksStoreClient][]ulid.ULID, error) { +func (m *blocksStoreSetMock) GetClientsFor(_ string, b []ulid.ULID, _ map[ulid.ULID][]string, _ map[ulid.ULID]map[string]int) (map[BlocksStoreClient][]ulid.ULID, error) { if m.nextResult >= len(m.mockedResponses) { panic("not enough mocked results") } + m.queriedBlocks = append(m.queriedBlocks, b...) res := m.mockedResponses[m.nextResult] m.nextResult++ @@ -2522,6 +2524,11 @@ func (m *blocksStoreSetMock) GetClientsFor(_ string, _ []ulid.ULID, _ map[ulid.U return nil, errors.New("unknown data type in the mocked result") } +func (m *blocksStoreSetMock) Reset() { + m.nextResult = 0 + m.queriedBlocks = nil +} + type blocksFinderMock struct { services.Service mock.Mock diff --git a/pkg/querier/parquet_queryable_test.go b/pkg/querier/parquet_queryable_test.go new file mode 100644 index 00000000000..d948ee58761 --- /dev/null +++ b/pkg/querier/parquet_queryable_test.go @@ -0,0 +1,269 @@ +package querier + +import ( + "context" + "testing" + + "github.com/go-kit/log" + "github.com/oklog/ulid" + "github.com/prometheus/client_golang/prometheus" + "github.com/prometheus/prometheus/model/labels" + "github.com/prometheus/prometheus/storage" + "github.com/prometheus/prometheus/util/annotations" + "github.com/stretchr/testify/mock" + "github.com/stretchr/testify/require" + "github.com/thanos-io/thanos/pkg/store/storepb" + "github.com/weaveworks/common/user" + + "github.com/cortexproject/cortex/pkg/cortexpb" + "github.com/cortexproject/cortex/pkg/querier/series" + "github.com/cortexproject/cortex/pkg/storage/parquet" + "github.com/cortexproject/cortex/pkg/storage/tsdb/bucketindex" +) + +func TestParquetQueryableFallbackLogic(t *testing.T) { + block1 := ulid.MustNew(1, nil) + block2 := ulid.MustNew(2, nil) + minT := int64(10) + maxT := int64(20) + + stores := &blocksStoreSetMock{mockedResponses: []interface{}{ + map[BlocksStoreClient][]ulid.ULID{ + &storeGatewayClientMock{remoteAddr: "1.1.1.1", + mockedSeriesResponses: []*storepb.SeriesResponse{ + mockSeriesResponse(labels.Labels{{Name: labels.MetricName, Value: "fromSg"}}, []cortexpb.Sample{{Value: 1, TimestampMs: minT}, {Value: 2, TimestampMs: minT + 1}}, nil, nil), + mockHintsResponse(block1, block2), + }, + mockedLabelNamesResponse: &storepb.LabelNamesResponse{ + Names: namesFromSeries(labels.FromMap(map[string]string{labels.MetricName: "fromSg", "fromSg": "fromSg"})), + Warnings: []string{}, + Hints: mockNamesHints(block1, block2), + }, + mockedLabelValuesResponse: &storepb.LabelValuesResponse{ + Values: valuesFromSeries(labels.MetricName, labels.FromMap(map[string]string{labels.MetricName: "fromSg", "fromSg": "fromSg"})), + Warnings: []string{}, + Hints: mockValuesHints(block1, block2), + }, + }: {block1, block2}}, + }, + } + + matchers := []*labels.Matcher{ + labels.MustNewMatcher(labels.MatchEqual, labels.MetricName, "fromSg"), + } + ctx := user.InjectOrgID(context.Background(), "user-1") + t.Run("should fallback all blocks", func(t *testing.T) { + finder := &blocksFinderMock{} + + 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 := &parquetQuerier{ + minT: minT, + maxT: maxT, + finder: finder, + blocksStoreQuerier: q, + parquetQuerier: mParquetQuerier, + } + + finder.On("GetBlocks", mock.Anything, "user-1", minT, maxT).Return(bucketindex.Blocks{ + &bucketindex.Block{ID: block1}, + &bucketindex.Block{ID: block2}, + }, 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("labelNames", func(t *testing.T) { + stores.Reset() + mParquetQuerier.Reset() + _, _, err := pq.LabelNames(ctx, nil, matchers...) + require.NoError(t, err) + require.Len(t, stores.queriedBlocks, 2) + require.Len(t, mParquetQuerier.queriedBlocks, 0) + }) + + t.Run("labelValues", func(t *testing.T) { + stores.Reset() + mParquetQuerier.Reset() + _, _, err := pq.LabelValues(ctx, labels.MetricName, nil, matchers...) + require.NoError(t, err) + require.Len(t, stores.queriedBlocks, 2) + require.Len(t, mParquetQuerier.queriedBlocks, 0) + }) + }) + + t.Run("should fallback partial blocks", func(t *testing.T) { + finder := &blocksFinderMock{} + + 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 := &parquetQuerier{ + minT: minT, + maxT: maxT, + finder: finder, + blocksStoreQuerier: q, + parquetQuerier: mParquetQuerier, + } + + 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}, + }, 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...) + require.NoError(t, ss.Err()) + require.Len(t, stores.queriedBlocks, 1) + require.Len(t, mParquetQuerier.queriedBlocks, 1) + }) + + t.Run("labelNames", func(t *testing.T) { + stores.Reset() + mParquetQuerier.Reset() + r, _, err := pq.LabelNames(ctx, nil, matchers...) + require.NoError(t, err) + require.Len(t, stores.queriedBlocks, 1) + require.Len(t, mParquetQuerier.queriedBlocks, 1) + require.Contains(t, r, "fromSg") + require.Contains(t, r, "fromParquet") + }) + + t.Run("labelValues", func(t *testing.T) { + stores.Reset() + mParquetQuerier.Reset() + r, _, err := pq.LabelValues(ctx, labels.MetricName, nil, matchers...) + require.NoError(t, err) + require.Len(t, stores.queriedBlocks, 1) + require.Len(t, mParquetQuerier.queriedBlocks, 1) + require.Contains(t, r, "fromSg") + require.Contains(t, r, "fromParquet") + }) + }) + + t.Run("should query only parquet blocks when possible", func(t *testing.T) { + finder := &blocksFinderMock{} + + 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 := &parquetQuerier{ + minT: minT, + maxT: maxT, + finder: finder, + blocksStoreQuerier: q, + parquetQuerier: mParquetQuerier, + } + + 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) { + stores.Reset() + mParquetQuerier.Reset() + ss := pq.Select(ctx, true, nil, matchers...) + require.NoError(t, ss.Err()) + require.Len(t, stores.queriedBlocks, 0) + require.Len(t, mParquetQuerier.queriedBlocks, 2) + }) + + t.Run("labelNames", func(t *testing.T) { + stores.Reset() + mParquetQuerier.Reset() + r, _, err := pq.LabelNames(ctx, nil, matchers...) + require.NoError(t, err) + require.Len(t, stores.queriedBlocks, 0) + require.Len(t, mParquetQuerier.queriedBlocks, 2) + require.NotContains(t, r, "fromSg") + require.Contains(t, r, "fromParquet") + }) + + t.Run("labelValues", func(t *testing.T) { + stores.Reset() + mParquetQuerier.Reset() + r, _, err := pq.LabelValues(ctx, labels.MetricName, nil, matchers...) + require.NoError(t, err) + require.Len(t, stores.queriedBlocks, 0) + require.Len(t, mParquetQuerier.queriedBlocks, 2) + require.NotContains(t, r, "fromSg") + require.Contains(t, r, "fromParquet") + }) + }) + +} + +type mockParquetQuerier struct { + queriedBlocks []*bucketindex.Block +} + +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...) + } + + return series.NewConcreteSeriesSet(sortSeries, nil) +} + +func (m *mockParquetQuerier) LabelValues(ctx context.Context, name string, _ *storage.LabelHints, matchers ...*labels.Matcher) ([]string, annotations.Annotations, error) { + if blocks, ok := ExtractBlocksFromContext(ctx); ok { + m.queriedBlocks = append(m.queriedBlocks, blocks...) + } + return []string{"fromParquet"}, nil, nil +} + +func (m *mockParquetQuerier) LabelNames(ctx context.Context, _ *storage.LabelHints, matchers ...*labels.Matcher) ([]string, annotations.Annotations, error) { + if blocks, ok := ExtractBlocksFromContext(ctx); ok { + m.queriedBlocks = append(m.queriedBlocks, blocks...) + } + return []string{"fromParquet"}, nil, nil +} + +func (m *mockParquetQuerier) Reset() { + m.queriedBlocks = nil +} + +func (mockParquetQuerier) Close() error { + return nil +} From 2f028723f17f72cd598ad221ba7218e95e29d4a5 Mon Sep 17 00:00:00 2001 From: alanprot Date: Wed, 21 May 2025 10:17:13 -0700 Subject: [PATCH 06/12] changelog Signed-off-by: alanprot --- CHANGELOG.md | 2 +- pkg/cortex/modules.go | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index e906e883aa0..34934c971a5 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -11,7 +11,7 @@ * [FEATURE] Ingester: Support out-of-order native histogram ingestion. It automatically enabled when `-ingester.out-of-order-time-window > 0` and `-blocks-storage.tsdb.enable-native-histograms=true`. #6626 #6663 * [FEATURE] Ruler: Add support for percentage based sharding for rulers. #6680 * [FEATURE] Ruler: Add support for group labels. #6665 -* [FEATURE] Support Parquet format: Implement parquet converter service to convert a TSDB block into Parquet. #6716 +* [FEATURE] Experimental Support Parquet format: Implement parquet converter service to convert a TSDB block into Parquet and Parquet Queryable. #6716 #6743 * [FEATURE] Distributor/Ingester: Implemented experimental feature to use gRPC stream connection for push requests. This can be enabled by setting `-distributor.use-stream-push=true`. #6580 * [FEATURE] Compactor: Add support for percentage based sharding for compactors. #6738 * [ENHANCEMENT] Query Frontend: Change to return 400 when the tenant resolving fail. #6715 diff --git a/pkg/cortex/modules.go b/pkg/cortex/modules.go index a390b989d6d..eca2cc7d302 100644 --- a/pkg/cortex/modules.go +++ b/pkg/cortex/modules.go @@ -412,7 +412,7 @@ func (t *Cortex) initStoreQueryables() (services.Service, error) { } queriable = pq } - t.StoreQueryables = append(t.StoreQueryables, querier.UseAlwaysQueryable(q)) + t.StoreQueryables = append(t.StoreQueryables, querier.UseAlwaysQueryable(queriable)) if s, ok := queriable.(services.Service); ok { servs = append(servs, s) } From 9a183e09fec0ffaf9c725f1743b888657f942095 Mon Sep 17 00:00:00 2001 From: alanprot Date: Wed, 21 May 2025 17:15:19 -0700 Subject: [PATCH 07/12] Some bugfixes and change in the config Signed-off-by: alanprot --- pkg/parquetconverter/converter.go | 23 +++------ pkg/querier/parquet_queryable.go | 72 +++++++++++++-------------- pkg/querier/parquet_queryable_test.go | 3 ++ 3 files changed, 45 insertions(+), 53 deletions(-) diff --git a/pkg/parquetconverter/converter.go b/pkg/parquetconverter/converter.go index 4e1005f7bbd..379a60edca4 100644 --- a/pkg/parquetconverter/converter.go +++ b/pkg/parquetconverter/converter.go @@ -30,7 +30,6 @@ import ( cortex_tsdb "github.com/cortexproject/cortex/pkg/storage/tsdb" "github.com/cortexproject/cortex/pkg/storage/tsdb/bucketindex" "github.com/cortexproject/cortex/pkg/util" - "github.com/cortexproject/cortex/pkg/util/flagext" util_log "github.com/cortexproject/cortex/pkg/util/log" "github.com/cortexproject/cortex/pkg/util/services" "github.com/cortexproject/cortex/pkg/util/validation" @@ -46,10 +45,8 @@ const ( var RingOp = ring.NewOp([]ring.InstanceState{ring.ACTIVE}, nil) type Config struct { - EnabledTenants flagext.StringSliceCSV `yaml:"enabled_tenants"` - DisabledTenants flagext.StringSliceCSV `yaml:"disabled_tenants"` - MetaSyncConcurrency int `yaml:"meta_sync_concurrency"` - ConversionInterval time.Duration `yaml:"conversion_interval"` + MetaSyncConcurrency int `yaml:"meta_sync_concurrency"` + ConversionInterval time.Duration `yaml:"conversion_interval"` DataDir string `yaml:"data_dir"` @@ -64,8 +61,7 @@ type Converter struct { cfg Config storageCfg cortex_tsdb.BlocksStorageConfig - allowedTenants *util.AllowedTenants - limits *validation.Overrides + limits *validation.Overrides // Ring used for sharding compactions. ringLifecycler *ring.Lifecycler @@ -87,8 +83,6 @@ type Converter struct { func (cfg *Config) RegisterFlags(f *flag.FlagSet) { cfg.Ring.RegisterFlags(f) - f.Var(&cfg.EnabledTenants, "parquet-converter.enabled-tenants", "Comma separated list of tenants that can be converted. If specified, only these tenants will be converted, otherwise all tenants can be converted.") - f.Var(&cfg.DisabledTenants, "parquet-converter.disabled-tenants", "Comma separated list of tenants that cannot converted.") f.StringVar(&cfg.DataDir, "parquet-converter.data-dir", "./data", "Data directory in which to cache blocks and process conversions.") f.IntVar(&cfg.MetaSyncConcurrency, "parquet-converter.meta-sync-concurrency", 20, "Number of Go routines to use when syncing block meta files from the long term storage.") f.DurationVar(&cfg.ConversionInterval, "parquet-converter.conversion-interval", time.Minute, "The frequency at which the conversion job runs.") @@ -107,7 +101,6 @@ func newConverter(cfg Config, bkt objstore.InstrumentedBucket, storageCfg cortex reg: registerer, storageCfg: storageCfg, logger: logger, - allowedTenants: util.NewAllowedTenants(cfg.EnabledTenants, cfg.DisabledTenants), limits: limits, pool: chunkenc.NewPool(), blockRanges: blockRanges, @@ -379,15 +372,11 @@ func (c *Converter) convertUser(ctx context.Context, logger log.Logger, ring rin return nil } -func (c *Converter) ownUser(r ring.ReadRing, userID string) (bool, error) { - if !c.allowedTenants.IsAllowed(userID) { +func (c *Converter) ownUser(r ring.ReadRing, userId string) (bool, error) { + if userId == util.GlobalMarkersDir { + // __markers__ is reserved for global markers and no tenant should be allowed to have that name. return false, nil } - - if c.limits.ParquetConverterTenantShardSize(userID) <= 0 { - return true, nil - } - rs, err := r.GetAllHealthy(RingOp) if err != nil { return false, err diff --git a/pkg/querier/parquet_queryable.go b/pkg/querier/parquet_queryable.go index 439fd4806ed..41d03d1be4d 100644 --- a/pkg/querier/parquet_queryable.go +++ b/pkg/querier/parquet_queryable.go @@ -2,16 +2,18 @@ package querier import ( "context" - "fmt" "time" "github.com/go-kit/log" "github.com/pkg/errors" + "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" "github.com/prometheus/prometheus/model/labels" "github.com/prometheus/prometheus/storage" + "github.com/prometheus/prometheus/tsdb/chunkenc" "github.com/prometheus/prometheus/util/annotations" "github.com/thanos-io/thanos/pkg/strutil" @@ -24,6 +26,19 @@ import ( "github.com/cortexproject/cortex/pkg/util/services" ) +type parquetQueryableFallbackMetrics struct { + blocksQueriedTotal *prometheus.CounterVec +} + +func newParquetQueryableFallbackMetrics(reg prometheus.Registerer) *parquetQueryableFallbackMetrics { + return &parquetQueryableFallbackMetrics{ + blocksQueriedTotal: promauto.With(reg).NewCounterVec(prometheus.CounterOpts{ + Name: "cortex_parquet_queryable_blocks_queried_total", + Help: "Total number of blocks found to query.", + }, []string{"type"}), + } +} + type parquetQueryableWithFallback struct { services.Service @@ -36,6 +51,9 @@ type parquetQueryableWithFallback struct { // Subservices manager. subservices *services.Manager subservicesWatcher *services.FailureWatcher + + // metrics + metrics *parquetQueryableFallbackMetrics } func NewParquetQueryable( @@ -51,39 +69,14 @@ func NewParquetQueryable( if err != nil { return nil, err } - - // Create the blocks finder. - var finder BlocksFinder - if storageCfg.BucketStore.BucketIndex.Enabled { - finder = NewBucketIndexBlocksFinder(BucketIndexBlocksFinderConfig{ - IndexLoader: bucketindex.LoaderConfig{ - CheckInterval: time.Minute, - UpdateOnStaleInterval: storageCfg.BucketStore.SyncInterval, - UpdateOnErrorInterval: storageCfg.BucketStore.BucketIndex.UpdateOnErrorInterval, - IdleTimeout: storageCfg.BucketStore.BucketIndex.IdleTimeout, - }, - MaxStalePeriod: storageCfg.BucketStore.BucketIndex.MaxStalePeriod, - IgnoreDeletionMarksDelay: storageCfg.BucketStore.IgnoreDeletionMarksDelay, - IgnoreBlocksWithin: storageCfg.BucketStore.IgnoreBlocksWithin, - }, bucketClient, limits, logger, reg) - } else { - finder = NewBucketScanBlocksFinder(BucketScanBlocksFinderConfig{ - ScanInterval: storageCfg.BucketStore.SyncInterval, - TenantsConcurrency: storageCfg.BucketStore.TenantSyncConcurrency, - MetasConcurrency: storageCfg.BucketStore.MetaSyncConcurrency, - CacheDir: storageCfg.BucketStore.SyncDir, - IgnoreDeletionMarksDelay: storageCfg.BucketStore.IgnoreDeletionMarksDelay, - IgnoreBlocksWithin: storageCfg.BucketStore.IgnoreBlocksWithin, - BlockDiscoveryStrategy: storageCfg.BucketStore.BlockDiscoveryStrategy, - }, bucketClient, limits, logger, reg) - } - - manager, err := services.NewManager(finder, blockStorageQueryable) + manager, err := services.NewManager(blockStorageQueryable) if err != nil { return nil, err } - pq, err := search.NewParquetQueryable(nil, func(ctx context.Context, mint, maxt int64) ([]*parquet_storage.ParquetShard, error) { + cDecoder := schema.NewPrometheusParquetChunksDecoder(chunkenc.NewPool()) + + parquetQueryable, err := search.NewParquetQueryable(cDecoder, func(ctx context.Context, mint, maxt int64) ([]*parquet_storage.ParquetShard, error) { userID, err := tenant.TenantID(ctx) if err != nil { return nil, err @@ -98,8 +91,8 @@ func NewParquetQueryable( shards := make([]*parquet_storage.ParquetShard, 0, len(blocks)) for _, block := range blocks { - blockName := fmt.Sprintf("%v/block", block.ID.String()) - shard, err := parquet_storage.OpenParquetShard(ctx, userBkt, blockName, 0) + // we always only have 1 shard - shard 0 + shard, err := parquet_storage.OpenParquetShard(ctx, userBkt, block.ID.String(), 0) if err != nil { return nil, err } @@ -112,15 +105,16 @@ func NewParquetQueryable( p := &parquetQueryableWithFallback{ subservices: manager, blockStorageQueryable: blockStorageQueryable, - parquetQueryable: pq, + parquetQueryable: parquetQueryable, queryStoreAfter: config.QueryStoreAfter, subservicesWatcher: services.NewFailureWatcher(), - finder: finder, + finder: blockStorageQueryable.finder, + metrics: newParquetQueryableFallbackMetrics(reg), } p.Service = services.NewBasicService(p.starting, p.running, p.stopping) - return pq, err + return p, err } func (p *parquetQueryableWithFallback) starting(ctx context.Context) error { @@ -164,6 +158,7 @@ func (p *parquetQueryableWithFallback) Querier(mint, maxt int64) (storage.Querie queryStoreAfter: p.queryStoreAfter, blocksStoreQuerier: bsq, finder: p.finder, + metrics: p.metrics, }, nil } @@ -178,6 +173,9 @@ type parquetQuerier struct { // If set, the querier manipulates the max time to not be greater than // "now - queryStoreAfter" so that most recent blocks are not queried. queryStoreAfter time.Duration + + // metrics + metrics *parquetQueryableFallbackMetrics } func (q *parquetQuerier) LabelValues(ctx context.Context, name string, hints *storage.LabelHints, matchers ...*labels.Matcher) ([]string, annotations.Annotations, error) { @@ -185,7 +183,6 @@ func (q *parquetQuerier) LabelValues(ctx context.Context, name string, hints *st if err != nil { return nil, nil, err } - limit := 0 if hints != nil { @@ -342,5 +339,8 @@ func (q *parquetQuerier) getBlocks(ctx context.Context, minT, maxT int64) ([]*bu remaining = append(remaining, b) } + q.metrics.blocksQueriedTotal.WithLabelValues("parquet").Add(float64(len(parquetBlocks))) + q.metrics.blocksQueriedTotal.WithLabelValues("tsdb").Add(float64(len(remaining))) + return remaining, parquetBlocks, nil } diff --git a/pkg/querier/parquet_queryable_test.go b/pkg/querier/parquet_queryable_test.go index d948ee58761..5a5435a7356 100644 --- a/pkg/querier/parquet_queryable_test.go +++ b/pkg/querier/parquet_queryable_test.go @@ -75,6 +75,7 @@ func TestParquetQueryableFallbackLogic(t *testing.T) { finder: finder, blocksStoreQuerier: q, parquetQuerier: mParquetQuerier, + metrics: newParquetQueryableFallbackMetrics(prometheus.NewRegistry()), } finder.On("GetBlocks", mock.Anything, "user-1", minT, maxT).Return(bucketindex.Blocks{ @@ -131,6 +132,7 @@ func TestParquetQueryableFallbackLogic(t *testing.T) { finder: finder, blocksStoreQuerier: q, parquetQuerier: mParquetQuerier, + metrics: newParquetQueryableFallbackMetrics(prometheus.NewRegistry()), } finder.On("GetBlocks", mock.Anything, "user-1", minT, maxT).Return(bucketindex.Blocks{ @@ -193,6 +195,7 @@ func TestParquetQueryableFallbackLogic(t *testing.T) { finder: finder, blocksStoreQuerier: q, parquetQuerier: mParquetQuerier, + metrics: newParquetQueryableFallbackMetrics(prometheus.NewRegistry()), } finder.On("GetBlocks", mock.Anything, "user-1", minT, maxT).Return(bucketindex.Blocks{ From 5152e90c6f51ec4a98a9375aeb0f9f0f74582251 Mon Sep 17 00:00:00 2001 From: alanprot Date: Thu, 22 May 2025 09:44:37 -0700 Subject: [PATCH 08/12] addressing comments Signed-off-by: alanprot --- pkg/cortex/modules.go | 4 ++-- pkg/querier/parquet_queryable.go | 14 +++++++------- pkg/querier/parquet_queryable_test.go | 6 +++--- 3 files changed, 12 insertions(+), 12 deletions(-) diff --git a/pkg/cortex/modules.go b/pkg/cortex/modules.go index eca2cc7d302..be3aca7fc00 100644 --- a/pkg/cortex/modules.go +++ b/pkg/cortex/modules.go @@ -401,7 +401,7 @@ func (t *Cortex) initStoreQueryables() (services.Service, error) { //nolint:revive // I prefer this form over removing 'else', because it allows q to have smaller scope. var queriable prom_storage.Queryable - if q, err := initQueryableForEngine(t.Cfg, t.Overrides, prometheus.DefaultRegisterer); err != nil { + if q, err := initBlockStoreQueryable(t.Cfg, t.Overrides, prometheus.DefaultRegisterer); err != nil { return nil, fmt.Errorf("failed to initialize querier: %v", err) } else { queriable = q @@ -433,7 +433,7 @@ func (t *Cortex) initStoreQueryables() (services.Service, error) { } } -func initQueryableForEngine(cfg Config, limits *validation.Overrides, reg prometheus.Registerer) (*querier.BlocksStoreQueryable, error) { +func initBlockStoreQueryable(cfg Config, limits *validation.Overrides, reg prometheus.Registerer) (*querier.BlocksStoreQueryable, error) { // When running in single binary, if the blocks sharding is disabled and no custom // store-gateway address has been configured, we can set it to the running process. if cfg.isModuleEnabled(All) && !cfg.StoreGateway.ShardingEnabled && cfg.Querier.StoreGatewayAddresses == "" { diff --git a/pkg/querier/parquet_queryable.go b/pkg/querier/parquet_queryable.go index 41d03d1be4d..ccd5e8a0c73 100644 --- a/pkg/querier/parquet_queryable.go +++ b/pkg/querier/parquet_queryable.go @@ -151,7 +151,7 @@ func (p *parquetQueryableWithFallback) Querier(mint, maxt int64) (storage.Querie return nil, err } - return &parquetQuerier{ + return &parquetQuerierWithFallback{ minT: mint, maxT: maxt, parquetQuerier: pq, @@ -162,7 +162,7 @@ func (p *parquetQueryableWithFallback) Querier(mint, maxt int64) (storage.Querie }, nil } -type parquetQuerier struct { +type parquetQuerierWithFallback struct { minT, maxT int64 finder BlocksFinder @@ -178,7 +178,7 @@ type parquetQuerier struct { metrics *parquetQueryableFallbackMetrics } -func (q *parquetQuerier) LabelValues(ctx context.Context, name string, hints *storage.LabelHints, matchers ...*labels.Matcher) ([]string, annotations.Annotations, error) { +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) if err != nil { return nil, nil, err @@ -223,7 +223,7 @@ func (q *parquetQuerier) LabelValues(ctx context.Context, name string, hints *st return result, rAnnotations, nil } -func (q *parquetQuerier) LabelNames(ctx context.Context, hints *storage.LabelHints, matchers ...*labels.Matcher) ([]string, annotations.Annotations, error) { +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) if err != nil { return nil, nil, err @@ -269,7 +269,7 @@ func (q *parquetQuerier) LabelNames(ctx context.Context, hints *storage.LabelHin return result, rAnnotations, nil } -func (q *parquetQuerier) Select(ctx context.Context, sortSeries bool, hints *storage.SelectHints, matchers ...*labels.Matcher) storage.SeriesSet { +func (q *parquetQuerierWithFallback) Select(ctx context.Context, sortSeries bool, hints *storage.SelectHints, matchers ...*labels.Matcher) storage.SeriesSet { mint, maxt, limit := q.minT, q.maxT, 0 if hints != nil { @@ -298,14 +298,14 @@ func (q *parquetQuerier) Select(ctx context.Context, sortSeries bool, hints *sto return storage.NewMergeSeriesSet(serieSets, limit, storage.ChainedSeriesMerge) } -func (q *parquetQuerier) Close() error { +func (q *parquetQuerierWithFallback) Close() error { mErr := multierror.MultiError{} mErr.Add(q.parquetQuerier.Close()) mErr.Add(q.blocksStoreQuerier.Close()) return mErr.Err() } -func (q *parquetQuerier) getBlocks(ctx context.Context, minT, maxT int64) ([]*bucketindex.Block, []*bucketindex.Block, error) { +func (q *parquetQuerierWithFallback) getBlocks(ctx context.Context, minT, maxT int64) ([]*bucketindex.Block, []*bucketindex.Block, error) { // 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 diff --git a/pkg/querier/parquet_queryable_test.go b/pkg/querier/parquet_queryable_test.go index 5a5435a7356..17768e4eaa7 100644 --- a/pkg/querier/parquet_queryable_test.go +++ b/pkg/querier/parquet_queryable_test.go @@ -69,7 +69,7 @@ func TestParquetQueryableFallbackLogic(t *testing.T) { } mParquetQuerier := &mockParquetQuerier{} - pq := &parquetQuerier{ + pq := &parquetQuerierWithFallback{ minT: minT, maxT: maxT, finder: finder, @@ -126,7 +126,7 @@ func TestParquetQueryableFallbackLogic(t *testing.T) { } mParquetQuerier := &mockParquetQuerier{} - pq := &parquetQuerier{ + pq := &parquetQuerierWithFallback{ minT: minT, maxT: maxT, finder: finder, @@ -189,7 +189,7 @@ func TestParquetQueryableFallbackLogic(t *testing.T) { } mParquetQuerier := &mockParquetQuerier{} - pq := &parquetQuerier{ + pq := &parquetQuerierWithFallback{ minT: minT, maxT: maxT, finder: finder, From c7685c7aa48bba4d040d6c46a04c44a6e4b146e4 Mon Sep 17 00:00:00 2001 From: alanprot Date: Thu, 22 May 2025 11:11:44 -0700 Subject: [PATCH 09/12] adding cortex_parquet_queryable_selects_queried_total metric Signed-off-by: alanprot --- pkg/querier/parquet_queryable.go | 14 ++++++++++++++ 1 file changed, 14 insertions(+) diff --git a/pkg/querier/parquet_queryable.go b/pkg/querier/parquet_queryable.go index ccd5e8a0c73..616aa4e1b18 100644 --- a/pkg/querier/parquet_queryable.go +++ b/pkg/querier/parquet_queryable.go @@ -28,6 +28,7 @@ import ( type parquetQueryableFallbackMetrics struct { blocksQueriedTotal *prometheus.CounterVec + selectCount *prometheus.CounterVec } func newParquetQueryableFallbackMetrics(reg prometheus.Registerer) *parquetQueryableFallbackMetrics { @@ -36,6 +37,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"}), } } @@ -342,5 +347,14 @@ 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))) + switch { + case len(remaining) > 0 && len(parquetBlocks) > 0: + q.metrics.selectCount.WithLabelValues("tsdb+parquet").Inc() + case len(remaining) > 0 && len(parquetBlocks) == 0: + q.metrics.selectCount.WithLabelValues("tsdb").Inc() + case len(remaining) == 0 && len(parquetBlocks) > 0: + q.metrics.selectCount.WithLabelValues("parquet").Inc() + } + return remaining, parquetBlocks, nil } From 65421e83496a3f96b6cb3207bd00536f5cc3d2d8 Mon Sep 17 00:00:00 2001 From: alanprot Date: Thu, 22 May 2025 11:29:10 -0700 Subject: [PATCH 10/12] update parquet common Signed-off-by: alanprot --- go.mod | 2 +- go.sum | 4 +- .../parquet-common/search/materialize.go | 72 ++++++++++++------- .../search/parquet_queriable.go | 45 ++++++++++-- .../parquet-common/search/rowrange.go | 6 ++ .../parquet-common/util/util.go | 12 ++-- vendor/modules.txt | 2 +- 7 files changed, 104 insertions(+), 39 deletions(-) diff --git a/go.mod b/go.mod index 0c282404820..60528b76fda 100644 --- a/go.mod +++ b/go.mod @@ -82,7 +82,7 @@ require ( 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-20250514003255-382b6ec8ae40 + github.com/prometheus-community/parquet-common v0.0.0-20250522182606-e046c038dc73 github.com/prometheus/procfs v0.15.1 github.com/sercand/kuberesolver/v5 v5.1.1 github.com/tjhop/slog-gokit v0.1.3 diff --git a/go.sum b/go.sum index 67e199c927b..1ee8edc2ddd 100644 --- a/go.sum +++ b/go.sum @@ -1573,8 +1573,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-20250514003255-382b6ec8ae40 h1:45NOJV7a7QGKg7rITB8wCs/f5O4bpe3mPAiTFsqcX8s= -github.com/prometheus-community/parquet-common v0.0.0-20250514003255-382b6ec8ae40/go.mod h1:zRW/xXBlELf8v9h9uqWvDkjOr3N5BtQGZ6LsDX9Ea/A= +github.com/prometheus-community/parquet-common v0.0.0-20250522182606-e046c038dc73 h1:AogORrmarkYfUOI7/lqOhz9atYmLZo69vPQ/SFkPSxE= +github.com/prometheus-community/parquet-common v0.0.0-20250522182606-e046c038dc73/go.mod h1:zRW/xXBlELf8v9h9uqWvDkjOr3N5BtQGZ6LsDX9Ea/A= github.com/prometheus-community/prom-label-proxy v0.8.1-0.20240127162815-c1195f9aabc0 h1:owfYHh79h8Y5HvNMGyww+DaVwo10CKiRW1RQrrZzIwg= github.com/prometheus-community/prom-label-proxy v0.8.1-0.20240127162815-c1195f9aabc0/go.mod h1:rT989D4UtOcfd9tVqIZRVIM8rkg+9XbreBjFNEKXvVI= github.com/prometheus/alertmanager v0.28.1 h1:BK5pCoAtaKg01BYRUJhEDV1tqJMEtYBGzPw8QdvnnvA= diff --git a/vendor/github.com/prometheus-community/parquet-common/search/materialize.go b/vendor/github.com/prometheus-community/parquet-common/search/materialize.go index ffa3f862984..d64f54b4d70 100644 --- a/vendor/github.com/prometheus-community/parquet-common/search/materialize.go +++ b/vendor/github.com/prometheus-community/parquet-common/search/materialize.go @@ -17,9 +17,10 @@ import ( "context" "fmt" "io" - "runtime" + "maps" "slices" "sort" + "sync" "github.com/efficientgo/core/errors" "github.com/parquet-go/parquet-go" @@ -34,9 +35,10 @@ import ( ) type Materializer struct { - b *storage.ParquetShard - s *schema.TSDBSchema - d *schema.PrometheusParquetChunksDecoder + b *storage.ParquetShard + s *schema.TSDBSchema + d *schema.PrometheusParquetChunksDecoder + partitioner util.Partitioner colIdx int concurrency int @@ -44,7 +46,12 @@ type Materializer struct { dataColToIndex []int } -func NewMaterializer(s *schema.TSDBSchema, d *schema.PrometheusParquetChunksDecoder, block *storage.ParquetShard) (*Materializer, error) { +func NewMaterializer(s *schema.TSDBSchema, + d *schema.PrometheusParquetChunksDecoder, + block *storage.ParquetShard, + concurrency int, + maxGapPartitioning int, +) (*Materializer, error) { colIdx, ok := block.LabelsFile().Schema().Lookup(schema.ColIndexes) if !ok { return nil, errors.New(fmt.Sprintf("schema index %s not found", schema.ColIndexes)) @@ -65,7 +72,8 @@ func NewMaterializer(s *schema.TSDBSchema, d *schema.PrometheusParquetChunksDeco d: d, b: block, colIdx: colIdx.ColumnIndex, - concurrency: runtime.GOMAXPROCS(0), + concurrency: concurrency, + partitioner: util.NewGapBasedPartitioner(maxGapPartitioning), dataColToIndex: dataColToIndex, }, nil } @@ -257,15 +265,19 @@ func (m *Materializer) materializeAllLabels(ctx context.Context, rgi int, rr []R return results, nil } +func totalRows(rr []RowRange) int64 { + res := int64(0) + for _, r := range rr { + res += r.count + } + return res +} + func (m *Materializer) materializeChunks(ctx context.Context, rgi int, mint, maxt int64, rr []RowRange) ([][]chunks.Meta, error) { minDataCol := m.s.DataColumIdx(mint) maxDataCol := m.s.DataColumIdx(maxt) rg := m.b.ChunksFile().RowGroups()[rgi] - totalRows := int64(0) - for _, r := range rr { - totalRows += r.count - } - r := make([][]chunks.Meta, totalRows) + r := make([][]chunks.Meta, totalRows(rr)) for i := minDataCol; i <= min(maxDataCol, len(m.dataColToIndex)-1); i++ { values, err := m.materializeColumn(ctx, m.b.ChunksFile(), rg, rg.ColumnChunks()[m.dataColToIndex[i]], rr) @@ -314,20 +326,25 @@ func (m *Materializer) materializeColumn(ctx context.Context, file *storage.Parq for _, r := range rr { if pageRowRange.Overlaps(r) { - pagesToRowsMap[i] = append(pagesToRowsMap[i], r) + pagesToRowsMap[i] = append(pagesToRowsMap[i], pageRowRange.Intersection(r)) } } } - r := make(map[RowRange][]parquet.Value, len(rr)) - for _, v := range rr { - r[v] = []parquet.Value{} + pageRanges := m.coalescePageRanges(pagesToRowsMap, oidx) + + r := make(map[RowRange][]parquet.Value, len(pageRanges)) + rMutex := &sync.Mutex{} + for _, v := range pageRanges { + for _, rs := range v.rows { + r[rs] = make([]parquet.Value, 0, rs.count) + } } errGroup := &errgroup.Group{} errGroup.SetLimit(m.concurrency) - for _, p := range coalescePageRanges(pagesToRowsMap, oidx) { + for _, p := range pageRanges { errGroup.Go(func() error { pgs := file.GetPages(ctx, cc) defer func() { _ = pgs.Close() }() @@ -352,7 +369,9 @@ func (m *Materializer) materializeColumn(ctx context.Context, file *storage.Parq vi.Reset(page) for vi.Next() { if currentRow == next { + rMutex.Lock() r[currentRr] = append(r[currentRr], vi.At()) + rMutex.Unlock() remaining-- if remaining > 0 { next = next + 1 @@ -379,11 +398,16 @@ func (m *Materializer) materializeColumn(ctx context.Context, file *storage.Parq return nil, errors.Wrap(err, "failed to materialize columns") } - values := make([]parquet.Value, 0, len(rr)) - for _, v := range rr { - values = append(values, r[v]...) + ranges := slices.Collect(maps.Keys(r)) + slices.SortFunc(ranges, func(a, b RowRange) int { + return int(a.from - b.from) + }) + + res := make([]parquet.Value, 0, totalRows(rr)) + for _, v := range ranges { + res = append(res, r[v]...) } - return values, err + return res, nil } type pageEntryRead struct { @@ -393,9 +417,7 @@ type pageEntryRead struct { // Merge nearby pages to enable efficient sequential reads. // Pages that are not close to each other will be scheduled for concurrent reads. -func coalescePageRanges(pagedIdx map[int][]RowRange, offset parquet.OffsetIndex) []pageEntryRead { - // TODO: Add the max gap size as parameter - partitioner := util.NewGapBasedPartitioner(10 * 1024) +func (m *Materializer) coalescePageRanges(pagedIdx map[int][]RowRange, offset parquet.OffsetIndex) []pageEntryRead { if len(pagedIdx) == 0 { return []pageEntryRead{} } @@ -406,8 +428,8 @@ func coalescePageRanges(pagedIdx map[int][]RowRange, offset parquet.OffsetIndex) slices.Sort(idxs) - parts := partitioner.Partition(len(idxs), func(i int) (uint64, uint64) { - return uint64(offset.Offset(idxs[i])), uint64(offset.Offset(idxs[i]) + offset.CompressedPageSize(idxs[i])) + parts := m.partitioner.Partition(len(idxs), func(i int) (int, int) { + return int(offset.Offset(idxs[i])), int(offset.Offset(idxs[i]) + offset.CompressedPageSize(idxs[i])) }) r := make([]pageEntryRead, 0, len(parts)) diff --git a/vendor/github.com/prometheus-community/parquet-common/search/parquet_queriable.go b/vendor/github.com/prometheus-community/parquet-common/search/parquet_queriable.go index cf609b31169..96cb17ca47c 100644 --- a/vendor/github.com/prometheus-community/parquet-common/search/parquet_queriable.go +++ b/vendor/github.com/prometheus-community/parquet-common/search/parquet_queriable.go @@ -15,6 +15,7 @@ package search import ( "context" + "runtime" "sort" "github.com/prometheus/prometheus/model/labels" @@ -29,15 +30,49 @@ import ( type ShardsFinderFunction func(ctx context.Context, mint, maxt int64) ([]*storage.ParquetShard, error) +type queryableOpts struct { + concurrency int + pagePartitioningMaxGapSize int +} + +var DefaultQueryableOpts = queryableOpts{ + concurrency: runtime.GOMAXPROCS(0), + pagePartitioningMaxGapSize: 10 * 1024, +} + +type QueryableOpts func(*queryableOpts) + +// WithConcurrency set the concurrency that can be used to run the query +func WithConcurrency(concurrency int) QueryableOpts { + return func(opts *queryableOpts) { + opts.concurrency = concurrency + } +} + +// WithPageMaxGapSize set the max gap size between pages that should be downloaded together in a single read call +func WithPageMaxGapSize(pagePartitioningMaxGapSize int) QueryableOpts { + return func(opts *queryableOpts) { + opts.pagePartitioningMaxGapSize = pagePartitioningMaxGapSize + } +} + type parquetQueryable struct { shardsFinder ShardsFinderFunction d *schema.PrometheusParquetChunksDecoder + opts *queryableOpts } -func NewParquetQueryable(d *schema.PrometheusParquetChunksDecoder, shardFinder ShardsFinderFunction) (prom_storage.Queryable, error) { +func NewParquetQueryable(d *schema.PrometheusParquetChunksDecoder, shardFinder ShardsFinderFunction, opts ...QueryableOpts) (prom_storage.Queryable, error) { + cfg := DefaultQueryableOpts + + for _, opt := range opts { + opt(&cfg) + } + return &parquetQueryable{ shardsFinder: shardFinder, d: d, + opts: &cfg, }, nil } @@ -47,6 +82,7 @@ func (p parquetQueryable) Querier(mint, maxt int64) (prom_storage.Querier, error maxt: maxt, shardsFinder: p.shardsFinder, d: p.d, + opts: p.opts, }, nil } @@ -54,6 +90,7 @@ type parquetQuerier struct { mint, maxt int64 shardsFinder ShardsFinderFunction d *schema.PrometheusParquetChunksDecoder + opts *queryableOpts } func (p parquetQuerier) LabelValues(ctx context.Context, name string, hints *prom_storage.LabelHints, matchers ...*labels.Matcher) ([]string, annotations.Annotations, error) { @@ -144,7 +181,7 @@ func (p parquetQuerier) queryableShards(ctx context.Context, mint, maxt int64) ( } qBlocks := make([]*queryableShard, len(shards)) for i, shard := range shards { - qb, err := newQueryableShard(shard, p.d) + qb, err := newQueryableShard(p.opts, shard, p.d) if err != nil { return nil, err } @@ -158,12 +195,12 @@ type queryableShard struct { m *Materializer } -func newQueryableShard(block *storage.ParquetShard, d *schema.PrometheusParquetChunksDecoder) (*queryableShard, error) { +func newQueryableShard(opts *queryableOpts, block *storage.ParquetShard, d *schema.PrometheusParquetChunksDecoder) (*queryableShard, error) { s, err := block.TSDBSchema() if err != nil { return nil, err } - m, err := NewMaterializer(s, d, block) + m, err := NewMaterializer(s, d, block, opts.concurrency, opts.pagePartitioningMaxGapSize) if err != nil { return nil, err } diff --git a/vendor/github.com/prometheus-community/parquet-common/search/rowrange.go b/vendor/github.com/prometheus-community/parquet-common/search/rowrange.go index 10cf136136a..412bcdfeca0 100644 --- a/vendor/github.com/prometheus-community/parquet-common/search/rowrange.go +++ b/vendor/github.com/prometheus-community/parquet-common/search/rowrange.go @@ -37,6 +37,12 @@ func (rr RowRange) Overlaps(o RowRange) bool { return rr.from < endB && o.from < endA } +// Intersection returns the intersection of rr and o. Both are assumed to be overlapping +func (rr RowRange) Intersection(o RowRange) RowRange { + os, oe := max(rr.from, o.from), min(rr.from+rr.count, o.from+o.count) + return RowRange{from: os, count: oe - os} +} + // intersect intersects the row ranges from left hand sight with the row ranges from rhs // it assumes that lhs and rhs are simplified and returns a simplified result. // it operates in o(l+r) time by cursoring through ranges with a two pointer approach. diff --git a/vendor/github.com/prometheus-community/parquet-common/util/util.go b/vendor/github.com/prometheus-community/parquet-common/util/util.go index 4851a51a7a9..06cb64277e4 100644 --- a/vendor/github.com/prometheus-community/parquet-common/util/util.go +++ b/vendor/github.com/prometheus-community/parquet-common/util/util.go @@ -35,8 +35,8 @@ func CloneRows(rows []parquet.Row) []parquet.Row { // https://github.com/thanos-io/thanos/blob/2a5a856e34adb2653dda700c4d87637236afb2dd/pkg/store/bucket.go#L3466 type Part struct { - Start uint64 - End uint64 + Start int + End int ElemRng [2]int } @@ -46,14 +46,14 @@ type Partitioner interface { // input ranges // It supports overlapping ranges. // NOTE: It expects range to be sorted by start time. - Partition(length int, rng func(int) (uint64, uint64)) []Part + Partition(length int, rng func(int) (int, int)) []Part } type gapBasedPartitioner struct { - maxGapSize uint64 + maxGapSize int } -func NewGapBasedPartitioner(maxGapSize uint64) Partitioner { +func NewGapBasedPartitioner(maxGapSize int) Partitioner { return gapBasedPartitioner{ maxGapSize: maxGapSize, } @@ -62,7 +62,7 @@ func NewGapBasedPartitioner(maxGapSize uint64) Partitioner { // Partition partitions length entries into n <= length ranges that cover all // input ranges by combining entries that are separated by reasonably small gaps. // It is used to combine multiple small ranges from object storage into bigger, more efficient/cheaper ones. -func (g gapBasedPartitioner) Partition(length int, rng func(int) (uint64, uint64)) (parts []Part) { +func (g gapBasedPartitioner) Partition(length int, rng func(int) (int, int)) (parts []Part) { j := 0 k := 0 for k < length { diff --git a/vendor/modules.txt b/vendor/modules.txt index 6dde2be2516..622998041b4 100644 --- a/vendor/modules.txt +++ b/vendor/modules.txt @@ -837,7 +837,7 @@ github.com/pkg/errors # github.com/pmezard/go-difflib v1.0.1-0.20181226105442-5d4384ee4fb2 ## explicit github.com/pmezard/go-difflib/difflib -# github.com/prometheus-community/parquet-common v0.0.0-20250514003255-382b6ec8ae40 +# github.com/prometheus-community/parquet-common v0.0.0-20250522182606-e046c038dc73 ## explicit; go 1.23.4 github.com/prometheus-community/parquet-common/convert github.com/prometheus-community/parquet-common/schema From 7a2383be7812a1eaea381487f2982354ea2b017f Mon Sep 17 00:00:00 2001 From: alanprot Date: Thu, 22 May 2025 16:16:53 -0700 Subject: [PATCH 11/12] Creating test parquet fuzz integration test Signed-off-by: alanprot --- integration/parquet_querier_test.go | 174 ++++++++++++++++++++++++++++ pkg/querier/parquet_queryable.go | 2 +- 2 files changed, 175 insertions(+), 1 deletion(-) create mode 100644 integration/parquet_querier_test.go diff --git a/integration/parquet_querier_test.go b/integration/parquet_querier_test.go new file mode 100644 index 00000000000..5b9d89828c3 --- /dev/null +++ b/integration/parquet_querier_test.go @@ -0,0 +1,174 @@ +//go:build integration +// +build integration + +package integration + +import ( + "context" + "fmt" + "github.com/cortexproject/cortex/integration/e2e" + e2edb "github.com/cortexproject/cortex/integration/e2e/db" + "github.com/cortexproject/cortex/integration/e2ecortex" + "github.com/cortexproject/cortex/pkg/storage/bucket" + "github.com/cortexproject/cortex/pkg/storage/tsdb" + "github.com/cortexproject/cortex/pkg/util/log" + cortex_testutil "github.com/cortexproject/cortex/pkg/util/test" + "github.com/cortexproject/promqlsmith" + "github.com/prometheus/prometheus/model/labels" + "github.com/stretchr/testify/require" + "github.com/thanos-io/objstore" + "github.com/thanos-io/thanos/pkg/block" + "github.com/thanos-io/thanos/pkg/block/metadata" + "math/rand" + "path/filepath" + "strconv" + "testing" + "time" +) + +func TestParquetFuzz(t *testing.T) { + + s, err := e2e.NewScenario(networkName) + require.NoError(t, err) + defer s.Close() + + consul := e2edb.NewConsulWithName("consul") + require.NoError(t, s.StartAndWaitReady(consul)) + + baseFlags := mergeFlags(AlertmanagerLocalFlags(), BlocksStorageFlags()) + flags := mergeFlags( + baseFlags, + map[string]string{ + "-target": "all,parquet-converter", + "-blocks-storage.tsdb.ship-interval": "1s", + "-blocks-storage.bucket-store.sync-interval": "1s", + "-blocks-storage.bucket-store.metadata-cache.bucket-index-content-ttl": "1s", + "-blocks-storage.bucket-store.bucket-index.idle-timeout": "1s", + "-blocks-storage.bucket-store.bucket-index.enabled": "true", + "-blocks-storage.bucket-store.index-cache.backend": tsdb.IndexCacheBackendInMemory, + "-querier.query-store-for-labels-enabled": "true", + // compactor + "-compactor.cleanup-interval": "1s", + // Ingester. + "-ring.store": "consul", + "-consul.hostname": consul.NetworkHTTPEndpoint(), + // Distributor. + "-distributor.replication-factor": "1", + // Store-gateway. + "-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", + "-frontend.max-cache-freshness": "1m", + // enable experimental promQL funcs + "-querier.enable-promql-experimental-functions": "true", + // parquet-converter + "-parquet-converter.ring.consul.hostname": consul.NetworkHTTPEndpoint(), + "-parquet-converter.conversion-interval": "1s", + "-parquet-converter.enabled": "true", + // Querier + "-querier.query-parquet-files": "true", + }, + ) + + // make alert manager config dir + require.NoError(t, writeFileToSharedDir(s, "alertmanager_configs", []byte{})) + + ctx := context.Background() + rnd := rand.New(rand.NewSource(time.Now().Unix())) + dir := filepath.Join(s.SharedDir(), "data") + numSeries := 10 + numSamples := 60 + lbls := make([]labels.Labels, 0, numSeries*2) + scrapeInterval := time.Minute + statusCodes := []string{"200", "400", "404", "500", "502"} + now := time.Now() + start := now.Add(-time.Hour * 24) + end := now.Add(-time.Hour) + + for i := 0; i < numSeries; i++ { + lbls = append(lbls, labels.Labels{ + {Name: labels.MetricName, Value: "test_series_a"}, + {Name: "job", Value: "test"}, + {Name: "series", Value: strconv.Itoa(i % 3)}, + {Name: "status_code", Value: statusCodes[i%5]}, + }) + + lbls = append(lbls, labels.Labels{ + {Name: labels.MetricName, Value: "test_series_b"}, + {Name: "job", Value: "test"}, + {Name: "series", Value: strconv.Itoa((i + 1) % 3)}, + {Name: "status_code", Value: statusCodes[(i+1)%5]}, + }) + } + id, err := e2e.CreateBlock(ctx, rnd, dir, lbls, numSamples, start.UnixMilli(), end.UnixMilli(), scrapeInterval.Milliseconds(), 10) + require.NoError(t, err) + minio := e2edb.NewMinio(9000, flags["-blocks-storage.s3.bucket-name"]) + require.NoError(t, s.StartAndWaitReady(minio)) + + cortex := e2ecortex.NewSingleBinary("cortex", flags, "") + require.NoError(t, s.StartAndWaitReady(cortex)) + + storage, err := e2ecortex.NewS3ClientForMinio(minio, flags["-blocks-storage.s3.bucket-name"]) + bkt := bucket.NewUserBucketClient("user-1", storage.GetBucket(), nil) + + err = block.Upload(ctx, log.Logger, bkt, filepath.Join(dir, id.String()), metadata.NoneFunc) + require.NoError(t, err) + + // Wait until we convert the blocks + cortex_testutil.Poll(t, 30*time.Second, true, func() interface{} { + found := false + + bkt.Iter(context.Background(), "", func(name string) error { + fmt.Println(name) + if name == fmt.Sprintf("parquet-markers/%v-parquet-converter-mark.json", id.String()) { + found = true + } + return nil + }, objstore.WithRecursiveIter()) + return found + }) + + att, err := bkt.Attributes(context.Background(), "bucket-index.json.gz") + require.NoError(t, err) + numberOfIndexesUpdate := 0 + lastUpdate := att.LastModified + + cortex_testutil.Poll(t, 30*time.Second, 5, func() interface{} { + att, err := bkt.Attributes(context.Background(), "bucket-index.json.gz") + require.NoError(t, err) + if lastUpdate != att.LastModified { + lastUpdate = att.LastModified + numberOfIndexesUpdate++ + } + return numberOfIndexesUpdate + }) + + c1, err := e2ecortex.NewClient("", cortex.HTTPEndpoint(), "", "", "user-1") + require.NoError(t, err) + + err = writeFileToSharedDir(s, "prometheus.yml", []byte("")) + require.NoError(t, err) + prom := e2edb.NewPrometheus("", map[string]string{ + "--enable-feature": "promql-experimental-functions", + }) + require.NoError(t, s.StartAndWaitReady(prom)) + + c2, err := e2ecortex.NewPromQueryClient(prom.HTTPEndpoint()) + require.NoError(t, err) + waitUntilReady(t, ctx, c1, c2, `{job="test"}`, start, end) + + opts := []promqlsmith.Option{ + promqlsmith.WithEnableOffset(true), + promqlsmith.WithEnableAtModifier(true), + promqlsmith.WithEnabledFunctions(enabledFunctions), + } + ps := promqlsmith.New(rnd, lbls, opts...) + + runQueryFuzzTestCases(t, ps, c1, c2, end, start, end, scrapeInterval, 500, false) + fmt.Println(cortex.Metrics()) + + require.NoError(t, cortex.WaitSumMetricsWithOptions(e2e.Greater(0), []string{"cortex_parquet_queryable_blocks_queried_total"}, e2e.WithLabelMatchers( + labels.MustNewMatcher(labels.MatchEqual, "type", "parquet")))) +} diff --git a/pkg/querier/parquet_queryable.go b/pkg/querier/parquet_queryable.go index 616aa4e1b18..fce24c62d51 100644 --- a/pkg/querier/parquet_queryable.go +++ b/pkg/querier/parquet_queryable.go @@ -349,7 +349,7 @@ func (q *parquetQuerierWithFallback) getBlocks(ctx context.Context, minT, maxT i switch { case len(remaining) > 0 && len(parquetBlocks) > 0: - q.metrics.selectCount.WithLabelValues("tsdb+parquet").Inc() + q.metrics.selectCount.WithLabelValues("mixed").Inc() case len(remaining) > 0 && len(parquetBlocks) == 0: q.metrics.selectCount.WithLabelValues("tsdb").Inc() case len(remaining) == 0 && len(parquetBlocks) > 0: From cef5e5c90459601e172a5762d7e753eb3fbbd76c Mon Sep 17 00:00:00 2001 From: alanprot Date: Fri, 23 May 2025 07:58:17 -0700 Subject: [PATCH 12/12] using right build flag for fuxx testing Signed-off-by: alanprot --- integration/parquet_querier_test.go | 33 ++++++++++++++++------------- pkg/parquetconverter/converter.go | 4 ++-- 2 files changed, 20 insertions(+), 17 deletions(-) diff --git a/integration/parquet_querier_test.go b/integration/parquet_querier_test.go index 5b9d89828c3..12d3760a21f 100644 --- a/integration/parquet_querier_test.go +++ b/integration/parquet_querier_test.go @@ -1,11 +1,24 @@ -//go:build integration -// +build integration +//go:build integration_query_fuzz +// +build integration_query_fuzz package integration import ( "context" "fmt" + "math/rand" + "path/filepath" + "strconv" + "testing" + "time" + + "github.com/cortexproject/promqlsmith" + "github.com/prometheus/prometheus/model/labels" + "github.com/stretchr/testify/require" + "github.com/thanos-io/objstore" + "github.com/thanos-io/thanos/pkg/block" + "github.com/thanos-io/thanos/pkg/block/metadata" + "github.com/cortexproject/cortex/integration/e2e" e2edb "github.com/cortexproject/cortex/integration/e2e/db" "github.com/cortexproject/cortex/integration/e2ecortex" @@ -13,17 +26,6 @@ import ( "github.com/cortexproject/cortex/pkg/storage/tsdb" "github.com/cortexproject/cortex/pkg/util/log" cortex_testutil "github.com/cortexproject/cortex/pkg/util/test" - "github.com/cortexproject/promqlsmith" - "github.com/prometheus/prometheus/model/labels" - "github.com/stretchr/testify/require" - "github.com/thanos-io/objstore" - "github.com/thanos-io/thanos/pkg/block" - "github.com/thanos-io/thanos/pkg/block/metadata" - "math/rand" - "path/filepath" - "strconv" - "testing" - "time" ) func TestParquetFuzz(t *testing.T) { @@ -111,6 +113,7 @@ func TestParquetFuzz(t *testing.T) { require.NoError(t, s.StartAndWaitReady(cortex)) storage, err := e2ecortex.NewS3ClientForMinio(minio, flags["-blocks-storage.s3.bucket-name"]) + require.NoError(t, err) bkt := bucket.NewUserBucketClient("user-1", storage.GetBucket(), nil) err = block.Upload(ctx, log.Logger, bkt, filepath.Join(dir, id.String()), metadata.NoneFunc) @@ -120,13 +123,14 @@ func TestParquetFuzz(t *testing.T) { cortex_testutil.Poll(t, 30*time.Second, true, func() interface{} { found := false - bkt.Iter(context.Background(), "", func(name string) error { + err := bkt.Iter(context.Background(), "", func(name string) error { fmt.Println(name) if name == fmt.Sprintf("parquet-markers/%v-parquet-converter-mark.json", id.String()) { found = true } return nil }, objstore.WithRecursiveIter()) + require.NoError(t, err) return found }) @@ -167,7 +171,6 @@ func TestParquetFuzz(t *testing.T) { ps := promqlsmith.New(rnd, lbls, opts...) runQueryFuzzTestCases(t, ps, c1, c2, end, start, end, scrapeInterval, 500, false) - fmt.Println(cortex.Metrics()) require.NoError(t, cortex.WaitSumMetricsWithOptions(e2e.Greater(0), []string{"cortex_parquet_queryable_blocks_queried_total"}, e2e.WithLabelMatchers( labels.MustNewMatcher(labels.MatchEqual, "type", "parquet")))) diff --git a/pkg/parquetconverter/converter.go b/pkg/parquetconverter/converter.go index 379a60edca4..62496a6811f 100644 --- a/pkg/parquetconverter/converter.go +++ b/pkg/parquetconverter/converter.go @@ -29,7 +29,7 @@ import ( cortex_parquet "github.com/cortexproject/cortex/pkg/storage/parquet" cortex_tsdb "github.com/cortexproject/cortex/pkg/storage/tsdb" "github.com/cortexproject/cortex/pkg/storage/tsdb/bucketindex" - "github.com/cortexproject/cortex/pkg/util" + "github.com/cortexproject/cortex/pkg/tenant" util_log "github.com/cortexproject/cortex/pkg/util/log" "github.com/cortexproject/cortex/pkg/util/services" "github.com/cortexproject/cortex/pkg/util/validation" @@ -373,7 +373,7 @@ func (c *Converter) convertUser(ctx context.Context, logger log.Logger, ring rin } func (c *Converter) ownUser(r ring.ReadRing, userId string) (bool, error) { - if userId == util.GlobalMarkersDir { + if userId == tenant.GlobalMarkersDir { // __markers__ is reserved for global markers and no tenant should be allowed to have that name. return false, nil }