github.com/grafana/pyroscope@v1.18.0/pkg/block/section_profiles.go (about)

     1  package block
     2  
     3  import (
     4  	"context"
     5  	"encoding/binary"
     6  	"fmt"
     7  	"io"
     8  	"math"
     9  
    10  	"github.com/parquet-go/parquet-go"
    11  	"github.com/pkg/errors"
    12  	"github.com/prometheus/common/model"
    13  
    14  	"github.com/grafana/pyroscope/pkg/iter"
    15  	phlaremodel "github.com/grafana/pyroscope/pkg/model"
    16  	"github.com/grafana/pyroscope/pkg/objstore"
    17  	phlareparquet "github.com/grafana/pyroscope/pkg/parquet"
    18  	"github.com/grafana/pyroscope/pkg/phlaredb"
    19  	"github.com/grafana/pyroscope/pkg/phlaredb/query"
    20  	schemav1 "github.com/grafana/pyroscope/pkg/phlaredb/schemas/v1"
    21  	"github.com/grafana/pyroscope/pkg/phlaredb/tsdb/index"
    22  	"github.com/grafana/pyroscope/pkg/util/bufferpool"
    23  	"github.com/grafana/pyroscope/pkg/util/build"
    24  	"github.com/grafana/pyroscope/pkg/util/loser"
    25  )
    26  
    27  func openProfileTable(_ context.Context, s *Dataset) (err error) {
    28  	offset := s.sectionOffset(SectionProfiles)
    29  	size := s.sectionSize(SectionProfiles)
    30  	if buf := s.inMemoryBuffer(); buf != nil {
    31  		offset -= int64(s.offset())
    32  		s.profiles, err = openParquetFile(
    33  			s.inMemoryBucket(buf), s.obj.path, offset, size,
    34  			0, // Do not prefetch the footer.
    35  			parquet.SkipBloomFilters(true),
    36  			parquet.FileReadMode(parquet.ReadModeSync),
    37  			parquet.ReadBufferSize(4<<10))
    38  	} else {
    39  		s.profiles, err = openParquetFile(
    40  			s.obj.storage, s.obj.path, offset, size,
    41  			estimateFooterSize(size),
    42  			parquet.SkipBloomFilters(true),
    43  			parquet.FileReadMode(parquet.ReadModeAsync),
    44  			parquet.ReadBufferSize(estimateReadBufferSize(size)))
    45  	}
    46  	if err != nil {
    47  		return fmt.Errorf("opening profile parquet table: %w", err)
    48  	}
    49  	return nil
    50  }
    51  
    52  type ParquetFile struct {
    53  	*parquet.File
    54  
    55  	reader objstore.ReaderAtCloser
    56  	cancel context.CancelFunc
    57  
    58  	storage objstore.BucketReader
    59  	path    string
    60  	off     int64
    61  	size    int64
    62  }
    63  
    64  func openParquetFile(
    65  	storage objstore.BucketReader,
    66  	path string,
    67  	offset, size, footerSize int64,
    68  	options ...parquet.FileOption,
    69  ) (p *ParquetFile, err error) {
    70  	// The context is used for GetRange calls and should not
    71  	// be canceled until the parquet file is closed.
    72  	ctx, cancel := context.WithCancel(context.Background())
    73  	defer func() {
    74  		if err != nil {
    75  			cancel()
    76  		}
    77  	}()
    78  
    79  	p = &ParquetFile{
    80  		cancel:  cancel,
    81  		storage: storage,
    82  		path:    path,
    83  		off:     offset,
    84  		size:    size,
    85  	}
    86  
    87  	r, err := storage.ReaderAt(ctx, path)
    88  	if err != nil {
    89  		return nil, fmt.Errorf("creating object reader: %w", err)
    90  	}
    91  
    92  	var ra io.ReaderAt
    93  	ra = io.NewSectionReader(r, offset, size)
    94  	if footerSize > 0 {
    95  		buf := bufferpool.GetBuffer(int(footerSize))
    96  		defer func() {
    97  			// Footer is not used after the file was opened.
    98  			bufferpool.Put(buf)
    99  		}()
   100  		if err = p.fetchFooter(ctx, buf, footerSize); err != nil {
   101  			return nil, err
   102  		}
   103  		rf := newReaderWithFooter(ra, buf.B, size)
   104  		defer rf.free()
   105  		ra = rf
   106  	}
   107  
   108  	f, err := parquet.OpenFile(ra, size, options...)
   109  	if err != nil {
   110  		return nil, err
   111  	}
   112  
   113  	p.reader = r
   114  	p.File = f
   115  	return p, nil
   116  }
   117  
   118  func (f *ParquetFile) RowReader() parquet.RowReader {
   119  	return parquet.NewReader(f.File, schemav1.ProfilesSchema)
   120  }
   121  
   122  func footerSize(buf []byte) int64 {
   123  	sb := buf[len(buf)-8 : len(buf)-4]
   124  	s := int64(binary.LittleEndian.Uint32(sb))
   125  	s += 8 // Include the footer size itself and the magic signature.
   126  	return s
   127  }
   128  
   129  func (f *ParquetFile) fetchFooter(ctx context.Context, buf *bufferpool.Buffer, estimatedSize int64) error {
   130  	if f.size < 8 {
   131  		return fmt.Errorf("file size is too small to contain a footer")
   132  	}
   133  
   134  	// Ensure the footer is at least 8 bytes.
   135  	if estimatedSize < 8 {
   136  		estimatedSize = 8
   137  	}
   138  
   139  	// Ensure the footer is not bigger than the file.
   140  	if estimatedSize > f.size {
   141  		estimatedSize = f.size
   142  	}
   143  
   144  	// Fetch the footer of estimated size at the estimated offset.
   145  	estimatedOffset := f.off + f.size - estimatedSize
   146  	if err := objstore.ReadRange(ctx, buf, f.path, f.storage, estimatedOffset, estimatedSize); err != nil {
   147  		return err
   148  	}
   149  	s := footerSize(buf.B)
   150  	if estimatedSize >= s {
   151  		// The footer has been fetched.
   152  		return nil
   153  	}
   154  
   155  	// reset buffer
   156  	buf.B = buf.B[:0]
   157  
   158  	// Fetch exact footer to buf for sure.
   159  	return objstore.ReadRange(ctx, buf, f.path, f.storage, f.off+f.size-s, s)
   160  }
   161  
   162  func (f *ParquetFile) Close() error {
   163  	if f.cancel != nil {
   164  		f.cancel()
   165  	}
   166  	if f.reader != nil {
   167  		return f.reader.Close()
   168  	}
   169  	return nil
   170  }
   171  
   172  func (f *ParquetFile) Column(ctx context.Context, columnName string, predicate query.Predicate) query.Iterator {
   173  	idx, _ := query.GetColumnIndexByPath(f.Root(), columnName)
   174  	if idx == -1 {
   175  		return query.NewErrIterator(fmt.Errorf("column '%s' not found in parquet table", columnName))
   176  	}
   177  	return query.NewSyncIterator(ctx, f.RowGroups(), idx, columnName, 1<<10, predicate, columnName)
   178  }
   179  
   180  type profilesWriter struct {
   181  	*parquet.GenericWriter[*schemav1.Profile]
   182  	buf      []parquet.Row
   183  	profiles uint64
   184  }
   185  
   186  func newProfileWriter(pageBufferSize int, w io.Writer) *profilesWriter {
   187  	return &profilesWriter{
   188  		buf: make([]parquet.Row, 1),
   189  		GenericWriter: parquet.NewGenericWriter[*schemav1.Profile](w,
   190  			parquet.CreatedBy("github.com/grafana/pyroscope/", build.Version, build.Revision),
   191  			parquet.PageBufferSize(pageBufferSize),
   192  			// Note that parquet keeps ALL RG pages in memory (ColumnPageBuffers).
   193  			parquet.MaxRowsPerRowGroup(maxRowsPerRowGroup),
   194  			schemav1.ProfilesSchema,
   195  			// parquet.ColumnPageBuffers(),
   196  		),
   197  	}
   198  }
   199  
   200  func (p *profilesWriter) writeRow(e ProfileEntry) error {
   201  	p.buf[0] = parquet.Row(e.Row)
   202  	_, err := p.WriteRows(p.buf)
   203  	p.profiles++
   204  	return err
   205  }
   206  
   207  type readerWithFooter struct {
   208  	reader io.ReaderAt
   209  	footer []byte
   210  	offset int64
   211  	size   int64
   212  }
   213  
   214  func newReaderWithFooter(r io.ReaderAt, footer []byte, size int64) *readerWithFooter {
   215  	footerSize := int64(len(footer))
   216  	footerOffset := size - footerSize
   217  	return &readerWithFooter{
   218  		reader: r,
   219  		footer: footer,
   220  		offset: footerOffset,
   221  		size:   footerSize,
   222  	}
   223  }
   224  
   225  func (f *readerWithFooter) hitsHeaderMagic(off, length int64) bool {
   226  	return off == 0 && length == 4
   227  }
   228  
   229  func (f *readerWithFooter) hitsFooter(off, length int64) bool {
   230  	return length <= f.size && off >= f.offset && off+length <= f.offset+f.size
   231  }
   232  
   233  var parquetMagic = []byte("PAR1")
   234  
   235  func (f *readerWithFooter) free() {
   236  	f.footer = nil
   237  	f.size = -1
   238  }
   239  
   240  func (f *readerWithFooter) ReadAt(p []byte, off int64) (n int, err error) {
   241  	if f.hitsHeaderMagic(off, int64(len(p))) {
   242  		copy(p, parquetMagic)
   243  		return len(p), nil
   244  	}
   245  	if f.hitsFooter(off, int64(len(p))) {
   246  		copy(p, f.footer[off-f.offset:])
   247  		return len(p), nil
   248  	}
   249  	return f.reader.ReadAt(p, off)
   250  }
   251  
   252  type ProfileEntry struct {
   253  	Dataset *Dataset
   254  
   255  	Timestamp   int64
   256  	Fingerprint model.Fingerprint
   257  	Labels      phlaremodel.Labels
   258  	Row         schemav1.ProfileRow
   259  }
   260  
   261  func NewMergeRowProfileIterator(src []*Dataset) (iter.Iterator[ProfileEntry], error) {
   262  	its := make([]iter.Iterator[ProfileEntry], len(src))
   263  	for i, s := range src {
   264  		it, err := NewProfileRowIterator(s)
   265  		if err != nil {
   266  			return nil, err
   267  		}
   268  		its[i] = it
   269  	}
   270  	if len(its) == 1 {
   271  		return its[0], nil
   272  	}
   273  	return &DedupeProfileRowIterator{
   274  		Iterator: iter.NewTreeIterator(loser.New(
   275  			its,
   276  			ProfileEntry{
   277  				Timestamp: math.MaxInt64,
   278  			},
   279  			func(it iter.Iterator[ProfileEntry]) ProfileEntry { return it.At() },
   280  			func(r1, r2 ProfileEntry) bool {
   281  				// first handle max profileRow if it's either r1 or r2
   282  				if r1.Timestamp == math.MaxInt64 {
   283  					return false
   284  				}
   285  				if r2.Timestamp == math.MaxInt64 {
   286  					return true
   287  				}
   288  				// then handle normal profileRows
   289  				if cmp := phlaremodel.CompareLabelPairs(r1.Labels, r2.Labels); cmp != 0 {
   290  					return cmp < 0
   291  				}
   292  				return r1.Timestamp < r2.Timestamp
   293  			},
   294  			func(it iter.Iterator[ProfileEntry]) { _ = it.Close() },
   295  		)),
   296  	}, nil
   297  }
   298  
   299  type DedupeProfileRowIterator struct {
   300  	iter.Iterator[ProfileEntry]
   301  
   302  	prevFP        model.Fingerprint
   303  	prevTimeNanos int64
   304  }
   305  
   306  func (it *DedupeProfileRowIterator) Next() bool {
   307  	for {
   308  		if !it.Iterator.Next() {
   309  			return false
   310  		}
   311  		currentProfile := it.At()
   312  		if it.prevFP == currentProfile.Fingerprint && it.prevTimeNanos == currentProfile.Timestamp {
   313  			// skip duplicate profile
   314  			continue
   315  		}
   316  		it.prevFP = currentProfile.Fingerprint
   317  		it.prevTimeNanos = currentProfile.Timestamp
   318  		return true
   319  	}
   320  }
   321  
   322  type profileRowIterator struct {
   323  	reader      *Dataset
   324  	index       phlaredb.IndexReader
   325  	profiles    iter.Iterator[parquet.Row]
   326  	allPostings index.Postings
   327  	err         error
   328  
   329  	currentRow       ProfileEntry
   330  	currentSeriesIdx uint32
   331  	chunks           []index.ChunkMeta
   332  }
   333  
   334  func NewProfileRowIterator(s *Dataset) (iter.Iterator[ProfileEntry], error) {
   335  	k, v := index.AllPostingsKey()
   336  	tsdb := s.Index()
   337  	allPostings, err := tsdb.Postings(k, nil, v)
   338  	if err != nil {
   339  		return nil, err
   340  	}
   341  	return &profileRowIterator{
   342  		reader:           s,
   343  		index:            tsdb,
   344  		profiles:         phlareparquet.NewBufferedRowReaderIterator(s.ProfileRowReader(), 4),
   345  		allPostings:      allPostings,
   346  		currentSeriesIdx: math.MaxUint32,
   347  		chunks:           make([]index.ChunkMeta, 1),
   348  	}, nil
   349  }
   350  
   351  func (p *profileRowIterator) At() ProfileEntry {
   352  	return p.currentRow
   353  }
   354  
   355  func (p *profileRowIterator) Next() bool {
   356  	if !p.profiles.Next() {
   357  		return false
   358  	}
   359  	p.currentRow.Dataset = p.reader
   360  	p.currentRow.Row = schemav1.ProfileRow(p.profiles.At())
   361  	seriesIndex := p.currentRow.Row.SeriesIndex()
   362  	p.currentRow.Timestamp = p.currentRow.Row.TimeNanos()
   363  	// do we have a new series?
   364  	if seriesIndex == p.currentSeriesIdx {
   365  		return true
   366  	}
   367  	p.currentSeriesIdx = seriesIndex
   368  	if !p.allPostings.Next() {
   369  		if err := p.allPostings.Err(); err != nil {
   370  			p.err = err
   371  			return false
   372  		}
   373  		p.err = errors.New("unexpected end of postings")
   374  		return false
   375  	}
   376  
   377  	fp, err := p.index.Series(p.allPostings.At(), &p.currentRow.Labels, &p.chunks)
   378  	if err != nil {
   379  		p.err = err
   380  		return false
   381  	}
   382  	p.currentRow.Fingerprint = model.Fingerprint(fp)
   383  	return true
   384  }
   385  
   386  func (p *profileRowIterator) Err() error {
   387  	if p.err != nil {
   388  		return p.err
   389  	}
   390  	return p.profiles.Err()
   391  }
   392  
   393  func (p *profileRowIterator) Close() error {
   394  	return p.reader.Close()
   395  }