From 0ebf54de7e7d4b0c1f75d8d7c9eb233ba6ffb158 Mon Sep 17 00:00:00 2001 From: Christian Simon Date: Tue, 21 Jun 2022 08:36:37 +0100 Subject: [PATCH 01/35] Testing From 3c8b16e028e42bf56268f98877aad29ce72c2d18 Mon Sep 17 00:00:00 2001 From: Christian Simon Date: Tue, 21 Jun 2022 08:37:24 +0100 Subject: [PATCH 02/35] Testing v2 From 03274dfc0cda965f5a3ba93178684042a0f9c3d5 Mon Sep 17 00:00:00 2001 From: Cyril Tovena Date: Thu, 6 Jul 2023 16:59:56 +0200 Subject: [PATCH 03/35] Add first draft of block compaction --- pkg/iter/tree.go | 4 + pkg/phlaredb/compact.go | 301 ++++++++++++++++++++++++++++ pkg/phlaredb/profile_store.go | 14 +- pkg/phlaredb/schemas/v1/profiles.go | 21 ++ pkg/phlaredb/tsdb/builder.go | 21 ++ 5 files changed, 356 insertions(+), 5 deletions(-) create mode 100644 pkg/phlaredb/compact.go create mode 100644 pkg/phlaredb/tsdb/builder.go diff --git a/pkg/iter/tree.go b/pkg/iter/tree.go index 44c73ba430..fb810d64eb 100644 --- a/pkg/iter/tree.go +++ b/pkg/iter/tree.go @@ -1,7 +1,11 @@ package iter import ( +<<<<<<< HEAD "github.com/grafana/pyroscope/pkg/util/loser" +======= + "github.com/grafana/phlare/pkg/util/loser" +>>>>>>> ee8a92e04 (Add first draft of block compaction) ) var _ Iterator[interface{}] = &TreeIterator[interface{}]{} diff --git a/pkg/phlaredb/compact.go b/pkg/phlaredb/compact.go new file mode 100644 index 0000000000..6966595567 --- /dev/null +++ b/pkg/phlaredb/compact.go @@ -0,0 +1,301 @@ +package phlaredb + +import ( + "context" + "math" + "os" + "path/filepath" + + "github.com/pkg/errors" + "github.com/prometheus/common/model" + "github.com/prometheus/prometheus/storage" + "github.com/prometheus/prometheus/tsdb" + "github.com/segmentio/parquet-go" + + "github.com/grafana/phlare/pkg/iter" + phlaremodel "github.com/grafana/phlare/pkg/model" + phlareparquet "github.com/grafana/phlare/pkg/parquet" + "github.com/grafana/phlare/pkg/phlaredb/block" + schemav1 "github.com/grafana/phlare/pkg/phlaredb/schemas/v1" + "github.com/grafana/phlare/pkg/phlaredb/tsdb/index" + "github.com/grafana/phlare/pkg/util" + "github.com/grafana/phlare/pkg/util/loser" +) + +type BlockReader interface { + Profiles() []parquet.RowGroup + Index() IndexReader + // Symbols() SymbolReader +} + +type SymbolReader interface { + // todo +} + +func Compact(ctx context.Context, src []BlockReader, dst string) (block.Meta, error) { + meta := block.NewMeta() + blockPath := filepath.Join(dst, meta.ULID.String()) + if err := os.MkdirAll(blockPath, 0o777); err != nil { + return block.Meta{}, err + } + indexPath := filepath.Join(blockPath, block.IndexFilename) + indexw, err := prepareIndexWriter(ctx, indexPath, src) + if err != nil { + return block.Meta{}, err + } + profilePath := filepath.Join(blockPath, (&schemav1.ProfilePersister{}).Name()+block.ParquetSuffix) + profileFile, err := os.OpenFile(profilePath, os.O_RDWR|os.O_CREATE|os.O_EXCL, 0o644) + if err != nil { + return block.Meta{}, err + } + profileWriter := newProfileWriter(profileFile) + + // todo new symbdb + + rowsIt := newMergeRowProfileIterator(src) + rowsIt = newSeriesRewriter(rowsIt, indexw) + rowsIt = newSymbolsRewriter(rowsIt) + reader := phlareparquet.NewIteratorRowReader(newRowsIterator(rowsIt)) + + // todo size of rowgroups. + _, _, err = phlareparquet.CopyAsRowGroups(profileWriter, reader, 1024) + if err != nil { + return block.Meta{}, err + } + + // flush the index file. + if err := indexw.Close(); err != nil { + return block.Meta{}, err + } + + if err := profileWriter.Close(); err != nil { + return block.Meta{}, err + } + // todo: block meta + if _, err := meta.WriteToFile(util.Logger, blockPath); err != nil { + return block.Meta{}, err + } + return *meta, nil +} + +type profileRow struct { + timeNanos int64 + + seriesRef uint32 + labels phlaremodel.Labels + fp model.Fingerprint + row schemav1.ProfileRow +} + +type profileRowIterator struct { + profiles iter.Iterator[parquet.Row] + index IndexReader + err error + + currentRow profileRow + chunks []index.ChunkMeta +} + +func newProfileRowIterator(profiles iter.Iterator[parquet.Row], idx IndexReader) *profileRowIterator { + return &profileRowIterator{ + profiles: profiles, + index: idx, + currentRow: profileRow{ + seriesRef: math.MaxUint32, + }, + chunks: make([]index.ChunkMeta, 1), + } +} + +func (p *profileRowIterator) At() profileRow { + return p.currentRow +} + +func (p *profileRowIterator) Next() bool { + if !p.profiles.Next() { + return false + } + p.currentRow.row = schemav1.ProfileRow(p.profiles.At()) + seriesIndex := p.currentRow.row.SeriesIndex() + p.currentRow.timeNanos = p.currentRow.row.TimeNanos() + // do we have a new series? + if seriesIndex == p.currentRow.seriesRef { + return true + } + p.currentRow.seriesRef = seriesIndex + fp, err := p.index.Series(storage.SeriesRef(p.currentRow.seriesRef), &p.currentRow.labels, &p.chunks) + if err != nil { + p.err = err + return false + } + p.currentRow.fp = model.Fingerprint(fp) + return true +} + +func (p *profileRowIterator) Err() error { + if p.err != nil { + return p.err + } + return p.profiles.Err() +} + +func (p *profileRowIterator) Close() error { + return p.profiles.Close() +} + +func newMergeRowProfileIterator(src []BlockReader) iter.Iterator[profileRow] { + its := make([]iter.Iterator[profileRow], len(src)) + for i, s := range src { + // todo: may be we could merge rowgroups in parallel but that requires locking. + reader := parquet.MultiRowGroup(s.Profiles()...).Rows() + its[i] = newProfileRowIterator( + phlareparquet.NewBufferedRowReaderIterator(reader, 1024), + s.Index(), + ) + } + return &dedupeProfileRowIterator{ + Iterator: iter.NewTreeIterator(loser.New( + its, + profileRow{ + timeNanos: math.MaxInt64, + }, + func(it iter.Iterator[profileRow]) profileRow { return it.At() }, + func(r1, r2 profileRow) bool { + // first handle max profileRow if it's either r1 or r2 + if r1.timeNanos == math.MaxInt64 { + return false + } + if r2.timeNanos == math.MaxInt64 { + return true + } + // then handle normal profileRows + if cmp := phlaremodel.CompareLabelPairs(r1.labels, r2.labels); cmp != 0 { + return cmp < 0 + } + return r1.timeNanos < r2.timeNanos + }, + func(it iter.Iterator[profileRow]) { _ = it.Close() }, + )), + } +} + +type symbolsRewriter struct { + iter.Iterator[profileRow] +} + +// todo remap symbols & ingest symbols +func newSymbolsRewriter(it iter.Iterator[profileRow]) *symbolsRewriter { + return &symbolsRewriter{ + Iterator: it, + } +} + +type seriesRewriter struct { + iter.Iterator[profileRow] + + indexw *index.Writer + + seriesRef storage.SeriesRef + labels phlaremodel.Labels + previousFp model.Fingerprint + currentChunkMeta index.ChunkMeta + err error +} + +func newSeriesRewriter(it iter.Iterator[profileRow], indexw *index.Writer) *seriesRewriter { + return &seriesRewriter{ + Iterator: it, + indexw: indexw, + } +} + +func (s *seriesRewriter) Next() bool { + if !s.Iterator.Next() { + if s.previousFp != 0 { + if err := s.indexw.AddSeries(s.seriesRef, s.labels, s.previousFp, s.currentChunkMeta); err != nil { + s.err = err + return false + } + } + return false + } + currentProfile := s.Iterator.At() + if s.previousFp != currentProfile.fp { + // store the previous series. + if s.previousFp != 0 { + if err := s.indexw.AddSeries(s.seriesRef, s.labels, s.previousFp, s.currentChunkMeta); err != nil { + s.err = err + return false + } + } + s.seriesRef++ + s.labels = currentProfile.labels.Clone() + s.previousFp = currentProfile.fp + s.currentChunkMeta.MinTime = currentProfile.timeNanos + } + s.currentChunkMeta.MaxTime = currentProfile.timeNanos + currentProfile.row.SetSeriesIndex(uint32(s.seriesRef)) + return true +} + +type rowsIterator struct { + iter.Iterator[profileRow] +} + +func newRowsIterator(it iter.Iterator[profileRow]) *rowsIterator { + return &rowsIterator{ + Iterator: it, + } +} + +func (r *rowsIterator) At() parquet.Row { + return parquet.Row(r.Iterator.At().row) +} + +type dedupeProfileRowIterator struct { + iter.Iterator[profileRow] + + prevFP model.Fingerprint + prevTimeNanos int64 +} + +func (it *dedupeProfileRowIterator) Next() bool { + for { + if !it.Iterator.Next() { + return false + } + currentProfile := it.Iterator.At() + if it.prevFP == currentProfile.fp && it.prevTimeNanos == currentProfile.timeNanos { + // skip duplicate profile + continue + } + it.prevFP = currentProfile.fp + it.prevTimeNanos = currentProfile.timeNanos + return true + } +} + +func prepareIndexWriter(ctx context.Context, path string, readers []BlockReader) (*index.Writer, error) { + var symbols index.StringIter + indexw, err := index.NewWriter(ctx, path) + if err != nil { + return nil, err + } + for i, r := range readers { + if i == 0 { + symbols = r.Index().Symbols() + } + symbols = tsdb.NewMergedStringIter(symbols, r.Index().Symbols()) + } + + for symbols.Next() { + if err := indexw.AddSymbol(symbols.At()); err != nil { + return nil, errors.Wrap(err, "add symbol") + } + } + if symbols.Err() != nil { + return nil, errors.Wrap(symbols.Err(), "next symbol") + } + + return indexw, nil +} diff --git a/pkg/phlaredb/profile_store.go b/pkg/phlaredb/profile_store.go index fb25d8d992..94d5d0d465 100644 --- a/pkg/phlaredb/profile_store.go +++ b/pkg/phlaredb/profile_store.go @@ -63,6 +63,14 @@ type profileStore struct { flushBufferLbs []phlaremodel.Labels } +func newProfileWriter(writer io.Writer) *parquet.GenericWriter[*schemav1.Profile] { + return parquet.NewGenericWriter[*schemav1.Profile](writer, (&schemav1.ProfilePersister{}).Schema(), + parquet.ColumnPageBuffers(parquet.NewFileBufferPool(os.TempDir(), "phlaredb-parquet-buffers*")), + parquet.CreatedBy("github.com/grafana/phlare/", build.Version, build.Revision), + parquet.PageBufferSize(3*1024*1024), + ) +} + func newProfileStore(phlarectx context.Context) *profileStore { s := &profileStore{ logger: phlarecontext.Logger(phlarectx), @@ -76,11 +84,7 @@ func newProfileStore(phlarectx context.Context) *profileStore { go s.cutRowGroupLoop() // Initialize writer on /dev/null // TODO: Reuse parquet.Writer beyond life time of the head. - s.writer = parquet.NewGenericWriter[*schemav1.Profile](io.Discard, s.persister.Schema(), - parquet.ColumnPageBuffers(parquet.NewFileBufferPool(os.TempDir(), "phlaredb-parquet-buffers*")), - parquet.CreatedBy("github.com/grafana/pyroscope/", build.Version, build.Revision), - parquet.PageBufferSize(3*1024*1024), - ) + s.writer = newProfileWriter(io.Discard) return s } diff --git a/pkg/phlaredb/schemas/v1/profiles.go b/pkg/phlaredb/schemas/v1/profiles.go index 157d4edee1..cd04d17fd7 100644 --- a/pkg/phlaredb/schemas/v1/profiles.go +++ b/pkg/phlaredb/schemas/v1/profiles.go @@ -458,3 +458,24 @@ func lessProfileRows(r1, r2 parquet.Row) bool { } return ts1 < ts2 } + +type ProfileRow parquet.Row + +func (p ProfileRow) SeriesIndex() uint32 { + return p[seriesIndexColIndex].Uint32() +} + +func (p ProfileRow) TimeNanos() int64 { + var ts int64 + for i := len(p) - 1; i >= 0; i-- { + if p[i].Column() == timeNanoColIndex { + ts = p[i].Int64() + break + } + } + return ts +} + +func (p ProfileRow) SetSeriesIndex(v uint32) { + p[seriesIndexColIndex] = parquet.Int32Value(int32(v)).Level(0, 0, seriesIndexColIndex) +} diff --git a/pkg/phlaredb/tsdb/builder.go b/pkg/phlaredb/tsdb/builder.go new file mode 100644 index 0000000000..0125c86967 --- /dev/null +++ b/pkg/phlaredb/tsdb/builder.go @@ -0,0 +1,21 @@ +package tsdb + +import ( + phlaremodel "github.com/grafana/phlare/pkg/model" + "github.com/prometheus/common/model" +) + +type Series struct { + labels phlaremodel.Labels + fp model.Fingerprint +} + +type Builder struct { + series map[string]Series +} + +func NewBuilder() *Builder { + return &Builder{ + series: map[string]Series{}, + } +} From 021abc55d13ade2480c3bdcd2b618d5ff6b9ed8c Mon Sep 17 00:00:00 2001 From: Cyril Tovena Date: Thu, 6 Jul 2023 17:01:26 +0200 Subject: [PATCH 04/35] Removes unused file --- pkg/phlaredb/tsdb/builder.go | 21 --------------------- 1 file changed, 21 deletions(-) delete mode 100644 pkg/phlaredb/tsdb/builder.go diff --git a/pkg/phlaredb/tsdb/builder.go b/pkg/phlaredb/tsdb/builder.go deleted file mode 100644 index 0125c86967..0000000000 --- a/pkg/phlaredb/tsdb/builder.go +++ /dev/null @@ -1,21 +0,0 @@ -package tsdb - -import ( - phlaremodel "github.com/grafana/phlare/pkg/model" - "github.com/prometheus/common/model" -) - -type Series struct { - labels phlaremodel.Labels - fp model.Fingerprint -} - -type Builder struct { - series map[string]Series -} - -func NewBuilder() *Builder { - return &Builder{ - series: map[string]Series{}, - } -} From c8026ad9fc9b0371f1971752ffff96f39022b371 Mon Sep 17 00:00:00 2001 From: Cyril Tovena Date: Thu, 6 Jul 2023 17:05:57 +0200 Subject: [PATCH 05/35] Correct row count --- pkg/phlaredb/compact.go | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/pkg/phlaredb/compact.go b/pkg/phlaredb/compact.go index 6966595567..2a104502d1 100644 --- a/pkg/phlaredb/compact.go +++ b/pkg/phlaredb/compact.go @@ -57,8 +57,7 @@ func Compact(ctx context.Context, src []BlockReader, dst string) (block.Meta, er rowsIt = newSymbolsRewriter(rowsIt) reader := phlareparquet.NewIteratorRowReader(newRowsIterator(rowsIt)) - // todo size of rowgroups. - _, _, err = phlareparquet.CopyAsRowGroups(profileWriter, reader, 1024) + _, _, err = phlareparquet.CopyAsRowGroups(profileWriter, reader, defaultParquetConfig.MaxBufferRowCount) if err != nil { return block.Meta{}, err } From b6095ea888ed5f5eaddf90f8aa131f8c2ddb4f69 Mon Sep 17 00:00:00 2001 From: Cyril Tovena Date: Fri, 7 Jul 2023 01:06:36 +0200 Subject: [PATCH 06/35] WIP: Testing against dev. --- pkg/phlaredb/compact.go | 3 +++ pkg/phlaredb/compact_test.go | 44 ++++++++++++++++++++++++++++++++++++ 2 files changed, 47 insertions(+) create mode 100644 pkg/phlaredb/compact_test.go diff --git a/pkg/phlaredb/compact.go b/pkg/phlaredb/compact.go index 2a104502d1..6493f45b8c 100644 --- a/pkg/phlaredb/compact.go +++ b/pkg/phlaredb/compact.go @@ -33,6 +33,9 @@ type SymbolReader interface { } func Compact(ctx context.Context, src []BlockReader, dst string) (block.Meta, error) { + if len(src) <= 1 { + return block.Meta{}, errors.New("not enough blocks to compact") + } meta := block.NewMeta() blockPath := filepath.Join(dst, meta.ULID.String()) if err := os.MkdirAll(blockPath, 0o777); err != nil { diff --git a/pkg/phlaredb/compact_test.go b/pkg/phlaredb/compact_test.go new file mode 100644 index 0000000000..b88306049f --- /dev/null +++ b/pkg/phlaredb/compact_test.go @@ -0,0 +1,44 @@ +package phlaredb + +import ( + "context" + "sync" + "testing" + "time" + + "github.com/stretchr/testify/require" + + "github.com/grafana/phlare/pkg/objstore/client" + "github.com/grafana/phlare/pkg/objstore/providers/gcs" + "github.com/grafana/phlare/pkg/phlaredb/block" +) + +func TestCompact(t *testing.T) { + ctx := context.Background() + bkt, err := client.NewBucket(ctx, client.Config{ + StorageBackendConfig: client.StorageBackendConfig{ + Backend: client.GCS, + GCS: gcs.Config{ + BucketName: "dev-us-central-0-profiles-dev-001-data", + }, + }, + StoragePrefix: "1218/phlaredb/", + }, "test") + require.NoError(t, err) + now := time.Now() + var ( + src []BlockReader + mtx sync.Mutex + ) + + err = block.IterBlockMetas(ctx, bkt, now.Add(-6*time.Hour), now, func(m *block.Meta) { + mtx.Lock() + defer mtx.Unlock() + // todo: meta to blockreader + // src = append(src, NewSingleBlockQuerierFromMeta(ctx, bkt, m)) + }) + require.NoError(t, err) + new, err := Compact(ctx, src, "test/") + require.NoError(t, err) + t.Log(new) +} From bb924912b8a49a1460b15a94fdff55f214625fcc Mon Sep 17 00:00:00 2001 From: Cyril Tovena Date: Fri, 7 Jul 2023 18:06:26 +0200 Subject: [PATCH 07/35] WIP: Testing against dev. --- pkg/phlaredb/block_querier.go | 8 ++++++++ pkg/phlaredb/compact_test.go | 14 +++++++++----- 2 files changed, 17 insertions(+), 5 deletions(-) diff --git a/pkg/phlaredb/block_querier.go b/pkg/phlaredb/block_querier.go index 180257409e..29e26e0f94 100644 --- a/pkg/phlaredb/block_querier.go +++ b/pkg/phlaredb/block_querier.go @@ -427,6 +427,14 @@ func newStacktraceResolverV2(bucketReader phlareobj.Bucket) StacktraceDB { } } +func (b *singleBlockQuerier) Profiles() []parquet.RowGroup { + return b.profiles.file.RowGroups() +} + +func (b *singleBlockQuerier) Index() IndexReader { + return b.index +} + func (b *singleBlockQuerier) Close() error { b.openLock.Lock() defer func() { diff --git a/pkg/phlaredb/compact_test.go b/pkg/phlaredb/compact_test.go index b88306049f..ae8ed1609d 100644 --- a/pkg/phlaredb/compact_test.go +++ b/pkg/phlaredb/compact_test.go @@ -14,6 +14,7 @@ import ( ) func TestCompact(t *testing.T) { + t.TempDir() ctx := context.Background() bkt, err := client.NewBucket(ctx, client.Config{ StorageBackendConfig: client.StorageBackendConfig{ @@ -31,14 +32,17 @@ func TestCompact(t *testing.T) { mtx sync.Mutex ) - err = block.IterBlockMetas(ctx, bkt, now.Add(-6*time.Hour), now, func(m *block.Meta) { + err = block.IterBlockMetas(ctx, bkt, now.Add(-24*time.Hour), now, func(m *block.Meta) { mtx.Lock() defer mtx.Unlock() - // todo: meta to blockreader - // src = append(src, NewSingleBlockQuerierFromMeta(ctx, bkt, m)) + b := NewSingleBlockQuerierFromMeta(ctx, bkt, m) + err := b.Open(ctx) + require.NoError(t, err) + src = append(src, b) }) require.NoError(t, err) - new, err := Compact(ctx, src, "test/") + dst := t.TempDir() + new, err := Compact(ctx, src, dst) require.NoError(t, err) - t.Log(new) + t.Log(new, dst) } From 27345129c61d6e827e5b0e6819f9d0fad2b4339a Mon Sep 17 00:00:00 2001 From: Cyril Tovena Date: Tue, 11 Jul 2023 15:12:07 +0200 Subject: [PATCH 08/35] Fixes Profiles Iteration with Labels --- pkg/phlaredb/compact.go | 49 ++++++++++++++++------ pkg/phlaredb/compact_test.go | 80 ++++++++++++++++++++++++++++++++++++ 2 files changed, 116 insertions(+), 13 deletions(-) diff --git a/pkg/phlaredb/compact.go b/pkg/phlaredb/compact.go index 6493f45b8c..4dd37e775c 100644 --- a/pkg/phlaredb/compact.go +++ b/pkg/phlaredb/compact.go @@ -55,7 +55,10 @@ func Compact(ctx context.Context, src []BlockReader, dst string) (block.Meta, er // todo new symbdb - rowsIt := newMergeRowProfileIterator(src) + rowsIt, err := newMergeRowProfileIterator(src) + if err != nil { + return block.Meta{}, err + } rowsIt = newSeriesRewriter(rowsIt, indexw) rowsIt = newSymbolsRewriter(rowsIt) reader := phlareparquet.NewIteratorRowReader(newRowsIterator(rowsIt)) @@ -90,23 +93,30 @@ type profileRow struct { } type profileRowIterator struct { - profiles iter.Iterator[parquet.Row] - index IndexReader - err error + profiles iter.Iterator[parquet.Row] + index IndexReader + allPostings index.Postings + err error currentRow profileRow chunks []index.ChunkMeta } -func newProfileRowIterator(profiles iter.Iterator[parquet.Row], idx IndexReader) *profileRowIterator { +func newProfileRowIterator(reader parquet.RowReader, idx IndexReader) (*profileRowIterator, error) { + k, v := index.AllPostingsKey() + allPostings, err := idx.Postings(k, nil, v) + if err != nil { + return nil, err + } return &profileRowIterator{ - profiles: profiles, - index: idx, + profiles: phlareparquet.NewBufferedRowReaderIterator(reader, 1024), + index: idx, + allPostings: allPostings, currentRow: profileRow{ seriesRef: math.MaxUint32, }, chunks: make([]index.ChunkMeta, 1), - } + }, nil } func (p *profileRowIterator) At() profileRow { @@ -125,7 +135,16 @@ func (p *profileRowIterator) Next() bool { return true } p.currentRow.seriesRef = seriesIndex - fp, err := p.index.Series(storage.SeriesRef(p.currentRow.seriesRef), &p.currentRow.labels, &p.chunks) + if !p.allPostings.Next() { + if err := p.allPostings.Err(); err != nil { + p.err = err + return false + } + p.err = errors.New("unexpected end of postings") + return false + } + + fp, err := p.index.Series(p.allPostings.At(), &p.currentRow.labels, &p.chunks) if err != nil { p.err = err return false @@ -145,15 +164,19 @@ func (p *profileRowIterator) Close() error { return p.profiles.Close() } -func newMergeRowProfileIterator(src []BlockReader) iter.Iterator[profileRow] { +func newMergeRowProfileIterator(src []BlockReader) (iter.Iterator[profileRow], error) { its := make([]iter.Iterator[profileRow], len(src)) for i, s := range src { // todo: may be we could merge rowgroups in parallel but that requires locking. reader := parquet.MultiRowGroup(s.Profiles()...).Rows() - its[i] = newProfileRowIterator( - phlareparquet.NewBufferedRowReaderIterator(reader, 1024), + it, err := newProfileRowIterator( + reader, s.Index(), ) + if err != nil { + return nil, err + } + its[i] = it } return &dedupeProfileRowIterator{ Iterator: iter.NewTreeIterator(loser.New( @@ -178,7 +201,7 @@ func newMergeRowProfileIterator(src []BlockReader) iter.Iterator[profileRow] { }, func(it iter.Iterator[profileRow]) { _ = it.Close() }, )), - } + }, nil } type symbolsRewriter struct { diff --git a/pkg/phlaredb/compact_test.go b/pkg/phlaredb/compact_test.go index ae8ed1609d..8ed790ebe1 100644 --- a/pkg/phlaredb/compact_test.go +++ b/pkg/phlaredb/compact_test.go @@ -2,17 +2,33 @@ package phlaredb import ( "context" + "net/http" "sync" "testing" "time" + _ "net/http/pprof" + + "github.com/prometheus/common/model" + "github.com/prometheus/prometheus/storage" + "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" + typesv1 "github.com/grafana/phlare/api/gen/proto/go/types/v1" + phlaremodel "github.com/grafana/phlare/pkg/model" "github.com/grafana/phlare/pkg/objstore/client" "github.com/grafana/phlare/pkg/objstore/providers/gcs" "github.com/grafana/phlare/pkg/phlaredb/block" + schemav1 "github.com/grafana/phlare/pkg/phlaredb/schemas/v1" + "github.com/grafana/phlare/pkg/phlaredb/tsdb/index" ) +func init() { + go func() { + _ = http.ListenAndServe("localhost:6060", nil) + }() +} + func TestCompact(t *testing.T) { t.TempDir() ctx := context.Background() @@ -35,6 +51,10 @@ func TestCompact(t *testing.T) { err = block.IterBlockMetas(ctx, bkt, now.Add(-24*time.Hour), now, func(m *block.Meta) { mtx.Lock() defer mtx.Unlock() + // only test on the 3 latest blocks. + if len(src) >= 3 { + return + } b := NewSingleBlockQuerierFromMeta(ctx, bkt, m) err := b.Open(ctx) require.NoError(t, err) @@ -46,3 +66,63 @@ func TestCompact(t *testing.T) { require.NoError(t, err) t.Log(new, dst) } + +func TestProfileRowIterator(t *testing.T) { + filePath := t.TempDir() + "/index.tsdb" + idxw, err := index.NewWriter(context.Background(), filePath) + require.NoError(t, err) + require.NoError(t, idxw.AddSymbol("a")) + require.NoError(t, idxw.AddSymbol("b")) + require.NoError(t, idxw.AddSymbol("c")) + addSeries(t, idxw, 0, phlaremodel.Labels{ + &typesv1.LabelPair{Name: "a", Value: "b"}, + }) + addSeries(t, idxw, 1, phlaremodel.Labels{ + &typesv1.LabelPair{Name: "a", Value: "c"}, + }) + addSeries(t, idxw, 2, phlaremodel.Labels{ + &typesv1.LabelPair{Name: "b", Value: "a"}, + }) + require.NoError(t, idxw.Close()) + idxr, err := index.NewFileReader(filePath) + require.NoError(t, err) + + it, err := newProfileRowIterator(schemav1.NewInMemoryProfilesRowReader( + []schemav1.InMemoryProfile{ + {SeriesIndex: 0, TimeNanos: 1}, + {SeriesIndex: 1, TimeNanos: 2}, + {SeriesIndex: 2, TimeNanos: 3}, + }, + ), idxr) + require.NoError(t, err) + + assert.True(t, it.Next()) + require.Equal(t, it.At().labels, phlaremodel.Labels{ + &typesv1.LabelPair{Name: "a", Value: "b"}, + }) + require.Equal(t, it.At().timeNanos, int64(1)) + require.Equal(t, it.At().seriesRef, uint32(0)) + + assert.True(t, it.Next()) + require.Equal(t, it.At().labels, phlaremodel.Labels{ + &typesv1.LabelPair{Name: "a", Value: "c"}, + }) + require.Equal(t, it.At().timeNanos, int64(2)) + require.Equal(t, it.At().seriesRef, uint32(1)) + + assert.True(t, it.Next()) + require.Equal(t, it.At().labels, phlaremodel.Labels{ + &typesv1.LabelPair{Name: "b", Value: "a"}, + }) + require.Equal(t, it.At().timeNanos, int64(3)) + require.Equal(t, it.At().seriesRef, uint32(2)) + + assert.False(t, it.Next()) + require.NoError(t, it.Err()) + require.NoError(t, it.Close()) +} + +func addSeries(t *testing.T, idxw *index.Writer, idx int, labels phlaremodel.Labels) { + t.Helper() + require.NoError(t, idxw.AddSeries(storage.SeriesRef(idx), labels, model.Fingerprint(labels.Hash()), index.ChunkMeta{SeriesIndex: uint32(idx)})) +} From 4d434517b1019e51cee3073f5af7aa455336b7ba Mon Sep 17 00:00:00 2001 From: Cyril Tovena Date: Wed, 12 Jul 2023 10:35:48 +0200 Subject: [PATCH 09/35] Fixes a bug and test compact locally --- pkg/phlaredb/compact.go | 3 +- pkg/phlaredb/compact_test.go | 80 +++++++++++++++++++++++++++++++----- 2 files changed, 71 insertions(+), 12 deletions(-) diff --git a/pkg/phlaredb/compact.go b/pkg/phlaredb/compact.go index 4dd37e775c..d1c3f24e35 100644 --- a/pkg/phlaredb/compact.go +++ b/pkg/phlaredb/compact.go @@ -63,7 +63,7 @@ func Compact(ctx context.Context, src []BlockReader, dst string) (block.Meta, er rowsIt = newSymbolsRewriter(rowsIt) reader := phlareparquet.NewIteratorRowReader(newRowsIterator(rowsIt)) - _, _, err = phlareparquet.CopyAsRowGroups(profileWriter, reader, defaultParquetConfig.MaxBufferRowCount) + total, _, err := phlareparquet.CopyAsRowGroups(profileWriter, reader, defaultParquetConfig.MaxBufferRowCount) if err != nil { return block.Meta{}, err } @@ -77,6 +77,7 @@ func Compact(ctx context.Context, src []BlockReader, dst string) (block.Meta, er return block.Meta{}, err } // todo: block meta + meta.Stats.NumProfiles = total if _, err := meta.WriteToFile(util.Logger, blockPath); err != nil { return block.Meta{}, err } diff --git a/pkg/phlaredb/compact_test.go b/pkg/phlaredb/compact_test.go index 8ed790ebe1..e4d840d496 100644 --- a/pkg/phlaredb/compact_test.go +++ b/pkg/phlaredb/compact_test.go @@ -3,6 +3,7 @@ package phlaredb import ( "context" "net/http" + "sort" "sync" "testing" "time" @@ -13,10 +14,13 @@ import ( "github.com/prometheus/prometheus/storage" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" + "golang.org/x/sync/errgroup" typesv1 "github.com/grafana/phlare/api/gen/proto/go/types/v1" phlaremodel "github.com/grafana/phlare/pkg/model" + phlareobj "github.com/grafana/phlare/pkg/objstore" "github.com/grafana/phlare/pkg/objstore/client" + "github.com/grafana/phlare/pkg/objstore/providers/filesystem" "github.com/grafana/phlare/pkg/objstore/providers/gcs" "github.com/grafana/phlare/pkg/phlaredb/block" schemav1 "github.com/grafana/phlare/pkg/phlaredb/schemas/v1" @@ -44,27 +48,81 @@ func TestCompact(t *testing.T) { require.NoError(t, err) now := time.Now() var ( - src []BlockReader - mtx sync.Mutex + meta []*block.Meta + mtx sync.Mutex ) err = block.IterBlockMetas(ctx, bkt, now.Add(-24*time.Hour), now, func(m *block.Meta) { mtx.Lock() defer mtx.Unlock() - // only test on the 3 latest blocks. - if len(src) >= 3 { - return - } - b := NewSingleBlockQuerierFromMeta(ctx, bkt, m) - err := b.Open(ctx) - require.NoError(t, err) - src = append(src, b) + meta = append(meta, m) }) require.NoError(t, err) dst := t.TempDir() - new, err := Compact(ctx, src, dst) + + sort.Slice(meta, func(i, j int) bool { + return meta[i].MinTime.Before(meta[j].MinTime) + }) + + // only test on the 4 latest blocks. + meta = meta[len(meta)-4:] + testCompact(t, meta, bkt, dst) +} + +func TestCompactLocal(t *testing.T) { + t.TempDir() + ctx := context.Background() + bkt, err := client.NewBucket(ctx, client.Config{ + StorageBackendConfig: client.StorageBackendConfig{ + Backend: client.Filesystem, + Filesystem: filesystem.Config{ + Directory: "/Users/cyril/work/phlare-data/", + }, + }, + StoragePrefix: "", + }, "test") + require.NoError(t, err) + var metas []*block.Meta + + metaMap, err := block.ListBlocks("/Users/cyril/work/phlare-data/", time.Time{}) + require.NoError(t, err) + for _, m := range metaMap { + metas = append(metas, m) + } + dst := t.TempDir() + testCompact(t, metas, bkt, dst) +} + +func testCompact(t *testing.T, metas []*block.Meta, bkt phlareobj.Bucket, dst string) { + t.Helper() + g, ctx := errgroup.WithContext(context.Background()) + var src []BlockReader + now := time.Now() + for i, m := range metas { + t.Log("src block(#", i, ")", + "ID", m.ULID.String(), + "minTime", m.MinTime.Time().Format(time.RFC3339Nano), + "maxTime", m.MaxTime.Time().Format(time.RFC3339Nano), + "numSeries", m.Stats.NumSeries, + "numProfiles", m.Stats.NumProfiles, + "numSamples", m.Stats.NumSamples) + b := NewSingleBlockQuerierFromMeta(ctx, bkt, m) + g.Go(func() error { + return b.Open(ctx) + }) + + src = append(src, b) + } + + require.NoError(t, g.Wait()) + + new, err := Compact(context.Background(), src, dst) require.NoError(t, err) t.Log(new, dst) + t.Log("Compaction duration", time.Since(now)) + t.Log("numSeries", new.Stats.NumSeries, + "numProfiles", new.Stats.NumProfiles, + "numSamples", new.Stats.NumSamples) } func TestProfileRowIterator(t *testing.T) { From cd8433b874ddaeaf79df077e48be541495d2ab84 Mon Sep 17 00:00:00 2001 From: Cyril Tovena Date: Wed, 12 Jul 2023 11:11:39 +0200 Subject: [PATCH 10/35] add tests instructions --- pkg/phlaredb/compact_test.go | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/pkg/phlaredb/compact_test.go b/pkg/phlaredb/compact_test.go index e4d840d496..213ddac241 100644 --- a/pkg/phlaredb/compact_test.go +++ b/pkg/phlaredb/compact_test.go @@ -69,6 +69,13 @@ func TestCompact(t *testing.T) { testCompact(t, meta, bkt, dst) } +// to download the blocks: +// gsutil -m cp -r \ +// "gs://dev-us-central-0-profiles-dev-001-data/1218/phlaredb/01H53WJEAB43S3GJ26XMSNRSJA" \ +// "gs://dev-us-central-0-profiles-dev-001-data/1218/phlaredb/01H5454JBEV80V2J7CKYHPCBG8" \ +// "gs://dev-us-central-0-profiles-dev-001-data/1218/phlaredb/01H54553SYKH43FNJN5BVR1M2H" \ +// "gs://dev-us-central-0-profiles-dev-001-data/1218/phlaredb/01H5457Q89WYYH9FCK8PZ6XG75" \ +// . func TestCompactLocal(t *testing.T) { t.TempDir() ctx := context.Background() From ca97480f5e652ec4240faa0b22a879c4e7cc7577 Mon Sep 17 00:00:00 2001 From: Cyril Tovena Date: Wed, 12 Jul 2023 17:41:49 +0200 Subject: [PATCH 11/35] Add more meta informations and rewrite stacktraceIDs --- pkg/phlaredb/block_querier.go | 11 +- pkg/phlaredb/compact.go | 136 +++++++++++++++++++++-- pkg/phlaredb/schemas/v1/profiles.go | 31 +++++- pkg/phlaredb/schemas/v1/profiles_test.go | 60 ++++++++++ 4 files changed, 223 insertions(+), 15 deletions(-) diff --git a/pkg/phlaredb/block_querier.go b/pkg/phlaredb/block_querier.go index 29e26e0f94..7f675cc7fe 100644 --- a/pkg/phlaredb/block_querier.go +++ b/pkg/phlaredb/block_querier.go @@ -435,6 +435,13 @@ func (b *singleBlockQuerier) Index() IndexReader { return b.index } +func (b *singleBlockQuerier) Meta() block.Meta { + if b.meta == nil { + return block.Meta{} + } + return *b.meta +} + func (b *singleBlockQuerier) Close() error { b.openLock.Lock() defer func() { @@ -941,9 +948,7 @@ func (b *singleBlockQuerier) SelectMatchingProfiles(ctx context.Context, params } } - var ( - buf [][]parquet.Value - ) + var buf [][]parquet.Value pIt := query.NewBinaryJoinIterator( 0, diff --git a/pkg/phlaredb/compact.go b/pkg/phlaredb/compact.go index d1c3f24e35..1469faaa1d 100644 --- a/pkg/phlaredb/compact.go +++ b/pkg/phlaredb/compact.go @@ -6,6 +6,7 @@ import ( "os" "path/filepath" + "github.com/oklog/ulid" "github.com/pkg/errors" "github.com/prometheus/common/model" "github.com/prometheus/prometheus/storage" @@ -23,20 +24,17 @@ import ( ) type BlockReader interface { + Meta() block.Meta Profiles() []parquet.RowGroup Index() IndexReader - // Symbols() SymbolReader -} - -type SymbolReader interface { - // todo + // todo symbdb } func Compact(ctx context.Context, src []BlockReader, dst string) (block.Meta, error) { if len(src) <= 1 { return block.Meta{}, errors.New("not enough blocks to compact") } - meta := block.NewMeta() + meta := compactedMeta(src) blockPath := filepath.Join(dst, meta.ULID.String()) if err := os.MkdirAll(blockPath, 0o777); err != nil { return block.Meta{}, err @@ -59,9 +57,9 @@ func Compact(ctx context.Context, src []BlockReader, dst string) (block.Meta, er if err != nil { return block.Meta{}, err } - rowsIt = newSeriesRewriter(rowsIt, indexw) - rowsIt = newSymbolsRewriter(rowsIt) - reader := phlareparquet.NewIteratorRowReader(newRowsIterator(rowsIt)) + seriesRewriter := newSeriesRewriter(rowsIt, indexw) + symbolsRewriter := newSymbolsRewriter(seriesRewriter) + reader := phlareparquet.NewIteratorRowReader(newRowsIterator(symbolsRewriter)) total, _, err := phlareparquet.CopyAsRowGroups(profileWriter, reader, defaultParquetConfig.MaxBufferRowCount) if err != nil { @@ -78,12 +76,61 @@ func Compact(ctx context.Context, src []BlockReader, dst string) (block.Meta, er } // todo: block meta meta.Stats.NumProfiles = total + meta.Stats.NumSeries = seriesRewriter.NumSeries() + meta.Stats.NumSamples = symbolsRewriter.NumSamples() + if _, err := meta.WriteToFile(util.Logger, blockPath); err != nil { return block.Meta{}, err } return *meta, nil } +func compactedMeta(src []BlockReader) *block.Meta { + meta := block.NewMeta() + highestCompactionLevel := 0 + ulids := make([]ulid.ULID, len(src)) + parents := make([]tsdb.BlockDesc, len(src)) + minTime, maxTime := model.Latest, model.Earliest + labels := make(map[string]string) + for _, b := range src { + if b.Meta().Compaction.Level > highestCompactionLevel { + highestCompactionLevel = b.Meta().Compaction.Level + } + ulids = append(ulids, b.Meta().ULID) + parents = append(parents, tsdb.BlockDesc{ + ULID: b.Meta().ULID, + MinTime: int64(b.Meta().MinTime), + MaxTime: int64(b.Meta().MaxTime), + }) + if b.Meta().MinTime < minTime { + minTime = b.Meta().MinTime + } + if b.Meta().MaxTime > maxTime { + maxTime = b.Meta().MaxTime + } + for k, v := range b.Meta().Labels { + if k == block.HostnameLabel { + continue + } + labels[k] = v + } + } + if hostname, err := os.Hostname(); err == nil { + labels[block.HostnameLabel] = hostname + } + meta.Source = block.CompactorSource + meta.Compaction = tsdb.BlockMetaCompaction{ + Deletable: meta.Stats.NumSamples == 0, + Level: highestCompactionLevel + 1, + Sources: ulids, + Parents: parents, + } + meta.MaxTime = maxTime + meta.MinTime = minTime + meta.Labels = labels + return meta +} + type profileRow struct { timeNanos int64 @@ -205,14 +252,77 @@ func newMergeRowProfileIterator(src []BlockReader) (iter.Iterator[profileRow], e }, nil } +type noopStacktraceRewriter struct{} + +func (noopStacktraceRewriter) RewriteStacktraces(src, dst []uint32) error { + copy(dst, src) + return nil +} + +type StacktraceRewriter interface { + RewriteStacktraces(src, dst []uint32) error +} + type symbolsRewriter struct { iter.Iterator[profileRow] + err error + + rewriter StacktraceRewriter + src, dst []uint32 + numSamples uint64 } // todo remap symbols & ingest symbols func newSymbolsRewriter(it iter.Iterator[profileRow]) *symbolsRewriter { return &symbolsRewriter{ Iterator: it, + rewriter: noopStacktraceRewriter{}, + } +} + +func (s *symbolsRewriter) NumSamples() uint64 { + return s.numSamples +} + +func (s *symbolsRewriter) Next() bool { + if !s.Iterator.Next() { + return false + } + var err error + s.Iterator.At().row.ForStacktraceIDsValues(func(values []parquet.Value) { + s.numSamples += uint64(len(values)) + s.loadStacktracesID(values) + err = s.rewriter.RewriteStacktraces(s.src, s.dst) + if err != nil { + return + } + for i, v := range values { + values[i] = parquet.Int64Value(int64(s.dst[i])).Level(v.RepetitionLevel(), v.DefinitionLevel(), v.Column()) + } + }) + if err != nil { + s.err = err + return false + } + return true +} + +func (s *symbolsRewriter) Err() error { + if s.err != nil { + return s.err + } + return s.Iterator.Err() +} + +func (s *symbolsRewriter) loadStacktracesID(values []parquet.Value) { + if cap(s.src) < len(values) { + s.src = make([]uint32, len(values)*2) + s.dst = make([]uint32, len(values)*2) + } + s.src = s.src[:len(values)] + s.dst = s.dst[:len(values)] + for i := range values { + s.src[i] = values[i].Uint32() } } @@ -226,6 +336,8 @@ type seriesRewriter struct { previousFp model.Fingerprint currentChunkMeta index.ChunkMeta err error + + numSeries uint64 } func newSeriesRewriter(it iter.Iterator[profileRow], indexw *index.Writer) *seriesRewriter { @@ -235,6 +347,10 @@ func newSeriesRewriter(it iter.Iterator[profileRow], indexw *index.Writer) *seri } } +func (s *seriesRewriter) NumSeries() uint64 { + return s.numSeries +} + func (s *seriesRewriter) Next() bool { if !s.Iterator.Next() { if s.previousFp != 0 { @@ -242,6 +358,7 @@ func (s *seriesRewriter) Next() bool { s.err = err return false } + s.numSeries++ } return false } @@ -253,6 +370,7 @@ func (s *seriesRewriter) Next() bool { s.err = err return false } + s.numSeries++ } s.seriesRef++ s.labels = currentProfile.labels.Clone() diff --git a/pkg/phlaredb/schemas/v1/profiles.go b/pkg/phlaredb/schemas/v1/profiles.go index cd04d17fd7..d661ab6fb4 100644 --- a/pkg/phlaredb/schemas/v1/profiles.go +++ b/pkg/phlaredb/schemas/v1/profiles.go @@ -42,9 +42,10 @@ var ( phlareparquet.NewGroupField("DefaultSampleType", parquet.Optional(parquet.Int(64))), }) - maxProfileRow parquet.Row - seriesIndexColIndex int - timeNanoColIndex int + maxProfileRow parquet.Row + seriesIndexColIndex int + stacktraceIDColIndex int + timeNanoColIndex int ) func init() { @@ -62,6 +63,11 @@ func init() { panic(fmt.Errorf("TimeNanos column not found")) } timeNanoColIndex = timeCol.ColumnIndex + stacktraceIDCol, ok := profilesSchema.Lookup("Samples", "list", "element", "StacktraceID") + if !ok { + panic(fmt.Errorf("StacktraceID column not found")) + } + stacktraceIDColIndex = stacktraceIDCol.ColumnIndex } type Sample struct { @@ -479,3 +485,22 @@ func (p ProfileRow) TimeNanos() int64 { func (p ProfileRow) SetSeriesIndex(v uint32) { p[seriesIndexColIndex] = parquet.Int32Value(int32(v)).Level(0, 0, seriesIndexColIndex) } + +func (p ProfileRow) ForStacktraceIDsValues(fn func([]parquet.Value)) { + start := -1 + var i int + for i = 0; i < len(p); i++ { + col := p[i].Column() + if col == stacktraceIDColIndex && p[i].DefinitionLevel() == 1 { + if start == -1 { + start = i + } + } + if col > stacktraceIDColIndex { + break + } + } + if start != -1 { + fn(p[start:i]) + } +} diff --git a/pkg/phlaredb/schemas/v1/profiles_test.go b/pkg/phlaredb/schemas/v1/profiles_test.go index 6584844b44..c23a0ef145 100644 --- a/pkg/phlaredb/schemas/v1/profiles_test.go +++ b/pkg/phlaredb/schemas/v1/profiles_test.go @@ -207,6 +207,66 @@ func TestLessProfileRows(t *testing.T) { } } +func TestProfileRowStacktraceIDs(t *testing.T) { + for _, tc := range []struct { + name string + expected []uint32 + profile InMemoryProfile + }{ + {"empty", nil, InMemoryProfile{}}, + {"one sample", []uint32{1}, InMemoryProfile{ + SeriesIndex: 1, + StacktracePartition: 2, + TotalValue: 3, + Samples: Samples{StacktraceIDs: []uint32{1}, Values: []uint64{1}}, + }}, + {"many", []uint32{1, 1, 2, 3, 4}, InMemoryProfile{ + SeriesIndex: 1, + StacktracePartition: 2, + TotalValue: 3, + Samples: Samples{ + StacktraceIDs: []uint32{1, 1, 2, 3, 4}, + Values: []uint64{4, 2, 4, 5, 2}, + }, + }}, + } { + tc := tc + t.Run(tc.name, func(t *testing.T) { + rows := generateProfileRow([]InMemoryProfile{tc.profile}) + var ids []uint32 + ProfileRow(rows[0]).ForStacktraceIDsValues(func(values []parquet.Value) { + for _, v := range values { + ids = append(ids, v.Uint32()) + } + }) + require.Equal(t, tc.expected, ids) + }) + } +} + +func TestProfileRowMutateValues(t *testing.T) { + row := ProfileRow(generateProfileRow([]InMemoryProfile{ + { + Samples: Samples{ + StacktraceIDs: []uint32{1, 1, 2, 3, 4}, + Values: []uint64{4, 2, 4, 5, 2}, + }, + }, + })[0]) + row.ForStacktraceIDsValues(func(values []parquet.Value) { + for i := range values { + values[i] = parquet.Int32Value(1).Level(0, 1, values[i].Column()) + } + }) + var ids []uint32 + row.ForStacktraceIDsValues(func(values []parquet.Value) { + for _, v := range values { + ids = append(ids, v.Uint32()) + } + }) + require.Equal(t, []uint32{1, 1, 1, 1, 1}, ids) +} + func BenchmarkProfileRows(b *testing.B) { a := generateProfileRow([]InMemoryProfile{{SeriesIndex: 1, TimeNanos: 1}})[0] a1 := generateProfileRow([]InMemoryProfile{{SeriesIndex: 1, TimeNanos: 2}})[0] From 19bc955bd24f93696d1d620d634405e88af29d17 Mon Sep 17 00:00:00 2001 From: Cyril Tovena Date: Wed, 12 Jul 2023 17:43:44 +0200 Subject: [PATCH 12/35] nit todo --- pkg/phlaredb/compact.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pkg/phlaredb/compact.go b/pkg/phlaredb/compact.go index 1469faaa1d..7d1f573708 100644 --- a/pkg/phlaredb/compact.go +++ b/pkg/phlaredb/compact.go @@ -74,7 +74,7 @@ func Compact(ctx context.Context, src []BlockReader, dst string) (block.Meta, er if err := profileWriter.Close(); err != nil { return block.Meta{}, err } - // todo: block meta + // todo: block meta files. meta.Stats.NumProfiles = total meta.Stats.NumSeries = seriesRewriter.NumSeries() meta.Stats.NumSamples = symbolsRewriter.NumSamples() From 1a694022917881f738599503653fa638c2033746 Mon Sep 17 00:00:00 2001 From: Cyril Tovena Date: Thu, 13 Jul 2023 07:53:09 +0200 Subject: [PATCH 13/35] Refactoring code --- pkg/phlaredb/compact.go | 95 ++++++++++++++++++++++++++++++++--------- 1 file changed, 74 insertions(+), 21 deletions(-) diff --git a/pkg/phlaredb/compact.go b/pkg/phlaredb/compact.go index 7d1f573708..2249f50c03 100644 --- a/pkg/phlaredb/compact.go +++ b/pkg/phlaredb/compact.go @@ -2,11 +2,14 @@ package phlaredb import ( "context" + "io/fs" "math" "os" "path/filepath" + "strings" "github.com/oklog/ulid" + "github.com/opentracing/opentracing-go" "github.com/pkg/errors" "github.com/prometheus/common/model" "github.com/prometheus/prometheus/storage" @@ -30,21 +33,43 @@ type BlockReader interface { // todo symbdb } -func Compact(ctx context.Context, src []BlockReader, dst string) (block.Meta, error) { +func Compact(ctx context.Context, src []BlockReader, dst string) (meta block.Meta, err error) { + srcMetas := make([]block.Meta, len(src)) + ulids := make([]string, len(src)) + + for i, b := range src { + srcMetas[i] = b.Meta() + ulids[i] = b.Meta().ULID.String() + } + meta = compactMetas(srcMetas) + blockPath := filepath.Join(dst, meta.ULID.String()) + indexPath := filepath.Join(blockPath, block.IndexFilename) + profilePath := filepath.Join(blockPath, (&schemav1.ProfilePersister{}).Name()+block.ParquetSuffix) + + sp, ctx := opentracing.StartSpanFromContext(ctx, "Compact") + defer func() { + // todo: context propagation is not working through objstore + // This is because the BlockReader has no context. + sp.SetTag("src", ulids) + sp.SetTag("block_id", meta.ULID.String()) + if err != nil { + sp.SetTag("error", err) + } + sp.Finish() + }() + if len(src) <= 1 { return block.Meta{}, errors.New("not enough blocks to compact") } - meta := compactedMeta(src) - blockPath := filepath.Join(dst, meta.ULID.String()) if err := os.MkdirAll(blockPath, 0o777); err != nil { return block.Meta{}, err } - indexPath := filepath.Join(blockPath, block.IndexFilename) + indexw, err := prepareIndexWriter(ctx, indexPath, src) if err != nil { return block.Meta{}, err } - profilePath := filepath.Join(blockPath, (&schemav1.ProfilePersister{}).Name()+block.ParquetSuffix) + profileFile, err := os.OpenFile(profilePath, os.O_RDWR|os.O_CREATE|os.O_EXCL, 0o644) if err != nil { return block.Meta{}, err @@ -74,18 +99,46 @@ func Compact(ctx context.Context, src []BlockReader, dst string) (block.Meta, er if err := profileWriter.Close(); err != nil { return block.Meta{}, err } - // todo: block meta files. + + metaFiles, err := metaFilesFromDir(blockPath) + if err != nil { + return block.Meta{}, err + } + meta.Files = metaFiles meta.Stats.NumProfiles = total meta.Stats.NumSeries = seriesRewriter.NumSeries() meta.Stats.NumSamples = symbolsRewriter.NumSamples() - if _, err := meta.WriteToFile(util.Logger, blockPath); err != nil { return block.Meta{}, err } - return *meta, nil + return meta, nil +} + +// todo implement and tests +func metaFilesFromDir(dir string) ([]block.File, error) { + var files []block.File + err := filepath.Walk(dir, func(path string, info fs.FileInfo, err error) error { + if err != nil { + return err + } + if info.IsDir() { + return nil + } + switch filepath.Ext(info.Name()) { + case strings.TrimPrefix(block.ParquetSuffix, "."): + // todo parquet file + case filepath.Ext(block.IndexFilename): + // todo tsdb index file + default: + // todo other files + } + return nil + }) + return files, err } -func compactedMeta(src []BlockReader) *block.Meta { +// todo write tests +func compactMetas(src []block.Meta) block.Meta { meta := block.NewMeta() highestCompactionLevel := 0 ulids := make([]ulid.ULID, len(src)) @@ -93,22 +146,22 @@ func compactedMeta(src []BlockReader) *block.Meta { minTime, maxTime := model.Latest, model.Earliest labels := make(map[string]string) for _, b := range src { - if b.Meta().Compaction.Level > highestCompactionLevel { - highestCompactionLevel = b.Meta().Compaction.Level + if b.Compaction.Level > highestCompactionLevel { + highestCompactionLevel = b.Compaction.Level } - ulids = append(ulids, b.Meta().ULID) + ulids = append(ulids, b.ULID) parents = append(parents, tsdb.BlockDesc{ - ULID: b.Meta().ULID, - MinTime: int64(b.Meta().MinTime), - MaxTime: int64(b.Meta().MaxTime), + ULID: b.ULID, + MinTime: int64(b.MinTime), + MaxTime: int64(b.MaxTime), }) - if b.Meta().MinTime < minTime { - minTime = b.Meta().MinTime + if b.MinTime < minTime { + minTime = b.MinTime } - if b.Meta().MaxTime > maxTime { - maxTime = b.Meta().MaxTime + if b.MaxTime > maxTime { + maxTime = b.MaxTime } - for k, v := range b.Meta().Labels { + for k, v := range b.Labels { if k == block.HostnameLabel { continue } @@ -128,7 +181,7 @@ func compactedMeta(src []BlockReader) *block.Meta { meta.MaxTime = maxTime meta.MinTime = minTime meta.Labels = labels - return meta + return *meta } type profileRow struct { From c92672397e4e93dcfcf95697f34e6c8e5f25f0c1 Mon Sep 17 00:00:00 2001 From: Cyril Tovena Date: Mon, 17 Jul 2023 14:11:37 +0200 Subject: [PATCH 14/35] Adds meta files information to dst meta --- pkg/phlaredb/compact.go | 52 +++++++++++++--- pkg/phlaredb/compact_test.go | 103 +++++++++++++++++++++++++++++++ pkg/phlaredb/tsdb/index/index.go | 12 ++++ 3 files changed, 160 insertions(+), 7 deletions(-) diff --git a/pkg/phlaredb/compact.go b/pkg/phlaredb/compact.go index 2249f50c03..433b851fc1 100644 --- a/pkg/phlaredb/compact.go +++ b/pkg/phlaredb/compact.go @@ -6,7 +6,6 @@ import ( "math" "os" "path/filepath" - "strings" "github.com/oklog/ulid" "github.com/opentracing/opentracing-go" @@ -114,7 +113,7 @@ func Compact(ctx context.Context, src []BlockReader, dst string) (meta block.Met return meta, nil } -// todo implement and tests +// metaFilesFromDir returns a list of block files description from a directory. func metaFilesFromDir(dir string) ([]block.File, error) { var files []block.File err := filepath.Walk(dir, func(path string, info fs.FileInfo, err error) error { @@ -124,19 +123,58 @@ func metaFilesFromDir(dir string) ([]block.File, error) { if info.IsDir() { return nil } + var f block.File switch filepath.Ext(info.Name()) { - case strings.TrimPrefix(block.ParquetSuffix, "."): - // todo parquet file + case block.ParquetSuffix: + f, err = parquetMetaFile(path, info.Size()) + if err != nil { + return err + } case filepath.Ext(block.IndexFilename): - // todo tsdb index file - default: - // todo other files + f, err = tsdbMetaFile(path) + if err != nil { + return err + } + } + f.RelPath, err = filepath.Rel(dir, path) + if err != nil { + return err } + f.SizeBytes = uint64(info.Size()) + files = append(files, f) return nil }) return files, err } +func tsdbMetaFile(filePath string) (block.File, error) { + idxReader, err := index.NewFileReader(filePath) + if err != nil { + return block.File{}, err + } + + return idxReader.FileInfo(), nil +} + +func parquetMetaFile(filePath string, size int64) (block.File, error) { + f, err := os.Open(filePath) + if err != nil { + return block.File{}, err + } + defer f.Close() + + pqFile, err := parquet.OpenFile(f, size) + if err != nil { + return block.File{}, err + } + return block.File{ + Parquet: &block.ParquetFile{ + NumRowGroups: uint64(len(pqFile.RowGroups())), + NumRows: uint64(pqFile.NumRows()), + }, + }, nil +} + // todo write tests func compactMetas(src []block.Meta) block.Meta { meta := block.NewMeta() diff --git a/pkg/phlaredb/compact_test.go b/pkg/phlaredb/compact_test.go index 213ddac241..63a793cc47 100644 --- a/pkg/phlaredb/compact_test.go +++ b/pkg/phlaredb/compact_test.go @@ -2,7 +2,10 @@ package phlaredb import ( "context" + "fmt" "net/http" + "os" + "path/filepath" "sort" "sync" "testing" @@ -12,6 +15,7 @@ import ( "github.com/prometheus/common/model" "github.com/prometheus/prometheus/storage" + "github.com/segmentio/parquet-go" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" "golang.org/x/sync/errgroup" @@ -191,3 +195,102 @@ func addSeries(t *testing.T, idxw *index.Writer, idx int, labels phlaremodel.Lab t.Helper() require.NoError(t, idxw.AddSeries(storage.SeriesRef(idx), labels, model.Fingerprint(labels.Hash()), index.ChunkMeta{SeriesIndex: uint32(idx)})) } + +func TestMetaFilesFromDir(t *testing.T) { + dst := t.TempDir() + generateParquetFile(t, filepath.Join(dst, "foo.parquet")) + generateParquetFile(t, filepath.Join(dst, "symbols", "bar.parquet")) + generateFile(t, filepath.Join(dst, "symbols", "index.symdb"), 100) + generateFile(t, filepath.Join(dst, "symbols", "stacktraces.symdb"), 200) + generateIndexFile(t, dst) + actual, err := metaFilesFromDir(dst) + + require.NoError(t, err) + require.Equal(t, 5, len(actual)) + require.Equal(t, []block.File{ + { + Parquet: &block.ParquetFile{ + NumRows: 100, + NumRowGroups: 10, + }, + RelPath: "foo.parquet", + SizeBytes: fileSize(t, filepath.Join(dst, "foo.parquet")), + }, + { + RelPath: block.IndexFilename, + SizeBytes: fileSize(t, filepath.Join(dst, block.IndexFilename)), + TSDB: &block.TSDBFile{ + NumSeries: 3, + }, + }, + { + Parquet: &block.ParquetFile{ + NumRows: 100, + NumRowGroups: 10, + }, + RelPath: filepath.Join("symbols", "bar.parquet"), + SizeBytes: fileSize(t, filepath.Join(dst, "symbols", "bar.parquet")), + }, + { + RelPath: filepath.Join("symbols", "index.symdb"), + SizeBytes: fileSize(t, filepath.Join(dst, "symbols", "index.symdb")), + }, + { + RelPath: filepath.Join("symbols", "stacktraces.symdb"), + SizeBytes: fileSize(t, filepath.Join(dst, "symbols", "stacktraces.symdb")), + }, + }, actual) +} + +func fileSize(t *testing.T, path string) uint64 { + t.Helper() + fi, err := os.Stat(path) + require.NoError(t, err) + return uint64(fi.Size()) +} + +func generateFile(t *testing.T, path string, size int) { + t.Helper() + require.NoError(t, os.MkdirAll(filepath.Dir(path), 0o755)) + f, err := os.Create(path) + require.NoError(t, err) + defer f.Close() + require.NoError(t, f.Truncate(int64(size))) +} + +func generateIndexFile(t *testing.T, dir string) { + t.Helper() + filePath := filepath.Join(dir, block.IndexFilename) + idxw, err := index.NewWriter(context.Background(), filePath) + require.NoError(t, err) + require.NoError(t, idxw.AddSymbol("a")) + require.NoError(t, idxw.AddSymbol("b")) + require.NoError(t, idxw.AddSymbol("c")) + addSeries(t, idxw, 0, phlaremodel.Labels{ + &typesv1.LabelPair{Name: "a", Value: "b"}, + }) + addSeries(t, idxw, 1, phlaremodel.Labels{ + &typesv1.LabelPair{Name: "a", Value: "c"}, + }) + addSeries(t, idxw, 2, phlaremodel.Labels{ + &typesv1.LabelPair{Name: "b", Value: "a"}, + }) + require.NoError(t, idxw.Close()) +} + +func generateParquetFile(t *testing.T, path string) { + t.Helper() + require.NoError(t, os.MkdirAll(filepath.Dir(path), 0o755)) + file, err := os.OpenFile(path, os.O_RDWR|os.O_CREATE|os.O_EXCL, 0o644) + require.NoError(t, err) + defer file.Close() + + writer := parquet.NewGenericWriter[struct{ Name string }](file, parquet.MaxRowsPerRowGroup(10)) + defer writer.Close() + for i := 0; i < 100; i++ { + _, err := writer.Write([]struct{ Name string }{ + {Name: fmt.Sprintf("name-%d", i)}, + }) + require.NoError(t, err) + } +} diff --git a/pkg/phlaredb/tsdb/index/index.go b/pkg/phlaredb/tsdb/index/index.go index a1612b261e..5469252885 100644 --- a/pkg/phlaredb/tsdb/index/index.go +++ b/pkg/phlaredb/tsdb/index/index.go @@ -1334,9 +1334,21 @@ func (r *Reader) Version() int { // FileInfo returns some general stats about the underlying file func (r *Reader) FileInfo() block.File { + k, v := AllPostingsKey() + postings, err := r.Postings(k, nil, v) + if err != nil { + panic(err) + } + var numSeries uint64 + for postings.Next() { + numSeries++ + } return block.File{ RelPath: block.IndexFilename, SizeBytes: uint64(r.Size()), + TSDB: &block.TSDBFile{ + NumSeries: numSeries, + }, } } From 65639cd85b06199782a82608fb373802f1d6c7b6 Mon Sep 17 00:00:00 2001 From: Cyril Tovena Date: Wed, 19 Jul 2023 22:50:38 +0200 Subject: [PATCH 15/35] Updates import to grafana/pyroscsope --- pkg/phlaredb/compact.go | 16 ++++++++-------- pkg/phlaredb/compact_test.go | 18 +++++++++--------- 2 files changed, 17 insertions(+), 17 deletions(-) diff --git a/pkg/phlaredb/compact.go b/pkg/phlaredb/compact.go index 433b851fc1..84c31e6bf9 100644 --- a/pkg/phlaredb/compact.go +++ b/pkg/phlaredb/compact.go @@ -15,14 +15,14 @@ import ( "github.com/prometheus/prometheus/tsdb" "github.com/segmentio/parquet-go" - "github.com/grafana/phlare/pkg/iter" - phlaremodel "github.com/grafana/phlare/pkg/model" - phlareparquet "github.com/grafana/phlare/pkg/parquet" - "github.com/grafana/phlare/pkg/phlaredb/block" - schemav1 "github.com/grafana/phlare/pkg/phlaredb/schemas/v1" - "github.com/grafana/phlare/pkg/phlaredb/tsdb/index" - "github.com/grafana/phlare/pkg/util" - "github.com/grafana/phlare/pkg/util/loser" + "github.com/grafana/pyroscope/pkg/iter" + phlaremodel "github.com/grafana/pyroscope/pkg/model" + phlareparquet "github.com/grafana/pyroscope/pkg/parquet" + "github.com/grafana/pyroscope/pkg/phlaredb/block" + schemav1 "github.com/grafana/pyroscope/pkg/phlaredb/schemas/v1" + "github.com/grafana/pyroscope/pkg/phlaredb/tsdb/index" + "github.com/grafana/pyroscope/pkg/util" + "github.com/grafana/pyroscope/pkg/util/loser" ) type BlockReader interface { diff --git a/pkg/phlaredb/compact_test.go b/pkg/phlaredb/compact_test.go index 63a793cc47..acab55278e 100644 --- a/pkg/phlaredb/compact_test.go +++ b/pkg/phlaredb/compact_test.go @@ -20,15 +20,15 @@ import ( "github.com/stretchr/testify/require" "golang.org/x/sync/errgroup" - typesv1 "github.com/grafana/phlare/api/gen/proto/go/types/v1" - phlaremodel "github.com/grafana/phlare/pkg/model" - phlareobj "github.com/grafana/phlare/pkg/objstore" - "github.com/grafana/phlare/pkg/objstore/client" - "github.com/grafana/phlare/pkg/objstore/providers/filesystem" - "github.com/grafana/phlare/pkg/objstore/providers/gcs" - "github.com/grafana/phlare/pkg/phlaredb/block" - schemav1 "github.com/grafana/phlare/pkg/phlaredb/schemas/v1" - "github.com/grafana/phlare/pkg/phlaredb/tsdb/index" + typesv1 "github.com/grafana/pyroscope/api/gen/proto/go/types/v1" + phlaremodel "github.com/grafana/pyroscope/pkg/model" + phlareobj "github.com/grafana/pyroscope/pkg/objstore" + "github.com/grafana/pyroscope/pkg/objstore/client" + "github.com/grafana/pyroscope/pkg/objstore/providers/filesystem" + "github.com/grafana/pyroscope/pkg/objstore/providers/gcs" + "github.com/grafana/pyroscope/pkg/phlaredb/block" + schemav1 "github.com/grafana/pyroscope/pkg/phlaredb/schemas/v1" + "github.com/grafana/pyroscope/pkg/phlaredb/tsdb/index" ) func init() { From 6a337ac76ca7cbae5afb7f03bb746238b0895c67 Mon Sep 17 00:00:00 2001 From: Cyril Tovena Date: Thu, 20 Jul 2023 08:21:06 +0200 Subject: [PATCH 16/35] Update pkg/iter/tree.go Co-authored-by: Anton Kolesnikov --- pkg/iter/tree.go | 4 ---- 1 file changed, 4 deletions(-) diff --git a/pkg/iter/tree.go b/pkg/iter/tree.go index fb810d64eb..44c73ba430 100644 --- a/pkg/iter/tree.go +++ b/pkg/iter/tree.go @@ -1,11 +1,7 @@ package iter import ( -<<<<<<< HEAD "github.com/grafana/pyroscope/pkg/util/loser" -======= - "github.com/grafana/phlare/pkg/util/loser" ->>>>>>> ee8a92e04 (Add first draft of block compaction) ) var _ Iterator[interface{}] = &TreeIterator[interface{}]{} From a60fd9b37cea1a8af1b621785750da5a2bafa557 Mon Sep 17 00:00:00 2001 From: Anton Kolesnikov Date: Mon, 17 Jul 2023 16:23:49 +0800 Subject: [PATCH 17/35] Add stacktrace rewriter --- pkg/phlaredb/compact.go | 421 ++++++++++++++++++++++------ pkg/phlaredb/schemas/v1/profiles.go | 18 +- 2 files changed, 349 insertions(+), 90 deletions(-) diff --git a/pkg/phlaredb/compact.go b/pkg/phlaredb/compact.go index 84c31e6bf9..3c9b30780b 100644 --- a/pkg/phlaredb/compact.go +++ b/pkg/phlaredb/compact.go @@ -29,7 +29,37 @@ type BlockReader interface { Meta() block.Meta Profiles() []parquet.RowGroup Index() IndexReader - // todo symbdb + Symbols() SymbolsResolver +} + +// TODO(kolesnikovae): Refactor to symdb. + +// ProfileSymbols represents symbolic information associated with a profile. +type ProfileSymbols struct { + StacktracePartition uint64 + StacktraceIDs []uint32 + + Stacktraces []*schemav1.Stacktrace + Locations []*schemav1.InMemoryLocation + Mappings []*schemav1.InMemoryMapping + Functions []*schemav1.InMemoryFunction + Strings []string +} + +type SymbolsResolver interface { + Stacktraces(iter.Iterator[uint32]) iter.Iterator[*schemav1.Stacktrace] + Locations(iter.Iterator[uint32]) iter.Iterator[*schemav1.InMemoryLocation] + Mappings(iter.Iterator[uint32]) iter.Iterator[*schemav1.InMemoryMapping] + Functions(iter.Iterator[uint32]) iter.Iterator[*schemav1.InMemoryFunction] + Strings(iter.Iterator[uint32]) iter.Iterator[string] +} + +type SymbolsAppender interface { + AppendStacktrace(*schemav1.Stacktrace) uint32 + AppendLocation(*schemav1.InMemoryLocation) uint32 + AppendMapping(*schemav1.InMemoryMapping) uint32 + AppendFunction(*schemav1.InMemoryFunction) uint32 + AppendString(string) uint32 } func Compact(ctx context.Context, src []BlockReader, dst string) (meta block.Meta, err error) { @@ -74,16 +104,18 @@ func Compact(ctx context.Context, src []BlockReader, dst string) (meta block.Met return block.Meta{}, err } profileWriter := newProfileWriter(profileFile) - - // todo new symbdb + symw, err := newSymbolsWriter(dst) + if err != nil { + return block.Meta{}, err + } rowsIt, err := newMergeRowProfileIterator(src) if err != nil { return block.Meta{}, err } seriesRewriter := newSeriesRewriter(rowsIt, indexw) - symbolsRewriter := newSymbolsRewriter(seriesRewriter) - reader := phlareparquet.NewIteratorRowReader(newRowsIterator(symbolsRewriter)) + symRewriter := newSymbolsRewriter(seriesRewriter, src, symw) + reader := phlareparquet.NewIteratorRowReader(newRowsIterator(symRewriter)) total, _, err := phlareparquet.CopyAsRowGroups(profileWriter, reader, defaultParquetConfig.MaxBufferRowCount) if err != nil { @@ -229,10 +261,13 @@ type profileRow struct { labels phlaremodel.Labels fp model.Fingerprint row schemav1.ProfileRow + + blockReader BlockReader } type profileRowIterator struct { profiles iter.Iterator[parquet.Row] + blockReader BlockReader index IndexReader allPostings index.Postings err error @@ -241,15 +276,16 @@ type profileRowIterator struct { chunks []index.ChunkMeta } -func newProfileRowIterator(reader parquet.RowReader, idx IndexReader) (*profileRowIterator, error) { +func newProfileRowIterator(reader parquet.RowReader, s BlockReader) (*profileRowIterator, error) { k, v := index.AllPostingsKey() - allPostings, err := idx.Postings(k, nil, v) + allPostings, err := s.Index().Postings(k, nil, v) if err != nil { return nil, err } return &profileRowIterator{ profiles: phlareparquet.NewBufferedRowReaderIterator(reader, 1024), - index: idx, + blockReader: s, + index: s.Index(), allPostings: allPostings, currentRow: profileRow{ seriesRef: math.MaxUint32, @@ -266,6 +302,7 @@ func (p *profileRowIterator) Next() bool { if !p.profiles.Next() { return false } + p.currentRow.blockReader = p.blockReader p.currentRow.row = schemav1.ProfileRow(p.profiles.At()) seriesIndex := p.currentRow.row.SeriesIndex() p.currentRow.timeNanos = p.currentRow.row.TimeNanos() @@ -308,10 +345,7 @@ func newMergeRowProfileIterator(src []BlockReader) (iter.Iterator[profileRow], e for i, s := range src { // todo: may be we could merge rowgroups in parallel but that requires locking. reader := parquet.MultiRowGroup(s.Profiles()...).Rows() - it, err := newProfileRowIterator( - reader, - s.Index(), - ) + it, err := newProfileRowIterator(reader, s) if err != nil { return nil, err } @@ -343,80 +377,6 @@ func newMergeRowProfileIterator(src []BlockReader) (iter.Iterator[profileRow], e }, nil } -type noopStacktraceRewriter struct{} - -func (noopStacktraceRewriter) RewriteStacktraces(src, dst []uint32) error { - copy(dst, src) - return nil -} - -type StacktraceRewriter interface { - RewriteStacktraces(src, dst []uint32) error -} - -type symbolsRewriter struct { - iter.Iterator[profileRow] - err error - - rewriter StacktraceRewriter - src, dst []uint32 - numSamples uint64 -} - -// todo remap symbols & ingest symbols -func newSymbolsRewriter(it iter.Iterator[profileRow]) *symbolsRewriter { - return &symbolsRewriter{ - Iterator: it, - rewriter: noopStacktraceRewriter{}, - } -} - -func (s *symbolsRewriter) NumSamples() uint64 { - return s.numSamples -} - -func (s *symbolsRewriter) Next() bool { - if !s.Iterator.Next() { - return false - } - var err error - s.Iterator.At().row.ForStacktraceIDsValues(func(values []parquet.Value) { - s.numSamples += uint64(len(values)) - s.loadStacktracesID(values) - err = s.rewriter.RewriteStacktraces(s.src, s.dst) - if err != nil { - return - } - for i, v := range values { - values[i] = parquet.Int64Value(int64(s.dst[i])).Level(v.RepetitionLevel(), v.DefinitionLevel(), v.Column()) - } - }) - if err != nil { - s.err = err - return false - } - return true -} - -func (s *symbolsRewriter) Err() error { - if s.err != nil { - return s.err - } - return s.Iterator.Err() -} - -func (s *symbolsRewriter) loadStacktracesID(values []parquet.Value) { - if cap(s.src) < len(values) { - s.src = make([]uint32, len(values)*2) - s.dst = make([]uint32, len(values)*2) - } - s.src = s.src[:len(values)] - s.dst = s.dst[:len(values)] - for i := range values { - s.src[i] = values[i].Uint32() - } -} - type seriesRewriter struct { iter.Iterator[profileRow] @@ -534,3 +494,292 @@ func prepareIndexWriter(ctx context.Context, path string, readers []BlockReader) return indexw, nil } + +type symbolsRewriter struct { + profiles iter.Iterator[profileRow] + stacktraces, dst []uint32 + err error + + rewriters map[BlockReader]*stacktraceRewriter + + numSamples uint64 +} + +func newSymbolsRewriter(it iter.Iterator[profileRow], blocks []BlockReader, a SymbolsAppender) *symbolsRewriter { + sr := symbolsRewriter{ + profiles: it, + rewriters: make(map[BlockReader]*stacktraceRewriter, len(blocks)), + } + for _, b := range blocks { + sr.rewriters[b] = newStacktraceRewriter() + } + return &sr +} + +func (s *symbolsRewriter) NumSamples() uint64 { return s.numSamples } + +func (s *symbolsRewriter) At() profileRow { return s.profiles.At() } + +func (s *symbolsRewriter) Close() error { return s.profiles.Close() } + +func (s *symbolsRewriter) Err() error { + if s.err != nil { + return s.err + } + return s.profiles.Err() +} + +func (s *symbolsRewriter) Next() bool { + if !s.profiles.Next() { + return false + } + var err error + profile := s.profiles.At() + profile.row.ForStacktraceIDsValues(func(values []parquet.Value) { + s.loadStacktracesID(values) + r := s.rewriters[profile.blockReader] + if err = r.rewriteStacktraces(profile.row.StacktracePartitionID(), s.stacktraces); err != nil { + return + } + s.numSamples += uint64(len(values)) + for i, v := range values { + values[i] = parquet.Int64Value(int64(s.dst[i])).Level(v.RepetitionLevel(), v.DefinitionLevel(), v.Column()) + } + }) + if err != nil { + s.err = err + return false + } + return true +} + +func (s *symbolsRewriter) loadStacktracesID(values []parquet.Value) { + if cap(s.stacktraces) < len(values) { + s.stacktraces = make([]uint32, len(values)*2) + s.dst = make([]uint32, len(values)*2) + } + s.stacktraces = s.stacktraces[:len(values)] + s.dst = s.dst[:len(values)] + for i := range values { + s.stacktraces[i] = values[i].Uint32() + } +} + +type stacktraceRewriter struct { + partition uint64 + stacktraces map[uint64]*lookupTable[*schemav1.Stacktrace] + + locations *lookupTable[*schemav1.InMemoryLocation] + mappings *lookupTable[*schemav1.InMemoryMapping] + functions *lookupTable[*schemav1.InMemoryFunction] + strings *lookupTable[string] +} + +func newStacktraceRewriter() *stacktraceRewriter { + // TODO(kolesnikovae): + return new(stacktraceRewriter) +} + +const ( + marker = 1 << 31 + markedMask = math.MaxUint32 >> 1 +) + +type lookupTable[T any] struct { + // Index is source ID, and the value is the destination ID. + // If destination ID is not known, the element is index to 'unresolved' (marked). + resolved []uint32 + // Source IDs. + unresolved []uint32 + values []T +} + +func newLookupTable[T any](size int) *lookupTable[T] { + var t lookupTable[T] + t.init(size) + return &t +} + +func (t *lookupTable[T]) init(size int) { + if cap(t.resolved) < size { + t.resolved = make([]uint32, size) + return + } + t.resolved = t.resolved[:size] + for i := range t.resolved { + t.resolved[i] = 0 + } +} + +func (t *lookupTable[T]) reset() { t.unresolved = t.unresolved[:0] } + +func (t *lookupTable[T]) tryLookup(x uint32) uint32 { + if v := t.resolved[x]; v != 0 { + return v - 1 + } + v := uint32(len(t.unresolved)) | marker + t.unresolved = append(t.unresolved, x) + return v +} + +func (t *lookupTable[T]) storeResolved(i, v uint32) { t.resolved[i] = v + 1 } + +func (t *lookupTable[T]) lookupUnresolved(x uint32) uint32 { + if x&marker == 0 { + // Already resolved. + return x + } + return t.unresolved[x&markedMask] +} + +func (t *lookupTable[T]) iter() *lookupTableIterator[T] { + t.values = make([]T, len(t.resolved)) + return &lookupTableIterator[T]{ + values: t.values, + } +} + +// TODO(kolesnikovae): +type lookupTableIterator[T any] struct { + cur uint32 + values []T +} + +func (t *lookupTableIterator[T]) set(v T) { t.values[t.cur] = v } + +func (r *stacktraceRewriter) symbolsResolver() SymbolsResolver { + // TODO(kolesnikovae): + return nil +} + +func (r *stacktraceRewriter) symbolsAppender() SymbolsAppender { + // TODO(kolesnikovae): + return nil +} + +func (r *stacktraceRewriter) reset(partition uint64) { + r.partition = partition + r.stacktraces[partition].reset() + r.locations.reset() + r.mappings.reset() + r.functions.reset() + r.strings.reset() +} + +func (r *stacktraceRewriter) hasUnresolved() bool { + return len(r.stacktraces[r.partition].unresolved)+ + len(r.locations.unresolved)+ + len(r.mappings.unresolved)+ + len(r.functions.unresolved)+ + len(r.strings.unresolved) > 0 +} + +func (r *stacktraceRewriter) rewriteStacktraces(partition uint64, stacktraces []uint32) error { + r.reset(partition) + r.populateUnresolved(stacktraces) + if r.hasUnresolved() { + r.append(stacktraces) + } + return nil +} + +func (r *stacktraceRewriter) populateUnresolved(stacktraces []uint32) { + // Filter out all stack traces that have been already resolved. + src := r.stacktraces[r.partition] + for i, v := range stacktraces { + stacktraces[i] = src.tryLookup(v) + } + if len(src.unresolved) == 0 { + return + } + + // Resolve locations for new stack traces. + var stacktrace *schemav1.Stacktrace + unresolvedStacktraces := src.iter() + p := r.symbolsResolver() + for i := p.Stacktraces(unresolvedStacktraces); i.Next(); stacktrace = i.At() { + for i, loc := range stacktrace.LocationIDs { + stacktrace.LocationIDs[i] = uint64(r.locations.tryLookup(uint32(loc))) + } + unresolvedStacktraces.set(stacktrace) + } + + // Resolve functions and mappings for new locations. + var location *schemav1.InMemoryLocation + unresolvedLocs := r.locations.iter() + for i := p.Locations(unresolvedLocs); i.Next(); location = i.At() { + location.MappingId = r.mappings.tryLookup(location.MappingId) + for j, line := range location.Line { + location.Line[j].FunctionId = r.functions.tryLookup(line.FunctionId) + } + unresolvedLocs.set(location) + } + + // Resolve strings. + var mapping *schemav1.InMemoryMapping + unresolvedMappings := r.mappings.iter() + for i := p.Mappings(unresolvedMappings); i.Next(); mapping = i.At() { + mapping.BuildId = r.strings.tryLookup(mapping.BuildId) + mapping.Filename = r.strings.tryLookup(mapping.Filename) + unresolvedMappings.set(mapping) + } + var function *schemav1.InMemoryFunction + unresolvedFunctions := r.functions.iter() + for i := p.Functions(unresolvedFunctions); i.Next(); function = i.At() { + function.Name = r.strings.tryLookup(function.Name) + function.Filename = r.strings.tryLookup(function.Filename) + function.SystemName = r.strings.tryLookup(function.SystemName) + unresolvedFunctions.set(function) + } + var str string + unresolvedStrings := r.strings.iter() + for i := p.Strings(unresolvedStrings); i.Next(); str = i.At() { + unresolvedStrings.set(str) + } +} + +func (r *stacktraceRewriter) append(stacktraces []uint32) { + a := r.symbolsAppender() + for _, str := range r.strings.values { + r.functions.storeResolved(0, a.AppendString(str)) + } + + for _, function := range r.functions.values { + function.Name = r.strings.lookupUnresolved(function.Name) + function.Filename = r.strings.lookupUnresolved(function.Filename) + function.SystemName = r.strings.lookupUnresolved(function.SystemName) + r.functions.storeResolved(0, a.AppendFunction(function)) + } + + for _, mapping := range r.mappings.values { + mapping.BuildId = r.strings.lookupUnresolved(mapping.BuildId) + mapping.Filename = r.strings.lookupUnresolved(mapping.Filename) + r.mappings.storeResolved(0, a.AppendMapping(mapping)) + } + + for _, location := range r.locations.values { + location.MappingId = r.mappings.lookupUnresolved(location.MappingId) + for j, line := range location.Line { + location.Line[j].FunctionId = r.functions.lookupUnresolved(line.FunctionId) + } + r.locations.storeResolved(0, a.AppendLocation(location)) + } + + src := r.stacktraces[r.partition] + for _, stacktrace := range src.values { + for j, v := range stacktrace.LocationIDs { + stacktrace.LocationIDs[j] = uint64(r.locations.lookupUnresolved(uint32(v))) + } + src.storeResolved(0, a.AppendStacktrace(stacktrace)) + } + for i, v := range stacktraces { + stacktraces[i] = src.lookupUnresolved(v) + } +} + +type symbolsWriter struct { + // TODO(kolesnikovae): + SymbolsAppender +} + +func newSymbolsWriter(dst string) (*symbolsWriter, error) { return &symbolsWriter{}, nil } diff --git a/pkg/phlaredb/schemas/v1/profiles.go b/pkg/phlaredb/schemas/v1/profiles.go index d661ab6fb4..bfcc0ec298 100644 --- a/pkg/phlaredb/schemas/v1/profiles.go +++ b/pkg/phlaredb/schemas/v1/profiles.go @@ -42,10 +42,11 @@ var ( phlareparquet.NewGroupField("DefaultSampleType", parquet.Optional(parquet.Int(64))), }) - maxProfileRow parquet.Row - seriesIndexColIndex int - stacktraceIDColIndex int - timeNanoColIndex int + maxProfileRow parquet.Row + seriesIndexColIndex int + stacktraceIDColIndex int + timeNanoColIndex int + stacktracePartitionColIndex int ) func init() { @@ -68,6 +69,11 @@ func init() { panic(fmt.Errorf("StacktraceID column not found")) } stacktraceIDColIndex = stacktraceIDCol.ColumnIndex + stacktracePartitionCol, ok := profilesSchema.Lookup("StacktracePartition") + if !ok { + panic(fmt.Errorf("StacktracePartition column not found")) + } + stacktracePartitionColIndex = stacktracePartitionCol.ColumnIndex } type Sample struct { @@ -471,6 +477,10 @@ func (p ProfileRow) SeriesIndex() uint32 { return p[seriesIndexColIndex].Uint32() } +func (p ProfileRow) StacktracePartitionID() uint64 { + return p[stacktracePartitionColIndex].Uint64() +} + func (p ProfileRow) TimeNanos() int64 { var ts int64 for i := len(p) - 1; i >= 0; i-- { From 71f241637e42ca27f5e02d625631343e836433eb Mon Sep 17 00:00:00 2001 From: Anton Kolesnikov Date: Mon, 17 Jul 2023 20:28:55 +0800 Subject: [PATCH 18/35] Fixes --- pkg/phlaredb/compact.go | 408 ++++++++++++++++++++++++++-------------- 1 file changed, 262 insertions(+), 146 deletions(-) diff --git a/pkg/phlaredb/compact.go b/pkg/phlaredb/compact.go index 3c9b30780b..c89208d1e0 100644 --- a/pkg/phlaredb/compact.go +++ b/pkg/phlaredb/compact.go @@ -6,6 +6,7 @@ import ( "math" "os" "path/filepath" + "sort" "github.com/oklog/ulid" "github.com/opentracing/opentracing-go" @@ -29,21 +30,13 @@ type BlockReader interface { Meta() block.Meta Profiles() []parquet.RowGroup Index() IndexReader - Symbols() SymbolsResolver + SymbolsReader } // TODO(kolesnikovae): Refactor to symdb. -// ProfileSymbols represents symbolic information associated with a profile. -type ProfileSymbols struct { - StacktracePartition uint64 - StacktraceIDs []uint32 - - Stacktraces []*schemav1.Stacktrace - Locations []*schemav1.InMemoryLocation - Mappings []*schemav1.InMemoryMapping - Functions []*schemav1.InMemoryFunction - Strings []string +type SymbolsReader interface { + SymbolsResolver(partition uint64) (SymbolsResolver, error) } type SymbolsResolver interface { @@ -52,6 +45,19 @@ type SymbolsResolver interface { Mappings(iter.Iterator[uint32]) iter.Iterator[*schemav1.InMemoryMapping] Functions(iter.Iterator[uint32]) iter.Iterator[*schemav1.InMemoryFunction] Strings(iter.Iterator[uint32]) iter.Iterator[string] + WriteStats(*SymbolStats) +} + +type SymbolStats struct { + StacktracesTotal int + LocationsTotal int + MappingsTotal int + FunctionsTotal int + StringsTotal int +} + +type SymbolsWriter interface { + SymbolsAppender(partition uint64) (SymbolsAppender, error) } type SymbolsAppender interface { @@ -60,6 +66,7 @@ type SymbolsAppender interface { AppendMapping(*schemav1.InMemoryMapping) uint32 AppendFunction(*schemav1.InMemoryFunction) uint32 AppendString(string) uint32 + Flush() error } func Compact(ctx context.Context, src []BlockReader, dst string) (meta block.Meta, err error) { @@ -496,22 +503,21 @@ func prepareIndexWriter(ctx context.Context, path string, readers []BlockReader) } type symbolsRewriter struct { - profiles iter.Iterator[profileRow] - stacktraces, dst []uint32 - err error - - rewriters map[BlockReader]*stacktraceRewriter + profiles iter.Iterator[profileRow] + rewriters map[BlockReader]*stacktraceRewriter + stacktraces []uint32 + err error numSamples uint64 } -func newSymbolsRewriter(it iter.Iterator[profileRow], blocks []BlockReader, a SymbolsAppender) *symbolsRewriter { +func newSymbolsRewriter(it iter.Iterator[profileRow], blocks []BlockReader, w SymbolsWriter) *symbolsRewriter { sr := symbolsRewriter{ profiles: it, rewriters: make(map[BlockReader]*stacktraceRewriter, len(blocks)), } - for _, b := range blocks { - sr.rewriters[b] = newStacktraceRewriter() + for _, r := range blocks { + sr.rewriters[r] = newStacktraceRewriter(r, w) } return &sr } @@ -543,7 +549,7 @@ func (s *symbolsRewriter) Next() bool { } s.numSamples += uint64(len(values)) for i, v := range values { - values[i] = parquet.Int64Value(int64(s.dst[i])).Level(v.RepetitionLevel(), v.DefinitionLevel(), v.Column()) + values[i] = parquet.Int64Value(int64(s.stacktraces[i])).Level(v.RepetitionLevel(), v.DefinitionLevel(), v.Column()) } }) if err != nil { @@ -556,114 +562,71 @@ func (s *symbolsRewriter) Next() bool { func (s *symbolsRewriter) loadStacktracesID(values []parquet.Value) { if cap(s.stacktraces) < len(values) { s.stacktraces = make([]uint32, len(values)*2) - s.dst = make([]uint32, len(values)*2) } s.stacktraces = s.stacktraces[:len(values)] - s.dst = s.dst[:len(values)] for i := range values { s.stacktraces[i] = values[i].Uint32() } } type stacktraceRewriter struct { - partition uint64 - stacktraces map[uint64]*lookupTable[*schemav1.Stacktrace] + reader SymbolsReader + writer SymbolsWriter + // Stack trace identifiers are only valid within the partition. + stacktraces map[uint64]*lookupTable[*schemav1.Stacktrace] + // Objects below have global addressing. locations *lookupTable[*schemav1.InMemoryLocation] mappings *lookupTable[*schemav1.InMemoryMapping] functions *lookupTable[*schemav1.InMemoryFunction] strings *lookupTable[string] -} -func newStacktraceRewriter() *stacktraceRewriter { - // TODO(kolesnikovae): - return new(stacktraceRewriter) + partition uint64 + resolver SymbolsResolver + appender SymbolsAppender + stats SymbolStats } -const ( - marker = 1 << 31 - markedMask = math.MaxUint32 >> 1 -) - -type lookupTable[T any] struct { - // Index is source ID, and the value is the destination ID. - // If destination ID is not known, the element is index to 'unresolved' (marked). - resolved []uint32 - // Source IDs. - unresolved []uint32 - values []T -} - -func newLookupTable[T any](size int) *lookupTable[T] { - var t lookupTable[T] - t.init(size) - return &t +func newStacktraceRewriter(r SymbolsReader, w SymbolsWriter) *stacktraceRewriter { + return &stacktraceRewriter{ + reader: r, + writer: w, + } } -func (t *lookupTable[T]) init(size int) { - if cap(t.resolved) < size { - t.resolved = make([]uint32, size) - return +func (r *stacktraceRewriter) init(partition uint64) (err error) { + r.partition = partition + if r.appender, err = r.writer.SymbolsAppender(partition); err != nil { + return err } - t.resolved = t.resolved[:size] - for i := range t.resolved { - t.resolved[i] = 0 + if r.resolver, err = r.reader.SymbolsResolver(partition); err != nil { + return err } -} + r.resolver.WriteStats(&r.stats) -func (t *lookupTable[T]) reset() { t.unresolved = t.unresolved[:0] } - -func (t *lookupTable[T]) tryLookup(x uint32) uint32 { - if v := t.resolved[x]; v != 0 { - return v - 1 + // Only stacktraces are yet partitioned. + if r.stacktraces == nil { + r.stacktraces = make(map[uint64]*lookupTable[*schemav1.Stacktrace]) } - v := uint32(len(t.unresolved)) | marker - t.unresolved = append(t.unresolved, x) - return v -} - -func (t *lookupTable[T]) storeResolved(i, v uint32) { t.resolved[i] = v + 1 } - -func (t *lookupTable[T]) lookupUnresolved(x uint32) uint32 { - if x&marker == 0 { - // Already resolved. - return x + p, ok := r.stacktraces[partition] + if !ok { + p = newLookupTable[*schemav1.Stacktrace](r.stats.StacktracesTotal) + r.stacktraces[partition] = p } - return t.unresolved[x&markedMask] -} + p.reset() -func (t *lookupTable[T]) iter() *lookupTableIterator[T] { - t.values = make([]T, len(t.resolved)) - return &lookupTableIterator[T]{ - values: t.values, + if r.locations == nil { + r.locations = newLookupTable[*schemav1.InMemoryLocation](r.stats.LocationsTotal) + r.mappings = newLookupTable[*schemav1.InMemoryMapping](r.stats.MappingsTotal) + r.functions = newLookupTable[*schemav1.InMemoryFunction](r.stats.FunctionsTotal) + r.strings = newLookupTable[string](r.stats.StringsTotal) + return nil } -} - -// TODO(kolesnikovae): -type lookupTableIterator[T any] struct { - cur uint32 - values []T -} - -func (t *lookupTableIterator[T]) set(v T) { t.values[t.cur] = v } - -func (r *stacktraceRewriter) symbolsResolver() SymbolsResolver { - // TODO(kolesnikovae): - return nil -} - -func (r *stacktraceRewriter) symbolsAppender() SymbolsAppender { - // TODO(kolesnikovae): - return nil -} - -func (r *stacktraceRewriter) reset(partition uint64) { - r.partition = partition - r.stacktraces[partition].reset() r.locations.reset() r.mappings.reset() r.functions.reset() r.strings.reset() + return nil } func (r *stacktraceRewriter) hasUnresolved() bool { @@ -674,112 +637,265 @@ func (r *stacktraceRewriter) hasUnresolved() bool { len(r.strings.unresolved) > 0 } -func (r *stacktraceRewriter) rewriteStacktraces(partition uint64, stacktraces []uint32) error { - r.reset(partition) - r.populateUnresolved(stacktraces) +func (r *stacktraceRewriter) rewriteStacktraces(partition uint64, stacktraces []uint32) (err error) { + if err = r.init(partition); err != nil { + return err + } + if err = r.populateUnresolved(stacktraces); err != nil { + return err + } if r.hasUnresolved() { - r.append(stacktraces) + if err = r.appendRewrite(stacktraces); err != nil { + return err + } } return nil } -func (r *stacktraceRewriter) populateUnresolved(stacktraces []uint32) { +func (r *stacktraceRewriter) populateUnresolved(stacktraceIDs []uint32) error { // Filter out all stack traces that have been already resolved. src := r.stacktraces[r.partition] - for i, v := range stacktraces { - stacktraces[i] = src.tryLookup(v) + for i, v := range stacktraceIDs { + stacktraceIDs[i] = src.tryLookup(v) } if len(src.unresolved) == 0 { - return + return nil } // Resolve locations for new stack traces. var stacktrace *schemav1.Stacktrace unresolvedStacktraces := src.iter() - p := r.symbolsResolver() - for i := p.Stacktraces(unresolvedStacktraces); i.Next(); stacktrace = i.At() { - for i, loc := range stacktrace.LocationIDs { - stacktrace.LocationIDs[i] = uint64(r.locations.tryLookup(uint32(loc))) + stacktraces := r.resolver.Stacktraces(unresolvedStacktraces) + for ; stacktraces.Err() == nil && stacktraces.Next(); stacktrace = stacktraces.At() { + for j, loc := range stacktrace.LocationIDs { + stacktrace.LocationIDs[j] = uint64(r.locations.tryLookup(uint32(loc))) } - unresolvedStacktraces.set(stacktrace) + // TODO(kolesnikovae): Copy. + unresolvedStacktraces.setValue(stacktrace) + } + if err := stacktraces.Err(); err != nil { + return err } // Resolve functions and mappings for new locations. var location *schemav1.InMemoryLocation unresolvedLocs := r.locations.iter() - for i := p.Locations(unresolvedLocs); i.Next(); location = i.At() { + locations := r.resolver.Locations(unresolvedLocs) + for ; locations.Err() == nil && locations.Next(); location = locations.At() { location.MappingId = r.mappings.tryLookup(location.MappingId) for j, line := range location.Line { location.Line[j].FunctionId = r.functions.tryLookup(line.FunctionId) } - unresolvedLocs.set(location) + unresolvedLocs.setValue(location) + } + if err := locations.Err(); err != nil { + return err } // Resolve strings. var mapping *schemav1.InMemoryMapping unresolvedMappings := r.mappings.iter() - for i := p.Mappings(unresolvedMappings); i.Next(); mapping = i.At() { + mappings := r.resolver.Mappings(unresolvedMappings) + for ; mappings.Err() == nil && mappings.Next(); mapping = mappings.At() { mapping.BuildId = r.strings.tryLookup(mapping.BuildId) mapping.Filename = r.strings.tryLookup(mapping.Filename) - unresolvedMappings.set(mapping) + unresolvedMappings.setValue(mapping) + } + if err := mappings.Err(); err != nil { + return err } + var function *schemav1.InMemoryFunction unresolvedFunctions := r.functions.iter() - for i := p.Functions(unresolvedFunctions); i.Next(); function = i.At() { + functions := r.resolver.Functions(unresolvedFunctions) + for ; functions.Err() == nil && functions.Next(); function = functions.At() { function.Name = r.strings.tryLookup(function.Name) function.Filename = r.strings.tryLookup(function.Filename) function.SystemName = r.strings.tryLookup(function.SystemName) - unresolvedFunctions.set(function) + unresolvedFunctions.setValue(function) + } + if err := functions.Err(); err != nil { + return err } + var str string unresolvedStrings := r.strings.iter() - for i := p.Strings(unresolvedStrings); i.Next(); str = i.At() { - unresolvedStrings.set(str) + strings := r.resolver.Strings(unresolvedStrings) + for ; strings.Err() == nil && strings.Next(); str = strings.At() { + unresolvedStrings.setValue(str) } + return strings.Err() } -func (r *stacktraceRewriter) append(stacktraces []uint32) { - a := r.symbolsAppender() - for _, str := range r.strings.values { - r.functions.storeResolved(0, a.AppendString(str)) +func (r *stacktraceRewriter) appendRewrite(stacktraces []uint32) error { + for _, v := range r.strings.unresolved { + r.strings.storeResolved(v.uid, r.appender.AppendString(v.val)) } - for _, function := range r.functions.values { - function.Name = r.strings.lookupUnresolved(function.Name) - function.Filename = r.strings.lookupUnresolved(function.Filename) - function.SystemName = r.strings.lookupUnresolved(function.SystemName) - r.functions.storeResolved(0, a.AppendFunction(function)) + for _, v := range r.functions.unresolved { + function := v.val + function.Name = r.strings.lookupResolved(function.Name) + function.Filename = r.strings.lookupResolved(function.Filename) + function.SystemName = r.strings.lookupResolved(function.SystemName) + r.functions.storeResolved(v.uid, r.appender.AppendFunction(function)) } - for _, mapping := range r.mappings.values { - mapping.BuildId = r.strings.lookupUnresolved(mapping.BuildId) - mapping.Filename = r.strings.lookupUnresolved(mapping.Filename) - r.mappings.storeResolved(0, a.AppendMapping(mapping)) + for _, v := range r.mappings.unresolved { + mapping := v.val + mapping.BuildId = r.strings.lookupResolved(mapping.BuildId) + mapping.Filename = r.strings.lookupResolved(mapping.Filename) + r.mappings.storeResolved(v.uid, r.appender.AppendMapping(mapping)) } - for _, location := range r.locations.values { - location.MappingId = r.mappings.lookupUnresolved(location.MappingId) + for _, v := range r.locations.unresolved { + location := v.val + location.MappingId = r.mappings.lookupResolved(location.MappingId) for j, line := range location.Line { - location.Line[j].FunctionId = r.functions.lookupUnresolved(line.FunctionId) + location.Line[j].FunctionId = r.functions.lookupResolved(line.FunctionId) } - r.locations.storeResolved(0, a.AppendLocation(location)) + r.locations.storeResolved(v.uid, r.appender.AppendLocation(location)) } src := r.stacktraces[r.partition] - for _, stacktrace := range src.values { - for j, v := range stacktrace.LocationIDs { - stacktrace.LocationIDs[j] = uint64(r.locations.lookupUnresolved(uint32(v))) + for _, v := range src.unresolved { + stacktrace := v.val + for j, lid := range stacktrace.LocationIDs { + stacktrace.LocationIDs[j] = uint64(r.locations.lookupResolved(uint32(lid))) } - src.storeResolved(0, a.AppendStacktrace(stacktrace)) + src.storeResolved(v.uid, r.appender.AppendStacktrace(stacktrace)) } for i, v := range stacktraces { - stacktraces[i] = src.lookupUnresolved(v) + stacktraces[i] = src.lookupResolved(v) } + + return r.appender.Flush() +} + +const ( + marker = 1 << 31 + markedMask = math.MaxUint32 >> 1 +) + +type lookupTable[T any] struct { + // Index is source ID, and the value is the destination ID. + // If destination ID is not known, the element is index to 'unresolved' (marked). + resolved []uint32 + unresolved []lookupTableValue[T] + refs []lookupTableRef } -type symbolsWriter struct { - // TODO(kolesnikovae): - SymbolsAppender +type lookupTableValue[T any] struct { + rid uint32 // Index to resolved. + uid uint32 // Original index (unresolved). + val T } -func newSymbolsWriter(dst string) (*symbolsWriter, error) { return &symbolsWriter{}, nil } +type lookupTableRef struct{ rid, uid uint32 } + +func newLookupTable[T any](size int) *lookupTable[T] { + var t lookupTable[T] + t.init(size) + return &t +} + +func (t *lookupTable[T]) init(size int) { + if cap(t.resolved) < size { + t.resolved = make([]uint32, size) + return + } + t.resolved = t.resolved[:size] + for i := range t.resolved { + t.resolved[i] = 0 + } +} + +func (t *lookupTable[T]) reset() { + t.unresolved = t.unresolved[:0] + t.refs = t.refs[:0] +} + +// tryLookup looks up the value at x in resolved. +// If x is has not been resolved yet, the x is memorized +// for future resolve, and returned values is the marked +// index to unresolved. +func (t *lookupTable[T]) tryLookup(x uint32) uint32 { + if v := t.resolved[x]; v != 0 { + if v&marker > 0 { + return v // Already marked for resolve. + } + return v - 1 // Already resolved. + } + u := uint32(len(t.unresolved)) + t.unresolved = append(t.unresolved, lookupTableValue[T]{ + rid: x, + uid: u, + }) + u |= marker + t.resolved[x] = u + return u +} + +func (t *lookupTable[T]) storeResolved(uid, v uint32) { + t.resolved[t.unresolved[uid].rid] = v + 1 +} + +func (t *lookupTable[T]) lookupResolved(x uint32) uint32 { + if x&marker > 0 { + return t.resolved[t.unresolved[x&markedMask].rid] - 1 + } + return x // Already resolved. +} + +func (t *lookupTable[T]) iter() *lookupTableIterator[T] { + if cap(t.refs) < len(t.unresolved) { + t.refs = make([]lookupTableRef, len(t.unresolved)) + } else { + t.refs = t.refs[:len(t.unresolved)] + } + for i, v := range t.unresolved { + t.refs[i] = lookupTableRef{ + rid: v.rid, + uid: v.uid, + } + } + sort.Slice(t.refs, func(i, j int) bool { + return t.refs[i].rid < t.refs[j].rid + }) + return &lookupTableIterator[T]{table: t} +} + +type lookupTableIterator[T any] struct { + table *lookupTable[T] + cur uint32 +} + +func (t *lookupTableIterator[T]) Next() bool { + return t.cur < uint32(len(t.table.refs)) +} + +func (t *lookupTableIterator[T]) At() uint32 { + x := t.table.refs[t.cur].rid + t.cur++ + return x +} + +func (t *lookupTableIterator[T]) setValue(v T) { + uid := t.table.refs[t.cur].uid + t.table.unresolved[uid].val = v +} + +func (t *lookupTableIterator[T]) Close() error { return nil } + +func (t *lookupTableIterator[T]) Err() error { return nil } + +// TODO(kolesnikovae): + +type symbolsWriter struct{} + +func newSymbolsWriter(dst string) (*symbolsWriter, error) { + return &symbolsWriter{}, nil +} + +func (w *symbolsWriter) SymbolsAppender(partition uint64) (SymbolsAppender, error) { + return nil, nil +} From 59258be2fcc566c41831738437647eebc538a253 Mon Sep 17 00:00:00 2001 From: Anton Kolesnikov Date: Mon, 17 Jul 2023 21:06:28 +0800 Subject: [PATCH 19/35] Add lookup table test --- pkg/phlaredb/block_querier.go | 5 ++ pkg/phlaredb/compact.go | 58 +++++++++---------- pkg/phlaredb/compact_test.go | 104 +++++++++++++++++++++++++++++++++- 3 files changed, 137 insertions(+), 30 deletions(-) diff --git a/pkg/phlaredb/block_querier.go b/pkg/phlaredb/block_querier.go index 7f675cc7fe..140c9010c5 100644 --- a/pkg/phlaredb/block_querier.go +++ b/pkg/phlaredb/block_querier.go @@ -435,6 +435,11 @@ func (b *singleBlockQuerier) Index() IndexReader { return b.index } +func (b *singleBlockQuerier) Symbols() SymbolsReader { + // TODO(kolesnikovae) + return nil +} + func (b *singleBlockQuerier) Meta() block.Meta { if b.meta == nil { return block.Meta{} diff --git a/pkg/phlaredb/compact.go b/pkg/phlaredb/compact.go index c89208d1e0..8a7156088e 100644 --- a/pkg/phlaredb/compact.go +++ b/pkg/phlaredb/compact.go @@ -30,7 +30,7 @@ type BlockReader interface { Meta() block.Meta Profiles() []parquet.RowGroup Index() IndexReader - SymbolsReader + Symbols() SymbolsReader } // TODO(kolesnikovae): Refactor to symdb. @@ -517,7 +517,7 @@ func newSymbolsRewriter(it iter.Iterator[profileRow], blocks []BlockReader, w Sy rewriters: make(map[BlockReader]*stacktraceRewriter, len(blocks)), } for _, r := range blocks { - sr.rewriters[r] = newStacktraceRewriter(r, w) + sr.rewriters[r] = newStacktraceRewriter(r.Symbols(), w) } return &sr } @@ -549,6 +549,7 @@ func (s *symbolsRewriter) Next() bool { } s.numSamples += uint64(len(values)) for i, v := range values { + // FIXME: the original order is not preserved, which will affect encoding. values[i] = parquet.Int64Value(int64(s.stacktraces[i])).Level(v.RepetitionLevel(), v.DefinitionLevel(), v.Column()) } }) @@ -729,7 +730,7 @@ func (r *stacktraceRewriter) populateUnresolved(stacktraceIDs []uint32) error { func (r *stacktraceRewriter) appendRewrite(stacktraces []uint32) error { for _, v := range r.strings.unresolved { - r.strings.storeResolved(v.uid, r.appender.AppendString(v.val)) + r.strings.storeResolved(v.rid, r.appender.AppendString(v.val)) } for _, v := range r.functions.unresolved { @@ -737,14 +738,14 @@ func (r *stacktraceRewriter) appendRewrite(stacktraces []uint32) error { function.Name = r.strings.lookupResolved(function.Name) function.Filename = r.strings.lookupResolved(function.Filename) function.SystemName = r.strings.lookupResolved(function.SystemName) - r.functions.storeResolved(v.uid, r.appender.AppendFunction(function)) + r.functions.storeResolved(v.rid, r.appender.AppendFunction(function)) } for _, v := range r.mappings.unresolved { mapping := v.val mapping.BuildId = r.strings.lookupResolved(mapping.BuildId) mapping.Filename = r.strings.lookupResolved(mapping.Filename) - r.mappings.storeResolved(v.uid, r.appender.AppendMapping(mapping)) + r.mappings.storeResolved(v.rid, r.appender.AppendMapping(mapping)) } for _, v := range r.locations.unresolved { @@ -753,7 +754,7 @@ func (r *stacktraceRewriter) appendRewrite(stacktraces []uint32) error { for j, line := range location.Line { location.Line[j].FunctionId = r.functions.lookupResolved(line.FunctionId) } - r.locations.storeResolved(v.uid, r.appender.AppendLocation(location)) + r.locations.storeResolved(v.rid, r.appender.AppendLocation(location)) } src := r.stacktraces[r.partition] @@ -762,7 +763,7 @@ func (r *stacktraceRewriter) appendRewrite(stacktraces []uint32) error { for j, lid := range stacktrace.LocationIDs { stacktrace.LocationIDs[j] = uint64(r.locations.lookupResolved(uint32(lid))) } - src.storeResolved(v.uid, r.appender.AppendStacktrace(stacktrace)) + src.storeResolved(v.rid, r.appender.AppendStacktrace(stacktrace)) } for i, v := range stacktraces { stacktraces[i] = src.lookupResolved(v) @@ -773,7 +774,7 @@ func (r *stacktraceRewriter) appendRewrite(stacktraces []uint32) error { const ( marker = 1 << 31 - markedMask = math.MaxUint32 >> 1 + markerMask = math.MaxUint32 >> 1 ) type lookupTable[T any] struct { @@ -786,11 +787,13 @@ type lookupTable[T any] struct { type lookupTableValue[T any] struct { rid uint32 // Index to resolved. - uid uint32 // Original index (unresolved). val T } -type lookupTableRef struct{ rid, uid uint32 } +type lookupTableRef struct { + rid uint32 // Index to resolved. + uid uint32 // Original index (unresolved). +} func newLookupTable[T any](size int) *lookupTable[T] { var t lookupTable[T] @@ -825,45 +828,42 @@ func (t *lookupTable[T]) tryLookup(x uint32) uint32 { } return v - 1 // Already resolved. } - u := uint32(len(t.unresolved)) - t.unresolved = append(t.unresolved, lookupTableValue[T]{ - rid: x, - uid: u, - }) - u |= marker + t.unresolved = append(t.unresolved, lookupTableValue[T]{rid: x}) + u := uint32(len(t.unresolved)) | marker t.resolved[x] = u return u } -func (t *lookupTable[T]) storeResolved(uid, v uint32) { - t.resolved[t.unresolved[uid].rid] = v + 1 -} +func (t *lookupTable[T]) storeResolved(rid, v uint32) { t.resolved[rid] = v + 1 } func (t *lookupTable[T]) lookupResolved(x uint32) uint32 { if x&marker > 0 { - return t.resolved[t.unresolved[x&markedMask].rid] - 1 + return t.resolved[t.unresolved[x&markerMask-1].rid] - 1 } return x // Already resolved. } func (t *lookupTable[T]) iter() *lookupTableIterator[T] { + t.initRefs() + sort.Sort(t) + return &lookupTableIterator[T]{table: t} +} + +func (t *lookupTable[T]) initRefs() { if cap(t.refs) < len(t.unresolved) { t.refs = make([]lookupTableRef, len(t.unresolved)) } else { t.refs = t.refs[:len(t.unresolved)] } for i, v := range t.unresolved { - t.refs[i] = lookupTableRef{ - rid: v.rid, - uid: v.uid, - } + t.refs[i] = lookupTableRef{rid: v.rid, uid: uint32(i)} } - sort.Slice(t.refs, func(i, j int) bool { - return t.refs[i].rid < t.refs[j].rid - }) - return &lookupTableIterator[T]{table: t} } +func (t *lookupTable[T]) Len() int { return len(t.refs) } +func (t *lookupTable[T]) Less(i, j int) bool { return t.refs[i].rid < t.refs[j].rid } +func (t *lookupTable[T]) Swap(i, j int) { t.refs[i], t.refs[j] = t.refs[j], t.refs[i] } + type lookupTableIterator[T any] struct { table *lookupTable[T] cur uint32 @@ -880,7 +880,7 @@ func (t *lookupTableIterator[T]) At() uint32 { } func (t *lookupTableIterator[T]) setValue(v T) { - uid := t.table.refs[t.cur].uid + uid := t.table.refs[t.cur-1].uid t.table.unresolved[uid].val = v } diff --git a/pkg/phlaredb/compact_test.go b/pkg/phlaredb/compact_test.go index acab55278e..07f5b1580f 100644 --- a/pkg/phlaredb/compact_test.go +++ b/pkg/phlaredb/compact_test.go @@ -136,6 +136,15 @@ func testCompact(t *testing.T, metas []*block.Meta, bkt phlareobj.Bucket, dst st "numSamples", new.Stats.NumSamples) } +type blockReaderMock struct { + BlockReader + idxr IndexReader +} + +func (m *blockReaderMock) Index() IndexReader { + return m.idxr +} + func TestProfileRowIterator(t *testing.T) { filePath := t.TempDir() + "/index.tsdb" idxw, err := index.NewWriter(context.Background(), filePath) @@ -162,7 +171,7 @@ func TestProfileRowIterator(t *testing.T) { {SeriesIndex: 1, TimeNanos: 2}, {SeriesIndex: 2, TimeNanos: 3}, }, - ), idxr) + ), &blockReaderMock{idxr: idxr}) require.NoError(t, err) assert.True(t, it.Next()) @@ -294,3 +303,96 @@ func generateParquetFile(t *testing.T, path string) { require.NoError(t, err) } } + +func Test_lookupTable(t *testing.T) { + // Given the source data set. + // Copy arbitrary subsets of those items to dst. + var dst []string + src := []string{ + "zero", + "one", + "two", + "three", + "four", + "five", + "six", + "seven", + } + + type testCase struct { + description string + input []uint32 + expected []string + } + + testCases := []testCase{ + { + description: "empty table", + input: []uint32{5, 0, 3, 1, 2, 2, 4}, + expected: []string{"five", "zero", "three", "one", "two", "two", "four"}, + }, + { + description: "no new values", + input: []uint32{2, 1, 2, 3}, + expected: []string{"two", "one", "two", "three"}, + }, + { + description: "new value mixed", + input: []uint32{2, 1, 6, 2, 3}, + expected: []string{"two", "one", "six", "two", "three"}, + }, + } + + // Try to lookup values in src lazily. + // Table size must be greater or equal + // to the source data set. + l := newLookupTable[string](10) + + populate := func(t *testing.T, x []uint32) { + for i, v := range x { + x[i] = l.tryLookup(v) + } + // Resolve unknown yet values. + // Mind the order and deduplication. + p := -1 + for it := l.iter(); it.Err() == nil && it.Next(); { + m := int(it.At()) + if m <= p { + t.Fatal("iterator order invalid") + } + p = m + it.setValue(src[m]) + } + } + + resolveAppend := func() { + // Populate dst with the newly resolved values. + // Note that order in dst does not have to match src. + for _, n := range l.unresolved { + l.storeResolved(n.rid, uint32(len(dst))) + dst = append(dst, n.val) + } + } + + resolve := func(x []uint32) []string { + // Lookup resolved values. + var resolved []string + for _, v := range x { + resolved = append(resolved, dst[l.lookupResolved(v)]) + } + return resolved + } + + for _, tc := range testCases { + tc := tc + t.Run(tc.description, func(t *testing.T) { + l.reset() + populate(t, tc.input) + resolveAppend() + assert.Equal(t, tc.expected, resolve(tc.input)) + }) + } + + assert.Len(t, dst, 7) + assert.NotContains(t, dst, "seven") +} From c0f7566e33299407e1980508c8552c858c0401f6 Mon Sep 17 00:00:00 2001 From: Anton Kolesnikov Date: Tue, 18 Jul 2023 17:19:51 +0800 Subject: [PATCH 20/35] Symbols reader integration --- pkg/iter/iter.go | 14 ++ pkg/phlaredb/block_querier.go | 22 ++-- pkg/phlaredb/block_symbols_appender.go | 19 +++ pkg/phlaredb/block_symbols_reader.go | 92 +++++++++++++ pkg/phlaredb/compact.go | 176 ++++++++++++++----------- pkg/phlaredb/sample_merge.go | 16 +-- pkg/phlaredb/symdb/interfaces.go | 1 + 7 files changed, 238 insertions(+), 102 deletions(-) create mode 100644 pkg/phlaredb/block_symbols_appender.go create mode 100644 pkg/phlaredb/block_symbols_reader.go diff --git a/pkg/iter/iter.go b/pkg/iter/iter.go index 47fbe6f4e6..34a16f7db6 100644 --- a/pkg/iter/iter.go +++ b/pkg/iter/iter.go @@ -101,6 +101,20 @@ func NewSliceSeekIterator[A constraints.Ordered](s []A) SeekIterator[A, A] { } } +type slicePositionIterator[T constraints.Integer, M any] struct { + i Iterator[T] + s []M +} + +func NewSliceIndexIterator[T constraints.Integer, M any](s []M, i Iterator[T]) Iterator[M] { + return slicePositionIterator[T, M]{s: s, i: i} +} + +func (i slicePositionIterator[T, M]) Next() bool { return i.i.Next() } +func (i slicePositionIterator[T, M]) At() M { return i.s[i.i.At()] } +func (i slicePositionIterator[T, M]) Err() error { return i.i.Err() } +func (i slicePositionIterator[T, M]) Close() error { return i.i.Close() } + type sliceSeekIterator[A constraints.Ordered] struct { *sliceIterator[A] } diff --git a/pkg/phlaredb/block_querier.go b/pkg/phlaredb/block_querier.go index 140c9010c5..7795ba6ea2 100644 --- a/pkg/phlaredb/block_querier.go +++ b/pkg/phlaredb/block_querier.go @@ -305,7 +305,7 @@ type singleBlockQuerier struct { type StacktraceDB interface { Open(ctx context.Context) error Close() error - Resolve(ctx context.Context, mapping uint64, locs locationsIdsByStacktraceID, stacktraceIDs []uint32) error + Resolve(ctx context.Context, partition uint64, locs symdb.StacktraceInserter, stacktraceIDs []uint32) error } type stacktraceResolverV1 struct { @@ -321,14 +321,17 @@ func (r *stacktraceResolverV1) Close() error { return r.stacktraces.Close() } -func (r *stacktraceResolverV1) Resolve(ctx context.Context, mapping uint64, locs locationsIdsByStacktraceID, stacktraceIDs []uint32) error { +func (r *stacktraceResolverV1) Resolve(ctx context.Context, _ uint64, locs symdb.StacktraceInserter, stacktraceIDs []uint32) error { stacktraces := repeatedColumnIter(ctx, r.stacktraces.file, "LocationIDs.list.element", iter.NewSliceIterator(stacktraceIDs)) defer stacktraces.Close() - + t := make([]int32, 0, 64) for stacktraces.Next() { + t = t[:0] s := stacktraces.At() - locs.addFromParquet(int64(s.Row), s.Values) - + for i, v := range s.Values { + t[i] = v.Int32() + } + locs.InsertStacktrace(s.Row, t) } return stacktraces.Err() } @@ -351,19 +354,14 @@ func (r *stacktraceResolverV2) Close() error { return nil } -func (r *stacktraceResolverV2) Resolve(ctx context.Context, mapping uint64, locs locationsIdsByStacktraceID, stacktraceIDs []uint32) error { +func (r *stacktraceResolverV2) Resolve(ctx context.Context, mapping uint64, locs symdb.StacktraceInserter, stacktraceIDs []uint32) error { mr, ok := r.reader.MappingReader(mapping) if !ok { return nil } resolver := mr.StacktraceResolver() defer resolver.Release() - - return resolver.ResolveStacktraces(ctx, symdb.StacktraceInserterFn( - func(stacktraceID uint32, locations []int32) { - locs.add(int64(stacktraceID), locations) - }, - ), stacktraceIDs) + return resolver.ResolveStacktraces(ctx, locs, stacktraceIDs) } func NewSingleBlockQuerierFromMeta(phlarectx context.Context, bucketReader phlareobj.Bucket, meta *block.Meta) *singleBlockQuerier { diff --git a/pkg/phlaredb/block_symbols_appender.go b/pkg/phlaredb/block_symbols_appender.go new file mode 100644 index 0000000000..b39adb9e27 --- /dev/null +++ b/pkg/phlaredb/block_symbols_appender.go @@ -0,0 +1,19 @@ +package phlaredb + +import schemav1 "github.com/grafana/phlare/pkg/phlaredb/schemas/v1" + +// TODO(kolesnikovae): Refactor to symdb. + +type SymbolsWriter interface { + SymbolsAppender(partition uint64) (SymbolsAppender, error) +} + +type SymbolsAppender interface { + AppendStacktrace([]int32) uint32 + AppendLocation(*schemav1.InMemoryLocation) uint32 + AppendMapping(*schemav1.InMemoryMapping) uint32 + AppendFunction(*schemav1.InMemoryFunction) uint32 + AppendString(string) uint32 + + Flush() error +} diff --git a/pkg/phlaredb/block_symbols_reader.go b/pkg/phlaredb/block_symbols_reader.go new file mode 100644 index 0000000000..6c6df7a52d --- /dev/null +++ b/pkg/phlaredb/block_symbols_reader.go @@ -0,0 +1,92 @@ +package phlaredb + +import ( + "context" + + "github.com/grafana/phlare/pkg/iter" + schemav1 "github.com/grafana/phlare/pkg/phlaredb/schemas/v1" + "github.com/grafana/phlare/pkg/phlaredb/symdb" +) + +// TODO(kolesnikovae): Refactor to symdb. + +type SymbolsReader interface { + SymbolsResolver(partition uint64) (SymbolsResolver, error) +} + +type SymbolsResolver interface { + ResolveStacktraces(ctx context.Context, dst symdb.StacktraceInserter, stacktraces []uint32) error + + Locations(iter.Iterator[uint32]) iter.Iterator[*schemav1.InMemoryLocation] + Mappings(iter.Iterator[uint32]) iter.Iterator[*schemav1.InMemoryMapping] + Functions(iter.Iterator[uint32]) iter.Iterator[*schemav1.InMemoryFunction] + Strings(iter.Iterator[uint32]) iter.Iterator[string] + + WriteStats(*SymbolStats) +} + +type SymbolStats struct { + StacktracesTotal int + LocationsTotal int + MappingsTotal int + FunctionsTotal int + StringsTotal int +} + +type inMemorySymbolsReader struct { + partitions map[uint64]*inMemorySymbolsResolver + + // TODO(kolesnikovae): Split into partitions. + strings inMemoryparquetReader[string, *schemav1.StringPersister] + functions inMemoryparquetReader[*schemav1.InMemoryFunction, *schemav1.FunctionPersister] + locations inMemoryparquetReader[*schemav1.InMemoryLocation, *schemav1.LocationPersister] + mappings inMemoryparquetReader[*schemav1.InMemoryMapping, *schemav1.MappingPersister] + stacktraces StacktraceDB +} + +func (r *inMemorySymbolsReader) Symbols(partition uint64) SymbolsResolver { + p, ok := r.partitions[partition] + if !ok { + p = &inMemorySymbolsResolver{ + partition: 0, + ctx: nil, + reader: nil, + } + r.partitions[partition] = p + } + return p +} + +type inMemorySymbolsResolver struct { + partition uint64 + ctx context.Context + reader *inMemorySymbolsReader +} + +func (s inMemorySymbolsResolver) ResolveStacktraces(ctx context.Context, dst symdb.StacktraceInserter, stacktraces []uint32) error { + return s.reader.stacktraces.Resolve(ctx, s.partition, dst, stacktraces) +} + +func (s inMemorySymbolsResolver) Locations(i iter.Iterator[uint32]) iter.Iterator[*schemav1.InMemoryLocation] { + return iter.NewSliceIndexIterator(s.reader.locations.cache, i) +} + +func (s inMemorySymbolsResolver) Mappings(i iter.Iterator[uint32]) iter.Iterator[*schemav1.InMemoryMapping] { + return iter.NewSliceIndexIterator(s.reader.mappings.cache, i) +} + +func (s inMemorySymbolsResolver) Functions(i iter.Iterator[uint32]) iter.Iterator[*schemav1.InMemoryFunction] { + return iter.NewSliceIndexIterator(s.reader.functions.cache, i) +} + +func (s inMemorySymbolsResolver) Strings(i iter.Iterator[uint32]) iter.Iterator[string] { + return iter.NewSliceIndexIterator(s.reader.strings.cache, i) +} + +func (s inMemorySymbolsResolver) WriteStats(stats *SymbolStats) { + stats.StacktracesTotal = 0 // TODO + stats.LocationsTotal = int(s.reader.locations.NumRows()) + stats.MappingsTotal = int(s.reader.mappings.NumRows()) + stats.FunctionsTotal = int(s.reader.functions.NumRows()) + stats.StringsTotal = int(s.reader.strings.NumRows()) +} diff --git a/pkg/phlaredb/compact.go b/pkg/phlaredb/compact.go index 8a7156088e..ea2d1eed8e 100644 --- a/pkg/phlaredb/compact.go +++ b/pkg/phlaredb/compact.go @@ -33,42 +33,6 @@ type BlockReader interface { Symbols() SymbolsReader } -// TODO(kolesnikovae): Refactor to symdb. - -type SymbolsReader interface { - SymbolsResolver(partition uint64) (SymbolsResolver, error) -} - -type SymbolsResolver interface { - Stacktraces(iter.Iterator[uint32]) iter.Iterator[*schemav1.Stacktrace] - Locations(iter.Iterator[uint32]) iter.Iterator[*schemav1.InMemoryLocation] - Mappings(iter.Iterator[uint32]) iter.Iterator[*schemav1.InMemoryMapping] - Functions(iter.Iterator[uint32]) iter.Iterator[*schemav1.InMemoryFunction] - Strings(iter.Iterator[uint32]) iter.Iterator[string] - WriteStats(*SymbolStats) -} - -type SymbolStats struct { - StacktracesTotal int - LocationsTotal int - MappingsTotal int - FunctionsTotal int - StringsTotal int -} - -type SymbolsWriter interface { - SymbolsAppender(partition uint64) (SymbolsAppender, error) -} - -type SymbolsAppender interface { - AppendStacktrace(*schemav1.Stacktrace) uint32 - AppendLocation(*schemav1.InMemoryLocation) uint32 - AppendMapping(*schemav1.InMemoryMapping) uint32 - AppendFunction(*schemav1.InMemoryFunction) uint32 - AppendString(string) uint32 - Flush() error -} - func Compact(ctx context.Context, src []BlockReader, dst string) (meta block.Meta, err error) { srcMetas := make([]block.Meta, len(src)) ulids := make([]string, len(src)) @@ -575,7 +539,9 @@ type stacktraceRewriter struct { writer SymbolsWriter // Stack trace identifiers are only valid within the partition. - stacktraces map[uint64]*lookupTable[*schemav1.Stacktrace] + stacktraces map[uint64]*lookupTable[[]int32] + inserter *stacktraceInserter + // Objects below have global addressing. locations *lookupTable[*schemav1.InMemoryLocation] mappings *lookupTable[*schemav1.InMemoryMapping] @@ -607,11 +573,12 @@ func (r *stacktraceRewriter) init(partition uint64) (err error) { // Only stacktraces are yet partitioned. if r.stacktraces == nil { - r.stacktraces = make(map[uint64]*lookupTable[*schemav1.Stacktrace]) + r.stacktraces = make(map[uint64]*lookupTable[[]int32]) + r.inserter = new(stacktraceInserter) } p, ok := r.stacktraces[partition] if !ok { - p = newLookupTable[*schemav1.Stacktrace](r.stats.StacktracesTotal) + p = newLookupTable[[]int32](r.stats.StacktracesTotal) r.stacktraces[partition] = p } p.reset() @@ -627,6 +594,13 @@ func (r *stacktraceRewriter) init(partition uint64) (err error) { r.mappings.reset() r.functions.reset() r.strings.reset() + + r.inserter = &stacktraceInserter{ + slt: p, + llt: r.locations, + s: r.inserter.s, + } + return nil } @@ -654,35 +628,20 @@ func (r *stacktraceRewriter) rewriteStacktraces(partition uint64, stacktraces [] } func (r *stacktraceRewriter) populateUnresolved(stacktraceIDs []uint32) error { - // Filter out all stack traces that have been already resolved. - src := r.stacktraces[r.partition] - for i, v := range stacktraceIDs { - stacktraceIDs[i] = src.tryLookup(v) + // Filter out all stack traces that have been already + // resolved and populate locations lookup table. + if err := r.resolveStacktraces(stacktraceIDs); err != nil { + return err } - if len(src.unresolved) == 0 { + if len(r.locations.unresolved) == 0 { return nil } - // Resolve locations for new stack traces. - var stacktrace *schemav1.Stacktrace - unresolvedStacktraces := src.iter() - stacktraces := r.resolver.Stacktraces(unresolvedStacktraces) - for ; stacktraces.Err() == nil && stacktraces.Next(); stacktrace = stacktraces.At() { - for j, loc := range stacktrace.LocationIDs { - stacktrace.LocationIDs[j] = uint64(r.locations.tryLookup(uint32(loc))) - } - // TODO(kolesnikovae): Copy. - unresolvedStacktraces.setValue(stacktrace) - } - if err := stacktraces.Err(); err != nil { - return err - } - // Resolve functions and mappings for new locations. - var location *schemav1.InMemoryLocation unresolvedLocs := r.locations.iter() locations := r.resolver.Locations(unresolvedLocs) - for ; locations.Err() == nil && locations.Next(); location = locations.At() { + for locations.Err() == nil && locations.Next() { + location := locations.At() location.MappingId = r.mappings.tryLookup(location.MappingId) for j, line := range location.Line { location.Line[j].FunctionId = r.functions.tryLookup(line.FunctionId) @@ -694,10 +653,10 @@ func (r *stacktraceRewriter) populateUnresolved(stacktraceIDs []uint32) error { } // Resolve strings. - var mapping *schemav1.InMemoryMapping unresolvedMappings := r.mappings.iter() mappings := r.resolver.Mappings(unresolvedMappings) - for ; mappings.Err() == nil && mappings.Next(); mapping = mappings.At() { + for mappings.Err() == nil && mappings.Next() { + mapping := mappings.At() mapping.BuildId = r.strings.tryLookup(mapping.BuildId) mapping.Filename = r.strings.tryLookup(mapping.Filename) unresolvedMappings.setValue(mapping) @@ -706,10 +665,10 @@ func (r *stacktraceRewriter) populateUnresolved(stacktraceIDs []uint32) error { return err } - var function *schemav1.InMemoryFunction unresolvedFunctions := r.functions.iter() functions := r.resolver.Functions(unresolvedFunctions) - for ; functions.Err() == nil && functions.Next(); function = functions.At() { + for functions.Err() == nil && functions.Next() { + function := functions.At() function.Name = r.strings.tryLookup(function.Name) function.Filename = r.strings.tryLookup(function.Filename) function.SystemName = r.strings.tryLookup(function.SystemName) @@ -719,11 +678,10 @@ func (r *stacktraceRewriter) populateUnresolved(stacktraceIDs []uint32) error { return err } - var str string unresolvedStrings := r.strings.iter() strings := r.resolver.Strings(unresolvedStrings) - for ; strings.Err() == nil && strings.Next(); str = strings.At() { - unresolvedStrings.setValue(str) + for strings.Err() == nil && strings.Next() { + unresolvedStrings.setValue(strings.At()) } return strings.Err() } @@ -760,8 +718,8 @@ func (r *stacktraceRewriter) appendRewrite(stacktraces []uint32) error { src := r.stacktraces[r.partition] for _, v := range src.unresolved { stacktrace := v.val - for j, lid := range stacktrace.LocationIDs { - stacktrace.LocationIDs[j] = uint64(r.locations.lookupResolved(uint32(lid))) + for j, lid := range stacktrace { + stacktrace[j] = int32(r.locations.lookupResolved(uint32(lid))) } src.storeResolved(v.rid, r.appender.AppendStacktrace(stacktrace)) } @@ -772,6 +730,48 @@ func (r *stacktraceRewriter) appendRewrite(stacktraces []uint32) error { return r.appender.Flush() } +func (r *stacktraceRewriter) resolveStacktraces(stacktraceIDs []uint32) error { + stacktraces := r.stacktraces[r.partition] + for i, v := range stacktraceIDs { + stacktraceIDs[i] = stacktraces.tryLookup(v) + } + if len(stacktraces.unresolved) == 0 { + return nil + } + + // Gather and sort references to unresolved stacks. + stacktraces.initRefs() + sort.Sort(stacktraces) + grow(r.inserter.s, len(stacktraces.refs)) + for j, u := range stacktraces.refs { + r.inserter.s[j] = u.rid + } + + return r.resolver.ResolveStacktraces(context.TODO(), r.inserter, r.inserter.s) +} + +type stacktraceInserter struct { + slt *lookupTable[[]int32] + llt *lookupTable[*schemav1.InMemoryLocation] + s []uint32 + c int +} + +func (i *stacktraceInserter) InsertStacktrace(stacktrace uint32, locations []int32) { + // Resolve locations for new stack traces. + for j, loc := range locations { + locations[j] = int32(i.llt.tryLookup(uint32(loc))) + } + // Update the unresolved value. + v := i.slt.referenceAt(i.c) + if v.rid != stacktrace { + panic("unexpected stack trace") + } + grow(v.val, len(locations)) + copy(v.val, locations) + i.c++ +} + const ( marker = 1 << 31 markerMask = math.MaxUint32 >> 1 @@ -828,16 +828,33 @@ func (t *lookupTable[T]) tryLookup(x uint32) uint32 { } return v - 1 // Already resolved. } - t.unresolved = append(t.unresolved, lookupTableValue[T]{rid: x}) - u := uint32(len(t.unresolved)) | marker + u := t.newUnresolvedValue(x) | marker t.resolved[x] = u return u } +func (t *lookupTable[T]) newUnresolvedValue(rid uint32) uint32 { + x := len(t.unresolved) + if x < cap(t.unresolved) { + // Try to reuse previously allocated value. + x++ + t.unresolved = t.unresolved[:x] + t.unresolved[x].rid = rid + } else { + t.unresolved = append(t.unresolved, lookupTableValue[T]{rid: rid}) + } + return uint32(x) +} + +func (t *lookupTable[T]) referenceAt(x int) *lookupTableValue[T] { + u := t.refs[x].uid + return &t.unresolved[u] +} + func (t *lookupTable[T]) storeResolved(rid, v uint32) { t.resolved[rid] = v + 1 } func (t *lookupTable[T]) lookupResolved(x uint32) uint32 { - if x&marker > 0 { + if x&marker > 0 { // TODO: why? return t.resolved[t.unresolved[x&markerMask-1].rid] - 1 } return x // Already resolved. @@ -850,11 +867,7 @@ func (t *lookupTable[T]) iter() *lookupTableIterator[T] { } func (t *lookupTable[T]) initRefs() { - if cap(t.refs) < len(t.unresolved) { - t.refs = make([]lookupTableRef, len(t.unresolved)) - } else { - t.refs = t.refs[:len(t.unresolved)] - } + grow(t.refs, len(t.unresolved)) for i, v := range t.unresolved { t.refs[i] = lookupTableRef{rid: v.rid, uid: uint32(i)} } @@ -888,6 +901,13 @@ func (t *lookupTableIterator[T]) Close() error { return nil } func (t *lookupTableIterator[T]) Err() error { return nil } +func grow[T any](s []T, n int) []T { + if cap(s) < n { + return make([]T, n, 2*n) + } + return s[:n] +} + // TODO(kolesnikovae): type symbolsWriter struct{} diff --git a/pkg/phlaredb/sample_merge.go b/pkg/phlaredb/sample_merge.go index e9fcd2671b..1c44f24a76 100644 --- a/pkg/phlaredb/sample_merge.go +++ b/pkg/phlaredb/sample_merge.go @@ -56,20 +56,12 @@ func newLocationsIdsByStacktraceID(size int) locationsIdsByStacktraceID { } } -func (l locationsIdsByStacktraceID) addFromParquet(stacktraceID int64, locs []parquet.Value) { - l.byStacktraceID[stacktraceID] = make([]int32, len(locs)) - for i, locationID := range locs { - locID := locationID.Uint64() - l.ids[int64(locID)] = struct{}{} - l.byStacktraceID[stacktraceID][i] = int32(locID) - } -} - -func (l locationsIdsByStacktraceID) add(stacktraceID int64, locs []int32) { - l.byStacktraceID[stacktraceID] = make([]int32, len(locs)) +func (l locationsIdsByStacktraceID) InsertStacktrace(stacktraceID uint32, locs []int32) { + s := make([]int32, len(locs)) + l.byStacktraceID[int64(stacktraceID)] = s for i, locationID := range locs { l.ids[int64(locationID)] = struct{}{} - l.byStacktraceID[stacktraceID][i] = locationID + s[i] = locationID } } diff --git a/pkg/phlaredb/symdb/interfaces.go b/pkg/phlaredb/symdb/interfaces.go index 2b9baaac84..f997437470 100644 --- a/pkg/phlaredb/symdb/interfaces.go +++ b/pkg/phlaredb/symdb/interfaces.go @@ -10,6 +10,7 @@ import ( // collection. https://github.com/google/pprof/blob/main/proto/README.md // // In the package, Mapping represents all the version of a binary. +// TODO(kolesnikovae): Rename mapping to Partition type MappingWriter interface { // StacktraceAppender provides exclusive write access From 040679117defe3001bfc190db4744d3148f89388 Mon Sep 17 00:00:00 2001 From: Anton Kolesnikov Date: Tue, 18 Jul 2023 19:53:46 +0800 Subject: [PATCH 21/35] Add SymbolsResolver.WriteStats --- pkg/phlaredb/block_querier.go | 19 +- pkg/phlaredb/block_symbols_reader.go | 24 +- ...ls_appender.go => block_symbols_writer.go} | 10 + pkg/phlaredb/compact.go | 229 +++++++++--------- pkg/phlaredb/head.go | 6 +- pkg/phlaredb/symdb/format.go | 26 +- pkg/phlaredb/symdb/interfaces.go | 27 +-- ...{mapping_memory.go => partition_memory.go} | 87 ++++--- ...emory_test.go => partition_memory_test.go} | 44 ++-- pkg/phlaredb/symdb/stacktrace_tree_test.go | 2 +- ..._reader_file.go => symbols_reader_file.go} | 98 ++++---- ...le_test.go => symbols_reader_file_test.go} | 4 +- ..._writer_file.go => symbols_writer_file.go} | 6 +- ...le_test.go => symbols_writer_file_test.go} | 36 +-- pkg/phlaredb/symdb/symdb.go | 50 ++-- 15 files changed, 347 insertions(+), 321 deletions(-) rename pkg/phlaredb/{block_symbols_appender.go => block_symbols_writer.go} (68%) rename pkg/phlaredb/symdb/{mapping_memory.go => partition_memory.go} (76%) rename pkg/phlaredb/symdb/{mapping_memory_test.go => partition_memory_test.go} (95%) rename pkg/phlaredb/symdb/{mapping_reader_file.go => symbols_reader_file.go} (70%) rename pkg/phlaredb/symdb/{mapping_reader_file_test.go => symbols_reader_file_test.go} (95%) rename pkg/phlaredb/symdb/{mapping_writer_file.go => symbols_writer_file.go} (96%) rename pkg/phlaredb/symdb/{mapping_writer_file_test.go => symbols_writer_file_test.go} (84%) diff --git a/pkg/phlaredb/block_querier.go b/pkg/phlaredb/block_querier.go index 7795ba6ea2..85f39c5e02 100644 --- a/pkg/phlaredb/block_querier.go +++ b/pkg/phlaredb/block_querier.go @@ -306,6 +306,7 @@ type StacktraceDB interface { Open(ctx context.Context) error Close() error Resolve(ctx context.Context, partition uint64, locs symdb.StacktraceInserter, stacktraceIDs []uint32) error + WriteStats(partition uint64, s *symdb.Stats) } type stacktraceResolverV1 struct { @@ -336,6 +337,11 @@ func (r *stacktraceResolverV1) Resolve(ctx context.Context, _ uint64, locs symdb return stacktraces.Err() } +func (r *stacktraceResolverV1) WriteStats(_ uint64, s *symdb.Stats) { + s.StacktracesTotal = int(r.stacktraces.file.NumRows()) + s.MaxStacktraceID = s.StacktracesTotal +} + type stacktraceResolverV2 struct { reader *symdb.Reader bucketReader phlareobj.Bucket @@ -354,8 +360,8 @@ func (r *stacktraceResolverV2) Close() error { return nil } -func (r *stacktraceResolverV2) Resolve(ctx context.Context, mapping uint64, locs symdb.StacktraceInserter, stacktraceIDs []uint32) error { - mr, ok := r.reader.MappingReader(mapping) +func (r *stacktraceResolverV2) Resolve(ctx context.Context, partition uint64, locs symdb.StacktraceInserter, stacktraceIDs []uint32) error { + mr, ok := r.reader.SymbolsResolver(partition) if !ok { return nil } @@ -364,6 +370,15 @@ func (r *stacktraceResolverV2) Resolve(ctx context.Context, mapping uint64, locs return resolver.ResolveStacktraces(ctx, locs, stacktraceIDs) } +func (r *stacktraceResolverV2) WriteStats(partition uint64, s *symdb.Stats) { + mr, ok := r.reader.SymbolsResolver(partition) + if !ok { + return + } + mr.WriteStats(s) + return +} + func NewSingleBlockQuerierFromMeta(phlarectx context.Context, bucketReader phlareobj.Bucket, meta *block.Meta) *singleBlockQuerier { q := &singleBlockQuerier{ logger: phlarecontext.Logger(phlarectx), diff --git a/pkg/phlaredb/block_symbols_reader.go b/pkg/phlaredb/block_symbols_reader.go index 6c6df7a52d..6e6455a6c8 100644 --- a/pkg/phlaredb/block_symbols_reader.go +++ b/pkg/phlaredb/block_symbols_reader.go @@ -22,15 +22,7 @@ type SymbolsResolver interface { Functions(iter.Iterator[uint32]) iter.Iterator[*schemav1.InMemoryFunction] Strings(iter.Iterator[uint32]) iter.Iterator[string] - WriteStats(*SymbolStats) -} - -type SymbolStats struct { - StacktracesTotal int - LocationsTotal int - MappingsTotal int - FunctionsTotal int - StringsTotal int + WriteStats(*symdb.Stats) } type inMemorySymbolsReader struct { @@ -44,22 +36,20 @@ type inMemorySymbolsReader struct { stacktraces StacktraceDB } -func (r *inMemorySymbolsReader) Symbols(partition uint64) SymbolsResolver { +func (r *inMemorySymbolsReader) SymbolsResolver(partition uint64) (SymbolsResolver, error) { p, ok := r.partitions[partition] if !ok { p = &inMemorySymbolsResolver{ - partition: 0, - ctx: nil, - reader: nil, + partition: partition, + reader: r, } r.partitions[partition] = p } - return p + return p, nil } type inMemorySymbolsResolver struct { partition uint64 - ctx context.Context reader *inMemorySymbolsReader } @@ -83,8 +73,8 @@ func (s inMemorySymbolsResolver) Strings(i iter.Iterator[uint32]) iter.Iterator[ return iter.NewSliceIndexIterator(s.reader.strings.cache, i) } -func (s inMemorySymbolsResolver) WriteStats(stats *SymbolStats) { - stats.StacktracesTotal = 0 // TODO +func (s inMemorySymbolsResolver) WriteStats(stats *symdb.Stats) { + s.reader.stacktraces.WriteStats(s.partition, stats) stats.LocationsTotal = int(s.reader.locations.NumRows()) stats.MappingsTotal = int(s.reader.mappings.NumRows()) stats.FunctionsTotal = int(s.reader.functions.NumRows()) diff --git a/pkg/phlaredb/block_symbols_appender.go b/pkg/phlaredb/block_symbols_writer.go similarity index 68% rename from pkg/phlaredb/block_symbols_appender.go rename to pkg/phlaredb/block_symbols_writer.go index b39adb9e27..cc0b156c02 100644 --- a/pkg/phlaredb/block_symbols_appender.go +++ b/pkg/phlaredb/block_symbols_writer.go @@ -17,3 +17,13 @@ type SymbolsAppender interface { Flush() error } + +type symbolsWriter struct{} + +func newSymbolsWriter(dst string) (*symbolsWriter, error) { + return &symbolsWriter{}, nil +} + +func (w *symbolsWriter) SymbolsAppender(partition uint64) (SymbolsAppender, error) { + return nil, nil +} diff --git a/pkg/phlaredb/compact.go b/pkg/phlaredb/compact.go index ea2d1eed8e..1df37e4d21 100644 --- a/pkg/phlaredb/compact.go +++ b/pkg/phlaredb/compact.go @@ -525,10 +525,7 @@ func (s *symbolsRewriter) Next() bool { } func (s *symbolsRewriter) loadStacktracesID(values []parquet.Value) { - if cap(s.stacktraces) < len(values) { - s.stacktraces = make([]uint32, len(values)*2) - } - s.stacktraces = s.stacktraces[:len(values)] + grow(s.stacktraces, len(values)) for i := range values { s.stacktraces[i] = values[i].Uint32() } @@ -538,20 +535,26 @@ type stacktraceRewriter struct { reader SymbolsReader writer SymbolsWriter - // Stack trace identifiers are only valid within the partition. - stacktraces map[uint64]*lookupTable[[]int32] - inserter *stacktraceInserter + partitions map[uint64]*symdbPartition + inserter *stacktraceInserter // Objects below have global addressing. + // TODO(kolesnikovae): Move to symdbPartition. locations *lookupTable[*schemav1.InMemoryLocation] mappings *lookupTable[*schemav1.InMemoryMapping] functions *lookupTable[*schemav1.InMemoryFunction] strings *lookupTable[string] +} - partition uint64 - resolver SymbolsResolver - appender SymbolsAppender - stats SymbolStats +type symdbPartition struct { + name uint64 + stats symdb.Stats + // Stacktrace identifiers are only valid within the partition. + stacktraces *lookupTable[[]int32] + resolver SymbolsResolver + appender SymbolsAppender + + r *stacktraceRewriter } func newStacktraceRewriter(r SymbolsReader, w SymbolsWriter) *stacktraceRewriter { @@ -561,90 +564,98 @@ func newStacktraceRewriter(r SymbolsReader, w SymbolsWriter) *stacktraceRewriter } } -func (r *stacktraceRewriter) init(partition uint64) (err error) { - r.partition = partition - if r.appender, err = r.writer.SymbolsAppender(partition); err != nil { - return err - } - if r.resolver, err = r.reader.SymbolsResolver(partition); err != nil { - return err - } - r.resolver.WriteStats(&r.stats) - - // Only stacktraces are yet partitioned. - if r.stacktraces == nil { - r.stacktraces = make(map[uint64]*lookupTable[[]int32]) - r.inserter = new(stacktraceInserter) +func (r *stacktraceRewriter) init(partition uint64) (p *symdbPartition, err error) { + if r.partitions == nil { + r.partitions = make(map[uint64]*symdbPartition) } - p, ok := r.stacktraces[partition] - if !ok { - p = newLookupTable[[]int32](r.stats.StacktracesTotal) - r.stacktraces[partition] = p + if p, err = r.getOrCreatePartition(partition); err != nil { + return nil, err } - p.reset() if r.locations == nil { - r.locations = newLookupTable[*schemav1.InMemoryLocation](r.stats.LocationsTotal) - r.mappings = newLookupTable[*schemav1.InMemoryMapping](r.stats.MappingsTotal) - r.functions = newLookupTable[*schemav1.InMemoryFunction](r.stats.FunctionsTotal) - r.strings = newLookupTable[string](r.stats.StringsTotal) - return nil + r.locations = newLookupTable[*schemav1.InMemoryLocation](p.stats.LocationsTotal) + r.mappings = newLookupTable[*schemav1.InMemoryMapping](p.stats.MappingsTotal) + r.functions = newLookupTable[*schemav1.InMemoryFunction](p.stats.FunctionsTotal) + r.strings = newLookupTable[string](p.stats.StringsTotal) + } else { + r.locations.reset() + r.mappings.reset() + r.functions.reset() + r.strings.reset() } - r.locations.reset() - r.mappings.reset() - r.functions.reset() - r.strings.reset() r.inserter = &stacktraceInserter{ - slt: p, + slt: p.stacktraces, llt: r.locations, s: r.inserter.s, } - return nil + return p, nil } -func (r *stacktraceRewriter) hasUnresolved() bool { - return len(r.stacktraces[r.partition].unresolved)+ - len(r.locations.unresolved)+ - len(r.mappings.unresolved)+ - len(r.functions.unresolved)+ - len(r.strings.unresolved) > 0 +func (r *stacktraceRewriter) getOrCreatePartition(partition uint64) (_ *symdbPartition, err error) { + p, ok := r.partitions[partition] + if ok { + p.reset() + return p, nil + } + n := &symdbPartition{name: partition} + if n.resolver, err = r.reader.SymbolsResolver(partition); err != nil { + return nil, err + } + if n.appender, err = r.writer.SymbolsAppender(partition); err != nil { + return nil, err + } + n.resolver.WriteStats(&n.stats) + n.stacktraces = newLookupTable[[]int32](n.stats.MaxStacktraceID) + r.partitions[partition] = n + return n, nil } -func (r *stacktraceRewriter) rewriteStacktraces(partition uint64, stacktraces []uint32) (err error) { - if err = r.init(partition); err != nil { +func (r *stacktraceRewriter) rewriteStacktraces(partition uint64, stacktraces []uint32) error { + p, err := r.init(partition) + if err != nil { return err } - if err = r.populateUnresolved(stacktraces); err != nil { + if err = p.populateUnresolved(stacktraces); err != nil { return err } - if r.hasUnresolved() { - if err = r.appendRewrite(stacktraces); err != nil { - return err - } + if p.hasUnresolved() { + return p.appendRewrite(stacktraces) } return nil } -func (r *stacktraceRewriter) populateUnresolved(stacktraceIDs []uint32) error { +func (p *symdbPartition) reset() { + p.stacktraces.reset() +} + +func (p *symdbPartition) hasUnresolved() bool { + return len(p.stacktraces.unresolved)+ + len(p.r.locations.unresolved)+ + len(p.r.mappings.unresolved)+ + len(p.r.functions.unresolved)+ + len(p.r.strings.unresolved) > 0 +} + +func (p *symdbPartition) populateUnresolved(stacktraceIDs []uint32) error { // Filter out all stack traces that have been already // resolved and populate locations lookup table. - if err := r.resolveStacktraces(stacktraceIDs); err != nil { + if err := p.resolveStacktraces(stacktraceIDs); err != nil { return err } - if len(r.locations.unresolved) == 0 { + if len(p.r.locations.unresolved) == 0 { return nil } // Resolve functions and mappings for new locations. - unresolvedLocs := r.locations.iter() - locations := r.resolver.Locations(unresolvedLocs) + unresolvedLocs := p.r.locations.iter() + locations := p.resolver.Locations(unresolvedLocs) for locations.Err() == nil && locations.Next() { location := locations.At() - location.MappingId = r.mappings.tryLookup(location.MappingId) + location.MappingId = p.r.mappings.tryLookup(location.MappingId) for j, line := range location.Line { - location.Line[j].FunctionId = r.functions.tryLookup(line.FunctionId) + location.Line[j].FunctionId = p.r.functions.tryLookup(line.FunctionId) } unresolvedLocs.setValue(location) } @@ -653,101 +664,99 @@ func (r *stacktraceRewriter) populateUnresolved(stacktraceIDs []uint32) error { } // Resolve strings. - unresolvedMappings := r.mappings.iter() - mappings := r.resolver.Mappings(unresolvedMappings) + unresolvedMappings := p.r.mappings.iter() + mappings := p.resolver.Mappings(unresolvedMappings) for mappings.Err() == nil && mappings.Next() { mapping := mappings.At() - mapping.BuildId = r.strings.tryLookup(mapping.BuildId) - mapping.Filename = r.strings.tryLookup(mapping.Filename) + mapping.BuildId = p.r.strings.tryLookup(mapping.BuildId) + mapping.Filename = p.r.strings.tryLookup(mapping.Filename) unresolvedMappings.setValue(mapping) } if err := mappings.Err(); err != nil { return err } - unresolvedFunctions := r.functions.iter() - functions := r.resolver.Functions(unresolvedFunctions) + unresolvedFunctions := p.r.functions.iter() + functions := p.resolver.Functions(unresolvedFunctions) for functions.Err() == nil && functions.Next() { function := functions.At() - function.Name = r.strings.tryLookup(function.Name) - function.Filename = r.strings.tryLookup(function.Filename) - function.SystemName = r.strings.tryLookup(function.SystemName) + function.Name = p.r.strings.tryLookup(function.Name) + function.Filename = p.r.strings.tryLookup(function.Filename) + function.SystemName = p.r.strings.tryLookup(function.SystemName) unresolvedFunctions.setValue(function) } if err := functions.Err(); err != nil { return err } - unresolvedStrings := r.strings.iter() - strings := r.resolver.Strings(unresolvedStrings) + unresolvedStrings := p.r.strings.iter() + strings := p.resolver.Strings(unresolvedStrings) for strings.Err() == nil && strings.Next() { unresolvedStrings.setValue(strings.At()) } return strings.Err() } -func (r *stacktraceRewriter) appendRewrite(stacktraces []uint32) error { - for _, v := range r.strings.unresolved { - r.strings.storeResolved(v.rid, r.appender.AppendString(v.val)) +func (p *symdbPartition) appendRewrite(stacktraces []uint32) error { + for _, v := range p.r.strings.unresolved { + p.r.strings.storeResolved(v.rid, p.appender.AppendString(v.val)) } - for _, v := range r.functions.unresolved { + for _, v := range p.r.functions.unresolved { function := v.val - function.Name = r.strings.lookupResolved(function.Name) - function.Filename = r.strings.lookupResolved(function.Filename) - function.SystemName = r.strings.lookupResolved(function.SystemName) - r.functions.storeResolved(v.rid, r.appender.AppendFunction(function)) + function.Name = p.r.strings.lookupResolved(function.Name) + function.Filename = p.r.strings.lookupResolved(function.Filename) + function.SystemName = p.r.strings.lookupResolved(function.SystemName) + p.r.functions.storeResolved(v.rid, p.appender.AppendFunction(function)) } - for _, v := range r.mappings.unresolved { + for _, v := range p.r.mappings.unresolved { mapping := v.val - mapping.BuildId = r.strings.lookupResolved(mapping.BuildId) - mapping.Filename = r.strings.lookupResolved(mapping.Filename) - r.mappings.storeResolved(v.rid, r.appender.AppendMapping(mapping)) + mapping.BuildId = p.r.strings.lookupResolved(mapping.BuildId) + mapping.Filename = p.r.strings.lookupResolved(mapping.Filename) + p.r.mappings.storeResolved(v.rid, p.appender.AppendMapping(mapping)) } - for _, v := range r.locations.unresolved { + for _, v := range p.r.locations.unresolved { location := v.val - location.MappingId = r.mappings.lookupResolved(location.MappingId) + location.MappingId = p.r.mappings.lookupResolved(location.MappingId) for j, line := range location.Line { - location.Line[j].FunctionId = r.functions.lookupResolved(line.FunctionId) + location.Line[j].FunctionId = p.r.functions.lookupResolved(line.FunctionId) } - r.locations.storeResolved(v.rid, r.appender.AppendLocation(location)) + p.r.locations.storeResolved(v.rid, p.appender.AppendLocation(location)) } - src := r.stacktraces[r.partition] - for _, v := range src.unresolved { + for _, v := range p.stacktraces.unresolved { stacktrace := v.val for j, lid := range stacktrace { - stacktrace[j] = int32(r.locations.lookupResolved(uint32(lid))) + stacktrace[j] = int32(p.r.locations.lookupResolved(uint32(lid))) } - src.storeResolved(v.rid, r.appender.AppendStacktrace(stacktrace)) + p.stacktraces.storeResolved(v.rid, p.appender.AppendStacktrace(stacktrace)) } for i, v := range stacktraces { - stacktraces[i] = src.lookupResolved(v) + stacktraces[i] = p.stacktraces.lookupResolved(v) } - return r.appender.Flush() + return p.appender.Flush() } -func (r *stacktraceRewriter) resolveStacktraces(stacktraceIDs []uint32) error { - stacktraces := r.stacktraces[r.partition] +func (p *symdbPartition) resolveStacktraces(stacktraceIDs []uint32) error { for i, v := range stacktraceIDs { - stacktraceIDs[i] = stacktraces.tryLookup(v) + stacktraceIDs[i] = p.stacktraces.tryLookup(v) } - if len(stacktraces.unresolved) == 0 { + if len(p.stacktraces.unresolved) == 0 { return nil } // Gather and sort references to unresolved stacks. - stacktraces.initRefs() - sort.Sort(stacktraces) - grow(r.inserter.s, len(stacktraces.refs)) - for j, u := range stacktraces.refs { - r.inserter.s[j] = u.rid + p.stacktraces.initRefs() + sort.Sort(p.stacktraces) + grow(p.r.inserter.s, len(p.stacktraces.refs)) + for j, u := range p.stacktraces.refs { + p.r.inserter.s[j] = u.rid } - return r.resolver.ResolveStacktraces(context.TODO(), r.inserter, r.inserter.s) + return p.resolver.ResolveStacktraces(context.TODO(), p.r.inserter, p.r.inserter.s) } type stacktraceInserter struct { @@ -907,15 +916,3 @@ func grow[T any](s []T, n int) []T { } return s[:n] } - -// TODO(kolesnikovae): - -type symbolsWriter struct{} - -func newSymbolsWriter(dst string) (*symbolsWriter, error) { - return &symbolsWriter{}, nil -} - -func (w *symbolsWriter) SymbolsAppender(partition uint64) (SymbolsAppender, error) { - return nil, nil -} diff --git a/pkg/phlaredb/head.go b/pkg/phlaredb/head.go index 703a691fb7..fad28dd428 100644 --- a/pkg/phlaredb/head.go +++ b/pkg/phlaredb/head.go @@ -317,7 +317,7 @@ func (h *Head) convertSamples(_ context.Context, r *rewriter, stacktracePartitio r.locations.rewriteUint64(&stacktraces[idxSample].LocationIDs[i]) } } - appender := h.symbolDB.MappingWriter(stacktracePartition).StacktraceAppender() + appender := h.symbolDB.SymbolsAppender(stacktracePartition).StacktraceAppender() defer appender.Release() if cap(stacktracesIds) < len(stacktraces) { @@ -609,7 +609,7 @@ func (h *Head) resolveStacktraces(ctx context.Context, stacktracesByMapping stac sp.LogFields(otlog.String("msg", "building MergeProfilesStacktracesResult")) _ = stacktracesByMapping.ForEach( func(mapping uint64, stacktraceSamples stacktraceSampleMap) error { - mp, ok := h.symbolDB.MappingReader(mapping) + mp, ok := h.symbolDB.SymbolsResolver(mapping) if !ok { return nil } @@ -672,7 +672,7 @@ func (h *Head) resolvePprof(ctx context.Context, stacktracesByMapping profileSam // now add locationIDs and stacktraces _ = stacktracesByMapping.ForEach( func(mapping uint64, stacktraceSamples profileSampleMap) error { - mp, ok := h.symbolDB.MappingReader(mapping) + mp, ok := h.symbolDB.SymbolsResolver(mapping) if !ok { return nil } diff --git a/pkg/phlaredb/symdb/format.go b/pkg/phlaredb/symdb/format.go index 83ba737cb2..04924bbf4b 100644 --- a/pkg/phlaredb/symdb/format.go +++ b/pkg/phlaredb/symdb/format.go @@ -80,8 +80,8 @@ type IndexFile struct { // Version-specific parts. - // StacktraceChunkHeaders are sorted by mapping - // name and chunk index in ascending order. + // StacktraceChunkHeaders are sorted by + // partition and chunk index in ascending order. StacktraceChunkHeaders StacktraceChunkHeaders CRC uint32 @@ -201,29 +201,29 @@ func (h *StacktraceChunkHeaders) UnmarshalBinary(b []byte) error { return nil } -type stacktraceChunkHeadersByMappingAndIndex StacktraceChunkHeaders +type stacktraceChunkHeadersByPartitionAndIndex StacktraceChunkHeaders -func (h stacktraceChunkHeadersByMappingAndIndex) Len() int { +func (h stacktraceChunkHeadersByPartitionAndIndex) Len() int { return len(h.Entries) } -func (h stacktraceChunkHeadersByMappingAndIndex) Less(i, j int) bool { +func (h stacktraceChunkHeadersByPartitionAndIndex) Less(i, j int) bool { a, b := h.Entries[i], h.Entries[j] - if a.MappingName == b.MappingName { + if a.Partition == b.Partition { return a.ChunkIndex < b.ChunkIndex } - return a.MappingName < b.MappingName + return a.Partition < b.Partition } -func (h stacktraceChunkHeadersByMappingAndIndex) Swap(i, j int) { +func (h stacktraceChunkHeadersByPartitionAndIndex) Swap(i, j int) { h.Entries[i], h.Entries[j] = h.Entries[j], h.Entries[i] } type StacktraceChunkHeader struct { - Offset int64 // Relative to the mapping offset. + Offset int64 Size int64 - MappingName uint64 // MappingName the chunk refers to. + Partition uint64 ChunkIndex uint16 ChunkEncoding ChunkEncoding _ [5]byte // Reserved. @@ -247,7 +247,7 @@ const ( func (h *StacktraceChunkHeader) marshal(b []byte) { binary.BigEndian.PutUint64(b[0:8], uint64(h.Offset)) binary.BigEndian.PutUint64(b[8:16], uint64(h.Size)) - binary.BigEndian.PutUint64(b[16:24], h.MappingName) + binary.BigEndian.PutUint64(b[16:24], h.Partition) binary.BigEndian.PutUint16(b[24:26], h.ChunkIndex) b[27] = byte(h.ChunkEncoding) // 5 bytes reserved. @@ -262,7 +262,7 @@ func (h *StacktraceChunkHeader) marshal(b []byte) { func (h *StacktraceChunkHeader) unmarshal(b []byte) { h.Offset = int64(binary.BigEndian.Uint64(b[0:8])) h.Size = int64(binary.BigEndian.Uint64(b[8:16])) - h.MappingName = binary.BigEndian.Uint64(b[16:24]) + h.Partition = binary.BigEndian.Uint64(b[16:24]) h.ChunkIndex = binary.BigEndian.Uint16(b[24:26]) h.ChunkEncoding = ChunkEncoding(b[27]) // 5 bytes reserved. @@ -333,7 +333,7 @@ func (f *IndexFile) WriteTo(dst io.Writer) (n int64, err error) { return w.offset, fmt.Errorf("toc write: %w", err) } - sort.Sort(stacktraceChunkHeadersByMappingAndIndex(f.StacktraceChunkHeaders)) + sort.Sort(stacktraceChunkHeadersByPartitionAndIndex(f.StacktraceChunkHeaders)) sch, _ := f.StacktraceChunkHeaders.MarshalBinary() if _, err = w.Write(sch); err != nil { return w.offset, fmt.Errorf("stacktrace chunk headers: %w", err) diff --git a/pkg/phlaredb/symdb/interfaces.go b/pkg/phlaredb/symdb/interfaces.go index f997437470..1010bea37e 100644 --- a/pkg/phlaredb/symdb/interfaces.go +++ b/pkg/phlaredb/symdb/interfaces.go @@ -10,26 +10,23 @@ import ( // collection. https://github.com/google/pprof/blob/main/proto/README.md // // In the package, Mapping represents all the version of a binary. -// TODO(kolesnikovae): Rename mapping to Partition -type MappingWriter interface { - // StacktraceAppender provides exclusive write access - // to the stack traces of the mapping. - // - // StacktraceAppender.Release must be called in order - // to dispose the object and release the lock. - // Released resolver must not be used. +type SymbolsAppender interface { StacktraceAppender() StacktraceAppender } -type MappingReader interface { - // StacktraceResolver provides non-exclusive read - // access to the stack traces of the mapping. - // - // StacktraceResolver.Release must be called in order - // to dispose the object and release the lock. - // Released resolver must not be used. +type SymbolsResolver interface { StacktraceResolver() StacktraceResolver + WriteStats(*Stats) +} + +type Stats struct { + StacktracesTotal int + LocationsTotal int + MappingsTotal int + FunctionsTotal int + StringsTotal int + MaxStacktraceID int } type StacktraceAppender interface { diff --git a/pkg/phlaredb/symdb/mapping_memory.go b/pkg/phlaredb/symdb/partition_memory.go similarity index 76% rename from pkg/phlaredb/symdb/mapping_memory.go rename to pkg/phlaredb/symdb/partition_memory.go index 2182f7b37e..58c78deed7 100644 --- a/pkg/phlaredb/symdb/mapping_memory.go +++ b/pkg/phlaredb/symdb/partition_memory.go @@ -12,21 +12,19 @@ import ( ) var ( - _ MappingReader = (*inMemoryMapping)(nil) - _ MappingWriter = (*inMemoryMapping)(nil) + _ SymbolsResolver = (*inMemoryPartition)(nil) + _ SymbolsAppender = (*inMemoryPartition)(nil) _ StacktraceAppender = (*stacktraceAppender)(nil) _ StacktraceResolver = (*stacktraceResolverMemory)(nil) ) -type inMemoryMapping struct { +type inMemoryPartition struct { name uint64 maxNodesPerChunk uint32 // maxStackDepth uint32 - // Stack traces originating from the mapping (binary): - // their bottom frames (roots) refer to this mapping. stacktraceMutex sync.RWMutex stacktraceHashToID map[uint64]uint32 stacktraceChunks []*stacktraceChunk @@ -34,33 +32,44 @@ type inMemoryMapping struct { stacktraceChunkHeaders []StacktraceChunkHeader } -func (b *inMemoryMapping) StacktraceAppender() StacktraceAppender { +func (b *inMemoryPartition) StacktraceAppender() StacktraceAppender { b.stacktraceMutex.RLock() - // Assuming there is at least one chunk. - c := b.stacktraceChunks[len(b.stacktraceChunks)-1] + c := b.currentStacktraceChunk() b.stacktraceMutex.RUnlock() return &stacktraceAppender{ - mapping: b, - chunk: c, + partition: b, + chunk: c, } } -func (b *inMemoryMapping) StacktraceResolver() StacktraceResolver { +func (b *inMemoryPartition) StacktraceResolver() StacktraceResolver { return &stacktraceResolverMemory{ - mapping: b, + partition: b, } } +func (b *inMemoryPartition) WriteStats(s *Stats) { + b.stacktraceMutex.RLock() + c := b.currentStacktraceChunk() + s.MaxStacktraceID = int(c.stid + c.tree.len()) + s.StacktracesTotal = len(b.stacktraceHashToID) + b.stacktraceMutex.RUnlock() +} + // stacktraceChunkForInsert returns a chunk for insertion: // if the existing one has capacity, or a new one, if the former is full. // Must be called with the stracktraces mutex write lock held. -func (b *inMemoryMapping) stacktraceChunkForInsert(x int) *stacktraceChunk { - c := b.stacktraceChunks[len(b.stacktraceChunks)-1] +func (b *inMemoryPartition) stacktraceChunkForInsert(x int) *stacktraceChunk { + c := b.currentStacktraceChunk() if n := c.tree.len() + uint32(x); b.maxNodesPerChunk > 0 && n >= b.maxNodesPerChunk { + // Calculate number of stacks in the chunk. + s := uint32(len(b.stacktraceHashToID)) + c.stacks = s - c.stacks c = &stacktraceChunk{ - mapping: b, - tree: newStacktraceTree(defaultStacktraceTreeSize), - stid: c.stid + b.maxNodesPerChunk, + parition: b, + tree: newStacktraceTree(defaultStacktraceTreeSize), + stid: c.stid + b.maxNodesPerChunk, + stacks: s, } b.stacktraceChunks = append(b.stacktraceChunks, c) } @@ -69,17 +78,23 @@ func (b *inMemoryMapping) stacktraceChunkForInsert(x int) *stacktraceChunk { // stacktraceChunkForRead returns a chunk for reads. // Must be called with the stracktraces mutex read lock held. -func (b *inMemoryMapping) stacktraceChunkForRead(i int) (*stacktraceChunk, bool) { +func (b *inMemoryPartition) stacktraceChunkForRead(i int) (*stacktraceChunk, bool) { if i < len(b.stacktraceChunks) { return b.stacktraceChunks[i], true } return nil, false } +func (b *inMemoryPartition) currentStacktraceChunk() *stacktraceChunk { + // Assuming there is at least one chunk. + return b.stacktraceChunks[len(b.stacktraceChunks)-1] +} + type stacktraceChunk struct { - mapping *inMemoryMapping - stid uint32 // Initial stack trace ID. - tree *stacktraceTree + parition *inMemoryPartition + tree *stacktraceTree + stid uint32 // Initial stack trace ID. + stacks uint32 // } func (s *stacktraceChunk) WriteTo(dst io.Writer) (int64, error) { @@ -87,7 +102,7 @@ func (s *stacktraceChunk) WriteTo(dst io.Writer) (int64, error) { } type stacktraceAppender struct { - mapping *inMemoryMapping + partition *inMemoryPartition chunk *stacktraceChunk releaseOnce sync.Once } @@ -103,13 +118,13 @@ func (a *stacktraceAppender) AppendStacktrace(dst []uint32, s []*v1.Stacktrace) misses int ) - a.mapping.stacktraceMutex.RLock() + a.partition.stacktraceMutex.RLock() for i, x := range s { - if dst[i], found = a.mapping.stacktraceHashToID[hashLocations(x.LocationIDs)]; !found { + if dst[i], found = a.partition.stacktraceHashToID[hashLocations(x.LocationIDs)]; !found { misses++ } } - a.mapping.stacktraceMutex.RUnlock() + a.partition.stacktraceMutex.RUnlock() if misses == 0 { return } @@ -125,10 +140,10 @@ func (a *stacktraceAppender) AppendStacktrace(dst []uint32, s []*v1.Stacktrace) // Instead of inserting stacks one by one, it is better to // build a tree, and merge it to the existing one. - a.mapping.stacktraceMutex.Lock() - defer a.mapping.stacktraceMutex.Unlock() + a.partition.stacktraceMutex.Lock() + defer a.partition.stacktraceMutex.Unlock() - m := int(a.mapping.maxNodesPerChunk) + m := int(a.partition.maxNodesPerChunk) t, j := a.chunk.tree, a.chunk.stid for i, v := range dst[:len(s)] { if v != 0 { @@ -142,7 +157,7 @@ func (a *stacktraceAppender) AppendStacktrace(dst []uint32, s []*v1.Stacktrace) // If we're close to the max nodes limit and can // potentially exceed it, we take the next chunk, // even if there are some space. - a.chunk = a.mapping.stacktraceChunkForInsert(len(x)) + a.chunk = a.partition.stacktraceChunkForInsert(len(x)) t, j = a.chunk.tree, a.chunk.stid } @@ -150,7 +165,7 @@ func (a *stacktraceAppender) AppendStacktrace(dst []uint32, s []*v1.Stacktrace) // we don't need to check the map. id = t.insert(x) + j h := hashLocations(x) - a.mapping.stacktraceHashToID[h] = id + a.partition.stacktraceHashToID[h] = id dst[i] = id } } @@ -174,7 +189,7 @@ func hashLocations(s []uint64) uint64 { } type stacktraceResolverMemory struct { - mapping *inMemoryMapping + partition *inMemoryPartition } const defaultStacktraceDepth = 64 @@ -196,7 +211,7 @@ func (p *stacktraceLocationsPool) put(x []int32) { func (r *stacktraceResolverMemory) ResolveStacktraces(_ context.Context, dst StacktraceInserter, stacktraces []uint32) (err error) { // TODO(kolesnikovae): Add option to do resolve concurrently. // Depends on StacktraceInserter implementation. - for _, sr := range SplitStacktraces(stacktraces, r.mapping.maxNodesPerChunk) { + for _, sr := range SplitStacktraces(stacktraces, r.partition.maxNodesPerChunk) { if err = r.ResolveStacktracesChunk(dst, sr); err != nil { return err } @@ -212,10 +227,10 @@ func (r *stacktraceResolverMemory) ResolveStacktraces(_ context.Context, dst Sta // the options, the package provides. func (r *stacktraceResolverMemory) ResolveStacktracesChunk(dst StacktraceInserter, sr StacktracesRange) error { - r.mapping.stacktraceMutex.RLock() - c, found := r.mapping.stacktraceChunkForRead(int(sr.chunk)) + r.partition.stacktraceMutex.RLock() + c, found := r.partition.stacktraceChunkForRead(int(sr.chunk)) if !found { - r.mapping.stacktraceMutex.RUnlock() + r.partition.stacktraceMutex.RUnlock() return ErrInvalidStacktraceRange } t := stacktraceTree{nodes: c.tree.nodes} @@ -227,7 +242,7 @@ func (r *stacktraceResolverMemory) ResolveStacktracesChunk(dst StacktraceInserte // races when the slice grows: in the worst case, the underlying // capacity will be retained and thus not be eligible for GC during // the call. - r.mapping.stacktraceMutex.RUnlock() + r.partition.stacktraceMutex.RUnlock() s := stacktraceLocations.get() // Restore the original stacktrace ID. off := sr.offset() diff --git a/pkg/phlaredb/symdb/mapping_memory_test.go b/pkg/phlaredb/symdb/partition_memory_test.go similarity index 95% rename from pkg/phlaredb/symdb/mapping_memory_test.go rename to pkg/phlaredb/symdb/partition_memory_test.go index 316703f6d9..82fc9b0bb9 100644 --- a/pkg/phlaredb/symdb/mapping_memory_test.go +++ b/pkg/phlaredb/symdb/partition_memory_test.go @@ -22,7 +22,7 @@ func Test_StacktraceAppender_shards(t *testing.T) { }, }) - w := db.MappingWriter(0) + w := db.SymbolsAppender(0) a := w.StacktraceAppender() defer a.Release() @@ -48,8 +48,8 @@ func Test_StacktraceAppender_shards(t *testing.T) { }) assert.Equal(t, []uint32{18}, sids[:1]) - require.Len(t, db.mappings, 1) - m := db.mappings[0] + require.Len(t, db.partitions, 1) + m := db.partitions[0] require.Len(t, m.stacktraceChunks, 3) c1 := m.stacktraceChunks[0] @@ -67,7 +67,7 @@ func Test_StacktraceAppender_shards(t *testing.T) { t.Run("WithoutMaxStacktraceTreeNodesPerChunk", func(t *testing.T) { db := NewSymDB(new(Config)) - w := db.MappingWriter(0) + w := db.SymbolsAppender(0) a := w.StacktraceAppender() defer a.Release() @@ -81,8 +81,8 @@ func Test_StacktraceAppender_shards(t *testing.T) { }) assert.Equal(t, []uint32{3, 2, 4, 5, 6}, sids) - require.Len(t, db.mappings, 1) - m := db.mappings[0] + require.Len(t, db.partitions, 1) + m := db.partitions[0] require.Len(t, m.stacktraceChunks, 1) c1 := m.stacktraceChunks[0] @@ -166,7 +166,7 @@ func Test_StacktraceResolver_stacktraces_split(t *testing.T) { func Test_Stacktrace_append_existing(t *testing.T) { db := NewSymDB(new(Config)) - w := db.MappingWriter(0) + w := db.SymbolsAppender(0) a := w.StacktraceAppender() defer a.Release() sids := make([]uint32, 2) @@ -185,7 +185,7 @@ func Test_Stacktrace_append_existing(t *testing.T) { func Test_Stacktrace_append_empty(t *testing.T) { db := NewSymDB(new(Config)) - w := db.MappingWriter(0) + w := db.SymbolsAppender(0) a := w.StacktraceAppender() defer a.Release() @@ -205,7 +205,7 @@ func Test_Stacktraces_append_resolve(t *testing.T) { t.Run("single chunk", func(t *testing.T) { db := NewSymDB(new(Config)) - w := db.MappingWriter(0) + w := db.SymbolsAppender(0) a := w.StacktraceAppender() defer a.Release() @@ -218,7 +218,7 @@ func Test_Stacktraces_append_resolve(t *testing.T) { {LocationIDs: []uint64{5, 2, 1}}, }) - mr, _ := db.MappingReader(0) + mr, _ := db.SymbolsResolver(0) r := mr.StacktraceResolver() defer r.Release() dst := new(mockStacktraceInserter) @@ -237,7 +237,7 @@ func Test_Stacktraces_append_resolve(t *testing.T) { }, }) - w := db.MappingWriter(0) + w := db.SymbolsAppender(0) a := w.StacktraceAppender() defer a.Release() @@ -274,10 +274,10 @@ func Test_Stacktraces_append_resolve(t *testing.T) { */ sids := make([]uint32, len(stacktraces)) a.AppendStacktrace(sids, stacktraces) - require.Len(t, db.mappings[0].stacktraceChunks, 6) + require.Len(t, db.partitions[0].stacktraceChunks, 6) t.Run("adjacent shards at beginning", func(t *testing.T) { - mr, _ := db.MappingReader(0) + mr, _ := db.SymbolsResolver(0) r := mr.StacktraceResolver() defer r.Release() dst := new(mockStacktraceInserter) @@ -290,7 +290,7 @@ func Test_Stacktraces_append_resolve(t *testing.T) { }) t.Run("adjacent shards at end", func(t *testing.T) { - mr, _ := db.MappingReader(0) + mr, _ := db.SymbolsResolver(0) r := mr.StacktraceResolver() defer r.Release() dst := new(mockStacktraceInserter) @@ -303,7 +303,7 @@ func Test_Stacktraces_append_resolve(t *testing.T) { }) t.Run("non-adjacent shards", func(t *testing.T) { - mr, _ := db.MappingReader(0) + mr, _ := db.SymbolsResolver(0) r := mr.StacktraceResolver() defer r.Release() dst := new(mockStacktraceInserter) @@ -348,13 +348,13 @@ func Test_Stacktraces_memory_resolve_pprof(t *testing.T) { sids := make([]uint32, len(stacktraces)) db := NewSymDB(new(Config)) - mw := db.MappingWriter(0) + mw := db.SymbolsAppender(0) a := mw.StacktraceAppender() defer a.Release() a.AppendStacktrace(sids, stacktraces) - mr, ok := db.MappingReader(0) + mr, ok := db.SymbolsResolver(0) require.True(t, ok) r := mr.StacktraceResolver() defer r.Release() @@ -378,13 +378,13 @@ func Test_Stacktraces_memory_resolve_chunked(t *testing.T) { }, } db := NewSymDB(cfg) - mw := db.MappingWriter(0) + mw := db.SymbolsAppender(0) a := mw.StacktraceAppender() defer a.Release() a.AppendStacktrace(sids, stacktraces) - mr, ok := db.MappingReader(0) + mr, ok := db.SymbolsResolver(0) require.True(t, ok) r := mr.StacktraceResolver() defer r.Release() @@ -417,7 +417,7 @@ func Test_Stacktraces_memory_resolve_concurrency(t *testing.T) { // Allocate stacktrace IDs. sids := make([]uint32, len(stacktraces)) db := NewSymDB(cfg) - mw := db.MappingWriter(0) + mw := db.SymbolsAppender(0) a := mw.StacktraceAppender() a.AppendStacktrace(sids, stacktraces) a.Release() @@ -438,7 +438,7 @@ func Test_Stacktraces_memory_resolve_concurrency(t *testing.T) { go func() { defer wg.Done() - a := db.MappingWriter(0).StacktraceAppender() + a := db.SymbolsAppender(0).StacktraceAppender() defer a.Release() for j := 0; j < appends; j++ { @@ -452,7 +452,7 @@ func Test_Stacktraces_memory_resolve_concurrency(t *testing.T) { go func() { defer wg.Done() - mr, ok := db.MappingReader(0) + mr, ok := db.SymbolsResolver(0) if !ok { return } diff --git a/pkg/phlaredb/symdb/stacktrace_tree_test.go b/pkg/phlaredb/symdb/stacktrace_tree_test.go index b8e3d776ae..94f1796844 100644 --- a/pkg/phlaredb/symdb/stacktrace_tree_test.go +++ b/pkg/phlaredb/symdb/stacktrace_tree_test.go @@ -88,7 +88,7 @@ func Test_stacktrace_tree_encoding_group(t *testing.T) { } func Test_stacktrace_tree_encoding_rand(t *testing.T) { - // TODO: Fuzzing. With random data it's easy to hit overflow. + // TODO: Fuzzing. nodes := make([]node, 1<<20) for i := range nodes { nodes[i] = node{ diff --git a/pkg/phlaredb/symdb/mapping_reader_file.go b/pkg/phlaredb/symdb/symbols_reader_file.go similarity index 70% rename from pkg/phlaredb/symdb/mapping_reader_file.go rename to pkg/phlaredb/symdb/symbols_reader_file.go index 1aa2dbdf41..e569006797 100644 --- a/pkg/phlaredb/symdb/mapping_reader_file.go +++ b/pkg/phlaredb/symdb/symbols_reader_file.go @@ -15,23 +15,23 @@ import ( ) var ( - _ MappingReader = (*mappingFileReader)(nil) + _ SymbolsResolver = (*partitionFileReader)(nil) _ StacktraceResolver = (*stacktraceResolverFile)(nil) ) type Reader struct { bucket objstore.BucketReader - maxConcurrentChunkFetch int - chunkFetchBufferSize int + maxConcurrentChunks int + chunkFetchBufferSize int - idx IndexFile - mappings map[uint64]*mappingFileReader + idx IndexFile + partitions map[uint64]*partitionFileReader } const ( - defaultMaxConcurrentChunkFetch = 8 - defaultChunkFetchBufferSize = 4096 + defaultMaxConcurrentChunks = 1 + defaultChunkFetchBufferSize = 4096 ) // NOTE(kolesnikovae): @@ -41,16 +41,16 @@ const ( type ReaderConfig struct { BucketReader objstore.BucketReader - MaxConcurrentChunkFetch int - ChunkFetchBufferSize int + MaxConcurrentChunks int + ChunkFetchBufferSize int } func Open(ctx context.Context, b objstore.BucketReader) (*Reader, error) { r := Reader{ bucket: b, - maxConcurrentChunkFetch: defaultMaxConcurrentChunkFetch, - chunkFetchBufferSize: defaultChunkFetchBufferSize, + maxConcurrentChunks: defaultMaxConcurrentChunks, + chunkFetchBufferSize: defaultChunkFetchBufferSize, } if err := r.open(ctx); err != nil { return nil, err @@ -71,46 +71,55 @@ func (r *Reader) open(ctx context.Context) error { return err } // TODO(kolesnikovae): Load in a smarter way as headers are ordered. - r.mappings = make(map[uint64]*mappingFileReader, len(r.idx.StacktraceChunkHeaders.Entries)/3) + r.partitions = make(map[uint64]*partitionFileReader, len(r.idx.StacktraceChunkHeaders.Entries)/3) for _, h := range r.idx.StacktraceChunkHeaders.Entries { - r.mapping(h.MappingName).addStacktracesChunk(h) + r.partition(h.Partition).addStacktracesChunk(h) } return nil } -func (r *Reader) mapping(n uint64) *mappingFileReader { - if m, ok := r.mappings[n]; ok { +func (r *Reader) partition(n uint64) *partitionFileReader { + if m, ok := r.partitions[n]; ok { return m } - m := &mappingFileReader{reader: r} - r.mappings[n] = m + m := &partitionFileReader{reader: r} + r.partitions[n] = m return m } -func (r *Reader) MappingReader(mappingName uint64) (MappingReader, bool) { - m, ok := r.mappings[mappingName] +func (r *Reader) SymbolsResolver(partition uint64) (SymbolsResolver, bool) { + m, ok := r.partitions[partition] return m, ok } -type mappingFileReader struct { +type partitionFileReader struct { reader *Reader stacktraceChunks []*stacktraceChunkFileReader } -func (m *mappingFileReader) StacktraceResolver() StacktraceResolver { +func (m *partitionFileReader) StacktraceResolver() StacktraceResolver { return &stacktraceResolverFile{ - mapping: m, + partition: m, } } -func (m *mappingFileReader) addStacktracesChunk(h StacktraceChunkHeader) { +func (m *partitionFileReader) WriteStats(s *Stats) { + var nodes uint32 + for _, c := range m.stacktraceChunks { + s.StacktracesTotal += int(c.header.Stacktraces) + nodes += c.header.StacktraceNodes + } + s.MaxStacktraceID = int(nodes) +} + +func (m *partitionFileReader) addStacktracesChunk(h StacktraceChunkHeader) { m.stacktraceChunks = append(m.stacktraceChunks, &stacktraceChunkFileReader{ reader: m.reader, header: h, }) } -func (m *mappingFileReader) stacktraceChunkReader(i uint32) *stacktraceChunkFileReader { +func (m *partitionFileReader) stacktraceChunkReader(i uint32) *stacktraceChunkFileReader { if int(i) < len(m.stacktraceChunks) { return m.stacktraceChunks[i] } @@ -118,7 +127,7 @@ func (m *mappingFileReader) stacktraceChunkReader(i uint32) *stacktraceChunkFile } type stacktraceResolverFile struct { - mapping *mappingFileReader + partition *partitionFileReader } func (r *stacktraceResolverFile) Release() {} @@ -126,37 +135,21 @@ func (r *stacktraceResolverFile) Release() {} var ErrInvalidStacktraceRange = fmt.Errorf("invalid range: stack traces can't be resolved") func (r *stacktraceResolverFile) ResolveStacktraces(ctx context.Context, dst StacktraceInserter, s []uint32) error { - if len(r.mapping.stacktraceChunks) == 0 { + if len(r.partition.stacktraceChunks) == 0 { return ErrInvalidStacktraceRange } // First, we determine the chunks needed for the range. // All chunks in a block must have the same StacktraceMaxNodes. - sr := SplitStacktraces(s, r.mapping.stacktraceChunks[0].header.StacktraceMaxNodes) - - // TODO(kolesnikovae): - // Chunks are fetched concurrently, but inserted to dst sequentially, - // to avoid race condition on the implementation end: - // - Add maxConcurrentChunkResolve option that controls the behaviour. - // - Caching: already fetched chunks should be cached (serialized or not). + sr := SplitStacktraces(s, r.partition.stacktraceChunks[0].header.StacktraceMaxNodes) g, ctx := errgroup.WithContext(ctx) - g.SetLimit(r.mapping.reader.maxConcurrentChunkFetch) - rs := make([]*stacktracesResolve, len(sr)) - for i, c := range sr { - rs[i] = r.newResolve(ctx, dst, c) - g.Go(rs[i].fetch) - } - if err := g.Wait(); err != nil { - return err - } - - for _, cr := range rs { - cr.resolveStacktracesChunk(dst) - cr.release() + g.SetLimit(r.partition.reader.maxConcurrentChunks) + for _, c := range sr { + g.Go(r.newResolve(ctx, dst, c).do) } - return nil + return g.Wait() } func (r *stacktraceResolverFile) newResolve(ctx context.Context, dst StacktraceInserter, c StacktracesRange) *stacktracesResolve { @@ -179,8 +172,17 @@ type stacktracesResolve struct { c StacktracesRange } +func (r *stacktracesResolve) do() error { + if err := r.fetch(); err != nil { + return err + } + r.resolveStacktracesChunk(r.dst) + r.release() + return nil +} + func (r *stacktracesResolve) fetch() (err error) { - if r.cr = r.r.mapping.stacktraceChunkReader(r.c.chunk); r.cr == nil { + if r.cr = r.r.partition.stacktraceChunkReader(r.c.chunk); r.cr == nil { return ErrInvalidStacktraceRange } if r.t, err = r.cr.fetch(r.ctx); err != nil { diff --git a/pkg/phlaredb/symdb/mapping_reader_file_test.go b/pkg/phlaredb/symdb/symbols_reader_file_test.go similarity index 95% rename from pkg/phlaredb/symdb/mapping_reader_file_test.go rename to pkg/phlaredb/symdb/symbols_reader_file_test.go index af46f9bf10..03be33139e 100644 --- a/pkg/phlaredb/symdb/mapping_reader_file_test.go +++ b/pkg/phlaredb/symdb/symbols_reader_file_test.go @@ -19,7 +19,7 @@ func Test_Reader_Open(t *testing.T) { } db := NewSymDB(cfg) - w := db.MappingWriter(1) + w := db.SymbolsAppender(1) a := w.StacktraceAppender() sids := make([]uint32, 5) a.AppendStacktrace(sids, []*schemav1.Stacktrace{ @@ -37,7 +37,7 @@ func Test_Reader_Open(t *testing.T) { require.NoError(t, err) x, err := Open(context.Background(), b) require.NoError(t, err) - mr, ok := x.MappingReader(1) + mr, ok := x.SymbolsResolver(1) require.True(t, ok) dst := new(mockStacktraceInserter) diff --git a/pkg/phlaredb/symdb/mapping_writer_file.go b/pkg/phlaredb/symdb/symbols_writer_file.go similarity index 96% rename from pkg/phlaredb/symdb/mapping_writer_file.go rename to pkg/phlaredb/symdb/symbols_writer_file.go index d14e741080..d0f2336c2a 100644 --- a/pkg/phlaredb/symdb/mapping_writer_file.go +++ b/pkg/phlaredb/symdb/symbols_writer_file.go @@ -38,13 +38,13 @@ func (w *Writer) writeStacktraceChunk(ci int, c *stacktraceChunk) (err error) { h := StacktraceChunkHeader{ Offset: w.scd.w.offset, Size: 0, // Set later. - MappingName: c.mapping.name, + Partition: c.parition.name, ChunkIndex: uint16(ci), ChunkEncoding: ChunkEncodingGroupVarint, - Stacktraces: 0, // TODO + Stacktraces: c.stacks, StacktraceNodes: c.tree.len(), StacktraceMaxDepth: 0, // TODO - StacktraceMaxNodes: c.mapping.maxNodesPerChunk, + StacktraceMaxNodes: c.parition.maxNodesPerChunk, CRC: 0, // Set later. } crc := crc32.New(castagnoli) diff --git a/pkg/phlaredb/symdb/mapping_writer_file_test.go b/pkg/phlaredb/symdb/symbols_writer_file_test.go similarity index 84% rename from pkg/phlaredb/symdb/mapping_writer_file_test.go rename to pkg/phlaredb/symdb/symbols_writer_file_test.go index 5f031d3fe9..0e38cd0904 100644 --- a/pkg/phlaredb/symdb/mapping_writer_file_test.go +++ b/pkg/phlaredb/symdb/symbols_writer_file_test.go @@ -22,7 +22,7 @@ func Test_Writer_IndexFile(t *testing.T) { sids := make([]uint32, 5) - w := db.MappingWriter(0) + w := db.SymbolsAppender(0) a := w.StacktraceAppender() a.AppendStacktrace(sids, []*schemav1.Stacktrace{ {LocationIDs: []uint64{3, 2, 1}}, @@ -34,7 +34,7 @@ func Test_Writer_IndexFile(t *testing.T) { assert.Equal(t, []uint32{3, 2, 11, 16, 18}, sids) a.Release() - w = db.MappingWriter(1) + w = db.SymbolsAppender(1) a = w.StacktraceAppender() a.AppendStacktrace(sids, []*schemav1.Stacktrace{ {LocationIDs: []uint64{3, 2, 1}}, @@ -46,9 +46,9 @@ func Test_Writer_IndexFile(t *testing.T) { assert.Equal(t, []uint32{3, 2, 11, 16, 18}, sids) a.Release() - require.Len(t, db.mappings, 2) - require.Len(t, db.mappings[0].stacktraceChunks, 3) - require.Len(t, db.mappings[1].stacktraceChunks, 3) + require.Len(t, db.partitions, 2) + require.Len(t, db.partitions[0].stacktraceChunks, 3) + require.Len(t, db.partitions[1].stacktraceChunks, 3) require.NoError(t, db.Flush()) @@ -75,10 +75,10 @@ func Test_Writer_IndexFile(t *testing.T) { { Offset: 0, Size: 10, - MappingName: 0x0, + Partition: 0x0, ChunkIndex: 0x0, ChunkEncoding: 0x1, - Stacktraces: 0x0, + Stacktraces: 0x2, StacktraceNodes: 0x4, StacktraceMaxDepth: 0x0, StacktraceMaxNodes: 0x7, @@ -87,10 +87,10 @@ func Test_Writer_IndexFile(t *testing.T) { { Offset: 10, Size: 15, - MappingName: 0x0, + Partition: 0x0, ChunkIndex: 0x1, ChunkEncoding: 0x1, - Stacktraces: 0x0, + Stacktraces: 0x1, StacktraceNodes: 0x5, StacktraceMaxDepth: 0x0, StacktraceMaxNodes: 0x7, @@ -99,10 +99,10 @@ func Test_Writer_IndexFile(t *testing.T) { { Offset: 25, Size: 15, - MappingName: 0x0, + Partition: 0x0, ChunkIndex: 0x2, ChunkEncoding: 0x1, - Stacktraces: 0x0, + Stacktraces: 0x3, StacktraceNodes: 0x5, StacktraceMaxDepth: 0x0, StacktraceMaxNodes: 0x7, @@ -111,10 +111,10 @@ func Test_Writer_IndexFile(t *testing.T) { { Offset: 40, Size: 10, - MappingName: 0x1, + Partition: 0x1, ChunkIndex: 0x0, ChunkEncoding: 0x1, - Stacktraces: 0x0, + Stacktraces: 0x2, StacktraceNodes: 0x4, StacktraceMaxDepth: 0x0, StacktraceMaxNodes: 0x7, @@ -123,10 +123,10 @@ func Test_Writer_IndexFile(t *testing.T) { { Offset: 50, Size: 15, - MappingName: 0x1, + Partition: 0x1, ChunkIndex: 0x1, ChunkEncoding: 0x1, - Stacktraces: 0x0, + Stacktraces: 0x1, StacktraceNodes: 0x5, StacktraceMaxDepth: 0x0, StacktraceMaxNodes: 0x7, @@ -135,10 +135,10 @@ func Test_Writer_IndexFile(t *testing.T) { { Offset: 65, Size: 15, - MappingName: 0x1, + Partition: 0x1, ChunkIndex: 0x2, ChunkEncoding: 0x1, - Stacktraces: 0x0, + Stacktraces: 0x3, StacktraceNodes: 0x5, StacktraceMaxDepth: 0x0, StacktraceMaxNodes: 0x7, @@ -146,7 +146,7 @@ func Test_Writer_IndexFile(t *testing.T) { }, }, }, - CRC: 0x5bbecabf, + CRC: 0x6418eaed, } assert.Equal(t, expected, idx) diff --git a/pkg/phlaredb/symdb/symdb.go b/pkg/phlaredb/symdb/symdb.go index 9c6e8ba3ca..38a640dd15 100644 --- a/pkg/phlaredb/symdb/symdb.go +++ b/pkg/phlaredb/symdb/symdb.go @@ -12,8 +12,8 @@ type SymDB struct { writer *Writer stats stats - m sync.RWMutex - mappings map[uint64]*inMemoryMapping + m sync.RWMutex + partitions map[uint64]*inMemoryPartition wg sync.WaitGroup stop chan struct{} @@ -32,7 +32,7 @@ const statsUpdateInterval = 10 * time.Second type stats struct { memorySize atomic.Uint64 - mappings atomic.Uint32 + partitions atomic.Uint32 } func DefaultConfig() *Config { @@ -57,27 +57,27 @@ func NewSymDB(c *Config) *SymDB { c = DefaultConfig() } db := &SymDB{ - config: c, - writer: NewWriter(c.Dir), - mappings: make(map[uint64]*inMemoryMapping), - stop: make(chan struct{}), + config: c, + writer: NewWriter(c.Dir), + partitions: make(map[uint64]*inMemoryPartition), + stop: make(chan struct{}), } db.wg.Add(1) go db.updateStats() return db } -func (s *SymDB) MappingWriter(mappingName uint64) MappingWriter { - return s.mapping(mappingName) +func (s *SymDB) SymbolsAppender(partition uint64) SymbolsAppender { + return s.partition(partition) } -func (s *SymDB) MappingReader(mappingName uint64) (MappingReader, bool) { - return s.lookupMapping(mappingName) +func (s *SymDB) SymbolsResolver(partition uint64) (SymbolsResolver, bool) { + return s.lookupPartition(partition) } -func (s *SymDB) lookupMapping(mappingName uint64) (*inMemoryMapping, bool) { +func (s *SymDB) lookupPartition(partition uint64) (*inMemoryPartition, bool) { s.m.RLock() - p, ok := s.mappings[mappingName] + p, ok := s.partitions[partition] if ok { s.m.RUnlock() return p, true @@ -86,26 +86,26 @@ func (s *SymDB) lookupMapping(mappingName uint64) (*inMemoryMapping, bool) { return nil, false } -func (s *SymDB) mapping(mappingName uint64) *inMemoryMapping { - p, ok := s.lookupMapping(mappingName) +func (s *SymDB) partition(partition uint64) *inMemoryPartition { + p, ok := s.lookupPartition(partition) if ok { return p } s.m.Lock() - if p, ok = s.mappings[mappingName]; ok { + if p, ok = s.partitions[partition]; ok { s.m.Unlock() return p } - p = &inMemoryMapping{ - name: mappingName, + p = &inMemoryPartition{ + name: partition, maxNodesPerChunk: s.config.Stacktraces.MaxNodesPerChunk, stacktraceHashToID: make(map[uint64]uint32, defaultStacktraceTreeSize/2), } p.stacktraceChunks = append(p.stacktraceChunks, &stacktraceChunk{ - tree: newStacktraceTree(defaultStacktraceTreeSize), - mapping: p, + tree: newStacktraceTree(defaultStacktraceTreeSize), + parition: p, }) - s.mappings[mappingName] = p + s.partitions[partition] = p s.m.Unlock() return p } @@ -114,9 +114,9 @@ func (s *SymDB) Flush() error { close(s.stop) s.wg.Wait() s.m.RLock() - m := make([]*inMemoryMapping, len(s.mappings)) + m := make([]*inMemoryPartition, len(s.partitions)) var i int - for _, v := range s.mappings { + for _, v := range s.partitions { m[i] = v i++ } @@ -156,7 +156,7 @@ func (s *SymDB) updateStats() { return case <-t.C: s.m.RLock() - s.stats.mappings.Store(uint32(len(s.mappings))) + s.stats.partitions.Store(uint32(len(s.partitions))) s.stats.memorySize.Store(uint64(s.calculateMemoryFootprint())) s.m.RUnlock() } @@ -165,7 +165,7 @@ func (s *SymDB) updateStats() { // calculateMemoryFootprint estimates the memory footprint. func (s *SymDB) calculateMemoryFootprint() (v int) { - for _, m := range s.mappings { + for _, m := range s.partitions { m.stacktraceMutex.RLock() v += len(m.stacktraceChunkHeaders) * stacktraceChunkHeaderSize for _, c := range m.stacktraceChunks { From fbc64e2ab05143fb9d4cd06fce39a0816de3890f Mon Sep 17 00:00:00 2001 From: Anton Kolesnikov Date: Tue, 18 Jul 2023 22:29:04 +0800 Subject: [PATCH 22/35] Fix lookup table --- pkg/phlaredb/compact.go | 15 +++++++-------- pkg/phlaredb/compact_test.go | 2 +- 2 files changed, 8 insertions(+), 9 deletions(-) diff --git a/pkg/phlaredb/compact.go b/pkg/phlaredb/compact.go index 1df37e4d21..5c3a9f27d5 100644 --- a/pkg/phlaredb/compact.go +++ b/pkg/phlaredb/compact.go @@ -525,7 +525,7 @@ func (s *symbolsRewriter) Next() bool { } func (s *symbolsRewriter) loadStacktracesID(values []parquet.Value) { - grow(s.stacktraces, len(values)) + s.stacktraces = grow(s.stacktraces, len(values)) for i := range values { s.stacktraces[i] = values[i].Uint32() } @@ -751,7 +751,7 @@ func (p *symdbPartition) resolveStacktraces(stacktraceIDs []uint32) error { // Gather and sort references to unresolved stacks. p.stacktraces.initRefs() sort.Sort(p.stacktraces) - grow(p.r.inserter.s, len(p.stacktraces.refs)) + p.r.inserter.s = grow(p.r.inserter.s, len(p.stacktraces.refs)) for j, u := range p.stacktraces.refs { p.r.inserter.s[j] = u.rid } @@ -776,7 +776,7 @@ func (i *stacktraceInserter) InsertStacktrace(stacktrace uint32, locations []int if v.rid != stacktrace { panic("unexpected stack trace") } - grow(v.val, len(locations)) + v.val = grow(v.val, len(locations)) copy(v.val, locations) i.c++ } @@ -846,8 +846,7 @@ func (t *lookupTable[T]) newUnresolvedValue(rid uint32) uint32 { x := len(t.unresolved) if x < cap(t.unresolved) { // Try to reuse previously allocated value. - x++ - t.unresolved = t.unresolved[:x] + t.unresolved = t.unresolved[:x+1] t.unresolved[x].rid = rid } else { t.unresolved = append(t.unresolved, lookupTableValue[T]{rid: rid}) @@ -863,8 +862,8 @@ func (t *lookupTable[T]) referenceAt(x int) *lookupTableValue[T] { func (t *lookupTable[T]) storeResolved(rid, v uint32) { t.resolved[rid] = v + 1 } func (t *lookupTable[T]) lookupResolved(x uint32) uint32 { - if x&marker > 0 { // TODO: why? - return t.resolved[t.unresolved[x&markerMask-1].rid] - 1 + if x&marker > 0 { + return t.resolved[t.unresolved[x&markerMask].rid] - 1 } return x // Already resolved. } @@ -876,7 +875,7 @@ func (t *lookupTable[T]) iter() *lookupTableIterator[T] { } func (t *lookupTable[T]) initRefs() { - grow(t.refs, len(t.unresolved)) + t.refs = grow(t.refs, len(t.unresolved)) for i, v := range t.unresolved { t.refs[i] = lookupTableRef{rid: v.rid, uid: uint32(i)} } diff --git a/pkg/phlaredb/compact_test.go b/pkg/phlaredb/compact_test.go index 07f5b1580f..bc393ed8f5 100644 --- a/pkg/phlaredb/compact_test.go +++ b/pkg/phlaredb/compact_test.go @@ -306,7 +306,7 @@ func generateParquetFile(t *testing.T, path string) { func Test_lookupTable(t *testing.T) { // Given the source data set. - // Copy arbitrary subsets of those items to dst. + // Copy arbitrary subsets of items from src to dst. var dst []string src := []string{ "zero", From 2ea1e198eb2b4a047b90764b3539e95c34f296e4 Mon Sep 17 00:00:00 2001 From: Anton Kolesnikov Date: Wed, 19 Jul 2023 22:48:06 +0800 Subject: [PATCH 23/35] Add symbols writer --- pkg/phlaredb/block_querier.go | 10 +- pkg/phlaredb/block_symbols_writer.go | 108 +++++++++-- pkg/phlaredb/compact.go | 210 +++++++++++----------- pkg/phlaredb/compact_test.go | 6 +- pkg/phlaredb/head.go | 6 +- pkg/phlaredb/symdb/symbols_reader_file.go | 6 + 6 files changed, 218 insertions(+), 128 deletions(-) diff --git a/pkg/phlaredb/block_querier.go b/pkg/phlaredb/block_querier.go index 85f39c5e02..1eb07ddecb 100644 --- a/pkg/phlaredb/block_querier.go +++ b/pkg/phlaredb/block_querier.go @@ -449,8 +449,14 @@ func (b *singleBlockQuerier) Index() IndexReader { } func (b *singleBlockQuerier) Symbols() SymbolsReader { - // TODO(kolesnikovae) - return nil + return &inMemorySymbolsReader{ + // TODO + // strings: b.strings, + functions: b.functions, + locations: b.locations, + mappings: b.mappings, + stacktraces: b.stacktraces, + } } func (b *singleBlockQuerier) Meta() block.Meta { diff --git a/pkg/phlaredb/block_symbols_writer.go b/pkg/phlaredb/block_symbols_writer.go index cc0b156c02..612de41c80 100644 --- a/pkg/phlaredb/block_symbols_writer.go +++ b/pkg/phlaredb/block_symbols_writer.go @@ -1,6 +1,13 @@ package phlaredb -import schemav1 "github.com/grafana/phlare/pkg/phlaredb/schemas/v1" +import ( + "context" + "fmt" + "path/filepath" + + schemav1 "github.com/grafana/phlare/pkg/phlaredb/schemas/v1" + "github.com/grafana/phlare/pkg/phlaredb/symdb" +) // TODO(kolesnikovae): Refactor to symdb. @@ -9,21 +16,102 @@ type SymbolsWriter interface { } type SymbolsAppender interface { - AppendStacktrace([]int32) uint32 - AppendLocation(*schemav1.InMemoryLocation) uint32 - AppendMapping(*schemav1.InMemoryMapping) uint32 - AppendFunction(*schemav1.InMemoryFunction) uint32 - AppendString(string) uint32 + AppendStacktraces([]uint32, [][]int32) + AppendLocations([]uint32, []*schemav1.InMemoryLocation) + AppendMappings([]uint32, []*schemav1.InMemoryMapping) + AppendFunctions([]uint32, []*schemav1.InMemoryFunction) + AppendStrings([]uint32, []string) Flush() error } -type symbolsWriter struct{} +type symbolsWriter struct { + partitions map[uint64]*symbolsAppender + + locations deduplicatingSlice[*schemav1.InMemoryLocation, locationsKey, *locationsHelper, *schemav1.LocationPersister] + mappings deduplicatingSlice[*schemav1.InMemoryMapping, mappingsKey, *mappingsHelper, *schemav1.MappingPersister] + functions deduplicatingSlice[*schemav1.InMemoryFunction, functionsKey, *functionsHelper, *schemav1.FunctionPersister] + strings deduplicatingSlice[string, string, *stringsHelper, *schemav1.StringPersister] + tables []Table -func newSymbolsWriter(dst string) (*symbolsWriter, error) { - return &symbolsWriter{}, nil + symdb *symdb.SymDB +} + +func newSymbolsWriter(dst string, cfg *ParquetConfig) (*symbolsWriter, error) { + w := symbolsWriter{ + partitions: make(map[uint64]*symbolsAppender), + } + dir := filepath.Join(dst, symdb.DefaultDirName) + w.symdb = symdb.NewSymDB(symdb.DefaultConfig().WithDirectory(dir)) + w.tables = []Table{ + &w.locations, + &w.mappings, + &w.functions, + &w.strings, + } + for _, t := range w.tables { + if err := t.Init(dst, cfg, contextHeadMetrics(context.Background())); err != nil { + return nil, err + } + } + return &w, nil } func (w *symbolsWriter) SymbolsAppender(partition uint64) (SymbolsAppender, error) { - return nil, nil + p, ok := w.partitions[partition] + if !ok { + appender := w.symdb.SymbolsAppender(partition) + x := &symbolsAppender{ + stacktraces: appender.StacktraceAppender(), + writer: w, + } + w.partitions[partition] = x + p = x + } + return p, nil +} + +func (w *symbolsWriter) Close() error { + for _, t := range w.tables { + _, _, err := t.Flush(context.Background()) + if err != nil { + return fmt.Errorf("flushing table %s: %w", t.Name(), err) + } + } + if err := w.symdb.Flush(); err != nil { + return fmt.Errorf("flushing symbol database: %w", err) + } + return nil +} + +type symbolsAppender struct { + stacktraces symdb.StacktraceAppender + writer *symbolsWriter +} + +func (s symbolsAppender) AppendStacktraces(dst []uint32, stacktraces [][]int32) { + // TODO: []*schemav1.Stacktrace -> [][]uint32. + s.stacktraces.AppendStacktrace(dst, stacktraces) +} + +func (s symbolsAppender) AppendLocations(dst []uint32, locations []*schemav1.InMemoryLocation) { + // TODO: rewriter -> dst. + _ = s.writer.locations.ingest(context.Background(), locations, nil) +} + +func (s symbolsAppender) AppendMappings(dst []uint32, mappings []*schemav1.InMemoryMapping) { + _ = s.writer.mappings.ingest(context.Background(), mappings, nil) +} + +func (s symbolsAppender) AppendFunctions(dst []uint32, functions []*schemav1.InMemoryFunction) { + _ = s.writer.functions.ingest(context.Background(), functions, nil) +} + +func (s symbolsAppender) AppendStrings(dst []uint32, strings []string) { + _ = s.writer.strings.ingest(context.Background(), strings, nil) +} + +func (s symbolsAppender) Flush() error { + // TODO: Reset state (e.g. rewriter). + return nil } diff --git a/pkg/phlaredb/compact.go b/pkg/phlaredb/compact.go index 5c3a9f27d5..1565259b51 100644 --- a/pkg/phlaredb/compact.go +++ b/pkg/phlaredb/compact.go @@ -75,7 +75,7 @@ func Compact(ctx context.Context, src []BlockReader, dst string) (meta block.Met return block.Meta{}, err } profileWriter := newProfileWriter(profileFile) - symw, err := newSymbolsWriter(dst) + symw, err := newSymbolsWriter(blockPath, defaultParquetConfig) if err != nil { return block.Meta{}, err } @@ -535,18 +535,18 @@ type stacktraceRewriter struct { reader SymbolsReader writer SymbolsWriter - partitions map[uint64]*symdbPartition + partitions map[uint64]*symPartitionRewriter inserter *stacktraceInserter // Objects below have global addressing. - // TODO(kolesnikovae): Move to symdbPartition. + // TODO(kolesnikovae): Move to partition. locations *lookupTable[*schemav1.InMemoryLocation] mappings *lookupTable[*schemav1.InMemoryMapping] functions *lookupTable[*schemav1.InMemoryFunction] strings *lookupTable[string] } -type symdbPartition struct { +type symPartitionRewriter struct { name uint64 stats symdb.Stats // Stacktrace identifiers are only valid within the partition. @@ -564,9 +564,9 @@ func newStacktraceRewriter(r SymbolsReader, w SymbolsWriter) *stacktraceRewriter } } -func (r *stacktraceRewriter) init(partition uint64) (p *symdbPartition, err error) { +func (r *stacktraceRewriter) init(partition uint64) (p *symPartitionRewriter, err error) { if r.partitions == nil { - r.partitions = make(map[uint64]*symdbPartition) + r.partitions = make(map[uint64]*symPartitionRewriter) } if p, err = r.getOrCreatePartition(partition); err != nil { return nil, err @@ -585,21 +585,20 @@ func (r *stacktraceRewriter) init(partition uint64) (p *symdbPartition, err erro } r.inserter = &stacktraceInserter{ - slt: p.stacktraces, - llt: r.locations, - s: r.inserter.s, + stacktraces: p.stacktraces, + locations: r.locations, } return p, nil } -func (r *stacktraceRewriter) getOrCreatePartition(partition uint64) (_ *symdbPartition, err error) { +func (r *stacktraceRewriter) getOrCreatePartition(partition uint64) (_ *symPartitionRewriter, err error) { p, ok := r.partitions[partition] if ok { p.reset() return p, nil } - n := &symdbPartition{name: partition} + n := &symPartitionRewriter{name: partition} if n.resolver, err = r.reader.SymbolsResolver(partition); err != nil { return nil, err } @@ -626,11 +625,11 @@ func (r *stacktraceRewriter) rewriteStacktraces(partition uint64, stacktraces [] return nil } -func (p *symdbPartition) reset() { +func (p *symPartitionRewriter) reset() { p.stacktraces.reset() } -func (p *symdbPartition) hasUnresolved() bool { +func (p *symPartitionRewriter) hasUnresolved() bool { return len(p.stacktraces.unresolved)+ len(p.r.locations.unresolved)+ len(p.r.mappings.unresolved)+ @@ -638,7 +637,7 @@ func (p *symdbPartition) hasUnresolved() bool { len(p.r.strings.unresolved) > 0 } -func (p *symdbPartition) populateUnresolved(stacktraceIDs []uint32) error { +func (p *symPartitionRewriter) populateUnresolved(stacktraceIDs []uint32) error { // Filter out all stack traces that have been already // resolved and populate locations lookup table. if err := p.resolveStacktraces(stacktraceIDs); err != nil { @@ -651,7 +650,7 @@ func (p *symdbPartition) populateUnresolved(stacktraceIDs []uint32) error { // Resolve functions and mappings for new locations. unresolvedLocs := p.r.locations.iter() locations := p.resolver.Locations(unresolvedLocs) - for locations.Err() == nil && locations.Next() { + for locations.Next() { location := locations.At() location.MappingId = p.r.mappings.tryLookup(location.MappingId) for j, line := range location.Line { @@ -666,7 +665,7 @@ func (p *symdbPartition) populateUnresolved(stacktraceIDs []uint32) error { // Resolve strings. unresolvedMappings := p.r.mappings.iter() mappings := p.resolver.Mappings(unresolvedMappings) - for mappings.Err() == nil && mappings.Next() { + for mappings.Next() { mapping := mappings.At() mapping.BuildId = p.r.strings.tryLookup(mapping.BuildId) mapping.Filename = p.r.strings.tryLookup(mapping.Filename) @@ -678,7 +677,7 @@ func (p *symdbPartition) populateUnresolved(stacktraceIDs []uint32) error { unresolvedFunctions := p.r.functions.iter() functions := p.resolver.Functions(unresolvedFunctions) - for functions.Err() == nil && functions.Next() { + for functions.Next() { function := functions.At() function.Name = p.r.strings.tryLookup(function.Name) function.Filename = p.r.strings.tryLookup(function.Filename) @@ -691,48 +690,48 @@ func (p *symdbPartition) populateUnresolved(stacktraceIDs []uint32) error { unresolvedStrings := p.r.strings.iter() strings := p.resolver.Strings(unresolvedStrings) - for strings.Err() == nil && strings.Next() { + for strings.Next() { unresolvedStrings.setValue(strings.At()) } return strings.Err() } -func (p *symdbPartition) appendRewrite(stacktraces []uint32) error { - for _, v := range p.r.strings.unresolved { - p.r.strings.storeResolved(v.rid, p.appender.AppendString(v.val)) - } +func (p *symPartitionRewriter) appendRewrite(stacktraces []uint32) error { + p.appender.AppendStrings(p.r.strings.buf, p.r.strings.values) + p.r.strings.updateResolved() - for _, v := range p.r.functions.unresolved { - function := v.val - function.Name = p.r.strings.lookupResolved(function.Name) - function.Filename = p.r.strings.lookupResolved(function.Filename) - function.SystemName = p.r.strings.lookupResolved(function.SystemName) - p.r.functions.storeResolved(v.rid, p.appender.AppendFunction(function)) + for _, v := range p.r.functions.values { + v.Name = p.r.strings.lookupResolved(v.Name) + v.Filename = p.r.strings.lookupResolved(v.Filename) + v.SystemName = p.r.strings.lookupResolved(v.SystemName) } + p.appender.AppendFunctions(p.r.functions.buf, p.r.functions.values) + p.r.functions.updateResolved() - for _, v := range p.r.mappings.unresolved { - mapping := v.val - mapping.BuildId = p.r.strings.lookupResolved(mapping.BuildId) - mapping.Filename = p.r.strings.lookupResolved(mapping.Filename) - p.r.mappings.storeResolved(v.rid, p.appender.AppendMapping(mapping)) + for _, v := range p.r.mappings.values { + v.BuildId = p.r.strings.lookupResolved(v.BuildId) + v.Filename = p.r.strings.lookupResolved(v.Filename) } + p.appender.AppendMappings(p.r.mappings.buf, p.r.mappings.values) + p.r.mappings.updateResolved() - for _, v := range p.r.locations.unresolved { - location := v.val - location.MappingId = p.r.mappings.lookupResolved(location.MappingId) - for j, line := range location.Line { - location.Line[j].FunctionId = p.r.functions.lookupResolved(line.FunctionId) + for _, v := range p.r.locations.values { + v.MappingId = p.r.mappings.lookupResolved(v.MappingId) + for j, line := range v.Line { + v.Line[j].FunctionId = p.r.functions.lookupResolved(line.FunctionId) } - p.r.locations.storeResolved(v.rid, p.appender.AppendLocation(location)) } + p.appender.AppendLocations(p.r.locations.buf, p.r.locations.values) + p.r.locations.updateResolved() - for _, v := range p.stacktraces.unresolved { - stacktrace := v.val - for j, lid := range stacktrace { - stacktrace[j] = int32(p.r.locations.lookupResolved(uint32(lid))) + for _, v := range p.stacktraces.values { + for j, location := range v { + v[j] = int32(p.r.locations.lookupResolved(uint32(location))) } - p.stacktraces.storeResolved(v.rid, p.appender.AppendStacktrace(stacktrace)) } + p.appender.AppendStacktraces(p.stacktraces.buf, p.stacktraces.values) + p.stacktraces.updateResolved() + for i, v := range stacktraces { stacktraces[i] = p.stacktraces.lookupResolved(v) } @@ -740,44 +739,35 @@ func (p *symdbPartition) appendRewrite(stacktraces []uint32) error { return p.appender.Flush() } -func (p *symdbPartition) resolveStacktraces(stacktraceIDs []uint32) error { +func (p *symPartitionRewriter) resolveStacktraces(stacktraceIDs []uint32) error { for i, v := range stacktraceIDs { stacktraceIDs[i] = p.stacktraces.tryLookup(v) } if len(p.stacktraces.unresolved) == 0 { return nil } - - // Gather and sort references to unresolved stacks. - p.stacktraces.initRefs() - sort.Sort(p.stacktraces) - p.r.inserter.s = grow(p.r.inserter.s, len(p.stacktraces.refs)) - for j, u := range p.stacktraces.refs { - p.r.inserter.s[j] = u.rid - } - - return p.resolver.ResolveStacktraces(context.TODO(), p.r.inserter, p.r.inserter.s) + p.stacktraces.initSorted() + return p.resolver.ResolveStacktraces(context.TODO(), p.r.inserter, p.stacktraces.buf) } type stacktraceInserter struct { - slt *lookupTable[[]int32] - llt *lookupTable[*schemav1.InMemoryLocation] - s []uint32 - c int + stacktraces *lookupTable[[]int32] + locations *lookupTable[*schemav1.InMemoryLocation] + c int } func (i *stacktraceInserter) InsertStacktrace(stacktrace uint32, locations []int32) { // Resolve locations for new stack traces. for j, loc := range locations { - locations[j] = int32(i.llt.tryLookup(uint32(loc))) - } - // Update the unresolved value. - v := i.slt.referenceAt(i.c) - if v.rid != stacktrace { - panic("unexpected stack trace") - } - v.val = grow(v.val, len(locations)) - copy(v.val, locations) + locations[j] = int32(i.locations.tryLookup(uint32(loc))) + } + // stacktrace points to resolved which should + // be a marked pointer to unresolved value. + v := &i.stacktraces.values[stacktrace&markerMask] + n := grow(*v, len(locations)) + copy(n, locations) + // Preserve allocated capacity. + v = &n i.c++ } @@ -790,18 +780,9 @@ type lookupTable[T any] struct { // Index is source ID, and the value is the destination ID. // If destination ID is not known, the element is index to 'unresolved' (marked). resolved []uint32 - unresolved []lookupTableValue[T] - refs []lookupTableRef -} - -type lookupTableValue[T any] struct { - rid uint32 // Index to resolved. - val T -} - -type lookupTableRef struct { - rid uint32 // Index to resolved. - uid uint32 // Original index (unresolved). + unresolved []uint32 // Points to resolved. Index matches values. + values []T // Values are populated for unresolved items. + buf []uint32 // Sorted unresolved values. } func newLookupTable[T any](size int) *lookupTable[T] { @@ -823,7 +804,8 @@ func (t *lookupTable[T]) init(size int) { func (t *lookupTable[T]) reset() { t.unresolved = t.unresolved[:0] - t.refs = t.refs[:0] + t.values = t.values[:0] + t.buf = t.buf[:0] } // tryLookup looks up the value at x in resolved. @@ -837,53 +819,61 @@ func (t *lookupTable[T]) tryLookup(x uint32) uint32 { } return v - 1 // Already resolved. } - u := t.newUnresolvedValue(x) | marker + u := t.newUnresolved(x) | marker t.resolved[x] = u return u } -func (t *lookupTable[T]) newUnresolvedValue(rid uint32) uint32 { - x := len(t.unresolved) - if x < cap(t.unresolved) { +func (t *lookupTable[T]) newUnresolved(rid uint32) uint32 { + t.unresolved = append(t.unresolved, rid) + x := len(t.values) + if x < cap(t.values) { // Try to reuse previously allocated value. - t.unresolved = t.unresolved[:x+1] - t.unresolved[x].rid = rid + t.values = t.values[:x+1] } else { - t.unresolved = append(t.unresolved, lookupTableValue[T]{rid: rid}) + var v T + t.values = append(t.values, v) } return uint32(x) } -func (t *lookupTable[T]) referenceAt(x int) *lookupTableValue[T] { - u := t.refs[x].uid - return &t.unresolved[u] +func (t *lookupTable[T]) storeResolved(i int, rid uint32) { + // The index is incremented to avoid 0 because it is + // used as sentinel and indicates absence (resolved is + // a sparse slice initialized with the maximal expected + // size). Correspondingly, lookupResolved should + // decrement the index on read. + t.resolved[t.unresolved[i]] = rid + 1 } -func (t *lookupTable[T]) storeResolved(rid, v uint32) { t.resolved[rid] = v + 1 } - func (t *lookupTable[T]) lookupResolved(x uint32) uint32 { if x&marker > 0 { - return t.resolved[t.unresolved[x&markerMask].rid] - 1 + return t.resolved[t.unresolved[x&markerMask]] - 1 } return x // Already resolved. } -func (t *lookupTable[T]) iter() *lookupTableIterator[T] { - t.initRefs() - sort.Sort(t) - return &lookupTableIterator[T]{table: t} +// updateResolved loads indices from buf to resolved. +// It is expected that the order matches values. +func (t *lookupTable[T]) updateResolved() { + for i, rid := range t.unresolved { + t.resolved[rid] = t.buf[i] + } } -func (t *lookupTable[T]) initRefs() { - t.refs = grow(t.refs, len(t.unresolved)) - for i, v := range t.unresolved { - t.refs[i] = lookupTableRef{rid: v.rid, uid: uint32(i)} - } +func (t *lookupTable[T]) initSorted() { + // Gather and sort references to unresolved values. + t.buf = grow(t.buf, len(t.unresolved)) + copy(t.buf, t.unresolved) + sort.Slice(t.buf, func(i, j int) bool { + return t.buf[i] < t.buf[j] + }) } -func (t *lookupTable[T]) Len() int { return len(t.refs) } -func (t *lookupTable[T]) Less(i, j int) bool { return t.refs[i].rid < t.refs[j].rid } -func (t *lookupTable[T]) Swap(i, j int) { t.refs[i], t.refs[j] = t.refs[j], t.refs[i] } +func (t *lookupTable[T]) iter() *lookupTableIterator[T] { + t.initSorted() + return &lookupTableIterator[T]{table: t} +} type lookupTableIterator[T any] struct { table *lookupTable[T] @@ -891,18 +881,18 @@ type lookupTableIterator[T any] struct { } func (t *lookupTableIterator[T]) Next() bool { - return t.cur < uint32(len(t.table.refs)) + return t.cur < uint32(len(t.table.buf)) } func (t *lookupTableIterator[T]) At() uint32 { - x := t.table.refs[t.cur].rid + x := t.table.buf[t.cur] t.cur++ return x } func (t *lookupTableIterator[T]) setValue(v T) { - uid := t.table.refs[t.cur-1].uid - t.table.unresolved[uid].val = v + u := t.table.resolved[t.table.buf[t.cur-1]] + t.table.values[u&markerMask] = v } func (t *lookupTableIterator[T]) Close() error { return nil } diff --git a/pkg/phlaredb/compact_test.go b/pkg/phlaredb/compact_test.go index bc393ed8f5..c528121534 100644 --- a/pkg/phlaredb/compact_test.go +++ b/pkg/phlaredb/compact_test.go @@ -368,9 +368,9 @@ func Test_lookupTable(t *testing.T) { resolveAppend := func() { // Populate dst with the newly resolved values. // Note that order in dst does not have to match src. - for _, n := range l.unresolved { - l.storeResolved(n.rid, uint32(len(dst))) - dst = append(dst, n.val) + for i, v := range l.values { + l.storeResolved(i, uint32(len(dst))) + dst = append(dst, v) } } diff --git a/pkg/phlaredb/head.go b/pkg/phlaredb/head.go index fad28dd428..8845c38122 100644 --- a/pkg/phlaredb/head.go +++ b/pkg/phlaredb/head.go @@ -971,9 +971,9 @@ func (h *Head) flush(ctx context.Context) error { } // add total size symdb - symbDBFiles, error := h.SymDBFiles() - if error != nil { - return error + symbDBFiles, err := h.SymDBFiles() + if err != nil { + return err } for _, file := range symbDBFiles { diff --git a/pkg/phlaredb/symdb/symbols_reader_file.go b/pkg/phlaredb/symdb/symbols_reader_file.go index e569006797..01a590223a 100644 --- a/pkg/phlaredb/symdb/symbols_reader_file.go +++ b/pkg/phlaredb/symdb/symbols_reader_file.go @@ -92,6 +92,9 @@ func (r *Reader) SymbolsResolver(partition uint64) (SymbolsResolver, bool) { return m, ok } +// Load causes reader to load all contents into memory. +func (r *Reader) Load() error { panic("implement me") } + type partitionFileReader struct { reader *Reader stacktraceChunks []*stacktraceChunkFileReader @@ -135,6 +138,9 @@ func (r *stacktraceResolverFile) Release() {} var ErrInvalidStacktraceRange = fmt.Errorf("invalid range: stack traces can't be resolved") func (r *stacktraceResolverFile) ResolveStacktraces(ctx context.Context, dst StacktraceInserter, s []uint32) error { + if len(s) == 0 { + return nil + } if len(r.partition.stacktraceChunks) == 0 { return ErrInvalidStacktraceRange } From bdf0c5689037eb103c864ba554b1f2c5d4e96035 Mon Sep 17 00:00:00 2001 From: Anton Kolesnikov Date: Thu, 20 Jul 2023 12:51:05 +0800 Subject: [PATCH 24/35] Load symdb block files at compaction --- pkg/phlaredb/block_symbols_reader.go | 14 +++++++------- pkg/phlaredb/block_symbols_writer.go | 4 ++-- pkg/phlaredb/compact.go | 10 ++++++++++ pkg/phlaredb/head.go | 22 +++++++++++++++++++++- 4 files changed, 40 insertions(+), 10 deletions(-) diff --git a/pkg/phlaredb/block_symbols_reader.go b/pkg/phlaredb/block_symbols_reader.go index 6e6455a6c8..fdc9789b9d 100644 --- a/pkg/phlaredb/block_symbols_reader.go +++ b/pkg/phlaredb/block_symbols_reader.go @@ -3,9 +3,9 @@ package phlaredb import ( "context" - "github.com/grafana/phlare/pkg/iter" - schemav1 "github.com/grafana/phlare/pkg/phlaredb/schemas/v1" - "github.com/grafana/phlare/pkg/phlaredb/symdb" + "github.com/grafana/pyroscope/pkg/iter" + schemav1 "github.com/grafana/pyroscope/pkg/phlaredb/schemas/v1" + "github.com/grafana/pyroscope/pkg/phlaredb/symdb" ) // TODO(kolesnikovae): Refactor to symdb. @@ -75,8 +75,8 @@ func (s inMemorySymbolsResolver) Strings(i iter.Iterator[uint32]) iter.Iterator[ func (s inMemorySymbolsResolver) WriteStats(stats *symdb.Stats) { s.reader.stacktraces.WriteStats(s.partition, stats) - stats.LocationsTotal = int(s.reader.locations.NumRows()) - stats.MappingsTotal = int(s.reader.mappings.NumRows()) - stats.FunctionsTotal = int(s.reader.functions.NumRows()) - stats.StringsTotal = int(s.reader.strings.NumRows()) + stats.LocationsTotal = int(s.reader.locations.file.NumRows()) + stats.MappingsTotal = int(s.reader.mappings.file.NumRows()) + stats.FunctionsTotal = int(s.reader.functions.file.NumRows()) + stats.StringsTotal = int(s.reader.strings.file.NumRows()) } diff --git a/pkg/phlaredb/block_symbols_writer.go b/pkg/phlaredb/block_symbols_writer.go index 612de41c80..3ec8a510b1 100644 --- a/pkg/phlaredb/block_symbols_writer.go +++ b/pkg/phlaredb/block_symbols_writer.go @@ -5,8 +5,8 @@ import ( "fmt" "path/filepath" - schemav1 "github.com/grafana/phlare/pkg/phlaredb/schemas/v1" - "github.com/grafana/phlare/pkg/phlaredb/symdb" + schemav1 "github.com/grafana/pyroscope/pkg/phlaredb/schemas/v1" + "github.com/grafana/pyroscope/pkg/phlaredb/symdb" ) // TODO(kolesnikovae): Refactor to symdb. diff --git a/pkg/phlaredb/compact.go b/pkg/phlaredb/compact.go index 1565259b51..7f67f8b4ed 100644 --- a/pkg/phlaredb/compact.go +++ b/pkg/phlaredb/compact.go @@ -21,6 +21,7 @@ import ( phlareparquet "github.com/grafana/pyroscope/pkg/parquet" "github.com/grafana/pyroscope/pkg/phlaredb/block" schemav1 "github.com/grafana/pyroscope/pkg/phlaredb/schemas/v1" + "github.com/grafana/pyroscope/pkg/phlaredb/symdb" "github.com/grafana/pyroscope/pkg/phlaredb/tsdb/index" "github.com/grafana/pyroscope/pkg/util" "github.com/grafana/pyroscope/pkg/util/loser" @@ -124,6 +125,13 @@ func metaFilesFromDir(dir string) ([]block.File, error) { return err } if info.IsDir() { + if info.Name() == symdb.DefaultDirName { + f, err := symdbMetaFiles(dir) + if err != nil { + return err + } + files = append(files, f...) + } return nil } var f block.File @@ -138,6 +146,8 @@ func metaFilesFromDir(dir string) ([]block.File, error) { if err != nil { return err } + default: + return nil } f.RelPath, err = filepath.Rel(dir, path) if err != nil { diff --git a/pkg/phlaredb/head.go b/pkg/phlaredb/head.go index 8845c38122..d514edb46b 100644 --- a/pkg/phlaredb/head.go +++ b/pkg/phlaredb/head.go @@ -971,7 +971,7 @@ func (h *Head) flush(ctx context.Context) error { } // add total size symdb - symbDBFiles, err := h.SymDBFiles() + symbDBFiles, err := symdbMetaFiles(h.headPath) if err != nil { return err } @@ -1020,6 +1020,26 @@ func (h *Head) SymDBFiles() ([]block.File, error) { return result, nil } +func symdbMetaFiles(dir string) ([]block.File, error) { + files, err := os.ReadDir(filepath.Join(dir, symdb.DefaultDirName)) + if err != nil { + return nil, err + } + result := make([]block.File, len(files)) + for idx, f := range files { + if f.IsDir() { + continue + } + result[idx].RelPath = filepath.Join(symdb.DefaultDirName, f.Name()) + info, err := f.Info() + if err != nil { + return nil, err + } + result[idx].SizeBytes = uint64(info.Size()) + } + return result, nil +} + // Move moves the head directory to local blocks. The call is not thread-safe: // no concurrent reads and writes are allowed. // From fbab0c499bca2a4bdb43ce16b7094bcc36e4a5af Mon Sep 17 00:00:00 2001 From: Anton Kolesnikov Date: Thu, 20 Jul 2023 12:59:41 +0800 Subject: [PATCH 25/35] Fix meta samples stats --- pkg/phlaredb/compact.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pkg/phlaredb/compact.go b/pkg/phlaredb/compact.go index 7f67f8b4ed..e37ecb6884 100644 --- a/pkg/phlaredb/compact.go +++ b/pkg/phlaredb/compact.go @@ -110,7 +110,7 @@ func Compact(ctx context.Context, src []BlockReader, dst string) (meta block.Met meta.Files = metaFiles meta.Stats.NumProfiles = total meta.Stats.NumSeries = seriesRewriter.NumSeries() - meta.Stats.NumSamples = symbolsRewriter.NumSamples() + meta.Stats.NumSamples = symRewriter.NumSamples() if _, err := meta.WriteToFile(util.Logger, blockPath); err != nil { return block.Meta{}, err } From f77e4fc45c78322a3edee92314b4432badfc6989 Mon Sep 17 00:00:00 2001 From: Anton Kolesnikov Date: Thu, 20 Jul 2023 17:27:41 +0800 Subject: [PATCH 26/35] Add dedup slice append --- pkg/phlaredb/block_symbols_writer.go | 9 ++++---- pkg/phlaredb/deduplicating_slice.go | 32 ++++++++++++++++++++++++++++ 2 files changed, 36 insertions(+), 5 deletions(-) diff --git a/pkg/phlaredb/block_symbols_writer.go b/pkg/phlaredb/block_symbols_writer.go index 3ec8a510b1..8b66a7ab7a 100644 --- a/pkg/phlaredb/block_symbols_writer.go +++ b/pkg/phlaredb/block_symbols_writer.go @@ -95,20 +95,19 @@ func (s symbolsAppender) AppendStacktraces(dst []uint32, stacktraces [][]int32) } func (s symbolsAppender) AppendLocations(dst []uint32, locations []*schemav1.InMemoryLocation) { - // TODO: rewriter -> dst. - _ = s.writer.locations.ingest(context.Background(), locations, nil) + s.writer.locations.append(dst, locations) } func (s symbolsAppender) AppendMappings(dst []uint32, mappings []*schemav1.InMemoryMapping) { - _ = s.writer.mappings.ingest(context.Background(), mappings, nil) + s.writer.mappings.append(dst, mappings) } func (s symbolsAppender) AppendFunctions(dst []uint32, functions []*schemav1.InMemoryFunction) { - _ = s.writer.functions.ingest(context.Background(), functions, nil) + s.writer.functions.append(dst, functions) } func (s symbolsAppender) AppendStrings(dst []uint32, strings []string) { - _ = s.writer.strings.ingest(context.Background(), strings, nil) + s.writer.strings.append(dst, strings) } func (s symbolsAppender) Flush() error { diff --git a/pkg/phlaredb/deduplicating_slice.go b/pkg/phlaredb/deduplicating_slice.go index 3442fb9e6f..ec082f7ec1 100644 --- a/pkg/phlaredb/deduplicating_slice.go +++ b/pkg/phlaredb/deduplicating_slice.go @@ -227,3 +227,35 @@ func (s *deduplicatingSlice[M, K, H, P]) ingest(_ context.Context, elems []M, re return nil } + +func (s *deduplicatingSlice[M, K, H, P]) append(dst []uint32, elems []M) { + missing := int64SlicePool.Get()[:0] + s.lock.RLock() + for i, v := range elems { + k := s.helper.key(v) + if x, ok := s.lookup[k]; ok { + dst[i] = uint32(x) + } else { + missing = append(missing, int64(i)) + } + } + s.lock.RUnlock() + if len(missing) > 0 { + s.lock.RLock() + p := uint32(len(s.slice)) + for _, i := range missing { + k := s.helper.key(elems[i]) + x, ok := s.lookup[k] + if !ok { + dst[i] = uint32(x) + continue + } + s.slice = append(s.slice, s.helper.clone(elems[i])) + s.lookup[k] = int64(p) + dst[i] = p + p++ + } + s.lock.RUnlock() + } + int64SlicePool.Put(missing) +} From 744d6333c023c50e7239621ca5b91f0a75a9073c Mon Sep 17 00:00:00 2001 From: Anton Kolesnikov Date: Thu, 20 Jul 2023 18:41:16 +0800 Subject: [PATCH 27/35] Cleanup --- pkg/phlaredb/block_symbols_writer.go | 12 ++---------- pkg/phlaredb/compact.go | 2 +- 2 files changed, 3 insertions(+), 11 deletions(-) diff --git a/pkg/phlaredb/block_symbols_writer.go b/pkg/phlaredb/block_symbols_writer.go index 8b66a7ab7a..0abdc0f86e 100644 --- a/pkg/phlaredb/block_symbols_writer.go +++ b/pkg/phlaredb/block_symbols_writer.go @@ -16,13 +16,11 @@ type SymbolsWriter interface { } type SymbolsAppender interface { - AppendStacktraces([]uint32, [][]int32) + AppendStacktraces([]uint32, []*schemav1.Stacktrace) AppendLocations([]uint32, []*schemav1.InMemoryLocation) AppendMappings([]uint32, []*schemav1.InMemoryMapping) AppendFunctions([]uint32, []*schemav1.InMemoryFunction) AppendStrings([]uint32, []string) - - Flush() error } type symbolsWriter struct { @@ -89,8 +87,7 @@ type symbolsAppender struct { writer *symbolsWriter } -func (s symbolsAppender) AppendStacktraces(dst []uint32, stacktraces [][]int32) { - // TODO: []*schemav1.Stacktrace -> [][]uint32. +func (s symbolsAppender) AppendStacktraces(dst []uint32, stacktraces []*schemav1.Stacktrace) { s.stacktraces.AppendStacktrace(dst, stacktraces) } @@ -109,8 +106,3 @@ func (s symbolsAppender) AppendFunctions(dst []uint32, functions []*schemav1.InM func (s symbolsAppender) AppendStrings(dst []uint32, strings []string) { s.writer.strings.append(dst, strings) } - -func (s symbolsAppender) Flush() error { - // TODO: Reset state (e.g. rewriter). - return nil -} diff --git a/pkg/phlaredb/compact.go b/pkg/phlaredb/compact.go index e37ecb6884..21c4f2ebda 100644 --- a/pkg/phlaredb/compact.go +++ b/pkg/phlaredb/compact.go @@ -746,7 +746,7 @@ func (p *symPartitionRewriter) appendRewrite(stacktraces []uint32) error { stacktraces[i] = p.stacktraces.lookupResolved(v) } - return p.appender.Flush() + return nil } func (p *symPartitionRewriter) resolveStacktraces(stacktraceIDs []uint32) error { From 4e8ca3f6d8acb3196899302c77f37be0e5c1ab7f Mon Sep 17 00:00:00 2001 From: Anton Kolesnikov Date: Sun, 23 Jul 2023 19:19:36 +0800 Subject: [PATCH 28/35] Convert locations to stacktrace --- pkg/phlaredb/compact.go | 23 ++++++++++++++++++++++- 1 file changed, 22 insertions(+), 1 deletion(-) diff --git a/pkg/phlaredb/compact.go b/pkg/phlaredb/compact.go index 21c4f2ebda..9971e687bc 100644 --- a/pkg/phlaredb/compact.go +++ b/pkg/phlaredb/compact.go @@ -565,6 +565,10 @@ type symPartitionRewriter struct { appender SymbolsAppender r *stacktraceRewriter + + // FIXME(kolesnikovae): schemav1.Stacktrace should be just a uint32 slice: + // type Stacktrace []uint32 + current []*schemav1.Stacktrace } func newStacktraceRewriter(r SymbolsReader, w SymbolsWriter) *stacktraceRewriter { @@ -637,6 +641,7 @@ func (r *stacktraceRewriter) rewriteStacktraces(partition uint64, stacktraces [] func (p *symPartitionRewriter) reset() { p.stacktraces.reset() + p.current = p.current[:0] } func (p *symPartitionRewriter) hasUnresolved() bool { @@ -739,7 +744,7 @@ func (p *symPartitionRewriter) appendRewrite(stacktraces []uint32) error { v[j] = int32(p.r.locations.lookupResolved(uint32(location))) } } - p.appender.AppendStacktraces(p.stacktraces.buf, p.stacktraces.values) + p.appender.AppendStacktraces(p.stacktraces.buf, p.stacktracesFromResolvedValues()) p.stacktraces.updateResolved() for i, v := range stacktraces { @@ -760,6 +765,22 @@ func (p *symPartitionRewriter) resolveStacktraces(stacktraceIDs []uint32) error return p.resolver.ResolveStacktraces(context.TODO(), p.r.inserter, p.stacktraces.buf) } +func (p *symPartitionRewriter) stacktracesFromResolvedValues() []*schemav1.Stacktrace { + p.current = grow(p.current, len(p.stacktraces.values)) + for i, v := range p.stacktraces.values { + s := p.current[i] + if s == nil { + s = &schemav1.Stacktrace{LocationIDs: make([]uint64, len(v))} + p.current[i] = s + } + s.LocationIDs = grow(s.LocationIDs, len(v)) + for j, m := range v { + s.LocationIDs[j] = uint64(m) + } + } + return p.current +} + type stacktraceInserter struct { stacktraces *lookupTable[[]int32] locations *lookupTable[*schemav1.InMemoryLocation] From 718883a18f74e6a1e856e6740a02cefefe25a20f Mon Sep 17 00:00:00 2001 From: Anton Kolesnikov Date: Sun, 23 Jul 2023 20:25:07 +0800 Subject: [PATCH 29/35] Implement symdb Reader.Load --- pkg/phlaredb/block_querier.go | 23 +++++-- pkg/phlaredb/compact_test.go | 9 +-- pkg/phlaredb/symdb/symbols_reader_file.go | 73 +++++++++++++++++++---- 3 files changed, 84 insertions(+), 21 deletions(-) diff --git a/pkg/phlaredb/block_querier.go b/pkg/phlaredb/block_querier.go index 1eb07ddecb..20e96d52d3 100644 --- a/pkg/phlaredb/block_querier.go +++ b/pkg/phlaredb/block_querier.go @@ -305,8 +305,13 @@ type singleBlockQuerier struct { type StacktraceDB interface { Open(ctx context.Context) error Close() error - Resolve(ctx context.Context, partition uint64, locs symdb.StacktraceInserter, stacktraceIDs []uint32) error + + // Load the database into memory entirely. + // This method is used at compaction. + Load(context.Context) error WriteStats(partition uint64, s *symdb.Stats) + + Resolve(ctx context.Context, partition uint64, locs symdb.StacktraceInserter, stacktraceIDs []uint32) error } type stacktraceResolverV1 struct { @@ -327,8 +332,8 @@ func (r *stacktraceResolverV1) Resolve(ctx context.Context, _ uint64, locs symdb defer stacktraces.Close() t := make([]int32, 0, 64) for stacktraces.Next() { - t = t[:0] s := stacktraces.At() + t = grow(t, len(s.Values)) for i, v := range s.Values { t[i] = v.Int32() } @@ -342,6 +347,13 @@ func (r *stacktraceResolverV1) WriteStats(_ uint64, s *symdb.Stats) { s.MaxStacktraceID = s.StacktracesTotal } +func (r *stacktraceResolverV1) Load(context.Context) error { + // FIXME(kolesnikovae): Loading all stacktraces from parquet file + // into memory is likely a bad choice. Instead we could convert + // it to symdb first. + return nil +} + type stacktraceResolverV2 struct { reader *symdb.Reader bucketReader phlareobj.Bucket @@ -370,6 +382,10 @@ func (r *stacktraceResolverV2) Resolve(ctx context.Context, partition uint64, lo return resolver.ResolveStacktraces(ctx, locs, stacktraceIDs) } +func (r *stacktraceResolverV2) Load(ctx context.Context) error { + return r.reader.Load(ctx) +} + func (r *stacktraceResolverV2) WriteStats(partition uint64, s *symdb.Stats) { mr, ok := r.reader.SymbolsResolver(partition) if !ok { @@ -450,8 +466,7 @@ func (b *singleBlockQuerier) Index() IndexReader { func (b *singleBlockQuerier) Symbols() SymbolsReader { return &inMemorySymbolsReader{ - // TODO - // strings: b.strings, + strings: b.strings, functions: b.functions, locations: b.locations, mappings: b.mappings, diff --git a/pkg/phlaredb/compact_test.go b/pkg/phlaredb/compact_test.go index c528121534..0c9bdd4788 100644 --- a/pkg/phlaredb/compact_test.go +++ b/pkg/phlaredb/compact_test.go @@ -4,6 +4,7 @@ import ( "context" "fmt" "net/http" + _ "net/http/pprof" "os" "path/filepath" "sort" @@ -11,8 +12,6 @@ import ( "testing" "time" - _ "net/http/pprof" - "github.com/prometheus/common/model" "github.com/prometheus/prometheus/storage" "github.com/segmentio/parquet-go" @@ -119,9 +118,11 @@ func testCompact(t *testing.T, metas []*block.Meta, bkt phlareobj.Bucket, dst st "numSamples", m.Stats.NumSamples) b := NewSingleBlockQuerierFromMeta(ctx, bkt, m) g.Go(func() error { - return b.Open(ctx) + if err := b.Open(ctx); err != nil { + return err + } + return b.stacktraces.Load(ctx) }) - src = append(src, b) } diff --git a/pkg/phlaredb/symdb/symbols_reader_file.go b/pkg/phlaredb/symdb/symbols_reader_file.go index 01a590223a..c2b8167be0 100644 --- a/pkg/phlaredb/symdb/symbols_reader_file.go +++ b/pkg/phlaredb/symdb/symbols_reader_file.go @@ -6,6 +6,7 @@ import ( "fmt" "hash/crc32" "io" + "sort" "sync" "github.com/grafana/dskit/multierror" @@ -93,7 +94,45 @@ func (r *Reader) SymbolsResolver(partition uint64) (SymbolsResolver, bool) { } // Load causes reader to load all contents into memory. -func (r *Reader) Load() error { panic("implement me") } +func (r *Reader) Load(ctx context.Context) error { + partitions := make([]*partitionFileReader, len(r.partitions)) + var i int + for _, v := range r.partitions { + partitions[i] = v + i++ + } + sort.Slice(partitions, func(i, j int) bool { + return partitions[i].stacktraceChunks[0].header.Offset < + partitions[j].stacktraceChunks[0].header.Offset + }) + + offset := partitions[0].stacktraceChunks[0].header.Offset + var size int64 + for i = range partitions { + for _, c := range partitions[i].stacktraceChunks { + size += c.header.Size + } + } + + rc, err := r.bucket.GetRange(ctx, StacktracesFileName, offset, size) + if err != nil { + return err + } + defer func() { + err = multierror.New(err, rc.Close()).Err() + }() + + buf := bufio.NewReaderSize(rc, r.chunkFetchBufferSize) + for _, p := range partitions { + for _, c := range p.stacktraceChunks { + if err = c.readFrom(io.LimitReader(buf, c.header.Size)); err != nil { + return err + } + } + } + + return nil +} type partitionFileReader struct { reader *Reader @@ -215,6 +254,10 @@ type stacktraceChunkFileReader struct { header StacktraceChunkHeader m sync.Mutex tree *parentPointerTree + // Indicates that the chunk has been loaded into + // memory with Load call and should not be released + // until the block is closed. + loaded bool } func (c *stacktraceChunkFileReader) fetch(ctx context.Context) (_ *parentPointerTree, err error) { @@ -223,7 +266,6 @@ func (c *stacktraceChunkFileReader) fetch(ctx context.Context) (_ *parentPointer if c.tree != nil { return c.tree, nil } - rc, err := c.reader.bucket.GetRange(ctx, StacktracesFileName, c.header.Offset, c.header.Size) if err != nil { return nil, err @@ -231,36 +273,41 @@ func (c *stacktraceChunkFileReader) fetch(ctx context.Context) (_ *parentPointer defer func() { err = multierror.New(err, rc.Close()).Err() }() + // Consider pooling the buffer. + buf := bufio.NewReaderSize(rc, c.reader.chunkFetchBufferSize) + if err = c.readFrom(buf); err != nil { + return nil, err + } + return c.tree, nil +} +func (c *stacktraceChunkFileReader) readFrom(r io.Reader) error { // NOTE(kolesnikovae): Pool of node chunks could reduce // the alloc size, but it may affect memory locality. // Although, properly aligned chunks of, say, 1-4K nodes // which is 8-32KiB respectively, should not make things // much worse than they are. Worth experimenting. t := newParentPointerTree(c.header.StacktraceNodes) - // We unmarshal the tree speculatively, before validating // the checksum. Even random bytes can be unmarshalled to // a tree not causing any errors, therefore it is vital // to verify the correctness of the data. crc := crc32.New(castagnoli) - tee := io.TeeReader(rc, crc) - - // Consider pooling the buffer. - buf := bufio.NewReaderSize(tee, c.reader.chunkFetchBufferSize) - if _, err = t.ReadFrom(buf); err != nil { - return nil, fmt.Errorf("failed to unmarshal stack treaces: %w", err) + tee := io.TeeReader(r, crc) + if _, err := t.ReadFrom(tee); err != nil { + return fmt.Errorf("failed to unmarshal stack treaces: %w", err) } if c.header.CRC != crc.Sum32() { - return nil, ErrInvalidCRC + return ErrInvalidCRC } - c.tree = t - return t, nil + return nil } func (c *stacktraceChunkFileReader) reset() { c.m.Lock() - c.tree = nil + if !c.loaded { + c.tree = nil + } c.m.Unlock() } From 113e69da0567f0d4b182c3e9554ebf19ce233438 Mon Sep 17 00:00:00 2001 From: Anton Kolesnikov Date: Sun, 23 Jul 2023 20:50:01 +0800 Subject: [PATCH 30/35] Fix stacktrace inserter --- pkg/phlaredb/compact.go | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/pkg/phlaredb/compact.go b/pkg/phlaredb/compact.go index 9971e687bc..eaa491e1ef 100644 --- a/pkg/phlaredb/compact.go +++ b/pkg/phlaredb/compact.go @@ -784,7 +784,6 @@ func (p *symPartitionRewriter) stacktracesFromResolvedValues() []*schemav1.Stack type stacktraceInserter struct { stacktraces *lookupTable[[]int32] locations *lookupTable[*schemav1.InMemoryLocation] - c int } func (i *stacktraceInserter) InsertStacktrace(stacktrace uint32, locations []int32) { @@ -794,12 +793,12 @@ func (i *stacktraceInserter) InsertStacktrace(stacktrace uint32, locations []int } // stacktrace points to resolved which should // be a marked pointer to unresolved value. - v := &i.stacktraces.values[stacktrace&markerMask] + idx := i.stacktraces.resolved[stacktrace] & markerMask + v := &i.stacktraces.values[idx] n := grow(*v, len(locations)) copy(n, locations) // Preserve allocated capacity. - v = &n - i.c++ + i.stacktraces.values[idx] = n } const ( From e7dce94d51e8976c6fa3cd7b979db41b61c7c91a Mon Sep 17 00:00:00 2001 From: Anton Kolesnikov Date: Sun, 23 Jul 2023 21:31:44 +0800 Subject: [PATCH 31/35] Fix symdb meta --- pkg/phlaredb/compact.go | 9 --------- 1 file changed, 9 deletions(-) diff --git a/pkg/phlaredb/compact.go b/pkg/phlaredb/compact.go index eaa491e1ef..2349fba809 100644 --- a/pkg/phlaredb/compact.go +++ b/pkg/phlaredb/compact.go @@ -125,13 +125,6 @@ func metaFilesFromDir(dir string) ([]block.File, error) { return err } if info.IsDir() { - if info.Name() == symdb.DefaultDirName { - f, err := symdbMetaFiles(dir) - if err != nil { - return err - } - files = append(files, f...) - } return nil } var f block.File @@ -146,8 +139,6 @@ func metaFilesFromDir(dir string) ([]block.File, error) { if err != nil { return err } - default: - return nil } f.RelPath, err = filepath.Rel(dir, path) if err != nil { From 141ebc1c28084bef982ae0fa56f7834d9aadd968 Mon Sep 17 00:00:00 2001 From: Anton Kolesnikov Date: Sun, 23 Jul 2023 21:33:30 +0800 Subject: [PATCH 32/35] Fix lint issues --- pkg/phlaredb/block_querier.go | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/pkg/phlaredb/block_querier.go b/pkg/phlaredb/block_querier.go index 20e96d52d3..3df2f2d26d 100644 --- a/pkg/phlaredb/block_querier.go +++ b/pkg/phlaredb/block_querier.go @@ -388,11 +388,9 @@ func (r *stacktraceResolverV2) Load(ctx context.Context) error { func (r *stacktraceResolverV2) WriteStats(partition uint64, s *symdb.Stats) { mr, ok := r.reader.SymbolsResolver(partition) - if !ok { - return + if ok { + mr.WriteStats(s) } - mr.WriteStats(s) - return } func NewSingleBlockQuerierFromMeta(phlarectx context.Context, bucketReader phlareobj.Bucket, meta *block.Meta) *singleBlockQuerier { From 26e163538559d2a01f4cccba0188c1e4c57b2734 Mon Sep 17 00:00:00 2001 From: Anton Kolesnikov Date: Mon, 24 Jul 2023 14:28:36 +0800 Subject: [PATCH 33/35] Fix symbols rewriter integration --- pkg/phlaredb/block_querier.go | 3 +++ pkg/phlaredb/block_symbols_reader.go | 8 ++++---- pkg/phlaredb/block_symbols_writer.go | 3 +++ pkg/phlaredb/compact.go | 19 +++++++++++++------ pkg/phlaredb/deduplicating_slice.go | 8 +++++--- 5 files changed, 28 insertions(+), 13 deletions(-) diff --git a/pkg/phlaredb/block_querier.go b/pkg/phlaredb/block_querier.go index 3df2f2d26d..009771dfb1 100644 --- a/pkg/phlaredb/block_querier.go +++ b/pkg/phlaredb/block_querier.go @@ -464,6 +464,8 @@ func (b *singleBlockQuerier) Index() IndexReader { func (b *singleBlockQuerier) Symbols() SymbolsReader { return &inMemorySymbolsReader{ + partitions: make(map[uint64]*inMemorySymbolsResolver), + strings: b.strings, functions: b.functions, locations: b.locations, @@ -1224,6 +1226,7 @@ type inMemoryparquetReader[M Models, P schemav1.Persister[M]] struct { persister P file *parquet.File size int64 + numRows int64 reader phlareobj.ReaderAtCloser cache []M } diff --git a/pkg/phlaredb/block_symbols_reader.go b/pkg/phlaredb/block_symbols_reader.go index fdc9789b9d..e8399e1342 100644 --- a/pkg/phlaredb/block_symbols_reader.go +++ b/pkg/phlaredb/block_symbols_reader.go @@ -75,8 +75,8 @@ func (s inMemorySymbolsResolver) Strings(i iter.Iterator[uint32]) iter.Iterator[ func (s inMemorySymbolsResolver) WriteStats(stats *symdb.Stats) { s.reader.stacktraces.WriteStats(s.partition, stats) - stats.LocationsTotal = int(s.reader.locations.file.NumRows()) - stats.MappingsTotal = int(s.reader.mappings.file.NumRows()) - stats.FunctionsTotal = int(s.reader.functions.file.NumRows()) - stats.StringsTotal = int(s.reader.strings.file.NumRows()) + stats.LocationsTotal = len(s.reader.locations.cache) + stats.MappingsTotal = len(s.reader.mappings.cache) + stats.FunctionsTotal = len(s.reader.functions.cache) + stats.StringsTotal = len(s.reader.strings.cache) } diff --git a/pkg/phlaredb/block_symbols_writer.go b/pkg/phlaredb/block_symbols_writer.go index 0abdc0f86e..43fd1a09a7 100644 --- a/pkg/phlaredb/block_symbols_writer.go +++ b/pkg/phlaredb/block_symbols_writer.go @@ -75,6 +75,9 @@ func (w *symbolsWriter) Close() error { if err != nil { return fmt.Errorf("flushing table %s: %w", t.Name(), err) } + if err = t.Close(); err != nil { + return fmt.Errorf("closing table %s: %w", t.Name(), err) + } } if err := w.symdb.Flush(); err != nil { return fmt.Errorf("flushing symbol database: %w", err) diff --git a/pkg/phlaredb/compact.go b/pkg/phlaredb/compact.go index 2349fba809..62dc48a1af 100644 --- a/pkg/phlaredb/compact.go +++ b/pkg/phlaredb/compact.go @@ -94,12 +94,19 @@ func Compact(ctx context.Context, src []BlockReader, dst string) (meta block.Met return block.Meta{}, err } + if err = symRewriter.Close(); err != nil { + return block.Meta{}, err + } + if err = symw.Close(); err != nil { + return block.Meta{}, err + } + // flush the index file. - if err := indexw.Close(); err != nil { + if err = indexw.Close(); err != nil { return block.Meta{}, err } - if err := profileWriter.Close(); err != nil { + if err = profileWriter.Close(); err != nil { return block.Meta{}, err } @@ -183,8 +190,8 @@ func parquetMetaFile(filePath string, size int64) (block.File, error) { func compactMetas(src []block.Meta) block.Meta { meta := block.NewMeta() highestCompactionLevel := 0 - ulids := make([]ulid.ULID, len(src)) - parents := make([]tsdb.BlockDesc, len(src)) + ulids := make([]ulid.ULID, 0, len(src)) + parents := make([]tsdb.BlockDesc, 0, len(src)) minTime, maxTime := model.Latest, model.Earliest labels := make(map[string]string) for _, b := range src { @@ -603,7 +610,7 @@ func (r *stacktraceRewriter) getOrCreatePartition(partition uint64) (_ *symParti p.reset() return p, nil } - n := &symPartitionRewriter{name: partition} + n := &symPartitionRewriter{r: r, name: partition} if n.resolver, err = r.reader.SymbolsResolver(partition); err != nil { return nil, err } @@ -878,7 +885,7 @@ func (t *lookupTable[T]) lookupResolved(x uint32) uint32 { // It is expected that the order matches values. func (t *lookupTable[T]) updateResolved() { for i, rid := range t.unresolved { - t.resolved[rid] = t.buf[i] + t.resolved[rid] = t.buf[i] + 1 } } diff --git a/pkg/phlaredb/deduplicating_slice.go b/pkg/phlaredb/deduplicating_slice.go index ec082f7ec1..cdeaf15ec0 100644 --- a/pkg/phlaredb/deduplicating_slice.go +++ b/pkg/phlaredb/deduplicating_slice.go @@ -244,13 +244,15 @@ func (s *deduplicatingSlice[M, K, H, P]) append(dst []uint32, elems []M) { s.lock.RLock() p := uint32(len(s.slice)) for _, i := range missing { - k := s.helper.key(elems[i]) + e := elems[i] + k := s.helper.key(e) x, ok := s.lookup[k] - if !ok { + if ok { dst[i] = uint32(x) continue } - s.slice = append(s.slice, s.helper.clone(elems[i])) + s.size.Add(s.helper.size(e)) + s.slice = append(s.slice, s.helper.clone(e)) s.lookup[k] = int64(p) dst[i] = p p++ From 233c723a78365bf1a5a1eecae92d2108285e929b Mon Sep 17 00:00:00 2001 From: Anton Kolesnikov Date: Mon, 24 Jul 2023 14:45:58 +0800 Subject: [PATCH 34/35] Remove unused rowNum field --- pkg/phlaredb/block_querier.go | 1 - 1 file changed, 1 deletion(-) diff --git a/pkg/phlaredb/block_querier.go b/pkg/phlaredb/block_querier.go index 009771dfb1..e6171e5c79 100644 --- a/pkg/phlaredb/block_querier.go +++ b/pkg/phlaredb/block_querier.go @@ -1226,7 +1226,6 @@ type inMemoryparquetReader[M Models, P schemav1.Persister[M]] struct { persister P file *parquet.File size int64 - numRows int64 reader phlareobj.ReaderAtCloser cache []M } From d3d7b9fca628a87365647ea2dce88f347adca24d Mon Sep 17 00:00:00 2001 From: Cyril Tovena Date: Wed, 26 Jul 2023 09:00:47 +0200 Subject: [PATCH 35/35] Remove bad tests --- pkg/phlaredb/compact_test.go | 148 ++++++++--------------------------- 1 file changed, 33 insertions(+), 115 deletions(-) diff --git a/pkg/phlaredb/compact_test.go b/pkg/phlaredb/compact_test.go index 0c9bdd4788..f386e51497 100644 --- a/pkg/phlaredb/compact_test.go +++ b/pkg/phlaredb/compact_test.go @@ -3,139 +3,57 @@ package phlaredb import ( "context" "fmt" - "net/http" _ "net/http/pprof" "os" "path/filepath" - "sort" - "sync" "testing" - "time" "github.com/prometheus/common/model" "github.com/prometheus/prometheus/storage" "github.com/segmentio/parquet-go" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" - "golang.org/x/sync/errgroup" typesv1 "github.com/grafana/pyroscope/api/gen/proto/go/types/v1" phlaremodel "github.com/grafana/pyroscope/pkg/model" - phlareobj "github.com/grafana/pyroscope/pkg/objstore" - "github.com/grafana/pyroscope/pkg/objstore/client" - "github.com/grafana/pyroscope/pkg/objstore/providers/filesystem" - "github.com/grafana/pyroscope/pkg/objstore/providers/gcs" "github.com/grafana/pyroscope/pkg/phlaredb/block" schemav1 "github.com/grafana/pyroscope/pkg/phlaredb/schemas/v1" "github.com/grafana/pyroscope/pkg/phlaredb/tsdb/index" ) -func init() { - go func() { - _ = http.ListenAndServe("localhost:6060", nil) - }() -} - -func TestCompact(t *testing.T) { - t.TempDir() - ctx := context.Background() - bkt, err := client.NewBucket(ctx, client.Config{ - StorageBackendConfig: client.StorageBackendConfig{ - Backend: client.GCS, - GCS: gcs.Config{ - BucketName: "dev-us-central-0-profiles-dev-001-data", - }, - }, - StoragePrefix: "1218/phlaredb/", - }, "test") - require.NoError(t, err) - now := time.Now() - var ( - meta []*block.Meta - mtx sync.Mutex - ) - - err = block.IterBlockMetas(ctx, bkt, now.Add(-24*time.Hour), now, func(m *block.Meta) { - mtx.Lock() - defer mtx.Unlock() - meta = append(meta, m) - }) - require.NoError(t, err) - dst := t.TempDir() - - sort.Slice(meta, func(i, j int) bool { - return meta[i].MinTime.Before(meta[j].MinTime) - }) - - // only test on the 4 latest blocks. - meta = meta[len(meta)-4:] - testCompact(t, meta, bkt, dst) -} - -// to download the blocks: -// gsutil -m cp -r \ -// "gs://dev-us-central-0-profiles-dev-001-data/1218/phlaredb/01H53WJEAB43S3GJ26XMSNRSJA" \ -// "gs://dev-us-central-0-profiles-dev-001-data/1218/phlaredb/01H5454JBEV80V2J7CKYHPCBG8" \ -// "gs://dev-us-central-0-profiles-dev-001-data/1218/phlaredb/01H54553SYKH43FNJN5BVR1M2H" \ -// "gs://dev-us-central-0-profiles-dev-001-data/1218/phlaredb/01H5457Q89WYYH9FCK8PZ6XG75" \ -// . -func TestCompactLocal(t *testing.T) { - t.TempDir() - ctx := context.Background() - bkt, err := client.NewBucket(ctx, client.Config{ - StorageBackendConfig: client.StorageBackendConfig{ - Backend: client.Filesystem, - Filesystem: filesystem.Config{ - Directory: "/Users/cyril/work/phlare-data/", - }, - }, - StoragePrefix: "", - }, "test") - require.NoError(t, err) - var metas []*block.Meta - - metaMap, err := block.ListBlocks("/Users/cyril/work/phlare-data/", time.Time{}) - require.NoError(t, err) - for _, m := range metaMap { - metas = append(metas, m) - } - dst := t.TempDir() - testCompact(t, metas, bkt, dst) -} - -func testCompact(t *testing.T, metas []*block.Meta, bkt phlareobj.Bucket, dst string) { - t.Helper() - g, ctx := errgroup.WithContext(context.Background()) - var src []BlockReader - now := time.Now() - for i, m := range metas { - t.Log("src block(#", i, ")", - "ID", m.ULID.String(), - "minTime", m.MinTime.Time().Format(time.RFC3339Nano), - "maxTime", m.MaxTime.Time().Format(time.RFC3339Nano), - "numSeries", m.Stats.NumSeries, - "numProfiles", m.Stats.NumProfiles, - "numSamples", m.Stats.NumSamples) - b := NewSingleBlockQuerierFromMeta(ctx, bkt, m) - g.Go(func() error { - if err := b.Open(ctx); err != nil { - return err - } - return b.stacktraces.Load(ctx) - }) - src = append(src, b) - } - - require.NoError(t, g.Wait()) - - new, err := Compact(context.Background(), src, dst) - require.NoError(t, err) - t.Log(new, dst) - t.Log("Compaction duration", time.Since(now)) - t.Log("numSeries", new.Stats.NumSeries, - "numProfiles", new.Stats.NumProfiles, - "numSamples", new.Stats.NumSamples) -} +// func testCompact(t *testing.T, metas []*block.Meta, bkt phlareobj.Bucket, dst string) { +// t.Helper() +// g, ctx := errgroup.WithContext(context.Background()) +// var src []BlockReader +// now := time.Now() +// for i, m := range metas { +// t.Log("src block(#", i, ")", +// "ID", m.ULID.String(), +// "minTime", m.MinTime.Time().Format(time.RFC3339Nano), +// "maxTime", m.MaxTime.Time().Format(time.RFC3339Nano), +// "numSeries", m.Stats.NumSeries, +// "numProfiles", m.Stats.NumProfiles, +// "numSamples", m.Stats.NumSamples) +// b := NewSingleBlockQuerierFromMeta(ctx, bkt, m) +// g.Go(func() error { +// if err := b.Open(ctx); err != nil { +// return err +// } +// return b.stacktraces.Load(ctx) +// }) +// src = append(src, b) +// } + +// require.NoError(t, g.Wait()) + +// new, err := Compact(context.Background(), src, dst) +// require.NoError(t, err) +// t.Log(new, dst) +// t.Log("Compaction duration", time.Since(now)) +// t.Log("numSeries", new.Stats.NumSeries, +// "numProfiles", new.Stats.NumProfiles, +// "numSamples", new.Stats.NumSamples) +// } type blockReaderMock struct { BlockReader