github.com/grafana/pyroscope@v1.18.0/pkg/phlaredb/query/repeated.go (about) 1 package query 2 3 import ( 4 "context" 5 "fmt" 6 "io" 7 "strings" 8 "sync" 9 "time" 10 11 "github.com/grafana/dskit/multierror" 12 "github.com/opentracing/opentracing-go" 13 otlog "github.com/opentracing/opentracing-go/log" 14 "github.com/parquet-go/parquet-go" 15 "github.com/prometheus/client_golang/prometheus" 16 17 "github.com/grafana/pyroscope/pkg/iter" 18 ) 19 20 type RepeatedRow[T any] struct { 21 Row T 22 Values [][]parquet.Value 23 } 24 25 type repeatedRowIterator[T any] struct { 26 columns iter.Iterator[[][]parquet.Value] 27 rows iter.Iterator[T] 28 } 29 30 const ( 31 // The value specifies how many individual values to be 32 // read (decoded) from the page. 33 // 34 // Too big read size does not make much sense: despite 35 // the fact that this does not impact read amplification 36 // as the page is already fully read, decoding of the 37 // values is not free. 38 // 39 // How many values we expect per a row, the upper boundary? 40 repeatedRowColumnIteratorReadSize = 2 << 10 41 42 // Batch size specifies how many rows to read from a column at once. 43 // Note that the batched rows are buffered in-memory but do not reference 44 // the pages from which they were read. 45 // 46 // The default value is extremely conservative, as in most cases, rows are 47 // quite large (e.g., profile samples). Given that we run many queries 48 // concurrently, the memory waste outweighs the benefits of the "read-ahead" 49 // optimization that batching is intended to provide. 50 // 51 // However, in cases where the rows are small (such as in time series), 52 // the value should be increased significantly. 53 defaultBatchSize = 4 54 ) 55 56 func NewRepeatedRowIterator[T any]( 57 ctx context.Context, 58 rows iter.Iterator[T], 59 rowGroups []parquet.RowGroup, 60 columns ...int, 61 ) iter.Iterator[RepeatedRow[T]] { 62 return NewRepeatedRowIteratorBatchSize(ctx, rows, rowGroups, defaultBatchSize, columns...) 63 } 64 65 func NewRepeatedRowIteratorBatchSize[T any]( 66 ctx context.Context, 67 rows iter.Iterator[T], 68 rowGroups []parquet.RowGroup, 69 batchSize int64, 70 columns ...int, 71 ) iter.Iterator[RepeatedRow[T]] { 72 rows, rowNumbers := iter.Tee(rows) 73 return &repeatedRowIterator[T]{ 74 rows: rows, 75 columns: NewMultiColumnIterator(ctx, 76 WrapWithRowNumber(rowNumbers), 77 int(batchSize), 78 rowGroups, 79 columns..., 80 ), 81 } 82 } 83 84 func (x *repeatedRowIterator[T]) Next() bool { 85 if !x.rows.Next() { 86 return false 87 } 88 return x.columns.Next() 89 } 90 91 func (x *repeatedRowIterator[T]) At() RepeatedRow[T] { 92 return RepeatedRow[T]{ 93 Values: x.columns.At(), 94 Row: x.rows.At(), 95 } 96 } 97 98 func (x *repeatedRowIterator[T]) Err() error { 99 return x.columns.Err() 100 } 101 102 func (x *repeatedRowIterator[T]) Close() error { 103 return x.columns.Close() 104 } 105 106 type rowNumberIterator[T any] struct{ it iter.Iterator[T] } 107 108 func WrapWithRowNumber[T any](it iter.Iterator[T]) iter.Iterator[int64] { 109 return &rowNumberIterator[T]{it} 110 } 111 112 func (x *rowNumberIterator[T]) Next() bool { return x.it.Next() } 113 func (x *rowNumberIterator[T]) Err() error { return x.it.Err() } 114 func (x *rowNumberIterator[T]) Close() error { return x.it.Close() } 115 116 func (x *rowNumberIterator[T]) At() int64 { 117 v := any(x.it.At()) 118 switch r := v.(type) { 119 case RowGetter: 120 return r.RowNumber() 121 case int64: 122 return r 123 case uint32: 124 return int64(r) 125 default: 126 panic(fmt.Sprintf("unknown row type: %T", v)) 127 } 128 } 129 130 type multiColumnIterator struct { 131 r []iter.Iterator[int64] 132 c []iter.Iterator[[]parquet.Value] 133 v [][]parquet.Value 134 } 135 136 func NewMultiColumnIterator( 137 ctx context.Context, 138 rows iter.Iterator[int64], 139 batchSize int, 140 rowGroups []parquet.RowGroup, 141 columns ...int, 142 ) iter.Iterator[[][]parquet.Value] { 143 m := multiColumnIterator{ 144 c: make([]iter.Iterator[[]parquet.Value], len(columns)), 145 v: make([][]parquet.Value, len(columns)), 146 // Even if there is just one column, we do need to tee it, 147 // as the source rows iterator is owned by caller, and we 148 // must never close it on our own. 149 r: iter.TeeN(rows, len(columns)), 150 } 151 for i, column := range columns { 152 m.c[i] = iter.NewAsyncBatchIterator[[]parquet.Value]( 153 NewRepeatedRowColumnIterator(ctx, m.r[i], rowGroups, column), 154 batchSize, 155 CloneParquetValues, 156 ReleaseParquetValues, 157 ) 158 } 159 return &m 160 } 161 162 func (m *multiColumnIterator) Next() bool { 163 for i, x := range m.c { 164 if !x.Next() { 165 return false 166 } 167 m.v[i] = x.At() 168 } 169 return true 170 } 171 172 func (m *multiColumnIterator) At() [][]parquet.Value { return m.v } 173 174 func (m *multiColumnIterator) Err() error { 175 var err multierror.MultiError 176 for i := range m.c { 177 err.Add(m.c[i].Err()) 178 err.Add(m.r[i].Err()) 179 } 180 return err.Err() 181 } 182 183 func (m *multiColumnIterator) Close() error { 184 var err multierror.MultiError 185 for i := range m.c { 186 err.Add(m.c[i].Close()) 187 err.Add(m.r[i].Close()) 188 } 189 return err.Err() 190 } 191 192 var ErrSeekOutOfRange = fmt.Errorf("bug: south row is out of range") 193 194 type repeatedRowColumnIterator struct { 195 ctx context.Context 196 span opentracing.Span 197 198 rows iter.Iterator[int64] 199 rgs []parquet.RowGroup 200 column int 201 readSize int 202 203 pages parquet.Pages 204 page parquet.Page 205 206 minRGRowNum int64 207 maxRGRowNum int64 208 maxPageRowNum int64 209 210 rowsRead int64 211 rowsFetched int64 212 pageBytes int64 213 214 pageReads prometheus.Counter 215 216 vit *repeatedValuePageIterator 217 prev int64 218 err error 219 } 220 221 func NewRepeatedRowColumnIterator(ctx context.Context, rows iter.Iterator[int64], rgs []parquet.RowGroup, column int) iter.Iterator[[]parquet.Value] { 222 r := repeatedRowColumnIterator{ 223 rows: rows, 224 rgs: rgs, 225 column: column, 226 vit: getRepeatedValuePageIteratorFromPool(), 227 readSize: repeatedRowColumnIteratorReadSize, 228 } 229 if len(rgs) == 0 { 230 return iter.NewEmptyIterator[[]parquet.Value]() 231 } 232 s := rgs[0].Schema() 233 if len(s.Columns()) <= column { 234 return iter.NewErrIterator[[]parquet.Value](fmt.Errorf("column %d not found", column)) 235 } 236 tableName := strings.ToLower(s.Name()) + "s" 237 columnName := strings.Join(s.Columns()[column], ".") 238 r.initMetrics(getMetricsFromContext(ctx), tableName, columnName) 239 r.span, r.ctx = opentracing.StartSpanFromContext(ctx, "RepeatedRowColumnIterator", opentracing.Tags{ 240 "table": tableName, 241 "column": columnName, 242 }) 243 return &r 244 } 245 246 func (x *repeatedRowColumnIterator) initMetrics(metrics *Metrics, table, column string) { 247 x.pageReads = metrics.pageReadsTotal.WithLabelValues(table, column) 248 } 249 250 func (x *repeatedRowColumnIterator) Next() bool { 251 if !x.rows.Next() || x.err != nil { 252 return false 253 } 254 rn := x.rows.At() 255 if rn >= x.maxRGRowNum { 256 if !x.seekRowGroup(rn) { 257 return false 258 } 259 } 260 rn -= x.minRGRowNum 261 if x.page == nil || rn >= x.maxPageRowNum { 262 if !x.readPage(rn) { 263 return false 264 } 265 // readPage ensures that the first row in the 266 // page matches rn, therefore we don't need to 267 // skip anything. 268 x.prev = rn - 1 269 } 270 // Skip rows to the rn. 271 next := int(rn - x.prev) 272 x.prev = rn 273 for i := 0; i < next; i++ { 274 if !x.vit.Next() { 275 x.err = ErrSeekOutOfRange 276 return false 277 } 278 } 279 x.rowsRead++ 280 return true 281 } 282 283 func (x *repeatedRowColumnIterator) seekRowGroup(rn int64) bool { 284 for i, rg := range x.rgs { 285 x.minRGRowNum = x.maxRGRowNum 286 x.maxRGRowNum += rg.NumRows() 287 if rn >= x.maxRGRowNum { 288 continue 289 } 290 x.rgs = x.rgs[i+1:] 291 return x.openChunk(rg) 292 } 293 return false 294 } 295 296 func (x *repeatedRowColumnIterator) openChunk(rg parquet.RowGroup) bool { 297 x.page = nil 298 x.vit.reset(nil, 0) 299 if x.pages != nil { 300 if x.err = x.pages.Close(); x.err != nil { 301 return false 302 } 303 } 304 x.pages = rg.ColumnChunks()[x.column].Pages() 305 return true 306 } 307 308 func (x *repeatedRowColumnIterator) readPage(rn int64) bool { 309 if x.err = x.ctx.Err(); x.err != nil { 310 return false 311 } 312 if x.err = x.pages.SeekToRow(rn); x.err != nil { 313 return false 314 } 315 readPageStart := time.Now() 316 if x.page, x.err = x.pages.ReadPage(); x.err != nil { 317 if x.err != io.EOF { 318 x.span.LogFields(otlog.Error(x.err)) 319 return false 320 } 321 x.err = nil 322 // ReadPage should never return page along with EOF, 323 // however this is not a strict contract. 324 if x.page == nil { 325 return false 326 } 327 } 328 x.pageReads.Add(1) 329 pageReadDurationMs := time.Since(readPageStart).Milliseconds() 330 // NumRows return the number of row in the page 331 // not counting skipped ones (because of SeekToRow). 332 // The implementation is quite expensive, therefore 333 // we should call it once per page. 334 pageNumRows := x.page.NumRows() 335 x.pageBytes += x.page.Size() 336 x.maxPageRowNum = rn + pageNumRows 337 x.rowsFetched += pageNumRows 338 x.vit.reset(x.page, x.readSize) 339 x.span.LogFields( 340 otlog.String("msg", "Page read"), 341 otlog.Int64("min_rg_row", x.minRGRowNum), 342 otlog.Int64("max_rg_row", x.maxRGRowNum), 343 otlog.Int64("seek_row", x.minRGRowNum+rn), 344 otlog.Int64("page_read_ms", pageReadDurationMs), 345 otlog.Int64("page_num_rows", pageNumRows), 346 ) 347 return true 348 } 349 350 func (x *repeatedRowColumnIterator) At() []parquet.Value { return x.vit.At() } 351 func (x *repeatedRowColumnIterator) Err() error { return x.err } 352 func (x *repeatedRowColumnIterator) Close() (err error) { 353 putRepeatedValuePageIteratorToPool(x.vit) 354 if x.pages != nil { 355 err = x.pages.Close() 356 } 357 x.span.LogFields( 358 otlog.Int64("page_bytes", x.pageBytes), 359 otlog.Int64("rows_fetched", x.rowsFetched), 360 otlog.Int64("rows_read", x.rowsRead), 361 ) 362 x.span.Finish() 363 return err 364 } 365 366 var repeatedValuePageIteratorPool = sync.Pool{New: func() any { return new(repeatedValuePageIterator) }} 367 368 func getRepeatedValuePageIteratorFromPool() *repeatedValuePageIterator { 369 return repeatedValuePageIteratorPool.Get().(*repeatedValuePageIterator) 370 } 371 372 func putRepeatedValuePageIteratorToPool(x *repeatedValuePageIterator) { 373 x.reset(nil, 0) 374 repeatedValuePageIteratorPool.Put(x) 375 } 376 377 // RepeatedValuePageIterator iterates over repeated fields. 378 // FIXME(kolesnikovae): Definition level is ignored. 379 type repeatedValuePageIterator struct { 380 page parquet.ValueReader 381 buf []parquet.Value 382 off int 383 row []parquet.Value 384 err error 385 } 386 387 func (x *repeatedValuePageIterator) At() []parquet.Value { return x.row } 388 func (x *repeatedValuePageIterator) Err() error { return x.err } 389 func (x *repeatedValuePageIterator) Close() error { return nil } 390 391 func (x *repeatedValuePageIterator) reset(page parquet.Page, readSize int) { 392 if cap(x.buf) < readSize { 393 x.buf = make([]parquet.Value, 0, readSize) 394 } 395 x.page = nil 396 if page != nil { 397 x.page = page.Values() 398 } 399 x.buf = x.buf[:0] 400 x.row = x.row[:0] 401 x.err = nil 402 x.off = 0 403 } 404 405 func (x *repeatedValuePageIterator) Next() bool { 406 if x.err != nil { 407 return false 408 } 409 x.row = x.row[:0] 410 var err error 411 var n int 412 loop: 413 for { 414 buf := x.buf[x.off:] 415 for _, v := range buf { 416 if v.RepetitionLevel() == 0 && len(x.row) > 0 { 417 // Found a new row. 418 break loop 419 } 420 x.row = append(x.row, v) 421 x.off++ 422 } 423 // Refill the buffer. 424 x.buf = x.buf[:cap(x.buf)] 425 x.off = 0 426 n, err = x.page.ReadValues(x.buf) 427 x.buf = x.buf[:n] 428 if err != nil && err != io.EOF { 429 x.err = err 430 } 431 if n == 0 { 432 break 433 } 434 } 435 return len(x.row) > 0 436 }