From 3d23a7e9f4a53dcc06756e0422b4bbd24e2c3eee Mon Sep 17 00:00:00 2001 From: Ashwanth Goli Date: Mon, 1 Dec 2025 15:11:22 +0530 Subject: [PATCH 01/13] move read_stats to use xcap --- pkg/dataobj/internal/dataset/read_stats.go | 316 ++---------------- pkg/dataobj/internal/dataset/reader.go | 40 ++- .../internal/dataset/reader_downloader.go | 30 +- .../sections/internal/columnar/decoder.go | 13 +- pkg/dataobj/sections/logs/reader.go | 12 +- pkg/dataobj/sections/logs/reader_test.go | 49 +-- pkg/engine/internal/executor/dataobjscan.go | 46 +-- .../internal/executor/dataobjscan_test.go | 14 +- pkg/engine/internal/executor/executor.go | 2 +- pkg/engine/internal/executor/stats.go | 23 -- 10 files changed, 113 insertions(+), 432 deletions(-) diff --git a/pkg/dataobj/internal/dataset/read_stats.go b/pkg/dataobj/internal/dataset/read_stats.go index 3d0d67072dbcb..9d6c5cef17c84 100644 --- a/pkg/dataobj/internal/dataset/read_stats.go +++ b/pkg/dataobj/internal/dataset/read_stats.go @@ -1,297 +1,41 @@ package dataset import ( - "context" - "time" - - "github.com/dustin/go-humanize" - "github.com/go-kit/log" - "github.com/go-kit/log/level" - - "github.com/grafana/loki/v3/pkg/logqlmodel/stats" + "github.com/grafana/loki/v3/pkg/xcap" ) -// DownloadStats holds statistics about the download operations performed by the dataset reader. -type DownloadStats struct { - // Use the following three stats together. - // 1. PagesScanned is the total number of ReadPage calls made to the downloader. - // 2. PagesFoundInCache is the number of pages that were found in cache and - // did not require a download. - // 3. BatchDownloadRequests is the number of batch download requests made by - // the downloader when a page is not found in cache. - PagesScanned uint64 - PagesFoundInCache uint64 - BatchDownloadRequests uint64 - - PageDownloadTime time.Duration // Total time taken for downloading paga data or metadata. - - PrimaryColumnPages uint64 // Number of pages downloaded for primary columns - SecondaryColumnPages uint64 // Number of pages downloaded for secondary columns - PrimaryColumnBytes uint64 // Total bytes downloaded for primary columns - SecondaryColumnBytes uint64 // Total bytes downloaded for secondary columns - PrimaryColumnUncompressedBytes uint64 // Total uncompressed bytes for primary columns - SecondaryColumnUncompressedBytes uint64 // Total uncompressed bytes for secondary columns -} - -func (ds *DownloadStats) Reset() { - ds.PagesScanned = 0 - ds.PagesFoundInCache = 0 - ds.BatchDownloadRequests = 0 - - ds.PageDownloadTime = 0 - - ds.PrimaryColumnPages = 0 - ds.SecondaryColumnPages = 0 - ds.PrimaryColumnBytes = 0 - ds.SecondaryColumnBytes = 0 - ds.PrimaryColumnUncompressedBytes = 0 - ds.SecondaryColumnUncompressedBytes = 0 -} - -// ReaderStats tracks statistics about dataset read operations. -type ReaderStats struct { - // TODO(ashwanth): global stats from [stats.Context] can be updated by the - // engine at the end of the query execution once we have a stats collection - // framework integrated with the execution pipeline. - // Below reference is a temporary stop gap. - // - // Reference to [stats.Context] to update relevant global statistics. - globalStats *stats.Context - +// xcap statistics for dataset reader operations. +var ( // Column statistics - PrimaryColumns uint64 // Number of primary columns to read from the dataset - SecondaryColumns uint64 // Number of secondary columns to read from the dataset + StatPrimaryColumns = xcap.NewStatisticInt64("primary_columns", xcap.AggregationTypeSum) + StatSecondaryColumns = xcap.NewStatisticInt64("secondary_columns", xcap.AggregationTypeSum) // Page statistics - PrimaryColumnPages uint64 // Total pages in primary columns - SecondaryColumnPages uint64 // Total pages in secondary columns - DownloadStats DownloadStats // Download statistics for primary and secondary columns - - ReadCalls int64 // Total number of read calls made to the reader + StatPrimaryColumnPages = xcap.NewStatisticInt64("primary_column_pages", xcap.AggregationTypeSum) + StatSecondaryColumnPages = xcap.NewStatisticInt64("secondary_column_pages", xcap.AggregationTypeSum) // Row statistics - MaxRows uint64 // Maximum number of rows across all columns - RowsToReadAfterPruning uint64 // Total number of primary rows to read after page pruning - - PrimaryRowsRead uint64 // Actual number of primary rows read. - SecondaryRowsRead uint64 // Actual number of secondary rows read. - - PrimaryRowBytes uint64 // Total bytes read for primary rows - SecondaryRowBytes uint64 // Total bytes read for secondary rows -} - -type ctxKeyType string - -const ( - readerStatsKey ctxKeyType = "reader_stats" + StatMaxRows = xcap.NewStatisticInt64("max_rows", xcap.AggregationTypeSum) + StatRowsAfterPruning = xcap.NewStatisticInt64("rows_after_pruning", xcap.AggregationTypeSum) + StatPrimaryRowsRead = xcap.NewStatisticInt64("primary_rows_read", xcap.AggregationTypeSum) + StatSecondaryRowsRead = xcap.NewStatisticInt64("secondary_rows_read", xcap.AggregationTypeSum) + StatPrimaryRowBytes = xcap.NewStatisticInt64("primary_row_bytes_read", xcap.AggregationTypeSum) + StatSecondaryRowBytes = xcap.NewStatisticInt64("secondary_row_bytes_read", xcap.AggregationTypeSum) + + // Download/Page scan statistics + StatPagesScanned = xcap.NewStatisticInt64("pages_scanned", xcap.AggregationTypeSum) + StatPagesFoundInCache = xcap.NewStatisticInt64("pages_found_in_cache", xcap.AggregationTypeSum) + StatBatchDownloadRequests = xcap.NewStatisticInt64("batch_download_requests", xcap.AggregationTypeSum) + StatPageDownloadTime = xcap.NewStatisticInt64("page_download_time_ns", xcap.AggregationTypeSum) + + // Page download byte statistics + StatPrimaryColumnPagesDownloaded = xcap.NewStatisticInt64("primary_column_pages_downloaded", xcap.AggregationTypeSum) + StatSecondaryColumnPagesDownloaded = xcap.NewStatisticInt64("secondary_column_pages_downloaded", xcap.AggregationTypeSum) + StatPrimaryColumnBytes = xcap.NewStatisticInt64("primary_column_bytes", xcap.AggregationTypeSum) + StatSecondaryColumnBytes = xcap.NewStatisticInt64("secondary_column_bytes", xcap.AggregationTypeSum) + StatPrimaryColumnUncompressedBytes = xcap.NewStatisticInt64("primary_column_uncompressed_bytes", xcap.AggregationTypeSum) + StatSecondaryColumnUncompressedBytes = xcap.NewStatisticInt64("secondary_column_uncompressed_bytes", xcap.AggregationTypeSum) + + // Read operation statistics + StatReadCalls = xcap.NewStatisticInt64("read_calls", xcap.AggregationTypeSum) ) - -func WithStats(ctx context.Context, stats *ReaderStats) context.Context { - if stats == nil { - return ctx - } - - return context.WithValue(ctx, readerStatsKey, stats) -} - -// StatsFromContext returns the reader stats from context. -func StatsFromContext(ctx context.Context) *ReaderStats { - v, ok := ctx.Value(readerStatsKey).(*ReaderStats) - if !ok { - return &ReaderStats{} - } - return v -} - -func (s *ReaderStats) LinkGlobalStats(stats *stats.Context) { - // If the global stats are already set, we don't override it. - if s.globalStats == nil { - s.globalStats = stats - } -} - -func (s *ReaderStats) AddReadCalls(count int) { - s.ReadCalls += int64(count) -} - -func (s *ReaderStats) AddPrimaryColumns(count uint64) { - s.PrimaryColumns += count -} - -func (s *ReaderStats) AddSecondaryColumns(count uint64) { - s.SecondaryColumns += count -} - -func (s *ReaderStats) AddPrimaryColumnPages(count uint64) { - s.PrimaryColumnPages += count -} - -func (s *ReaderStats) AddSecondaryColumnPages(count uint64) { - s.SecondaryColumnPages += count -} - -func (s *ReaderStats) AddPrimaryRowsRead(count uint64) { - s.PrimaryRowsRead += count - if s.globalStats != nil { - s.globalStats.AddPrePredicateDecompressedRows(int64(count)) - } -} - -func (s *ReaderStats) AddSecondaryRowsRead(count uint64) { - s.SecondaryRowsRead += count - if s.globalStats != nil { - s.globalStats.AddPostPredicateRows(int64(count)) - } -} - -func (s *ReaderStats) AddPrimaryRowBytes(count uint64) { - s.PrimaryRowBytes += count - if s.globalStats != nil { - s.globalStats.AddPrePredicateDecompressedBytes(int64(count)) - } -} - -func (s *ReaderStats) AddSecondaryRowBytes(count uint64) { - s.SecondaryRowBytes += count - if s.globalStats != nil { - s.globalStats.AddPostPredicateDecompressedBytes(int64(count)) - } -} - -func (s *ReaderStats) AddMaxRows(count uint64) { - s.MaxRows += count -} - -func (s *ReaderStats) AddRowsToReadAfterPruning(count uint64) { - s.RowsToReadAfterPruning += count -} - -func (s *ReaderStats) AddTotalRowsAvailable(count int64) { - s.MaxRows += uint64(count) - if s.globalStats != nil { - s.globalStats.AddTotalRowsAvailable(count) - } -} - -func (s *ReaderStats) AddPagesScanned(count uint64) { - s.DownloadStats.PagesScanned += count - if s.globalStats != nil { - s.globalStats.AddPagesScanned(int64(count)) - } -} - -func (s *ReaderStats) AddPagesFoundInCache(count uint64) { - s.DownloadStats.PagesFoundInCache += count -} - -func (s *ReaderStats) AddBatchDownloadRequests(count uint64) { - s.DownloadStats.BatchDownloadRequests += count - - if s.globalStats != nil { - s.globalStats.AddPageBatches(int64(count)) - } -} - -// AddPageDownloadTime adds the given duration into the total download time for -// page metadata and data. -func (s *ReaderStats) AddPageDownloadTime(duration time.Duration) { - s.DownloadStats.PageDownloadTime += duration - if s.globalStats != nil { - s.globalStats.AddPageDownloadTime(duration) - } -} - -func (s *ReaderStats) AddPrimaryColumnPagesDownloaded(count uint64) { - s.DownloadStats.PrimaryColumnPages += count - if s.globalStats != nil { - s.globalStats.AddPagesDownloaded(int64(count)) - - } -} - -func (s *ReaderStats) AddSecondaryColumnPagesDownloaded(count uint64) { - s.DownloadStats.SecondaryColumnPages += count - if s.globalStats != nil { - s.globalStats.AddPagesDownloaded(int64(count)) - - } -} - -func (s *ReaderStats) AddPrimaryColumnBytesDownloaded(bytes uint64) { - s.DownloadStats.PrimaryColumnBytes += bytes - if s.globalStats != nil { - s.globalStats.AddPagesDownloadedBytes(int64(bytes)) - } -} - -func (s *ReaderStats) AddSecondaryColumnBytesDownloaded(bytes uint64) { - s.DownloadStats.SecondaryColumnBytes += bytes - if s.globalStats != nil { - s.globalStats.AddPagesDownloadedBytes(int64(bytes)) - } -} - -func (s *ReaderStats) AddPrimaryColumnUncompressedBytes(count uint64) { - s.DownloadStats.PrimaryColumnUncompressedBytes += count -} - -func (s *ReaderStats) AddSecondaryColumnUncompressedBytes(count uint64) { - s.DownloadStats.SecondaryColumnUncompressedBytes += count -} - -func (s *ReaderStats) Reset() { - s.PrimaryColumns = 0 - s.SecondaryColumns = 0 - - s.PrimaryColumnPages = 0 - s.SecondaryColumnPages = 0 - s.DownloadStats.Reset() - - s.ReadCalls = 0 - - s.MaxRows = 0 - s.RowsToReadAfterPruning = 0 - - s.PrimaryRowsRead = 0 - s.SecondaryRowsRead = 0 - - s.PrimaryRowBytes = 0 - s.SecondaryRowBytes = 0 - - s.globalStats = nil // Reset the global stats reference -} - -// LogSummary logs a summary of the read statistics to the provided logger. -func (s *ReaderStats) LogSummary(logger log.Logger, execDuration time.Duration) { - logValues := make([]any, 0, 50) - logValues = append(logValues, "msg", "dataset reader stats", - "execution_duration", execDuration, - "read_calls", s.ReadCalls, - "max_rows", s.MaxRows, - "rows_to_read_after_pruning", s.RowsToReadAfterPruning, - "primary_column_rows_read", s.PrimaryRowsRead, - "secondary_column_rows_read", s.SecondaryRowsRead, - "primary_column_bytes_read", humanize.Bytes(s.PrimaryRowBytes), - "secondary_column_bytes_read", humanize.Bytes(s.SecondaryRowBytes), - - "primary_columns", s.PrimaryColumns, - "secondary_columns", s.SecondaryColumns, - "primary_column_pages", s.PrimaryColumnPages, - "secondary_column_pages", s.SecondaryColumnPages, - - "total_pages_read", s.DownloadStats.PagesScanned, - "pages_found_in_cache", s.DownloadStats.PagesFoundInCache, - "batch_download_requests", s.DownloadStats.BatchDownloadRequests, - - "total_download_time", s.DownloadStats.PageDownloadTime, - - "primary_pages_downloaded", s.DownloadStats.PrimaryColumnPages, - "secondary_pages_downloaded", s.DownloadStats.SecondaryColumnPages, - "primary_page_bytes_downloaded", humanize.Bytes(s.DownloadStats.PrimaryColumnBytes), - "secondary_page_bytes_downloaded", humanize.Bytes(s.DownloadStats.SecondaryColumnBytes), - "primary_page_uncompressed_bytes", humanize.Bytes(s.DownloadStats.PrimaryColumnUncompressedBytes), - "secondary_page_uncompressed_bytes", humanize.Bytes(s.DownloadStats.SecondaryColumnUncompressedBytes), - ) - - level.Debug(logger).Log(logValues...) -} diff --git a/pkg/dataobj/internal/dataset/reader.go b/pkg/dataobj/internal/dataset/reader.go index c3ee591b963a2..17d5de5eb1a3b 100644 --- a/pkg/dataobj/internal/dataset/reader.go +++ b/pkg/dataobj/internal/dataset/reader.go @@ -10,6 +10,7 @@ import ( "github.com/grafana/loki/v3/pkg/dataobj/internal/metadata/datasetmd" "github.com/grafana/loki/v3/pkg/dataobj/internal/util/bitmask" "github.com/grafana/loki/v3/pkg/dataobj/internal/util/sliceclear" + "github.com/grafana/loki/v3/pkg/xcap" ) // ReaderOptions configures how a [Reader] will read [Row]s. @@ -48,7 +49,6 @@ type Reader struct { row int64 // The current row being read. inner *basicReader // Underlying reader that reads from columns. ranges rowRanges // Valid ranges to read across the entire dataset. - stats ReaderStats // Stats about the read operation. } // NewReader creates a new Reader from the provided options. @@ -62,8 +62,8 @@ func NewReader(opts ReaderOptions) *Reader { // returns the number of rows read and any error encountered. At the end of the // Dataset, Read returns 0, [io.EOF]. func (r *Reader) Read(ctx context.Context, s []Row) (int, error) { - stats := StatsFromContext(ctx) - stats.AddReadCalls(1) + region := xcap.RegionFromContext(ctx) + region.Record(StatReadCalls.Observe(1)) if len(s) == 0 { return 0, nil @@ -142,14 +142,15 @@ func (r *Reader) Read(ctx context.Context, s []Row) (int, error) { rowsRead = count passCount = count - stats.AddPrimaryRowsRead(uint64(rowsRead)) var primaryColumnBytes int64 for i := range count { primaryColumnBytes += s[i].Size() } - stats.AddPrimaryRowBytes(uint64(primaryColumnBytes)) + + region.Record(StatPrimaryRowsRead.Observe(int64(rowsRead))) + region.Record(StatPrimaryRowBytes.Observe(primaryColumnBytes)) } else { - rowsRead, passCount, err = r.readAndFilterPrimaryColumns(ctx, readSize, s[:readSize], stats) + rowsRead, passCount, err = r.readAndFilterPrimaryColumns(ctx, readSize, s[:readSize]) if err != nil { return passCount, err } @@ -179,8 +180,8 @@ func (r *Reader) Read(ctx context.Context, s []Row) (int, error) { totalBytesFilled += s[i].Size() - s[i].SizeOfColumns(r.primaryColumnIndexes) } - stats.AddSecondaryRowsRead(uint64(count)) - stats.AddSecondaryRowBytes(uint64(totalBytesFilled)) + region.Record(StatSecondaryRowsRead.Observe(int64(count))) + region.Record(StatSecondaryRowBytes.Observe(totalBytesFilled)) } // We only advance r.row after we successfully read and filled rows. This @@ -197,7 +198,7 @@ func (r *Reader) Read(ctx context.Context, s []Row) (int, error) { // the columns on the reduced row range. // // It returns the max rows read, rows that passed all the predicates, and any error -func (r *Reader) readAndFilterPrimaryColumns(ctx context.Context, readSize int, s []Row, stats *ReaderStats) (int, int, error) { +func (r *Reader) readAndFilterPrimaryColumns(ctx context.Context, readSize int, s []Row) (int, int, error) { var ( rowsRead int // tracks max rows accessed to move the [r.row] cursor passCount int // number of rows that passed the predicate @@ -265,8 +266,11 @@ func (r *Reader) readAndFilterPrimaryColumns(ctx context.Context, readSize int, readSize = passCount } - stats.AddPrimaryRowsRead(uint64(rowsRead)) - stats.AddPrimaryRowBytes(uint64(primaryColumnBytes)) + if region := xcap.RegionFromContext(ctx); region != nil { + region.Record(StatPrimaryRowsRead.Observe(int64(rowsRead))) + region.Record(StatPrimaryRowBytes.Observe(primaryColumnBytes)) + } + return rowsRead, passCount, nil } @@ -539,18 +543,18 @@ func (r *Reader) initDownloader(ctx context.Context) error { mask := bitmask.New(len(r.opts.Columns)) r.fillPrimaryMask(mask) - stats := StatsFromContext(ctx) + region := xcap.RegionFromContext(ctx) for i, column := range r.opts.Columns { primary := mask.Test(i) r.dl.AddColumn(column, primary) if primary { r.primaryColumnIndexes = append(r.primaryColumnIndexes, i) - stats.AddPrimaryColumns(1) - stats.AddPrimaryColumnPages(uint64(column.ColumnDesc().PagesCount)) + region.Record(StatPrimaryColumns.Observe(1)) + region.Record(StatPrimaryColumnPages.Observe(int64(column.ColumnDesc().PagesCount))) } else { - stats.AddSecondaryColumns(1) - stats.AddSecondaryColumnPages(uint64(column.ColumnDesc().PagesCount)) + region.Record(StatSecondaryColumns.Observe(1)) + region.Record(StatSecondaryColumnPages.Observe(int64(column.ColumnDesc().PagesCount))) } } @@ -584,8 +588,8 @@ func (r *Reader) initDownloader(ctx context.Context) error { rowsCount = max(rowsCount, uint64(column.ColumnDesc().RowsCount)) } - stats.AddTotalRowsAvailable(int64(rowsCount)) - stats.AddRowsToReadAfterPruning(ranges.TotalRowCount()) + region.Record(StatMaxRows.Observe(int64(rowsCount))) + region.Record(StatRowsAfterPruning.Observe(int64(ranges.TotalRowCount()))) return nil } diff --git a/pkg/dataobj/internal/dataset/reader_downloader.go b/pkg/dataobj/internal/dataset/reader_downloader.go index 1002eb493fab5..b990181116299 100644 --- a/pkg/dataobj/internal/dataset/reader_downloader.go +++ b/pkg/dataobj/internal/dataset/reader_downloader.go @@ -5,6 +5,7 @@ import ( "github.com/grafana/loki/v3/pkg/dataobj/internal/result" "github.com/grafana/loki/v3/pkg/dataobj/internal/util/sliceclear" + "github.com/grafana/loki/v3/pkg/xcap" ) // readerDownloader is a utility for downloading pages in bulk from a @@ -228,16 +229,17 @@ func (dl *readerDownloader) downloadBatch(ctx context.Context, requestor *reader return err } - stats := StatsFromContext(ctx) - for _, page := range batch { - if page.column.primary { - stats.AddPrimaryColumnPagesDownloaded(1) - stats.AddPrimaryColumnBytesDownloaded(uint64(page.inner.PageDesc().CompressedSize)) - stats.AddPrimaryColumnUncompressedBytes(uint64(page.inner.PageDesc().UncompressedSize)) - } else { - stats.AddSecondaryColumnPagesDownloaded(1) - stats.AddSecondaryColumnBytesDownloaded(uint64(page.inner.PageDesc().CompressedSize)) - stats.AddSecondaryColumnUncompressedBytes(uint64(page.inner.PageDesc().UncompressedSize)) + if region := xcap.RegionFromContext(ctx); region != nil { + for _, page := range batch { + if page.column.primary { + region.Record(StatPrimaryColumnPagesDownloaded.Observe(1)) + region.Record(StatPrimaryColumnBytes.Observe(int64(page.inner.PageDesc().CompressedSize))) + region.Record(StatPrimaryColumnUncompressedBytes.Observe(int64(page.inner.PageDesc().UncompressedSize))) + } else { + region.Record(StatSecondaryColumnPagesDownloaded.Observe(1)) + region.Record(StatSecondaryColumnBytes.Observe(int64(page.inner.PageDesc().CompressedSize))) + region.Record(StatSecondaryColumnUncompressedBytes.Observe(int64(page.inner.PageDesc().UncompressedSize))) + } } } @@ -586,14 +588,14 @@ func (page *readerPage) PageDesc() *PageDesc { } func (page *readerPage) ReadPage(ctx context.Context) (PageData, error) { - stats := StatsFromContext(ctx) - stats.AddPagesScanned(1) + region := xcap.RegionFromContext(ctx) + region.Record(StatPagesScanned.Observe(1)) if page.data != nil { - stats.AddPagesFoundInCache(1) + region.Record(StatPagesFoundInCache.Observe(1)) return page.data, nil } - stats.AddBatchDownloadRequests(1) + region.Record(StatBatchDownloadRequests.Observe(1)) if err := page.column.dl.downloadBatch(ctx, page); err != nil { return nil, err } diff --git a/pkg/dataobj/sections/internal/columnar/decoder.go b/pkg/dataobj/sections/internal/columnar/decoder.go index 1f8d604b541f8..8488ac8e7c06f 100644 --- a/pkg/dataobj/sections/internal/columnar/decoder.go +++ b/pkg/dataobj/sections/internal/columnar/decoder.go @@ -14,6 +14,7 @@ import ( "github.com/grafana/loki/v3/pkg/dataobj/internal/util/bufpool" "github.com/grafana/loki/v3/pkg/dataobj/internal/util/protocodec" "github.com/grafana/loki/v3/pkg/util/rangeio" + "github.com/grafana/loki/v3/pkg/xcap" ) // A Decoder allows reading an encoded dataset-based section. @@ -75,9 +76,11 @@ func (dec *Decoder) getSectionInfo() (*datasetmd.SectionInfoExtension, error) { // first slice corresponds to the first column, and so on. func (dec *Decoder) Pages(ctx context.Context, columns []*datasetmd.ColumnDesc) result.Seq[[]*datasetmd.PageDesc] { return result.Iter(func(yield func([]*datasetmd.PageDesc) bool) error { - stats := dataset.StatsFromContext(ctx) + region := xcap.RegionFromContext(ctx) startTime := time.Now() - defer func() { stats.AddPageDownloadTime(time.Since(startTime)) }() + defer func() { + region.Record(dataset.StatPageDownloadTime.Observe(time.Since(startTime).Nanoseconds())) + }() ranges := make([]rangeio.Range, 0, len(columns)) for _, column := range columns { @@ -129,9 +132,11 @@ func (rr metadataRangeReader) ReadRange(ctx context.Context, r rangeio.Range) (i // error is emitted from the sequence and iteration stops. func (dec *Decoder) ReadPages(ctx context.Context, pages []*datasetmd.PageDesc) result.Seq[dataset.PageData] { return result.Iter(func(yield func(dataset.PageData) bool) error { - stats := dataset.StatsFromContext(ctx) + region := xcap.RegionFromContext(ctx) startTime := time.Now() - defer func() { stats.AddPageDownloadTime(time.Since(startTime)) }() + defer func() { + region.Record(dataset.StatPageDownloadTime.Observe(time.Since(startTime).Nanoseconds())) + }() ranges := make([]rangeio.Range, 0, len(pages)) for _, page := range pages { diff --git a/pkg/dataobj/sections/logs/reader.go b/pkg/dataobj/sections/logs/reader.go index cb6a950745524..b19cdb6f9b0ee 100644 --- a/pkg/dataobj/sections/logs/reader.go +++ b/pkg/dataobj/sections/logs/reader.go @@ -16,7 +16,6 @@ import ( "github.com/grafana/loki/v3/pkg/dataobj/internal/metadata/datasetmd" "github.com/grafana/loki/v3/pkg/dataobj/internal/util/slicegrow" "github.com/grafana/loki/v3/pkg/dataobj/sections/internal/columnar" - "github.com/grafana/loki/v3/pkg/logqlmodel/stats" ) // ReaderOptions customizes the behavior of a [Reader]. @@ -117,7 +116,6 @@ type Reader struct { ready bool inner *dataset.Reader buf []dataset.Row - stats dataset.ReaderStats } // NewReader creates a new Reader from the provided options. Options are not @@ -167,7 +165,7 @@ func (r *Reader) Read(ctx context.Context, batchSize int) (arrow.RecordBatch, er builder := array.NewRecordBuilder(r.opts.Allocator, r.schema) - n, readErr := r.inner.Read(dataset.WithStats(ctx, &r.stats), r.buf) + n, readErr := r.inner.Read(ctx, r.buf) for rowIndex := range n { row := r.buf[rowIndex] @@ -255,9 +253,6 @@ func (r *Reader) init(ctx context.Context) error { return fmt.Errorf("mapping predicates: %w", err) } - // TODO(ashwanth): remove when global stats are updated by the executor. - r.stats.LinkGlobalStats(stats.FromContext(ctx)) - innerOptions := dataset.ReaderOptions{ Dataset: dset, Columns: dset.Columns(), @@ -413,7 +408,6 @@ func mustConvertType(dtype arrow.DataType) datasetmd.PhysicalType { func (r *Reader) Reset(opts ReaderOptions) { r.opts = opts r.schema = columnsSchema(opts.Columns) - r.stats.Reset() r.ready = false @@ -424,10 +418,6 @@ func (r *Reader) Reset(opts ReaderOptions) { } } -func (r *Reader) Stats() *dataset.ReaderStats { - return &r.stats -} - // Close closes the Reader and releases any resources it holds. Closed Readers // can be reused by calling [Reader.Reset]. func (r *Reader) Close() error { diff --git a/pkg/dataobj/sections/logs/reader_test.go b/pkg/dataobj/sections/logs/reader_test.go index eed50c101a22d..2921fc818fd80 100644 --- a/pkg/dataobj/sections/logs/reader_test.go +++ b/pkg/dataobj/sections/logs/reader_test.go @@ -16,9 +16,10 @@ import ( "github.com/stretchr/testify/require" "github.com/grafana/loki/v3/pkg/dataobj" + "github.com/grafana/loki/v3/pkg/dataobj/internal/dataset" "github.com/grafana/loki/v3/pkg/dataobj/sections/logs" - "github.com/grafana/loki/v3/pkg/logqlmodel/stats" "github.com/grafana/loki/v3/pkg/util/arrowtest" + "github.com/grafana/loki/v3/pkg/xcap" ) // TestReader does a basic end-to-end test over a reader with a predicate applied. @@ -161,7 +162,7 @@ func readTable(ctx context.Context, r *logs.Reader) (arrow.Table, error) { return array.NewTableFromRecords(recs[0].Schema(), recs), nil } -// TestReaderStats tests that the reader properly tracks statistics +// TestReaderStats tests that the reader properly tracks statistics via xcap regions. func TestReaderStats(t *testing.T) { sec := buildSection(t, []logs.Record{ {StreamID: 2, Timestamp: unixTime(40), Metadata: labels.FromStrings("trace_id", "789012"), Line: []byte("baz qux")}, @@ -202,35 +203,37 @@ func TestReaderStats(t *testing.T) { }, }) - // Create stats context - statsCtx, ctx := stats.NewContext(context.Background()) + // Create xcap capture and region + ctx, _ := xcap.NewCapture(context.Background(), nil) + ctx, region := xcap.StartRegion(ctx, "test-reader") // Read the data _, err := readTable(ctx, r) require.NoError(t, err) - // Get the reader stats - readerStats := r.Stats() + region.End() + + // Get observations from the region + observations := region.Observations() + obsMap := make(map[string]int64) + for _, obs := range observations { + obsMap[obs.Statistic.Name()] = obs.Value.(int64) + } // Verify the stats are properly populated - require.Equal(t, int64(2), readerStats.ReadCalls) - require.Equal(t, uint64(2), readerStats.PrimaryColumns) // from 2 predicates - require.Equal(t, uint64(1), readerStats.SecondaryColumns) - require.Equal(t, uint64(2), readerStats.PrimaryColumnPages) - require.Equal(t, uint64(1), readerStats.SecondaryColumnPages) + require.Equal(t, int64(2), obsMap[dataset.StatReadCalls.Name()]) + require.Equal(t, int64(2), obsMap[dataset.StatPrimaryColumns.Name()]) // from 2 predicates + require.Equal(t, int64(1), obsMap[dataset.StatSecondaryColumns.Name()]) // 1 secondary column + require.Equal(t, int64(2), obsMap[dataset.StatPrimaryColumnPages.Name()]) + require.Equal(t, int64(1), obsMap[dataset.StatSecondaryColumnPages.Name()]) - require.Equal(t, uint64(4), readerStats.MaxRows) - require.Equal(t, uint64(4), readerStats.RowsToReadAfterPruning) - require.Equal(t, uint64(4), readerStats.PrimaryRowsRead) - require.Equal(t, uint64(2), readerStats.SecondaryRowsRead) // 2 rows pass the predicate + require.Equal(t, int64(4), obsMap[dataset.StatTotalRowsAvailable.Name()]) + require.Equal(t, int64(4), obsMap[dataset.StatRowsAfterPruning.Name()]) + require.Equal(t, int64(4), obsMap[dataset.StatPrimaryRowsRead.Name()]) + require.Equal(t, int64(2), obsMap[dataset.StatSecondaryRowsRead.Name()]) // 2 rows pass the predicate // Verify download stats - these should be populated by the downloader - require.Equal(t, uint64(3), readerStats.DownloadStats.PagesScanned) // one page per column - require.Equal(t, uint64(2), readerStats.DownloadStats.PrimaryColumnPages) - require.Equal(t, uint64(1), readerStats.DownloadStats.SecondaryColumnPages) - - // Verify global stats are updated - result := statsCtx.Result(0, 0, 0) - require.Equal(t, int64(4), result.Querier.Store.Dataobj.PrePredicateDecompressedRows) - require.Equal(t, int64(2), result.Querier.Store.Dataobj.PostPredicateRows) + require.Equal(t, int64(3), obsMap[dataset.StatPagesScanned.Name()]) // one page per column + require.Equal(t, int64(2), obsMap[dataset.StatPrimaryColumnPagesDownloaded.Name()]) + require.Equal(t, int64(1), obsMap[dataset.StatSecondaryColumnPagesDownloaded.Name()]) } diff --git a/pkg/engine/internal/executor/dataobjscan.go b/pkg/engine/internal/executor/dataobjscan.go index bb91da1f5cbf0..d5178d7bfa8c7 100644 --- a/pkg/engine/internal/executor/dataobjscan.go +++ b/pkg/engine/internal/executor/dataobjscan.go @@ -19,7 +19,6 @@ import ( "github.com/grafana/loki/v3/pkg/engine/internal/planner/physical" "github.com/grafana/loki/v3/pkg/engine/internal/semconv" "github.com/grafana/loki/v3/pkg/engine/internal/types" - "github.com/grafana/loki/v3/pkg/xcap" ) type dataobjScanOptions struct { @@ -35,7 +34,6 @@ type dataobjScanOptions struct { type dataobjScan struct { opts dataobjScanOptions logger log.Logger - region *xcap.Region initialized bool initializedAt time.Time @@ -51,11 +49,10 @@ var _ Pipeline = (*dataobjScan)(nil) // [arrow.RecordBatch] composed of the requested log section in a data object. Rows // in the returned record are ordered by timestamp in the direction specified // by opts.Direction. -func newDataobjScanPipeline(opts dataobjScanOptions, logger log.Logger, region *xcap.Region) *dataobjScan { +func newDataobjScanPipeline(opts dataobjScanOptions, logger log.Logger) *dataobjScan { return &dataobjScan{ opts: opts, logger: logger, - region: region, } } @@ -373,10 +370,6 @@ func (s *dataobjScan) read(ctx context.Context) (arrow.RecordBatch, error) { // Close closes s and releases all resources. func (s *dataobjScan) Close() { - if s.region != nil && s.reader != nil { - s.recordReaderStats() - s.region.End() - } if s.streams != nil { s.streams.Close() } @@ -388,43 +381,6 @@ func (s *dataobjScan) Close() { s.streams = nil s.streamsInjector = nil s.reader = nil - s.region = nil -} - -// recordReaderStats records statistics from the [logs.Reader] to the xcap region. -// TODO: [dataset.ReaderStats] should be replaced by xcap statistics. -func (s *dataobjScan) recordReaderStats() { - if s.region == nil || s.reader == nil { - return - } - - stats := s.reader.Stats() - if stats == nil { - return - } - - // Record basic stats - s.region.Record(statDatasetPrimaryColumns.Observe(int64(stats.PrimaryColumns))) - s.region.Record(statDatasetSecondaryColumns.Observe(int64(stats.SecondaryColumns))) - s.region.Record(statDatasetPrimaryColumnPages.Observe(int64(stats.PrimaryColumnPages))) - s.region.Record(statDatasetSecondaryColumnPages.Observe(int64(stats.SecondaryColumnPages))) - s.region.Record(statDatasetMaxRows.Observe(int64(stats.MaxRows))) - s.region.Record(statDatasetRowsAfterPruning.Observe(int64(stats.RowsToReadAfterPruning))) - s.region.Record(statDatasetPrimaryRowsRead.Observe(int64(stats.PrimaryRowsRead))) - s.region.Record(statDatasetSecondaryRowsRead.Observe(int64(stats.SecondaryRowsRead))) - s.region.Record(statDatasetPrimaryRowBytes.Observe(int64(stats.PrimaryRowBytes))) - s.region.Record(statDatasetSecondaryRowBytes.Observe(int64(stats.SecondaryRowBytes))) - - // Record download stats - downloadStats := stats.DownloadStats - s.region.Record(statDatasetPagesScanned.Observe(int64(downloadStats.PagesScanned))) - s.region.Record(statDatasetPagesFoundInCache.Observe(int64(downloadStats.PagesFoundInCache))) - s.region.Record(statDatasetBatchDownloadRequests.Observe(int64(downloadStats.BatchDownloadRequests))) - s.region.Record(statDatasetPageDownloadTime.Observe(downloadStats.PageDownloadTime.Nanoseconds())) - s.region.Record(statDatasetPrimaryColumnBytes.Observe(int64(downloadStats.PrimaryColumnBytes))) - s.region.Record(statDatasetSecondaryColumnBytes.Observe(int64(downloadStats.SecondaryColumnBytes))) - s.region.Record(statDatasetPrimaryColumnUncompressedBytes.Observe(int64(downloadStats.PrimaryColumnUncompressedBytes))) - s.region.Record(statDatasetSecondaryColumnUncompressedBytes.Observe(int64(downloadStats.SecondaryColumnUncompressedBytes))) } // Region implements RegionProvider. diff --git a/pkg/engine/internal/executor/dataobjscan_test.go b/pkg/engine/internal/executor/dataobjscan_test.go index 8192fd1aa27e8..d54942927f988 100644 --- a/pkg/engine/internal/executor/dataobjscan_test.go +++ b/pkg/engine/internal/executor/dataobjscan_test.go @@ -82,7 +82,7 @@ func Test_dataobjScan(t *testing.T) { Projections: nil, // All columns BatchSize: 512, - }, log.NewNopLogger(), nil) + }, log.NewNopLogger()) expectFields := []arrow.Field{ semconv.FieldFromFQN("utf8.label.env", true), @@ -115,7 +115,7 @@ prod,notloki,NULL,notloki-pod-1,1970-01-01 00:00:02,hello world` }, BatchSize: 512, - }, log.NewNopLogger(), nil) + }, log.NewNopLogger()) expectFields := []arrow.Field{ semconv.FieldFromFQN("utf8.label.env", true), @@ -141,7 +141,7 @@ prod,1970-01-01 00:00:02` Projections: nil, // All columns BatchSize: 512, - }, log.NewNopLogger(), nil) + }, log.NewNopLogger()) expectFields := []arrow.Field{ semconv.FieldFromFQN("utf8.label.env", true), @@ -172,7 +172,7 @@ prod,notloki,NULL,notloki-pod-1,1970-01-01 00:00:02,hello world` &physical.ColumnExpr{Ref: types.ColumnRef{Column: "env", Type: types.ColumnTypeAmbiguous}}, }, BatchSize: 512, - }, log.NewNopLogger(), nil) + }, log.NewNopLogger()) expectFields := []arrow.Field{ semconv.FieldFromFQN("utf8.label.env", true), @@ -251,7 +251,7 @@ func Test_dataobjScan_DuplicateColumns(t *testing.T) { StreamIDs: []int64{1, 2, 3}, // All streams Projections: nil, // All columns BatchSize: 512, - }, log.NewNopLogger(), nil) + }, log.NewNopLogger()) expectFields := []arrow.Field{ semconv.FieldFromFQN("utf8.label.env", true), @@ -285,7 +285,7 @@ prod,NULL,pod-1,loki,NULL,override,1970-01-01 00:00:01,message 1` &physical.ColumnExpr{Ref: types.ColumnRef{Column: "pod", Type: types.ColumnTypeAmbiguous}}, }, BatchSize: 512, - }, log.NewNopLogger(), nil) + }, log.NewNopLogger()) expectFields := []arrow.Field{ semconv.FieldFromFQN("utf8.label.pod", true), @@ -311,7 +311,7 @@ pod-1,override` &physical.ColumnExpr{Ref: types.ColumnRef{Column: "namespace", Type: types.ColumnTypeAmbiguous}}, }, BatchSize: 512, - }, log.NewNopLogger(), nil) + }, log.NewNopLogger()) expectFields := []arrow.Field{ semconv.FieldFromFQN("utf8.label.namespace", true), diff --git a/pkg/engine/internal/executor/executor.go b/pkg/engine/internal/executor/executor.go index 467cf5abd9fc2..cfd17c188e40e 100644 --- a/pkg/engine/internal/executor/executor.go +++ b/pkg/engine/internal/executor/executor.go @@ -203,7 +203,7 @@ func (c *Context) executeDataObjScan(ctx context.Context, node *physical.DataObj Projections: node.Projections, BatchSize: c.batchSize, - }, log.With(c.logger, "location", string(node.Location), "section", node.Section), region) + }, log.With(c.logger, "location", string(node.Location), "section", node.Section)) return pipeline } diff --git a/pkg/engine/internal/executor/stats.go b/pkg/engine/internal/executor/stats.go index 22fc63921c278..b86a1a82cd470 100644 --- a/pkg/engine/internal/executor/stats.go +++ b/pkg/engine/internal/executor/stats.go @@ -11,27 +11,4 @@ var ( // [ColumnCompat] statistics. statCompatCollisionFound = xcap.NewStatisticFlag("collision_found") - - // [DataObjScan] statistics - statDatasetPrimaryColumns = xcap.NewStatisticInt64("dataset_primary_columns", xcap.AggregationTypeSum) - statDatasetSecondaryColumns = xcap.NewStatisticInt64("dataset_secondary_columns", xcap.AggregationTypeSum) - statDatasetPrimaryColumnPages = xcap.NewStatisticInt64("dataset_primary_column_pages", xcap.AggregationTypeSum) - statDatasetSecondaryColumnPages = xcap.NewStatisticInt64("dataset_secondary_column_pages", xcap.AggregationTypeSum) - statDatasetMaxRows = xcap.NewStatisticInt64("dataset_max_rows", xcap.AggregationTypeSum) - statDatasetRowsAfterPruning = xcap.NewStatisticInt64("dataset_rows_after_pruning", xcap.AggregationTypeSum) - - statDatasetPrimaryRowsRead = xcap.NewStatisticInt64("primary_rows_read", xcap.AggregationTypeSum) - statDatasetSecondaryRowsRead = xcap.NewStatisticInt64("secondary_rows_read", xcap.AggregationTypeSum) - statDatasetPrimaryRowBytes = xcap.NewStatisticInt64("primary_row_bytes_read", xcap.AggregationTypeSum) - statDatasetSecondaryRowBytes = xcap.NewStatisticInt64("secondary_row_bytes_read", xcap.AggregationTypeSum) - - statDatasetPagesScanned = xcap.NewStatisticInt64("pages_scanned", xcap.AggregationTypeSum) - statDatasetPagesFoundInCache = xcap.NewStatisticInt64("pages_found_in_cache", xcap.AggregationTypeSum) - statDatasetBatchDownloadRequests = xcap.NewStatisticInt64("batch_download_requests", xcap.AggregationTypeSum) - statDatasetPageDownloadTime = xcap.NewStatisticInt64("page_download_time_nanos", xcap.AggregationTypeSum) - - statDatasetPrimaryColumnBytes = xcap.NewStatisticInt64("primary_column_bytes", xcap.AggregationTypeSum) - statDatasetSecondaryColumnBytes = xcap.NewStatisticInt64("secondary_column_bytes", xcap.AggregationTypeSum) - statDatasetPrimaryColumnUncompressedBytes = xcap.NewStatisticInt64("primary_column_uncompressed_bytes", xcap.AggregationTypeSum) - statDatasetSecondaryColumnUncompressedBytes = xcap.NewStatisticInt64("secondary_column_uncompressed_bytes", xcap.AggregationTypeSum) ) From 8dd8057089d239709be9d627fa14640f0b6a89c4 Mon Sep 17 00:00:00 2001 From: Ashwanth Goli Date: Tue, 2 Dec 2025 11:14:38 +0530 Subject: [PATCH 02/13] reader: inject region in ctx --- pkg/dataobj/internal/dataset/reader.go | 41 +++++++++++++++----------- 1 file changed, 23 insertions(+), 18 deletions(-) diff --git a/pkg/dataobj/internal/dataset/reader.go b/pkg/dataobj/internal/dataset/reader.go index 17d5de5eb1a3b..ea2ae3dfd567a 100644 --- a/pkg/dataobj/internal/dataset/reader.go +++ b/pkg/dataobj/internal/dataset/reader.go @@ -49,6 +49,8 @@ type Reader struct { row int64 // The current row being read. inner *basicReader // Underlying reader that reads from columns. ranges rowRanges // Valid ranges to read across the entire dataset. + + region *xcap.Region // Region for recording statistics. } // NewReader creates a new Reader from the provided options. @@ -62,9 +64,6 @@ func NewReader(opts ReaderOptions) *Reader { // returns the number of rows read and any error encountered. At the end of the // Dataset, Read returns 0, [io.EOF]. func (r *Reader) Read(ctx context.Context, s []Row) (int, error) { - region := xcap.RegionFromContext(ctx) - region.Record(StatReadCalls.Observe(1)) - if len(s) == 0 { return 0, nil } @@ -76,6 +75,9 @@ func (r *Reader) Read(ctx context.Context, s []Row) (int, error) { } } + r.region.Record(StatReadCalls.Observe(1)) + ctx = xcap.ContextWithRegion(ctx, r.region) + // Our Read implementation works by: // // 1. Determining the next row to read (aligned to a valid range), @@ -147,8 +149,8 @@ func (r *Reader) Read(ctx context.Context, s []Row) (int, error) { primaryColumnBytes += s[i].Size() } - region.Record(StatPrimaryRowsRead.Observe(int64(rowsRead))) - region.Record(StatPrimaryRowBytes.Observe(primaryColumnBytes)) + r.region.Record(StatPrimaryRowsRead.Observe(int64(rowsRead))) + r.region.Record(StatPrimaryRowBytes.Observe(primaryColumnBytes)) } else { rowsRead, passCount, err = r.readAndFilterPrimaryColumns(ctx, readSize, s[:readSize]) if err != nil { @@ -180,8 +182,8 @@ func (r *Reader) Read(ctx context.Context, s []Row) (int, error) { totalBytesFilled += s[i].Size() - s[i].SizeOfColumns(r.primaryColumnIndexes) } - region.Record(StatSecondaryRowsRead.Observe(int64(count))) - region.Record(StatSecondaryRowBytes.Observe(totalBytesFilled)) + r.region.Record(StatSecondaryRowsRead.Observe(int64(count))) + r.region.Record(StatSecondaryRowBytes.Observe(totalBytesFilled)) } // We only advance r.row after we successfully read and filled rows. This @@ -266,10 +268,8 @@ func (r *Reader) readAndFilterPrimaryColumns(ctx context.Context, readSize int, readSize = passCount } - if region := xcap.RegionFromContext(ctx); region != nil { - region.Record(StatPrimaryRowsRead.Observe(int64(rowsRead))) - region.Record(StatPrimaryRowBytes.Observe(primaryColumnBytes)) - } + r.region.Record(StatPrimaryRowsRead.Observe(int64(rowsRead))) + r.region.Record(StatPrimaryRowBytes.Observe(primaryColumnBytes)) return rowsRead, passCount, nil } @@ -398,9 +398,12 @@ func buildMask(full rowRange, s []Row) iter.Seq[rowRange] { // Close closes the Reader. Closed Readers can be reused by calling // [Reader.Reset]. func (r *Reader) Close() error { + r.region.End() + if r.inner != nil { return r.inner.Close() } + return nil } @@ -425,12 +428,15 @@ func (r *Reader) Reset(opts ReaderOptions) { r.ranges = sliceclear.Clear(r.ranges) r.primaryColumnIndexes = sliceclear.Clear(r.primaryColumnIndexes) r.ready = false + r.region = nil } func (r *Reader) init(ctx context.Context) error { // Reader.init is kept close to the defition of Reader.Reset to make it // easier to follow the correctness of resetting + initializing. + _, r.region = xcap.StartRegion(ctx, "dataset.Reader") + // r.validatePredicate must be called before initializing anything else; for // simplicity, other functions assume that the predicate is valid and can // panic if it isn't. @@ -543,18 +549,17 @@ func (r *Reader) initDownloader(ctx context.Context) error { mask := bitmask.New(len(r.opts.Columns)) r.fillPrimaryMask(mask) - region := xcap.RegionFromContext(ctx) for i, column := range r.opts.Columns { primary := mask.Test(i) r.dl.AddColumn(column, primary) if primary { r.primaryColumnIndexes = append(r.primaryColumnIndexes, i) - region.Record(StatPrimaryColumns.Observe(1)) - region.Record(StatPrimaryColumnPages.Observe(int64(column.ColumnDesc().PagesCount))) + r.region.Record(StatPrimaryColumns.Observe(1)) + r.region.Record(StatPrimaryColumnPages.Observe(int64(column.ColumnDesc().PagesCount))) } else { - region.Record(StatSecondaryColumns.Observe(1)) - region.Record(StatSecondaryColumnPages.Observe(int64(column.ColumnDesc().PagesCount))) + r.region.Record(StatSecondaryColumns.Observe(1)) + r.region.Record(StatSecondaryColumnPages.Observe(int64(column.ColumnDesc().PagesCount))) } } @@ -588,8 +593,8 @@ func (r *Reader) initDownloader(ctx context.Context) error { rowsCount = max(rowsCount, uint64(column.ColumnDesc().RowsCount)) } - region.Record(StatMaxRows.Observe(int64(rowsCount))) - region.Record(StatRowsAfterPruning.Observe(int64(ranges.TotalRowCount()))) + r.region.Record(StatMaxRows.Observe(int64(rowsCount))) + r.region.Record(StatRowsAfterPruning.Observe(int64(ranges.TotalRowCount()))) return nil } From 18f57607f6cc4b4375daebcd93c5f2679e4198c0 Mon Sep 17 00:00:00 2001 From: Ashwanth Goli Date: Wed, 3 Dec 2025 12:09:25 +0530 Subject: [PATCH 03/13] fixup! reader: inject region in ctx --- pkg/engine/internal/executor/dataobjscan.go | 9 ++++++++- pkg/engine/internal/executor/dataobjscan_test.go | 14 +++++++------- pkg/engine/internal/executor/executor.go | 2 +- 3 files changed, 16 insertions(+), 9 deletions(-) diff --git a/pkg/engine/internal/executor/dataobjscan.go b/pkg/engine/internal/executor/dataobjscan.go index d5178d7bfa8c7..11e4eed412ec1 100644 --- a/pkg/engine/internal/executor/dataobjscan.go +++ b/pkg/engine/internal/executor/dataobjscan.go @@ -19,6 +19,7 @@ import ( "github.com/grafana/loki/v3/pkg/engine/internal/planner/physical" "github.com/grafana/loki/v3/pkg/engine/internal/semconv" "github.com/grafana/loki/v3/pkg/engine/internal/types" + "github.com/grafana/loki/v3/pkg/xcap" ) type dataobjScanOptions struct { @@ -34,6 +35,7 @@ type dataobjScanOptions struct { type dataobjScan struct { opts dataobjScanOptions logger log.Logger + region *xcap.Region initialized bool initializedAt time.Time @@ -49,10 +51,11 @@ var _ Pipeline = (*dataobjScan)(nil) // [arrow.RecordBatch] composed of the requested log section in a data object. Rows // in the returned record are ordered by timestamp in the direction specified // by opts.Direction. -func newDataobjScanPipeline(opts dataobjScanOptions, logger log.Logger) *dataobjScan { +func newDataobjScanPipeline(opts dataobjScanOptions, logger log.Logger, region *xcap.Region) *dataobjScan { return &dataobjScan{ opts: opts, logger: logger, + region: region, } } @@ -370,6 +373,9 @@ func (s *dataobjScan) read(ctx context.Context) (arrow.RecordBatch, error) { // Close closes s and releases all resources. func (s *dataobjScan) Close() { + if s.region != nil { + s.region.End() + } if s.streams != nil { s.streams.Close() } @@ -381,6 +387,7 @@ func (s *dataobjScan) Close() { s.streams = nil s.streamsInjector = nil s.reader = nil + s.region = nil } // Region implements RegionProvider. diff --git a/pkg/engine/internal/executor/dataobjscan_test.go b/pkg/engine/internal/executor/dataobjscan_test.go index d54942927f988..8192fd1aa27e8 100644 --- a/pkg/engine/internal/executor/dataobjscan_test.go +++ b/pkg/engine/internal/executor/dataobjscan_test.go @@ -82,7 +82,7 @@ func Test_dataobjScan(t *testing.T) { Projections: nil, // All columns BatchSize: 512, - }, log.NewNopLogger()) + }, log.NewNopLogger(), nil) expectFields := []arrow.Field{ semconv.FieldFromFQN("utf8.label.env", true), @@ -115,7 +115,7 @@ prod,notloki,NULL,notloki-pod-1,1970-01-01 00:00:02,hello world` }, BatchSize: 512, - }, log.NewNopLogger()) + }, log.NewNopLogger(), nil) expectFields := []arrow.Field{ semconv.FieldFromFQN("utf8.label.env", true), @@ -141,7 +141,7 @@ prod,1970-01-01 00:00:02` Projections: nil, // All columns BatchSize: 512, - }, log.NewNopLogger()) + }, log.NewNopLogger(), nil) expectFields := []arrow.Field{ semconv.FieldFromFQN("utf8.label.env", true), @@ -172,7 +172,7 @@ prod,notloki,NULL,notloki-pod-1,1970-01-01 00:00:02,hello world` &physical.ColumnExpr{Ref: types.ColumnRef{Column: "env", Type: types.ColumnTypeAmbiguous}}, }, BatchSize: 512, - }, log.NewNopLogger()) + }, log.NewNopLogger(), nil) expectFields := []arrow.Field{ semconv.FieldFromFQN("utf8.label.env", true), @@ -251,7 +251,7 @@ func Test_dataobjScan_DuplicateColumns(t *testing.T) { StreamIDs: []int64{1, 2, 3}, // All streams Projections: nil, // All columns BatchSize: 512, - }, log.NewNopLogger()) + }, log.NewNopLogger(), nil) expectFields := []arrow.Field{ semconv.FieldFromFQN("utf8.label.env", true), @@ -285,7 +285,7 @@ prod,NULL,pod-1,loki,NULL,override,1970-01-01 00:00:01,message 1` &physical.ColumnExpr{Ref: types.ColumnRef{Column: "pod", Type: types.ColumnTypeAmbiguous}}, }, BatchSize: 512, - }, log.NewNopLogger()) + }, log.NewNopLogger(), nil) expectFields := []arrow.Field{ semconv.FieldFromFQN("utf8.label.pod", true), @@ -311,7 +311,7 @@ pod-1,override` &physical.ColumnExpr{Ref: types.ColumnRef{Column: "namespace", Type: types.ColumnTypeAmbiguous}}, }, BatchSize: 512, - }, log.NewNopLogger()) + }, log.NewNopLogger(), nil) expectFields := []arrow.Field{ semconv.FieldFromFQN("utf8.label.namespace", true), diff --git a/pkg/engine/internal/executor/executor.go b/pkg/engine/internal/executor/executor.go index cfd17c188e40e..467cf5abd9fc2 100644 --- a/pkg/engine/internal/executor/executor.go +++ b/pkg/engine/internal/executor/executor.go @@ -203,7 +203,7 @@ func (c *Context) executeDataObjScan(ctx context.Context, node *physical.DataObj Projections: node.Projections, BatchSize: c.batchSize, - }, log.With(c.logger, "location", string(node.Location), "section", node.Section)) + }, log.With(c.logger, "location", string(node.Location), "section", node.Section), region) return pipeline } From 9f31f932d6f1c64d8fdeea2926cc134382bfa73a Mon Sep 17 00:00:00 2001 From: Ashwanth Goli Date: Thu, 4 Dec 2025 11:12:37 +0530 Subject: [PATCH 04/13] update tests --- pkg/dataobj/internal/dataset/reader_test.go | 45 ++++++++++++ pkg/dataobj/sections/logs/reader.go | 2 +- pkg/dataobj/sections/logs/reader_test.go | 78 --------------------- pkg/xcap/region.go | 13 ++++ 4 files changed, 59 insertions(+), 79 deletions(-) diff --git a/pkg/dataobj/internal/dataset/reader_test.go b/pkg/dataobj/internal/dataset/reader_test.go index 4b29e449ddc6e..a0fc57a95d6d8 100644 --- a/pkg/dataobj/internal/dataset/reader_test.go +++ b/pkg/dataobj/internal/dataset/reader_test.go @@ -16,6 +16,7 @@ import ( "github.com/grafana/loki/v3/pkg/dataobj/internal/metadata/datasetmd" "github.com/grafana/loki/v3/pkg/dataobj/internal/result" + "github.com/grafana/loki/v3/pkg/xcap" ) func Test_Reader_ReadAll(t *testing.T) { @@ -883,3 +884,47 @@ func Test_DatasetGenerator(t *testing.T) { t.Logf("timestamp column size: %s", humanize.Bytes(uint64(cols[0].ColumnDesc().UncompressedSize))) t.Logf("label column size: %s", humanize.Bytes(uint64(cols[1].ColumnDesc().UncompressedSize))) } + +// Test_Reader_Stats tests that the reader properly tracks statistics via xcap regions. +func Test_Reader_Stats(t *testing.T) { + dset, columns := buildTestDataset(t) + + r := NewReader(ReaderOptions{ + Dataset: dset, + Columns: columns, + Predicates: []Predicate{ + GreaterThanPredicate{ + Column: columns[3], // birth_year + Value: Int64Value(1985), + }, + EqualPredicate{ + Column: columns[0], // first_name + Value: BinaryValue([]byte("Alice")), + }, + }, + }) + defer r.Close() + + ctx, _ := xcap.NewCapture(context.Background(), nil) + _, err := readDatasetWithContext(ctx, r, 3) + require.NoError(t, err) + + require.NotNil(t, r.region, "region should be available after reading") + + observations := r.region.Observations() + obsMap := make(map[string]int64) + for _, obs := range observations { + obsMap[obs.Statistic.Name()] = obs.Value.(int64) + } + + require.Equal(t, int64(2), obsMap[StatReadCalls.Name()]) + require.Equal(t, int64(2), obsMap[StatPrimaryColumns.Name()]) + require.Equal(t, int64(2), obsMap[StatSecondaryColumns.Name()]) + require.Equal(t, int64(5), obsMap[StatPrimaryColumnPages.Name()]) + require.Equal(t, int64(8), obsMap[StatSecondaryColumnPages.Name()]) + + require.Equal(t, int64(len(basicReaderTestData)), obsMap[StatMaxRows.Name()]) + require.Equal(t, int64(3), obsMap[StatRowsAfterPruning.Name()]) + require.Equal(t, int64(3), obsMap[StatPrimaryRowsRead.Name()]) + require.Equal(t, int64(1), obsMap[StatSecondaryRowsRead.Name()]) +} diff --git a/pkg/dataobj/sections/logs/reader.go b/pkg/dataobj/sections/logs/reader.go index b19cdb6f9b0ee..5c76e3f910793 100644 --- a/pkg/dataobj/sections/logs/reader.go +++ b/pkg/dataobj/sections/logs/reader.go @@ -213,7 +213,7 @@ func (r *Reader) Read(ctx context.Context, batchSize int) (arrow.RecordBatch, er return builder.NewRecordBatch(), readErr } -func (r *Reader) init(ctx context.Context) error { +func (r *Reader) init(_ context.Context) error { if err := r.opts.Validate(); err != nil { return fmt.Errorf("invalid options: %w", err) } else if r.opts.Allocator == nil { diff --git a/pkg/dataobj/sections/logs/reader_test.go b/pkg/dataobj/sections/logs/reader_test.go index 2921fc818fd80..ed99f9df45332 100644 --- a/pkg/dataobj/sections/logs/reader_test.go +++ b/pkg/dataobj/sections/logs/reader_test.go @@ -16,10 +16,8 @@ import ( "github.com/stretchr/testify/require" "github.com/grafana/loki/v3/pkg/dataobj" - "github.com/grafana/loki/v3/pkg/dataobj/internal/dataset" "github.com/grafana/loki/v3/pkg/dataobj/sections/logs" "github.com/grafana/loki/v3/pkg/util/arrowtest" - "github.com/grafana/loki/v3/pkg/xcap" ) // TestReader does a basic end-to-end test over a reader with a predicate applied. @@ -161,79 +159,3 @@ func readTable(ctx context.Context, r *logs.Reader) (arrow.Table, error) { return array.NewTableFromRecords(recs[0].Schema(), recs), nil } - -// TestReaderStats tests that the reader properly tracks statistics via xcap regions. -func TestReaderStats(t *testing.T) { - sec := buildSection(t, []logs.Record{ - {StreamID: 2, Timestamp: unixTime(40), Metadata: labels.FromStrings("trace_id", "789012"), Line: []byte("baz qux")}, - {StreamID: 2, Timestamp: unixTime(30), Metadata: labels.FromStrings("trace_id", "123456"), Line: []byte("foo bar")}, - {StreamID: 1, Timestamp: unixTime(20), Metadata: labels.FromStrings("trace_id", "abcdef"), Line: []byte("goodbye, world!")}, - {StreamID: 1, Timestamp: unixTime(10), Metadata: labels.EmptyLabels(), Line: []byte("hello, world!")}, - }) - - var ( - streamID = sec.Columns()[0] - traceID = sec.Columns()[2] - message = sec.Columns()[3] - ) - - // Create a reader with predicates - r := logs.NewReader(logs.ReaderOptions{ - Columns: []*logs.Column{streamID, traceID, message}, - Allocator: memory.DefaultAllocator, - Predicates: []logs.Predicate{ - logs.FuncPredicate{ - Column: traceID, - Keep: func(_ *logs.Column, value scalar.Scalar) bool { - if !value.IsValid() { - return false - } - - bb := value.(*scalar.String).Value.Bytes() - return bytes.Equal(bb, []byte("abcdef")) || bytes.Equal(bb, []byte("123456")) - }, - }, - logs.InPredicate{ - Column: streamID, - Values: []scalar.Scalar{ - scalar.NewInt64Scalar(1), - scalar.NewInt64Scalar(2), - }, - }, - }, - }) - - // Create xcap capture and region - ctx, _ := xcap.NewCapture(context.Background(), nil) - ctx, region := xcap.StartRegion(ctx, "test-reader") - - // Read the data - _, err := readTable(ctx, r) - require.NoError(t, err) - - region.End() - - // Get observations from the region - observations := region.Observations() - obsMap := make(map[string]int64) - for _, obs := range observations { - obsMap[obs.Statistic.Name()] = obs.Value.(int64) - } - - // Verify the stats are properly populated - require.Equal(t, int64(2), obsMap[dataset.StatReadCalls.Name()]) - require.Equal(t, int64(2), obsMap[dataset.StatPrimaryColumns.Name()]) // from 2 predicates - require.Equal(t, int64(1), obsMap[dataset.StatSecondaryColumns.Name()]) // 1 secondary column - require.Equal(t, int64(2), obsMap[dataset.StatPrimaryColumnPages.Name()]) - require.Equal(t, int64(1), obsMap[dataset.StatSecondaryColumnPages.Name()]) - - require.Equal(t, int64(4), obsMap[dataset.StatTotalRowsAvailable.Name()]) - require.Equal(t, int64(4), obsMap[dataset.StatRowsAfterPruning.Name()]) - require.Equal(t, int64(4), obsMap[dataset.StatPrimaryRowsRead.Name()]) - require.Equal(t, int64(2), obsMap[dataset.StatSecondaryRowsRead.Name()]) // 2 rows pass the predicate - - // Verify download stats - these should be populated by the downloader - require.Equal(t, int64(3), obsMap[dataset.StatPagesScanned.Name()]) // one page per column - require.Equal(t, int64(2), obsMap[dataset.StatPrimaryColumnPagesDownloaded.Name()]) - require.Equal(t, int64(1), obsMap[dataset.StatSecondaryColumnPagesDownloaded.Name()]) -} diff --git a/pkg/xcap/region.go b/pkg/xcap/region.go index ced604c9dcfa0..9f51555de579f 100644 --- a/pkg/xcap/region.go +++ b/pkg/xcap/region.go @@ -211,6 +211,19 @@ func (r *Region) RecordError(err error) { r.status = Status{Code: codes.Error, Message: err.Error()} } +// Observations returns all aggregated observations recorded in the region. +func (r *Region) Observations() []AggregatedObservation { + r.mu.RLock() + defer r.mu.RUnlock() + + observations := make([]AggregatedObservation, 0, len(r.observations)) + for _, agg := range r.observations { + observations = append(observations, *agg) + } + + return observations +} + // End completes the Region. Updates to the Region are ignored after calling End. func (r *Region) End() { if r == nil { From 601dfcc9d5b4073f2677d6c2cb6c7bdf31d17d35 Mon Sep 17 00:00:00 2001 From: Ashwanth Goli Date: Wed, 3 Dec 2025 15:09:16 +0530 Subject: [PATCH 05/13] use dots in stat name --- pkg/dataobj/internal/dataset/read_stats.go | 42 +++++++++---------- .../internal/dataset/reader_downloader.go | 6 +-- 2 files changed, 24 insertions(+), 24 deletions(-) diff --git a/pkg/dataobj/internal/dataset/read_stats.go b/pkg/dataobj/internal/dataset/read_stats.go index 9d6c5cef17c84..532a2e6436708 100644 --- a/pkg/dataobj/internal/dataset/read_stats.go +++ b/pkg/dataobj/internal/dataset/read_stats.go @@ -7,35 +7,35 @@ import ( // xcap statistics for dataset reader operations. var ( // Column statistics - StatPrimaryColumns = xcap.NewStatisticInt64("primary_columns", xcap.AggregationTypeSum) - StatSecondaryColumns = xcap.NewStatisticInt64("secondary_columns", xcap.AggregationTypeSum) + StatPrimaryColumns = xcap.NewStatisticInt64("primary.columns", xcap.AggregationTypeSum) + StatSecondaryColumns = xcap.NewStatisticInt64("secondary.columns", xcap.AggregationTypeSum) // Page statistics - StatPrimaryColumnPages = xcap.NewStatisticInt64("primary_column_pages", xcap.AggregationTypeSum) - StatSecondaryColumnPages = xcap.NewStatisticInt64("secondary_column_pages", xcap.AggregationTypeSum) + StatPrimaryColumnPages = xcap.NewStatisticInt64("primary.column.pages", xcap.AggregationTypeSum) + StatSecondaryColumnPages = xcap.NewStatisticInt64("secondary.column.pages", xcap.AggregationTypeSum) // Row statistics - StatMaxRows = xcap.NewStatisticInt64("max_rows", xcap.AggregationTypeSum) - StatRowsAfterPruning = xcap.NewStatisticInt64("rows_after_pruning", xcap.AggregationTypeSum) - StatPrimaryRowsRead = xcap.NewStatisticInt64("primary_rows_read", xcap.AggregationTypeSum) - StatSecondaryRowsRead = xcap.NewStatisticInt64("secondary_rows_read", xcap.AggregationTypeSum) - StatPrimaryRowBytes = xcap.NewStatisticInt64("primary_row_bytes_read", xcap.AggregationTypeSum) - StatSecondaryRowBytes = xcap.NewStatisticInt64("secondary_row_bytes_read", xcap.AggregationTypeSum) + StatMaxRows = xcap.NewStatisticInt64("row.max", xcap.AggregationTypeSum) + StatRowsAfterPruning = xcap.NewStatisticInt64("rows.after.pruning", xcap.AggregationTypeSum) + StatPrimaryRowsRead = xcap.NewStatisticInt64("primary.rows.read", xcap.AggregationTypeSum) + StatSecondaryRowsRead = xcap.NewStatisticInt64("secondary.rows.read", xcap.AggregationTypeSum) + StatPrimaryRowBytes = xcap.NewStatisticInt64("primary.row.read.bytes", xcap.AggregationTypeSum) + StatSecondaryRowBytes = xcap.NewStatisticInt64("secondary.row.read.bytes", xcap.AggregationTypeSum) // Download/Page scan statistics - StatPagesScanned = xcap.NewStatisticInt64("pages_scanned", xcap.AggregationTypeSum) - StatPagesFoundInCache = xcap.NewStatisticInt64("pages_found_in_cache", xcap.AggregationTypeSum) - StatBatchDownloadRequests = xcap.NewStatisticInt64("batch_download_requests", xcap.AggregationTypeSum) - StatPageDownloadTime = xcap.NewStatisticInt64("page_download_time_ns", xcap.AggregationTypeSum) + StatPagesScanned = xcap.NewStatisticInt64("pages.scanned", xcap.AggregationTypeSum) + StatPagesFoundInCache = xcap.NewStatisticInt64("pages.cache.hit", xcap.AggregationTypeSum) + StatPageDownloadRequests = xcap.NewStatisticInt64("pages.download.requests", xcap.AggregationTypeSum) + StatPageDownloadTime = xcap.NewStatisticInt64("pages.download.duration.ns", xcap.AggregationTypeSum) // Page download byte statistics - StatPrimaryColumnPagesDownloaded = xcap.NewStatisticInt64("primary_column_pages_downloaded", xcap.AggregationTypeSum) - StatSecondaryColumnPagesDownloaded = xcap.NewStatisticInt64("secondary_column_pages_downloaded", xcap.AggregationTypeSum) - StatPrimaryColumnBytes = xcap.NewStatisticInt64("primary_column_bytes", xcap.AggregationTypeSum) - StatSecondaryColumnBytes = xcap.NewStatisticInt64("secondary_column_bytes", xcap.AggregationTypeSum) - StatPrimaryColumnUncompressedBytes = xcap.NewStatisticInt64("primary_column_uncompressed_bytes", xcap.AggregationTypeSum) - StatSecondaryColumnUncompressedBytes = xcap.NewStatisticInt64("secondary_column_uncompressed_bytes", xcap.AggregationTypeSum) + StatPrimaryPagesDownloaded = xcap.NewStatisticInt64("primary.pages.downloaded", xcap.AggregationTypeSum) + StatSecondaryPagesDownloaded = xcap.NewStatisticInt64("secondary.pages.downloaded", xcap.AggregationTypeSum) + StatPrimaryColumnBytes = xcap.NewStatisticInt64("primary.pages.compressed.bytes", xcap.AggregationTypeSum) + StatSecondaryColumnBytes = xcap.NewStatisticInt64("secondary.pages.compressed.bytes", xcap.AggregationTypeSum) + StatPrimaryColumnUncompressedBytes = xcap.NewStatisticInt64("primary.pages.uncompressed.bytes", xcap.AggregationTypeSum) + StatSecondaryColumnUncompressedBytes = xcap.NewStatisticInt64("secondary.pages.uncompressed.bytes", xcap.AggregationTypeSum) // Read operation statistics - StatReadCalls = xcap.NewStatisticInt64("read_calls", xcap.AggregationTypeSum) + StatReadCalls = xcap.NewStatisticInt64("read.calls", xcap.AggregationTypeSum) ) diff --git a/pkg/dataobj/internal/dataset/reader_downloader.go b/pkg/dataobj/internal/dataset/reader_downloader.go index b990181116299..e2d870932a43c 100644 --- a/pkg/dataobj/internal/dataset/reader_downloader.go +++ b/pkg/dataobj/internal/dataset/reader_downloader.go @@ -232,11 +232,11 @@ func (dl *readerDownloader) downloadBatch(ctx context.Context, requestor *reader if region := xcap.RegionFromContext(ctx); region != nil { for _, page := range batch { if page.column.primary { - region.Record(StatPrimaryColumnPagesDownloaded.Observe(1)) + region.Record(StatPrimaryPagesDownloaded.Observe(1)) region.Record(StatPrimaryColumnBytes.Observe(int64(page.inner.PageDesc().CompressedSize))) region.Record(StatPrimaryColumnUncompressedBytes.Observe(int64(page.inner.PageDesc().UncompressedSize))) } else { - region.Record(StatSecondaryColumnPagesDownloaded.Observe(1)) + region.Record(StatSecondaryPagesDownloaded.Observe(1)) region.Record(StatSecondaryColumnBytes.Observe(int64(page.inner.PageDesc().CompressedSize))) region.Record(StatSecondaryColumnUncompressedBytes.Observe(int64(page.inner.PageDesc().UncompressedSize))) } @@ -595,7 +595,7 @@ func (page *readerPage) ReadPage(ctx context.Context) (PageData, error) { return page.data, nil } - region.Record(StatBatchDownloadRequests.Observe(1)) + region.Record(StatPageDownloadRequests.Observe(1)) if err := page.column.dl.downloadBatch(ctx, page); err != nil { return nil, err } From 2d361250a916ac9d11762642749db920af771df3 Mon Sep 17 00:00:00 2001 From: Ashwanth Goli Date: Wed, 3 Dec 2025 16:16:46 +0530 Subject: [PATCH 06/13] add log exporter --- pkg/engine/engine.go | 2 + pkg/xcap/aggregation.go | 25 +++-- pkg/xcap/exporter.go | 70 ++++++++++++ pkg/xcap/summary.go | 231 +++++++++++++++++++++++++++++++++++++++ pkg/xcap/summary_test.go | 122 +++++++++++++++++++++ 5 files changed, 440 insertions(+), 10 deletions(-) create mode 100644 pkg/xcap/summary.go create mode 100644 pkg/xcap/summary_test.go diff --git a/pkg/engine/engine.go b/pkg/engine/engine.go index f1744f30208a9..b2eb5ba2221bb 100644 --- a/pkg/engine/engine.go +++ b/pkg/engine/engine.go @@ -467,5 +467,7 @@ func exportCapture(ctx context.Context, capture *xcap.Capture, plan *physical.Pl return parentID, ok }) + xcap.ExportLog(capture, logger) + return xcap.ExportTrace(ctx, capture, logger) } diff --git a/pkg/xcap/aggregation.go b/pkg/xcap/aggregation.go index fab8b30cbafca..a6ccb2e9a6899 100644 --- a/pkg/xcap/aggregation.go +++ b/pkg/xcap/aggregation.go @@ -10,10 +10,21 @@ type AggregatedObservation struct { // Record aggregates a new observation into this aggregated observation. // It updates the value according to the statistic's aggregation type. func (a *AggregatedObservation) Record(obs Observation) { - stat := obs.statistic() - val := obs.value() + a.aggregate(obs.statistic().Aggregation(), obs.value()) + a.Count++ +} + +// Merge aggregates another AggregatedObservation into this one. +func (a *AggregatedObservation) Merge(other *AggregatedObservation) { + if other == nil { + return + } + a.aggregate(a.Statistic.Aggregation(), other.Value) + a.Count += other.Count +} - switch stat.Aggregation() { +func (a *AggregatedObservation) aggregate(aggType AggregationType, val any) { + switch aggType { case AggregationTypeSum: switch v := val.(type) { case int64: @@ -21,7 +32,6 @@ func (a *AggregatedObservation) Record(obs Observation) { case float64: a.Value = a.Value.(float64) + v } - case AggregationTypeMin: switch v := val.(type) { case int64: @@ -33,7 +43,6 @@ func (a *AggregatedObservation) Record(obs Observation) { a.Value = v } } - case AggregationTypeMax: switch v := val.(type) { case int64: @@ -50,16 +59,12 @@ func (a *AggregatedObservation) Record(obs Observation) { a.Value = v } } - case AggregationTypeLast: - // Last value overwrites a.Value = val - case AggregationTypeFirst: + // Keep the first value, don't update if a.Value == nil { a.Value = val } } - - a.Count++ } diff --git a/pkg/xcap/exporter.go b/pkg/xcap/exporter.go index 523c3ed568dbd..10fdd3f2bd283 100644 --- a/pkg/xcap/exporter.go +++ b/pkg/xcap/exporter.go @@ -115,3 +115,73 @@ func observationToAttribute(key StatisticKey, obs *AggregatedObservation) attrib // Fallback: convert to string return attrKey.String(fmt.Sprintf("%v", obs.Value)) } + +// ExportLog exports a Capture as a structured log line with aggregated statistics. +func ExportLog(capture *Capture, logger log.Logger) { + if capture == nil || logger == nil { + return + } + + summary := summarizeObservations(capture) + level.Info(logger).Log(summary.toLogValues()...) +} + +// summarizeObservations collects and summarizes observations from the capture. +func summarizeObservations(capture *Capture) *observations { + if capture == nil { + return nil + } + + collect := newObservationCollector(capture) + result := newObservations() + + // collect observations from all DataObjScan regions. observations from + // child regions are rolled-up to include dataset reader and bucket stats. + // streamView is excluded as it is handled separately below. + result.merge( + collect.fromRegions("DataObjScan", true, "streamsView.init"). + filter( + // object store calls + "bucket.get", "bucket.getrange", "bucket.attributes", + // dataset reader stats + "row.max", "rows.after.pruning", "read.calls", + "primary.pages.downloaded", "secondary.pages.downloaded", + "primary.pages.compressed.bytes", "secondary.pages.compressed.bytes", + "primary.rows.read", "secondary.rows.read", + "primary.row.read.bytes", "secondary.row.read.bytes", + "pages.scanned", "pages.cache.hit", + "pages.download.requests", "pages.download.duration.ns", + ). + prefix("logs_dataset_"). + normalizeKeys(), + ) + + // metastore index and resolved section stats + result.merge( + collect.fromRegions("ObjectMetastore.Sections", true). + filter("metastore.index.objects", "metastore.resolved.sections"). + normalizeKeys(), + ) + + // metastore bucket and dataset reader stats + result.merge( + collect.fromRegions("ObjectMetastore.Sections", true). + filter("bucket.get", "bucket.getrange", "bucket.attributes", + "primary.pages.downloaded", "secondary.pages.downloaded", + "primary.pages.compressed.bytes", "secondary.pages.compressed.bytes"). + prefix("metastore_"). + normalizeKeys(), + ) + + // streamsView bucket and dataset reader stats + result.merge( + collect.fromRegions("streamsView.init", true). + filter("bucket.get", "bucket.getrange", "bucket.attributes", + "primary.pages.downloaded", "secondary.pages.downloaded", + "primary.pages.compressed.bytes", "secondary.pages.compressed.bytes"). + prefix("streams_"). + normalizeKeys(), + ) + + return result +} diff --git a/pkg/xcap/summary.go b/pkg/xcap/summary.go new file mode 100644 index 0000000000000..7fbf6316e1304 --- /dev/null +++ b/pkg/xcap/summary.go @@ -0,0 +1,231 @@ +package xcap + +import ( + "sort" + "strings" + "time" + + "github.com/dustin/go-humanize" +) + +// observations holds aggregated observations that can be transformed and merged. +// +// All transformation methods (filter, prefix, normalizeKeys) return new instances, +// leaving the original unchanged. +type observations struct { + data map[StatisticKey]*AggregatedObservation +} + +// newObservations creates an empty observations. +func newObservations() *observations { + return &observations{data: make(map[StatisticKey]*AggregatedObservation)} +} + +// filter returns a new observations containing only entries with matching stat names. +func (o *observations) filter(names ...string) *observations { + if len(names) == 0 || o == nil { + return o + } + + nameSet := make(map[string]struct{}, len(names)) + for _, n := range names { + nameSet[n] = struct{}{} + } + + result := newObservations() + for k, obs := range o.data { + if _, ok := nameSet[k.Name]; ok { + result.data[k] = obs + } + } + return result +} + +// prefix returns a new observations with all stat names prefixed. +func (o *observations) prefix(p string) *observations { + if p == "" || o == nil { + return o + } + + result := newObservations() + for k, obs := range o.data { + newKey := StatisticKey{ + Name: p + k.Name, + DataType: k.DataType, + Aggregation: k.Aggregation, + } + result.data[newKey] = obs + } + return result +} + +// normalizeKeys returns a new observations with dots replaced by underscores in stat names. +func (o *observations) normalizeKeys() *observations { + if o == nil { + return o + } + + result := newObservations() + for k, obs := range o.data { + newKey := StatisticKey{ + Name: strings.ReplaceAll(k.Name, ".", "_"), + DataType: k.DataType, + Aggregation: k.Aggregation, + } + result.data[newKey] = obs + } + return result +} + +// merge merges another observations into this one. +func (o *observations) merge(other *observations) { + if other == nil { + return + } + for k, obs := range other.data { + if existing, ok := o.data[k]; ok { + existing.Merge(obs) + } else { + o.data[k] = &AggregatedObservation{ + Statistic: obs.Statistic, + Value: obs.Value, + Count: obs.Count, + } + } + } +} + +// ToLogValues converts observations to a slice suitable for go-kit/log. +// Keys are sorted for deterministic output. +func (o *observations) toLogValues() []any { + if o == nil { + return nil + } + + // Collect key-value pairs for sorting by name. + type kv struct { + name string + value any + } + pairs := make([]kv, 0, len(o.data)) + for k, obs := range o.data { + pairs = append(pairs, kv{name: k.Name, value: obs.Value}) + } + sort.Slice(pairs, func(i, j int) bool { + return strings.Compare(pairs[i].name, pairs[j].name) < 0 + }) + + result := make([]any, 0, len(pairs)*2) + for _, p := range pairs { + value := p.value + + // Format bytes values (keys ending with "_bytes") + if strings.HasSuffix(p.name, "_bytes") { + switch val := value.(type) { + case uint64: + value = humanize.Bytes(val) + case int64: + value = humanize.Bytes(uint64(val)) + } + } + + // Format duration values (keys ending with "duration_ns") + if strings.HasSuffix(p.name, "duration_ns") { + switch val := value.(type) { + case int64: + value = time.Duration(val).String() + case uint64: + value = time.Duration(val).String() + } + } + + result = append(result, p.name, value) + } + return result +} + +// observationCollector provides methods to collect observations from a Capture. +type observationCollector struct { + capture *Capture + childrenMap map[identifier][]*Region +} + +// newObservationCollector creates a new collector for gathering observations from the given capture. +func newObservationCollector(capture *Capture) *observationCollector { + if capture == nil { + return nil + } + + // Build parent -> children map + childrenMap := make(map[identifier][]*Region) + for _, r := range capture.Regions() { + childrenMap[r.parentID] = append(childrenMap[r.parentID], r) + } + + return &observationCollector{ + capture: capture, + childrenMap: childrenMap, + } +} + +// fromRegions collects observations from regions with the given name. +// If rollUp is true, each region's stats include all its descendant stats +// aggregated according to each stat's aggregation type. +func (c *observationCollector) fromRegions(name string, rollUp bool, excluded ...string) *observations { + if c == nil { + return newObservations() + } + + excludedSet := make(map[string]struct{}, len(excluded)) + for _, name := range excluded { + excludedSet[name] = struct{}{} + } + + result := newObservations() + for _, region := range c.capture.Regions() { + if region.name != name { + continue + } + + var obs *observations + if rollUp { + obs = c.rollUpObservations(region, excludedSet) + } else { + obs = c.getRegionObservations(region) + } + + result.merge(obs) + } + + return result +} + +// getRegionObservations returns a copy of a region's observations. +func (c *observationCollector) getRegionObservations(region *Region) *observations { + result := newObservations() + for k, obs := range region.observations { + result.data[k] = &AggregatedObservation{ + Statistic: obs.Statistic, + Value: obs.Value, + Count: obs.Count, + } + } + return result +} + +// rollUpObservations computes observations for a region including all its descendants. +// Stats are aggregated according to their aggregation type. +func (c *observationCollector) rollUpObservations(region *Region, excludedSet map[string]struct{}) *observations { + result := c.getRegionObservations(region) + + // Recursively aggregate from children. + for _, child := range c.childrenMap[region.id] { + // Skip children with excluded names. + if _, excluded := excludedSet[child.name]; excluded { + continue + } + result.merge(c.rollUpObservations(child, excludedSet)) + } + + return result +} diff --git a/pkg/xcap/summary_test.go b/pkg/xcap/summary_test.go new file mode 100644 index 0000000000000..1ad0dd19f50c8 --- /dev/null +++ b/pkg/xcap/summary_test.go @@ -0,0 +1,122 @@ +package xcap + +import ( + "context" + "testing" + + "github.com/stretchr/testify/require" +) + +func TestObservations(t *testing.T) { + ctx, capture := NewCapture(context.Background(), nil) + + statA := NewStatisticInt64("stat.one", AggregationTypeSum) + statB := NewStatisticInt64("stat.two", AggregationTypeSum) + statC := NewStatisticInt64("stat.three", AggregationTypeSum) + + _, region := StartRegion(ctx, "Test") + region.Record(statA.Observe(10)) + region.Record(statB.Observe(20)) + region.Record(statC.Observe(30)) + region.End() + + collector := newObservationCollector(capture) + obs := collector.fromRegions("Test", false) + + t.Run("filter", func(t *testing.T) { + filtered := obs.filter("stat.one", "stat.two") + require.Len(t, filtered.data, 2) + require.Equal(t, int64(10), filtered.data[statA.Key()].Value) + require.Equal(t, int64(20), filtered.data[statB.Key()].Value) + }) + + t.Run("prefix", func(t *testing.T) { + prefixed := obs.filter("stat.one").prefix("metastore_") + expectedKey := StatisticKey{Name: "metastore_stat.one", DataType: DataTypeInt64, Aggregation: AggregationTypeSum} + require.Len(t, prefixed.data, 1) + require.Equal(t, int64(10), prefixed.data[expectedKey].Value) + }) + + t.Run("normalizeKeys", func(t *testing.T) { + normalized := obs.filter("stat.three").normalizeKeys() + expectedKey := StatisticKey{Name: "stat_three", DataType: DataTypeInt64, Aggregation: AggregationTypeSum} + require.Len(t, normalized.data, 1) + require.Equal(t, int64(30), normalized.data[expectedKey].Value) + }) + + t.Run("merge", func(t *testing.T) { + target := newObservations() + target.merge(obs.filter("stat.one", "stat.two")) + target.merge(obs.filter("stat.two", "stat.three")) + + require.Len(t, target.data, 3) + require.Equal(t, int64(10), target.data[statA.Key()].Value) + require.Equal(t, int64(40), target.data[statB.Key()].Value) + require.Equal(t, int64(30), target.data[statC.Key()].Value) + }) + + t.Run("toLogValues", func(t *testing.T) { + logValues := obs.toLogValues() + // Sorted: stat.one, stat.two, stat.three + require.Equal(t, []any{"stat.one", int64(10), "stat.three", int64(30), "stat.two", int64(20)}, logValues) + }) + + t.Run("chaining", func(t *testing.T) { + result := obs.filter("stat.three").prefix("logs_").normalizeKeys().toLogValues() + require.Equal(t, []any{"logs_stat_three", int64(30)}, result) + }) +} + +func TestRollups(t *testing.T) { + t.Run("includes child observations when rollup=true", func(t *testing.T) { + ctx, capture := NewCapture(context.Background(), nil) + stat := NewStatisticInt64("count", AggregationTypeSum) + + ctx, parent := StartRegion(ctx, "Parent") + parent.Record(stat.Observe(10)) + + ctx, child := StartRegion(ctx, "Child") + child.Record(stat.Observe(20)) + + _, grandchild := StartRegion(ctx, "Grandchild") + grandchild.Record(stat.Observe(30)) + + grandchild.End() + child.End() + parent.End() + + collector := newObservationCollector(capture) + + withRollup := collector.fromRegions("Parent", true) + require.Equal(t, int64(60), withRollup.data[stat.Key()].Value) // 10 + 20 + 30 + + withoutRollup := collector.fromRegions("Parent", false) + require.Equal(t, int64(10), withoutRollup.data[stat.Key()].Value) // parent only + }) + + t.Run("excludes matching descendants", func(t *testing.T) { + ctx, capture := NewCapture(context.Background(), nil) + stat := NewStatisticInt64("count", AggregationTypeSum) + + ctx, parent := StartRegion(ctx, "Parent") + parent.Record(stat.Observe(1)) + + _, included := StartRegion(ctx, "included") + included.Record(stat.Observe(10)) + included.End() + + ctx2, excluded := StartRegion(ctx, "excluded") + excluded.Record(stat.Observe(100)) + + _, excludedChild := StartRegion(ctx2, "excludedChild") + excludedChild.Record(stat.Observe(1000)) + excludedChild.End() + + excluded.End() + parent.End() + + collector := newObservationCollector(capture) + stats := collector.fromRegions("Parent", true, "excluded") + require.Equal(t, int64(11), stats.data[stat.Key()].Value) // 1 + 10, excludes 100 + 1000 + }) +} From 5b796891a3a41385f2fff3a983628e128ec633d2 Mon Sep 17 00:00:00 2001 From: Ashwanth Goli Date: Thu, 4 Dec 2025 13:04:43 +0530 Subject: [PATCH 07/13] set result.Summary from capture --- pkg/engine/engine.go | 1 + pkg/xcap/aggregation.go | 24 +++++ pkg/xcap/stats_summary.go | 81 +++++++++++++++ pkg/xcap/stats_summary_test.go | 176 +++++++++++++++++++++++++++++++++ pkg/xcap/summary.go | 62 ++++++++++++ 5 files changed, 344 insertions(+) create mode 100644 pkg/xcap/stats_summary.go create mode 100644 pkg/xcap/stats_summary_test.go diff --git a/pkg/engine/engine.go b/pkg/engine/engine.go index b2eb5ba2221bb..095dc64f81d3b 100644 --- a/pkg/engine/engine.go +++ b/pkg/engine/engine.go @@ -237,6 +237,7 @@ func (e *Engine) Execute(ctx context.Context, params logql.Params) (logqlmodel.R md := metadata.FromContext(ctx) region.SetStatus(codes.Ok, "") + stats.Summary = capture.ToStatsSummary(durFull, 0, builder.Len()) result := builder.Build(stats, md) logql.RecordRangeAndInstantQueryMetrics(ctx, logger, params, strconv.Itoa(http.StatusOK), stats, result.Data) diff --git a/pkg/xcap/aggregation.go b/pkg/xcap/aggregation.go index a6ccb2e9a6899..31826762e2d2f 100644 --- a/pkg/xcap/aggregation.go +++ b/pkg/xcap/aggregation.go @@ -68,3 +68,27 @@ func (a *AggregatedObservation) aggregate(aggType AggregationType, val any) { } } } + +func (a *AggregatedObservation) Int64() (int64, bool) { + if a.Statistic.DataType() != DataTypeInt64 { + return 0, false + } + + return a.Value.(int64), true +} + +func (a *AggregatedObservation) Float64() (float64, bool) { + if a.Statistic.DataType() != DataTypeFloat64 { + return 0, false + } + + return a.Value.(float64), true +} + +func (a *AggregatedObservation) Bool() bool { + if a.Statistic.DataType() != DataTypeBool { + return false + } + + return a.Value.(bool) +} diff --git a/pkg/xcap/stats_summary.go b/pkg/xcap/stats_summary.go new file mode 100644 index 0000000000000..3cd7e0e215580 --- /dev/null +++ b/pkg/xcap/stats_summary.go @@ -0,0 +1,81 @@ +package xcap + +import ( + "time" + + "github.com/grafana/loki/v3/pkg/logqlmodel/stats" +) + +// Region name for data object scan operations. +const regionNameDataObjScan = "DataObjScan" + +// Statistic keys used for Summary computation. +// These must match the statistic names used in the executor package. +const ( + // Row statistics + statKeyRowsOut = "rows_out" + statKeyPrimaryRowsRead = "primary_rows_read" + + // Byte statistics + statKeyPrimaryColumnUncompressedBytes = "primary_column_uncompressed_bytes" + statKeySecondaryColumnUncompressedBytes = "secondary_column_uncompressed_bytes" +) + +// ToStatsSummary computes a stats.Summary from observations in the capture. +func (c *Capture) ToStatsSummary(execTime, queueTime time.Duration, totalEntriesReturned int) stats.Summary { + summary := stats.Summary{ + ExecTime: execTime.Seconds(), + QueueTime: queueTime.Seconds(), + TotalEntriesReturned: int64(totalEntriesReturned), + } + + if c == nil { + return summary + } + + // Collect observations from DataObjScan as the summary stats mainly relate to log lines. + // In practice, new engine would process more bytes while scanning metastore objects and stream sections. + collector := newObservationCollector(c) + obs := collector.fromRegions(regionNameDataObjScan, true).filter( + statKeyRowsOut, + statKeyPrimaryRowsRead, + statKeyPrimaryColumnUncompressedBytes, + statKeySecondaryColumnUncompressedBytes, + ) + + // TotalBytesProcessed: sum of uncompressed bytes from primary and secondary columns + summary.TotalBytesProcessed = getObservation(obs, statKeyPrimaryColumnUncompressedBytes) + + getObservation(obs, statKeySecondaryColumnUncompressedBytes) + + // TotalLinesProcessed: primary rows read + summary.TotalLinesProcessed = getObservation(obs, statKeyPrimaryRowsRead) + + // TotalPostFilterLines: rows output after filtering + // TODO: this will report the wrong value if the plan has a filter stage. + // pick the min of row_out from filter and scan nodes. + summary.TotalPostFilterLines = getObservation(obs, statKeyRowsOut) + + if execTime > 0 { + execSeconds := execTime.Seconds() + summary.BytesProcessedPerSecond = int64(float64(summary.TotalBytesProcessed) / execSeconds) + summary.LinesProcessedPerSecond = int64(float64(summary.TotalLinesProcessed) / execSeconds) + } + + return summary +} + +// getObservation retrieves an int64 value from observations by statistic name. +func getObservation(obs *observations, name string) int64 { + if obs == nil { + return 0 + } + + for key, agg := range obs.data { + if key.Name == name { + if v, ok := agg.Value.(int64); ok { + return v + } + } + } + return 0 +} diff --git a/pkg/xcap/stats_summary_test.go b/pkg/xcap/stats_summary_test.go new file mode 100644 index 0000000000000..c1b296611b04f --- /dev/null +++ b/pkg/xcap/stats_summary_test.go @@ -0,0 +1,176 @@ +package xcap + +import ( + "context" + "testing" + "time" + + "github.com/stretchr/testify/require" +) + +func TestSummaryFromCapture(t *testing.T) { + t.Run("nil capture returns empty summary with provided values", func(t *testing.T) { + execTime := 2 * time.Second + queueTime := 100 * time.Millisecond + entriesReturned := 42 + + summary := SummaryFromCapture(nil, execTime, queueTime, entriesReturned) + + require.Equal(t, execTime.Seconds(), summary.ExecTime) + require.Equal(t, queueTime.Seconds(), summary.QueueTime) + require.Equal(t, int64(entriesReturned), summary.TotalEntriesReturned) + require.Equal(t, int64(0), summary.TotalBytesProcessed) + require.Equal(t, int64(0), summary.TotalLinesProcessed) + require.Equal(t, int64(0), summary.TotalPostFilterLines) + }) + + t.Run("computes bytes and lines from DataObjScan regions", func(t *testing.T) { + ctx, capture := NewCapture(context.Background(), nil) + + // Define statistics matching the executor package names + primaryBytes := NewStatisticInt64("primary_column_uncompressed_bytes", AggregationTypeSum) + secondaryBytes := NewStatisticInt64("secondary_column_uncompressed_bytes", AggregationTypeSum) + primaryRows := NewStatisticInt64("primary_rows_read", AggregationTypeSum) + secondaryRows := NewStatisticInt64("secondary_rows_read", AggregationTypeSum) + rowsOut := NewStatisticInt64("rows_out", AggregationTypeSum) + + // Create DataObjScan regions with observations + _, region1 := StartRegion(ctx, "DataObjScan") + region1.Record(primaryBytes.Observe(1000)) + region1.Record(secondaryBytes.Observe(500)) + region1.Record(primaryRows.Observe(100)) + region1.Record(secondaryRows.Observe(50)) + region1.Record(rowsOut.Observe(80)) + region1.End() + + _, region2 := StartRegion(ctx, "DataObjScan") + region2.Record(primaryBytes.Observe(2000)) + region2.Record(secondaryBytes.Observe(1000)) + region2.Record(primaryRows.Observe(200)) + region2.Record(secondaryRows.Observe(100)) + region2.Record(rowsOut.Observe(150)) + region2.End() + + capture.End() + + execTime := 2 * time.Second + queueTime := 100 * time.Millisecond + entriesReturned := 42 + + summary := SummaryFromCapture(capture, execTime, queueTime, entriesReturned) + + require.Equal(t, execTime.Seconds(), summary.ExecTime) + require.Equal(t, queueTime.Seconds(), summary.QueueTime) + require.Equal(t, int64(entriesReturned), summary.TotalEntriesReturned) + + // TotalBytesProcessed = primary + secondary = (1000+2000) + (500+1000) = 4500 + require.Equal(t, int64(4500), summary.TotalBytesProcessed) + + // TotalLinesProcessed = primary + secondary = (100+200) + (50+100) = 450 + require.Equal(t, int64(450), summary.TotalLinesProcessed) + + // TotalPostFilterLines = rows_out = 80 + 150 = 230 + require.Equal(t, int64(230), summary.TotalPostFilterLines) + + // BytesProcessedPerSecond = 4500 / 2 = 2250 + require.Equal(t, int64(2250), summary.BytesProcessedPerSecond) + + // LinesProcessedPerSecond = 450 / 2 = 225 + require.Equal(t, int64(225), summary.LinesProcessedPerSecond) + }) + + t.Run("ignores non-DataObjScan regions", func(t *testing.T) { + ctx, capture := NewCapture(context.Background(), nil) + + primaryBytes := NewStatisticInt64("primary_column_uncompressed_bytes", AggregationTypeSum) + + // DataObjScan region - should be counted + _, scanRegion := StartRegion(ctx, "DataObjScan") + scanRegion.Record(primaryBytes.Observe(1000)) + scanRegion.End() + + // Other region - should be ignored + _, otherRegion := StartRegion(ctx, "OtherRegion") + otherRegion.Record(primaryBytes.Observe(5000)) + otherRegion.End() + + capture.End() + + summary := SummaryFromCapture(capture, time.Second, 0, 0) + + // Only DataObjScan region bytes counted + require.Equal(t, int64(1000), summary.TotalBytesProcessed) + }) + + t.Run("zero exec time results in zero per-second rates", func(t *testing.T) { + ctx, capture := NewCapture(context.Background(), nil) + + primaryBytes := NewStatisticInt64("primary_column_uncompressed_bytes", AggregationTypeSum) + _, region := StartRegion(ctx, "DataObjScan") + region.Record(primaryBytes.Observe(1000)) + region.End() + capture.End() + + summary := SummaryFromCapture(capture, 0, 0, 10) + + require.Equal(t, int64(1000), summary.TotalBytesProcessed) + require.Equal(t, int64(0), summary.BytesProcessedPerSecond) + require.Equal(t, int64(0), summary.LinesProcessedPerSecond) + }) + + t.Run("missing statistics result in zero values", func(t *testing.T) { + ctx, capture := NewCapture(context.Background(), nil) + + // Only record some statistics + primaryBytes := NewStatisticInt64("primary_column_uncompressed_bytes", AggregationTypeSum) + _, region := StartRegion(ctx, "DataObjScan") + region.Record(primaryBytes.Observe(1000)) + region.End() + capture.End() + + execTime := 1 * time.Second + summary := SummaryFromCapture(capture, execTime, 0, 0) + + // Only primary bytes recorded + require.Equal(t, int64(1000), summary.TotalBytesProcessed) + // No rows recorded + require.Equal(t, int64(0), summary.TotalLinesProcessed) + require.Equal(t, int64(0), summary.TotalPostFilterLines) + }) + + t.Run("rolls up child region observations into DataObjScan", func(t *testing.T) { + ctx, capture := NewCapture(context.Background(), nil) + + primaryBytes := NewStatisticInt64("primary_column_uncompressed_bytes", AggregationTypeSum) + + // Parent DataObjScan region + ctx, parent := StartRegion(ctx, "DataObjScan") + parent.Record(primaryBytes.Observe(500)) + + // Child region (should be rolled up into parent) + _, child := StartRegion(ctx, "child_operation") + child.Record(primaryBytes.Observe(300)) + child.End() + + parent.End() + capture.End() + + summary := SummaryFromCapture(capture, time.Second, 0, 0) + + // Child observations rolled up into DataObjScan: 500 + 300 = 800 + require.Equal(t, int64(800), summary.TotalBytesProcessed) + }) + + t.Run("empty capture returns zero stats", func(t *testing.T) { + _, capture := NewCapture(context.Background(), nil) + capture.End() + + summary := SummaryFromCapture(capture, time.Second, time.Millisecond, 10) + + require.Equal(t, float64(1), summary.ExecTime) + require.Equal(t, float64(0.001), summary.QueueTime) + require.Equal(t, int64(10), summary.TotalEntriesReturned) + require.Equal(t, int64(0), summary.TotalBytesProcessed) + require.Equal(t, int64(0), summary.TotalLinesProcessed) + }) +} diff --git a/pkg/xcap/summary.go b/pkg/xcap/summary.go index 7fbf6316e1304..6baefe19b9190 100644 --- a/pkg/xcap/summary.go +++ b/pkg/xcap/summary.go @@ -6,6 +6,7 @@ import ( "time" "github.com/dustin/go-humanize" + "github.com/grafana/loki/v3/pkg/logqlmodel/stats" ) // observations holds aggregated observations that can be transformed and merged. @@ -229,3 +230,64 @@ func (c *observationCollector) rollUpObservations(region *Region, excludedSet ma return result } + +// Region name for data object scan operations. +const regionNameDataObjScan = "DataObjScan" + +// ToStatsSummary computes a stats.Summary from observations in the capture. +func (c *Capture) ToStatsSummary(execTime, queueTime time.Duration, totalEntriesReturned int) stats.Summary { + summary := stats.Summary{ + ExecTime: execTime.Seconds(), + QueueTime: queueTime.Seconds(), + TotalEntriesReturned: int64(totalEntriesReturned), + } + + if c == nil { + return summary + } + + // Collect observations from DataObjScan as the summary stats mainly relate to log lines. + // In practice, new engine would process more bytes while scanning metastore objects and stream sections. + collector := newObservationCollector(c) + observations := collector.fromRegions(regionNameDataObjScan, true).filter( + StatRowsOut.Key(), + StatPrimaryRowsRead.Key(), + StatPrimaryColumnUncompressedBytes.Key(), + StatSecondaryColumnUncompressedBytes.Key(), + ) + + // TotalBytesProcessed: sum of uncompressed bytes from primary and secondary columns + summary.TotalBytesProcessed = readInt64(observations, StatPrimaryColumnUncompressedBytes.Key()) + + readInt64(observations, StatSecondaryColumnUncompressedBytes.Key()) + + // TotalLinesProcessed: primary rows read + summary.TotalLinesProcessed = readInt64(observations, StatPrimaryRowsRead.Key()) + + // TotalPostFilterLines: rows output after filtering + // TODO: this will report the wrong value if the plan has a filter stage. + // pick the min of row_out from filter and scan nodes. + summary.TotalPostFilterLines = readInt64(observations, StatRowsOut.Key()) + + if execTime > 0 { + execSeconds := execTime.Seconds() + summary.BytesProcessedPerSecond = int64(float64(summary.TotalBytesProcessed) / execSeconds) + summary.LinesProcessedPerSecond = int64(float64(summary.TotalLinesProcessed) / execSeconds) + } + + return summary +} + +// readInt64 reads an int64 observation for the given stat key. +func readInt64(o *observations, key StatisticKey) int64 { + if o == nil { + return 0 + } + + if agg, ok := o.data[key]; ok { + if v, ok := agg.Int64(); ok { + return v + } + } + + return 0 +} From a4703012778053628cfaa5b05dc6d20d3b81923e Mon Sep 17 00:00:00 2001 From: Ashwanth Goli Date: Thu, 4 Dec 2025 12:30:50 +0530 Subject: [PATCH 08/13] move stats defs to xcap pkg --- pkg/dataobj/internal/dataset/read_stats.go | 41 ---------- pkg/dataobj/internal/dataset/reader.go | 26 +++--- .../internal/dataset/reader_downloader.go | 18 ++--- pkg/dataobj/internal/dataset/reader_test.go | 20 ++--- pkg/dataobj/metastore/object.go | 9 +-- .../sections/internal/columnar/decoder.go | 4 +- pkg/engine/internal/executor/compat.go | 2 +- pkg/engine/internal/executor/pipeline.go | 6 +- pkg/engine/internal/executor/stats.go | 14 ---- pkg/storage/bucket/xcap_bucket.go | 18 ++--- pkg/util/rangeio/rangeio.go | 20 ++--- pkg/xcap/exporter.go | 34 ++++---- pkg/xcap/stats_definitions.go | 71 ++++++++++++++++ pkg/xcap/stats_summary.go | 81 ------------------- pkg/xcap/stats_summary_test.go | 72 +++++++---------- pkg/xcap/summary.go | 49 ++++++----- pkg/xcap/summary_test.go | 12 +-- 17 files changed, 204 insertions(+), 293 deletions(-) delete mode 100644 pkg/dataobj/internal/dataset/read_stats.go delete mode 100644 pkg/engine/internal/executor/stats.go create mode 100644 pkg/xcap/stats_definitions.go delete mode 100644 pkg/xcap/stats_summary.go diff --git a/pkg/dataobj/internal/dataset/read_stats.go b/pkg/dataobj/internal/dataset/read_stats.go deleted file mode 100644 index 532a2e6436708..0000000000000 --- a/pkg/dataobj/internal/dataset/read_stats.go +++ /dev/null @@ -1,41 +0,0 @@ -package dataset - -import ( - "github.com/grafana/loki/v3/pkg/xcap" -) - -// xcap statistics for dataset reader operations. -var ( - // Column statistics - StatPrimaryColumns = xcap.NewStatisticInt64("primary.columns", xcap.AggregationTypeSum) - StatSecondaryColumns = xcap.NewStatisticInt64("secondary.columns", xcap.AggregationTypeSum) - - // Page statistics - StatPrimaryColumnPages = xcap.NewStatisticInt64("primary.column.pages", xcap.AggregationTypeSum) - StatSecondaryColumnPages = xcap.NewStatisticInt64("secondary.column.pages", xcap.AggregationTypeSum) - - // Row statistics - StatMaxRows = xcap.NewStatisticInt64("row.max", xcap.AggregationTypeSum) - StatRowsAfterPruning = xcap.NewStatisticInt64("rows.after.pruning", xcap.AggregationTypeSum) - StatPrimaryRowsRead = xcap.NewStatisticInt64("primary.rows.read", xcap.AggregationTypeSum) - StatSecondaryRowsRead = xcap.NewStatisticInt64("secondary.rows.read", xcap.AggregationTypeSum) - StatPrimaryRowBytes = xcap.NewStatisticInt64("primary.row.read.bytes", xcap.AggregationTypeSum) - StatSecondaryRowBytes = xcap.NewStatisticInt64("secondary.row.read.bytes", xcap.AggregationTypeSum) - - // Download/Page scan statistics - StatPagesScanned = xcap.NewStatisticInt64("pages.scanned", xcap.AggregationTypeSum) - StatPagesFoundInCache = xcap.NewStatisticInt64("pages.cache.hit", xcap.AggregationTypeSum) - StatPageDownloadRequests = xcap.NewStatisticInt64("pages.download.requests", xcap.AggregationTypeSum) - StatPageDownloadTime = xcap.NewStatisticInt64("pages.download.duration.ns", xcap.AggregationTypeSum) - - // Page download byte statistics - StatPrimaryPagesDownloaded = xcap.NewStatisticInt64("primary.pages.downloaded", xcap.AggregationTypeSum) - StatSecondaryPagesDownloaded = xcap.NewStatisticInt64("secondary.pages.downloaded", xcap.AggregationTypeSum) - StatPrimaryColumnBytes = xcap.NewStatisticInt64("primary.pages.compressed.bytes", xcap.AggregationTypeSum) - StatSecondaryColumnBytes = xcap.NewStatisticInt64("secondary.pages.compressed.bytes", xcap.AggregationTypeSum) - StatPrimaryColumnUncompressedBytes = xcap.NewStatisticInt64("primary.pages.uncompressed.bytes", xcap.AggregationTypeSum) - StatSecondaryColumnUncompressedBytes = xcap.NewStatisticInt64("secondary.pages.uncompressed.bytes", xcap.AggregationTypeSum) - - // Read operation statistics - StatReadCalls = xcap.NewStatisticInt64("read.calls", xcap.AggregationTypeSum) -) diff --git a/pkg/dataobj/internal/dataset/reader.go b/pkg/dataobj/internal/dataset/reader.go index ea2ae3dfd567a..7a47ea2939f59 100644 --- a/pkg/dataobj/internal/dataset/reader.go +++ b/pkg/dataobj/internal/dataset/reader.go @@ -75,7 +75,7 @@ func (r *Reader) Read(ctx context.Context, s []Row) (int, error) { } } - r.region.Record(StatReadCalls.Observe(1)) + r.region.Record(xcap.StatDatasetReadCalls.Observe(1)) ctx = xcap.ContextWithRegion(ctx, r.region) // Our Read implementation works by: @@ -149,8 +149,8 @@ func (r *Reader) Read(ctx context.Context, s []Row) (int, error) { primaryColumnBytes += s[i].Size() } - r.region.Record(StatPrimaryRowsRead.Observe(int64(rowsRead))) - r.region.Record(StatPrimaryRowBytes.Observe(primaryColumnBytes)) + r.region.Record(xcap.StatPrimaryRowsRead.Observe(int64(rowsRead))) + r.region.Record(xcap.StatPrimaryRowBytes.Observe(primaryColumnBytes)) } else { rowsRead, passCount, err = r.readAndFilterPrimaryColumns(ctx, readSize, s[:readSize]) if err != nil { @@ -182,8 +182,8 @@ func (r *Reader) Read(ctx context.Context, s []Row) (int, error) { totalBytesFilled += s[i].Size() - s[i].SizeOfColumns(r.primaryColumnIndexes) } - r.region.Record(StatSecondaryRowsRead.Observe(int64(count))) - r.region.Record(StatSecondaryRowBytes.Observe(totalBytesFilled)) + r.region.Record(xcap.StatSecondaryRowsRead.Observe(int64(count))) + r.region.Record(xcap.StatSecondaryRowBytes.Observe(totalBytesFilled)) } // We only advance r.row after we successfully read and filled rows. This @@ -268,8 +268,8 @@ func (r *Reader) readAndFilterPrimaryColumns(ctx context.Context, readSize int, readSize = passCount } - r.region.Record(StatPrimaryRowsRead.Observe(int64(rowsRead))) - r.region.Record(StatPrimaryRowBytes.Observe(primaryColumnBytes)) + r.region.Record(xcap.StatPrimaryRowsRead.Observe(int64(rowsRead))) + r.region.Record(xcap.StatPrimaryRowBytes.Observe(primaryColumnBytes)) return rowsRead, passCount, nil } @@ -555,11 +555,11 @@ func (r *Reader) initDownloader(ctx context.Context) error { if primary { r.primaryColumnIndexes = append(r.primaryColumnIndexes, i) - r.region.Record(StatPrimaryColumns.Observe(1)) - r.region.Record(StatPrimaryColumnPages.Observe(int64(column.ColumnDesc().PagesCount))) + r.region.Record(xcap.StatPrimaryColumns.Observe(1)) + r.region.Record(xcap.StatPrimaryColumnPages.Observe(int64(column.ColumnDesc().PagesCount))) } else { - r.region.Record(StatSecondaryColumns.Observe(1)) - r.region.Record(StatSecondaryColumnPages.Observe(int64(column.ColumnDesc().PagesCount))) + r.region.Record(xcap.StatSecondaryColumns.Observe(1)) + r.region.Record(xcap.StatSecondaryColumnPages.Observe(int64(column.ColumnDesc().PagesCount))) } } @@ -593,8 +593,8 @@ func (r *Reader) initDownloader(ctx context.Context) error { rowsCount = max(rowsCount, uint64(column.ColumnDesc().RowsCount)) } - r.region.Record(StatMaxRows.Observe(int64(rowsCount))) - r.region.Record(StatRowsAfterPruning.Observe(int64(ranges.TotalRowCount()))) + r.region.Record(xcap.StatMaxRows.Observe(int64(rowsCount))) + r.region.Record(xcap.StatRowsAfterPruning.Observe(int64(ranges.TotalRowCount()))) return nil } diff --git a/pkg/dataobj/internal/dataset/reader_downloader.go b/pkg/dataobj/internal/dataset/reader_downloader.go index e2d870932a43c..daed18cf6016f 100644 --- a/pkg/dataobj/internal/dataset/reader_downloader.go +++ b/pkg/dataobj/internal/dataset/reader_downloader.go @@ -232,13 +232,13 @@ func (dl *readerDownloader) downloadBatch(ctx context.Context, requestor *reader if region := xcap.RegionFromContext(ctx); region != nil { for _, page := range batch { if page.column.primary { - region.Record(StatPrimaryPagesDownloaded.Observe(1)) - region.Record(StatPrimaryColumnBytes.Observe(int64(page.inner.PageDesc().CompressedSize))) - region.Record(StatPrimaryColumnUncompressedBytes.Observe(int64(page.inner.PageDesc().UncompressedSize))) + region.Record(xcap.StatPrimaryPagesDownloaded.Observe(1)) + region.Record(xcap.StatPrimaryColumnBytes.Observe(int64(page.inner.PageDesc().CompressedSize))) + region.Record(xcap.StatPrimaryColumnUncompressedBytes.Observe(int64(page.inner.PageDesc().UncompressedSize))) } else { - region.Record(StatSecondaryPagesDownloaded.Observe(1)) - region.Record(StatSecondaryColumnBytes.Observe(int64(page.inner.PageDesc().CompressedSize))) - region.Record(StatSecondaryColumnUncompressedBytes.Observe(int64(page.inner.PageDesc().UncompressedSize))) + region.Record(xcap.StatSecondaryPagesDownloaded.Observe(1)) + region.Record(xcap.StatSecondaryColumnBytes.Observe(int64(page.inner.PageDesc().CompressedSize))) + region.Record(xcap.StatSecondaryColumnUncompressedBytes.Observe(int64(page.inner.PageDesc().UncompressedSize))) } } } @@ -589,13 +589,13 @@ func (page *readerPage) PageDesc() *PageDesc { func (page *readerPage) ReadPage(ctx context.Context) (PageData, error) { region := xcap.RegionFromContext(ctx) - region.Record(StatPagesScanned.Observe(1)) + region.Record(xcap.StatPagesScanned.Observe(1)) if page.data != nil { - region.Record(StatPagesFoundInCache.Observe(1)) + region.Record(xcap.StatPagesFoundInCache.Observe(1)) return page.data, nil } - region.Record(StatPageDownloadRequests.Observe(1)) + region.Record(xcap.StatPageDownloadRequests.Observe(1)) if err := page.column.dl.downloadBatch(ctx, page); err != nil { return nil, err } diff --git a/pkg/dataobj/internal/dataset/reader_test.go b/pkg/dataobj/internal/dataset/reader_test.go index a0fc57a95d6d8..701d22817a90a 100644 --- a/pkg/dataobj/internal/dataset/reader_test.go +++ b/pkg/dataobj/internal/dataset/reader_test.go @@ -917,14 +917,14 @@ func Test_Reader_Stats(t *testing.T) { obsMap[obs.Statistic.Name()] = obs.Value.(int64) } - require.Equal(t, int64(2), obsMap[StatReadCalls.Name()]) - require.Equal(t, int64(2), obsMap[StatPrimaryColumns.Name()]) - require.Equal(t, int64(2), obsMap[StatSecondaryColumns.Name()]) - require.Equal(t, int64(5), obsMap[StatPrimaryColumnPages.Name()]) - require.Equal(t, int64(8), obsMap[StatSecondaryColumnPages.Name()]) - - require.Equal(t, int64(len(basicReaderTestData)), obsMap[StatMaxRows.Name()]) - require.Equal(t, int64(3), obsMap[StatRowsAfterPruning.Name()]) - require.Equal(t, int64(3), obsMap[StatPrimaryRowsRead.Name()]) - require.Equal(t, int64(1), obsMap[StatSecondaryRowsRead.Name()]) + require.Equal(t, int64(2), obsMap[xcap.StatDatasetReadCalls.Name()]) + require.Equal(t, int64(2), obsMap[xcap.StatPrimaryColumns.Name()]) + require.Equal(t, int64(2), obsMap[xcap.StatSecondaryColumns.Name()]) + require.Equal(t, int64(5), obsMap[xcap.StatPrimaryColumnPages.Name()]) + require.Equal(t, int64(8), obsMap[xcap.StatSecondaryColumnPages.Name()]) + + require.Equal(t, int64(len(basicReaderTestData)), obsMap[xcap.StatMaxRows.Name()]) + require.Equal(t, int64(3), obsMap[xcap.StatRowsAfterPruning.Name()]) + require.Equal(t, int64(3), obsMap[xcap.StatPrimaryRowsRead.Name()]) + require.Equal(t, int64(1), obsMap[xcap.StatSecondaryRowsRead.Name()]) } diff --git a/pkg/dataobj/metastore/object.go b/pkg/dataobj/metastore/object.go index cb0901657c1a5..3efbf0b1a69f7 100644 --- a/pkg/dataobj/metastore/object.go +++ b/pkg/dataobj/metastore/object.go @@ -39,11 +39,6 @@ const ( metastoreWindowSize = 12 * time.Hour ) -var ( - statIndexObjects = xcap.NewStatisticInt64("metastore.index.objects", xcap.AggregationTypeSum) - statResolvedSections = xcap.NewStatisticInt64("metastore.resolved.sections", xcap.AggregationTypeSum) -) - type ObjectMetastore struct { bucket objstore.Bucket parallelism int @@ -193,7 +188,7 @@ func (m *ObjectMetastore) Sections(ctx context.Context, start, end time.Time, ma } m.metrics.indexObjectsTotal.Observe(float64(len(indexPaths))) - region.Record(statIndexObjects.Observe(int64(len(indexPaths)))) + region.Record(xcap.StatIndexObjects.Observe(int64(len(indexPaths)))) // Return early if no index files are found if len(indexPaths) == 0 { @@ -250,7 +245,7 @@ func (m *ObjectMetastore) Sections(ctx context.Context, start, end time.Time, ma duration := sectionsTimer.ObserveDuration() m.metrics.resolvedSectionsTotal.Observe(float64(len(streamSectionPointers))) m.metrics.resolvedSectionsRatio.Observe(float64(len(streamSectionPointers)) / float64(initialSectionPointersCount)) - region.Record(statResolvedSections.Observe(int64(len(streamSectionPointers)))) + region.Record(xcap.StatResolvedSections.Observe(int64(len(streamSectionPointers)))) level.Debug(utillog.WithContext(ctx, m.logger)).Log( "msg", "resolved sections", diff --git a/pkg/dataobj/sections/internal/columnar/decoder.go b/pkg/dataobj/sections/internal/columnar/decoder.go index 8488ac8e7c06f..88650a24644b1 100644 --- a/pkg/dataobj/sections/internal/columnar/decoder.go +++ b/pkg/dataobj/sections/internal/columnar/decoder.go @@ -79,7 +79,7 @@ func (dec *Decoder) Pages(ctx context.Context, columns []*datasetmd.ColumnDesc) region := xcap.RegionFromContext(ctx) startTime := time.Now() defer func() { - region.Record(dataset.StatPageDownloadTime.Observe(time.Since(startTime).Nanoseconds())) + region.Record(xcap.StatPageDownloadTime.Observe(time.Since(startTime).Nanoseconds())) }() ranges := make([]rangeio.Range, 0, len(columns)) @@ -135,7 +135,7 @@ func (dec *Decoder) ReadPages(ctx context.Context, pages []*datasetmd.PageDesc) region := xcap.RegionFromContext(ctx) startTime := time.Now() defer func() { - region.Record(dataset.StatPageDownloadTime.Observe(time.Since(startTime).Nanoseconds())) + region.Record(xcap.StatPageDownloadTime.Observe(time.Since(startTime).Nanoseconds())) }() ranges := make([]rangeio.Range, 0, len(pages)) diff --git a/pkg/engine/internal/executor/compat.go b/pkg/engine/internal/executor/compat.go index 4b81e4225c4f8..d4470b263ee42 100644 --- a/pkg/engine/internal/executor/compat.go +++ b/pkg/engine/internal/executor/compat.go @@ -81,7 +81,7 @@ func newColumnCompatibilityPipeline(compat *physical.ColumnCompat, input Pipelin return cmp.Compare(a.name, b.name) }) - region.Record(statCompatCollisionFound.Observe(true)) + region.Record(xcap.StatCompatCollisionFound.Observe(true)) // Next, update the schema with the new columns that have the _extracted suffix. newSchema := batch.Schema() diff --git a/pkg/engine/internal/executor/pipeline.go b/pkg/engine/internal/executor/pipeline.go index 14f4c7770e948..3948dbcddcf77 100644 --- a/pkg/engine/internal/executor/pipeline.go +++ b/pkg/engine/internal/executor/pipeline.go @@ -337,17 +337,17 @@ func (p *observedPipeline) Read(ctx context.Context) (arrow.RecordBatch, error) start := time.Now() if p.region != nil { - p.region.Record(statReadCalls.Observe(1)) + p.region.Record(xcap.StatReadCalls.Observe(1)) } rec, err := p.inner.Read(ctx) if p.region != nil { if rec != nil { - p.region.Record(statRowsOut.Observe(rec.NumRows())) + p.region.Record(xcap.StatRowsOut.Observe(rec.NumRows())) } - p.region.Record(statReadDuration.Observe(time.Since(start).Nanoseconds())) + p.region.Record(xcap.StatReadDuration.Observe(time.Since(start).Nanoseconds())) } return rec, err diff --git a/pkg/engine/internal/executor/stats.go b/pkg/engine/internal/executor/stats.go deleted file mode 100644 index b86a1a82cd470..0000000000000 --- a/pkg/engine/internal/executor/stats.go +++ /dev/null @@ -1,14 +0,0 @@ -package executor - -import "github.com/grafana/loki/v3/pkg/xcap" - -// xcap statistics used in executor pkg. -var ( - // Common statistics tracked for all pipeline nodes. - statRowsOut = xcap.NewStatisticInt64("rows_out", xcap.AggregationTypeSum) - statReadCalls = xcap.NewStatisticInt64("read_calls", xcap.AggregationTypeSum) - statReadDuration = xcap.NewStatisticInt64("read_duration_ns", xcap.AggregationTypeSum) - - // [ColumnCompat] statistics. - statCompatCollisionFound = xcap.NewStatisticFlag("collision_found") -) diff --git a/pkg/storage/bucket/xcap_bucket.go b/pkg/storage/bucket/xcap_bucket.go index 26f3eca8d4447..687d6cd5bcc80 100644 --- a/pkg/storage/bucket/xcap_bucket.go +++ b/pkg/storage/bucket/xcap_bucket.go @@ -9,14 +9,6 @@ import ( "github.com/grafana/loki/v3/pkg/xcap" ) -// Statistics for tracking bucket operation counts. -var ( - statBucketGet = xcap.NewStatisticInt64("bucket.get", xcap.AggregationTypeSum) - statBucketGetRange = xcap.NewStatisticInt64("bucket.getrange", xcap.AggregationTypeSum) - statBucketIter = xcap.NewStatisticInt64("bucket.iter", xcap.AggregationTypeSum) - statBucketAttributes = xcap.NewStatisticInt64("bucket.attributes", xcap.AggregationTypeSum) -) - // XCapBucket wraps an objstore.Bucket and records request counts to the xcap // Region found in the context. If no Region is present in the context, the // wrapper simply delegates to the underlying bucket without recording. @@ -51,13 +43,13 @@ func (b *XCapBucket) Close() error { // Iter calls f for each entry in the given directory (not recursive.). func (b *XCapBucket) Iter(ctx context.Context, dir string, f func(string) error, options ...objstore.IterOption) error { - recordOp(ctx, statBucketIter) + recordOp(ctx, xcap.StatBucketIter) return b.bkt.Iter(ctx, dir, f, options...) } // IterWithAttributes calls f for each entry in the given directory similar to Iter. func (b *XCapBucket) IterWithAttributes(ctx context.Context, dir string, f func(objstore.IterObjectAttributes) error, options ...objstore.IterOption) error { - recordOp(ctx, statBucketIter) + recordOp(ctx, xcap.StatBucketIter) return b.bkt.IterWithAttributes(ctx, dir, f, options...) } @@ -68,13 +60,13 @@ func (b *XCapBucket) SupportedIterOptions() []objstore.IterOptionType { // Get returns a reader for the given object name. func (b *XCapBucket) Get(ctx context.Context, name string) (io.ReadCloser, error) { - recordOp(ctx, statBucketGet) + recordOp(ctx, xcap.StatBucketGet) return b.bkt.Get(ctx, name) } // GetRange returns a new range reader for the given object name and range. func (b *XCapBucket) GetRange(ctx context.Context, name string, off, length int64) (io.ReadCloser, error) { - recordOp(ctx, statBucketGetRange) + recordOp(ctx, xcap.StatBucketGetRange) return b.bkt.GetRange(ctx, name, off, length) } @@ -100,7 +92,7 @@ func (b *XCapBucket) IsAccessDeniedErr(err error) bool { // Attributes returns information about the specified object. func (b *XCapBucket) Attributes(ctx context.Context, name string) (objstore.ObjectAttributes, error) { - recordOp(ctx, statBucketAttributes) + recordOp(ctx, xcap.StatBucketAttributes) return b.bkt.Attributes(ctx, name) } diff --git a/pkg/util/rangeio/rangeio.go b/pkg/util/rangeio/rangeio.go index 613f05c0d23c5..548667bc2e72d 100644 --- a/pkg/util/rangeio/rangeio.go +++ b/pkg/util/rangeio/rangeio.go @@ -24,16 +24,6 @@ import ( "github.com/grafana/loki/v3/pkg/xcap" ) -// xcap statistics for RangeIO operations. -var ( - statInputRangesCount = xcap.NewStatisticInt64("input.ranges", xcap.AggregationTypeSum) - statInputRangesSize = xcap.NewStatisticInt64("input.ranges.size.bytes", xcap.AggregationTypeSum) - statOptimizedRangesCount = xcap.NewStatisticInt64("optimized.ranges", xcap.AggregationTypeSum) - statOptimizedRangesSize = xcap.NewStatisticInt64("optimized.ranges.size.bytes", xcap.AggregationTypeSum) - // pick the min value when aggregating to track the slowest read. - statOptimizedThroughput = xcap.NewStatisticFloat64("optimized.ranges.min.throughput", xcap.AggregationTypeMin) -) - // Range represents a range of data to be read. type Range struct { // Data to read into; exactly len(Data) bytes will be read, or an error will @@ -439,16 +429,16 @@ func recordRangeStats(ranges, optimizedRanges []Range, region *xcap.Region) { origSize := rangesSize(ranges) optimizedSize := rangesSize(optimizedRanges) - region.Record(statInputRangesCount.Observe(int64(len(ranges)))) - region.Record(statInputRangesSize.Observe(int64(origSize))) - region.Record(statOptimizedRangesCount.Observe(int64(len(optimizedRanges)))) - region.Record(statOptimizedRangesSize.Observe(int64(optimizedSize))) + region.Record(xcap.StatInputRangesCount.Observe(int64(len(ranges)))) + region.Record(xcap.StatInputRangesSize.Observe(int64(origSize))) + region.Record(xcap.StatOptimizedRangesCount.Observe(int64(len(optimizedRanges)))) + region.Record(xcap.StatOptimizedRangesSize.Observe(int64(optimizedSize))) } func recordThroughputStat(region *xcap.Region, startTime time.Time, optimizedRanges []Range) { size := rangesSize(optimizedRanges) bytesPerSec := float64(size) / time.Since(startTime).Seconds() - region.Record(statOptimizedThroughput.Observe(bytesPerSec)) + region.Record(xcap.StatOptimizedThroughput.Observe(bytesPerSec)) } type bytesStringer uint64 diff --git a/pkg/xcap/exporter.go b/pkg/xcap/exporter.go index 10fdd3f2bd283..7ad8fc18623c5 100644 --- a/pkg/xcap/exporter.go +++ b/pkg/xcap/exporter.go @@ -142,15 +142,15 @@ func summarizeObservations(capture *Capture) *observations { collect.fromRegions("DataObjScan", true, "streamsView.init"). filter( // object store calls - "bucket.get", "bucket.getrange", "bucket.attributes", + StatBucketGet.Key(), StatBucketGetRange.Key(), StatBucketAttributes.Key(), // dataset reader stats - "row.max", "rows.after.pruning", "read.calls", - "primary.pages.downloaded", "secondary.pages.downloaded", - "primary.pages.compressed.bytes", "secondary.pages.compressed.bytes", - "primary.rows.read", "secondary.rows.read", - "primary.row.read.bytes", "secondary.row.read.bytes", - "pages.scanned", "pages.cache.hit", - "pages.download.requests", "pages.download.duration.ns", + StatMaxRows.Key(), StatRowsAfterPruning.Key(), StatDatasetReadCalls.Key(), + StatPrimaryPagesDownloaded.Key(), StatSecondaryPagesDownloaded.Key(), + StatPrimaryColumnBytes.Key(), StatSecondaryColumnBytes.Key(), + StatPrimaryRowsRead.Key(), StatSecondaryRowsRead.Key(), + StatPrimaryRowBytes.Key(), StatSecondaryRowBytes.Key(), + StatPagesScanned.Key(), StatPagesFoundInCache.Key(), + StatPageDownloadRequests.Key(), StatPageDownloadTime.Key(), ). prefix("logs_dataset_"). normalizeKeys(), @@ -159,16 +159,18 @@ func summarizeObservations(capture *Capture) *observations { // metastore index and resolved section stats result.merge( collect.fromRegions("ObjectMetastore.Sections", true). - filter("metastore.index.objects", "metastore.resolved.sections"). + filter(StatIndexObjects.Key(), StatResolvedSections.Key()). normalizeKeys(), ) // metastore bucket and dataset reader stats result.merge( collect.fromRegions("ObjectMetastore.Sections", true). - filter("bucket.get", "bucket.getrange", "bucket.attributes", - "primary.pages.downloaded", "secondary.pages.downloaded", - "primary.pages.compressed.bytes", "secondary.pages.compressed.bytes"). + filter( + StatBucketGet.Key(), StatBucketGetRange.Key(), StatBucketAttributes.Key(), + StatPrimaryPagesDownloaded.Key(), StatSecondaryPagesDownloaded.Key(), + StatPrimaryColumnBytes.Key(), StatSecondaryColumnBytes.Key(), + ). prefix("metastore_"). normalizeKeys(), ) @@ -176,9 +178,11 @@ func summarizeObservations(capture *Capture) *observations { // streamsView bucket and dataset reader stats result.merge( collect.fromRegions("streamsView.init", true). - filter("bucket.get", "bucket.getrange", "bucket.attributes", - "primary.pages.downloaded", "secondary.pages.downloaded", - "primary.pages.compressed.bytes", "secondary.pages.compressed.bytes"). + filter( + StatBucketGet.Key(), StatBucketGetRange.Key(), StatBucketAttributes.Key(), + StatPrimaryPagesDownloaded.Key(), StatSecondaryPagesDownloaded.Key(), + StatPrimaryColumnBytes.Key(), StatSecondaryColumnBytes.Key(), + ). prefix("streams_"). normalizeKeys(), ) diff --git a/pkg/xcap/stats_definitions.go b/pkg/xcap/stats_definitions.go new file mode 100644 index 0000000000000..4aab20ea38ffd --- /dev/null +++ b/pkg/xcap/stats_definitions.go @@ -0,0 +1,71 @@ +package xcap + +// Common pipeline statistics tracked across executor nodes. +var ( + StatRowsOut = NewStatisticInt64("rows.out", AggregationTypeSum) + StatReadCalls = NewStatisticInt64("read.calls", AggregationTypeSum) + StatReadDuration = NewStatisticInt64("read.duration.ns", AggregationTypeSum) +) + +// ColumnCompat statistics. +var ( + StatCompatCollisionFound = NewStatisticFlag("collision.found") +) + +var ( + // Dataset column statistics. + StatPrimaryColumns = NewStatisticInt64("primary.columns", AggregationTypeSum) + StatSecondaryColumns = NewStatisticInt64("secondary.columns", AggregationTypeSum) + StatPrimaryColumnPages = NewStatisticInt64("primary.column.pages", AggregationTypeSum) + StatSecondaryColumnPages = NewStatisticInt64("secondary.column.pages", AggregationTypeSum) + + // Dataset row statistics. + StatMaxRows = NewStatisticInt64("row.max", AggregationTypeSum) + StatRowsAfterPruning = NewStatisticInt64("rows.after.pruning", AggregationTypeSum) + StatPrimaryRowsRead = NewStatisticInt64("primary.rows.read", AggregationTypeSum) + StatSecondaryRowsRead = NewStatisticInt64("secondary.rows.read", AggregationTypeSum) + StatPrimaryRowBytes = NewStatisticInt64("primary.row.read.bytes", AggregationTypeSum) + StatSecondaryRowBytes = NewStatisticInt64("secondary.row.read.bytes", AggregationTypeSum) + + // Dataset page scan statistics. + StatPagesScanned = NewStatisticInt64("pages.scanned", AggregationTypeSum) + StatPagesFoundInCache = NewStatisticInt64("pages.cache.hit", AggregationTypeSum) + StatPageDownloadRequests = NewStatisticInt64("pages.download.requests", AggregationTypeSum) + StatPageDownloadTime = NewStatisticInt64("pages.download.duration.ns", AggregationTypeSum) + + // Dataset page download byte statistics. + StatPrimaryPagesDownloaded = NewStatisticInt64("primary.pages.downloaded", AggregationTypeSum) + StatSecondaryPagesDownloaded = NewStatisticInt64("secondary.pages.downloaded", AggregationTypeSum) + StatPrimaryColumnBytes = NewStatisticInt64("primary.pages.compressed.bytes", AggregationTypeSum) + StatSecondaryColumnBytes = NewStatisticInt64("secondary.pages.compressed.bytes", AggregationTypeSum) + StatPrimaryColumnUncompressedBytes = NewStatisticInt64("primary.column.uncompressed.bytes", AggregationTypeSum) + StatSecondaryColumnUncompressedBytes = NewStatisticInt64("secondary.column.uncompressed.bytes", AggregationTypeSum) +) + +// Dataset read operation statistics. +var ( + StatDatasetReadCalls = NewStatisticInt64("dataset.read.calls", AggregationTypeSum) +) + +// Range IO statistics. +var ( + StatInputRangesCount = NewStatisticInt64("input.ranges", AggregationTypeSum) + StatInputRangesSize = NewStatisticInt64("input.ranges.size.bytes", AggregationTypeSum) + StatOptimizedRangesCount = NewStatisticInt64("optimized.ranges", AggregationTypeSum) + StatOptimizedRangesSize = NewStatisticInt64("optimized.ranges.size.bytes", AggregationTypeSum) + StatOptimizedThroughput = NewStatisticFloat64("optimized.ranges.min.throughput", AggregationTypeMin) +) + +// Bucket operation statistics. +var ( + StatBucketGet = NewStatisticInt64("bucket.get", AggregationTypeSum) + StatBucketGetRange = NewStatisticInt64("bucket.getrange", AggregationTypeSum) + StatBucketIter = NewStatisticInt64("bucket.iter", AggregationTypeSum) + StatBucketAttributes = NewStatisticInt64("bucket.attributes", AggregationTypeSum) +) + +// Metastore statistics. +var ( + StatIndexObjects = NewStatisticInt64("metastore.index.objects", AggregationTypeSum) + StatResolvedSections = NewStatisticInt64("metastore.resolved.sections", AggregationTypeSum) +) diff --git a/pkg/xcap/stats_summary.go b/pkg/xcap/stats_summary.go deleted file mode 100644 index 3cd7e0e215580..0000000000000 --- a/pkg/xcap/stats_summary.go +++ /dev/null @@ -1,81 +0,0 @@ -package xcap - -import ( - "time" - - "github.com/grafana/loki/v3/pkg/logqlmodel/stats" -) - -// Region name for data object scan operations. -const regionNameDataObjScan = "DataObjScan" - -// Statistic keys used for Summary computation. -// These must match the statistic names used in the executor package. -const ( - // Row statistics - statKeyRowsOut = "rows_out" - statKeyPrimaryRowsRead = "primary_rows_read" - - // Byte statistics - statKeyPrimaryColumnUncompressedBytes = "primary_column_uncompressed_bytes" - statKeySecondaryColumnUncompressedBytes = "secondary_column_uncompressed_bytes" -) - -// ToStatsSummary computes a stats.Summary from observations in the capture. -func (c *Capture) ToStatsSummary(execTime, queueTime time.Duration, totalEntriesReturned int) stats.Summary { - summary := stats.Summary{ - ExecTime: execTime.Seconds(), - QueueTime: queueTime.Seconds(), - TotalEntriesReturned: int64(totalEntriesReturned), - } - - if c == nil { - return summary - } - - // Collect observations from DataObjScan as the summary stats mainly relate to log lines. - // In practice, new engine would process more bytes while scanning metastore objects and stream sections. - collector := newObservationCollector(c) - obs := collector.fromRegions(regionNameDataObjScan, true).filter( - statKeyRowsOut, - statKeyPrimaryRowsRead, - statKeyPrimaryColumnUncompressedBytes, - statKeySecondaryColumnUncompressedBytes, - ) - - // TotalBytesProcessed: sum of uncompressed bytes from primary and secondary columns - summary.TotalBytesProcessed = getObservation(obs, statKeyPrimaryColumnUncompressedBytes) + - getObservation(obs, statKeySecondaryColumnUncompressedBytes) - - // TotalLinesProcessed: primary rows read - summary.TotalLinesProcessed = getObservation(obs, statKeyPrimaryRowsRead) - - // TotalPostFilterLines: rows output after filtering - // TODO: this will report the wrong value if the plan has a filter stage. - // pick the min of row_out from filter and scan nodes. - summary.TotalPostFilterLines = getObservation(obs, statKeyRowsOut) - - if execTime > 0 { - execSeconds := execTime.Seconds() - summary.BytesProcessedPerSecond = int64(float64(summary.TotalBytesProcessed) / execSeconds) - summary.LinesProcessedPerSecond = int64(float64(summary.TotalLinesProcessed) / execSeconds) - } - - return summary -} - -// getObservation retrieves an int64 value from observations by statistic name. -func getObservation(obs *observations, name string) int64 { - if obs == nil { - return 0 - } - - for key, agg := range obs.data { - if key.Name == name { - if v, ok := agg.Value.(int64); ok { - return v - } - } - } - return 0 -} diff --git a/pkg/xcap/stats_summary_test.go b/pkg/xcap/stats_summary_test.go index c1b296611b04f..c4061e489a5c7 100644 --- a/pkg/xcap/stats_summary_test.go +++ b/pkg/xcap/stats_summary_test.go @@ -8,13 +8,14 @@ import ( "github.com/stretchr/testify/require" ) -func TestSummaryFromCapture(t *testing.T) { +func TestToStatsSummary(t *testing.T) { t.Run("nil capture returns empty summary with provided values", func(t *testing.T) { execTime := 2 * time.Second queueTime := 100 * time.Millisecond entriesReturned := 42 - summary := SummaryFromCapture(nil, execTime, queueTime, entriesReturned) + var capture *Capture + summary := capture.ToStatsSummary(execTime, queueTime, entriesReturned) require.Equal(t, execTime.Seconds(), summary.ExecTime) require.Equal(t, queueTime.Seconds(), summary.QueueTime) @@ -27,28 +28,21 @@ func TestSummaryFromCapture(t *testing.T) { t.Run("computes bytes and lines from DataObjScan regions", func(t *testing.T) { ctx, capture := NewCapture(context.Background(), nil) - // Define statistics matching the executor package names - primaryBytes := NewStatisticInt64("primary_column_uncompressed_bytes", AggregationTypeSum) - secondaryBytes := NewStatisticInt64("secondary_column_uncompressed_bytes", AggregationTypeSum) - primaryRows := NewStatisticInt64("primary_rows_read", AggregationTypeSum) - secondaryRows := NewStatisticInt64("secondary_rows_read", AggregationTypeSum) - rowsOut := NewStatisticInt64("rows_out", AggregationTypeSum) - - // Create DataObjScan regions with observations + // Create DataObjScan regions with observations using registry stats _, region1 := StartRegion(ctx, "DataObjScan") - region1.Record(primaryBytes.Observe(1000)) - region1.Record(secondaryBytes.Observe(500)) - region1.Record(primaryRows.Observe(100)) - region1.Record(secondaryRows.Observe(50)) - region1.Record(rowsOut.Observe(80)) + region1.Record(StatPrimaryColumnUncompressedBytes.Observe(1000)) + region1.Record(StatSecondaryColumnUncompressedBytes.Observe(500)) + region1.Record(StatPrimaryRowsRead.Observe(100)) + region1.Record(StatSecondaryRowsRead.Observe(50)) + region1.Record(StatRowsOut.Observe(80)) region1.End() _, region2 := StartRegion(ctx, "DataObjScan") - region2.Record(primaryBytes.Observe(2000)) - region2.Record(secondaryBytes.Observe(1000)) - region2.Record(primaryRows.Observe(200)) - region2.Record(secondaryRows.Observe(100)) - region2.Record(rowsOut.Observe(150)) + region2.Record(StatPrimaryColumnUncompressedBytes.Observe(2000)) + region2.Record(StatSecondaryColumnUncompressedBytes.Observe(1000)) + region2.Record(StatPrimaryRowsRead.Observe(200)) + region2.Record(StatSecondaryRowsRead.Observe(100)) + region2.Record(StatRowsOut.Observe(150)) region2.End() capture.End() @@ -57,7 +51,7 @@ func TestSummaryFromCapture(t *testing.T) { queueTime := 100 * time.Millisecond entriesReturned := 42 - summary := SummaryFromCapture(capture, execTime, queueTime, entriesReturned) + summary := capture.ToStatsSummary(execTime, queueTime, entriesReturned) require.Equal(t, execTime.Seconds(), summary.ExecTime) require.Equal(t, queueTime.Seconds(), summary.QueueTime) @@ -66,8 +60,8 @@ func TestSummaryFromCapture(t *testing.T) { // TotalBytesProcessed = primary + secondary = (1000+2000) + (500+1000) = 4500 require.Equal(t, int64(4500), summary.TotalBytesProcessed) - // TotalLinesProcessed = primary + secondary = (100+200) + (50+100) = 450 - require.Equal(t, int64(450), summary.TotalLinesProcessed) + // TotalLinesProcessed = primary_rows_read = 100 + 200 = 300 + require.Equal(t, int64(300), summary.TotalLinesProcessed) // TotalPostFilterLines = rows_out = 80 + 150 = 230 require.Equal(t, int64(230), summary.TotalPostFilterLines) @@ -75,28 +69,26 @@ func TestSummaryFromCapture(t *testing.T) { // BytesProcessedPerSecond = 4500 / 2 = 2250 require.Equal(t, int64(2250), summary.BytesProcessedPerSecond) - // LinesProcessedPerSecond = 450 / 2 = 225 - require.Equal(t, int64(225), summary.LinesProcessedPerSecond) + // LinesProcessedPerSecond = 300 / 2 = 150 + require.Equal(t, int64(150), summary.LinesProcessedPerSecond) }) t.Run("ignores non-DataObjScan regions", func(t *testing.T) { ctx, capture := NewCapture(context.Background(), nil) - primaryBytes := NewStatisticInt64("primary_column_uncompressed_bytes", AggregationTypeSum) - // DataObjScan region - should be counted _, scanRegion := StartRegion(ctx, "DataObjScan") - scanRegion.Record(primaryBytes.Observe(1000)) + scanRegion.Record(StatPrimaryColumnUncompressedBytes.Observe(1000)) scanRegion.End() // Other region - should be ignored _, otherRegion := StartRegion(ctx, "OtherRegion") - otherRegion.Record(primaryBytes.Observe(5000)) + otherRegion.Record(StatPrimaryColumnUncompressedBytes.Observe(5000)) otherRegion.End() capture.End() - summary := SummaryFromCapture(capture, time.Second, 0, 0) + summary := capture.ToStatsSummary(time.Second, 0, 0) // Only DataObjScan region bytes counted require.Equal(t, int64(1000), summary.TotalBytesProcessed) @@ -105,13 +97,12 @@ func TestSummaryFromCapture(t *testing.T) { t.Run("zero exec time results in zero per-second rates", func(t *testing.T) { ctx, capture := NewCapture(context.Background(), nil) - primaryBytes := NewStatisticInt64("primary_column_uncompressed_bytes", AggregationTypeSum) _, region := StartRegion(ctx, "DataObjScan") - region.Record(primaryBytes.Observe(1000)) + region.Record(StatPrimaryColumnUncompressedBytes.Observe(1000)) region.End() capture.End() - summary := SummaryFromCapture(capture, 0, 0, 10) + summary := capture.ToStatsSummary(0, 0, 10) require.Equal(t, int64(1000), summary.TotalBytesProcessed) require.Equal(t, int64(0), summary.BytesProcessedPerSecond) @@ -122,14 +113,13 @@ func TestSummaryFromCapture(t *testing.T) { ctx, capture := NewCapture(context.Background(), nil) // Only record some statistics - primaryBytes := NewStatisticInt64("primary_column_uncompressed_bytes", AggregationTypeSum) _, region := StartRegion(ctx, "DataObjScan") - region.Record(primaryBytes.Observe(1000)) + region.Record(StatPrimaryColumnUncompressedBytes.Observe(1000)) region.End() capture.End() execTime := 1 * time.Second - summary := SummaryFromCapture(capture, execTime, 0, 0) + summary := capture.ToStatsSummary(execTime, 0, 0) // Only primary bytes recorded require.Equal(t, int64(1000), summary.TotalBytesProcessed) @@ -141,21 +131,19 @@ func TestSummaryFromCapture(t *testing.T) { t.Run("rolls up child region observations into DataObjScan", func(t *testing.T) { ctx, capture := NewCapture(context.Background(), nil) - primaryBytes := NewStatisticInt64("primary_column_uncompressed_bytes", AggregationTypeSum) - // Parent DataObjScan region ctx, parent := StartRegion(ctx, "DataObjScan") - parent.Record(primaryBytes.Observe(500)) + parent.Record(StatPrimaryColumnUncompressedBytes.Observe(500)) // Child region (should be rolled up into parent) _, child := StartRegion(ctx, "child_operation") - child.Record(primaryBytes.Observe(300)) + child.Record(StatPrimaryColumnUncompressedBytes.Observe(300)) child.End() parent.End() capture.End() - summary := SummaryFromCapture(capture, time.Second, 0, 0) + summary := capture.ToStatsSummary(time.Second, 0, 0) // Child observations rolled up into DataObjScan: 500 + 300 = 800 require.Equal(t, int64(800), summary.TotalBytesProcessed) @@ -165,7 +153,7 @@ func TestSummaryFromCapture(t *testing.T) { _, capture := NewCapture(context.Background(), nil) capture.End() - summary := SummaryFromCapture(capture, time.Second, time.Millisecond, 10) + summary := capture.ToStatsSummary(time.Second, time.Millisecond, 10) require.Equal(t, float64(1), summary.ExecTime) require.Equal(t, float64(0.001), summary.QueueTime) diff --git a/pkg/xcap/summary.go b/pkg/xcap/summary.go index 6baefe19b9190..d08bfab0caf4d 100644 --- a/pkg/xcap/summary.go +++ b/pkg/xcap/summary.go @@ -22,20 +22,20 @@ func newObservations() *observations { return &observations{data: make(map[StatisticKey]*AggregatedObservation)} } -// filter returns a new observations containing only entries with matching stat names. -func (o *observations) filter(names ...string) *observations { - if len(names) == 0 || o == nil { +// filter returns a new observations containing only entries with matching stat keys. +func (o *observations) filter(keys ...StatisticKey) *observations { + if len(keys) == 0 || o == nil { return o } - nameSet := make(map[string]struct{}, len(names)) - for _, n := range names { - nameSet[n] = struct{}{} + keySet := make(map[StatisticKey]struct{}, len(keys)) + for _, k := range keys { + keySet[k] = struct{}{} } result := newObservations() for k, obs := range o.data { - if _, ok := nameSet[k.Name]; ok { + if _, ok := keySet[k]; ok { result.data[k] = obs } } @@ -147,8 +147,9 @@ func (o *observations) toLogValues() []any { // observationCollector provides methods to collect observations from a Capture. type observationCollector struct { - capture *Capture - childrenMap map[identifier][]*Region + capture *Capture + childrenMap map[identifier][]*Region + nameToRegions map[string][]*Region } // newObservationCollector creates a new collector for gathering observations from the given capture. @@ -157,15 +158,20 @@ func newObservationCollector(capture *Capture) *observationCollector { return nil } - // Build parent -> children map + // Build + // - parent -> children + // - name -> matching regions childrenMap := make(map[identifier][]*Region) - for _, r := range capture.Regions() { + nameToRegions := make(map[string][]*Region) + for _, r := range capture.regions { childrenMap[r.parentID] = append(childrenMap[r.parentID], r) + nameToRegions[r.name] = append(nameToRegions[r.name], r) } return &observationCollector{ - capture: capture, - childrenMap: childrenMap, + capture: capture, + childrenMap: childrenMap, + nameToRegions: nameToRegions, } } @@ -173,8 +179,15 @@ func newObservationCollector(capture *Capture) *observationCollector { // If rollUp is true, each region's stats include all its descendant stats // aggregated according to each stat's aggregation type. func (c *observationCollector) fromRegions(name string, rollUp bool, excluded ...string) *observations { + result := newObservations() + if c == nil { - return newObservations() + return result + } + + regions := c.nameToRegions[name] + if len(regions) == 0 { + return result } excludedSet := make(map[string]struct{}, len(excluded)) @@ -182,12 +195,7 @@ func (c *observationCollector) fromRegions(name string, rollUp bool, excluded .. excludedSet[name] = struct{}{} } - result := newObservations() - for _, region := range c.capture.Regions() { - if region.name != name { - continue - } - + for _, region := range regions { var obs *observations if rollUp { obs = c.rollUpObservations(region, excludedSet) @@ -288,6 +296,5 @@ func readInt64(o *observations, key StatisticKey) int64 { return v } } - return 0 } diff --git a/pkg/xcap/summary_test.go b/pkg/xcap/summary_test.go index 1ad0dd19f50c8..d640d0bdd3b31 100644 --- a/pkg/xcap/summary_test.go +++ b/pkg/xcap/summary_test.go @@ -24,21 +24,21 @@ func TestObservations(t *testing.T) { obs := collector.fromRegions("Test", false) t.Run("filter", func(t *testing.T) { - filtered := obs.filter("stat.one", "stat.two") + filtered := obs.filter(statA.Key(), statB.Key()) require.Len(t, filtered.data, 2) require.Equal(t, int64(10), filtered.data[statA.Key()].Value) require.Equal(t, int64(20), filtered.data[statB.Key()].Value) }) t.Run("prefix", func(t *testing.T) { - prefixed := obs.filter("stat.one").prefix("metastore_") + prefixed := obs.filter(statA.Key()).prefix("metastore_") expectedKey := StatisticKey{Name: "metastore_stat.one", DataType: DataTypeInt64, Aggregation: AggregationTypeSum} require.Len(t, prefixed.data, 1) require.Equal(t, int64(10), prefixed.data[expectedKey].Value) }) t.Run("normalizeKeys", func(t *testing.T) { - normalized := obs.filter("stat.three").normalizeKeys() + normalized := obs.filter(statC.Key()).normalizeKeys() expectedKey := StatisticKey{Name: "stat_three", DataType: DataTypeInt64, Aggregation: AggregationTypeSum} require.Len(t, normalized.data, 1) require.Equal(t, int64(30), normalized.data[expectedKey].Value) @@ -46,8 +46,8 @@ func TestObservations(t *testing.T) { t.Run("merge", func(t *testing.T) { target := newObservations() - target.merge(obs.filter("stat.one", "stat.two")) - target.merge(obs.filter("stat.two", "stat.three")) + target.merge(obs.filter(statA.Key(), statB.Key())) + target.merge(obs.filter(statB.Key(), statC.Key())) require.Len(t, target.data, 3) require.Equal(t, int64(10), target.data[statA.Key()].Value) @@ -62,7 +62,7 @@ func TestObservations(t *testing.T) { }) t.Run("chaining", func(t *testing.T) { - result := obs.filter("stat.three").prefix("logs_").normalizeKeys().toLogValues() + result := obs.filter(statC.Key()).prefix("logs_").normalizeKeys().toLogValues() require.Equal(t, []any{"logs_stat_three", int64(30)}, result) }) } From 54de16c7f2265295f7e76fd411c4fe7270870fb8 Mon Sep 17 00:00:00 2001 From: Ashwanth Goli Date: Thu, 4 Dec 2025 13:16:23 +0530 Subject: [PATCH 09/13] tidy up tests --- pkg/xcap/stats_summary_test.go | 163 --------------------------------- pkg/xcap/summary.go | 1 + pkg/xcap/summary_test.go | 110 ++++++++++++++++++++++ 3 files changed, 111 insertions(+), 163 deletions(-) diff --git a/pkg/xcap/stats_summary_test.go b/pkg/xcap/stats_summary_test.go index c4061e489a5c7..c7a71730da29d 100644 --- a/pkg/xcap/stats_summary_test.go +++ b/pkg/xcap/stats_summary_test.go @@ -1,164 +1 @@ package xcap - -import ( - "context" - "testing" - "time" - - "github.com/stretchr/testify/require" -) - -func TestToStatsSummary(t *testing.T) { - t.Run("nil capture returns empty summary with provided values", func(t *testing.T) { - execTime := 2 * time.Second - queueTime := 100 * time.Millisecond - entriesReturned := 42 - - var capture *Capture - summary := capture.ToStatsSummary(execTime, queueTime, entriesReturned) - - require.Equal(t, execTime.Seconds(), summary.ExecTime) - require.Equal(t, queueTime.Seconds(), summary.QueueTime) - require.Equal(t, int64(entriesReturned), summary.TotalEntriesReturned) - require.Equal(t, int64(0), summary.TotalBytesProcessed) - require.Equal(t, int64(0), summary.TotalLinesProcessed) - require.Equal(t, int64(0), summary.TotalPostFilterLines) - }) - - t.Run("computes bytes and lines from DataObjScan regions", func(t *testing.T) { - ctx, capture := NewCapture(context.Background(), nil) - - // Create DataObjScan regions with observations using registry stats - _, region1 := StartRegion(ctx, "DataObjScan") - region1.Record(StatPrimaryColumnUncompressedBytes.Observe(1000)) - region1.Record(StatSecondaryColumnUncompressedBytes.Observe(500)) - region1.Record(StatPrimaryRowsRead.Observe(100)) - region1.Record(StatSecondaryRowsRead.Observe(50)) - region1.Record(StatRowsOut.Observe(80)) - region1.End() - - _, region2 := StartRegion(ctx, "DataObjScan") - region2.Record(StatPrimaryColumnUncompressedBytes.Observe(2000)) - region2.Record(StatSecondaryColumnUncompressedBytes.Observe(1000)) - region2.Record(StatPrimaryRowsRead.Observe(200)) - region2.Record(StatSecondaryRowsRead.Observe(100)) - region2.Record(StatRowsOut.Observe(150)) - region2.End() - - capture.End() - - execTime := 2 * time.Second - queueTime := 100 * time.Millisecond - entriesReturned := 42 - - summary := capture.ToStatsSummary(execTime, queueTime, entriesReturned) - - require.Equal(t, execTime.Seconds(), summary.ExecTime) - require.Equal(t, queueTime.Seconds(), summary.QueueTime) - require.Equal(t, int64(entriesReturned), summary.TotalEntriesReturned) - - // TotalBytesProcessed = primary + secondary = (1000+2000) + (500+1000) = 4500 - require.Equal(t, int64(4500), summary.TotalBytesProcessed) - - // TotalLinesProcessed = primary_rows_read = 100 + 200 = 300 - require.Equal(t, int64(300), summary.TotalLinesProcessed) - - // TotalPostFilterLines = rows_out = 80 + 150 = 230 - require.Equal(t, int64(230), summary.TotalPostFilterLines) - - // BytesProcessedPerSecond = 4500 / 2 = 2250 - require.Equal(t, int64(2250), summary.BytesProcessedPerSecond) - - // LinesProcessedPerSecond = 300 / 2 = 150 - require.Equal(t, int64(150), summary.LinesProcessedPerSecond) - }) - - t.Run("ignores non-DataObjScan regions", func(t *testing.T) { - ctx, capture := NewCapture(context.Background(), nil) - - // DataObjScan region - should be counted - _, scanRegion := StartRegion(ctx, "DataObjScan") - scanRegion.Record(StatPrimaryColumnUncompressedBytes.Observe(1000)) - scanRegion.End() - - // Other region - should be ignored - _, otherRegion := StartRegion(ctx, "OtherRegion") - otherRegion.Record(StatPrimaryColumnUncompressedBytes.Observe(5000)) - otherRegion.End() - - capture.End() - - summary := capture.ToStatsSummary(time.Second, 0, 0) - - // Only DataObjScan region bytes counted - require.Equal(t, int64(1000), summary.TotalBytesProcessed) - }) - - t.Run("zero exec time results in zero per-second rates", func(t *testing.T) { - ctx, capture := NewCapture(context.Background(), nil) - - _, region := StartRegion(ctx, "DataObjScan") - region.Record(StatPrimaryColumnUncompressedBytes.Observe(1000)) - region.End() - capture.End() - - summary := capture.ToStatsSummary(0, 0, 10) - - require.Equal(t, int64(1000), summary.TotalBytesProcessed) - require.Equal(t, int64(0), summary.BytesProcessedPerSecond) - require.Equal(t, int64(0), summary.LinesProcessedPerSecond) - }) - - t.Run("missing statistics result in zero values", func(t *testing.T) { - ctx, capture := NewCapture(context.Background(), nil) - - // Only record some statistics - _, region := StartRegion(ctx, "DataObjScan") - region.Record(StatPrimaryColumnUncompressedBytes.Observe(1000)) - region.End() - capture.End() - - execTime := 1 * time.Second - summary := capture.ToStatsSummary(execTime, 0, 0) - - // Only primary bytes recorded - require.Equal(t, int64(1000), summary.TotalBytesProcessed) - // No rows recorded - require.Equal(t, int64(0), summary.TotalLinesProcessed) - require.Equal(t, int64(0), summary.TotalPostFilterLines) - }) - - t.Run("rolls up child region observations into DataObjScan", func(t *testing.T) { - ctx, capture := NewCapture(context.Background(), nil) - - // Parent DataObjScan region - ctx, parent := StartRegion(ctx, "DataObjScan") - parent.Record(StatPrimaryColumnUncompressedBytes.Observe(500)) - - // Child region (should be rolled up into parent) - _, child := StartRegion(ctx, "child_operation") - child.Record(StatPrimaryColumnUncompressedBytes.Observe(300)) - child.End() - - parent.End() - capture.End() - - summary := capture.ToStatsSummary(time.Second, 0, 0) - - // Child observations rolled up into DataObjScan: 500 + 300 = 800 - require.Equal(t, int64(800), summary.TotalBytesProcessed) - }) - - t.Run("empty capture returns zero stats", func(t *testing.T) { - _, capture := NewCapture(context.Background(), nil) - capture.End() - - summary := capture.ToStatsSummary(time.Second, time.Millisecond, 10) - - require.Equal(t, float64(1), summary.ExecTime) - require.Equal(t, float64(0.001), summary.QueueTime) - require.Equal(t, int64(10), summary.TotalEntriesReturned) - require.Equal(t, int64(0), summary.TotalBytesProcessed) - require.Equal(t, int64(0), summary.TotalLinesProcessed) - }) -} diff --git a/pkg/xcap/summary.go b/pkg/xcap/summary.go index d08bfab0caf4d..680792b29807b 100644 --- a/pkg/xcap/summary.go +++ b/pkg/xcap/summary.go @@ -6,6 +6,7 @@ import ( "time" "github.com/dustin/go-humanize" + "github.com/grafana/loki/v3/pkg/logqlmodel/stats" ) diff --git a/pkg/xcap/summary_test.go b/pkg/xcap/summary_test.go index d640d0bdd3b31..1544e7ae5acc5 100644 --- a/pkg/xcap/summary_test.go +++ b/pkg/xcap/summary_test.go @@ -3,6 +3,7 @@ package xcap import ( "context" "testing" + "time" "github.com/stretchr/testify/require" ) @@ -120,3 +121,112 @@ func TestRollups(t *testing.T) { require.Equal(t, int64(11), stats.data[stat.Key()].Value) // 1 + 10, excludes 100 + 1000 }) } + +func TestToStatsSummary(t *testing.T) { + t.Run("nil capture returns empty summary with provided values", func(t *testing.T) { + execTime := 2 * time.Second + queueTime := 100 * time.Millisecond + entriesReturned := 42 + + var capture *Capture + summary := capture.ToStatsSummary(execTime, queueTime, entriesReturned) + + require.Equal(t, execTime.Seconds(), summary.ExecTime) + require.Equal(t, queueTime.Seconds(), summary.QueueTime) + require.Equal(t, int64(entriesReturned), summary.TotalEntriesReturned) + require.Equal(t, int64(0), summary.TotalBytesProcessed) + require.Equal(t, int64(0), summary.TotalLinesProcessed) + require.Equal(t, int64(0), summary.TotalPostFilterLines) + }) + + t.Run("computes bytes and lines from DataObjScan regions", func(t *testing.T) { + ctx, capture := NewCapture(context.Background(), nil) + + // Create DataObjScan regions with observations using registry stats + _, region1 := StartRegion(ctx, "DataObjScan") + region1.Record(StatPrimaryColumnUncompressedBytes.Observe(1000)) + region1.Record(StatSecondaryColumnUncompressedBytes.Observe(500)) + region1.Record(StatPrimaryRowsRead.Observe(100)) + region1.Record(StatRowsOut.Observe(80)) + region1.End() + + _, region2 := StartRegion(ctx, "DataObjScan") + region2.Record(StatPrimaryColumnUncompressedBytes.Observe(2000)) + region2.Record(StatSecondaryColumnUncompressedBytes.Observe(1000)) + region2.Record(StatPrimaryRowsRead.Observe(200)) + region2.Record(StatRowsOut.Observe(150)) + region2.End() + + // Other region - should be ignored + _, otherRegion := StartRegion(ctx, "OtherRegion") + otherRegion.Record(StatPrimaryColumnUncompressedBytes.Observe(5000)) + otherRegion.End() + + capture.End() + + execTime := 2 * time.Second + queueTime := 100 * time.Millisecond + entriesReturned := 42 + + summary := capture.ToStatsSummary(execTime, queueTime, entriesReturned) + + require.Equal(t, execTime.Seconds(), summary.ExecTime) + require.Equal(t, queueTime.Seconds(), summary.QueueTime) + require.Equal(t, int64(entriesReturned), summary.TotalEntriesReturned) + + // TotalBytesProcessed = primary + secondary = (1000+2000) + (500+1000) = 4500 + require.Equal(t, int64(4500), summary.TotalBytesProcessed) + + // TotalLinesProcessed = primary_rows_read = 100 + 200 = 300 + require.Equal(t, int64(300), summary.TotalLinesProcessed) + + // TotalPostFilterLines = rows_out = 80 + 150 = 230 + require.Equal(t, int64(230), summary.TotalPostFilterLines) + + // BytesProcessedPerSecond = 4500 / 2 = 2250 + require.Equal(t, int64(2250), summary.BytesProcessedPerSecond) + + // LinesProcessedPerSecond = 300 / 2 = 150 + require.Equal(t, int64(150), summary.LinesProcessedPerSecond) + }) + + t.Run("missing statistics result in zero values", func(t *testing.T) { + ctx, capture := NewCapture(context.Background(), nil) + + // Only record some statistics + _, region := StartRegion(ctx, "DataObjScan") + region.Record(StatPrimaryColumnUncompressedBytes.Observe(1000)) + region.End() + capture.End() + + execTime := 1 * time.Second + summary := capture.ToStatsSummary(execTime, 0, 0) + + // Only primary bytes recorded + require.Equal(t, int64(1000), summary.TotalBytesProcessed) + // No rows recorded + require.Equal(t, int64(0), summary.TotalLinesProcessed) + require.Equal(t, int64(0), summary.TotalPostFilterLines) + }) + + t.Run("rolls up child region observations into DataObjScan", func(t *testing.T) { + ctx, capture := NewCapture(context.Background(), nil) + + // Parent DataObjScan region + ctx, parent := StartRegion(ctx, "DataObjScan") + parent.Record(StatPrimaryColumnUncompressedBytes.Observe(500)) + + // Child region (should be rolled up into parent) + _, child := StartRegion(ctx, "child_operation") + child.Record(StatPrimaryColumnUncompressedBytes.Observe(300)) + child.End() + + parent.End() + capture.End() + + summary := capture.ToStatsSummary(time.Second, 0, 0) + + // Child observations rolled up into DataObjScan: 500 + 300 = 800 + require.Equal(t, int64(800), summary.TotalBytesProcessed) + }) +} From eca788baf745cc14429cf1962d4f66f7bbc6658f Mon Sep 17 00:00:00 2001 From: Ashwanth Goli Date: Thu, 4 Dec 2025 13:30:03 +0530 Subject: [PATCH 10/13] remove old stats collection --- pkg/engine/engine.go | 20 +++++------- pkg/engine/internal/worker/thread.go | 9 ++---- pkg/xcap/summary.go | 35 +++++++++++++-------- pkg/xcap/summary_test.go | 46 ++++++++++++++-------------- 4 files changed, 54 insertions(+), 56 deletions(-) diff --git a/pkg/engine/engine.go b/pkg/engine/engine.go index 095dc64f81d3b..f72e0d26ef1f7 100644 --- a/pkg/engine/engine.go +++ b/pkg/engine/engine.go @@ -29,7 +29,6 @@ import ( "github.com/grafana/loki/v3/pkg/logql/syntax" "github.com/grafana/loki/v3/pkg/logqlmodel" "github.com/grafana/loki/v3/pkg/logqlmodel/metadata" - "github.com/grafana/loki/v3/pkg/logqlmodel/stats" "github.com/grafana/loki/v3/pkg/storage/bucket" "github.com/grafana/loki/v3/pkg/util/httpreq" util_log "github.com/grafana/loki/v3/pkg/util/log" @@ -219,6 +218,11 @@ func (e *Engine) Execute(ctx context.Context, params logql.Params) (logqlmodel.R "duration_full", durFull, ) + // Close the pipeline to calculate the stats. + pipeline.Close() + + region.SetStatus(codes.Ok, "") + // explicitly call End() before exporting even though we have a defer above. // It is safe to call End() multiple times. region.End() @@ -227,17 +231,9 @@ func (e *Engine) Execute(ctx context.Context, params logql.Params) (logqlmodel.R level.Error(logger).Log("msg", "failed to export capture as trace", "err", err) } - // Close the pipeline to calculate the stats. - pipeline.Close() - - queueTime, _ := ctx.Value(httpreq.QueryQueueTimeHTTPHeader).(time.Duration) - statsCtx := stats.FromContext(ctx) - statsCtx.AddQuerierExecTime(durFull) - stats := statsCtx.Result(durFull, queueTime, builder.Len()) + // TODO: capture and report queue time md := metadata.FromContext(ctx) - - region.SetStatus(codes.Ok, "") - stats.Summary = capture.ToStatsSummary(durFull, 0, builder.Len()) + stats := capture.ToStatsSummary(durFull, 0, builder.Len()) result := builder.Build(stats, md) logql.RecordRangeAndInstantQueryMetrics(ctx, logger, params, strconv.Itoa(http.StatusOK), stats, result.Data) @@ -246,14 +242,12 @@ func (e *Engine) Execute(ctx context.Context, params logql.Params) (logqlmodel.R // buildContext initializes a request-scoped context prior to execution. func (e *Engine) buildContext(ctx context.Context) context.Context { - statsContext, ctx := stats.NewContext(ctx) metadataContext, ctx := metadata.NewContext(ctx) // Inject the range config into the context for any calls to // [rangeio.ReadRanges] to make use of. ctx = rangeio.WithConfig(ctx, &e.rangeConfig) - statsContext.SetQueryUsedV2Engine() metadataContext.AddWarning("Query was executed using the new experimental query engine and dataobj storage.") return ctx } diff --git a/pkg/engine/internal/worker/thread.go b/pkg/engine/internal/worker/thread.go index bc230525e5be9..e1325123e0ec4 100644 --- a/pkg/engine/internal/worker/thread.go +++ b/pkg/engine/internal/worker/thread.go @@ -16,7 +16,6 @@ import ( "github.com/grafana/loki/v3/pkg/engine/internal/planner/physical" "github.com/grafana/loki/v3/pkg/engine/internal/scheduler/wire" "github.com/grafana/loki/v3/pkg/engine/internal/workflow" - "github.com/grafana/loki/v3/pkg/logqlmodel/stats" "github.com/grafana/loki/v3/pkg/storage/bucket" utillog "github.com/grafana/loki/v3/pkg/util/log" "github.com/grafana/loki/v3/pkg/xcap" @@ -158,7 +157,6 @@ func (t *thread) runJob(ctx context.Context, job *threadJob) { }, } - statsCtx, ctx := stats.NewContext(ctx) ctx = user.InjectOrgID(ctx, job.Task.TenantID) ctx, capture := xcap.NewCapture(ctx, nil) @@ -199,8 +197,7 @@ func (t *thread) runJob(ctx context.Context, job *threadJob) { level.Warn(logger).Log("msg", "failed to inform scheduler of task status", "err", err) } - var totalRows int - totalRows, err = t.drainPipeline(ctx, pipeline, job, logger) + _, err = t.drainPipeline(ctx, pipeline, job, logger) if err != nil { level.Warn(logger).Log("msg", "task failed", "err", err) _ = job.Scheduler.SendMessageAsync(ctx, wire.TaskStatusMessage{ @@ -229,8 +226,6 @@ func (t *thread) runJob(ctx context.Context, job *threadJob) { // to finalize the capture before it's included in the TaskStatusMessage. capture.End() - // TODO(rfratto): We should find a way to expose queue time here. - result := statsCtx.Result(time.Since(startTime), 0, totalRows) level.Info(logger).Log("msg", "task completed", "duration", time.Since(startTime)) // Wait for the scheduler to confirm the task has completed before @@ -238,7 +233,7 @@ func (t *thread) runJob(ctx context.Context, job *threadJob) { // for how many threads have capacity for requesting tasks. err = job.Scheduler.SendMessage(ctx, wire.TaskStatusMessage{ ID: job.Task.ULID, - Status: workflow.TaskStatus{State: workflow.TaskStateCompleted, Statistics: &result, Capture: capture}, + Status: workflow.TaskStatus{State: workflow.TaskStateCompleted, Capture: capture}, }) if err != nil { level.Warn(logger).Log("msg", "failed to inform scheduler of task status", "err", err) diff --git a/pkg/xcap/summary.go b/pkg/xcap/summary.go index 680792b29807b..5cd8b4eca840d 100644 --- a/pkg/xcap/summary.go +++ b/pkg/xcap/summary.go @@ -243,16 +243,23 @@ func (c *observationCollector) rollUpObservations(region *Region, excludedSet ma // Region name for data object scan operations. const regionNameDataObjScan = "DataObjScan" -// ToStatsSummary computes a stats.Summary from observations in the capture. -func (c *Capture) ToStatsSummary(execTime, queueTime time.Duration, totalEntriesReturned int) stats.Summary { - summary := stats.Summary{ - ExecTime: execTime.Seconds(), - QueueTime: queueTime.Seconds(), - TotalEntriesReturned: int64(totalEntriesReturned), +// ToStatsSummary computes a stats.Result from observations in the capture. +func (c *Capture) ToStatsSummary(execTime, queueTime time.Duration, totalEntriesReturned int) stats.Result { + result := stats.Result{ + Summary: stats.Summary{ + ExecTime: execTime.Seconds(), + QueueTime: queueTime.Seconds(), + TotalEntriesReturned: int64(totalEntriesReturned), + }, + Querier: stats.Querier{ + Store: stats.Store{ + QueryUsedV2Engine: true, + }, + }, } if c == nil { - return summary + return result } // Collect observations from DataObjScan as the summary stats mainly relate to log lines. @@ -266,24 +273,26 @@ func (c *Capture) ToStatsSummary(execTime, queueTime time.Duration, totalEntries ) // TotalBytesProcessed: sum of uncompressed bytes from primary and secondary columns - summary.TotalBytesProcessed = readInt64(observations, StatPrimaryColumnUncompressedBytes.Key()) + + result.Summary.TotalBytesProcessed = readInt64(observations, StatPrimaryColumnUncompressedBytes.Key()) + readInt64(observations, StatSecondaryColumnUncompressedBytes.Key()) // TotalLinesProcessed: primary rows read - summary.TotalLinesProcessed = readInt64(observations, StatPrimaryRowsRead.Key()) + result.Summary.TotalLinesProcessed = readInt64(observations, StatPrimaryRowsRead.Key()) // TotalPostFilterLines: rows output after filtering // TODO: this will report the wrong value if the plan has a filter stage. // pick the min of row_out from filter and scan nodes. - summary.TotalPostFilterLines = readInt64(observations, StatRowsOut.Key()) + result.Summary.TotalPostFilterLines = readInt64(observations, StatRowsOut.Key()) + + // TODO: track and report TotalStructuredMetadataBytesProcessed if execTime > 0 { execSeconds := execTime.Seconds() - summary.BytesProcessedPerSecond = int64(float64(summary.TotalBytesProcessed) / execSeconds) - summary.LinesProcessedPerSecond = int64(float64(summary.TotalLinesProcessed) / execSeconds) + result.Summary.BytesProcessedPerSecond = int64(float64(result.Summary.TotalBytesProcessed) / execSeconds) + result.Summary.LinesProcessedPerSecond = int64(float64(result.Summary.TotalLinesProcessed) / execSeconds) } - return summary + return result } // readInt64 reads an int64 observation for the given stat key. diff --git a/pkg/xcap/summary_test.go b/pkg/xcap/summary_test.go index 1544e7ae5acc5..0db547e73fe71 100644 --- a/pkg/xcap/summary_test.go +++ b/pkg/xcap/summary_test.go @@ -129,14 +129,14 @@ func TestToStatsSummary(t *testing.T) { entriesReturned := 42 var capture *Capture - summary := capture.ToStatsSummary(execTime, queueTime, entriesReturned) - - require.Equal(t, execTime.Seconds(), summary.ExecTime) - require.Equal(t, queueTime.Seconds(), summary.QueueTime) - require.Equal(t, int64(entriesReturned), summary.TotalEntriesReturned) - require.Equal(t, int64(0), summary.TotalBytesProcessed) - require.Equal(t, int64(0), summary.TotalLinesProcessed) - require.Equal(t, int64(0), summary.TotalPostFilterLines) + result := capture.ToStatsSummary(execTime, queueTime, entriesReturned) + + require.Equal(t, execTime.Seconds(), result.Summary.ExecTime) + require.Equal(t, queueTime.Seconds(), result.Summary.QueueTime) + require.Equal(t, int64(entriesReturned), result.Summary.TotalEntriesReturned) + require.Equal(t, int64(0), result.Summary.TotalBytesProcessed) + require.Equal(t, int64(0), result.Summary.TotalLinesProcessed) + require.Equal(t, int64(0), result.Summary.TotalPostFilterLines) }) t.Run("computes bytes and lines from DataObjScan regions", func(t *testing.T) { @@ -168,26 +168,26 @@ func TestToStatsSummary(t *testing.T) { queueTime := 100 * time.Millisecond entriesReturned := 42 - summary := capture.ToStatsSummary(execTime, queueTime, entriesReturned) + result := capture.ToStatsSummary(execTime, queueTime, entriesReturned) - require.Equal(t, execTime.Seconds(), summary.ExecTime) - require.Equal(t, queueTime.Seconds(), summary.QueueTime) - require.Equal(t, int64(entriesReturned), summary.TotalEntriesReturned) + require.Equal(t, execTime.Seconds(), result.Summary.ExecTime) + require.Equal(t, queueTime.Seconds(), result.Summary.QueueTime) + require.Equal(t, int64(entriesReturned), result.Summary.TotalEntriesReturned) // TotalBytesProcessed = primary + secondary = (1000+2000) + (500+1000) = 4500 - require.Equal(t, int64(4500), summary.TotalBytesProcessed) + require.Equal(t, int64(4500), result.Summary.TotalBytesProcessed) // TotalLinesProcessed = primary_rows_read = 100 + 200 = 300 - require.Equal(t, int64(300), summary.TotalLinesProcessed) + require.Equal(t, int64(300), result.Summary.TotalLinesProcessed) // TotalPostFilterLines = rows_out = 80 + 150 = 230 - require.Equal(t, int64(230), summary.TotalPostFilterLines) + require.Equal(t, int64(230), result.Summary.TotalPostFilterLines) // BytesProcessedPerSecond = 4500 / 2 = 2250 - require.Equal(t, int64(2250), summary.BytesProcessedPerSecond) + require.Equal(t, int64(2250), result.Summary.BytesProcessedPerSecond) // LinesProcessedPerSecond = 300 / 2 = 150 - require.Equal(t, int64(150), summary.LinesProcessedPerSecond) + require.Equal(t, int64(150), result.Summary.LinesProcessedPerSecond) }) t.Run("missing statistics result in zero values", func(t *testing.T) { @@ -200,13 +200,13 @@ func TestToStatsSummary(t *testing.T) { capture.End() execTime := 1 * time.Second - summary := capture.ToStatsSummary(execTime, 0, 0) + result := capture.ToStatsSummary(execTime, 0, 0) // Only primary bytes recorded - require.Equal(t, int64(1000), summary.TotalBytesProcessed) + require.Equal(t, int64(1000), result.Summary.TotalBytesProcessed) // No rows recorded - require.Equal(t, int64(0), summary.TotalLinesProcessed) - require.Equal(t, int64(0), summary.TotalPostFilterLines) + require.Equal(t, int64(0), result.Summary.TotalLinesProcessed) + require.Equal(t, int64(0), result.Summary.TotalPostFilterLines) }) t.Run("rolls up child region observations into DataObjScan", func(t *testing.T) { @@ -224,9 +224,9 @@ func TestToStatsSummary(t *testing.T) { parent.End() capture.End() - summary := capture.ToStatsSummary(time.Second, 0, 0) + result := capture.ToStatsSummary(time.Second, 0, 0) // Child observations rolled up into DataObjScan: 500 + 300 = 800 - require.Equal(t, int64(800), summary.TotalBytesProcessed) + require.Equal(t, int64(800), result.Summary.TotalBytesProcessed) }) } From d19972508365cb2b4581774df6102ef05c5a122f Mon Sep 17 00:00:00 2001 From: Ashwanth Goli Date: Thu, 4 Dec 2025 15:49:50 +0530 Subject: [PATCH 11/13] remove empty test file --- pkg/xcap/stats_summary_test.go | 1 - 1 file changed, 1 deletion(-) delete mode 100644 pkg/xcap/stats_summary_test.go diff --git a/pkg/xcap/stats_summary_test.go b/pkg/xcap/stats_summary_test.go deleted file mode 100644 index c7a71730da29d..0000000000000 --- a/pkg/xcap/stats_summary_test.go +++ /dev/null @@ -1 +0,0 @@ -package xcap From 8edc55bc98e178dceb26e0f2e81ee09dc6117216 Mon Sep 17 00:00:00 2001 From: Ashwanth Goli Date: Thu, 4 Dec 2025 18:06:55 +0530 Subject: [PATCH 12/13] rename stats --- pkg/dataobj/internal/dataset/reader.go | 24 +++---- .../internal/dataset/reader_downloader.go | 18 +++--- pkg/dataobj/internal/dataset/reader_test.go | 18 +++--- pkg/dataobj/metastore/object.go | 4 +- .../sections/internal/columnar/decoder.go | 4 +- pkg/engine/internal/executor/pipeline.go | 6 +- pkg/util/rangeio/rangeio.go | 10 +-- pkg/xcap/exporter.go | 24 +++---- pkg/xcap/stats_definitions.go | 64 +++++++++---------- pkg/xcap/summary.go | 16 ++--- pkg/xcap/summary_test.go | 24 +++---- 11 files changed, 105 insertions(+), 107 deletions(-) diff --git a/pkg/dataobj/internal/dataset/reader.go b/pkg/dataobj/internal/dataset/reader.go index 7a47ea2939f59..b9da611971bd9 100644 --- a/pkg/dataobj/internal/dataset/reader.go +++ b/pkg/dataobj/internal/dataset/reader.go @@ -149,8 +149,8 @@ func (r *Reader) Read(ctx context.Context, s []Row) (int, error) { primaryColumnBytes += s[i].Size() } - r.region.Record(xcap.StatPrimaryRowsRead.Observe(int64(rowsRead))) - r.region.Record(xcap.StatPrimaryRowBytes.Observe(primaryColumnBytes)) + r.region.Record(xcap.StatDatasetPrimaryRowsRead.Observe(int64(rowsRead))) + r.region.Record(xcap.StatDatasetPrimaryRowBytes.Observe(primaryColumnBytes)) } else { rowsRead, passCount, err = r.readAndFilterPrimaryColumns(ctx, readSize, s[:readSize]) if err != nil { @@ -182,8 +182,8 @@ func (r *Reader) Read(ctx context.Context, s []Row) (int, error) { totalBytesFilled += s[i].Size() - s[i].SizeOfColumns(r.primaryColumnIndexes) } - r.region.Record(xcap.StatSecondaryRowsRead.Observe(int64(count))) - r.region.Record(xcap.StatSecondaryRowBytes.Observe(totalBytesFilled)) + r.region.Record(xcap.StatDatasetSecondaryRowsRead.Observe(int64(count))) + r.region.Record(xcap.StatDatasetSecondaryRowBytes.Observe(totalBytesFilled)) } // We only advance r.row after we successfully read and filled rows. This @@ -268,8 +268,8 @@ func (r *Reader) readAndFilterPrimaryColumns(ctx context.Context, readSize int, readSize = passCount } - r.region.Record(xcap.StatPrimaryRowsRead.Observe(int64(rowsRead))) - r.region.Record(xcap.StatPrimaryRowBytes.Observe(primaryColumnBytes)) + r.region.Record(xcap.StatDatasetPrimaryRowsRead.Observe(int64(rowsRead))) + r.region.Record(xcap.StatDatasetPrimaryRowBytes.Observe(primaryColumnBytes)) return rowsRead, passCount, nil } @@ -555,11 +555,11 @@ func (r *Reader) initDownloader(ctx context.Context) error { if primary { r.primaryColumnIndexes = append(r.primaryColumnIndexes, i) - r.region.Record(xcap.StatPrimaryColumns.Observe(1)) - r.region.Record(xcap.StatPrimaryColumnPages.Observe(int64(column.ColumnDesc().PagesCount))) + r.region.Record(xcap.StatDatasetPrimaryColumns.Observe(1)) + r.region.Record(xcap.StatDatasetPrimaryColumnPages.Observe(int64(column.ColumnDesc().PagesCount))) } else { - r.region.Record(xcap.StatSecondaryColumns.Observe(1)) - r.region.Record(xcap.StatSecondaryColumnPages.Observe(int64(column.ColumnDesc().PagesCount))) + r.region.Record(xcap.StatDatasetSecondaryColumns.Observe(1)) + r.region.Record(xcap.StatDatasetSecondaryColumnPages.Observe(int64(column.ColumnDesc().PagesCount))) } } @@ -593,8 +593,8 @@ func (r *Reader) initDownloader(ctx context.Context) error { rowsCount = max(rowsCount, uint64(column.ColumnDesc().RowsCount)) } - r.region.Record(xcap.StatMaxRows.Observe(int64(rowsCount))) - r.region.Record(xcap.StatRowsAfterPruning.Observe(int64(ranges.TotalRowCount()))) + r.region.Record(xcap.StatDatasetMaxRows.Observe(int64(rowsCount))) + r.region.Record(xcap.StatDatasetRowsAfterPruning.Observe(int64(ranges.TotalRowCount()))) return nil } diff --git a/pkg/dataobj/internal/dataset/reader_downloader.go b/pkg/dataobj/internal/dataset/reader_downloader.go index daed18cf6016f..b0f5d1cbc0254 100644 --- a/pkg/dataobj/internal/dataset/reader_downloader.go +++ b/pkg/dataobj/internal/dataset/reader_downloader.go @@ -232,13 +232,13 @@ func (dl *readerDownloader) downloadBatch(ctx context.Context, requestor *reader if region := xcap.RegionFromContext(ctx); region != nil { for _, page := range batch { if page.column.primary { - region.Record(xcap.StatPrimaryPagesDownloaded.Observe(1)) - region.Record(xcap.StatPrimaryColumnBytes.Observe(int64(page.inner.PageDesc().CompressedSize))) - region.Record(xcap.StatPrimaryColumnUncompressedBytes.Observe(int64(page.inner.PageDesc().UncompressedSize))) + region.Record(xcap.StatDatasetPrimaryPagesDownloaded.Observe(1)) + region.Record(xcap.StatDatasetPrimaryColumnBytes.Observe(int64(page.inner.PageDesc().CompressedSize))) + region.Record(xcap.StatDatasetPrimaryColumnUncompressedBytes.Observe(int64(page.inner.PageDesc().UncompressedSize))) } else { - region.Record(xcap.StatSecondaryPagesDownloaded.Observe(1)) - region.Record(xcap.StatSecondaryColumnBytes.Observe(int64(page.inner.PageDesc().CompressedSize))) - region.Record(xcap.StatSecondaryColumnUncompressedBytes.Observe(int64(page.inner.PageDesc().UncompressedSize))) + region.Record(xcap.StatDatasetSecondaryPagesDownloaded.Observe(1)) + region.Record(xcap.StatDatasetSecondaryColumnBytes.Observe(int64(page.inner.PageDesc().CompressedSize))) + region.Record(xcap.StatDatasetSecondaryColumnUncompressedBytes.Observe(int64(page.inner.PageDesc().UncompressedSize))) } } } @@ -589,13 +589,13 @@ func (page *readerPage) PageDesc() *PageDesc { func (page *readerPage) ReadPage(ctx context.Context) (PageData, error) { region := xcap.RegionFromContext(ctx) - region.Record(xcap.StatPagesScanned.Observe(1)) + region.Record(xcap.StatDatasetPagesScanned.Observe(1)) if page.data != nil { - region.Record(xcap.StatPagesFoundInCache.Observe(1)) + region.Record(xcap.StatDatasetPagesFoundInCache.Observe(1)) return page.data, nil } - region.Record(xcap.StatPageDownloadRequests.Observe(1)) + region.Record(xcap.StatDatasetPageDownloadRequests.Observe(1)) if err := page.column.dl.downloadBatch(ctx, page); err != nil { return nil, err } diff --git a/pkg/dataobj/internal/dataset/reader_test.go b/pkg/dataobj/internal/dataset/reader_test.go index 701d22817a90a..a304d7253f6fd 100644 --- a/pkg/dataobj/internal/dataset/reader_test.go +++ b/pkg/dataobj/internal/dataset/reader_test.go @@ -918,13 +918,13 @@ func Test_Reader_Stats(t *testing.T) { } require.Equal(t, int64(2), obsMap[xcap.StatDatasetReadCalls.Name()]) - require.Equal(t, int64(2), obsMap[xcap.StatPrimaryColumns.Name()]) - require.Equal(t, int64(2), obsMap[xcap.StatSecondaryColumns.Name()]) - require.Equal(t, int64(5), obsMap[xcap.StatPrimaryColumnPages.Name()]) - require.Equal(t, int64(8), obsMap[xcap.StatSecondaryColumnPages.Name()]) - - require.Equal(t, int64(len(basicReaderTestData)), obsMap[xcap.StatMaxRows.Name()]) - require.Equal(t, int64(3), obsMap[xcap.StatRowsAfterPruning.Name()]) - require.Equal(t, int64(3), obsMap[xcap.StatPrimaryRowsRead.Name()]) - require.Equal(t, int64(1), obsMap[xcap.StatSecondaryRowsRead.Name()]) + require.Equal(t, int64(2), obsMap[xcap.StatDatasetPrimaryColumns.Name()]) + require.Equal(t, int64(2), obsMap[xcap.StatDatasetSecondaryColumns.Name()]) + require.Equal(t, int64(5), obsMap[xcap.StatDatasetPrimaryColumnPages.Name()]) + require.Equal(t, int64(8), obsMap[xcap.StatDatasetSecondaryColumnPages.Name()]) + + require.Equal(t, int64(len(basicReaderTestData)), obsMap[xcap.StatDatasetMaxRows.Name()]) + require.Equal(t, int64(3), obsMap[xcap.StatDatasetRowsAfterPruning.Name()]) + require.Equal(t, int64(3), obsMap[xcap.StatDatasetPrimaryRowsRead.Name()]) + require.Equal(t, int64(1), obsMap[xcap.StatDatasetSecondaryRowsRead.Name()]) } diff --git a/pkg/dataobj/metastore/object.go b/pkg/dataobj/metastore/object.go index 3efbf0b1a69f7..4d8662f7a9e0a 100644 --- a/pkg/dataobj/metastore/object.go +++ b/pkg/dataobj/metastore/object.go @@ -188,7 +188,7 @@ func (m *ObjectMetastore) Sections(ctx context.Context, start, end time.Time, ma } m.metrics.indexObjectsTotal.Observe(float64(len(indexPaths))) - region.Record(xcap.StatIndexObjects.Observe(int64(len(indexPaths)))) + region.Record(xcap.StatMetastoreIndexObjects.Observe(int64(len(indexPaths)))) // Return early if no index files are found if len(indexPaths) == 0 { @@ -245,7 +245,7 @@ func (m *ObjectMetastore) Sections(ctx context.Context, start, end time.Time, ma duration := sectionsTimer.ObserveDuration() m.metrics.resolvedSectionsTotal.Observe(float64(len(streamSectionPointers))) m.metrics.resolvedSectionsRatio.Observe(float64(len(streamSectionPointers)) / float64(initialSectionPointersCount)) - region.Record(xcap.StatResolvedSections.Observe(int64(len(streamSectionPointers)))) + region.Record(xcap.StatMetastoreResolvedSections.Observe(int64(len(streamSectionPointers)))) level.Debug(utillog.WithContext(ctx, m.logger)).Log( "msg", "resolved sections", diff --git a/pkg/dataobj/sections/internal/columnar/decoder.go b/pkg/dataobj/sections/internal/columnar/decoder.go index 88650a24644b1..b4b47df5034a0 100644 --- a/pkg/dataobj/sections/internal/columnar/decoder.go +++ b/pkg/dataobj/sections/internal/columnar/decoder.go @@ -79,7 +79,7 @@ func (dec *Decoder) Pages(ctx context.Context, columns []*datasetmd.ColumnDesc) region := xcap.RegionFromContext(ctx) startTime := time.Now() defer func() { - region.Record(xcap.StatPageDownloadTime.Observe(time.Since(startTime).Nanoseconds())) + region.Record(xcap.StatDatasetPageDownloadTime.Observe(time.Since(startTime).Nanoseconds())) }() ranges := make([]rangeio.Range, 0, len(columns)) @@ -135,7 +135,7 @@ func (dec *Decoder) ReadPages(ctx context.Context, pages []*datasetmd.PageDesc) region := xcap.RegionFromContext(ctx) startTime := time.Now() defer func() { - region.Record(xcap.StatPageDownloadTime.Observe(time.Since(startTime).Nanoseconds())) + region.Record(xcap.StatDatasetPageDownloadTime.Observe(time.Since(startTime).Nanoseconds())) }() ranges := make([]rangeio.Range, 0, len(pages)) diff --git a/pkg/engine/internal/executor/pipeline.go b/pkg/engine/internal/executor/pipeline.go index 3948dbcddcf77..c75ef51472625 100644 --- a/pkg/engine/internal/executor/pipeline.go +++ b/pkg/engine/internal/executor/pipeline.go @@ -337,17 +337,17 @@ func (p *observedPipeline) Read(ctx context.Context) (arrow.RecordBatch, error) start := time.Now() if p.region != nil { - p.region.Record(xcap.StatReadCalls.Observe(1)) + p.region.Record(xcap.StatPipelineReadCalls.Observe(1)) } rec, err := p.inner.Read(ctx) if p.region != nil { if rec != nil { - p.region.Record(xcap.StatRowsOut.Observe(rec.NumRows())) + p.region.Record(xcap.StatPipelineRowsOut.Observe(rec.NumRows())) } - p.region.Record(xcap.StatReadDuration.Observe(time.Since(start).Nanoseconds())) + p.region.Record(xcap.StatPipelineReadDuration.Observe(time.Since(start).Nanoseconds())) } return rec, err diff --git a/pkg/util/rangeio/rangeio.go b/pkg/util/rangeio/rangeio.go index 548667bc2e72d..bc577a78bd380 100644 --- a/pkg/util/rangeio/rangeio.go +++ b/pkg/util/rangeio/rangeio.go @@ -429,16 +429,16 @@ func recordRangeStats(ranges, optimizedRanges []Range, region *xcap.Region) { origSize := rangesSize(ranges) optimizedSize := rangesSize(optimizedRanges) - region.Record(xcap.StatInputRangesCount.Observe(int64(len(ranges)))) - region.Record(xcap.StatInputRangesSize.Observe(int64(origSize))) - region.Record(xcap.StatOptimizedRangesCount.Observe(int64(len(optimizedRanges)))) - region.Record(xcap.StatOptimizedRangesSize.Observe(int64(optimizedSize))) + region.Record(xcap.StatRangeIOInputCount.Observe(int64(len(ranges)))) + region.Record(xcap.StatRangeIOInputSize.Observe(int64(origSize))) + region.Record(xcap.StatRangeIOOptimizedCount.Observe(int64(len(optimizedRanges)))) + region.Record(xcap.StatRangeIOOptimizedSize.Observe(int64(optimizedSize))) } func recordThroughputStat(region *xcap.Region, startTime time.Time, optimizedRanges []Range) { size := rangesSize(optimizedRanges) bytesPerSec := float64(size) / time.Since(startTime).Seconds() - region.Record(xcap.StatOptimizedThroughput.Observe(bytesPerSec)) + region.Record(xcap.StatRangeIOThroughput.Observe(bytesPerSec)) } type bytesStringer uint64 diff --git a/pkg/xcap/exporter.go b/pkg/xcap/exporter.go index 7ad8fc18623c5..bb68eab72bb0b 100644 --- a/pkg/xcap/exporter.go +++ b/pkg/xcap/exporter.go @@ -144,13 +144,13 @@ func summarizeObservations(capture *Capture) *observations { // object store calls StatBucketGet.Key(), StatBucketGetRange.Key(), StatBucketAttributes.Key(), // dataset reader stats - StatMaxRows.Key(), StatRowsAfterPruning.Key(), StatDatasetReadCalls.Key(), - StatPrimaryPagesDownloaded.Key(), StatSecondaryPagesDownloaded.Key(), - StatPrimaryColumnBytes.Key(), StatSecondaryColumnBytes.Key(), - StatPrimaryRowsRead.Key(), StatSecondaryRowsRead.Key(), - StatPrimaryRowBytes.Key(), StatSecondaryRowBytes.Key(), - StatPagesScanned.Key(), StatPagesFoundInCache.Key(), - StatPageDownloadRequests.Key(), StatPageDownloadTime.Key(), + StatDatasetMaxRows.Key(), StatDatasetRowsAfterPruning.Key(), StatDatasetReadCalls.Key(), + StatDatasetPrimaryPagesDownloaded.Key(), StatDatasetSecondaryPagesDownloaded.Key(), + StatDatasetPrimaryColumnBytes.Key(), StatDatasetSecondaryColumnBytes.Key(), + StatDatasetPrimaryRowsRead.Key(), StatDatasetSecondaryRowsRead.Key(), + StatDatasetPrimaryRowBytes.Key(), StatDatasetSecondaryRowBytes.Key(), + StatDatasetPagesScanned.Key(), StatDatasetPagesFoundInCache.Key(), + StatDatasetPageDownloadRequests.Key(), StatDatasetPageDownloadTime.Key(), ). prefix("logs_dataset_"). normalizeKeys(), @@ -159,7 +159,7 @@ func summarizeObservations(capture *Capture) *observations { // metastore index and resolved section stats result.merge( collect.fromRegions("ObjectMetastore.Sections", true). - filter(StatIndexObjects.Key(), StatResolvedSections.Key()). + filter(StatMetastoreIndexObjects.Key(), StatMetastoreResolvedSections.Key()). normalizeKeys(), ) @@ -168,8 +168,8 @@ func summarizeObservations(capture *Capture) *observations { collect.fromRegions("ObjectMetastore.Sections", true). filter( StatBucketGet.Key(), StatBucketGetRange.Key(), StatBucketAttributes.Key(), - StatPrimaryPagesDownloaded.Key(), StatSecondaryPagesDownloaded.Key(), - StatPrimaryColumnBytes.Key(), StatSecondaryColumnBytes.Key(), + StatDatasetPrimaryPagesDownloaded.Key(), StatDatasetSecondaryPagesDownloaded.Key(), + StatDatasetPrimaryColumnBytes.Key(), StatDatasetSecondaryColumnBytes.Key(), ). prefix("metastore_"). normalizeKeys(), @@ -180,8 +180,8 @@ func summarizeObservations(capture *Capture) *observations { collect.fromRegions("streamsView.init", true). filter( StatBucketGet.Key(), StatBucketGetRange.Key(), StatBucketAttributes.Key(), - StatPrimaryPagesDownloaded.Key(), StatSecondaryPagesDownloaded.Key(), - StatPrimaryColumnBytes.Key(), StatSecondaryColumnBytes.Key(), + StatDatasetPrimaryPagesDownloaded.Key(), StatDatasetSecondaryPagesDownloaded.Key(), + StatDatasetPrimaryColumnBytes.Key(), StatDatasetSecondaryColumnBytes.Key(), ). prefix("streams_"). normalizeKeys(), diff --git a/pkg/xcap/stats_definitions.go b/pkg/xcap/stats_definitions.go index 4aab20ea38ffd..30808e2147200 100644 --- a/pkg/xcap/stats_definitions.go +++ b/pkg/xcap/stats_definitions.go @@ -2,9 +2,9 @@ package xcap // Common pipeline statistics tracked across executor nodes. var ( - StatRowsOut = NewStatisticInt64("rows.out", AggregationTypeSum) - StatReadCalls = NewStatisticInt64("read.calls", AggregationTypeSum) - StatReadDuration = NewStatisticInt64("read.duration.ns", AggregationTypeSum) + StatPipelineRowsOut = NewStatisticInt64("rows.out", AggregationTypeSum) + StatPipelineReadCalls = NewStatisticInt64("read.calls", AggregationTypeSum) + StatPipelineReadDuration = NewStatisticInt64("read.duration.ns", AggregationTypeSum) ) // ColumnCompat statistics. @@ -14,46 +14,44 @@ var ( var ( // Dataset column statistics. - StatPrimaryColumns = NewStatisticInt64("primary.columns", AggregationTypeSum) - StatSecondaryColumns = NewStatisticInt64("secondary.columns", AggregationTypeSum) - StatPrimaryColumnPages = NewStatisticInt64("primary.column.pages", AggregationTypeSum) - StatSecondaryColumnPages = NewStatisticInt64("secondary.column.pages", AggregationTypeSum) + StatDatasetPrimaryColumns = NewStatisticInt64("primary.columns", AggregationTypeSum) + StatDatasetSecondaryColumns = NewStatisticInt64("secondary.columns", AggregationTypeSum) + StatDatasetPrimaryColumnPages = NewStatisticInt64("primary.column.pages", AggregationTypeSum) + StatDatasetSecondaryColumnPages = NewStatisticInt64("secondary.column.pages", AggregationTypeSum) // Dataset row statistics. - StatMaxRows = NewStatisticInt64("row.max", AggregationTypeSum) - StatRowsAfterPruning = NewStatisticInt64("rows.after.pruning", AggregationTypeSum) - StatPrimaryRowsRead = NewStatisticInt64("primary.rows.read", AggregationTypeSum) - StatSecondaryRowsRead = NewStatisticInt64("secondary.rows.read", AggregationTypeSum) - StatPrimaryRowBytes = NewStatisticInt64("primary.row.read.bytes", AggregationTypeSum) - StatSecondaryRowBytes = NewStatisticInt64("secondary.row.read.bytes", AggregationTypeSum) + StatDatasetMaxRows = NewStatisticInt64("rows.max", AggregationTypeSum) + StatDatasetRowsAfterPruning = NewStatisticInt64("rows.after.pruning", AggregationTypeSum) + StatDatasetPrimaryRowsRead = NewStatisticInt64("primary.rows.read", AggregationTypeSum) + StatDatasetSecondaryRowsRead = NewStatisticInt64("secondary.rows.read", AggregationTypeSum) + StatDatasetPrimaryRowBytes = NewStatisticInt64("primary.row.read.bytes", AggregationTypeSum) + StatDatasetSecondaryRowBytes = NewStatisticInt64("secondary.row.read.bytes", AggregationTypeSum) // Dataset page scan statistics. - StatPagesScanned = NewStatisticInt64("pages.scanned", AggregationTypeSum) - StatPagesFoundInCache = NewStatisticInt64("pages.cache.hit", AggregationTypeSum) - StatPageDownloadRequests = NewStatisticInt64("pages.download.requests", AggregationTypeSum) - StatPageDownloadTime = NewStatisticInt64("pages.download.duration.ns", AggregationTypeSum) + StatDatasetPagesScanned = NewStatisticInt64("pages.scanned", AggregationTypeSum) + StatDatasetPagesFoundInCache = NewStatisticInt64("pages.cache.hit", AggregationTypeSum) + StatDatasetPageDownloadRequests = NewStatisticInt64("pages.download.requests", AggregationTypeSum) + StatDatasetPageDownloadTime = NewStatisticInt64("pages.download.duration.ns", AggregationTypeSum) // Dataset page download byte statistics. - StatPrimaryPagesDownloaded = NewStatisticInt64("primary.pages.downloaded", AggregationTypeSum) - StatSecondaryPagesDownloaded = NewStatisticInt64("secondary.pages.downloaded", AggregationTypeSum) - StatPrimaryColumnBytes = NewStatisticInt64("primary.pages.compressed.bytes", AggregationTypeSum) - StatSecondaryColumnBytes = NewStatisticInt64("secondary.pages.compressed.bytes", AggregationTypeSum) - StatPrimaryColumnUncompressedBytes = NewStatisticInt64("primary.column.uncompressed.bytes", AggregationTypeSum) - StatSecondaryColumnUncompressedBytes = NewStatisticInt64("secondary.column.uncompressed.bytes", AggregationTypeSum) -) + StatDatasetPrimaryPagesDownloaded = NewStatisticInt64("primary.pages.downloaded", AggregationTypeSum) + StatDatasetSecondaryPagesDownloaded = NewStatisticInt64("secondary.pages.downloaded", AggregationTypeSum) + StatDatasetPrimaryColumnBytes = NewStatisticInt64("primary.pages.compressed.bytes", AggregationTypeSum) + StatDatasetSecondaryColumnBytes = NewStatisticInt64("secondary.pages.compressed.bytes", AggregationTypeSum) + StatDatasetPrimaryColumnUncompressedBytes = NewStatisticInt64("primary.column.uncompressed.bytes", AggregationTypeSum) + StatDatasetSecondaryColumnUncompressedBytes = NewStatisticInt64("secondary.column.uncompressed.bytes", AggregationTypeSum) -// Dataset read operation statistics. -var ( + // Dataset read operation statistics. StatDatasetReadCalls = NewStatisticInt64("dataset.read.calls", AggregationTypeSum) ) // Range IO statistics. var ( - StatInputRangesCount = NewStatisticInt64("input.ranges", AggregationTypeSum) - StatInputRangesSize = NewStatisticInt64("input.ranges.size.bytes", AggregationTypeSum) - StatOptimizedRangesCount = NewStatisticInt64("optimized.ranges", AggregationTypeSum) - StatOptimizedRangesSize = NewStatisticInt64("optimized.ranges.size.bytes", AggregationTypeSum) - StatOptimizedThroughput = NewStatisticFloat64("optimized.ranges.min.throughput", AggregationTypeMin) + StatRangeIOInputCount = NewStatisticInt64("input.ranges", AggregationTypeSum) + StatRangeIOInputSize = NewStatisticInt64("input.ranges.size.bytes", AggregationTypeSum) + StatRangeIOOptimizedCount = NewStatisticInt64("optimized.ranges", AggregationTypeSum) + StatRangeIOOptimizedSize = NewStatisticInt64("optimized.ranges.size.bytes", AggregationTypeSum) + StatRangeIOThroughput = NewStatisticFloat64("optimized.ranges.min.throughput", AggregationTypeMin) ) // Bucket operation statistics. @@ -66,6 +64,6 @@ var ( // Metastore statistics. var ( - StatIndexObjects = NewStatisticInt64("metastore.index.objects", AggregationTypeSum) - StatResolvedSections = NewStatisticInt64("metastore.resolved.sections", AggregationTypeSum) + StatMetastoreIndexObjects = NewStatisticInt64("metastore.index.objects", AggregationTypeSum) + StatMetastoreResolvedSections = NewStatisticInt64("metastore.resolved.sections", AggregationTypeSum) ) diff --git a/pkg/xcap/summary.go b/pkg/xcap/summary.go index 5cd8b4eca840d..6faa437ec2d93 100644 --- a/pkg/xcap/summary.go +++ b/pkg/xcap/summary.go @@ -266,23 +266,23 @@ func (c *Capture) ToStatsSummary(execTime, queueTime time.Duration, totalEntries // In practice, new engine would process more bytes while scanning metastore objects and stream sections. collector := newObservationCollector(c) observations := collector.fromRegions(regionNameDataObjScan, true).filter( - StatRowsOut.Key(), - StatPrimaryRowsRead.Key(), - StatPrimaryColumnUncompressedBytes.Key(), - StatSecondaryColumnUncompressedBytes.Key(), + StatPipelineRowsOut.Key(), + StatDatasetPrimaryRowsRead.Key(), + StatDatasetPrimaryColumnUncompressedBytes.Key(), + StatDatasetSecondaryColumnUncompressedBytes.Key(), ) // TotalBytesProcessed: sum of uncompressed bytes from primary and secondary columns - result.Summary.TotalBytesProcessed = readInt64(observations, StatPrimaryColumnUncompressedBytes.Key()) + - readInt64(observations, StatSecondaryColumnUncompressedBytes.Key()) + result.Summary.TotalBytesProcessed = readInt64(observations, StatDatasetPrimaryColumnUncompressedBytes.Key()) + + readInt64(observations, StatDatasetSecondaryColumnUncompressedBytes.Key()) // TotalLinesProcessed: primary rows read - result.Summary.TotalLinesProcessed = readInt64(observations, StatPrimaryRowsRead.Key()) + result.Summary.TotalLinesProcessed = readInt64(observations, StatDatasetPrimaryRowsRead.Key()) // TotalPostFilterLines: rows output after filtering // TODO: this will report the wrong value if the plan has a filter stage. // pick the min of row_out from filter and scan nodes. - result.Summary.TotalPostFilterLines = readInt64(observations, StatRowsOut.Key()) + result.Summary.TotalPostFilterLines = readInt64(observations, StatPipelineRowsOut.Key()) // TODO: track and report TotalStructuredMetadataBytesProcessed diff --git a/pkg/xcap/summary_test.go b/pkg/xcap/summary_test.go index 0db547e73fe71..2f2ba4b807356 100644 --- a/pkg/xcap/summary_test.go +++ b/pkg/xcap/summary_test.go @@ -144,22 +144,22 @@ func TestToStatsSummary(t *testing.T) { // Create DataObjScan regions with observations using registry stats _, region1 := StartRegion(ctx, "DataObjScan") - region1.Record(StatPrimaryColumnUncompressedBytes.Observe(1000)) - region1.Record(StatSecondaryColumnUncompressedBytes.Observe(500)) - region1.Record(StatPrimaryRowsRead.Observe(100)) - region1.Record(StatRowsOut.Observe(80)) + region1.Record(StatDatasetPrimaryColumnUncompressedBytes.Observe(1000)) + region1.Record(StatDatasetSecondaryColumnUncompressedBytes.Observe(500)) + region1.Record(StatDatasetPrimaryRowsRead.Observe(100)) + region1.Record(StatPipelineRowsOut.Observe(80)) region1.End() _, region2 := StartRegion(ctx, "DataObjScan") - region2.Record(StatPrimaryColumnUncompressedBytes.Observe(2000)) - region2.Record(StatSecondaryColumnUncompressedBytes.Observe(1000)) - region2.Record(StatPrimaryRowsRead.Observe(200)) - region2.Record(StatRowsOut.Observe(150)) + region2.Record(StatDatasetPrimaryColumnUncompressedBytes.Observe(2000)) + region2.Record(StatDatasetSecondaryColumnUncompressedBytes.Observe(1000)) + region2.Record(StatDatasetPrimaryRowsRead.Observe(200)) + region2.Record(StatPipelineRowsOut.Observe(150)) region2.End() // Other region - should be ignored _, otherRegion := StartRegion(ctx, "OtherRegion") - otherRegion.Record(StatPrimaryColumnUncompressedBytes.Observe(5000)) + otherRegion.Record(StatDatasetPrimaryColumnUncompressedBytes.Observe(5000)) otherRegion.End() capture.End() @@ -195,7 +195,7 @@ func TestToStatsSummary(t *testing.T) { // Only record some statistics _, region := StartRegion(ctx, "DataObjScan") - region.Record(StatPrimaryColumnUncompressedBytes.Observe(1000)) + region.Record(StatDatasetPrimaryColumnUncompressedBytes.Observe(1000)) region.End() capture.End() @@ -214,11 +214,11 @@ func TestToStatsSummary(t *testing.T) { // Parent DataObjScan region ctx, parent := StartRegion(ctx, "DataObjScan") - parent.Record(StatPrimaryColumnUncompressedBytes.Observe(500)) + parent.Record(StatDatasetPrimaryColumnUncompressedBytes.Observe(500)) // Child region (should be rolled up into parent) _, child := StartRegion(ctx, "child_operation") - child.Record(StatPrimaryColumnUncompressedBytes.Observe(300)) + child.Record(StatDatasetPrimaryColumnUncompressedBytes.Observe(300)) child.End() parent.End() From 5a411e510189b6adfdb2f1bffd8f5c06071caaec Mon Sep 17 00:00:00 2001 From: Ashwanth Goli Date: Thu, 4 Dec 2025 18:17:32 +0530 Subject: [PATCH 13/13] use secs for duration --- pkg/dataobj/sections/internal/columnar/decoder.go | 4 ++-- pkg/engine/internal/executor/pipeline.go | 2 +- pkg/xcap/stats_definitions.go | 4 ++-- pkg/xcap/summary.go | 10 ++++++---- 4 files changed, 11 insertions(+), 9 deletions(-) diff --git a/pkg/dataobj/sections/internal/columnar/decoder.go b/pkg/dataobj/sections/internal/columnar/decoder.go index b4b47df5034a0..8ff85c014901f 100644 --- a/pkg/dataobj/sections/internal/columnar/decoder.go +++ b/pkg/dataobj/sections/internal/columnar/decoder.go @@ -79,7 +79,7 @@ func (dec *Decoder) Pages(ctx context.Context, columns []*datasetmd.ColumnDesc) region := xcap.RegionFromContext(ctx) startTime := time.Now() defer func() { - region.Record(xcap.StatDatasetPageDownloadTime.Observe(time.Since(startTime).Nanoseconds())) + region.Record(xcap.StatDatasetPageDownloadTime.Observe(time.Since(startTime).Seconds())) }() ranges := make([]rangeio.Range, 0, len(columns)) @@ -135,7 +135,7 @@ func (dec *Decoder) ReadPages(ctx context.Context, pages []*datasetmd.PageDesc) region := xcap.RegionFromContext(ctx) startTime := time.Now() defer func() { - region.Record(xcap.StatDatasetPageDownloadTime.Observe(time.Since(startTime).Nanoseconds())) + region.Record(xcap.StatDatasetPageDownloadTime.Observe(time.Since(startTime).Seconds())) }() ranges := make([]rangeio.Range, 0, len(pages)) diff --git a/pkg/engine/internal/executor/pipeline.go b/pkg/engine/internal/executor/pipeline.go index c75ef51472625..23fea975ffa17 100644 --- a/pkg/engine/internal/executor/pipeline.go +++ b/pkg/engine/internal/executor/pipeline.go @@ -347,7 +347,7 @@ func (p *observedPipeline) Read(ctx context.Context) (arrow.RecordBatch, error) p.region.Record(xcap.StatPipelineRowsOut.Observe(rec.NumRows())) } - p.region.Record(xcap.StatPipelineReadDuration.Observe(time.Since(start).Nanoseconds())) + p.region.Record(xcap.StatPipelineReadDuration.Observe(time.Since(start).Seconds())) } return rec, err diff --git a/pkg/xcap/stats_definitions.go b/pkg/xcap/stats_definitions.go index 30808e2147200..46fa31774a11a 100644 --- a/pkg/xcap/stats_definitions.go +++ b/pkg/xcap/stats_definitions.go @@ -4,7 +4,7 @@ package xcap var ( StatPipelineRowsOut = NewStatisticInt64("rows.out", AggregationTypeSum) StatPipelineReadCalls = NewStatisticInt64("read.calls", AggregationTypeSum) - StatPipelineReadDuration = NewStatisticInt64("read.duration.ns", AggregationTypeSum) + StatPipelineReadDuration = NewStatisticFloat64("read.duration", AggregationTypeSum) ) // ColumnCompat statistics. @@ -31,7 +31,7 @@ var ( StatDatasetPagesScanned = NewStatisticInt64("pages.scanned", AggregationTypeSum) StatDatasetPagesFoundInCache = NewStatisticInt64("pages.cache.hit", AggregationTypeSum) StatDatasetPageDownloadRequests = NewStatisticInt64("pages.download.requests", AggregationTypeSum) - StatDatasetPageDownloadTime = NewStatisticInt64("pages.download.duration.ns", AggregationTypeSum) + StatDatasetPageDownloadTime = NewStatisticFloat64("pages.download.duration", AggregationTypeSum) // Dataset page download byte statistics. StatDatasetPrimaryPagesDownloaded = NewStatisticInt64("primary.pages.downloaded", AggregationTypeSum) diff --git a/pkg/xcap/summary.go b/pkg/xcap/summary.go index 6faa437ec2d93..a7e36fb539ca9 100644 --- a/pkg/xcap/summary.go +++ b/pkg/xcap/summary.go @@ -131,13 +131,15 @@ func (o *observations) toLogValues() []any { } } - // Format duration values (keys ending with "duration_ns") - if strings.HasSuffix(p.name, "duration_ns") { + // Format duration values (keys ending with "duration") + if strings.HasSuffix(p.name, "duration") { switch val := value.(type) { + case float64: + value = time.Duration(val * 1000).String() case int64: - value = time.Duration(val).String() + value = time.Duration(val * 1000).String() case uint64: - value = time.Duration(val).String() + value = time.Duration(val * 1000).String() } }