github.com/grafana/pyroscope@v1.18.0/pkg/parquet/row_reader.go (about)

     1  package parquet
     2  
     3  import (
     4  	"io"
     5  
     6  	"github.com/grafana/dskit/runutil"
     7  	"github.com/parquet-go/parquet-go"
     8  
     9  	"github.com/grafana/pyroscope/pkg/iter"
    10  	"github.com/grafana/pyroscope/pkg/util"
    11  	"github.com/grafana/pyroscope/pkg/util/loser"
    12  )
    13  
    14  const (
    15  	defaultRowBufferSize = 64
    16  )
    17  
    18  var (
    19  	_ parquet.RowReader          = (*emptyRowReader)(nil)
    20  	_ parquet.RowReader          = (*ErrRowReader)(nil)
    21  	_ parquet.RowReader          = (*IteratorRowReader)(nil)
    22  	_ iter.Iterator[parquet.Row] = (*BufferedRowReaderIterator)(nil)
    23  
    24  	EmptyRowReader = &emptyRowReader{}
    25  )
    26  
    27  type emptyRowReader struct{}
    28  
    29  func (e *emptyRowReader) ReadRows(rows []parquet.Row) (int, error) { return 0, io.EOF }
    30  
    31  type ErrRowReader struct{ err error }
    32  
    33  func NewErrRowReader(err error) *ErrRowReader { return &ErrRowReader{err: err} }
    34  
    35  func (e ErrRowReader) ReadRows(rows []parquet.Row) (int, error) { return 0, e.err }
    36  
    37  // NewMergeRowReader returns a RowReader that k-way merges the given readers using the less function.
    38  // Each reader must be sorted according to the less function already.
    39  func NewMergeRowReader(readers []parquet.RowReader, maxValue parquet.Row, less func(parquet.Row, parquet.Row) bool) parquet.RowReader {
    40  	if len(readers) == 0 {
    41  		return EmptyRowReader
    42  	}
    43  	if len(readers) == 1 {
    44  		return readers[0]
    45  	}
    46  	its := make([]iter.Iterator[parquet.Row], len(readers))
    47  	for i := range readers {
    48  		its[i] = NewBufferedRowReaderIterator(readers[i], defaultRowBufferSize)
    49  	}
    50  
    51  	return NewIteratorRowReader(
    52  		iter.NewTreeIterator[parquet.Row](
    53  			loser.New(
    54  				its,
    55  				maxValue,
    56  				func(it iter.Iterator[parquet.Row]) parquet.Row { return it.At() },
    57  				less,
    58  				func(it iter.Iterator[parquet.Row]) { _ = it.Close() },
    59  			),
    60  		),
    61  	)
    62  }
    63  
    64  type IteratorRowReader struct {
    65  	iter.Iterator[parquet.Row]
    66  }
    67  
    68  // NewIteratorRowReader returns a RowReader that reads rows from the given iterator.
    69  func NewIteratorRowReader(it iter.Iterator[parquet.Row]) *IteratorRowReader {
    70  	return &IteratorRowReader{
    71  		Iterator: it,
    72  	}
    73  }
    74  
    75  func (it *IteratorRowReader) ReadRows(rows []parquet.Row) (int, error) {
    76  	var n int
    77  	if len(rows) == 0 {
    78  		return 0, nil
    79  	}
    80  	for n != len(rows) {
    81  
    82  		if !it.Next() {
    83  			runutil.CloseWithLogOnErr(util.Logger, it.Iterator, "failed to close iterator")
    84  			if it.Err() != nil {
    85  				return n, it.Err()
    86  			}
    87  			return n, io.EOF
    88  		}
    89  		rows[n] = rows[n][:0]
    90  		for _, c := range it.At() {
    91  			rows[n] = append(rows[n], c.Clone())
    92  		}
    93  		n++
    94  	}
    95  	return n, nil
    96  }
    97  
    98  type BufferedRowReaderIterator struct {
    99  	reader parquet.RowReader
   100  	buf    []parquet.Row
   101  	err    error
   102  	r      int
   103  	w      int
   104  }
   105  
   106  // NewBufferedRowReaderIterator returns a new `iter.Iterator[parquet.Row]` from a RowReader.
   107  // The iterator will buffer `bufferSize` rows from the reader.
   108  func NewBufferedRowReaderIterator(reader parquet.RowReader, bufferSize int) *BufferedRowReaderIterator {
   109  	return &BufferedRowReaderIterator{
   110  		reader: reader,
   111  		buf:    make([]parquet.Row, bufferSize),
   112  	}
   113  }
   114  
   115  func (r *BufferedRowReaderIterator) Next() bool {
   116  	if r.r < r.w-1 {
   117  		r.r++
   118  		return true
   119  	}
   120  	var err error
   121  	if r.w, err = r.reader.ReadRows(r.buf); err != nil && err != io.EOF {
   122  		r.err = err
   123  		return false
   124  	}
   125  	if r.w > 0 {
   126  		r.r = 0
   127  		return true
   128  	}
   129  	return false
   130  }
   131  
   132  func (r *BufferedRowReaderIterator) At() parquet.Row {
   133  	return r.buf[r.r]
   134  }
   135  
   136  func (r *BufferedRowReaderIterator) Err() error {
   137  	return r.err
   138  }
   139  
   140  func (r *BufferedRowReaderIterator) Close() error {
   141  	return r.err
   142  }
   143  
   144  func ReadAll(r parquet.RowReader) ([]parquet.Row, error) {
   145  	return ReadAllWithBufferSize(r, defaultRowBufferSize)
   146  }
   147  
   148  func ReadAllWithBufferSize(r parquet.RowReader, bufferSize int) ([]parquet.Row, error) {
   149  	var rows []parquet.Row
   150  	batch := make([]parquet.Row, bufferSize)
   151  	for {
   152  		n, err := r.ReadRows(batch)
   153  		if err != nil && err != io.EOF {
   154  			return rows, err
   155  		}
   156  		if n != 0 {
   157  			for i := range batch[:n] {
   158  				rows = append(rows, batch[i].Clone())
   159  			}
   160  		}
   161  		if n == 0 || err == io.EOF {
   162  			break
   163  		}
   164  	}
   165  	return rows, nil
   166  }