github.com/grafana/pyroscope@v1.18.0/pkg/phlaredb/compact.go (about) 1 package phlaredb 2 3 import ( 4 "context" 5 "crypto/rand" 6 "fmt" 7 "io" 8 "io/fs" 9 "math" 10 "os" 11 "path/filepath" 12 "sort" 13 14 "github.com/go-kit/log" 15 "github.com/go-kit/log/level" 16 "github.com/grafana/dskit/multierror" 17 "github.com/grafana/dskit/runutil" 18 "github.com/oklog/ulid/v2" 19 "github.com/opentracing/opentracing-go" 20 "github.com/opentracing/opentracing-go/ext" 21 "github.com/parquet-go/parquet-go" 22 "github.com/pkg/errors" 23 "github.com/prometheus/common/model" 24 "github.com/prometheus/prometheus/storage" 25 26 "github.com/grafana/pyroscope/pkg/iter" 27 phlaremodel "github.com/grafana/pyroscope/pkg/model" 28 phlareparquet "github.com/grafana/pyroscope/pkg/parquet" 29 "github.com/grafana/pyroscope/pkg/phlaredb/block" 30 "github.com/grafana/pyroscope/pkg/phlaredb/downsample" 31 schemav1 "github.com/grafana/pyroscope/pkg/phlaredb/schemas/v1" 32 "github.com/grafana/pyroscope/pkg/phlaredb/sharding" 33 "github.com/grafana/pyroscope/pkg/phlaredb/symdb" 34 "github.com/grafana/pyroscope/pkg/phlaredb/tsdb/index" 35 "github.com/grafana/pyroscope/pkg/util" 36 "github.com/grafana/pyroscope/pkg/util/loser" 37 ) 38 39 type BlockReader interface { 40 Open(context.Context) error 41 Meta() block.Meta 42 Profiles() ProfileReader 43 Index() IndexReader 44 Symbols() symdb.SymbolsReader 45 Close() error 46 } 47 48 type ProfileReader interface { 49 io.ReaderAt 50 Schema() *parquet.Schema 51 Root() *parquet.Column 52 RowGroups() []parquet.RowGroup 53 } 54 55 type CompactWithSplittingOpts struct { 56 Src []BlockReader 57 Dst string 58 SplitCount uint64 59 StageSize uint64 60 SplitBy SplitByFunc 61 DownsamplerEnabled bool 62 Logger log.Logger 63 } 64 65 func Compact(ctx context.Context, src []BlockReader, dst string) (meta block.Meta, err error) { 66 metas, err := CompactWithSplitting(ctx, CompactWithSplittingOpts{ 67 Src: src, 68 Dst: dst, 69 SplitCount: 1, 70 StageSize: 0, 71 SplitBy: SplitByFingerprint, 72 DownsamplerEnabled: true, 73 Logger: util.Logger, 74 }) 75 if err != nil { 76 return block.Meta{}, err 77 } 78 return metas[0], nil 79 } 80 81 func CompactWithSplitting(ctx context.Context, opts CompactWithSplittingOpts) ( 82 []block.Meta, error, 83 ) { 84 if len(opts.Src) <= 1 && opts.SplitCount == 1 { 85 return nil, errors.New("not enough blocks to compact") 86 } 87 if opts.SplitCount == 0 { 88 opts.SplitCount = 1 89 } 90 if opts.StageSize == 0 || opts.StageSize > opts.SplitCount { 91 opts.StageSize = opts.SplitCount 92 } 93 var ( 94 writers = make([]*blockWriter, opts.SplitCount) 95 srcMetas = make([]block.Meta, len(opts.Src)) 96 outMetas = make([]block.Meta, 0, len(writers)) 97 err error 98 ) 99 for i, b := range opts.Src { 100 srcMetas[i] = b.Meta() 101 } 102 103 symbolsCompactor := newSymbolsCompactor(opts.Dst, symdb.FormatV2) 104 defer runutil.CloseWithLogOnErr(util.Logger, symbolsCompactor, "close symbols compactor") 105 106 outMeta := compactMetas(srcMetas...) 107 for _, stage := range splitStages(len(writers), int(opts.StageSize)) { 108 for _, idx := range stage { 109 if writers[idx], err = createBlockWriter(blockWriterOpts{ 110 dst: opts.Dst, 111 meta: outMeta, 112 splitCount: opts.SplitCount, 113 shard: idx, 114 rewriterFn: symbolsCompactor.Rewriter, 115 downsamplerEnabled: opts.DownsamplerEnabled, 116 logger: opts.Logger, 117 }); err != nil { 118 return nil, fmt.Errorf("create block writer: %w", err) 119 } 120 } 121 var metas []block.Meta 122 sp, ctx := opentracing.StartSpanFromContext(ctx, "compact.Stage", opentracing.Tag{Key: "stage", Value: stage}) 123 if metas, err = compact(ctx, writers, opts.Src, opts.SplitBy, opts.SplitCount); err != nil { 124 sp.Finish() 125 ext.LogError(sp, err) 126 return nil, err 127 } 128 sp.Finish() 129 outMetas = append(outMetas, metas...) 130 // Writers are already closed, and must be GCed. 131 for j := range writers { 132 writers[j] = nil 133 } 134 } 135 136 return outMetas, nil 137 } 138 139 // splitStages splits n into sequences of size s: 140 // For n=7, s=3: [[0 1 2] [3 4 5] [6]] 141 func splitStages(n, s int) (stages [][]int) { 142 for i := 0; i < n; i += s { 143 end := i + s 144 if end > n { 145 end = n 146 } 147 b := make([]int, end-i) 148 for j := i; j < end; j++ { 149 b[j-i] = j 150 } 151 stages = append(stages, b) 152 } 153 return stages 154 } 155 156 func createBlockWriter(opts blockWriterOpts) (*blockWriter, error) { 157 meta := opts.meta.Clone() 158 meta.ULID = ulid.MustNew(meta.ULID.Time(), rand.Reader) 159 if opts.splitCount > 1 { 160 if meta.Labels == nil { 161 meta.Labels = make(map[string]string) 162 } 163 meta.Labels[sharding.CompactorShardIDLabel] = sharding.FormatShardIDLabelValue(uint64(opts.shard), opts.splitCount) 164 } 165 opts.meta = *meta 166 return newBlockWriter(opts) 167 } 168 169 func compact(ctx context.Context, writers []*blockWriter, readers []BlockReader, splitBy SplitByFunc, splitCount uint64) ([]block.Meta, error) { 170 rowsIt, err := newMergeRowProfileIterator(readers) 171 if err != nil { 172 return nil, err 173 } 174 defer runutil.CloseWithLogOnErr(util.Logger, rowsIt, "close rows iterator") 175 // iterate and splits the rows into series. 176 for rowsIt.Next() { 177 r := rowsIt.At() 178 shard := int(splitBy(r, splitCount)) 179 w := writers[shard] 180 if w == nil { 181 continue 182 } 183 if err = w.WriteRow(r); err != nil { 184 return nil, err 185 } 186 } 187 if err = rowsIt.Err(); err != nil { 188 return nil, err 189 } 190 sp, ctx := opentracing.StartSpanFromContext(ctx, "compact.Close") 191 defer sp.Finish() 192 193 // Close all blocks 194 errs := multierror.New() 195 for _, w := range writers { 196 if w == nil { 197 continue 198 } 199 if err = w.Close(ctx); err != nil { 200 errs.Add(err) 201 } 202 } 203 204 out := make([]block.Meta, 0, len(writers)) 205 for _, w := range writers { 206 if w == nil { 207 continue 208 } 209 if w.meta.Stats.NumSamples > 0 { 210 out = append(out, *w.meta) 211 } 212 } 213 214 // Returns all Metas 215 return out, errs.Err() 216 } 217 218 type SplitByFunc func(r profileRow, shardsCount uint64) uint64 219 220 var SplitByFingerprint = func(r profileRow, shardsCount uint64) uint64 { 221 return uint64(r.fp) % shardsCount 222 } 223 224 var SplitByStacktracePartition = func(r profileRow, shardsCount uint64) uint64 { 225 return r.row.StacktracePartitionID() % shardsCount 226 } 227 228 type blockWriter struct { 229 indexRewriter *indexRewriter 230 symbolsRewriter SymbolsRewriter 231 profilesWriter *profilesWriter 232 downsampler *downsample.Downsampler 233 path string 234 meta *block.Meta 235 totalProfiles uint64 236 } 237 238 type SymbolsRewriter interface { 239 ReWriteRow(profile profileRow) error 240 Close() (uint64, error) 241 } 242 243 type SymbolsRewriterFn func(blockPath string) SymbolsRewriter 244 245 type blockWriterOpts struct { 246 dst string 247 splitCount uint64 248 shard int 249 meta block.Meta 250 rewriterFn SymbolsRewriterFn 251 downsamplerEnabled bool 252 logger log.Logger 253 } 254 255 func newBlockWriter(opts blockWriterOpts) (*blockWriter, error) { 256 blockPath := filepath.Join(opts.dst, opts.meta.ULID.String()) 257 258 if err := os.MkdirAll(blockPath, 0o777); err != nil { 259 return nil, err 260 } 261 262 profileWriter, err := newProfileWriter(blockPath) 263 if err != nil { 264 return nil, err 265 } 266 267 var downsampler *downsample.Downsampler 268 if opts.downsamplerEnabled && opts.meta.Compaction.Level > 2 { 269 level.Debug(opts.logger).Log("msg", "downsampling enabled for block writer", "path", blockPath) 270 downsampler, err = downsample.NewDownsampler(blockPath, opts.logger) 271 if err != nil { 272 return nil, err 273 } 274 } 275 276 return &blockWriter{ 277 indexRewriter: newIndexRewriter(blockPath), 278 symbolsRewriter: opts.rewriterFn(blockPath), 279 profilesWriter: profileWriter, 280 downsampler: downsampler, 281 path: blockPath, 282 meta: &opts.meta, 283 }, nil 284 } 285 286 func (bw *blockWriter) WriteRow(r profileRow) error { 287 err := bw.indexRewriter.ReWriteRow(r) 288 if err != nil { 289 return err 290 } 291 err = bw.symbolsRewriter.ReWriteRow(r) 292 if err != nil { 293 return err 294 } 295 296 if err := bw.profilesWriter.WriteRow(r); err != nil { 297 return err 298 } 299 if bw.downsampler != nil { 300 err := bw.downsampler.AddRow(r.row, r.fp) 301 if err != nil { 302 return err 303 } 304 } 305 bw.totalProfiles++ 306 return nil 307 } 308 309 func (bw *blockWriter) Close(ctx context.Context) error { 310 if err := bw.indexRewriter.Close(ctx); err != nil { 311 return err 312 } 313 numSamples, err := bw.symbolsRewriter.Close() 314 if err != nil { 315 return err 316 } 317 if err := bw.profilesWriter.Close(); err != nil { 318 return err 319 } 320 if bw.downsampler != nil { 321 if err := bw.downsampler.Close(); err != nil { 322 return err 323 } 324 } 325 metaFiles, err := metaFilesFromDir(bw.path) 326 if err != nil { 327 return err 328 } 329 bw.meta.Files = metaFiles 330 bw.meta.Stats.NumProfiles = bw.totalProfiles 331 bw.meta.Stats.NumSeries = bw.indexRewriter.NumSeries() 332 bw.meta.Stats.NumSamples = numSamples 333 bw.meta.Compaction.Deletable = bw.totalProfiles == 0 334 if _, err := bw.meta.WriteToFile(util.Logger, bw.path); err != nil { 335 return err 336 } 337 return nil 338 } 339 340 type profilesWriter struct { 341 *parquet.GenericWriter[*schemav1.Profile] 342 file *os.File 343 344 buf []parquet.Row 345 } 346 347 func newProfileWriter(path string) (*profilesWriter, error) { 348 profilePath := filepath.Join(path, (&schemav1.ProfilePersister{}).Name()+block.ParquetSuffix) 349 profileFile, err := os.OpenFile(profilePath, os.O_RDWR|os.O_CREATE|os.O_EXCL, 0o644) 350 if err != nil { 351 return nil, err 352 } 353 return &profilesWriter{ 354 GenericWriter: newParquetProfileWriter(profileFile, parquet.MaxRowsPerRowGroup(int64(defaultParquetConfig.MaxBufferRowCount))), 355 file: profileFile, 356 buf: make([]parquet.Row, 1), 357 }, nil 358 } 359 360 func (p *profilesWriter) WriteRow(r profileRow) error { 361 p.buf[0] = parquet.Row(r.row) 362 _, err := p.WriteRows(p.buf) 363 if err != nil { 364 return err 365 } 366 367 return nil 368 } 369 370 func (p *profilesWriter) Close() error { 371 err := p.GenericWriter.Close() 372 if err != nil { 373 return err 374 } 375 return p.file.Close() 376 } 377 378 func newIndexRewriter(path string) *indexRewriter { 379 return &indexRewriter{ 380 symbols: make(map[string]struct{}), 381 path: path, 382 } 383 } 384 385 type indexRewriter struct { 386 series []struct { 387 labels phlaremodel.Labels 388 fp model.Fingerprint 389 } 390 symbols map[string]struct{} 391 chunks []index.ChunkMeta // one chunk per series 392 393 previousFp model.Fingerprint 394 395 path string 396 } 397 398 func (idxRw *indexRewriter) ReWriteRow(r profileRow) error { 399 if idxRw.previousFp != r.fp || len(idxRw.series) == 0 { 400 series := r.labels.Clone() 401 for _, l := range series { 402 idxRw.symbols[l.Name] = struct{}{} 403 idxRw.symbols[l.Value] = struct{}{} 404 } 405 idxRw.series = append(idxRw.series, struct { 406 labels phlaremodel.Labels 407 fp model.Fingerprint 408 }{ 409 labels: series, 410 fp: r.fp, 411 }) 412 idxRw.chunks = append(idxRw.chunks, index.ChunkMeta{ 413 MinTime: r.timeNanos, 414 MaxTime: r.timeNanos, 415 SeriesIndex: uint32(len(idxRw.series) - 1), 416 }) 417 idxRw.previousFp = r.fp 418 } 419 idxRw.chunks[len(idxRw.chunks)-1].MaxTime = r.timeNanos 420 r.row.SetSeriesIndex(idxRw.chunks[len(idxRw.chunks)-1].SeriesIndex) 421 return nil 422 } 423 424 func (idxRw *indexRewriter) NumSeries() uint64 { 425 return uint64(len(idxRw.series)) 426 } 427 428 // Close writes the index to given folder. 429 func (idxRw *indexRewriter) Close(ctx context.Context) error { 430 indexw, err := index.NewWriter(ctx, filepath.Join(idxRw.path, block.IndexFilename)) 431 if err != nil { 432 return err 433 } 434 435 // Sort symbols 436 symbols := make([]string, 0, len(idxRw.symbols)) 437 for s := range idxRw.symbols { 438 symbols = append(symbols, s) 439 } 440 sort.Strings(symbols) 441 442 // Add symbols 443 for _, symbol := range symbols { 444 if err := indexw.AddSymbol(symbol); err != nil { 445 return err 446 } 447 } 448 449 // Add Series 450 for i, series := range idxRw.series { 451 if err := indexw.AddSeries(storage.SeriesRef(i), series.labels, series.fp, idxRw.chunks[i]); err != nil { 452 return err 453 } 454 } 455 456 return indexw.Close() 457 } 458 459 // metaFilesFromDir returns a list of block files description from a directory. 460 func metaFilesFromDir(dir string) ([]block.File, error) { 461 var files []block.File 462 err := filepath.Walk(dir, func(path string, info fs.FileInfo, err error) error { 463 if err != nil { 464 return err 465 } 466 if info.IsDir() { 467 return nil 468 } 469 var f block.File 470 switch filepath.Ext(info.Name()) { 471 case block.ParquetSuffix: 472 f, err = parquetMetaFile(path, info.Size()) 473 if err != nil { 474 return err 475 } 476 case filepath.Ext(block.IndexFilename): 477 f, err = tsdbMetaFile(path) 478 if err != nil { 479 return err 480 } 481 } 482 f.RelPath, err = filepath.Rel(dir, path) 483 if err != nil { 484 return err 485 } 486 f.SizeBytes = uint64(info.Size()) 487 files = append(files, f) 488 return nil 489 }) 490 return files, err 491 } 492 493 func tsdbMetaFile(filePath string) (block.File, error) { 494 idxReader, err := index.NewFileReader(filePath) 495 if err != nil { 496 return block.File{}, err 497 } 498 499 return idxReader.FileInfo(), nil 500 } 501 502 func parquetMetaFile(filePath string, size int64) (block.File, error) { 503 f, err := os.Open(filePath) 504 if err != nil { 505 return block.File{}, err 506 } 507 defer f.Close() 508 509 pqFile, err := parquet.OpenFile(f, size) 510 if err != nil { 511 return block.File{}, err 512 } 513 return block.File{ 514 Parquet: &block.ParquetFile{ 515 NumRowGroups: uint64(len(pqFile.RowGroups())), 516 NumRows: uint64(pqFile.NumRows()), 517 }, 518 }, nil 519 } 520 521 func compactMetas(src ...block.Meta) block.Meta { 522 meta := block.NewMeta() 523 highestCompactionLevel := 0 524 sources := map[ulid.ULID]struct{}{} 525 parents := make([]block.BlockDesc, 0, len(src)) 526 minTime, maxTime := model.Latest, model.Earliest 527 labels := make(map[string]string) 528 for _, b := range src { 529 if b.Compaction.Level > highestCompactionLevel { 530 highestCompactionLevel = b.Compaction.Level 531 } 532 for _, s := range b.Compaction.Sources { 533 sources[s] = struct{}{} 534 } 535 parents = append(parents, block.BlockDesc{ 536 ULID: b.ULID, 537 MinTime: b.MinTime, 538 MaxTime: b.MaxTime, 539 }) 540 if b.MinTime < minTime { 541 minTime = b.MinTime 542 } 543 if b.MaxTime > maxTime { 544 maxTime = b.MaxTime 545 } 546 for k, v := range b.Labels { 547 if k == block.HostnameLabel { 548 continue 549 } 550 labels[k] = v 551 } 552 } 553 meta.Source = block.CompactorSource 554 meta.Compaction = block.BlockMetaCompaction{ 555 Deletable: false, 556 Level: highestCompactionLevel + 1, 557 Parents: parents, 558 } 559 for s := range sources { 560 meta.Compaction.Sources = append(meta.Compaction.Sources, s) 561 } 562 sort.Slice(meta.Compaction.Sources, func(i, j int) bool { 563 return meta.Compaction.Sources[i].Compare(meta.Compaction.Sources[j]) < 0 564 }) 565 meta.MaxTime = maxTime 566 meta.MinTime = minTime 567 meta.Labels = labels 568 meta.ULID = ulid.MustNew(uint64(minTime), rand.Reader) 569 return *meta 570 } 571 572 type profileRow struct { 573 timeNanos int64 574 575 labels phlaremodel.Labels 576 fp model.Fingerprint 577 row schemav1.ProfileRow 578 579 blockReader BlockReader 580 } 581 582 type profileRowIterator struct { 583 profiles iter.Iterator[parquet.Row] 584 blockReader BlockReader 585 closer io.Closer 586 index IndexReader 587 allPostings index.Postings 588 err error 589 590 currentRow profileRow 591 currentSeriesIdx uint32 592 chunks []index.ChunkMeta 593 } 594 595 func newProfileRowIterator(s BlockReader) (*profileRowIterator, error) { 596 k, v := index.AllPostingsKey() 597 allPostings, err := s.Index().Postings(k, nil, v) 598 if err != nil { 599 return nil, err 600 } 601 // todo close once https://github.com/grafana/pyroscope/issues/2172 is done. 602 reader := parquet.NewReader(s.Profiles(), schemav1.ProfilesSchema) 603 return &profileRowIterator{ 604 profiles: phlareparquet.NewBufferedRowReaderIterator(reader, 32), 605 blockReader: s, 606 closer: reader, 607 index: s.Index(), 608 allPostings: allPostings, 609 currentSeriesIdx: math.MaxUint32, 610 chunks: make([]index.ChunkMeta, 1), 611 }, nil 612 } 613 614 func (p *profileRowIterator) At() profileRow { 615 return p.currentRow 616 } 617 618 func (p *profileRowIterator) Next() bool { 619 if !p.profiles.Next() { 620 return false 621 } 622 p.currentRow.blockReader = p.blockReader 623 p.currentRow.row = schemav1.ProfileRow(p.profiles.At()) 624 seriesIndex := p.currentRow.row.SeriesIndex() 625 p.currentRow.timeNanos = p.currentRow.row.TimeNanos() 626 // do we have a new series? 627 if seriesIndex == p.currentSeriesIdx { 628 return true 629 } 630 p.currentSeriesIdx = seriesIndex 631 if !p.allPostings.Next() { 632 if err := p.allPostings.Err(); err != nil { 633 p.err = err 634 return false 635 } 636 p.err = errors.New("unexpected end of postings") 637 return false 638 } 639 640 fp, err := p.index.Series(p.allPostings.At(), &p.currentRow.labels, &p.chunks) 641 if err != nil { 642 p.err = err 643 return false 644 } 645 p.currentRow.fp = model.Fingerprint(fp) 646 return true 647 } 648 649 func (p *profileRowIterator) Err() error { 650 if p.err != nil { 651 return p.err 652 } 653 return p.profiles.Err() 654 } 655 656 func (p *profileRowIterator) Close() error { 657 err := p.profiles.Close() 658 if p.closer != nil { 659 if err := p.closer.Close(); err != nil { 660 return err 661 } 662 } 663 return err 664 } 665 666 func newMergeRowProfileIterator(src []BlockReader) (iter.Iterator[profileRow], error) { 667 its := make([]iter.Iterator[profileRow], len(src)) 668 for i, s := range src { 669 it, err := newProfileRowIterator(s) 670 if err != nil { 671 return nil, err 672 } 673 its[i] = it 674 } 675 if len(its) == 1 { 676 return its[0], nil 677 } 678 return &dedupeProfileRowIterator{ 679 Iterator: iter.NewTreeIterator(loser.New( 680 its, 681 profileRow{ 682 timeNanos: math.MaxInt64, 683 }, 684 func(it iter.Iterator[profileRow]) profileRow { return it.At() }, 685 func(r1, r2 profileRow) bool { 686 // first handle max profileRow if it's either r1 or r2 687 if r1.timeNanos == math.MaxInt64 { 688 return false 689 } 690 if r2.timeNanos == math.MaxInt64 { 691 return true 692 } 693 // then handle normal profileRows 694 if cmp := phlaremodel.CompareLabelPairs(r1.labels, r2.labels); cmp != 0 { 695 return cmp < 0 696 } 697 return r1.timeNanos < r2.timeNanos 698 }, 699 func(it iter.Iterator[profileRow]) { _ = it.Close() }, 700 )), 701 }, nil 702 } 703 704 type dedupeProfileRowIterator struct { 705 iter.Iterator[profileRow] 706 707 prevFP model.Fingerprint 708 prevTimeNanos int64 709 } 710 711 func (it *dedupeProfileRowIterator) Next() bool { 712 for { 713 if !it.Iterator.Next() { 714 return false 715 } 716 currentProfile := it.At() 717 if it.prevFP == currentProfile.fp && it.prevTimeNanos == currentProfile.timeNanos { 718 // skip duplicate profile 719 continue 720 } 721 it.prevFP = currentProfile.fp 722 it.prevTimeNanos = currentProfile.timeNanos 723 return true 724 } 725 } 726 727 type symbolsCompactor struct { 728 version symdb.FormatVersion 729 rewriters map[BlockReader]*symdb.Rewriter 730 w *symdb.SymDB 731 stacktraces []uint32 732 733 dst string 734 flushed bool 735 } 736 737 func newSymbolsCompactor(path string, version symdb.FormatVersion) *symbolsCompactor { 738 if version == symdb.FormatV3 { 739 return &symbolsCompactor{ 740 version: version, 741 w: symdb.NewSymDB(symdb.DefaultConfig(). 742 WithVersion(symdb.FormatV3). 743 WithDirectory(path)), 744 dst: path, 745 rewriters: make(map[BlockReader]*symdb.Rewriter), 746 } 747 } 748 dst := filepath.Join(path, symdb.DefaultDirName) 749 return &symbolsCompactor{ 750 version: symdb.FormatV2, 751 w: symdb.NewSymDB(symdb.DefaultConfig(). 752 WithVersion(symdb.FormatV2). 753 WithDirectory(dst)), 754 dst: dst, 755 rewriters: make(map[BlockReader]*symdb.Rewriter), 756 } 757 } 758 759 func (s *symbolsCompactor) Rewriter(dst string) SymbolsRewriter { 760 return &symbolsRewriter{ 761 symbolsCompactor: s, 762 dst: dst, 763 } 764 } 765 766 type symbolsRewriter struct { 767 *symbolsCompactor 768 769 numSamples uint64 770 dst string 771 } 772 773 func (s *symbolsRewriter) NumSamples() uint64 { return s.numSamples } 774 775 func (s *symbolsRewriter) ReWriteRow(profile profileRow) error { 776 total, err := s.symbolsCompactor.ReWriteRow(profile) 777 s.numSamples += total 778 return err 779 } 780 781 func (s *symbolsRewriter) Close() (uint64, error) { 782 if err := s.Flush(); err != nil { 783 return 0, err 784 } 785 if s.version == symdb.FormatV3 { 786 dst := filepath.Join(s.dst, symdb.DefaultFileName) 787 src := filepath.Join(s.symbolsCompactor.dst, symdb.DefaultFileName) 788 return s.numSamples, util.CopyFile(src, dst) 789 } else { 790 return s.numSamples, util.CopyDir(s.symbolsCompactor.dst, filepath.Join(s.dst, symdb.DefaultDirName)) 791 } 792 } 793 794 func (s *symbolsCompactor) ReWriteRow(profile profileRow) (uint64, error) { 795 var ( 796 err error 797 rewrittenSamples uint64 798 ) 799 profile.row.ForStacktraceIDsValues(func(values []parquet.Value) { 800 s.loadStacktracesID(values) 801 r, ok := s.rewriters[profile.blockReader] 802 if !ok { 803 r = symdb.NewRewriter(s.w, profile.blockReader.Symbols(), nil) 804 s.rewriters[profile.blockReader] = r 805 } 806 if err = r.Rewrite(profile.row.StacktracePartitionID(), s.stacktraces); err != nil { 807 return 808 } 809 rewrittenSamples += uint64(len(values)) 810 for i, v := range values { 811 // FIXME: the original order is not preserved, which will affect encoding. 812 values[i] = parquet.Int64Value(int64(s.stacktraces[i])).Level(v.RepetitionLevel(), v.DefinitionLevel(), v.Column()) 813 } 814 }) 815 if err != nil { 816 return rewrittenSamples, err 817 } 818 return rewrittenSamples, nil 819 } 820 821 func (s *symbolsCompactor) Flush() error { 822 if s.flushed { 823 return nil 824 } 825 if err := s.w.Flush(); err != nil { 826 return err 827 } 828 s.flushed = true 829 return nil 830 } 831 832 func (s *symbolsCompactor) Close() error { 833 if s.version == symdb.FormatV3 { 834 return os.RemoveAll(filepath.Join(s.dst, symdb.DefaultFileName)) 835 } 836 return os.RemoveAll(s.dst) 837 } 838 839 func (s *symbolsCompactor) loadStacktracesID(values []parquet.Value) { 840 s.stacktraces = grow(s.stacktraces, len(values)) 841 for i := range values { 842 s.stacktraces[i] = values[i].Uint32() 843 } 844 }