From 3943d073a0902dd191e11ab13b316a694cc86268 Mon Sep 17 00:00:00 2001 From: Alexandr Yudin Date: Tue, 29 Jul 2025 13:25:27 +0000 Subject: [PATCH 01/96] for save --- pp-pkg/handler/interface.go | 2 +- pp-pkg/receiver/receiver.go | 255 +++++++++--------- pp/go/relabeler/appender/appender.go | 40 ++- pp/go/relabeler/appender/appender_test.go | 26 +- pp/go/relabeler/head/manager/manager.go | 5 +- pp/go/relabeler/headcontainer/active.go | 224 +++++++++++++++ .../headcontainer/discardable_rotatable.go | 61 +++++ .../headcontainer/heap_profile_writable.go | 30 +++ pp/go/relabeler/headcontainer/rotatable.go | 118 ++++++++ pp/go/relabeler/querier/metrics.go | 29 -- pp/go/relabeler/querier/multi.go | 26 +- web/api/v1/api.go | 30 +-- web/api/v1/errors_test.go | 1 - web/api/v1/pp_api.go | 10 +- web/web.go | 2 +- 15 files changed, 662 insertions(+), 197 deletions(-) create mode 100644 pp/go/relabeler/headcontainer/active.go create mode 100644 pp/go/relabeler/headcontainer/discardable_rotatable.go create mode 100644 pp/go/relabeler/headcontainer/heap_profile_writable.go create mode 100644 pp/go/relabeler/headcontainer/rotatable.go diff --git a/pp-pkg/handler/interface.go b/pp-pkg/handler/interface.go index 7f5c6db92f..f2fd23edd2 100644 --- a/pp-pkg/handler/interface.go +++ b/pp-pkg/handler/interface.go @@ -22,7 +22,7 @@ type Receiver interface { commitToWal bool, ) (cppbridge.RelabelerStats, error) RelabelerIDIsExist(relabelerID string) bool - HeadQueryable() storage.Queryable + HeadQuerier(ctx context.Context, mint, maxt int64) (storage.Querier, error) HeadStatus(ctx context.Context, limit int) relabeler.HeadStatus // MergeOutOfOrderChunks merge chunks with out of order data chunks. MergeOutOfOrderChunks(ctx context.Context) diff --git a/pp-pkg/receiver/receiver.go b/pp-pkg/receiver/receiver.go index af9863401c..2dc38742c6 100644 --- a/pp-pkg/receiver/receiver.go +++ b/pp-pkg/receiver/receiver.go @@ -9,7 +9,6 @@ import ( "os" "path" "path/filepath" - "sync" "time" "github.com/go-kit/log" @@ -35,6 +34,7 @@ import ( "github.com/prometheus/prometheus/pp/go/relabeler/head/catalog" headmanager "github.com/prometheus/prometheus/pp/go/relabeler/head/manager" "github.com/prometheus/prometheus/pp/go/relabeler/head/ready" + "github.com/prometheus/prometheus/pp/go/relabeler/headcontainer" rlogger "github.com/prometheus/prometheus/pp/go/relabeler/logger" "github.com/prometheus/prometheus/pp/go/relabeler/querier" "github.com/prometheus/prometheus/pp/go/util" @@ -71,8 +71,9 @@ func (s *HeadConfigStorage) Store(headConfig *HeadConfig) { type Receiver struct { ctx context.Context - distributor *distributor.Distributor - appender *appender.QueryableAppender + distributor *distributor.Distributor + // appender *appender.QueryableAppender + activeHead *headcontainer.Active storage *appender.QueryableStorage rotator *appender.RotateCommiter metricsWriteTrigger *appender.MetricsWriteTrigger @@ -88,6 +89,9 @@ type Receiver struct { clientID string cgogc *cppbridge.CGOGC shutdowner *util.GracefulShutdowner + + activeQuerierMetrics *querier.Metrics + storageQuerierMetrics *querier.Metrics } type RotationInfo struct { @@ -182,15 +186,16 @@ func NewReceiver( return nil, fmt.Errorf("failed to create head manager: %w", err) } - activeHead, rotatedHeads, err := headManager.Restore(rotationInfo.BlockDuration) + currentHead, rotatedHeads, err := headManager.Restore(rotationInfo.BlockDuration) if err != nil { return nil, fmt.Errorf("failed to restore heads: %w", err) } readyNotifier.NotifyReady() + storageQuerierMetrics := querier.NewMetrics(registerer, querier.QueryableStorageSource) queryableStorage := appender.NewQueryableStorageWithWriteNotifier( block.NewBlockWriter(dataDir, block.DefaultChunkSegmentSize, rotationInfo.BlockDuration, registerer), registerer, - querier.NewMetrics(registerer, querier.QueryableStorageSource), + storageQuerierMetrics, triggerNotifier, clock, maxRetentionDuration, @@ -199,32 +204,38 @@ func NewReceiver( rotatedHeads..., ) - hd := appender.NewRotatableHead(activeHead, queryableStorage, headManager, newHeadActivator(headCatalog)) + var containeredHead relabeler.Head + containeredHead = headcontainer.NewRotatable(currentHead, queryableStorage, headManager, newHeadActivator(headCatalog)) - var appenderHead relabeler.Head = hd if len(os.Getenv("OPCORE_ROTATION_HEAP_DEBUG")) > 0 { - heapProfileWriter := util.NewHeapProfileWriter(filepath.Join(dataDir, "heap_profiles")) - appenderHead = appender.NewHeapProfileWritableHead(appenderHead, heapProfileWriter) + containeredHead = headcontainer.NewHeapProfileWritable( + containeredHead, + util.NewHeapProfileWriter(filepath.Join(dataDir, "heap_profiles")), + ) } dstrb := distributor.NewDistributor(*destinationGroups) - app := appender.NewQueryableAppender( - ctx, - appenderHead, - dstrb, - querier.NewMetrics(registerer, querier.QueryableAppenderSource), - ) - mwt := appender.NewMetricsWriteTrigger(ctx, appender.DefaultMetricWriteInterval, app, queryableStorage) + activeQuerierMetrics := querier.NewMetrics(registerer, querier.QueryableAppenderSource) + activeHead := headcontainer.NewActive(containeredHead, registerer) + // app := appender.NewQueryableAppender( + // ctx, + // appenderHead, + // dstrb, + // activeQuerierMetrics, + // registerer, + // ) + mwt := appender.NewMetricsWriteTrigger(ctx, appender.DefaultMetricWriteInterval, activeHead, queryableStorage) r := &Receiver{ - ctx: ctx, - distributor: dstrb, - appender: app, + ctx: ctx, + distributor: dstrb, + // appender: app, + activeHead: activeHead, storage: queryableStorage, headConfigStorage: headConfigStorage, rotator: appender.NewRotateCommiter( ctx, - app, + activeHead, relabeler.NewRotateTimerWithSeed(clock, rotationInfo.BlockDuration, rotationInfo.Seed), appender.NewConstantIntervalTimer(clock, commitInterval), appender.NewConstantIntervalTimer(clock, appender.DefaultMergeDuration), @@ -242,6 +253,9 @@ func NewReceiver( clientID: clientID, cgogc: cppbridge.NewCGOGC(registerer), shutdowner: util.NewGracefulShutdowner(), + + activeQuerierMetrics: activeQuerierMetrics, + storageQuerierMetrics: storageQuerierMetrics, } level.Info(logger).Log("msg", "created") @@ -260,7 +274,7 @@ func (rr *Receiver) AppendHashdex( return nil } incomingData := &relabeler.IncomingData{Hashdex: hashdex} - _, err := rr.appender.Append(ctx, incomingData, nil, relabelerID, commitToWal) + _, err := rr.activeHead.Append(ctx, incomingData, nil, relabelerID, commitToWal) return err } @@ -282,7 +296,7 @@ func (rr *Receiver) AppendSnappyProtobuf( } incomingData := &relabeler.IncomingData{Hashdex: hx} - _, err = rr.appender.Append(ctx, incomingData, nil, relabelerID, commitToWal) + _, err = rr.activeHead.Append(ctx, incomingData, nil, relabelerID, commitToWal) return err } @@ -305,7 +319,7 @@ func (rr *Receiver) AppendTimeSeries( return cppbridge.RelabelerStats{}, nil } incomingData := &relabeler.IncomingData{Hashdex: hx, Data: data} - return rr.appender.AppendWithStaleNans( + return rr.activeHead.Append( ctx, incomingData, state, @@ -321,7 +335,7 @@ func (rr *Receiver) AppendTimeSeriesHashdex( relabelerID string, commitToWal bool, ) (cppbridge.RelabelerStats, error) { - return rr.appender.AppendWithStaleNans( + return rr.activeHead.Append( ctx, &relabeler.IncomingData{Hashdex: hashdex}, state, @@ -355,96 +369,96 @@ func (rr *Receiver) ApplyConfig(cfg *prom_config.Config) error { numberOfShards: numberOfShards, }) - err = rr.appender.Reconfigure( + err = rr.activeHead.Reconfigure( rr.ctx, HeadConfigureFunc(func(head relabeler.Head) error { return head.Reconfigure(rr.ctx, rCfg.Configs, numberOfShards) }), - DistributorConfigureFunc(func(dstrb relabeler.Distributor) error { - mxdgupds := new(sync.Mutex) - dgupds, err := makeDestinationGroupUpdates( - cfg.RemoteWriteConfigs, - rr.workingDir, - rr.clientID, - numberOfShards, - ) - if err != nil { - level.Error(rr.logger).Log("msg", "failed to init destination group update", "err", err) - return err - } - mxDelete := new(sync.Mutex) - toDelete := []int{} - - dgs := dstrb.DestinationGroups() - if err = dgs.RangeGo(func(destinationGroupID int, dg *relabeler.DestinationGroup) error { - var rangeErr error - dgu, ok := dgupds[dg.Name()] - if !ok { - mxDelete.Lock() - toDelete = append(toDelete, destinationGroupID) - mxDelete.Unlock() - ctxShutdown, cancel := context.WithTimeout(rr.ctx, defaultShutdownTimeout) - if rangeErr = dg.Shutdown(ctxShutdown); err != nil { - level.Error(rr.logger).Log("msg", "failed shutdown DestinationGroup", "err", rangeErr) - } - cancel() - return nil - } - - if !dg.Equal(dgu.DestinationGroupConfig) || - !dg.EqualDialers(dgu.DialersConfigs) { - var dialers []relabeler.Dialer - if !dg.EqualDialers(dgu.DialersConfigs) { - dialers, rangeErr = makeDialers(rr.clock, rr.registerer, dgu.DialersConfigs) - if rangeErr != nil { - return rangeErr - } - } - - if rangeErr = dg.ResetTo(dgu.DestinationGroupConfig, dialers); err != nil { - return rangeErr - } - } - mxdgupds.Lock() - delete(dgupds, dg.Name()) - mxdgupds.Unlock() - return nil - }); err != nil { - level.Error(rr.logger).Log("msg", "failed to apply config DestinationGroups", "err", err) - return err - } - // delete unused DestinationGroup - dgs.RemoveByID(toDelete) - - // create new DestinationGroup - for _, dgupd := range dgupds { - dialers, err := makeDialers(rr.clock, rr.registerer, dgupd.DialersConfigs) - if err != nil { - level.Error(rr.logger).Log("msg", "failed to make new dialers", "err", err) - return err - } - - dg, err := relabeler.NewDestinationGroup( - rr.ctx, - dgupd.DestinationGroupConfig, - encoderSelector, - refillCtor, - refillSenderCtor, - rr.clock, - dialers, - rr.registerer, - ) - if err != nil { - level.Error(rr.logger).Log("msg", "failed to init DestinationGroup", "err", err) - return err - } - - dgs.Add(dg) - } - dstrb.SetDestinationGroups(dgs) - - return nil - }), + // DistributorConfigureFunc(func(dstrb relabeler.Distributor) error { + // mxdgupds := new(sync.Mutex) + // dgupds, err := makeDestinationGroupUpdates( + // cfg.RemoteWriteConfigs, + // rr.workingDir, + // rr.clientID, + // numberOfShards, + // ) + // if err != nil { + // level.Error(rr.logger).Log("msg", "failed to init destination group update", "err", err) + // return err + // } + // mxDelete := new(sync.Mutex) + // toDelete := []int{} + + // dgs := dstrb.DestinationGroups() + // if err = dgs.RangeGo(func(destinationGroupID int, dg *relabeler.DestinationGroup) error { + // var rangeErr error + // dgu, ok := dgupds[dg.Name()] + // if !ok { + // mxDelete.Lock() + // toDelete = append(toDelete, destinationGroupID) + // mxDelete.Unlock() + // ctxShutdown, cancel := context.WithTimeout(rr.ctx, defaultShutdownTimeout) + // if rangeErr = dg.Shutdown(ctxShutdown); err != nil { + // level.Error(rr.logger).Log("msg", "failed shutdown DestinationGroup", "err", rangeErr) + // } + // cancel() + // return nil + // } + + // if !dg.Equal(dgu.DestinationGroupConfig) || + // !dg.EqualDialers(dgu.DialersConfigs) { + // var dialers []relabeler.Dialer + // if !dg.EqualDialers(dgu.DialersConfigs) { + // dialers, rangeErr = makeDialers(rr.clock, rr.registerer, dgu.DialersConfigs) + // if rangeErr != nil { + // return rangeErr + // } + // } + + // if rangeErr = dg.ResetTo(dgu.DestinationGroupConfig, dialers); err != nil { + // return rangeErr + // } + // } + // mxdgupds.Lock() + // delete(dgupds, dg.Name()) + // mxdgupds.Unlock() + // return nil + // }); err != nil { + // level.Error(rr.logger).Log("msg", "failed to apply config DestinationGroups", "err", err) + // return err + // } + // // delete unused DestinationGroup + // dgs.RemoveByID(toDelete) + + // // create new DestinationGroup + // for _, dgupd := range dgupds { + // dialers, err := makeDialers(rr.clock, rr.registerer, dgupd.DialersConfigs) + // if err != nil { + // level.Error(rr.logger).Log("msg", "failed to make new dialers", "err", err) + // return err + // } + + // dg, err := relabeler.NewDestinationGroup( + // rr.ctx, + // dgupd.DestinationGroupConfig, + // encoderSelector, + // refillCtor, + // refillSenderCtor, + // rr.clock, + // dialers, + // rr.registerer, + // ) + // if err != nil { + // level.Error(rr.logger).Log("msg", "failed to init DestinationGroup", "err", err) + // return err + // } + + // dgs.Add(dg) + // } + // dstrb.SetDestinationGroups(dgs) + + // return nil + // }), ) if err != nil { return err @@ -458,12 +472,13 @@ func (rr *Receiver) GetState() *cppbridge.State { return cppbridge.NewState(rr.headConfigStorage.Load().numberOfShards) } -func (rr *Receiver) HeadQueryable() storage.Queryable { - return rr.appender +// HeadQuerier returns [storage.Querier] from active head. +func (rr *Receiver) HeadQuerier(ctx context.Context, mint, maxt int64) (storage.Querier, error) { + return rr.activeHead.Querier(ctx, rr.activeQuerierMetrics, mint, maxt) } func (rr *Receiver) HeadStatus(ctx context.Context, limit int) relabeler.HeadStatus { - return rr.appender.HeadStatus(ctx, limit) + return rr.activeHead.HeadStatus(ctx, limit) } // LowestSentTimestamp returns the lowest sent timestamp across all queues. @@ -473,27 +488,27 @@ func (*Receiver) LowestSentTimestamp() int64 { // MergeOutOfOrderChunks merge chunks with out of order data chunks. func (rr *Receiver) MergeOutOfOrderChunks(ctx context.Context) { - rr.appender.MergeOutOfOrderChunks(ctx) + rr.activeHead.MergeOutOfOrderChunks(ctx) } // Querier calls f() with the given parameters. // Returns a querier.MultiQuerier combining of appenderQuerier and storageQuerier. func (rr *Receiver) Querier(mint, maxt int64) (storage.Querier, error) { - appenderQuerier, err := rr.appender.Querier(mint, maxt) + activeQuerier, err := rr.activeHead.Querier(rr.ctx, rr.activeQuerierMetrics, mint, maxt) if err != nil { return nil, err } storageQuerier, err := rr.storage.Querier(mint, maxt) if err != nil { - return nil, errors.Join(err, appenderQuerier.Close()) + return nil, errors.Join(err, activeQuerier.Close()) } - return querier.NewMultiQuerier([]storage.Querier{appenderQuerier, storageQuerier}, nil), nil + return querier.NewMultiQuerier([]storage.Querier{activeQuerier, storageQuerier}, nil), nil } func (rr *Receiver) ChunkQuerier(mint, maxt int64) (storage.ChunkQuerier, error) { - appenderQuerier, err := rr.appender.ChunkQuerier(mint, maxt) + appenderQuerier, err := rr.activeHead.ChunkQuerier(rr.ctx, mint, maxt) if err != nil { return nil, err } @@ -538,9 +553,9 @@ func (rr *Receiver) Shutdown(ctx context.Context) error { rotatorErr := rr.rotator.Close() storageErr := rr.storage.Close() distributorErr := rr.distributor.Shutdown(ctx) - appendErr := rr.appender.Close(ctx) + activeHeadErr := rr.activeHead.Close(ctx) err := rr.shutdowner.Shutdown(ctx) - return errors.Join(cgogcErr, metricWriteErr, rotatorErr, storageErr, distributorErr, appendErr, err) + return errors.Join(cgogcErr, metricWriteErr, rotatorErr, storageErr, distributorErr, activeHeadErr, err) } // makeDestinationGroups create DestinationGroups from configs. diff --git a/pp/go/relabeler/appender/appender.go b/pp/go/relabeler/appender/appender.go index f68149312d..8b28d8170c 100644 --- a/pp/go/relabeler/appender/appender.go +++ b/pp/go/relabeler/appender/appender.go @@ -6,10 +6,12 @@ import ( "fmt" "time" + "github.com/prometheus/client_golang/prometheus" "github.com/prometheus/prometheus/pp/go/cppbridge" "github.com/prometheus/prometheus/pp/go/relabeler" "github.com/prometheus/prometheus/pp/go/relabeler/logger" "github.com/prometheus/prometheus/pp/go/relabeler/querier" + "github.com/prometheus/prometheus/pp/go/util" "github.com/prometheus/prometheus/pp/go/util/locker" "github.com/prometheus/prometheus/storage" ) @@ -20,6 +22,10 @@ type QueryableAppender struct { head relabeler.Head distributor relabeler.Distributor querierMetrics *querier.Metrics + + appendDuration prometheus.Histogram + waitLockRotateDuration prometheus.Gauge + rotationDuration prometheus.Gauge } func NewQueryableAppender( @@ -27,13 +33,41 @@ func NewQueryableAppender( head relabeler.Head, distributor relabeler.Distributor, querierMetrics *querier.Metrics, + registerer prometheus.Registerer, ) *QueryableAppender { + factory := util.NewUnconflictRegisterer(registerer) return &QueryableAppender{ ctx: ctx, wlocker: locker.NewWeighted(2 * head.Concurrency()), // x2 for back pressure head: head, distributor: distributor, querierMetrics: querierMetrics, + + appendDuration: factory.NewHistogram( + prometheus.HistogramOpts{ + Name: "prompp_head_append_duration", + Help: "Append to head duration in microseconds", + Buckets: []float64{ + 50, 100, 250, 500, 750, + 1000, 2500, 5000, 7500, + 10000, 25000, 50000, 75000, + 100000, 500000, + }, + }, + ), + + waitLockRotateDuration: factory.NewGauge( + prometheus.GaugeOpts{ + Name: "prompp_head_wait_lock_rotate_duration", + Help: "The duration of the lock wait for rotation in nanoseconds", + }, + ), + rotationDuration: factory.NewGauge( + prometheus.GaugeOpts{ + Name: "prompp_head_rotate_duration", + Help: "The duration of the rotate in nanoseconds", + }, + ), } } @@ -63,7 +97,7 @@ func (qa *QueryableAppender) AppendWithStaleNans( defer runlock() defer func() { - qa.querierMetrics.AppendDuration.Observe(float64(time.Since(start).Microseconds())) + qa.appendDuration.Observe(float64(time.Since(start).Microseconds())) }() data, stats, err := qa.head.Append(ctx, incomingData, state, relabelerID, commitToWal) @@ -130,11 +164,11 @@ func (qa *QueryableAppender) Rotate(ctx context.Context) error { if err != nil { return fmt.Errorf("Rotate: weighted locker: %w", err) } - qa.querierMetrics.WaitLockRotateDuration.Set(float64(time.Since(start).Nanoseconds())) + qa.waitLockRotateDuration.Set(float64(time.Since(start).Nanoseconds())) defer unlock() defer func() { - qa.querierMetrics.RotationDuration.Set(float64(time.Since(start).Nanoseconds())) + qa.rotationDuration.Set(float64(time.Since(start).Nanoseconds())) }() qa.head.MergeOutOfOrderChunks() diff --git a/pp/go/relabeler/appender/appender_test.go b/pp/go/relabeler/appender/appender_test.go index f3d71d2a89..70a002b903 100644 --- a/pp/go/relabeler/appender/appender_test.go +++ b/pp/go/relabeler/appender/appender_test.go @@ -126,7 +126,7 @@ func (s *AppenderSuite) TestManagerRelabelerKeep() { defer func() { _ = hd.Close() }() s.T().Log("make appender") - app := appender.NewQueryableAppender(s.baseCtx, hd, dstrb, s.metrics) + app := appender.NewQueryableAppender(s.baseCtx, hd, dstrb, s.metrics, nil) hlimits := cppbridge.DefaultWALHashdexLimits() @@ -269,7 +269,7 @@ func (s *AppenderSuite) TestManagerRelabelerRelabeling() { s.Require().NoError(err) defer func() { _ = hd.Close() }() s.T().Log("make appender") - app := appender.NewQueryableAppender(s.baseCtx, hd, dstrb, s.metrics) + app := appender.NewQueryableAppender(s.baseCtx, hd, dstrb, s.metrics, nil) hlimits := cppbridge.DefaultWALHashdexLimits() @@ -423,7 +423,7 @@ func (s *AppenderSuite) TestManagerRelabelerRelabelingAddNewLabel() { s.Require().NoError(err) defer func() { _ = hd.Close() }() s.T().Log("make appender") - app := appender.NewQueryableAppender(s.baseCtx, hd, dstrb, s.metrics) + app := appender.NewQueryableAppender(s.baseCtx, hd, dstrb, s.metrics, nil) hlimits := cppbridge.DefaultWALHashdexLimits() @@ -582,7 +582,7 @@ func (s *AppenderSuite) TestManagerRelabelerRelabelingWithExternalLabelsEnd() { defer func() { _ = hd.Close() }() s.Require().NoError(err) s.T().Log("make appender") - app := appender.NewQueryableAppender(s.baseCtx, hd, dstrb, s.metrics) + app := appender.NewQueryableAppender(s.baseCtx, hd, dstrb, s.metrics, nil) hlimits := cppbridge.DefaultWALHashdexLimits() s.T().Log("append first data") @@ -739,7 +739,7 @@ func (s *AppenderSuite) TestManagerRelabelerRelabelingWithExternalLabelsRelabel( s.Require().NoError(err) defer func() { _ = hd.Close() }() s.T().Log("make appender") - app := appender.NewQueryableAppender(s.baseCtx, hd, dstrb, s.metrics) + app := appender.NewQueryableAppender(s.baseCtx, hd, dstrb, s.metrics, nil) hlimits := cppbridge.DefaultWALHashdexLimits() @@ -901,7 +901,7 @@ func (s *AppenderSuite) TestManagerRelabelerRelabelingWithTargetLabels() { s.Require().NoError(err) defer func() { _ = hd.Close() }() s.T().Log("make appender") - app := appender.NewQueryableAppender(s.baseCtx, hd, dstrb, s.metrics) + app := appender.NewQueryableAppender(s.baseCtx, hd, dstrb, s.metrics, nil) hlimits := cppbridge.DefaultWALHashdexLimits() @@ -1093,7 +1093,7 @@ func (s *AppenderSuite) TestManagerRelabelerRelabelingWithTargetLabels_Conflicti s.Require().NoError(err) defer func() { _ = hd.Close() }() s.T().Log("make appender") - app := appender.NewQueryableAppender(s.baseCtx, hd, dstrb, s.metrics) + app := appender.NewQueryableAppender(s.baseCtx, hd, dstrb, s.metrics, nil) hlimits := cppbridge.DefaultWALHashdexLimits() @@ -1286,7 +1286,7 @@ func (s *AppenderSuite) TestManagerRelabelerRelabelingWithTargetLabels_Conflicti s.Require().NoError(err) defer func() { _ = hd.Close() }() s.T().Log("make appender") - app := appender.NewQueryableAppender(s.baseCtx, hd, dstrb, s.metrics) + app := appender.NewQueryableAppender(s.baseCtx, hd, dstrb, s.metrics, nil) hlimits := cppbridge.DefaultWALHashdexLimits() @@ -1515,7 +1515,7 @@ func (s *AppenderSuite) TestManagerRelabelerRelabelingWithRotate() { rotatableHead := appender.NewRotatableHead(hd, noOpStorage{}, builder, appender.NoOpHeadActivator{}) s.T().Log("make appender") - app := appender.NewQueryableAppender(s.baseCtx, rotatableHead, dstrb, s.metrics) + app := appender.NewQueryableAppender(s.baseCtx, rotatableHead, dstrb, s.metrics, nil) rotationTimer := relabeler.NewRotateTimer(clock, appender.DefaultRotateDuration) commitTimer := appender.NewConstantIntervalTimer(clock, appender.DefaultCommitTimeout) @@ -1974,7 +1974,7 @@ func (s *AppenderSuite) TestManagerRelabelerKeepWithStaleNans() { s.Require().NoError(err) defer func() { _ = hd.Close() }() s.T().Log("make appender") - app := appender.NewQueryableAppender(s.baseCtx, hd, dstrb, s.metrics) + app := appender.NewQueryableAppender(s.baseCtx, hd, dstrb, s.metrics, nil) hlimits := cppbridge.DefaultWALHashdexLimits() @@ -2108,7 +2108,7 @@ func (s *AppenderSuite) TestManagerRelabelerKeepWithStaleNans_WithNullTimestamp( s.Require().NoError(err) defer func() { _ = hd.Close() }() s.T().Log("make appender") - app := appender.NewQueryableAppender(s.baseCtx, hd, dstrb, s.metrics) + app := appender.NewQueryableAppender(s.baseCtx, hd, dstrb, s.metrics, nil) hlimits := cppbridge.DefaultWALHashdexLimits() @@ -2239,7 +2239,7 @@ func (s *AppenderSuite) TestManagerRelabelerKeepWithStaleNans_HonorTimestamps() s.Require().NoError(err) defer func() { _ = hd.Close() }() s.T().Log("make appender") - app := appender.NewQueryableAppender(s.baseCtx, hd, dstrb, s.metrics) + app := appender.NewQueryableAppender(s.baseCtx, hd, dstrb, s.metrics, nil) hlimits := cppbridge.DefaultWALHashdexLimits() @@ -2419,7 +2419,7 @@ func (s *AppenderSuite) TestManagerRelabelerRelabelingWithRotateWithStaleNans() rotatableHead := appender.NewRotatableHead(hd, noOpStorage{}, builder, appender.NoOpHeadActivator{}) s.T().Log("make appender") - app := appender.NewQueryableAppender(s.baseCtx, rotatableHead, dstrb, s.metrics) + app := appender.NewQueryableAppender(s.baseCtx, rotatableHead, dstrb, s.metrics, nil) rotationTimer := relabeler.NewRotateTimer(clock, appender.DefaultRotateDuration) commitTimer := appender.NewConstantIntervalTimer(clock, appender.DefaultCommitTimeout) diff --git a/pp/go/relabeler/head/manager/manager.go b/pp/go/relabeler/head/manager/manager.go index fe69bac892..2a676189c1 100644 --- a/pp/go/relabeler/head/manager/manager.go +++ b/pp/go/relabeler/head/manager/manager.go @@ -14,6 +14,7 @@ import ( "github.com/prometheus/prometheus/pp/go/relabeler/config" "github.com/prometheus/prometheus/pp/go/relabeler/head" "github.com/prometheus/prometheus/pp/go/relabeler/head/catalog" + "github.com/prometheus/prometheus/pp/go/relabeler/headcontainer" "github.com/prometheus/prometheus/pp/go/relabeler/logger" "github.com/prometheus/prometheus/pp/go/util" ) @@ -295,9 +296,9 @@ func (m *Manager) BuildWithConfig( } // createDiscardableRotatableHead create discardable and rotatable head. -func (m *Manager) createDiscardableRotatableHead(h relabeler.Head, releaseHeadFn func()) *DiscardableRotatableHead { +func (m *Manager) createDiscardableRotatableHead(h relabeler.Head, releaseHeadFn func()) relabeler.Head { m.counter.With(prometheus.Labels{"type": "created"}).Inc() - return NewDiscardableRotatableHead( + return headcontainer.NewDiscardableRotatable( h, func(id string, err error) error { if _, rotateErr := m.catalog.SetStatus(id, catalog.StatusRotated); rotateErr != nil { diff --git a/pp/go/relabeler/headcontainer/active.go b/pp/go/relabeler/headcontainer/active.go new file mode 100644 index 0000000000..edb113165b --- /dev/null +++ b/pp/go/relabeler/headcontainer/active.go @@ -0,0 +1,224 @@ +package headcontainer + +import ( + "context" + "errors" + "fmt" + "time" + + "github.com/prometheus/client_golang/prometheus" + "github.com/prometheus/prometheus/pp/go/cppbridge" + "github.com/prometheus/prometheus/pp/go/relabeler" + "github.com/prometheus/prometheus/pp/go/relabeler/logger" + "github.com/prometheus/prometheus/pp/go/relabeler/querier" + "github.com/prometheus/prometheus/pp/go/util" + "github.com/prometheus/prometheus/pp/go/util/locker" + "github.com/prometheus/prometheus/storage" +) + +type Active struct { + wlocker *locker.Weighted + head relabeler.Head + + appendDuration prometheus.Histogram + waitLockRotateDuration prometheus.Gauge + rotationDuration prometheus.Gauge +} + +func NewActive( + head relabeler.Head, + registerer prometheus.Registerer, +) *Active { + factory := util.NewUnconflictRegisterer(registerer) + return &Active{ + wlocker: locker.NewWeighted(2 * head.Concurrency()), // x2 for back pressure + head: head, + + appendDuration: factory.NewHistogram( + prometheus.HistogramOpts{ + Name: "prompp_head_append_duration", + Help: "Append to head duration in microseconds", + Buckets: []float64{ + 50, 100, 250, 500, 750, + 1000, 2500, 5000, 7500, + 10000, 25000, 50000, 75000, + 100000, 500000, + }, + }, + ), + + waitLockRotateDuration: factory.NewGauge( + prometheus.GaugeOpts{ + Name: "prompp_head_wait_lock_rotate_duration", + Help: "The duration of the lock wait for rotation in nanoseconds", + }, + ), + rotationDuration: factory.NewGauge( + prometheus.GaugeOpts{ + Name: "prompp_head_rotate_duration", + Help: "The duration of the rotate in nanoseconds", + }, + ), + } +} + +func (h *Active) Append( + ctx context.Context, + incomingData *relabeler.IncomingData, + state *cppbridge.State, + relabelerID string, + commitToWal bool, +) (cppbridge.RelabelerStats, error) { + start := time.Now() + + runlock, err := h.wlocker.RLock(ctx) + if err != nil { + return cppbridge.RelabelerStats{}, fmt.Errorf("Append: weighted locker: %w", err) + } + defer runlock() + + defer func() { + h.appendDuration.Observe(float64(time.Since(start).Microseconds())) + }() + + _, stats, err := h.head.Append(ctx, incomingData, state, relabelerID, commitToWal) + if err != nil { + return cppbridge.RelabelerStats{}, err + } + + return stats, nil +} + +func (h *Active) ChunkQuerier(ctx context.Context, mint, maxt int64) (storage.ChunkQuerier, error) { + runlock, err := h.wlocker.RLock(ctx) + if err != nil { + return nil, fmt.Errorf("ChunkQuerier: weighted locker: %w", err) + } + head := h.head.Raw() + runlock() + return querier.NewChunkQuerier( + head, + querier.NoOpShardedDeduplicatorFactory(), + mint, + maxt, + nil, + ), nil +} + +func (h *Active) Close(ctx context.Context) error { + unlock, err := h.wlocker.LockWithPriority(ctx) + if err != nil { + return fmt.Errorf("Close: weighted locker: %w", err) + } + defer unlock() + + return errors.Join(h.head.CommitToWal(), h.head.Flush(), h.head.Close()) +} + +func (h *Active) CommitToWal(ctx context.Context) error { + runlock, err := h.wlocker.RLock(ctx) + if err != nil { + return fmt.Errorf("CommitToWal: weighted locker: %w", err) + } + defer runlock() + + return h.head.CommitToWal() +} + +func (h *Active) HeadStatus(ctx context.Context, limit int) relabeler.HeadStatus { + runlock, err := h.wlocker.RLock(ctx) + if err != nil { + logger.Warnf("[ActiveHead] HeadStatus: weighted locker: %s", err) + return relabeler.HeadStatus{} + } + defer runlock() + + return h.head.Status(limit) +} + +// MergeOutOfOrderChunks merge chunks with out of order data chunks. +func (h *Active) MergeOutOfOrderChunks(ctx context.Context) { + runlock, err := h.wlocker.RLock(ctx) + if err != nil { + logger.Warnf("[ActiveHead] MergeOutOfOrderChunks: weighted locker: %s", err) + return + } + defer runlock() + + h.head.MergeOutOfOrderChunks() +} + +func (h *Active) Querier( + ctx context.Context, + querierMetrics *querier.Metrics, + mint, maxt int64, +) (storage.Querier, error) { + runlock, err := h.wlocker.RLock(ctx) + if err != nil { + return nil, fmt.Errorf("Querier: weighted locker: %w", err) + } + head := h.head.Raw() + runlock() + + return querier.NewQuerier( + head, + querier.NoOpShardedDeduplicatorFactory(), + mint, + maxt, + nil, + querierMetrics, + ), nil +} + +func (h *Active) Reconfigure( + ctx context.Context, + headConfigurator relabeler.HeadConfigurator, +) error { + unlock, err := h.wlocker.LockWithPriority(ctx) + if err != nil { + return fmt.Errorf("Reconfigure: weighted locker: %w", err) + } + defer unlock() + + if err := headConfigurator.Configure(h.head); err != nil { + return fmt.Errorf("failed to reconfigure head: %w", err) + } + + h.wlocker.Resize(2 * h.head.Concurrency()) // x2 for back pressure + + return nil +} + +func (h *Active) Rotate(ctx context.Context) error { + start := time.Now() + + unlock, err := h.wlocker.LockWithPriority(ctx) + if err != nil { + return fmt.Errorf("Rotate: weighted locker: %w", err) + } + h.waitLockRotateDuration.Set(float64(time.Since(start).Nanoseconds())) + defer unlock() + + defer func() { + h.rotationDuration.Set(float64(time.Since(start).Nanoseconds())) + }() + + if err := h.head.Rotate(); err != nil { + return fmt.Errorf("failed to rotate head: %w", err) + } + + h.wlocker.Resize(2 * h.head.Concurrency()) // x2 for back pressure + + return nil +} + +func (h *Active) WriteMetrics(ctx context.Context) { + runlock, err := h.wlocker.RLock(ctx) + if err != nil { + logger.Warnf("[ActiveHead] writeMetrics: weighted locker: %s", err) + return + } + defer runlock() + + h.head.WriteMetrics(ctx) +} diff --git a/pp/go/relabeler/headcontainer/discardable_rotatable.go b/pp/go/relabeler/headcontainer/discardable_rotatable.go new file mode 100644 index 0000000000..ff1fa51550 --- /dev/null +++ b/pp/go/relabeler/headcontainer/discardable_rotatable.go @@ -0,0 +1,61 @@ +package headcontainer + +import ( + "errors" + + "github.com/prometheus/prometheus/pp/go/relabeler" +) + +// +// DiscardableRotatable +// + +type DiscardableRotatable struct { + onRotate func(id string, err error) error + onDiscard func(id string) error + afterClose func(id string) error + relabeler.Head +} + +func NewDiscardableRotatable( + head relabeler.Head, + onRotate func(id string, err error) error, + onDiscard func(id string) error, + afterClose func(id string) error, +) *DiscardableRotatable { + return &DiscardableRotatable{ + onRotate: onRotate, + onDiscard: onDiscard, + afterClose: afterClose, + Head: head, + } +} + +func (h *DiscardableRotatable) Rotate() error { + err := h.Head.Rotate() + if h.onRotate != nil { + err = errors.Join(err, h.onRotate(h.ID(), err)) + h.onRotate = nil + } + + return err +} + +func (h *DiscardableRotatable) Close() error { + err := h.Head.Close() + if h.afterClose != nil { + err = errors.Join(err, h.afterClose(h.ID())) + } + + return err +} + +func (h *DiscardableRotatable) Discard() (err error) { + err = h.Head.Discard() + if h.onDiscard != nil { + err = errors.Join(err, h.onDiscard(h.ID())) + h.onDiscard = nil + } + + return err +} diff --git a/pp/go/relabeler/headcontainer/heap_profile_writable.go b/pp/go/relabeler/headcontainer/heap_profile_writable.go new file mode 100644 index 0000000000..59a2e5fe7a --- /dev/null +++ b/pp/go/relabeler/headcontainer/heap_profile_writable.go @@ -0,0 +1,30 @@ +package headcontainer + +import ( + "github.com/prometheus/prometheus/pp/go/relabeler" +) + +// +// HeapProfileWritable +// + +type HeapProfileWriter interface { + WriteHeapProfile() error +} + +type HeapProfileWritable struct { + heapProfileWriter HeapProfileWriter + relabeler.Head +} + +func NewHeapProfileWritable(head relabeler.Head, heapProfileWriter HeapProfileWriter) *HeapProfileWritable { + return &HeapProfileWritable{Head: head, heapProfileWriter: heapProfileWriter} +} + +func (h *HeapProfileWritable) Rotate() error { + if err := h.Head.Rotate(); err != nil { + return err + } + + return h.heapProfileWriter.WriteHeapProfile() +} diff --git a/pp/go/relabeler/headcontainer/rotatable.go b/pp/go/relabeler/headcontainer/rotatable.go new file mode 100644 index 0000000000..68f0691dce --- /dev/null +++ b/pp/go/relabeler/headcontainer/rotatable.go @@ -0,0 +1,118 @@ +package headcontainer + +import ( + "context" + + "github.com/prometheus/prometheus/pp/go/relabeler" + "github.com/prometheus/prometheus/pp/go/relabeler/config" + "github.com/prometheus/prometheus/pp/go/relabeler/logger" +) + +// CopySeriesOnRotate copy active series from the current head to the new head during rotation. +var CopySeriesOnRotate = false + +// Storage - head storage. +type Storage interface { + Add(head relabeler.Head) +} + +// HeadBuilder - head builder. +type HeadBuilder interface { + Build() (relabeler.Head, error) + BuildWithConfig(inputRelabelerConfigs []*config.InputRelabelerConfig, numberOfShards uint16) (relabeler.Head, error) +} + +type HeadActivator interface { + Activate(headID string) error +} + +type NoOpHeadActivator struct{} + +func (NoOpHeadActivator) Activate(headID string) error { return nil } + +// Rotatable head wrapper, allows rotations. +type Rotatable struct { + storage Storage + builder HeadBuilder + headActivator HeadActivator + relabeler.Head +} + +// NewRotatable init new [*Rotatable] container head. +func NewRotatable( + head relabeler.Head, + storage Storage, + builder HeadBuilder, + headActivator HeadActivator, +) *Rotatable { + return &Rotatable{ + storage: storage, + builder: builder, + headActivator: headActivator, + Head: head, + } +} + +// Reconfigure relabeler.Head interface implementation. +func (h *Rotatable) Reconfigure( + ctx context.Context, + inputRelabelerConfigs []*config.InputRelabelerConfig, + numberOfShards uint16, +) error { + if h.Head.NumberOfShards() != numberOfShards { + return h.RotateWithConfig(inputRelabelerConfigs, numberOfShards) + } + return h.Head.Reconfigure(ctx, inputRelabelerConfigs, numberOfShards) +} + +// Rotate - relabeler.Head interface implementation. +func (h *Rotatable) Rotate() error { + h.Head.MergeOutOfOrderChunks() + + newHead, err := h.builder.Build() + if err != nil { + return err + } + + if CopySeriesOnRotate { + newHead.CopySeriesFrom(h.Head) + } + + if err = h.headActivator.Activate(newHead.ID()); err != nil { + return err + } + + if err = h.Head.CommitToWal(); err != nil { + logger.Errorf("failed to commit wal on rotation: %v", err) + } + h.Head.Stop() + + h.storage.Add(h.Head) + h.Head = newHead + return nil +} + +func (h *Rotatable) RotateWithConfig( + inputRelabelerConfigs []*config.InputRelabelerConfig, + numberOfShards uint16, +) error { + h.Head.MergeOutOfOrderChunks() + + newHead, err := h.builder.BuildWithConfig(inputRelabelerConfigs, numberOfShards) + if err != nil { + return err + } + + if err = h.headActivator.Activate(newHead.ID()); err != nil { + return err + } + + if err = h.Head.CommitToWal(); err != nil { + logger.Errorf("failed to commit wal on rotation: %v", err) + } + h.Head.Stop() + + h.storage.Add(h.Head) + h.Head = newHead + return nil +} diff --git a/pp/go/relabeler/querier/metrics.go b/pp/go/relabeler/querier/metrics.go index 493ee255fa..530ee752ca 100644 --- a/pp/go/relabeler/querier/metrics.go +++ b/pp/go/relabeler/querier/metrics.go @@ -16,10 +16,6 @@ type Metrics struct { LabelNamesDuration prometheus.Histogram LabelValuesDuration prometheus.Histogram SelectDuration *prometheus.HistogramVec - AppendDuration prometheus.Histogram - - WaitLockRotateDuration prometheus.Gauge - RotationDuration prometheus.Gauge } func NewMetrics(registerer prometheus.Registerer, source string) *Metrics { @@ -65,30 +61,5 @@ func NewMetrics(registerer prometheus.Registerer, source string) *Metrics { }, []string{"query_type"}, ), - AppendDuration: factory.NewHistogram( - prometheus.HistogramOpts{ - Name: "prompp_head_append_duration", - Help: "Append to head duration in microseconds", - Buckets: []float64{ - 50, 100, 250, 500, 750, - 1000, 2500, 5000, 7500, - 10000, 25000, 50000, 75000, - 100000, 500000, - }, - }, - ), - - WaitLockRotateDuration: factory.NewGauge( - prometheus.GaugeOpts{ - Name: "prompp_head_wait_lock_rotate_duration", - Help: "The duration of the lock wait for rotation in nanoseconds", - }, - ), - RotationDuration: factory.NewGauge( - prometheus.GaugeOpts{ - Name: "prompp_head_rotate_duration", - Help: "The duration of the rotate in nanoseconds", - }, - ), } } diff --git a/pp/go/relabeler/querier/multi.go b/pp/go/relabeler/querier/multi.go index aabe2c80cc..805f3c2444 100644 --- a/pp/go/relabeler/querier/multi.go +++ b/pp/go/relabeler/querier/multi.go @@ -12,15 +12,23 @@ import ( ) type MultiQuerier struct { - mint int64 - maxt int64 queriers []storage.Querier closer func() error } func NewMultiQuerier(queriers []storage.Querier, closer func() error) *MultiQuerier { + qs := make([]storage.Querier, 0, len(queriers)) + for _, q := range queriers { + if rawQ, ok := q.(*MultiQuerier); ok { + qs = append(qs, rawQ.queriers...) + continue + } + + qs = append(qs, q) + } + return &MultiQuerier{ - queriers: queriers, + queriers: qs, closer: closer, } } @@ -30,6 +38,10 @@ func (q *MultiQuerier) LabelValues( name string, matchers ...*labels.Matcher, ) ([]string, annotations.Annotations, error) { + if len(q.queriers) == 1 { + return q.queriers[0].LabelValues(ctx, name, matchers...) + } + labelValuesResults := make([][]string, len(q.queriers)) annotationResults := make([]annotations.Annotations, len(q.queriers)) errs := make([]error, len(q.queriers)) @@ -57,6 +69,10 @@ func (q *MultiQuerier) LabelNames( ctx context.Context, matchers ...*labels.Matcher, ) ([]string, annotations.Annotations, error) { + if len(q.queriers) == 1 { + return q.queriers[0].LabelNames(ctx, matchers...) + } + labelNamesResults := make([][]string, len(q.queriers)) annotationResults := make([]annotations.Annotations, len(q.queriers)) errs := make([]error, len(q.queriers)) @@ -94,6 +110,10 @@ func (q *MultiQuerier) Select( hints *storage.SelectHints, matchers ...*labels.Matcher, ) storage.SeriesSet { + if len(q.queriers) == 1 { + return q.queriers[0].Select(ctx, sortSeries, hints, matchers...) + } + seriesSets := make([]storage.SeriesSet, len(q.queriers)) wg := &sync.WaitGroup{} diff --git a/web/api/v1/api.go b/web/api/v1/api.go index 7210260abb..0c27c57062 100644 --- a/web/api/v1/api.go +++ b/web/api/v1/api.go @@ -193,9 +193,8 @@ type API struct { QueryEngine promql.QueryEngine ExemplarQueryable storage.ExemplarQueryable - HeadQueryable storage.Queryable // PP_CHANGES.md: rebuild on cpp - headStatusGetter HeadStatusGetter // PP_CHANGES.md: rebuild on cpp - opHandler *handler.PPHandler // PP_CHANGES.md: rebuild on cpp + receiver handler.Receiver // PP_CHANGES.md: rebuild on cpp + opHandler *handler.PPHandler // PP_CHANGES.md: rebuild on cpp scrapePoolsRetriever func(context.Context) ScrapePoolsRetriever targetRetriever func(context.Context) TargetRetriever @@ -218,9 +217,9 @@ type API struct { isAgent bool statsRenderer StatsRenderer - remoteWriteHandler http.Handler - remoteReadHandler http.Handler - otlpWriteHandler http.Handler + // remoteWriteHandler http.Handler // PP_CHANGES.md: rebuild on cpp + remoteReadHandler http.Handler + otlpWriteHandler http.Handler codecs []Codec } @@ -232,7 +231,6 @@ func NewAPI( ap storage.Appendable, eq storage.ExemplarQueryable, - hq storage.Queryable, // PP_CHANGES.md: rebuild on cpp receiver handler.Receiver, // PP_CHANGES.md: rebuild on cpp spsr func(context.Context) ScrapePoolsRetriever, @@ -265,8 +263,7 @@ func NewAPI( Queryable: q, ExemplarQueryable: eq, - HeadQueryable: hq, // PP_CHANGES.md: rebuild on cpp - headStatusGetter: receiver, // PP_CHANGES.md: rebuild on cpp + receiver: receiver, // PP_CHANGES.md: rebuild on cpp scrapePoolsRetriever: spsr, targetRetriever: tr, @@ -1663,13 +1660,14 @@ func (api *API) remoteRead(w http.ResponseWriter, r *http.Request) { } } -func (api *API) remoteWrite(w http.ResponseWriter, r *http.Request) { - if api.remoteWriteHandler != nil { - api.remoteWriteHandler.ServeHTTP(w, r) - } else { - http.Error(w, "remote write receiver needs to be enabled with --web.enable-remote-write-receiver", http.StatusNotFound) - } -} +// PP_CHANGES.md: rebuild on cpp +// func (api *API) remoteWrite(w http.ResponseWriter, r *http.Request) { +// if api.remoteWriteHandler != nil { +// api.remoteWriteHandler.ServeHTTP(w, r) +// } else { +// http.Error(w, "remote write receiver needs to be enabled with --web.enable-remote-write-receiver", http.StatusNotFound) +// } +// } func (api *API) otlpWrite(w http.ResponseWriter, r *http.Request) { if api.otlpWriteHandler != nil { diff --git a/web/api/v1/errors_test.go b/web/api/v1/errors_test.go index ebde91e59c..3191fc9e55 100644 --- a/web/api/v1/errors_test.go +++ b/web/api/v1/errors_test.go @@ -115,7 +115,6 @@ func createPrometheusAPI(q storage.SampleAndChunkQueryable) *route.Router { nil, nil, nil, - nil, func(context.Context) ScrapePoolsRetriever { return &DummyScrapePoolsRetriever{} }, func(context.Context) TargetRetriever { return &DummyTargetRetriever{} }, func(context.Context) AlertmanagerRetriever { return &DummyAlertmanagerRetriever{} }, diff --git a/web/api/v1/pp_api.go b/web/api/v1/pp_api.go index 2b3c585ea2..ac03b638b6 100644 --- a/web/api/v1/pp_api.go +++ b/web/api/v1/pp_api.go @@ -10,17 +10,11 @@ import ( "github.com/prometheus/prometheus/model/labels" "github.com/prometheus/prometheus/pp-pkg/handler/middleware" - "github.com/prometheus/prometheus/pp/go/relabeler" "github.com/prometheus/prometheus/promql" "github.com/prometheus/prometheus/promql/parser" "github.com/prometheus/prometheus/tsdb/chunkenc" ) -// HeadStatusGetter getter head status from relabeler. -type HeadStatusGetter interface { - HeadStatus(ctx context.Context, limit int) relabeler.HeadStatus -} - // Register the API's endpoints in the given router from op. func (api *API) opRegister(r *route.Router, wrapAgent func(f apiFunc) http.HandlerFunc) { r.Get("/query_head", wrapAgent(api.queryHead)) @@ -76,7 +70,7 @@ func (api *API) queryHead(r *http.Request) apiFuncResult { matchers = append(matchers, selector...) } - q, err := api.HeadQueryable.Querier(start.UnixMilli(), end.UnixMilli()) + q, err := api.receiver.HeadQuerier(ctx, start.UnixMilli(), end.UnixMilli()) if err != nil { return apiFuncResult{nil, &apiError{errorBadData, err}, nil, nil} } @@ -115,7 +109,7 @@ func (api *API) serveHeadStatus(r *http.Request) apiFuncResult { } } - return apiFuncResult{api.headStatusGetter.HeadStatus(r.Context(), limit), nil, nil, nil} + return apiFuncResult{api.receiver.HeadStatus(r.Context(), limit), nil, nil, nil} } func (api *API) opRemoteWrite(middlewares ...middleware.Middleware) http.HandlerFunc { diff --git a/web/web.go b/web/web.go index a2b4ed62bc..51115e0ce7 100644 --- a/web/web.go +++ b/web/web.go @@ -326,7 +326,7 @@ func New(logger log.Logger, o *Options, receiver handler.Receiver) *Handler { // } // PP_CHANGES.md: rebuild on cpp start - h.apiV1 = api_v1.NewAPI(h.queryEngine, h.storage, app, h.exemplarStorage, receiver.HeadQueryable(), receiver, factorySPr, factoryTr, factoryAr, + h.apiV1 = api_v1.NewAPI(h.queryEngine, h.storage, app, h.exemplarStorage, receiver, factorySPr, factoryTr, factoryAr, func() config.Config { h.mtx.RLock() defer h.mtx.RUnlock() From 0bd578cab74bef5390c5bdd2912029706ada0b43 Mon Sep 17 00:00:00 2001 From: Alexandr Yudin Date: Mon, 4 Aug 2025 08:12:07 +0000 Subject: [PATCH 02/96] for save --- pp-pkg/.golangci.yml | 254 ++++++++++++++ pp-pkg/configadapter/headkeeper.go | 28 ++ pp-pkg/configadapter/remotewriter.go | 54 +++ pp-pkg/storage/adapter.go | 158 +++++++++ pp-pkg/storage/appender.go | 30 ++ pp/.golangci.yml | 50 ++- pp/go/storage/block/chunk_writer.go | 253 ++++++++++++++ pp/go/storage/block/file_writer.go | 54 +++ pp/go/storage/block/writer.go | 323 ++++++++++++++++++ pp/go/storage/catalog/catalog.go | 1 + pp/go/storage/head/builder/builder.go | 1 + pp/go/storage/head/container/weighted.go | 65 ++++ pp/go/storage/head/container/weighted_test.go | 111 ++++++ pp/go/storage/head/keeper/keeper.go | 153 +++++++++ pp/go/storage/head/manager/manager.go | 152 +++++++++ pp/go/storage/head/manager/timer.go | 121 +++++++ pp/go/storage/head/shard/data_storage.go | 40 +++ pp/go/storage/head/shard/lss.go | 67 ++++ pp/go/storage/head/shard/shard.go | 73 ++++ pp/go/storage/head/shard/wal.go | 275 +++++++++++++++ pp/go/storage/interface.go | 150 ++++++++ pp/go/storage/logger/logger.go | 19 ++ pp/go/storage/models.go | 54 +++ pp/go/storage/task.go | 200 +++++++++++ 24 files changed, 2673 insertions(+), 13 deletions(-) create mode 100644 pp-pkg/.golangci.yml create mode 100644 pp-pkg/configadapter/headkeeper.go create mode 100644 pp-pkg/configadapter/remotewriter.go create mode 100644 pp-pkg/storage/adapter.go create mode 100644 pp-pkg/storage/appender.go create mode 100644 pp/go/storage/block/chunk_writer.go create mode 100644 pp/go/storage/block/file_writer.go create mode 100644 pp/go/storage/block/writer.go create mode 100644 pp/go/storage/catalog/catalog.go create mode 100644 pp/go/storage/head/builder/builder.go create mode 100644 pp/go/storage/head/container/weighted.go create mode 100644 pp/go/storage/head/container/weighted_test.go create mode 100644 pp/go/storage/head/keeper/keeper.go create mode 100644 pp/go/storage/head/manager/manager.go create mode 100644 pp/go/storage/head/manager/timer.go create mode 100644 pp/go/storage/head/shard/data_storage.go create mode 100644 pp/go/storage/head/shard/lss.go create mode 100644 pp/go/storage/head/shard/shard.go create mode 100644 pp/go/storage/head/shard/wal.go create mode 100644 pp/go/storage/interface.go create mode 100644 pp/go/storage/logger/logger.go create mode 100644 pp/go/storage/models.go create mode 100644 pp/go/storage/task.go diff --git a/pp-pkg/.golangci.yml b/pp-pkg/.golangci.yml new file mode 100644 index 0000000000..06738ffb0c --- /dev/null +++ b/pp-pkg/.golangci.yml @@ -0,0 +1,254 @@ +--- +# This file contains all available configuration options +# with their default values. + +# options for analysis running +run: + # default concurrency is a available CPU number + concurrency: 4 + + # # timeout for analysis, e.g. 30s, 5m, default is 1m + # deadline: 10m + + # exit code when at least one issue was found, default is 1 + issues-exit-code: 1 + + # include test files or not, default is true + tests: true +# output configuration options +output: + # # colored-line-number|line-number|json|tab|checkstyle|code-climate, default is "colored-line-number" + # format: colored-line-number + + + # print lines of code with issue, default is true + print-issued-lines: true + + # print linter name in the end of issue text, default is true + print-linter-name: true + +# all available settings of specific linters +linters-settings: + govet: + # # report about shadowed variables + # check-shadowing: true + # settings per analyzer + settings: + printf: # analyzer name, run `go tool vet help` to see all analyzers + funcs: # run `go tool vet help printf` to see available settings for `printf` analyzer + - (github.com/golangci/golangci-lint/pkg/logutils.Log).Infof + - (github.com/golangci/golangci-lint/pkg/logutils.Log).Warnf + - (github.com/golangci/golangci-lint/pkg/logutils.Log).Errorf + - (github.com/golangci/golangci-lint/pkg/logutils.Log).Fatalf + shadow: + # Whether to be strict about shadowing; can be noisy. + # Default: false + strict: true + gofmt: + # simplify code: gofmt with `-s` option, true by default + simplify: true + goimports: + # put imports beginning with prefix after 3rd-party packages; + # it's a comma-separated list of prefixes + local-prefixes: fox.flant.com + dupl: + # tokens count to trigger issue, 150 by default + threshold: 100 + goconst: + # minimal length of string constant, 3 by default + min-len: 3 + # minimal occurrences count to trigger, 3 by default + min-occurrences: 3 + misspell: + # Correct spellings using locale preferences for US or UK. + # Default is to use a neutral variety of English. + # Setting locale to US will correct the British spelling of 'colour' to 'color'. + locale: US + ignore-words: + - someword + # unused: + # # treat code as a program (not a library) and report unused exported identifiers; default is false. + # # XXX: if you enable this setting, unused will report a lot of false-positives in text editors: + # # if it's called for subdir of a project it can't find funcs usages. All text editor integrations + # # with golangci-lint call it on a directory with the changed file. + # check-exported: false + # # Mark all exported fields as used. + # # default: true + # exported-fields-are-used: false + nakedret: + # make an issue if func has more lines of code than this setting and it has naked returns; default is 30 + max-func-lines: 30 + gocritic: + # Enable multiple checks by tags, run `GL_DEBUG=gocritic golangci-lint` run to see all tags and checks. + # Empty list by default. See https://github.com/go-critic/go-critic#usage -> section "Tags". + enabled-tags: + - style + - performance + settings: # settings passed to gocritic + captLocal: # must be valid enabled check name + paramsOnly: true + revive: + ignore-generated-header: true + rules: + - name: context-keys-type + - name: time-naming + - name: var-declaration + - name: unexported-return + - name: errorf + - name: blank-imports + - name: context-as-argument + - name: dot-imports + - name: error-return + - name: error-strings + - name: error-naming + - name: exported + - name: if-return + - name: increment-decrement + - name: range + - name: receiver-naming + - name: indent-error-flow + - name: argument-limit + arguments: + - 10 + - name: cyclomatic + arguments: + - 10 + - name: empty-block + - name: superfluous-else + - name: confusing-naming + - name: get-return + - name: modifies-parameter + - name: confusing-results + - name: deep-exit + - name: unused-parameter + - name: unreachable-code + - name: add-constant + arguments: + - allowInts: "0,1" + allowStrs: '"","/"' + - name: flag-parameter + - name: unnecessary-stmt + - name: struct-tag + - name: modifies-value-receiver + - name: constant-logical-expr + - name: bool-literal-in-expr + - name: redefines-builtin-id + - name: function-result-limit + arguments: + - 3 + - name: imports-blocklist # imports-blacklist + arguments: + - github.com/davecgh/go-spew/spew + - name: range-val-in-closure + - name: range-val-address + - name: waitgroup-by-value + - name: atomic + - name: empty-lines + - name: line-length-limit + arguments: + - 120 + - name: duplicated-imports + - name: import-shadowing + - name: bare-return + - name: unused-receiver + - name: unhandled-error + - name: cognitive-complexity + arguments: + - 20 + - name: string-of-int + - name: early-return + - name: unconditional-recursion + - name: identical-branches + - name: function-length + arguments: + - 30 + - 50 + testifylint: + disable: + - float-compare + - go-require + enable: + - bool-compare + - compares + - empty + - error-is-as + - error-nil + - expected-actual + - len + - require-error + - suite-dont-use-pkg + - suite-extra-assert-call + +linters: + disable-all: true + enable: + - bodyclose # checks whether HTTP response body is closed successfully [fast: false, auto-fix: false] + - dupl # Tool for code clone detection [fast: true, auto-fix: false] + - goconst # Finds repeated strings that could be replaced by a constant [fast: true, auto-fix: false] + - gocritic # The most opinionated Go source code linter [fast: true, auto-fix: false] + - godox # Tool for detection of FIXME, TODO and other comment keywords [fast: true, auto-fix: false] + - gofmt + - goimports # Goimports does everything that gofmt does. Additionally it checks unused imports [fast: true, auto-fix: true] + - gosec # Inspects source code for security problems [fast: true, auto-fix: false] + - gosimple # Linter for Go source code that specializes in simplifying a code [fast: false, auto-fix: false] + - govet # Vet examines Go source code and reports suspicious constructs, such as Printf calls whose arguments do not align with the format string [fast: false, auto-fix: false] + - ineffassign # Detects when assignments to existing variables are not used [fast: true, auto-fix: false] + - misspell # Finds commonly misspelled English words in comments [fast: true, auto-fix: true] + - nakedret # Finds naked returns in functions greater than a specified function length [fast: true, auto-fix: false] + - staticcheck # Staticcheck is a go vet on steroids, applying a ton of static analysis checks [fast: false, auto-fix: false] + - typecheck # Like the front-end of a Go compiler, parses and type-checks Go code [fast: true, auto-fix: false] + - unconvert # Remove unnecessary type conversions [fast: true, auto-fix: false] + - unused # Checks Go code for unused constants, variables, functions and types [fast: false, auto-fix: false] + - revive + - testifylint # Checks usage of github.com/stretchr/testify. + fast: false + +issues: + # List of regexps of issue texts to exclude, empty list by default. + # But independently from this option we use default exclude patterns, + # it can be disabled by `exclude-use-default: false`. To list all + # excluded by default patterns execute `golangci-lint run --help` + exclude: + - abcdef + + # Excluding configuration per-path, per-linter, per-text and per-source + exclude-rules: + # Exclude some linters from running on tests files. + - path: _test\.go + linters: + - gocyclo + - errcheck + - dupl + - gosec + - scopelint + + - path: _test\.go + linters: + - revive + text: "add-constant" + + - path: _test\.go + linters: + - revive + text: "line-length-limit" + + - path: _test\.go + linters: + - revive + text: "function-length" + + # Exclude some staticcheck messages + - linters: + - staticcheck + text: "SA9003:" # Empty body in an if or else branch + + # Exclude some stylecheck messages + - linters: + - stylecheck + text: "ST1000:" # Incorrect or missing package comment + + # Independently from option `exclude` we use default exclude patterns, + # it can be disabled by this option. To list all + # excluded by default patterns execute `golangci-lint run --help`. + # Default value for this option is true. + exclude-use-default: false diff --git a/pp-pkg/configadapter/headkeeper.go b/pp-pkg/configadapter/headkeeper.go new file mode 100644 index 0000000000..0867f9f29e --- /dev/null +++ b/pp-pkg/configadapter/headkeeper.go @@ -0,0 +1,28 @@ +package configadapter + +import ( + "context" + + prom_config "github.com/prometheus/prometheus/config" + "github.com/prometheus/prometheus/pp/go/storage/head/manager" +) + +// DefaultNumberOfShards default value for number of shards [pp_storage.Head]. +var DefaultNumberOfShards uint16 = 2 + +// HeadKeeperApplyConfig returns func-adapter for apply config on [headkeeper.HeadKeeper]. +func HeadKeeperApplyConfig(ctx context.Context, hk *manager.Manager) func(cfg *prom_config.Config) error { + return func(cfg *prom_config.Config) error { + rCfg, err := cfg.GetReceiverConfig() + if err != nil { + return err + } + + numberOfShards := rCfg.NumberOfShards + if numberOfShards == 0 { + numberOfShards = DefaultNumberOfShards + } + + return hk.ApplyConfig(ctx, rCfg.Configs, numberOfShards) + } +} diff --git a/pp-pkg/configadapter/remotewriter.go b/pp-pkg/configadapter/remotewriter.go new file mode 100644 index 0000000000..6beb74a5db --- /dev/null +++ b/pp-pkg/configadapter/remotewriter.go @@ -0,0 +1,54 @@ +package configadapter + +import ( + "crypto/md5" // #nosec G501 // cryptographic strength is not required + "encoding/hex" + "time" + + "gopkg.in/yaml.v2" + + "github.com/prometheus/prometheus/config" + "github.com/prometheus/prometheus/pp/go/relabeler/remotewriter" +) + +// RemoteWriterApplyConfig returns func-adapter for apply config on [remotewriter.RemoteWriter]. +func RemoteWriterApplyConfig(remoteWriter *remotewriter.RemoteWriter) func(promConfig *config.Config) error { + return func(promConfig *config.Config) error { + destinationConfigs := make([]remotewriter.DestinationConfig, 0, len(promConfig.RemoteWriteConfigs)) + for _, rwc := range promConfig.RemoteWriteConfigs { + if !rwc.IsPrometheusProtocol() { + continue + } + + if rwc.Name == "" { + rwcHash, err := toHash(rwc) + if err != nil { + return err + } + rwc.Name = rwcHash + } + + if rwc.QueueConfig.SampleAgeLimit == 0 { + rwc.QueueConfig.SampleAgeLimit = remotewriter.DefaultSampleAgeLimit + } + + destinationConfigs = append(destinationConfigs, remotewriter.DestinationConfig{ + RemoteWriteConfig: rwc.RemoteWriteConfig, + ExternalLabels: promConfig.GlobalConfig.ExternalLabels, + ReadTimeout: time.Duration(promConfig.GlobalConfig.ScrapeInterval), + }, + ) + } + return remoteWriter.ApplyConfig(destinationConfigs...) + } +} + +// toHash used for hashing configs and diff'ing hashes in ApplyConfig. +func toHash(data interface{}) (string, error) { + bytes, err := yaml.Marshal(data) + if err != nil { + return "", err + } + hash := md5.Sum(bytes) // #nosec G401 // cryptographic strength is not required + return hex.EncodeToString(hash[:]), nil +} diff --git a/pp-pkg/storage/adapter.go b/pp-pkg/storage/adapter.go new file mode 100644 index 0000000000..7670c357e5 --- /dev/null +++ b/pp-pkg/storage/adapter.go @@ -0,0 +1,158 @@ +package storage + +import ( + "context" + + "github.com/prometheus/common/model" + "github.com/prometheus/prometheus/pp/go/cppbridge" + "github.com/prometheus/prometheus/pp/go/relabeler/querier" + pp_storage "github.com/prometheus/prometheus/pp/go/storage" + "github.com/prometheus/prometheus/storage" +) + +var _ storage.Storage = (*Adapter)(nil) + +// Adapter for implementing the [Queryable] interface and append data. +type Adapter struct { + // +} + +// AppendHashdex append incoming [cppbridge.HashdexContent] to [pp_storage.Head]. +func (ar *Adapter) AppendHashdex( + ctx context.Context, + hashdex cppbridge.ShardedData, + relabelerID string, + commitToWal bool, +) error { + // if rr.haTracker.IsDrop(hashdex.Cluster(), hashdex.Replica()) { + // return nil + // } + // incomingData := &relabeler.IncomingData{Hashdex: hashdex} + // _, err := rr.activeHead.Append(ctx, incomingData, nil, relabelerID, commitToWal) + return nil +} + +// AppendScraperHashdex append ScraperHashdex data to [pp_storage.Head]. +func (ar *Adapter) AppendScraperHashdex( + ctx context.Context, + hashdex cppbridge.ShardedData, + state *cppbridge.State, + relabelerID string, + commitToWal bool, +) (cppbridge.RelabelerStats, error) { + // return rr.activeHead.Append( + // ctx, + // &relabeler.IncomingData{Hashdex: hashdex}, + // state, + // relabelerID, + // commitToWal, + // ) + + return cppbridge.RelabelerStats{}, nil +} + +// AppendSnappyProtobuf append compressed via snappy Protobuf data to [pp_storage.Head]. +func (ar *Adapter) AppendSnappyProtobuf( + ctx context.Context, + compressedData pp_storage.ProtobufData, + relabelerID string, + commitToWal bool, +) error { + // hx, err := cppbridge.NewWALSnappyProtobufHashdex(compressedData.Bytes(), rr.hashdexLimits) + // compressedData.Destroy() + // if err != nil { + // return err + // } + + // if rr.haTracker.IsDrop(hx.Cluster(), hx.Replica()) { + // return nil + // } + + // incomingData := &relabeler.IncomingData{Hashdex: hx} + // _, err = rr.activeHead.Append(ctx, incomingData, nil, relabelerID, commitToWal) + return nil +} + +// AppendTimeSeries append TimeSeries data to [pp_storage.Head]. +func (ar *Adapter) AppendTimeSeries( + ctx context.Context, + data pp_storage.TimeSeriesBatch, + state *cppbridge.State, + relabelerID string, + commitToWal bool, +) (cppbridge.RelabelerStats, error) { + // hx, err := rr.hashdexFactory.GoModel(data.TimeSeries(), rr.hashdexLimits) + // if err != nil { + // data.Destroy() + // return cppbridge.RelabelerStats{}, err + // } + + // if rr.haTracker.IsDrop(hx.Cluster(), hx.Replica()) { + // data.Destroy() + // return cppbridge.RelabelerStats{}, nil + // } + // incomingData := &relabeler.IncomingData{Hashdex: hx, Data: data} + // return rr.activeHead.Append( + // ctx, + // incomingData, + // state, + // relabelerID, + // commitToWal, + // ) + + return cppbridge.RelabelerStats{}, nil +} + +// Appender create a new [storage.Appender] for [pp_storage.Head]. +func (ar *Adapter) Appender(ctx context.Context) storage.Appender { + // return newPromAppender(ctx, rr, prom_config.TransparentRelabeler) + return nil +} + +// ChunkQuerier provides querying access over time series data of a fixed time range. +// Returns new Chunk Querier that merges results of given primary and secondary chunk queriers. +func (ar *Adapter) ChunkQuerier(mint, maxt int64) (storage.ChunkQuerier, error) { + // TODO + return storage.NewMergeChunkQuerier( + nil, + []storage.ChunkQuerier{}, + storage.NewConcatenatingChunkSeriesMerger(), + ), nil +} + +// Close closes the storage and all its underlying resources. +// Implements the [storage.Storage] interface. +func (*Adapter) Close() error { + return nil +} + +// HeadQuerier returns [storage.Querier] from active head. +func (ar *Adapter) HeadQuerier(ctx context.Context, mint, maxt int64) (storage.Querier, error) { + // TODO + return nil, nil +} + +// HeadStatus returns stats of Head. +func (ar *Adapter) HeadStatus(ctx context.Context, limit int) pp_storage.HeadStatus { + // TODO + return pp_storage.HeadStatus{} +} + +// Querier calls f() with the given parameters. +// Returns a [querier.MultiQuerier] combining of primary and secondary queriers. +func (ar *Adapter) Querier(mint, maxt int64) (storage.Querier, error) { + // TODO + return querier.NewMultiQuerier([]storage.Querier{}, nil), nil +} + +// RelabelerIDIsExist check on exist relabelerID. +func (ar *Adapter) RelabelerIDIsExist(relabelerID string) bool { + // TODO + return true +} + +// StartTime returns the oldest timestamp stored in the storage. +// Implements the [storage.Storage] interface. +func (*Adapter) StartTime() (int64, error) { + return int64(model.Latest), nil +} diff --git a/pp-pkg/storage/appender.go b/pp-pkg/storage/appender.go new file mode 100644 index 0000000000..e89550a871 --- /dev/null +++ b/pp-pkg/storage/appender.go @@ -0,0 +1,30 @@ +package storage + +import ( + "context" + + "github.com/prometheus/prometheus/pp/go/model" +) + +// timeSeriesBatch implementation buffer of [ppstorage.TimeSeriesData]. +type timeSeriesBatch struct { + timeSeries []model.TimeSeries +} + +// TimeSeries returns slice [model.TimeSeries]. +func (d *timeSeriesBatch) TimeSeries() []model.TimeSeries { + return d.timeSeries +} + +// Destroy buffered data. +func (d *timeSeriesBatch) Destroy() { + d.timeSeries = nil +} + +// TimeSeriesAppender +type TimeSeriesAppender struct { + ctx context.Context + // receiver *Receiver + relabelerID string + data *timeSeriesBatch +} diff --git a/pp/.golangci.yml b/pp/.golangci.yml index 9572d7ffe2..06738ffb0c 100644 --- a/pp/.golangci.yml +++ b/pp/.golangci.yml @@ -7,8 +7,8 @@ run: # default concurrency is a available CPU number concurrency: 4 - # timeout for analysis, e.g. 30s, 5m, default is 1m - deadline: 10m + # # timeout for analysis, e.g. 30s, 5m, default is 1m + # deadline: 10m # exit code when at least one issue was found, default is 1 issues-exit-code: 1 @@ -17,8 +17,9 @@ run: tests: true # output configuration options output: - # colored-line-number|line-number|json|tab|checkstyle|code-climate, default is "colored-line-number" - format: colored-line-number + # # colored-line-number|line-number|json|tab|checkstyle|code-climate, default is "colored-line-number" + # format: colored-line-number + # print lines of code with issue, default is true print-issued-lines: true @@ -29,8 +30,8 @@ output: # all available settings of specific linters linters-settings: govet: - # report about shadowed variables - check-shadowing: true + # # report about shadowed variables + # check-shadowing: true # settings per analyzer settings: printf: # analyzer name, run `go tool vet help` to see all analyzers @@ -39,6 +40,10 @@ linters-settings: - (github.com/golangci/golangci-lint/pkg/logutils.Log).Warnf - (github.com/golangci/golangci-lint/pkg/logutils.Log).Errorf - (github.com/golangci/golangci-lint/pkg/logutils.Log).Fatalf + shadow: + # Whether to be strict about shadowing; can be noisy. + # Default: false + strict: true gofmt: # simplify code: gofmt with `-s` option, true by default simplify: true @@ -61,12 +66,15 @@ linters-settings: locale: US ignore-words: - someword - unused: - # treat code as a program (not a library) and report unused exported identifiers; default is false. - # XXX: if you enable this setting, unused will report a lot of false-positives in text editors: - # if it's called for subdir of a project it can't find funcs usages. All text editor integrations - # with golangci-lint call it on a directory with the changed file. - check-exported: false + # unused: + # # treat code as a program (not a library) and report unused exported identifiers; default is false. + # # XXX: if you enable this setting, unused will report a lot of false-positives in text editors: + # # if it's called for subdir of a project it can't find funcs usages. All text editor integrations + # # with golangci-lint call it on a directory with the changed file. + # check-exported: false + # # Mark all exported fields as used. + # # default: true + # exported-fields-are-used: false nakedret: # make an issue if func has more lines of code than this setting and it has naked returns; default is 30 max-func-lines: 30 @@ -128,7 +136,7 @@ linters-settings: - name: function-result-limit arguments: - 3 - - name: imports-blacklist + - name: imports-blocklist # imports-blacklist arguments: - github.com/davecgh/go-spew/spew - name: range-val-in-closure @@ -155,6 +163,21 @@ linters-settings: arguments: - 30 - 50 + testifylint: + disable: + - float-compare + - go-require + enable: + - bool-compare + - compares + - empty + - error-is-as + - error-nil + - expected-actual + - len + - require-error + - suite-dont-use-pkg + - suite-extra-assert-call linters: disable-all: true @@ -177,6 +200,7 @@ linters: - unconvert # Remove unnecessary type conversions [fast: true, auto-fix: false] - unused # Checks Go code for unused constants, variables, functions and types [fast: false, auto-fix: false] - revive + - testifylint # Checks usage of github.com/stretchr/testify. fast: false issues: diff --git a/pp/go/storage/block/chunk_writer.go b/pp/go/storage/block/chunk_writer.go new file mode 100644 index 0000000000..ed3bb0f509 --- /dev/null +++ b/pp/go/storage/block/chunk_writer.go @@ -0,0 +1,253 @@ +package block + +import ( + "bufio" + "encoding/binary" + "errors" + "fmt" + "hash" + "hash/crc32" + "io" + "os" + "path/filepath" + + "github.com/prometheus/prometheus/tsdb/chunks" + "github.com/prometheus/prometheus/tsdb/fileutil" +) + +const ( + chunksFormatV1 = 1 +) + +type ChunkMetadata struct { + MinT int64 + MaxT int64 + Ref uint64 +} + +type ChunkWriter struct { + dirFile *os.File + files []*os.File + wbuf *bufio.Writer + n int64 + crc32 hash.Hash + segmentSize int64 + buf [binary.MaxVarintLen32]byte +} + +func NewChunkWriter(dir string, segmentSize int64) (*ChunkWriter, error) { + if segmentSize < 0 { + segmentSize = DefaultChunkSegmentSize + } + + if err := os.MkdirAll(dir, 0o777); err != nil { + return nil, fmt.Errorf("failed to create all dirs: %w", err) + } + + dirFile, err := fileutil.OpenDir(dir) + if err != nil { + return nil, fmt.Errorf("failed to open dir: %w", err) + } + + return &ChunkWriter{ + dirFile: dirFile, + crc32: crc32.New(crc32.MakeTable(crc32.Castagnoli)), + segmentSize: segmentSize, + }, nil +} + +func (w *ChunkWriter) Write(chunk Chunk) (meta ChunkMetadata, err error) { + // calculate chunk size + chunkSize := int64(chunks.MaxChunkLengthFieldSize) + chunkSize += chunks.ChunkEncodingSize + chunkSize += int64(len(chunk.Bytes())) + chunkSize += crc32.Size + + // check segment boundaries and cut if needed + if w.n == 0 || w.n+chunkSize > w.segmentSize { + if err = w.cut(); err != nil { + return meta, fmt.Errorf("failed to cut file: %w", err) + } + } + + // write chunk + return w.writeChunk(chunk) +} + +func (w *ChunkWriter) writeChunk(chunk Chunk) (meta ChunkMetadata, err error) { + meta.Ref = uint64(chunks.NewBlockChunkRef(uint64(w.seq()), uint64(w.n))) + + n := binary.PutUvarint(w.buf[:], uint64(len(chunk.Bytes()))) + if err = w.write(w.buf[:n]); err != nil { + return meta, err + } + + w.buf[0] = byte(chunk.Encoding()) + if err = w.write(w.buf[:1]); err != nil { + return meta, err + } + + if err = w.write(chunk.Bytes()); err != nil { + return meta, err + } + + w.crc32.Reset() + + buf := append(w.buf[:0], byte(chunk.Encoding())) + if _, err = w.crc32.Write(buf[:1]); err != nil { + return meta, err + } + + if _, err = w.crc32.Write(chunk.Bytes()); err != nil { + return meta, err + } + + if err = w.write(w.crc32.Sum(w.buf[:0])); err != nil { + return meta, err + } + + meta.MinT = chunk.MinT() + meta.MaxT = chunk.MaxT() + + return meta, nil +} + +func (w *ChunkWriter) Close() (err error) { + if err = w.finalizeTail(); err != nil { + return fmt.Errorf("failed to finalize tail on close: %w", err) + } + + return w.dirFile.Close() +} + +func (w *ChunkWriter) write(b []byte) error { + n, err := w.wbuf.Write(b) + w.n += int64(n) + return err +} + +func (w *ChunkWriter) seq() int { + return len(w.files) - 1 +} + +func (w *ChunkWriter) tail() *os.File { + if len(w.files) == 0 { + return nil + } + return w.files[len(w.files)-1] +} + +// finalizeTail writes all pending data to the current tail file, +// truncates its size, and closes it. +func (w *ChunkWriter) finalizeTail() error { + tf := w.tail() + if tf == nil { + return nil + } + + if err := w.wbuf.Flush(); err != nil { + return err + } + + if err := tf.Sync(); err != nil { + return err + } + // As the file was pre-allocated, we truncate any superfluous zero bytes. + off, err := tf.Seek(0, io.SeekCurrent) + if err != nil { + return err + } + if err := tf.Truncate(off); err != nil { + return err + } + + return tf.Close() +} + +func (w *ChunkWriter) cut() error { + // Sync current tail to disk and close. + if err := w.finalizeTail(); err != nil { + return err + } + + n, f, _, err := cutSegmentFile(w.dirFile, w.seq(), chunks.MagicChunks, chunksFormatV1, w.segmentSize) + if err != nil { + return err + } + w.n = int64(n) + + w.files = append(w.files, f) + if w.wbuf != nil { + w.wbuf.Reset(f) + } else { + w.wbuf = bufio.NewWriterSize(f, 8*1024*1024) + } + + return nil +} + +func cutSegmentFile(dirFile *os.File, currentSeq int, magicNumber uint32, chunksFormat byte, allocSize int64) (headerSize int, newFile *os.File, seq int, returnErr error) { + p, seq, err := nextSequenceFile(dirFile.Name(), currentSeq) + if err != nil { + return 0, nil, 0, fmt.Errorf("next sequence file: %w", err) + } + ptmp := p + ".tmp" + f, err := os.Create(ptmp) + if err != nil { + return 0, nil, 0, fmt.Errorf("open temp file: %w", err) + } + defer func() { + if returnErr != nil { + if f != nil { + returnErr = errors.Join(returnErr, f.Close()) + } + // Calling RemoveAll on a non-existent file does not return error. + returnErr = errors.Join(returnErr, os.RemoveAll(ptmp)) + } + }() + if allocSize > 0 { + if err = fileutil.Preallocate(f, allocSize, true); err != nil { + return 0, nil, 0, fmt.Errorf("preallocate: %w", err) + } + } + + if err = dirFile.Sync(); err != nil { + return 0, nil, 0, fmt.Errorf("sync directory: %w", err) + } + + // Write header metadata for new file. + metab := make([]byte, chunks.SegmentHeaderSize) + binary.BigEndian.PutUint32(metab[:chunks.MagicChunksSize], magicNumber) + metab[4] = chunksFormat + + n, err := f.Write(metab) + if err != nil { + return 0, nil, 0, fmt.Errorf("write header: %w", err) + } + if err := f.Close(); err != nil { + return 0, nil, 0, fmt.Errorf("close temp file: %w", err) + } + f = nil + + if err := fileutil.Rename(ptmp, p); err != nil { + return 0, nil, 0, fmt.Errorf("replace file: %w", err) + } + + f, err = os.OpenFile(p, os.O_WRONLY, 0o666) + if err != nil { + return 0, nil, 0, fmt.Errorf("open final file: %w", err) + } + // Skip header for further writes. + if _, err := f.Seek(int64(n), 0); err != nil { + return 0, nil, 0, fmt.Errorf("seek in final file: %w", err) + } + return n, f, seq, nil +} + +func nextSequenceFile(dir string, currentSeq int) (string, int, error) { + return segmentFile(dir, currentSeq+1), currentSeq + 1, nil +} + +func segmentFile(baseDir string, index int) string { + return filepath.Join(baseDir, fmt.Sprintf("%0.6d", index)) +} diff --git a/pp/go/storage/block/file_writer.go b/pp/go/storage/block/file_writer.go new file mode 100644 index 0000000000..1b93d572f4 --- /dev/null +++ b/pp/go/storage/block/file_writer.go @@ -0,0 +1,54 @@ +package block + +import ( + "bufio" + "fmt" + "os" + "path/filepath" + + "github.com/prometheus/prometheus/tsdb/fileutil" +) + +type FileWriter struct { + file *os.File + writeBuffer *bufio.Writer +} + +func NewFileWriter(fileName string) (*FileWriter, error) { + dir := filepath.Dir(fileName) + df, err := fileutil.OpenDir(dir) + if err != nil { + return nil, fmt.Errorf("failed to open parent dir {%s}: %w", dir, err) + } + defer func() { _ = df.Close() }() + + if err := os.RemoveAll(fileName); err != nil { + return nil, fmt.Errorf("failed to cleanup {%s}: %w", fileName, err) + } + + indexFile, err := os.OpenFile(fileName, os.O_CREATE|os.O_RDWR, 0o666) + if err != nil { + return nil, fmt.Errorf(" failed to open file {%s}: %w", fileName, err) + } + + return &FileWriter{ + file: indexFile, + writeBuffer: bufio.NewWriterSize(indexFile, 1<<22), + }, nil +} + +func (w *FileWriter) Write(p []byte) (n int, err error) { + return w.writeBuffer.Write(p) +} + +func (w *FileWriter) Close() error { + if err := w.writeBuffer.Flush(); err != nil { + return fmt.Errorf("failed to flush write buffer: %w", err) + } + + if err := w.file.Sync(); err != nil { + return fmt.Errorf("failed to sync index file: %w", err) + } + + return w.file.Close() +} diff --git a/pp/go/storage/block/writer.go b/pp/go/storage/block/writer.go new file mode 100644 index 0000000000..1e9fc95f4e --- /dev/null +++ b/pp/go/storage/block/writer.go @@ -0,0 +1,323 @@ +package block + +import ( + "crypto/rand" + "encoding/json" + "errors" + "fmt" + "io" + "math" + "os" + "path/filepath" + "time" + + "github.com/oklog/ulid" + "github.com/prometheus/client_golang/prometheus" + "github.com/prometheus/prometheus/pp/go/util" + "github.com/prometheus/prometheus/tsdb" + "github.com/prometheus/prometheus/tsdb/chunkenc" + "github.com/prometheus/prometheus/tsdb/fileutil" +) + +const ( + // DefaultChunkSegmentSize is the default chunks segment size. + DefaultChunkSegmentSize = 512 * 1024 * 1024 + // DefaultBlockDuration is the default block duration. + DefaultBlockDuration = 2 * time.Hour + tmpForCreationBlockDirSuffix = ".tmp-for-creation" + indexFilename = "index" + metaFilename = "meta.json" + metaVersion1 = 1 +) + +type HBlockWriter[T any] interface { + Write(block T) error +} + +var _ HBlockWriter[Block] = (*BlockWriter)(nil) + +type BlockWriter struct { + dataDir string + maxBlockChunkSegmentSize int64 + blockDurationMs int64 + blockWriteDuration *prometheus.GaugeVec +} + +func NewBlockWriter( + dataDir string, + maxBlockChunkSegmentSize int64, + blockDuration time.Duration, + registerer prometheus.Registerer, +) *BlockWriter { + factory := util.NewUnconflictRegisterer(registerer) + return &BlockWriter{ + dataDir: dataDir, + maxBlockChunkSegmentSize: maxBlockChunkSegmentSize, + blockDurationMs: blockDuration.Milliseconds(), + blockWriteDuration: factory.NewGaugeVec(prometheus.GaugeOpts{ + Name: "prompp_block_write_duration", + Help: "Block write duration in milliseconds.", + }, []string{"block_id"}), + } +} + +type Chunk interface { + MinT() int64 + MaxT() int64 + SeriesID() uint32 + Encoding() chunkenc.Encoding + SampleCount() uint8 + Bytes() []byte +} + +type ChunkIterator interface { + Next() bool + At() Chunk +} + +type IndexWriter interface { + WriteSeriesTo(uint32, []ChunkMetadata, io.Writer) (int64, error) + WriteRestTo(io.Writer) (int64, error) +} + +type Block interface { + TimeBounds() (minT, maxT int64) + ChunkIterator(minT, maxT int64) ChunkIterator + IndexWriter() IndexWriter +} + +func (w *BlockWriter) Write(block Block) error { + blockMinT, blockMaxT := block.TimeBounds() + + quantStart := (blockMinT / w.blockDurationMs) * w.blockDurationMs + for ; quantStart <= blockMaxT; quantStart += w.blockDurationMs { + minT, maxT := quantStart, quantStart+w.blockDurationMs-1 + if minT < blockMinT { + minT = blockMinT + } + if maxT > blockMaxT { + maxT = blockMaxT + } + if err := w.write(block, minT, maxT); err != nil { + return err + } + } + + return nil +} + +func (w *BlockWriter) write(block Block, minT, maxT int64) (err error) { + start := time.Now() + uid := ulid.MustNew(ulid.Now(), rand.Reader) + dir := filepath.Join(w.dataDir, uid.String()) + tmp := dir + tmpForCreationBlockDirSuffix + var closers []io.Closer + defer func() { + err = errors.Join(err, closeAll(closers...)) + if cleanUpErr := os.RemoveAll(tmp); err != nil { + // todo: log error + _ = cleanUpErr + } + }() + + if err = os.RemoveAll(tmp); err != nil { + return fmt.Errorf("failed to cleanup tmp directory {%s}: %w", tmp, err) + } + + if err = os.MkdirAll(tmp, 0o777); err != nil { + return fmt.Errorf("failed to create tmp directory {%s}: %w", tmp, err) + } + + chunkw, err := NewChunkWriter(chunkDir(tmp), w.maxBlockChunkSegmentSize) + if err != nil { + return fmt.Errorf("failed to create chunk writer: %w", err) + } + closers = append(closers, chunkw) + + indexFileWriter, err := NewFileWriter(filepath.Join(tmp, indexFilename)) + if err != nil { + return fmt.Errorf("failed to create index file writer: %w", err) + } + + closers = append(closers, indexFileWriter) + indexWriter := block.IndexWriter() + + chunkIterator := block.ChunkIterator(minT, maxT) + var chunksMetadata []ChunkMetadata + var chunkMetadata ChunkMetadata + var previousSeriesID uint32 = math.MaxUint32 + var chunk Chunk + + writeSeries := func() error { + if len(chunksMetadata) == 0 { + return nil + } + _, err = indexWriter.WriteSeriesTo(previousSeriesID, chunksMetadata, indexFileWriter) + if err != nil { + return fmt.Errorf("failed to write series %d: %w", previousSeriesID, err) + } + return nil + } + + blockMeta := &tsdb.BlockMeta{ + ULID: uid, + MinTime: math.MaxInt64, + MaxTime: math.MinInt64, + Version: metaVersion1, + Compaction: tsdb.BlockMetaCompaction{ + Level: 1, + Sources: []ulid.ULID{uid}, + }, + } + + var hasChunks bool + for chunkIterator.Next() { + hasChunks = true + chunk = chunkIterator.At() + chunkMetadata, err = chunkw.Write(chunk) + if err != nil { + return fmt.Errorf("failed to write chunk: %w", err) + } + + adjustBlockMetaTimeRange(blockMeta, chunk.MinT(), chunk.MaxT()) + blockMeta.Stats.NumChunks++ + blockMeta.Stats.NumSamples += uint64(chunk.SampleCount()) + seriesID := chunk.SeriesID() + + if previousSeriesID == seriesID { + chunksMetadata = append(chunksMetadata, chunkMetadata) + } else { + if err = writeSeries(); err != nil { + return err + } + blockMeta.Stats.NumSeries++ + chunksMetadata = append(chunksMetadata[:0], chunkMetadata) + previousSeriesID = seriesID + } + } + + if !hasChunks { + return nil + } + + if err = writeSeries(); err != nil { + return err + } + indexFileSize, err := indexWriter.WriteRestTo(indexFileWriter) + if err != nil { + return fmt.Errorf("failed to write index: %w", err) + } + // todo: logs & metrics + _ = indexFileSize + + // write meta + blockMeta.MaxTime += 1 + metaFileSize, err := writeBlockMetaFile(filepath.Join(tmp, metaFilename), blockMeta) + if err != nil { + return fmt.Errorf("failed to write block meta file: %w", err) + } + // todo: log & metrics + _ = metaFileSize + + closeErr := err + for _, closer := range closers { + closeErr = errors.Join(closeErr, closer.Close()) + } + closers = closers[:0] + + if closeErr != nil { + return closeErr + } + + var df *os.File + df, err = fileutil.OpenDir(tmp) + if err != nil { + return fmt.Errorf("failed to open temporary block dir: %w", err) + } + defer func() { + if df != nil { + _ = df.Close() + } + }() + + if err = df.Sync(); err != nil { + return fmt.Errorf("failed to sync temporary block dir: %w", err) + } + + if err = df.Close(); err != nil { + return fmt.Errorf("failed to close temporary block dir: %w", err) + } + df = nil + + if err = fileutil.Replace(tmp, dir); err != nil { + return fmt.Errorf("failed to move temporary block dir {%s} to {%s}: %w", tmp, dir, err) + } + + w.blockWriteDuration.With(prometheus.Labels{ + "block_id": blockMeta.ULID.String(), + }).Set(float64(time.Since(start).Milliseconds())) + + return +} + +func chunkDir(dir string) string { return filepath.Join(dir, "chunks") } + +func closeAll(closers ...io.Closer) error { + errs := make([]error, len(closers)) + for i, closer := range closers { + errs[i] = closer.Close() + } + return errors.Join(errs...) +} + +func adjustBlockMetaTimeRange(blockMeta *tsdb.BlockMeta, mint, maxt int64) { + if mint < blockMeta.MinTime { + blockMeta.MinTime = mint + } + + if maxt > blockMeta.MaxTime { + blockMeta.MaxTime = maxt + } +} + +func writeBlockMetaFile(fileName string, blockMeta *tsdb.BlockMeta) (int64, error) { + tmp := fileName + ".tmp" + defer func() { + if err := os.RemoveAll(tmp); err != nil { + // todo: log error + } + }() + + metaFile, err := os.Create(tmp) + if err != nil { + return 0, fmt.Errorf("failed to create block meta file: %w", err) + } + defer func() { + if metaFile != nil { + if err = metaFile.Close(); err != nil { + // todo: log error + } + } + }() + + jsonBlockMeta, err := json.MarshalIndent(blockMeta, "", "\t") + if err != nil { + return 0, fmt.Errorf("failed to marshal meta json: %w", err) + } + + n, err := metaFile.Write(jsonBlockMeta) + if err != nil { + return 0, fmt.Errorf("failed to write meta json: %w", err) + } + + if err = metaFile.Sync(); err != nil { + return 0, fmt.Errorf("failed to sync meta file: %w", err) + } + + if err = metaFile.Close(); err != nil { + return 0, fmt.Errorf("faield to close meta file: %w", err) + } + metaFile = nil + + return int64(n), fileutil.Replace(tmp, fileName) +} diff --git a/pp/go/storage/catalog/catalog.go b/pp/go/storage/catalog/catalog.go new file mode 100644 index 0000000000..e571e24c61 --- /dev/null +++ b/pp/go/storage/catalog/catalog.go @@ -0,0 +1 @@ +package catalog diff --git a/pp/go/storage/head/builder/builder.go b/pp/go/storage/head/builder/builder.go new file mode 100644 index 0000000000..e647fb226b --- /dev/null +++ b/pp/go/storage/head/builder/builder.go @@ -0,0 +1 @@ +package builder diff --git a/pp/go/storage/head/container/weighted.go b/pp/go/storage/head/container/weighted.go new file mode 100644 index 0000000000..f9efff564d --- /dev/null +++ b/pp/go/storage/head/container/weighted.go @@ -0,0 +1,65 @@ +package container + +import ( + "context" + "fmt" + "sync/atomic" + "unsafe" + + "github.com/prometheus/prometheus/pp/go/util/locker" +) + +// Head the minimum required Head implementation for a container. +type Head[T any] interface { + Concurrency() int64 + *T +} + +// Weighted container for [Head] with weighted locker. +type Weighted[T any, H Head[T]] struct { + wlocker *locker.Weighted + head *T +} + +// NewWeighted init new [Weighted]. +func NewWeighted[T any, H Head[T]](head H) *Weighted[T, H] { + return &Weighted[T, H]{ + wlocker: locker.NewWeighted(2 * head.Concurrency()), // x2 for back pressure + head: head, + } +} + +// Get the active head [Head] under the non-exlusive lock and return. +func (c *Weighted[T, H]) Get() H { + return (*T)(atomic.LoadPointer( + (*unsafe.Pointer)(unsafe.Pointer(&c.head))), // #nosec G103 // it's meant to be that way + ) +} + +// Replace the active head [Head] with a new head under the exlusive priority lock. +func (c *Weighted[T, H]) Replace(ctx context.Context, newHead H) error { + unlock, err := c.wlocker.LockWithPriority(ctx) + if err != nil { + return fmt.Errorf("weighted lock with priority: %w", err) + } + + atomic.StorePointer( + (*unsafe.Pointer)(unsafe.Pointer(&c.head)), // #nosec G103 // it's meant to be that way + unsafe.Pointer(newHead), // #nosec G103 // it's meant to be that way + ) + + unlock() + + return nil +} + +// With calls fn(h Head) under the non-exlusive lock. +func (c *Weighted[T, H]) With(ctx context.Context, fn func(h H) error) error { + runlock, err := c.wlocker.RLock(ctx) + if err != nil { + return fmt.Errorf("weighted rlock: %w", err) + } + defer runlock() + + return fn(c.head) +} diff --git a/pp/go/storage/head/container/weighted_test.go b/pp/go/storage/head/container/weighted_test.go new file mode 100644 index 0000000000..87b879fa84 --- /dev/null +++ b/pp/go/storage/head/container/weighted_test.go @@ -0,0 +1,111 @@ +package container_test + +import ( + "context" + "fmt" + "testing" + "unsafe" + + "github.com/stretchr/testify/suite" + + "github.com/prometheus/prometheus/pp/go/storage/head/container" +) + +type WeightedSuite struct { + suite.Suite +} + +func TestWeightedSuite(t *testing.T) { + suite.Run(t, new(WeightedSuite)) +} + +func (s *WeightedSuite) TestGet() { + expectedHead := &testHead{c: 2} + c := container.NewWeighted(expectedHead) + + actualHead := c.Get() + + s.Equal(expectedHead, actualHead) +} + +func (s *WeightedSuite) TestReplace() { + baseCtx := context.Background() + expectedHead := &testHead{c: 2} + newHead := &testHead{c: 3} + c := container.NewWeighted(expectedHead) + + err := c.Replace(baseCtx, newHead) + s.Require().NoError(err) + + actualHead := c.Get() + + s.NotEqual(expectedHead, actualHead) + s.NotEqual(unsafe.Pointer(expectedHead), unsafe.Pointer(actualHead)) + s.Equal(newHead, actualHead) + s.Equal(unsafe.Pointer(newHead), unsafe.Pointer(actualHead)) +} + +func (s *WeightedSuite) TestReplaceError() { + expectedHead := &testHead{c: 2} + newHead := &testHead{c: 3} + c := container.NewWeighted(expectedHead) + ctx, cancel := context.WithCancel(context.Background()) + cancel() + + err := c.Replace(ctx, newHead) + s.Error(err) +} + +func (s *WeightedSuite) TestWith() { + baseCtx := context.Background() + expectedHead := &testHead{c: 2} + c := container.NewWeighted(expectedHead) + + err := c.With(baseCtx, func(h *testHead) error { + if expectedHead.c != h.c { + return fmt.Errorf("expectedHead(%d) not equal actual(%d)", expectedHead.c, h.c) + } + + return nil + }) + + s.NoError(err) +} + +func (s *WeightedSuite) TestWithError() { + baseCtx := context.Background() + expectedHead := &testHead{c: 1} + c := container.NewWeighted(expectedHead) + step1 := make(chan struct{}) + step2 := make(chan struct{}) + ctx, cancel := context.WithCancel(baseCtx) + + go c.With(baseCtx, func(_ *testHead) error { + close(step1) + cancel() + <-step2 + return nil + }) + + <-step1 + err := c.With(ctx, func(_ *testHead) error { + return nil + }) + close(step2) + + s.Error(err) +} + +// +// testHead +// + +// testHead implementation [container.Head]. +type testHead struct { + c int64 +} + +// Concurrency implementation [container.Head]. +func (h *testHead) Concurrency() int64 { + return h.c +} diff --git a/pp/go/storage/head/keeper/keeper.go b/pp/go/storage/head/keeper/keeper.go new file mode 100644 index 0000000000..fc669d875d --- /dev/null +++ b/pp/go/storage/head/keeper/keeper.go @@ -0,0 +1,153 @@ +package keeper + +import ( + "sync" + "time" + + "github.com/jonboulle/clockwork" + "github.com/prometheus/client_golang/prometheus" + "github.com/prometheus/prometheus/pp/go/relabeler" + "github.com/prometheus/prometheus/pp/go/relabeler/querier" + "github.com/prometheus/prometheus/pp/go/storage/logger" + "github.com/prometheus/prometheus/pp/go/util" +) + +// type Block interface { +// TimeBounds() (minT, maxT int64) +// // ChunkIterator(minT, maxT int64) ChunkIterator +// // IndexWriter() IndexWriter +// } + +// HeadBlockWriter writes block on disk from [Head]. +type HeadBlockWriter[TBlock any] interface { + Write(block TBlock) error +} + +type WriteNotifier interface { + NotifyWritten() +} + +type HeadBlockBuilder[TBlock any] func() TBlock + +type Keeper[TBlock any] struct { + hbWriter HeadBlockWriter[TBlock] + headBlockBuilder HeadBlockBuilder[TBlock] + + writeNotifier WriteNotifier + mtx sync.Mutex + heads []relabeler.Head + headRetentionTimeout time.Duration + + writeTimer clockwork.Timer + writeTimeout time.Duration + addCount uint32 + closer *util.Closer + + clock clockwork.Clock + maxRetentionDuration time.Duration + headPersistenceDuration prometheus.Histogram + querierMetrics *querier.Metrics +} + +func (k *Keeper[TBlock]) write() bool { + bl := k.headBlockBuilder() + + k.hbWriter.Write(bl) + + k.mtx.Lock() + lenHeads := len(k.heads) + if lenHeads == 0 { + // quick exit + k.mtx.Unlock() + return true + } + heads := make([]relabeler.Head, lenHeads) + copy(heads, k.heads) + k.mtx.Unlock() + + successful := true + shouldNotify := false + persisted := make([]string, 0, lenHeads) + for _, head := range heads { + start := k.clock.Now() + if k.headIsOutdated(head) { + persisted = append(persisted, head.ID()) + shouldNotify = true + continue + } + if err := head.Flush(); err != nil { + logger.Errorf("QUERYABLE STORAGE: failed to flush head %s: %s", head.String(), err.Error()) + successful = false + continue + } + if err := head.Rotate(); err != nil { + logger.Errorf("QUERYABLE STORAGE: failed to rotate head %s: %s", head.String(), err.Error()) + successful = false + continue + } + + tBlockWrite := head.CreateTask( + relabeler.BlockWrite, + func(shard relabeler.Shard) error { + shard.LSSLock() + defer shard.LSSUnlock() + + return k.hbWriter.Write(relabeler.NewBlock(shard.LSS().Raw(), shard.DataStorage().Raw())) + }, + relabeler.ForLSSTask, + ) + head.Enqueue(tBlockWrite) + if err := tBlockWrite.Wait(); err != nil { + logger.Errorf("QUERYABLE STORAGE: failed to write head %s: %s", head.String(), err.Error()) + successful = false + continue + } + + k.headPersistenceDuration.Observe(float64(k.clock.Since(start).Milliseconds())) + persisted = append(persisted, head.ID()) + shouldNotify = true + logger.Infof("QUERYABLE STORAGE: head %s persisted, duration: %v", head.String(), k.clock.Since(start)) + } + + if shouldNotify { + k.writeNotifier.NotifyWritten() + } + + time.AfterFunc(k.headRetentionTimeout, func() { + select { + case <-k.closer.Signal(): + return + default: + k.shrink(persisted...) + } + }) + + return successful +} + +func (k *Keeper[TBlock]) headIsOutdated(head relabeler.Head) bool { + headMaxTimestampMs := head.Status(1).HeadStats.MaxTime + return k.clock.Now().Sub(time.Unix(headMaxTimestampMs/1000, 0)) > k.maxRetentionDuration +} + +func (k *Keeper[TBlock]) shrink(persisted ...string) { + k.mtx.Lock() + defer k.mtx.Unlock() + + persistedMap := make(map[string]struct{}) + for _, headID := range persisted { + persistedMap[headID] = struct{}{} + } + + var heads []relabeler.Head + for _, head := range k.heads { + if _, ok := persistedMap[head.ID()]; ok { + _ = head.Close() + _ = head.Discard() + logger.Infof("QUERYABLE STORAGE: head %s persisted, closed and discarded", head.String()) + continue + } + heads = append(heads, head) + } + k.heads = heads +} diff --git a/pp/go/storage/head/manager/manager.go b/pp/go/storage/head/manager/manager.go new file mode 100644 index 0000000000..bbdb28fdb6 --- /dev/null +++ b/pp/go/storage/head/manager/manager.go @@ -0,0 +1,152 @@ +package manager + +import ( + "context" + "fmt" + "time" + + "github.com/prometheus/client_golang/prometheus" + "github.com/prometheus/prometheus/pp/go/relabeler/config" + "github.com/prometheus/prometheus/pp/go/storage" + "github.com/prometheus/prometheus/pp/go/storage/logger" + "github.com/prometheus/prometheus/pp/go/util" +) + +type Timer interface { + Chan() <-chan time.Time + Reset() + Stop() +} + +type ActiveHeadContainer interface { + Get(ctx context.Context) (storage.Head, error) + Replace(ctx context.Context, newHead storage.Head) error + With(ctx context.Context, fn func(h storage.Head) error) error +} + +// type ActiveHeadContainer[T any] interface { +// Get() *T +// Replace(ctx context.Context, newHead *T) error +// With(ctx context.Context, fn func(h *T) error) error +// } + +// var _ ActiveHeadContainer[testHead] = (*container.Weighted[testHead, *testHead])(nil) + +// HeadBuilder builder for the [storage.Head]. +type HeadBuilder interface { + // inputRelabelerConfigs []*config.InputRelabelerConfig, + BuildWithConfig(numberOfShards uint16) (storage.Head, error) +} + +type Manager struct { + // + activeHead ActiveHeadContainer + headBuilder HeadBuilder + rotateTimer Timer + commitTimer Timer + mergeTimer Timer + // TODO closer vs shutdowner + closer *util.Closer + shutdowner *util.GracefulShutdowner + + rotateCounter prometheus.Counter + + numberOfShards uint16 +} + +// ApplyConfig update config. +func (m *Manager) ApplyConfig( + ctx context.Context, + inputRelabelerConfigs []*config.InputRelabelerConfig, + numberOfShards uint16, +) error { + logger.Infof("reconfiguration start") + defer logger.Infof("reconfiguration completed") + + // TODO HeadConfigStorage + + return nil +} + +// MergeOutOfOrderChunks merge chunks with out of order data chunks. +func (m *Manager) MergeOutOfOrderChunks(ctx context.Context) error { + return m.activeHead.With(ctx, func(h storage.Head) error { + h.MergeOutOfOrderChunks() + + return nil + }) +} + +// Run starts processing of the [Manager]. +// TODO implementation. +func (m *Manager) Run(ctx context.Context) error { + go m.loop(ctx) + return nil +} + +// Shutdown safe shutdown [Manager]. +func (m *Manager) Shutdown(ctx context.Context) error { + return nil +} + +// commitToWal commit the accumulated data into the wal. +func (m *Manager) commitToWal(ctx context.Context) error { + return m.activeHead.With(ctx, func(h storage.Head) error { + return h.CommitToWal() + }) +} + +// TODO implementation. +func (m *Manager) loop(ctx context.Context) { + defer m.closer.Done() + + for { + select { + case <-m.closer.Signal(): + return + + case <-m.commitTimer.Chan(): + if err := m.commitToWal(ctx); err != nil { + logger.Errorf("wal commit failed: %v", err) + } + m.commitTimer.Reset() + + case <-m.mergeTimer.Chan(): + if err := m.MergeOutOfOrderChunks(ctx); err != nil { + logger.Errorf("merge out of order chunks failed: %v", err) + } + m.mergeTimer.Reset() + + case <-m.rotateTimer.Chan(): + logger.Debugf("start rotation") + + if err := m.rotate(ctx); err != nil { + logger.Errorf("rotation failed: %v", err) + } + m.rotateCounter.Inc() + + m.rotateTimer.Reset() + m.commitTimer.Reset() + m.mergeTimer.Reset() + } + } +} + +func (m *Manager) rotate(ctx context.Context) error { + newHead, err := m.headBuilder.BuildWithConfig(m.numberOfShards) + if err != nil { + return fmt.Errorf("failed to build a new head: %w", err) + } + + oldHead, err := m.activeHead.Get(ctx) + if err != nil { + return fmt.Errorf("getting active head failed: %w", err) + } + + newHead.CopySeriesFrom(oldHead) + + // storgae.Add(oldHead) + + // TODO if replace error? + return m.activeHead.Replace(ctx, newHead) +} diff --git a/pp/go/storage/head/manager/timer.go b/pp/go/storage/head/manager/timer.go new file mode 100644 index 0000000000..a35889a84b --- /dev/null +++ b/pp/go/storage/head/manager/timer.go @@ -0,0 +1,121 @@ +package manager + +import ( + "math" + "math/rand" + "sync" + "time" + + "github.com/jonboulle/clockwork" +) + +// +// RotateTimer +// + +// RotateTimer - custom timer with reset the timer for the delay time. +type RotateTimer struct { + clock clockwork.Clock + timer clockwork.Timer + rotateAt time.Time + mx *sync.Mutex + durationBlock int64 + rndDurationBlock int64 +} + +// NewRotateTimer - init new RotateTimer. The duration durationBlock and delayAfterNotify must be greater than zero; +// if not, Ticker will panic. Stop the ticker to release associated resources. +func NewRotateTimer(clock clockwork.Clock, desiredBlockFormationDuration time.Duration) *RotateTimer { + return NewRotateTimerWithSeed(clock, desiredBlockFormationDuration, uint64(clock.Now().UnixNano())) +} + +// NewRotateTimerWithSeed - init new RotateTimer. The duration durationBlock and delayAfterNotify must be greater than zero; +// if not, Ticker will panic. Stop the ticker to release associated resources. +func NewRotateTimerWithSeed( + clock clockwork.Clock, + desiredBlockFormationDuration time.Duration, + seed uint64, +) *RotateTimer { + bd := desiredBlockFormationDuration.Milliseconds() + //nolint:gosec // there is no need for cryptographic strength here + rnd := rand.New(rand.NewSource(int64(seed))) + rt := &RotateTimer{ + clock: clock, + durationBlock: bd, + rndDurationBlock: rnd.Int63n(bd), + mx: new(sync.Mutex), + } + + rt.rotateAt = rt.RotateAtNext() + rt.timer = clock.NewTimer(rt.rotateAt.Sub(rt.clock.Now())) + + return rt +} + +// Chan - return chan with ticker time. +func (rt *RotateTimer) Chan() <-chan time.Time { + return rt.timer.Chan() +} + +// Reset - changes the timer to expire after duration Block and clearing channels. +func (rt *RotateTimer) Reset() { + rt.mx.Lock() + rt.rotateAt = rt.RotateAtNext() + if !rt.timer.Stop() { + select { + case <-rt.timer.Chan(): + default: + } + } + rt.timer.Reset(rt.rotateAt.Sub(rt.clock.Now())) + rt.mx.Unlock() +} + +// RotateAtNext - calculated next rotate time. +func (rt *RotateTimer) RotateAtNext() time.Time { + now := rt.clock.Now().UnixMilli() + k := now % rt.durationBlock + startBlock := math.Floor(float64(now)/float64(rt.durationBlock)) * float64(rt.durationBlock) + + if rt.rndDurationBlock > k { + return time.UnixMilli(int64(startBlock) + rt.rndDurationBlock) + } + + return time.UnixMilli(int64(startBlock) + rt.durationBlock + rt.rndDurationBlock) +} + +// Stop - prevents the Timer from firing. +// Stop does not close the channel, to prevent a read from the channel succeeding incorrectly. +func (rt *RotateTimer) Stop() { + if !rt.timer.Stop() { + <-rt.timer.Chan() + } +} + +// +// ConstantIntervalTimer +// + +type ConstantIntervalTimer struct { + timer clockwork.Timer + interval time.Duration +} + +func NewConstantIntervalTimer(clock clockwork.Clock, interval time.Duration) *ConstantIntervalTimer { + return &ConstantIntervalTimer{ + timer: clock.NewTimer(interval), + interval: interval, + } +} + +func (t *ConstantIntervalTimer) Chan() <-chan time.Time { + return t.timer.Chan() +} + +func (t *ConstantIntervalTimer) Reset() { + t.timer.Reset(t.interval) +} + +func (t *ConstantIntervalTimer) Stop() { + t.timer.Stop() +} diff --git a/pp/go/storage/head/shard/data_storage.go b/pp/go/storage/head/shard/data_storage.go new file mode 100644 index 0000000000..2425dff06a --- /dev/null +++ b/pp/go/storage/head/shard/data_storage.go @@ -0,0 +1,40 @@ +package shard + +import "github.com/prometheus/prometheus/pp/go/cppbridge" + +type DataStorage struct { + dataStorage *cppbridge.HeadDataStorage + encoder *cppbridge.HeadEncoder +} + +func NewDataStorage() *DataStorage { + dataStorage := cppbridge.NewHeadDataStorage() + return &DataStorage{ + dataStorage: dataStorage, + encoder: cppbridge.NewHeadEncoderWithDataStorage(dataStorage), + } +} + +func (ds *DataStorage) AppendInnerSeriesSlice(innerSeriesSlice []*cppbridge.InnerSeries) { + ds.encoder.EncodeInnerSeriesSlice(innerSeriesSlice) +} + +func (ds *DataStorage) Raw() *cppbridge.HeadDataStorage { + return ds.dataStorage +} + +func (ds *DataStorage) MergeOutOfOrderChunks() { + ds.encoder.MergeOutOfOrderChunks() +} + +func (ds *DataStorage) Query(query cppbridge.HeadDataStorageQuery) *cppbridge.HeadDataStorageSerializedChunks { + return ds.dataStorage.Query(query) +} + +func (ds *DataStorage) InstantQuery(targetTimestamp, notFoundValueTimestampValue int64, seriesIDs []uint32) []cppbridge.Sample { + return ds.dataStorage.InstantQuery(targetTimestamp, notFoundValueTimestampValue, seriesIDs) +} + +func (ds *DataStorage) AllocatedMemory() uint64 { + return ds.dataStorage.AllocatedMemory() +} diff --git a/pp/go/storage/head/shard/lss.go b/pp/go/storage/head/shard/lss.go new file mode 100644 index 0000000000..e90cfa5279 --- /dev/null +++ b/pp/go/storage/head/shard/lss.go @@ -0,0 +1,67 @@ +package shard + +import ( + "sync" + + "github.com/prometheus/prometheus/pp/go/cppbridge" + "github.com/prometheus/prometheus/pp/go/model" +) + +// LSS labelset storage for [shard]. +type LSS struct { + input *cppbridge.LabelSetStorage + target *cppbridge.LabelSetStorage + snapshot *cppbridge.LabelSetSnapshot + once sync.Once +} + +func (w *LSS) Raw() *cppbridge.LabelSetStorage { + return w.target +} + +func (w *LSS) AllocatedMemory() uint64 { + return w.input.AllocatedMemory() + w.target.AllocatedMemory() +} + +func (w *LSS) QueryLabelValues( + label_name string, + matchers []model.LabelMatcher, +) *cppbridge.LSSQueryLabelValuesResult { + return w.target.QueryLabelValues(label_name, matchers) +} + +func (w *LSS) QueryLabelNames(matchers []model.LabelMatcher) *cppbridge.LSSQueryLabelNamesResult { + return w.target.QueryLabelNames(matchers) +} + +// QuerySelector returns a created selector that matches the given label matchers. +func (w *LSS) QuerySelector(matchers []model.LabelMatcher) (selector uintptr, status uint32) { + return w.target.QuerySelector(matchers) +} + +func (w *LSS) GetLabelSets(labelSetIDs []uint32) *cppbridge.LabelSetStorageGetLabelSetsResult { + return w.target.GetLabelSets(labelSetIDs) +} + +// GetSnapshot return the actual snapshot. +func (w *LSS) GetSnapshot() *cppbridge.LabelSetSnapshot { + w.once.Do(func() { + w.snapshot = w.target.CreateLabelSetSnapshot() + }) + + return w.snapshot +} + +// ResetSnapshot resets the current snapshot. +func (w *LSS) ResetSnapshot() { + w.snapshot = nil + w.once = sync.Once{} +} + +func (w *LSS) Input() *cppbridge.LabelSetStorage { + return w.input +} + +func (w *LSS) Target() *cppbridge.LabelSetStorage { + return w.target +} diff --git a/pp/go/storage/head/shard/shard.go b/pp/go/storage/head/shard/shard.go new file mode 100644 index 0000000000..50187dce0c --- /dev/null +++ b/pp/go/storage/head/shard/shard.go @@ -0,0 +1,73 @@ +package shard + +import "sync" + +// +// RWLockable +// + +// RWLockable implementation [sync.RWMutex]. +type RWLockable interface { + Lock() + RLock() + RUnlock() + Unlock() +} + +// +// Shard +// + +// Shard +type Shard struct { + lss *LSS + dataStorage *DataStorage + wal *Wal + lssLocker RWLockable + dataStorageLocker RWLockable + id uint16 +} + +// NewShard init new [Shard]. +func NewShard( + lss *LSS, + dataStorage *DataStorage, + wal *Wal, + shardID uint16, + withLocker bool, +) *Shard { + s := &Shard{ + id: shardID, + lss: lss, + dataStorage: dataStorage, + wal: wal, + lssLocker: &noopRWLockable{}, + dataStorageLocker: &noopRWLockable{}, + } + + if withLocker { + s.lssLocker = &sync.RWMutex{} + s.dataStorageLocker = &sync.RWMutex{} + } + + return s +} + +// +// noopRWLockable +// + +// noopRWLockable implementation sync.RWMutex, does nothing. +type noopRWLockable struct{} + +// Lock implementation [RWLockable]. +func (*noopRWLockable) Lock() {} + +// RLock implementation [RWLockable]. +func (*noopRWLockable) RLock() {} + +// RUnlock implementation [RWLockable]. +func (*noopRWLockable) RUnlock() {} + +// Unlock implementation [RWLockable]. +func (*noopRWLockable) Unlock() {} diff --git a/pp/go/storage/head/shard/wal.go b/pp/go/storage/head/shard/wal.go new file mode 100644 index 0000000000..4660648210 --- /dev/null +++ b/pp/go/storage/head/shard/wal.go @@ -0,0 +1,275 @@ +package shard + +import ( + "encoding/binary" + "fmt" + "hash/crc32" + "io" + + "github.com/prometheus/prometheus/pp/go/cppbridge" +) + +const ( + // FileFormatVersion wal file version. + FileFormatVersion = 1 +) + +// SegmentWriter writer for wal segments. +type SegmentWriter interface { + // CurrentSize return current shard wal size. + CurrentSize() int64 + // Write encoded segment to writer. + Write(segment EncodedSegment) error + // Flush write all buffered segments. + Flush() error + // Close closes the storage. + Close() error +} + +// EncodedSegment encoded segment. +type EncodedSegment interface { + // Size returns the number of bytes in the segment. + Size() int64 + // CRC32 checksum for segment. + CRC32() uint32 + io.WriterTo + cppbridge.SegmentStats +} + +// Wal write-ahead log for [Shard]. +type Wal struct { + encoder *cppbridge.HeadWalEncoder + segmentWriter SegmentWriter + maxSegmentSize uint32 + corrupted bool + limitExhausted bool +} + +// NewWal init new [Wal]. +func NewWal(encoder *cppbridge.HeadWalEncoder, maxSegmentSize uint32, segmentWriter SegmentWriter) *Wal { + return &Wal{ + encoder: encoder, + segmentWriter: segmentWriter, + maxSegmentSize: maxSegmentSize, + } +} + +// NewCorruptedWal init new corrupted [Wal]. +func NewCorruptedWal() *Wal { + return &Wal{ + corrupted: true, + } +} + +// CurrentSize returns current wal size. +func (w *Wal) CurrentSize() int64 { + return w.segmentWriter.CurrentSize() +} + +// Write the incoming inner series to wal encoder. +func (w *Wal) Write(innerSeriesSlice []*cppbridge.InnerSeries) (bool, error) { + if w.corrupted { + return false, fmt.Errorf("writing in corrupted wal") + } + + stats, err := w.encoder.Encode(innerSeriesSlice) + if err != nil { + return false, fmt.Errorf("failed to encode inner series: %w", err) + } + + if w.maxSegmentSize == 0 { + return false, nil + } + + // memoize reaching of limits to deduplicate triggers + if !w.limitExhausted && stats.Samples() >= w.maxSegmentSize { + w.limitExhausted = true + return true, nil + } + + return false, nil +} + +// Commit finalize segment from encoder and write to [SegmentWriter]. +func (w *Wal) Commit() error { + if w.corrupted { + return fmt.Errorf("committing corrupted wal") + } + + segment, err := w.encoder.Finalize() + if err != nil { + return fmt.Errorf("failed to finalize segment: %w", err) + } + w.limitExhausted = false + + if err = w.segmentWriter.Write(segment); err != nil { + return fmt.Errorf("failed to write segment: %w", err) + } + + if err = w.segmentWriter.Flush(); err != nil { + return fmt.Errorf("failed to flush segment writer: %w", err) + } + + return nil +} + +// Flush wal [SegmentWriter]. +func (w *Wal) Flush() error { + return w.segmentWriter.Flush() +} + +// Close closes the wal segmentWriter. +func (w *Wal) Close() error { + if w.segmentWriter != nil { + return w.segmentWriter.Close() + } + + return nil +} + +// WriteHeader write header to writer. +func WriteHeader(writer io.Writer, fileFormatVersion, encoderVersion uint8) (n int, err error) { + var buf [binary.MaxVarintLen32]byte + var size int + var bytesWritten int + + size = binary.PutUvarint(buf[:], uint64(fileFormatVersion)) + bytesWritten, err = writer.Write(buf[:size]) + if err != nil { + return n, fmt.Errorf("failed to write file format version: %w", err) + } + n += bytesWritten + + size = binary.PutUvarint(buf[:], uint64(encoderVersion)) + bytesWritten, err = writer.Write(buf[:size]) + if err != nil { + return n, fmt.Errorf("failed to write encoder version: %w", err) + } + n += bytesWritten + + return n, nil +} + +type byteReader struct { + r io.Reader + n int +} + +func (r *byteReader) ReadByte() (byte, error) { + b := make([]byte, 1) + n, err := io.ReadFull(r.r, b) + if err != nil { + return 0, err + } + r.n += n + return b[0], nil +} + +// ReadHeader read header from reader. +// +//revive:disable-next-line:function-result-limit there is no point in packing it into a structure. +func ReadHeader(reader io.Reader) (fileFormatVersion, encoderVersion uint8, n int, err error) { + br := &byteReader{r: reader} + fileFormatVersionU64, err := binary.ReadUvarint(br) + if err != nil { + return 0, 0, n, fmt.Errorf("failed to read file format version: %w", err) + } + fileFormatVersion = uint8(fileFormatVersionU64) // #nosec G115 // no overflow + n = br.n + + encoderVersionU64, err := binary.ReadUvarint(br) + if err != nil { + return 0, 0, n, fmt.Errorf("failed to read encoder version: %w", err) + } + encoderVersion = uint8(encoderVersionU64) // #nosec G115 // no overflow + n = br.n + + return fileFormatVersion, encoderVersion, n, nil +} + +// WriteSegment write encoded segment to writer. +func WriteSegment(writer io.Writer, segment EncodedSegment) (n int, err error) { + var buf [binary.MaxVarintLen32]byte + var size int + var bytesWritten int + + segmentSize := uint64(segment.Size()) // #nosec G115 // no overflow + size = binary.PutUvarint(buf[:], segmentSize) + bytesWritten, err = writer.Write(buf[:size]) + if err != nil { + return n, fmt.Errorf("failed to write segment size: %w", err) + } + n += bytesWritten + + size = binary.PutUvarint(buf[:], uint64(segment.CRC32())) + bytesWritten, err = writer.Write(buf[:size]) + if err != nil { + return n, fmt.Errorf("failed to write segment crc32 hash: %w", err) + } + n += bytesWritten + + size = binary.PutUvarint(buf[:], uint64(segment.Samples())) + bytesWritten, err = writer.Write(buf[:size]) + if err != nil { + return n, fmt.Errorf("failed to write segment sample count: %w", err) + } + n += bytesWritten + + var bytesWritten64 int64 + bytesWritten64, err = segment.WriteTo(writer) + if err != nil { + return n, fmt.Errorf("failed to write segment data: %w", err) + } + n += int(bytesWritten64) + + return n, nil +} + +type DecodedSegment struct { + data []byte + sampleCount uint32 +} + +func (d DecodedSegment) Data() []byte { + return d.data +} + +func (d DecodedSegment) SampleCount() uint32 { + return d.sampleCount +} + +func ReadSegment(reader io.Reader) (decodedSegment DecodedSegment, n int, err error) { + br := &byteReader{r: reader} + var size uint64 + size, err = binary.ReadUvarint(br) + if err != nil { + return decodedSegment, br.n, fmt.Errorf("failed to read segment size: %w", err) + } + + crc32HashU64, err := binary.ReadUvarint(br) + if err != nil { + return decodedSegment, br.n, fmt.Errorf("failed to read segment crc32 hash: %w", err) + } + crc32Hash := uint32(crc32HashU64) // #nosec G115 // no overflow + + sampleCountU64, err := binary.ReadUvarint(br) + if err != nil { + return decodedSegment, br.n, fmt.Errorf("failed to read segment sample count: %w", err) + } + decodedSegment.sampleCount = uint32(sampleCountU64) // #nosec G115 // no overflow + + decodedSegment.data = make([]byte, size) + n, err = io.ReadFull(reader, decodedSegment.data) + if err != nil { + return decodedSegment, br.n, fmt.Errorf("failed to read segment data: %w", err) + } + n += br.n + + if crc32Hash != crc32.ChecksumIEEE(decodedSegment.data) { + return decodedSegment, n, fmt.Errorf( + "crc32 did not match, want: %d, have: %d", crc32Hash, crc32.ChecksumIEEE(decodedSegment.data), + ) + } + + return decodedSegment, n, nil +} diff --git a/pp/go/storage/interface.go b/pp/go/storage/interface.go new file mode 100644 index 0000000000..fd01b37cd6 --- /dev/null +++ b/pp/go/storage/interface.go @@ -0,0 +1,150 @@ +package storage + +import ( + "context" + + "github.com/prometheus/prometheus/pp/go/cppbridge" + "github.com/prometheus/prometheus/pp/go/model" + "github.com/prometheus/prometheus/pp/go/relabeler/config" +) + +// +// Head +// + +// Head implementation of the head with added metrics. +type Head interface { + ID() string + Generation() uint64 + Append( + ctx context.Context, + incomingData *IncomingData, + state *cppbridge.State, + relabelerID string, + commitToWal bool, + ) ([][]*cppbridge.InnerSeries, cppbridge.RelabelerStats, error) + CommitToWal() error + // MergeOutOfOrderChunks merge chunks with out of order data chunks. + MergeOutOfOrderChunks() + NumberOfShards() uint16 + Stop() + Flush() error + Reconfigure(ctx context.Context, inputRelabelerConfigs []*config.InputRelabelerConfig, numberOfShards uint16) error + WriteMetrics(ctx context.Context) + Status(limit int) HeadStatus + Rotate() error + Close() error + Discard() error + String() string + CopySeriesFrom(other Head) + Enqueue(t *GenericTask) + EnqueueOnShard(t *GenericTask, shardID uint16) + CreateTask(taskName string, fn ShardFn, isLss bool) *GenericTask + Concurrency() int64 + RLockQuery(ctx context.Context) (runlock func(), err error) + Raw() Head +} + +// +// Shard +// + +// Shard interface for shards [Head]. +type Shard interface { + // DataStorage returns [DataStorage] shard. + DataStorage() DataStorage + // lock for DataStorage + DataStorageLock() + DataStorageRLock() + DataStorageRUnlock() + DataStorageUnlock() + // LSS returns [LSS] shard. + LSS() LSS + // lock for LSS + LSSLock() + LSSRLock() + LSSRUnlock() + LSSUnlock() + // ShardID returns ID shard. + ShardID() uint16 + // Wal returns [Wal] shard. + Wal() Wal +} + +// ShardFn function executing on a [Shard]. +type ShardFn func(shard Shard) error + +// +// DataStorage +// + +// DataStorage sample storage interface. +type DataStorage interface { + AllocatedMemory() uint64 + // AppendInnerSeriesSlice append slice of [cppbridge.InnerSeries](samples with label IDs) to the storage. + AppendInnerSeriesSlice(innerSeriesSlice []*cppbridge.InnerSeries) + InstantQuery(targetTimestamp, notFoundValueTimestampValue int64, seriesIDs []uint32) []cppbridge.Sample + MergeOutOfOrderChunks() + Query(query cppbridge.HeadDataStorageQuery) *cppbridge.HeadDataStorageSerializedChunks + Raw() *cppbridge.HeadDataStorage +} + +// +// LSS +// + +// LSS labelset storage interface. +type LSS interface { + AllocatedMemory() uint64 + GetLabelSets(labelSetIDs []uint32) *cppbridge.LabelSetStorageGetLabelSetsResult + GetSnapshot() *cppbridge.LabelSetSnapshot + Input() *cppbridge.LabelSetStorage + QueryLabelNames(matchers []model.LabelMatcher) *cppbridge.LSSQueryLabelNamesResult + QueryLabelValues(label_name string, matchers []model.LabelMatcher) *cppbridge.LSSQueryLabelValuesResult + QuerySelector(matchers []model.LabelMatcher) (selector uintptr, status uint32) + Raw() *cppbridge.LabelSetStorage + ResetSnapshot() + Target() *cppbridge.LabelSetStorage +} + +// +// Wal +// + +// Wal write-ahead log for [Shard]. +type Wal interface { + // DO NOT USE in public interfaces like ForEachShard + Commit() error + Flush() error + Write(innerSeriesSlice []*cppbridge.InnerSeries) (bool, error) +} + +// +// MetricData +// + +// MetricData is an universal interface for blob protobuf data or batch [model.TimeSeries]. +type MetricData interface { + // Destroy incoming data. + Destroy() +} + +// +// ProtobufData +// + +// ProtobufData is an universal interface for blob protobuf data. +type ProtobufData interface { + Bytes() []byte + Destroy() +} + +// +// TimeSeriesData +// + +// TimeSeriesBatch is an universal interface for batch [model.TimeSeries]. +type TimeSeriesBatch interface { + TimeSeries() []model.TimeSeries + Destroy() +} diff --git a/pp/go/storage/logger/logger.go b/pp/go/storage/logger/logger.go new file mode 100644 index 0000000000..548517806c --- /dev/null +++ b/pp/go/storage/logger/logger.go @@ -0,0 +1,19 @@ +package logger + +func noop(string, ...any) {} + +// These variables are set by the common log package. +var ( + Errorf = noop + Warnf = noop + Infof = noop + Debugf = noop +) + +// Unset logger funcs to NoOp +func Unset() { + Errorf = noop + Warnf = noop + Infof = noop + Debugf = noop +} diff --git a/pp/go/storage/models.go b/pp/go/storage/models.go new file mode 100644 index 0000000000..5dc72e0b37 --- /dev/null +++ b/pp/go/storage/models.go @@ -0,0 +1,54 @@ +package storage + +import "github.com/prometheus/prometheus/pp/go/cppbridge" + +// +// IncomingData +// + +// IncomingData implements. +type IncomingData struct { + Hashdex cppbridge.ShardedData + Data MetricData +} + +// ShardedData return hashdex. +func (i *IncomingData) ShardedData() cppbridge.ShardedData { + return i.Hashdex +} + +// Destroy increment or destroy IncomingData. +func (i *IncomingData) Destroy() { + i.Hashdex = nil + if i.Data != nil { + i.Data.Destroy() + } +} + +// +// HeadStatus +// + +// HeadStatus holds information about all shards. +type HeadStatus struct { + HeadStats HeadStats `json:"headStats"` + SeriesCountByMetricName []HeadStat `json:"seriesCountByMetricName"` + LabelValueCountByLabelName []HeadStat `json:"labelValueCountByLabelName"` + MemoryInBytesByLabelName []HeadStat `json:"memoryInBytesByLabelName"` + SeriesCountByLabelValuePair []HeadStat `json:"seriesCountByLabelValuePair"` +} + +// HeadStat holds the information about individual cardinality. +type HeadStat struct { + Name string `json:"name"` + Value uint64 `json:"value"` +} + +// HeadStats has information about the head. +type HeadStats struct { + NumSeries uint64 `json:"numSeries"` + NumLabelPairs int `json:"numLabelPairs"` + ChunkCount int64 `json:"chunkCount"` + MinTime int64 `json:"minTime"` + MaxTime int64 `json:"maxTime"` +} diff --git a/pp/go/storage/task.go b/pp/go/storage/task.go new file mode 100644 index 0000000000..a3c9a2d282 --- /dev/null +++ b/pp/go/storage/task.go @@ -0,0 +1,200 @@ +package storage + +import ( + "errors" + "sync" + "sync/atomic" + "time" + + "github.com/prometheus/client_golang/prometheus" +) + +const ( + // ForLSSTask task for LSS. + ForLSSTask = true + // ForDataStorageTask task for DataStorage. + ForDataStorageTask = false + // ExclusiveTask exclusive task(write). + ExclusiveTask = true + // NonExclusiveTask non-exclusive task(read). + NonExclusiveTask = false +) + +const ( + // LSSInputRelabeling name of task. + LSSInputRelabeling = "lss_input_relabeling" + // LSSAppendRelabelerSeries name of task. + LSSAppendRelabelerSeries = "lss_append_relabeler_series" + + // LSSWalCommit name of task. + LSSWalCommit = "lss_wal_commit" + // LSSWalFlush name of task. + LSSWalFlush = "lss_wal_flush" + // LSSWalWrite name of task. + LSSWalWrite = "lss_wal_write" + + // LSSCopyAddedSeries name of task. + LSSCopyAddedSeries = "lss_copy_added_series" + + // LSSOutputRelabeling name of task. + LSSOutputRelabeling = "lss_output_relabeling" + + // LSSAllocatedMemory name of task. + LSSAllocatedMemory = "lss_allocated_memory" + + // LSSHeadStatus name of task. + LSSHeadStatus = "lss_head_status" + + // LSSQueryChunkQuerySelector name of task. + LSSQueryChunkQuerySelector = "lss_query_chunk_query_selector" + // LSSLabelValuesChunkQuerier name of task. + LSSLabelValuesChunkQuerier = "lss_label_values_chunk_querier" + // LSSLabelNamesChunkQuerier name of task. + LSSLabelNamesChunkQuerier = "lss_label_names_chunk_querier" + + // LSSQueryInstantQuerySelector name of task. + LSSQueryInstantQuerySelector = "lss_query_instant_query_selector" + // LSSQueryRangeQuerySelector name of task. + LSSQueryRangeQuerySelector = "lss_query_range_query_selector" + // LSSLabelValuesQuerier name of task. + LSSLabelValuesQuerier = "lss_label_values_querier" + // LSSLabelNamesQuerier name of task. + LSSLabelNamesQuerier = "lss_label_names_querier" + + // DSAppendInnerSeries name of task. + DSAppendInnerSeries = "data_storage_append_inner_series" + // DSMergeOutOfOrderChunks name of task. + DSMergeOutOfOrderChunks = "data_storage_merge_out_of_order_chunks" + + // DSAllocatedMemory name of task. + DSAllocatedMemory = "data_storage_allocated_memory" + + // DSHeadStatus name of task. + DSHeadStatus = "data_storage_head_status" + + // DSQueryChunkQuerier name of task. + DSQueryChunkQuerier = "data_storage_query_chunk_querier" + + // DSQueryInstantQuerier name of task. + DSQueryInstantQuerier = "data_storage_query_instant_querier" + // DSQueryRangeQuerier name of task. + DSQueryRangeQuerier = "data_storage_query_range_querier" + + // Read Only + + // BlockWrite name of task. + BlockWrite = "block_write" +) + +// +// GenericTask +// + +// GenericTask generic task, will be executed on each shard. +type GenericTask struct { + errs []error + shardFn ShardFn + wg sync.WaitGroup + createdTS int64 + executeTS int64 + created prometheus.Counter + done prometheus.Counter + live prometheus.Counter + execute prometheus.Counter + forLSS bool +} + +// NewGenericTask init new [GenericTask]. +func NewGenericTask( + shardFn ShardFn, + created, done, live, execute prometheus.Counter, + forLSS bool, +) *GenericTask { + t := &GenericTask{ + shardFn: shardFn, + wg: sync.WaitGroup{}, + createdTS: time.Now().UnixMicro(), + created: created, + done: done, + live: live, + execute: execute, + forLSS: forLSS, + } + t.created.Inc() + + return t +} + +// NewReadOnlyGenericTask init new GenericTask for read only head. +func NewReadOnlyGenericTask(shardFn ShardFn) *GenericTask { + t := &GenericTask{ + shardFn: shardFn, + wg: sync.WaitGroup{}, + } + + return t +} + +// SetShardsNumber set shards number +func (t *GenericTask) SetShardsNumber(number uint16) { + t.errs = make([]error, number) + t.wg.Add(int(number)) +} + +// ExecuteOnShard execute task on shard. +func (t *GenericTask) ExecuteOnShard(shard Shard) { + atomic.CompareAndSwapInt64(&t.executeTS, 0, time.Now().UnixMicro()) + t.errs[shard.ShardID()] = t.shardFn(shard) + t.wg.Done() +} + +// ForLSS indicates that the task is for operation on lss. +func (t *GenericTask) ForLSS() bool { + return t.forLSS +} + +// Wait for the task to complete on all shards. +func (t *GenericTask) Wait() error { + t.wg.Wait() + if t.done == nil { + return errors.Join(t.errs...) + } + + now := time.Now().UnixMicro() + t.done.Inc() + t.execute.Add(float64(now - t.executeTS)) + t.live.Add(float64(now - t.createdTS)) + + return errors.Join(t.errs...) +} + +// +// TaskWaiter +// + +// TaskWaiter aggregates the wait for tasks to be completed. +type TaskWaiter struct { + tasks []*GenericTask +} + +// NewTaskWaiter init new TaskWaiter for n task. +func NewTaskWaiter(n int) *TaskWaiter { + return &TaskWaiter{ + tasks: make([]*GenericTask, 0, n), + } +} + +// Add task to waiter. +func (tw *TaskWaiter) Add(t *GenericTask) { + tw.tasks = append(tw.tasks, t) +} + +// Wait for tasks to be completed. +func (tw *TaskWaiter) Wait() error { + errs := make([]error, len(tw.tasks)) + for _, t := range tw.tasks { + errs = append(errs, t.Wait()) + } + + return errors.Join(errs...) +} From 574c9f8779efc473b8d98652791e4576198f4a45 Mon Sep 17 00:00:00 2001 From: Alexandr Yudin Date: Tue, 5 Aug 2025 13:14:05 +0000 Subject: [PATCH 03/96] for save --- .../shard/segmentwriter/segment_writer.go | 53 ++++ pp/go/storage/head/shard/wal/wal.go | 275 ++++++++++++++++++ 2 files changed, 328 insertions(+) create mode 100644 pp/go/storage/head/shard/segmentwriter/segment_writer.go create mode 100644 pp/go/storage/head/shard/wal/wal.go diff --git a/pp/go/storage/head/shard/segmentwriter/segment_writer.go b/pp/go/storage/head/shard/segmentwriter/segment_writer.go new file mode 100644 index 0000000000..bd4b4118c1 --- /dev/null +++ b/pp/go/storage/head/shard/segmentwriter/segment_writer.go @@ -0,0 +1,53 @@ +package segmentwriter + +import ( + "bytes" + "io" + "os" + "sync/atomic" +) + +type SegmentIsWrittenNotifier interface { + NotifySegmentIsWritten(shardID uint16) +} + +type WriteSyncCloser interface { + io.WriteCloser + Sync() error + Stat() (os.FileInfo, error) +} + +type SegmentWriter struct { + shardID uint16 + segments []EncodedSegment + buffer *bytes.Buffer + notifier SegmentIsWrittenNotifier + writer WriteSyncCloser + currentSize int64 + writeCompleted bool +} + +func NewSegmentWriter( + shardID uint16, + writer WriteSyncCloser, + notifier SegmentIsWrittenNotifier, +) (*SegmentWriter, error) { + info, err := writer.Stat() + if err != nil { + return nil, err + } + + return &SegmentWriter{ + shardID: shardID, + buffer: bytes.NewBuffer(nil), + notifier: notifier, + writer: writer, + currentSize: info.Size(), + writeCompleted: true, + }, nil +} + +// CurrentSize return current shard wal size. +func (w *SegmentWriter) CurrentSize() int64 { + return atomic.LoadInt64(&w.currentSize) +} diff --git a/pp/go/storage/head/shard/wal/wal.go b/pp/go/storage/head/shard/wal/wal.go new file mode 100644 index 0000000000..6783f20b37 --- /dev/null +++ b/pp/go/storage/head/shard/wal/wal.go @@ -0,0 +1,275 @@ +package wal + +import ( + "encoding/binary" + "fmt" + "hash/crc32" + "io" + + "github.com/prometheus/prometheus/pp/go/cppbridge" +) + +const ( + // FileFormatVersion wal file version. + FileFormatVersion = 1 +) + +// SegmentWriter writer for wal segments. +type SegmentWriter interface { + // CurrentSize return current shard wal size. + CurrentSize() int64 + // Write encoded segment to writer. + Write(segment EncodedSegment) error + // Flush write all buffered segments. + Flush() error + // Close closes the storage. + Close() error +} + +// EncodedSegment encoded segment. +type EncodedSegment interface { + // Size returns the number of bytes in the segment. + Size() int64 + // CRC32 checksum for segment. + CRC32() uint32 + io.WriterTo + cppbridge.SegmentStats +} + +// Wal write-ahead log for [Shard]. +type Wal struct { + encoder *cppbridge.HeadWalEncoder + segmentWriter SegmentWriter + maxSegmentSize uint32 + corrupted bool + limitExhausted bool +} + +// NewWal init new [Wal]. +func NewWal(encoder *cppbridge.HeadWalEncoder, maxSegmentSize uint32, segmentWriter SegmentWriter) *Wal { + return &Wal{ + encoder: encoder, + segmentWriter: segmentWriter, + maxSegmentSize: maxSegmentSize, + } +} + +// NewCorruptedWal init new corrupted [Wal]. +func NewCorruptedWal() *Wal { + return &Wal{ + corrupted: true, + } +} + +// CurrentSize returns current wal size. +func (w *Wal) CurrentSize() int64 { + return w.segmentWriter.CurrentSize() +} + +// Write the incoming inner series to wal encoder. +func (w *Wal) Write(innerSeriesSlice []*cppbridge.InnerSeries) (bool, error) { + if w.corrupted { + return false, fmt.Errorf("writing in corrupted wal") + } + + stats, err := w.encoder.Encode(innerSeriesSlice) + if err != nil { + return false, fmt.Errorf("failed to encode inner series: %w", err) + } + + if w.maxSegmentSize == 0 { + return false, nil + } + + // memoize reaching of limits to deduplicate triggers + if !w.limitExhausted && stats.Samples() >= w.maxSegmentSize { + w.limitExhausted = true + return true, nil + } + + return false, nil +} + +// Commit finalize segment from encoder and write to [SegmentWriter]. +func (w *Wal) Commit() error { + if w.corrupted { + return fmt.Errorf("committing corrupted wal") + } + + segment, err := w.encoder.Finalize() + if err != nil { + return fmt.Errorf("failed to finalize segment: %w", err) + } + w.limitExhausted = false + + if err = w.segmentWriter.Write(segment); err != nil { + return fmt.Errorf("failed to write segment: %w", err) + } + + if err = w.segmentWriter.Flush(); err != nil { + return fmt.Errorf("failed to flush segment writer: %w", err) + } + + return nil +} + +// Flush wal [SegmentWriter]. +func (w *Wal) Flush() error { + return w.segmentWriter.Flush() +} + +// Close closes the wal segmentWriter. +func (w *Wal) Close() error { + if w.segmentWriter != nil { + return w.segmentWriter.Close() + } + + return nil +} + +// WriteHeader write header to writer. +func WriteHeader(writer io.Writer, fileFormatVersion, encoderVersion uint8) (n int, err error) { + var buf [binary.MaxVarintLen32]byte + var size int + var bytesWritten int + + size = binary.PutUvarint(buf[:], uint64(fileFormatVersion)) + bytesWritten, err = writer.Write(buf[:size]) + if err != nil { + return n, fmt.Errorf("failed to write file format version: %w", err) + } + n += bytesWritten + + size = binary.PutUvarint(buf[:], uint64(encoderVersion)) + bytesWritten, err = writer.Write(buf[:size]) + if err != nil { + return n, fmt.Errorf("failed to write encoder version: %w", err) + } + n += bytesWritten + + return n, nil +} + +type byteReader struct { + r io.Reader + n int +} + +func (r *byteReader) ReadByte() (byte, error) { + b := make([]byte, 1) + n, err := io.ReadFull(r.r, b) + if err != nil { + return 0, err + } + r.n += n + return b[0], nil +} + +// ReadHeader read header from reader. +// +//revive:disable-next-line:function-result-limit there is no point in packing it into a structure. +func ReadHeader(reader io.Reader) (fileFormatVersion, encoderVersion uint8, n int, err error) { + br := &byteReader{r: reader} + fileFormatVersionU64, err := binary.ReadUvarint(br) + if err != nil { + return 0, 0, n, fmt.Errorf("failed to read file format version: %w", err) + } + fileFormatVersion = uint8(fileFormatVersionU64) // #nosec G115 // no overflow + n = br.n + + encoderVersionU64, err := binary.ReadUvarint(br) + if err != nil { + return 0, 0, n, fmt.Errorf("failed to read encoder version: %w", err) + } + encoderVersion = uint8(encoderVersionU64) // #nosec G115 // no overflow + n = br.n + + return fileFormatVersion, encoderVersion, n, nil +} + +// WriteSegment write encoded segment to writer. +func WriteSegment(writer io.Writer, segment EncodedSegment) (n int, err error) { + var buf [binary.MaxVarintLen32]byte + var size int + var bytesWritten int + + segmentSize := uint64(segment.Size()) // #nosec G115 // no overflow + size = binary.PutUvarint(buf[:], segmentSize) + bytesWritten, err = writer.Write(buf[:size]) + if err != nil { + return n, fmt.Errorf("failed to write segment size: %w", err) + } + n += bytesWritten + + size = binary.PutUvarint(buf[:], uint64(segment.CRC32())) + bytesWritten, err = writer.Write(buf[:size]) + if err != nil { + return n, fmt.Errorf("failed to write segment crc32 hash: %w", err) + } + n += bytesWritten + + size = binary.PutUvarint(buf[:], uint64(segment.Samples())) + bytesWritten, err = writer.Write(buf[:size]) + if err != nil { + return n, fmt.Errorf("failed to write segment sample count: %w", err) + } + n += bytesWritten + + var bytesWritten64 int64 + bytesWritten64, err = segment.WriteTo(writer) + if err != nil { + return n, fmt.Errorf("failed to write segment data: %w", err) + } + n += int(bytesWritten64) + + return n, nil +} + +type DecodedSegment struct { + data []byte + sampleCount uint32 +} + +func (d DecodedSegment) Data() []byte { + return d.data +} + +func (d DecodedSegment) SampleCount() uint32 { + return d.sampleCount +} + +func ReadSegment(reader io.Reader) (decodedSegment DecodedSegment, n int, err error) { + br := &byteReader{r: reader} + var size uint64 + size, err = binary.ReadUvarint(br) + if err != nil { + return decodedSegment, br.n, fmt.Errorf("failed to read segment size: %w", err) + } + + crc32HashU64, err := binary.ReadUvarint(br) + if err != nil { + return decodedSegment, br.n, fmt.Errorf("failed to read segment crc32 hash: %w", err) + } + crc32Hash := uint32(crc32HashU64) // #nosec G115 // no overflow + + sampleCountU64, err := binary.ReadUvarint(br) + if err != nil { + return decodedSegment, br.n, fmt.Errorf("failed to read segment sample count: %w", err) + } + decodedSegment.sampleCount = uint32(sampleCountU64) // #nosec G115 // no overflow + + decodedSegment.data = make([]byte, size) + n, err = io.ReadFull(reader, decodedSegment.data) + if err != nil { + return decodedSegment, br.n, fmt.Errorf("failed to read segment data: %w", err) + } + n += br.n + + if crc32Hash != crc32.ChecksumIEEE(decodedSegment.data) { + return decodedSegment, n, fmt.Errorf( + "crc32 did not match, want: %d, have: %d", crc32Hash, crc32.ChecksumIEEE(decodedSegment.data), + ) + } + + return decodedSegment, n, nil +} From 63dc68021a90fa12451f64c498ef7144eaa560fe Mon Sep 17 00:00:00 2001 From: Alexandr Yudin Date: Tue, 5 Aug 2025 13:49:50 +0000 Subject: [PATCH 04/96] for save --- .../shard/segmentwriter/segment_writer.go | 92 ++++++++++++++++++- 1 file changed, 87 insertions(+), 5 deletions(-) diff --git a/pp/go/storage/head/shard/segmentwriter/segment_writer.go b/pp/go/storage/head/shard/segmentwriter/segment_writer.go index bd4b4118c1..02c3509280 100644 --- a/pp/go/storage/head/shard/segmentwriter/segment_writer.go +++ b/pp/go/storage/head/shard/segmentwriter/segment_writer.go @@ -2,22 +2,26 @@ package segmentwriter import ( "bytes" + "fmt" "io" "os" "sync/atomic" ) +// SegmentIsWrittenNotifier notify when new segment write. type SegmentIsWrittenNotifier interface { NotifySegmentIsWritten(shardID uint16) } +// WriteSyncCloser writer implementation [os.File]. type WriteSyncCloser interface { io.WriteCloser Sync() error Stat() (os.FileInfo, error) } -type SegmentWriter struct { +// SegmentWriter writer for segments. +type SegmentWriter[EncodedSegment any] struct { shardID uint16 segments []EncodedSegment buffer *bytes.Buffer @@ -27,17 +31,18 @@ type SegmentWriter struct { writeCompleted bool } -func NewSegmentWriter( +// NewSegmentWriter init new [SegmentWriter]. +func NewSegmentWriter[EncodedSegment any]( shardID uint16, writer WriteSyncCloser, notifier SegmentIsWrittenNotifier, -) (*SegmentWriter, error) { +) (*SegmentWriter[EncodedSegment], error) { info, err := writer.Stat() if err != nil { return nil, err } - return &SegmentWriter{ + return &SegmentWriter[EncodedSegment]{ shardID: shardID, buffer: bytes.NewBuffer(nil), notifier: notifier, @@ -47,7 +52,84 @@ func NewSegmentWriter( }, nil } +// Close closes the writer [WriteSyncCloser]. +func (w *SegmentWriter[EncodedSegment]) Close() error { + return w.writer.Close() +} + // CurrentSize return current shard wal size. -func (w *SegmentWriter) CurrentSize() int64 { +func (w *SegmentWriter[EncodedSegment]) CurrentSize() int64 { return atomic.LoadInt64(&w.currentSize) } + +// Flush and sync buffer and collected segments to [WriteSyncCloser]. +func (w *SegmentWriter[EncodedSegment]) Flush() error { + if !w.writeCompleted { + if err := w.flushAndSync(); err != nil { + return fmt.Errorf("flush and sync: %w", err) + } + } + + for index, segment := range w.segments { + if encoded, err := w.writeToBufferAndFlush(segment); err != nil { + if encoded { + index++ + } + // shift encoded segments to the left + copy(w.segments, w.segments[index:]) + w.segments = w.segments[:len(w.segments)-index] + return fmt.Errorf("flush segment: %w", err) + } + } + + w.segments = nil + return nil +} + +// Write to buffer [SegmentWriter] incoming [EncodedSegment]. +func (w *SegmentWriter[EncodedSegment]) Write(segment EncodedSegment) error { + w.segments = append(w.segments, segment) + return nil +} + +// flushAndSync write the contents from buffer to [WriteSyncCloser] and sync. +func (w *SegmentWriter[EncodedSegment]) flushAndSync() error { + n, err := w.buffer.WriteTo(w.writer) + atomic.AddInt64(&w.currentSize, n) + if err != nil { + return fmt.Errorf("buffer write: %w", err) + } + + if err := w.sync(); err != nil { + return fmt.Errorf("writer sync: %w", err) + } + + return nil +} + +// sync commits the current contents of the [WriteSyncCloser] and notify [SegmentIsWrittenNotifier]. +func (w *SegmentWriter[EncodedSegment]) sync() error { + if err := w.writer.Sync(); err != nil { + return fmt.Errorf("writer sync: %w", err) + } + + w.notifier.NotifySegmentIsWritten(w.shardID) + w.writeCompleted = true + return nil +} + +// writeToBufferAndFlush write [EncodedSegment] as slice byte to buffer and flush to [WriteSyncCloser]. +func (w *SegmentWriter[EncodedSegment]) writeToBufferAndFlush(segment EncodedSegment) (encoded bool, err error) { + if _, err := WriteSegment(w.buffer, segment); err != nil { + w.buffer.Reset() + return false, fmt.Errorf("encode segment: %w", err) + } + + w.writeCompleted = false + + if err := w.flushAndSync(); err != nil { + return true, err + } + + return true, nil +} From 6fddb82275d86bde6b564303fa7bbe60bd100619 Mon Sep 17 00:00:00 2001 From: Alexandr Yudin Date: Tue, 5 Aug 2025 16:02:31 +0000 Subject: [PATCH 05/96] for save --- pp-pkg/.golangci.yml | 12 +- pp/.golangci.yml | 12 +- pp/go/storage/head/keeper/keeper.go | 7 +- pp/go/storage/head/shard/wal/wal.go | 187 ++---------------- pp/go/storage/head/shard/wal/wal_test.go | 49 +++++ .../segment_writer.go => writer/buffered.go} | 42 ++-- .../head/shard/writer/buffered_test.go | 42 ++++ pp/go/storage/head/shard/writer/header.go | 30 +++ pp/go/storage/head/shard/writer/segment.go | 53 +++++ 9 files changed, 230 insertions(+), 204 deletions(-) create mode 100644 pp/go/storage/head/shard/wal/wal_test.go rename pp/go/storage/head/shard/{segmentwriter/segment_writer.go => writer/buffered.go} (68%) create mode 100644 pp/go/storage/head/shard/writer/buffered_test.go create mode 100644 pp/go/storage/head/shard/writer/header.go create mode 100644 pp/go/storage/head/shard/writer/segment.go diff --git a/pp-pkg/.golangci.yml b/pp-pkg/.golangci.yml index 06738ffb0c..89e6704200 100644 --- a/pp-pkg/.golangci.yml +++ b/pp-pkg/.golangci.yml @@ -30,8 +30,8 @@ output: # all available settings of specific linters linters-settings: govet: - # # report about shadowed variables - # check-shadowing: true + # report about shadowed variables + check-shadowing: true # settings per analyzer settings: printf: # analyzer name, run `go tool vet help` to see all analyzers @@ -40,10 +40,10 @@ linters-settings: - (github.com/golangci/golangci-lint/pkg/logutils.Log).Warnf - (github.com/golangci/golangci-lint/pkg/logutils.Log).Errorf - (github.com/golangci/golangci-lint/pkg/logutils.Log).Fatalf - shadow: - # Whether to be strict about shadowing; can be noisy. - # Default: false - strict: true + # shadow: + # # Whether to be strict about shadowing; can be noisy. + # # Default: false + # strict: true gofmt: # simplify code: gofmt with `-s` option, true by default simplify: true diff --git a/pp/.golangci.yml b/pp/.golangci.yml index 06738ffb0c..89e6704200 100644 --- a/pp/.golangci.yml +++ b/pp/.golangci.yml @@ -30,8 +30,8 @@ output: # all available settings of specific linters linters-settings: govet: - # # report about shadowed variables - # check-shadowing: true + # report about shadowed variables + check-shadowing: true # settings per analyzer settings: printf: # analyzer name, run `go tool vet help` to see all analyzers @@ -40,10 +40,10 @@ linters-settings: - (github.com/golangci/golangci-lint/pkg/logutils.Log).Warnf - (github.com/golangci/golangci-lint/pkg/logutils.Log).Errorf - (github.com/golangci/golangci-lint/pkg/logutils.Log).Fatalf - shadow: - # Whether to be strict about shadowing; can be noisy. - # Default: false - strict: true + # shadow: + # # Whether to be strict about shadowing; can be noisy. + # # Default: false + # strict: true gofmt: # simplify code: gofmt with `-s` option, true by default simplify: true diff --git a/pp/go/storage/head/keeper/keeper.go b/pp/go/storage/head/keeper/keeper.go index fc669d875d..7c55267844 100644 --- a/pp/go/storage/head/keeper/keeper.go +++ b/pp/go/storage/head/keeper/keeper.go @@ -50,10 +50,6 @@ type Keeper[TBlock any] struct { } func (k *Keeper[TBlock]) write() bool { - bl := k.headBlockBuilder() - - k.hbWriter.Write(bl) - k.mtx.Lock() lenHeads := len(k.heads) if lenHeads == 0 { @@ -92,7 +88,8 @@ func (k *Keeper[TBlock]) write() bool { shard.LSSLock() defer shard.LSSUnlock() - return k.hbWriter.Write(relabeler.NewBlock(shard.LSS().Raw(), shard.DataStorage().Raw())) + bl := k.headBlockBuilder() // relabeler.NewBlock(shard.LSS().Raw(), shard.DataStorage().Raw()) + return k.hbWriter.Write(bl) }, relabeler.ForLSSTask, ) diff --git a/pp/go/storage/head/shard/wal/wal.go b/pp/go/storage/head/shard/wal/wal.go index 6783f20b37..4daaf00202 100644 --- a/pp/go/storage/head/shard/wal/wal.go +++ b/pp/go/storage/head/shard/wal/wal.go @@ -1,9 +1,7 @@ package wal import ( - "encoding/binary" "fmt" - "hash/crc32" "io" "github.com/prometheus/prometheus/pp/go/cppbridge" @@ -26,28 +24,30 @@ type SegmentWriter interface { Close() error } -// EncodedSegment encoded segment. +// EncodedSegment the minimum required Segment implementation for a [Wal]. type EncodedSegment interface { - // Size returns the number of bytes in the segment. Size() int64 - // CRC32 checksum for segment. CRC32() uint32 + Samples() uint32 io.WriterTo - cppbridge.SegmentStats } // Wal write-ahead log for [Shard]. -type Wal struct { +type Wal[Writer SegmentWriter] struct { encoder *cppbridge.HeadWalEncoder - segmentWriter SegmentWriter + segmentWriter Writer maxSegmentSize uint32 corrupted bool limitExhausted bool } // NewWal init new [Wal]. -func NewWal(encoder *cppbridge.HeadWalEncoder, maxSegmentSize uint32, segmentWriter SegmentWriter) *Wal { - return &Wal{ +func NewWal[Writer SegmentWriter]( + encoder *cppbridge.HeadWalEncoder, + maxSegmentSize uint32, + segmentWriter Writer, +) *Wal[Writer] { + return &Wal[Writer]{ encoder: encoder, segmentWriter: segmentWriter, maxSegmentSize: maxSegmentSize, @@ -55,19 +55,19 @@ func NewWal(encoder *cppbridge.HeadWalEncoder, maxSegmentSize uint32, segmentWri } // NewCorruptedWal init new corrupted [Wal]. -func NewCorruptedWal() *Wal { - return &Wal{ +func NewCorruptedWal[Writer SegmentWriter]() *Wal[Writer] { + return &Wal[Writer]{ corrupted: true, } } // CurrentSize returns current wal size. -func (w *Wal) CurrentSize() int64 { +func (w *Wal[Writer]) CurrentSize() int64 { return w.segmentWriter.CurrentSize() } // Write the incoming inner series to wal encoder. -func (w *Wal) Write(innerSeriesSlice []*cppbridge.InnerSeries) (bool, error) { +func (w *Wal[Writer]) Write(innerSeriesSlice []*cppbridge.InnerSeries) (bool, error) { if w.corrupted { return false, fmt.Errorf("writing in corrupted wal") } @@ -91,7 +91,7 @@ func (w *Wal) Write(innerSeriesSlice []*cppbridge.InnerSeries) (bool, error) { } // Commit finalize segment from encoder and write to [SegmentWriter]. -func (w *Wal) Commit() error { +func (w *Wal[Writer]) Commit() error { if w.corrupted { return fmt.Errorf("committing corrupted wal") } @@ -114,162 +114,11 @@ func (w *Wal) Commit() error { } // Flush wal [SegmentWriter]. -func (w *Wal) Flush() error { +func (w *Wal[Writer]) Flush() error { return w.segmentWriter.Flush() } // Close closes the wal segmentWriter. -func (w *Wal) Close() error { - if w.segmentWriter != nil { - return w.segmentWriter.Close() - } - - return nil -} - -// WriteHeader write header to writer. -func WriteHeader(writer io.Writer, fileFormatVersion, encoderVersion uint8) (n int, err error) { - var buf [binary.MaxVarintLen32]byte - var size int - var bytesWritten int - - size = binary.PutUvarint(buf[:], uint64(fileFormatVersion)) - bytesWritten, err = writer.Write(buf[:size]) - if err != nil { - return n, fmt.Errorf("failed to write file format version: %w", err) - } - n += bytesWritten - - size = binary.PutUvarint(buf[:], uint64(encoderVersion)) - bytesWritten, err = writer.Write(buf[:size]) - if err != nil { - return n, fmt.Errorf("failed to write encoder version: %w", err) - } - n += bytesWritten - - return n, nil -} - -type byteReader struct { - r io.Reader - n int -} - -func (r *byteReader) ReadByte() (byte, error) { - b := make([]byte, 1) - n, err := io.ReadFull(r.r, b) - if err != nil { - return 0, err - } - r.n += n - return b[0], nil -} - -// ReadHeader read header from reader. -// -//revive:disable-next-line:function-result-limit there is no point in packing it into a structure. -func ReadHeader(reader io.Reader) (fileFormatVersion, encoderVersion uint8, n int, err error) { - br := &byteReader{r: reader} - fileFormatVersionU64, err := binary.ReadUvarint(br) - if err != nil { - return 0, 0, n, fmt.Errorf("failed to read file format version: %w", err) - } - fileFormatVersion = uint8(fileFormatVersionU64) // #nosec G115 // no overflow - n = br.n - - encoderVersionU64, err := binary.ReadUvarint(br) - if err != nil { - return 0, 0, n, fmt.Errorf("failed to read encoder version: %w", err) - } - encoderVersion = uint8(encoderVersionU64) // #nosec G115 // no overflow - n = br.n - - return fileFormatVersion, encoderVersion, n, nil -} - -// WriteSegment write encoded segment to writer. -func WriteSegment(writer io.Writer, segment EncodedSegment) (n int, err error) { - var buf [binary.MaxVarintLen32]byte - var size int - var bytesWritten int - - segmentSize := uint64(segment.Size()) // #nosec G115 // no overflow - size = binary.PutUvarint(buf[:], segmentSize) - bytesWritten, err = writer.Write(buf[:size]) - if err != nil { - return n, fmt.Errorf("failed to write segment size: %w", err) - } - n += bytesWritten - - size = binary.PutUvarint(buf[:], uint64(segment.CRC32())) - bytesWritten, err = writer.Write(buf[:size]) - if err != nil { - return n, fmt.Errorf("failed to write segment crc32 hash: %w", err) - } - n += bytesWritten - - size = binary.PutUvarint(buf[:], uint64(segment.Samples())) - bytesWritten, err = writer.Write(buf[:size]) - if err != nil { - return n, fmt.Errorf("failed to write segment sample count: %w", err) - } - n += bytesWritten - - var bytesWritten64 int64 - bytesWritten64, err = segment.WriteTo(writer) - if err != nil { - return n, fmt.Errorf("failed to write segment data: %w", err) - } - n += int(bytesWritten64) - - return n, nil -} - -type DecodedSegment struct { - data []byte - sampleCount uint32 -} - -func (d DecodedSegment) Data() []byte { - return d.data -} - -func (d DecodedSegment) SampleCount() uint32 { - return d.sampleCount -} - -func ReadSegment(reader io.Reader) (decodedSegment DecodedSegment, n int, err error) { - br := &byteReader{r: reader} - var size uint64 - size, err = binary.ReadUvarint(br) - if err != nil { - return decodedSegment, br.n, fmt.Errorf("failed to read segment size: %w", err) - } - - crc32HashU64, err := binary.ReadUvarint(br) - if err != nil { - return decodedSegment, br.n, fmt.Errorf("failed to read segment crc32 hash: %w", err) - } - crc32Hash := uint32(crc32HashU64) // #nosec G115 // no overflow - - sampleCountU64, err := binary.ReadUvarint(br) - if err != nil { - return decodedSegment, br.n, fmt.Errorf("failed to read segment sample count: %w", err) - } - decodedSegment.sampleCount = uint32(sampleCountU64) // #nosec G115 // no overflow - - decodedSegment.data = make([]byte, size) - n, err = io.ReadFull(reader, decodedSegment.data) - if err != nil { - return decodedSegment, br.n, fmt.Errorf("failed to read segment data: %w", err) - } - n += br.n - - if crc32Hash != crc32.ChecksumIEEE(decodedSegment.data) { - return decodedSegment, n, fmt.Errorf( - "crc32 did not match, want: %d, have: %d", crc32Hash, crc32.ChecksumIEEE(decodedSegment.data), - ) - } - - return decodedSegment, n, nil +func (w *Wal[Writer]) Close() error { + return w.segmentWriter.Close() } diff --git a/pp/go/storage/head/shard/wal/wal_test.go b/pp/go/storage/head/shard/wal/wal_test.go new file mode 100644 index 0000000000..ac95ab0748 --- /dev/null +++ b/pp/go/storage/head/shard/wal/wal_test.go @@ -0,0 +1,49 @@ +package wal_test + +import ( + "fmt" + "os" + "path/filepath" + "testing" + + "github.com/prometheus/prometheus/pp/go/cppbridge" + "github.com/prometheus/prometheus/pp/go/storage/head/shard/wal" + "github.com/prometheus/prometheus/pp/go/storage/head/shard/writer" + "github.com/stretchr/testify/require" +) + +func TestXxx(t *testing.T) { + shardID := uint16(0) + tmpDir, err := os.MkdirTemp("", "shard") + require.NoError(t, err) + defer func() { + _ = os.RemoveAll(tmpDir) + }() + + shardFile, err := os.Create(filepath.Join(filepath.Clean(tmpDir), fmt.Sprintf("shard_%d.wal", shardID))) + require.NoError(t, err) + + swn := &segmentWriteNotifier{} + + defer func() { + if err == nil { + return + } + _ = shardFile.Close() + }() + + sw, err := writer.NewBuffered(shardID, shardFile, writer.WriteSegment[wal.EncodedSegment], swn) + require.NoError(t, err) + + shardWalEncoder := &cppbridge.HeadWalEncoder{} + + wal.NewWal(shardWalEncoder, 10, sw) +} + +// segmentWriteNotifier test implementation [writer.SegmentIsWrittenNotifier]. +type segmentWriteNotifier struct{} + +// NotifySegmentIsWritten test implementation [writer.SegmentIsWrittenNotifier]. +func (*segmentWriteNotifier) NotifySegmentIsWritten(shardID uint16) { + _ = shardID +} diff --git a/pp/go/storage/head/shard/segmentwriter/segment_writer.go b/pp/go/storage/head/shard/writer/buffered.go similarity index 68% rename from pp/go/storage/head/shard/segmentwriter/segment_writer.go rename to pp/go/storage/head/shard/writer/buffered.go index 02c3509280..e6fbc9b69f 100644 --- a/pp/go/storage/head/shard/segmentwriter/segment_writer.go +++ b/pp/go/storage/head/shard/writer/buffered.go @@ -1,4 +1,4 @@ -package segmentwriter +package writer import ( "bytes" @@ -20,32 +20,38 @@ type WriteSyncCloser interface { Stat() (os.FileInfo, error) } -// SegmentWriter writer for segments. -type SegmentWriter[EncodedSegment any] struct { +// SegmentWriterFN encode to slice byte and write to [io.Writer]. +type SegmentWriterFN[Segment any] func(writer io.Writer, segment Segment) (n int, err error) + +// Buffered writer for segments. +type Buffered[Segment any] struct { shardID uint16 - segments []EncodedSegment + segments []Segment buffer *bytes.Buffer notifier SegmentIsWrittenNotifier + swriter SegmentWriterFN[Segment] writer WriteSyncCloser currentSize int64 writeCompleted bool } -// NewSegmentWriter init new [SegmentWriter]. -func NewSegmentWriter[EncodedSegment any]( +// NewBuffered init new [Buffered]. +func NewBuffered[Segment any]( shardID uint16, writer WriteSyncCloser, + swriter SegmentWriterFN[Segment], notifier SegmentIsWrittenNotifier, -) (*SegmentWriter[EncodedSegment], error) { +) (*Buffered[Segment], error) { info, err := writer.Stat() if err != nil { return nil, err } - return &SegmentWriter[EncodedSegment]{ + return &Buffered[Segment]{ shardID: shardID, buffer: bytes.NewBuffer(nil), notifier: notifier, + swriter: swriter, writer: writer, currentSize: info.Size(), writeCompleted: true, @@ -53,17 +59,17 @@ func NewSegmentWriter[EncodedSegment any]( } // Close closes the writer [WriteSyncCloser]. -func (w *SegmentWriter[EncodedSegment]) Close() error { +func (w *Buffered[Segment]) Close() error { return w.writer.Close() } // CurrentSize return current shard wal size. -func (w *SegmentWriter[EncodedSegment]) CurrentSize() int64 { +func (w *Buffered[Segment]) CurrentSize() int64 { return atomic.LoadInt64(&w.currentSize) } // Flush and sync buffer and collected segments to [WriteSyncCloser]. -func (w *SegmentWriter[EncodedSegment]) Flush() error { +func (w *Buffered[Segment]) Flush() error { if !w.writeCompleted { if err := w.flushAndSync(); err != nil { return fmt.Errorf("flush and sync: %w", err) @@ -86,14 +92,14 @@ func (w *SegmentWriter[EncodedSegment]) Flush() error { return nil } -// Write to buffer [SegmentWriter] incoming [EncodedSegment]. -func (w *SegmentWriter[EncodedSegment]) Write(segment EncodedSegment) error { +// Write to buffer [Buffered] incoming [Segment]. +func (w *Buffered[Segment]) Write(segment Segment) error { w.segments = append(w.segments, segment) return nil } // flushAndSync write the contents from buffer to [WriteSyncCloser] and sync. -func (w *SegmentWriter[EncodedSegment]) flushAndSync() error { +func (w *Buffered[Segment]) flushAndSync() error { n, err := w.buffer.WriteTo(w.writer) atomic.AddInt64(&w.currentSize, n) if err != nil { @@ -108,7 +114,7 @@ func (w *SegmentWriter[EncodedSegment]) flushAndSync() error { } // sync commits the current contents of the [WriteSyncCloser] and notify [SegmentIsWrittenNotifier]. -func (w *SegmentWriter[EncodedSegment]) sync() error { +func (w *Buffered[Segment]) sync() error { if err := w.writer.Sync(); err != nil { return fmt.Errorf("writer sync: %w", err) } @@ -118,9 +124,9 @@ func (w *SegmentWriter[EncodedSegment]) sync() error { return nil } -// writeToBufferAndFlush write [EncodedSegment] as slice byte to buffer and flush to [WriteSyncCloser]. -func (w *SegmentWriter[EncodedSegment]) writeToBufferAndFlush(segment EncodedSegment) (encoded bool, err error) { - if _, err := WriteSegment(w.buffer, segment); err != nil { +// writeToBufferAndFlush write [Segment] as slice byte to buffer and flush to [WriteSyncCloser]. +func (w *Buffered[Segment]) writeToBufferAndFlush(segment Segment) (encoded bool, err error) { + if _, err := w.swriter(w.buffer, segment); err != nil { w.buffer.Reset() return false, fmt.Errorf("encode segment: %w", err) } diff --git a/pp/go/storage/head/shard/writer/buffered_test.go b/pp/go/storage/head/shard/writer/buffered_test.go new file mode 100644 index 0000000000..66a6cce524 --- /dev/null +++ b/pp/go/storage/head/shard/writer/buffered_test.go @@ -0,0 +1,42 @@ +package writer_test + +import ( + "fmt" + "os" + "path/filepath" + "testing" + + "github.com/prometheus/prometheus/pp/go/storage/head/shard/writer" + "github.com/stretchr/testify/require" +) + +func TestXxx(t *testing.T) { + // + shardID := uint16(0) + tmpDir, err := os.MkdirTemp("", "shard") + require.NoError(t, err) + defer func() { + _ = os.RemoveAll(tmpDir) + }() + + shardFile, err := os.Create(filepath.Join(filepath.Clean(tmpDir), fmt.Sprintf("shard_%d.wal", shardID))) + require.NoError(t, err) + + swn := &segmentWriteNotifier{} + + defer func() { + if err == nil { + return + } + _ = shardFile.Close() + }() + writer.NewBuffered(shardID, shardFile, writer.WriteSegment[writer.EncodedSegment], swn) +} + +// segmentWriteNotifier test implementation [writer.SegmentIsWrittenNotifier]. +type segmentWriteNotifier struct{} + +// NotifySegmentIsWritten test implementation [writer.SegmentIsWrittenNotifier]. +func (*segmentWriteNotifier) NotifySegmentIsWritten(shardID uint16) { + _ = shardID +} diff --git a/pp/go/storage/head/shard/writer/header.go b/pp/go/storage/head/shard/writer/header.go new file mode 100644 index 0000000000..778e0ea5b5 --- /dev/null +++ b/pp/go/storage/head/shard/writer/header.go @@ -0,0 +1,30 @@ +package writer + +import ( + "encoding/binary" + "fmt" + "io" +) + +// WriteHeader encode Header to slice byte and write to [io.Writer]. +func WriteHeader(writer io.Writer, fileFormatVersion, encoderVersion uint8) (n int, err error) { + var buf [binary.MaxVarintLen32]byte + var size int + var bytesWritten int + + size = binary.PutUvarint(buf[:], uint64(fileFormatVersion)) + bytesWritten, err = writer.Write(buf[:size]) + if err != nil { + return n, fmt.Errorf("failed to write file format version: %w", err) + } + n += bytesWritten + + size = binary.PutUvarint(buf[:], uint64(encoderVersion)) + bytesWritten, err = writer.Write(buf[:size]) + if err != nil { + return n, fmt.Errorf("failed to write encoder version: %w", err) + } + n += bytesWritten + + return n, nil +} diff --git a/pp/go/storage/head/shard/writer/segment.go b/pp/go/storage/head/shard/writer/segment.go new file mode 100644 index 0000000000..aea4fbee11 --- /dev/null +++ b/pp/go/storage/head/shard/writer/segment.go @@ -0,0 +1,53 @@ +package writer + +import ( + "encoding/binary" + "fmt" + "io" +) + +// EncodedSegment the minimum required Segment implementation for a [WriteSegment]. +type EncodedSegment interface { + Size() int64 + CRC32() uint32 + Samples() uint32 + io.WriterTo +} + +// WriteSegment encode [EncodedSegment] to slice byte and write to [io.Writer]. +func WriteSegment[Segment EncodedSegment](writer io.Writer, segment Segment) (n int, err error) { + var buf [binary.MaxVarintLen32]byte + var size int + var bytesWritten int + + segmentSize := uint64(segment.Size()) // #nosec G115 // no overflow + size = binary.PutUvarint(buf[:], segmentSize) + bytesWritten, err = writer.Write(buf[:size]) + if err != nil { + return n, fmt.Errorf("failed to write segment size: %w", err) + } + n += bytesWritten + + size = binary.PutUvarint(buf[:], uint64(segment.CRC32())) + bytesWritten, err = writer.Write(buf[:size]) + if err != nil { + return n, fmt.Errorf("failed to write segment crc32 hash: %w", err) + } + n += bytesWritten + + size = binary.PutUvarint(buf[:], uint64(segment.Samples())) + bytesWritten, err = writer.Write(buf[:size]) + if err != nil { + return n, fmt.Errorf("failed to write segment sample count: %w", err) + } + n += bytesWritten + + var bytesWritten64 int64 + bytesWritten64, err = segment.WriteTo(writer) + if err != nil { + return n, fmt.Errorf("failed to write segment data: %w", err) + } + n += int(bytesWritten64) + + return n, nil +} From 841f2479d9cfe3ea3009f7ba72a4a3c42a45d68b Mon Sep 17 00:00:00 2001 From: Alexandr Yudin Date: Thu, 7 Aug 2025 06:32:40 +0000 Subject: [PATCH 06/96] for save --- pp/go/storage/head/head/head.go | 41 +++ pp/go/storage/head/shard/data_storage.go | 26 +- pp/go/storage/head/shard/lss.go | 47 ++- pp/go/storage/head/shard/shard.go | 94 +++--- pp/go/storage/head/shard/wal.go | 275 ------------------ .../head/shard/wal/reader/byte_reader.go | 30 ++ .../head/shard/wal/reader/byte_reader_test.go | 32 ++ pp/go/storage/head/shard/wal/reader/header.go | 29 ++ .../storage/head/shard/wal/reader/segment.go | 78 +++++ pp/go/storage/head/shard/wal/wal.go | 49 ++-- pp/go/storage/head/shard/wal/wal_test.go | 10 +- .../head/shard/{ => wal}/writer/buffered.go | 30 +- .../shard/{ => wal}/writer/buffered_test.go | 2 +- .../head/shard/{ => wal}/writer/header.go | 0 .../head/shard/{ => wal}/writer/segment.go | 2 +- pp/go/storage/querier/dedup.go | 46 +++ 16 files changed, 399 insertions(+), 392 deletions(-) create mode 100644 pp/go/storage/head/head/head.go delete mode 100644 pp/go/storage/head/shard/wal.go create mode 100644 pp/go/storage/head/shard/wal/reader/byte_reader.go create mode 100644 pp/go/storage/head/shard/wal/reader/byte_reader_test.go create mode 100644 pp/go/storage/head/shard/wal/reader/header.go create mode 100644 pp/go/storage/head/shard/wal/reader/segment.go rename pp/go/storage/head/shard/{ => wal}/writer/buffered.go (79%) rename pp/go/storage/head/shard/{ => wal}/writer/buffered_test.go (92%) rename pp/go/storage/head/shard/{ => wal}/writer/header.go (100%) rename pp/go/storage/head/shard/{ => wal}/writer/segment.go (92%) create mode 100644 pp/go/storage/querier/dedup.go diff --git a/pp/go/storage/head/head/head.go b/pp/go/storage/head/head/head.go new file mode 100644 index 0000000000..20aaed207c --- /dev/null +++ b/pp/go/storage/head/head/head.go @@ -0,0 +1,41 @@ +package head + +import ( + "sync" + + "github.com/prometheus/prometheus/pp/go/relabeler" + "github.com/prometheus/prometheus/pp/go/util/locker" +) + +type Shard interface { + ShardID() uint16 +} + +type Head struct { + id string + generation uint64 + readOnly bool + + shards []Shard + lssTaskChs []chan *relabeler.GenericTask + dataStorageTaskChs []chan *relabeler.GenericTask + queryLocker *locker.Weighted + + numberOfShards uint16 + stopc chan struct{} + wg sync.WaitGroup + + // // stat + // appendedSegmentCount prometheus.Counter + // memoryInUse *prometheus.GaugeVec + // series prometheus.Gauge + // walSize *prometheus.GaugeVec + // // TODO refactoring + // queueLSS *prometheus.GaugeVec + // queueDataStorage *prometheus.GaugeVec + + // tasksCreated *prometheus.CounterVec + // tasksDone *prometheus.CounterVec + // tasksLive *prometheus.CounterVec + // tasksExecute *prometheus.CounterVec +} diff --git a/pp/go/storage/head/shard/data_storage.go b/pp/go/storage/head/shard/data_storage.go index 2425dff06a..dcf10700ea 100644 --- a/pp/go/storage/head/shard/data_storage.go +++ b/pp/go/storage/head/shard/data_storage.go @@ -2,11 +2,13 @@ package shard import "github.com/prometheus/prometheus/pp/go/cppbridge" +// DataStorage samles storage with labels IDs. type DataStorage struct { dataStorage *cppbridge.HeadDataStorage encoder *cppbridge.HeadEncoder } +// NewDataStorage int new [DataStorage]. func NewDataStorage() *DataStorage { dataStorage := cppbridge.NewHeadDataStorage() return &DataStorage{ @@ -15,26 +17,34 @@ func NewDataStorage() *DataStorage { } } +// AllocatedMemory return size of allocated memory for DataStorage. +func (ds *DataStorage) AllocatedMemory() uint64 { + return ds.dataStorage.AllocatedMemory() +} + +// AppendInnerSeriesSlice add InnerSeries to storage. func (ds *DataStorage) AppendInnerSeriesSlice(innerSeriesSlice []*cppbridge.InnerSeries) { ds.encoder.EncodeInnerSeriesSlice(innerSeriesSlice) } -func (ds *DataStorage) Raw() *cppbridge.HeadDataStorage { - return ds.dataStorage +// InstantQuery make instant query to data storage and returns serialazed chunks. +func (ds *DataStorage) InstantQuery( + targetTimestamp, notFoundValueTimestampValue int64, seriesIDs []uint32, +) []cppbridge.Sample { + return ds.dataStorage.InstantQuery(targetTimestamp, notFoundValueTimestampValue, seriesIDs) } +// MergeOutOfOrderChunks merge chunks with out of order data chunks. func (ds *DataStorage) MergeOutOfOrderChunks() { ds.encoder.MergeOutOfOrderChunks() } +// Query make query to data storage and returns serialazed chunks. func (ds *DataStorage) Query(query cppbridge.HeadDataStorageQuery) *cppbridge.HeadDataStorageSerializedChunks { return ds.dataStorage.Query(query) } -func (ds *DataStorage) InstantQuery(targetTimestamp, notFoundValueTimestampValue int64, seriesIDs []uint32) []cppbridge.Sample { - return ds.dataStorage.InstantQuery(targetTimestamp, notFoundValueTimestampValue, seriesIDs) -} - -func (ds *DataStorage) AllocatedMemory() uint64 { - return ds.dataStorage.AllocatedMemory() +// Raw returns raw [cppbridge.HeadDataStorage]. +func (ds *DataStorage) Raw() *cppbridge.HeadDataStorage { + return ds.dataStorage } diff --git a/pp/go/storage/head/shard/lss.go b/pp/go/storage/head/shard/lss.go index e90cfa5279..6906fb9800 100644 --- a/pp/go/storage/head/shard/lss.go +++ b/pp/go/storage/head/shard/lss.go @@ -15,14 +15,31 @@ type LSS struct { once sync.Once } -func (w *LSS) Raw() *cppbridge.LabelSetStorage { - return w.target -} - +// AllocatedMemory return size of allocated memory for labelset storages. func (w *LSS) AllocatedMemory() uint64 { return w.input.AllocatedMemory() + w.target.AllocatedMemory() } +// GetSnapshot return the actual snapshot. +func (w *LSS) GetSnapshot() *cppbridge.LabelSetSnapshot { + w.once.Do(func() { + w.snapshot = w.target.CreateLabelSetSnapshot() + }) + + return w.snapshot +} + +// Input returns input lss. +func (w *LSS) Input() *cppbridge.LabelSetStorage { + return w.input +} + +// QueryLabelNames returns a LSSQueryLabelNamesResult that matches the given label matchers. +func (w *LSS) QueryLabelNames(matchers []model.LabelMatcher) *cppbridge.LSSQueryLabelNamesResult { + return w.target.QueryLabelNames(matchers) +} + +// QueryLabelValues returns a LSSQueryLabelValuesResult that matches the given label matchers. func (w *LSS) QueryLabelValues( label_name string, matchers []model.LabelMatcher, @@ -30,38 +47,18 @@ func (w *LSS) QueryLabelValues( return w.target.QueryLabelValues(label_name, matchers) } -func (w *LSS) QueryLabelNames(matchers []model.LabelMatcher) *cppbridge.LSSQueryLabelNamesResult { - return w.target.QueryLabelNames(matchers) -} - // QuerySelector returns a created selector that matches the given label matchers. func (w *LSS) QuerySelector(matchers []model.LabelMatcher) (selector uintptr, status uint32) { return w.target.QuerySelector(matchers) } -func (w *LSS) GetLabelSets(labelSetIDs []uint32) *cppbridge.LabelSetStorageGetLabelSetsResult { - return w.target.GetLabelSets(labelSetIDs) -} - -// GetSnapshot return the actual snapshot. -func (w *LSS) GetSnapshot() *cppbridge.LabelSetSnapshot { - w.once.Do(func() { - w.snapshot = w.target.CreateLabelSetSnapshot() - }) - - return w.snapshot -} - // ResetSnapshot resets the current snapshot. func (w *LSS) ResetSnapshot() { w.snapshot = nil w.once = sync.Once{} } -func (w *LSS) Input() *cppbridge.LabelSetStorage { - return w.input -} - +// Target returns main lss. func (w *LSS) Target() *cppbridge.LabelSetStorage { return w.target } diff --git a/pp/go/storage/head/shard/shard.go b/pp/go/storage/head/shard/shard.go index 50187dce0c..913054aa31 100644 --- a/pp/go/storage/head/shard/shard.go +++ b/pp/go/storage/head/shard/shard.go @@ -1,73 +1,75 @@ -package shard +// Shard - TODO write description on package -import "sync" +package shard -// -// RWLockable -// +import ( + "fmt" + "runtime" + "sync" -// RWLockable implementation [sync.RWMutex]. -type RWLockable interface { - Lock() - RLock() - RUnlock() - Unlock() -} + "github.com/prometheus/prometheus/pp/go/cppbridge" + "github.com/prometheus/prometheus/pp/go/model" +) // // Shard // -// Shard -type Shard struct { +// Shard bridge to labelset storage, data storage and wal. +type Shard[TWal any] struct { lss *LSS dataStorage *DataStorage - wal *Wal - lssLocker RWLockable - dataStorageLocker RWLockable - id uint16 + wal TWal + lssLocker sync.RWMutex + dataStorageLocker sync.RWMutex + // write -> append samples walLocker.Lock + // commit -> lssLocker.Rlock walLocker.Lock + // flush -> walLocker.Lock + walLocker sync.Mutex + id uint16 } // NewShard init new [Shard]. -func NewShard( +func NewShard[TWal any]( lss *LSS, dataStorage *DataStorage, - wal *Wal, + wal TWal, shardID uint16, - withLocker bool, -) *Shard { - s := &Shard{ +) *Shard[TWal] { + return &Shard[TWal]{ id: shardID, lss: lss, dataStorage: dataStorage, wal: wal, - lssLocker: &noopRWLockable{}, - dataStorageLocker: &noopRWLockable{}, - } - - if withLocker { - s.lssLocker = &sync.RWMutex{} - s.dataStorageLocker = &sync.RWMutex{} + lssLocker: sync.RWMutex{}, + dataStorageLocker: sync.RWMutex{}, + walLocker: sync.Mutex{}, } - - return s } -// -// noopRWLockable -// - -// noopRWLockable implementation sync.RWMutex, does nothing. -type noopRWLockable struct{} +// QueryLabelValues query labels values to lss and add values to +// the dedup-container that matches the given label matchers. +func (s *Shard[TWal]) QueryLabelValues( + name string, + matchers []model.LabelMatcher, + dedupAdd func(shardID uint16, snapshot *cppbridge.LabelSetSnapshot, values []string), +) error { + s.lssLocker.RLock() + queryLabelValuesResult := s.lss.QueryLabelValues(name, matchers) + snapshot := s.lss.GetSnapshot() + s.lssLocker.RUnlock() -// Lock implementation [RWLockable]. -func (*noopRWLockable) Lock() {} + if queryLabelValuesResult.Status() != cppbridge.LSSQueryStatusMatch { + return fmt.Errorf("no matches on shard: %d", s.id) + } -// RLock implementation [RWLockable]. -func (*noopRWLockable) RLock() {} + dedupAdd(s.id, snapshot, queryLabelValuesResult.Values()) + runtime.KeepAlive(queryLabelValuesResult) -// RUnlock implementation [RWLockable]. -func (*noopRWLockable) RUnlock() {} + return nil +} -// Unlock implementation [RWLockable]. -func (*noopRWLockable) Unlock() {} +// ShardID returns the shard ID. +func (s *Shard[TWal]) ShardID() uint16 { + return s.id +} diff --git a/pp/go/storage/head/shard/wal.go b/pp/go/storage/head/shard/wal.go deleted file mode 100644 index 4660648210..0000000000 --- a/pp/go/storage/head/shard/wal.go +++ /dev/null @@ -1,275 +0,0 @@ -package shard - -import ( - "encoding/binary" - "fmt" - "hash/crc32" - "io" - - "github.com/prometheus/prometheus/pp/go/cppbridge" -) - -const ( - // FileFormatVersion wal file version. - FileFormatVersion = 1 -) - -// SegmentWriter writer for wal segments. -type SegmentWriter interface { - // CurrentSize return current shard wal size. - CurrentSize() int64 - // Write encoded segment to writer. - Write(segment EncodedSegment) error - // Flush write all buffered segments. - Flush() error - // Close closes the storage. - Close() error -} - -// EncodedSegment encoded segment. -type EncodedSegment interface { - // Size returns the number of bytes in the segment. - Size() int64 - // CRC32 checksum for segment. - CRC32() uint32 - io.WriterTo - cppbridge.SegmentStats -} - -// Wal write-ahead log for [Shard]. -type Wal struct { - encoder *cppbridge.HeadWalEncoder - segmentWriter SegmentWriter - maxSegmentSize uint32 - corrupted bool - limitExhausted bool -} - -// NewWal init new [Wal]. -func NewWal(encoder *cppbridge.HeadWalEncoder, maxSegmentSize uint32, segmentWriter SegmentWriter) *Wal { - return &Wal{ - encoder: encoder, - segmentWriter: segmentWriter, - maxSegmentSize: maxSegmentSize, - } -} - -// NewCorruptedWal init new corrupted [Wal]. -func NewCorruptedWal() *Wal { - return &Wal{ - corrupted: true, - } -} - -// CurrentSize returns current wal size. -func (w *Wal) CurrentSize() int64 { - return w.segmentWriter.CurrentSize() -} - -// Write the incoming inner series to wal encoder. -func (w *Wal) Write(innerSeriesSlice []*cppbridge.InnerSeries) (bool, error) { - if w.corrupted { - return false, fmt.Errorf("writing in corrupted wal") - } - - stats, err := w.encoder.Encode(innerSeriesSlice) - if err != nil { - return false, fmt.Errorf("failed to encode inner series: %w", err) - } - - if w.maxSegmentSize == 0 { - return false, nil - } - - // memoize reaching of limits to deduplicate triggers - if !w.limitExhausted && stats.Samples() >= w.maxSegmentSize { - w.limitExhausted = true - return true, nil - } - - return false, nil -} - -// Commit finalize segment from encoder and write to [SegmentWriter]. -func (w *Wal) Commit() error { - if w.corrupted { - return fmt.Errorf("committing corrupted wal") - } - - segment, err := w.encoder.Finalize() - if err != nil { - return fmt.Errorf("failed to finalize segment: %w", err) - } - w.limitExhausted = false - - if err = w.segmentWriter.Write(segment); err != nil { - return fmt.Errorf("failed to write segment: %w", err) - } - - if err = w.segmentWriter.Flush(); err != nil { - return fmt.Errorf("failed to flush segment writer: %w", err) - } - - return nil -} - -// Flush wal [SegmentWriter]. -func (w *Wal) Flush() error { - return w.segmentWriter.Flush() -} - -// Close closes the wal segmentWriter. -func (w *Wal) Close() error { - if w.segmentWriter != nil { - return w.segmentWriter.Close() - } - - return nil -} - -// WriteHeader write header to writer. -func WriteHeader(writer io.Writer, fileFormatVersion, encoderVersion uint8) (n int, err error) { - var buf [binary.MaxVarintLen32]byte - var size int - var bytesWritten int - - size = binary.PutUvarint(buf[:], uint64(fileFormatVersion)) - bytesWritten, err = writer.Write(buf[:size]) - if err != nil { - return n, fmt.Errorf("failed to write file format version: %w", err) - } - n += bytesWritten - - size = binary.PutUvarint(buf[:], uint64(encoderVersion)) - bytesWritten, err = writer.Write(buf[:size]) - if err != nil { - return n, fmt.Errorf("failed to write encoder version: %w", err) - } - n += bytesWritten - - return n, nil -} - -type byteReader struct { - r io.Reader - n int -} - -func (r *byteReader) ReadByte() (byte, error) { - b := make([]byte, 1) - n, err := io.ReadFull(r.r, b) - if err != nil { - return 0, err - } - r.n += n - return b[0], nil -} - -// ReadHeader read header from reader. -// -//revive:disable-next-line:function-result-limit there is no point in packing it into a structure. -func ReadHeader(reader io.Reader) (fileFormatVersion, encoderVersion uint8, n int, err error) { - br := &byteReader{r: reader} - fileFormatVersionU64, err := binary.ReadUvarint(br) - if err != nil { - return 0, 0, n, fmt.Errorf("failed to read file format version: %w", err) - } - fileFormatVersion = uint8(fileFormatVersionU64) // #nosec G115 // no overflow - n = br.n - - encoderVersionU64, err := binary.ReadUvarint(br) - if err != nil { - return 0, 0, n, fmt.Errorf("failed to read encoder version: %w", err) - } - encoderVersion = uint8(encoderVersionU64) // #nosec G115 // no overflow - n = br.n - - return fileFormatVersion, encoderVersion, n, nil -} - -// WriteSegment write encoded segment to writer. -func WriteSegment(writer io.Writer, segment EncodedSegment) (n int, err error) { - var buf [binary.MaxVarintLen32]byte - var size int - var bytesWritten int - - segmentSize := uint64(segment.Size()) // #nosec G115 // no overflow - size = binary.PutUvarint(buf[:], segmentSize) - bytesWritten, err = writer.Write(buf[:size]) - if err != nil { - return n, fmt.Errorf("failed to write segment size: %w", err) - } - n += bytesWritten - - size = binary.PutUvarint(buf[:], uint64(segment.CRC32())) - bytesWritten, err = writer.Write(buf[:size]) - if err != nil { - return n, fmt.Errorf("failed to write segment crc32 hash: %w", err) - } - n += bytesWritten - - size = binary.PutUvarint(buf[:], uint64(segment.Samples())) - bytesWritten, err = writer.Write(buf[:size]) - if err != nil { - return n, fmt.Errorf("failed to write segment sample count: %w", err) - } - n += bytesWritten - - var bytesWritten64 int64 - bytesWritten64, err = segment.WriteTo(writer) - if err != nil { - return n, fmt.Errorf("failed to write segment data: %w", err) - } - n += int(bytesWritten64) - - return n, nil -} - -type DecodedSegment struct { - data []byte - sampleCount uint32 -} - -func (d DecodedSegment) Data() []byte { - return d.data -} - -func (d DecodedSegment) SampleCount() uint32 { - return d.sampleCount -} - -func ReadSegment(reader io.Reader) (decodedSegment DecodedSegment, n int, err error) { - br := &byteReader{r: reader} - var size uint64 - size, err = binary.ReadUvarint(br) - if err != nil { - return decodedSegment, br.n, fmt.Errorf("failed to read segment size: %w", err) - } - - crc32HashU64, err := binary.ReadUvarint(br) - if err != nil { - return decodedSegment, br.n, fmt.Errorf("failed to read segment crc32 hash: %w", err) - } - crc32Hash := uint32(crc32HashU64) // #nosec G115 // no overflow - - sampleCountU64, err := binary.ReadUvarint(br) - if err != nil { - return decodedSegment, br.n, fmt.Errorf("failed to read segment sample count: %w", err) - } - decodedSegment.sampleCount = uint32(sampleCountU64) // #nosec G115 // no overflow - - decodedSegment.data = make([]byte, size) - n, err = io.ReadFull(reader, decodedSegment.data) - if err != nil { - return decodedSegment, br.n, fmt.Errorf("failed to read segment data: %w", err) - } - n += br.n - - if crc32Hash != crc32.ChecksumIEEE(decodedSegment.data) { - return decodedSegment, n, fmt.Errorf( - "crc32 did not match, want: %d, have: %d", crc32Hash, crc32.ChecksumIEEE(decodedSegment.data), - ) - } - - return decodedSegment, n, nil -} diff --git a/pp/go/storage/head/shard/wal/reader/byte_reader.go b/pp/go/storage/head/shard/wal/reader/byte_reader.go new file mode 100644 index 0000000000..54fe2354e5 --- /dev/null +++ b/pp/go/storage/head/shard/wal/reader/byte_reader.go @@ -0,0 +1,30 @@ +package reader + +import "io" + +// byteReader reads from the reader 1 byte at a time. +type byteReader struct { + r io.Reader + buf []byte + n int +} + +// newByteReader init new [byteReader] +func newByteReader(r io.Reader) *byteReader { + return &byteReader{ + r: r, + buf: make([]byte, 1), + } +} + +// ReadByte reads from the reader 1 byte. +func (r *byteReader) ReadByte() (byte, error) { + n, err := io.ReadFull(r.r, r.buf) + if err != nil { + return 0, err + } + + r.n += n + + return r.buf[0], nil +} diff --git a/pp/go/storage/head/shard/wal/reader/byte_reader_test.go b/pp/go/storage/head/shard/wal/reader/byte_reader_test.go new file mode 100644 index 0000000000..ab133840bd --- /dev/null +++ b/pp/go/storage/head/shard/wal/reader/byte_reader_test.go @@ -0,0 +1,32 @@ +package reader + +import ( + "bytes" + "testing" +) + +func TestXxx(t *testing.T) { + bb := &bytes.Buffer{} + br := newByteReader(bb) + + bb.Write([]byte{1, 2, 3, 0}) + + t.Log(br.ReadByte()) + t.Log(br.ReadByte()) + t.Log(br.ReadByte()) + t.Log(br.ReadByte()) +} + +func BenchmarkBR1(b *testing.B) { + bb := &bytes.Buffer{} + br := newByteReader(bb) + + buf := []byte{1, 2, 3} + + for i := 0; i < b.N; i++ { + bb.Write(buf) + br.ReadByte() + br.ReadByte() + br.ReadByte() + } +} diff --git a/pp/go/storage/head/shard/wal/reader/header.go b/pp/go/storage/head/shard/wal/reader/header.go new file mode 100644 index 0000000000..2097f1b4e4 --- /dev/null +++ b/pp/go/storage/head/shard/wal/reader/header.go @@ -0,0 +1,29 @@ +package reader + +import ( + "encoding/binary" + "fmt" + "io" +) + +// ReadHeader read header from reader. +// +//revive:disable-next-line:function-result-limit there is no point in packing it into a structure. +func ReadHeader(reader io.Reader) (fileFormatVersion, encoderVersion uint8, n int, err error) { + br := newByteReader(reader) + fileFormatVersionU64, err := binary.ReadUvarint(br) + if err != nil { + return 0, 0, n, fmt.Errorf("failed to read file format version: %w", err) + } + fileFormatVersion = uint8(fileFormatVersionU64) // #nosec G115 // no overflow + n = br.n + + encoderVersionU64, err := binary.ReadUvarint(br) + if err != nil { + return 0, 0, n, fmt.Errorf("failed to read encoder version: %w", err) + } + encoderVersion = uint8(encoderVersionU64) // #nosec G115 // no overflow + n = br.n + + return fileFormatVersion, encoderVersion, n, nil +} diff --git a/pp/go/storage/head/shard/wal/reader/segment.go b/pp/go/storage/head/shard/wal/reader/segment.go new file mode 100644 index 0000000000..94734ddc7d --- /dev/null +++ b/pp/go/storage/head/shard/wal/reader/segment.go @@ -0,0 +1,78 @@ +package reader + +import ( + "encoding/binary" + "fmt" + "hash/crc32" + "io" +) + +// TODO rebuild ReadFrom + +// Segment encoded segment from wal. +type Segment struct { + data []byte + sampleCount uint32 +} + +// Bytes returns the data read. +func (s *Segment) Bytes() []byte { + return s.data +} + +// Reset segment data. +func (s *Segment) Reset() { + s.data = s.data[:0] + s.sampleCount = 0 +} + +// Samples returns count of samples in segment. +func (s *Segment) Samples() uint32 { + return s.sampleCount +} + +// resize segment data. +func (s *Segment) resize(size int) { + if cap(s.data) < size { + s.data = make([]byte, size) + } else { + s.data = s.data[:size] + } +} + +// ReadSegment read and decode [Segment] from [io.Reader] and returns. +func ReadSegment(reader io.Reader, segment *Segment) (n int, err error) { + br := newByteReader(reader) + var size uint64 + size, err = binary.ReadUvarint(br) + if err != nil { + return br.n, fmt.Errorf("failed to read segment size: %w", err) + } + + crc32HashU64, err := binary.ReadUvarint(br) + if err != nil { + return br.n, fmt.Errorf("failed to read segment crc32 hash: %w", err) + } + crc32Hash := uint32(crc32HashU64) // #nosec G115 // no overflow + + sampleCountU64, err := binary.ReadUvarint(br) + if err != nil { + return br.n, fmt.Errorf("failed to read segment sample count: %w", err) + } + segment.sampleCount = uint32(sampleCountU64) // #nosec G115 // no overflow + + segment.resize(int(size)) // #nosec G115 // no overflow + n, err = io.ReadFull(reader, segment.data) + if err != nil { + return br.n, fmt.Errorf("failed to read segment data: %w", err) + } + n += br.n + + if crc32Hash != crc32.ChecksumIEEE(segment.data) { + return n, fmt.Errorf( + "crc32 did not match, want: %d, have: %d", crc32Hash, crc32.ChecksumIEEE(segment.data), + ) + } + + return n, nil +} diff --git a/pp/go/storage/head/shard/wal/wal.go b/pp/go/storage/head/shard/wal/wal.go index 4daaf00202..3474b6e479 100644 --- a/pp/go/storage/head/shard/wal/wal.go +++ b/pp/go/storage/head/shard/wal/wal.go @@ -13,17 +13,28 @@ const ( ) // SegmentWriter writer for wal segments. -type SegmentWriter interface { +type SegmentWriter[TSegment EncodedSegment] interface { // CurrentSize return current shard wal size. CurrentSize() int64 // Write encoded segment to writer. - Write(segment EncodedSegment) error + Write(segment TSegment) error // Flush write all buffered segments. Flush() error // Close closes the storage. Close() error } +// Encoder the minimum required Encoder implementation for a [Wal]. +type Encoder[TSegment EncodedSegment, TStats StatsSegment] interface { + Encode(innerSeriesSlice []*cppbridge.InnerSeries) (TStats, error) + Finalize() (TSegment, error) +} + +// StatsSegment stats data for [Encoder]. +type StatsSegment interface { + Samples() uint32 +} + // EncodedSegment the minimum required Segment implementation for a [Wal]. type EncodedSegment interface { Size() int64 @@ -33,21 +44,21 @@ type EncodedSegment interface { } // Wal write-ahead log for [Shard]. -type Wal[Writer SegmentWriter] struct { - encoder *cppbridge.HeadWalEncoder - segmentWriter Writer +type Wal[TSegment EncodedSegment, TStats StatsSegment, TWriter SegmentWriter[TSegment]] struct { + encoder Encoder[TSegment, TStats] // *cppbridge.HeadWalEncoder + segmentWriter TWriter maxSegmentSize uint32 corrupted bool limitExhausted bool } // NewWal init new [Wal]. -func NewWal[Writer SegmentWriter]( - encoder *cppbridge.HeadWalEncoder, +func NewWal[TSegment EncodedSegment, TStats StatsSegment, TWriter SegmentWriter[TSegment]]( + encoder Encoder[TSegment, TStats], + segmentWriter TWriter, maxSegmentSize uint32, - segmentWriter Writer, -) *Wal[Writer] { - return &Wal[Writer]{ +) *Wal[TSegment, TStats, TWriter] { + return &Wal[TSegment, TStats, TWriter]{ encoder: encoder, segmentWriter: segmentWriter, maxSegmentSize: maxSegmentSize, @@ -55,19 +66,23 @@ func NewWal[Writer SegmentWriter]( } // NewCorruptedWal init new corrupted [Wal]. -func NewCorruptedWal[Writer SegmentWriter]() *Wal[Writer] { - return &Wal[Writer]{ +func NewCorruptedWal[ + TSegment EncodedSegment, + TStats StatsSegment, + TWriter SegmentWriter[TSegment], +]() *Wal[TSegment, TStats, TWriter] { + return &Wal[TSegment, TStats, TWriter]{ corrupted: true, } } // CurrentSize returns current wal size. -func (w *Wal[Writer]) CurrentSize() int64 { +func (w *Wal[TSegment, TStats, TWriter]) CurrentSize() int64 { return w.segmentWriter.CurrentSize() } // Write the incoming inner series to wal encoder. -func (w *Wal[Writer]) Write(innerSeriesSlice []*cppbridge.InnerSeries) (bool, error) { +func (w *Wal[TSegment, TStats, TWriter]) Write(innerSeriesSlice []*cppbridge.InnerSeries) (bool, error) { if w.corrupted { return false, fmt.Errorf("writing in corrupted wal") } @@ -91,7 +106,7 @@ func (w *Wal[Writer]) Write(innerSeriesSlice []*cppbridge.InnerSeries) (bool, er } // Commit finalize segment from encoder and write to [SegmentWriter]. -func (w *Wal[Writer]) Commit() error { +func (w *Wal[TSegment, TStats, TWriter]) Commit() error { if w.corrupted { return fmt.Errorf("committing corrupted wal") } @@ -114,11 +129,11 @@ func (w *Wal[Writer]) Commit() error { } // Flush wal [SegmentWriter]. -func (w *Wal[Writer]) Flush() error { +func (w *Wal[TSegment, TStats, TWriter]) Flush() error { return w.segmentWriter.Flush() } // Close closes the wal segmentWriter. -func (w *Wal[Writer]) Close() error { +func (w *Wal[TSegment, TStats, TWriter]) Close() error { return w.segmentWriter.Close() } diff --git a/pp/go/storage/head/shard/wal/wal_test.go b/pp/go/storage/head/shard/wal/wal_test.go index ac95ab0748..d5f3083a0b 100644 --- a/pp/go/storage/head/shard/wal/wal_test.go +++ b/pp/go/storage/head/shard/wal/wal_test.go @@ -6,10 +6,11 @@ import ( "path/filepath" "testing" + "github.com/stretchr/testify/require" + "github.com/prometheus/prometheus/pp/go/cppbridge" "github.com/prometheus/prometheus/pp/go/storage/head/shard/wal" - "github.com/prometheus/prometheus/pp/go/storage/head/shard/writer" - "github.com/stretchr/testify/require" + "github.com/prometheus/prometheus/pp/go/storage/head/shard/wal/writer" ) func TestXxx(t *testing.T) { @@ -32,12 +33,13 @@ func TestXxx(t *testing.T) { _ = shardFile.Close() }() - sw, err := writer.NewBuffered(shardID, shardFile, writer.WriteSegment[wal.EncodedSegment], swn) + sw, err := writer.NewBuffered(shardID, shardFile, writer.WriteSegment[*cppbridge.EncodedSegment], swn) require.NoError(t, err) shardWalEncoder := &cppbridge.HeadWalEncoder{} - wal.NewWal(shardWalEncoder, 10, sw) + wl := wal.NewWal(shardWalEncoder, sw, 10) + _ = wl } // segmentWriteNotifier test implementation [writer.SegmentIsWrittenNotifier]. diff --git a/pp/go/storage/head/shard/writer/buffered.go b/pp/go/storage/head/shard/wal/writer/buffered.go similarity index 79% rename from pp/go/storage/head/shard/writer/buffered.go rename to pp/go/storage/head/shard/wal/writer/buffered.go index e6fbc9b69f..2b8ea348fe 100644 --- a/pp/go/storage/head/shard/writer/buffered.go +++ b/pp/go/storage/head/shard/wal/writer/buffered.go @@ -21,33 +21,33 @@ type WriteSyncCloser interface { } // SegmentWriterFN encode to slice byte and write to [io.Writer]. -type SegmentWriterFN[Segment any] func(writer io.Writer, segment Segment) (n int, err error) +type SegmentWriterFN[TSegment any] func(writer io.Writer, segment TSegment) (n int, err error) // Buffered writer for segments. -type Buffered[Segment any] struct { +type Buffered[TSegment any] struct { shardID uint16 - segments []Segment + segments []TSegment buffer *bytes.Buffer notifier SegmentIsWrittenNotifier - swriter SegmentWriterFN[Segment] + swriter SegmentWriterFN[TSegment] writer WriteSyncCloser currentSize int64 writeCompleted bool } // NewBuffered init new [Buffered]. -func NewBuffered[Segment any]( +func NewBuffered[TSegment any]( shardID uint16, writer WriteSyncCloser, - swriter SegmentWriterFN[Segment], + swriter SegmentWriterFN[TSegment], notifier SegmentIsWrittenNotifier, -) (*Buffered[Segment], error) { +) (*Buffered[TSegment], error) { info, err := writer.Stat() if err != nil { return nil, err } - return &Buffered[Segment]{ + return &Buffered[TSegment]{ shardID: shardID, buffer: bytes.NewBuffer(nil), notifier: notifier, @@ -59,17 +59,17 @@ func NewBuffered[Segment any]( } // Close closes the writer [WriteSyncCloser]. -func (w *Buffered[Segment]) Close() error { +func (w *Buffered[TSegment]) Close() error { return w.writer.Close() } // CurrentSize return current shard wal size. -func (w *Buffered[Segment]) CurrentSize() int64 { +func (w *Buffered[TSegment]) CurrentSize() int64 { return atomic.LoadInt64(&w.currentSize) } // Flush and sync buffer and collected segments to [WriteSyncCloser]. -func (w *Buffered[Segment]) Flush() error { +func (w *Buffered[TSegment]) Flush() error { if !w.writeCompleted { if err := w.flushAndSync(); err != nil { return fmt.Errorf("flush and sync: %w", err) @@ -93,13 +93,13 @@ func (w *Buffered[Segment]) Flush() error { } // Write to buffer [Buffered] incoming [Segment]. -func (w *Buffered[Segment]) Write(segment Segment) error { +func (w *Buffered[TSegment]) Write(segment TSegment) error { w.segments = append(w.segments, segment) return nil } // flushAndSync write the contents from buffer to [WriteSyncCloser] and sync. -func (w *Buffered[Segment]) flushAndSync() error { +func (w *Buffered[TSegment]) flushAndSync() error { n, err := w.buffer.WriteTo(w.writer) atomic.AddInt64(&w.currentSize, n) if err != nil { @@ -114,7 +114,7 @@ func (w *Buffered[Segment]) flushAndSync() error { } // sync commits the current contents of the [WriteSyncCloser] and notify [SegmentIsWrittenNotifier]. -func (w *Buffered[Segment]) sync() error { +func (w *Buffered[TSegment]) sync() error { if err := w.writer.Sync(); err != nil { return fmt.Errorf("writer sync: %w", err) } @@ -125,7 +125,7 @@ func (w *Buffered[Segment]) sync() error { } // writeToBufferAndFlush write [Segment] as slice byte to buffer and flush to [WriteSyncCloser]. -func (w *Buffered[Segment]) writeToBufferAndFlush(segment Segment) (encoded bool, err error) { +func (w *Buffered[TSegment]) writeToBufferAndFlush(segment TSegment) (encoded bool, err error) { if _, err := w.swriter(w.buffer, segment); err != nil { w.buffer.Reset() return false, fmt.Errorf("encode segment: %w", err) diff --git a/pp/go/storage/head/shard/writer/buffered_test.go b/pp/go/storage/head/shard/wal/writer/buffered_test.go similarity index 92% rename from pp/go/storage/head/shard/writer/buffered_test.go rename to pp/go/storage/head/shard/wal/writer/buffered_test.go index 66a6cce524..8f373cc07a 100644 --- a/pp/go/storage/head/shard/writer/buffered_test.go +++ b/pp/go/storage/head/shard/wal/writer/buffered_test.go @@ -6,7 +6,7 @@ import ( "path/filepath" "testing" - "github.com/prometheus/prometheus/pp/go/storage/head/shard/writer" + "github.com/prometheus/prometheus/pp/go/storage/head/shard/wal/writer" "github.com/stretchr/testify/require" ) diff --git a/pp/go/storage/head/shard/writer/header.go b/pp/go/storage/head/shard/wal/writer/header.go similarity index 100% rename from pp/go/storage/head/shard/writer/header.go rename to pp/go/storage/head/shard/wal/writer/header.go diff --git a/pp/go/storage/head/shard/writer/segment.go b/pp/go/storage/head/shard/wal/writer/segment.go similarity index 92% rename from pp/go/storage/head/shard/writer/segment.go rename to pp/go/storage/head/shard/wal/writer/segment.go index aea4fbee11..ce3b3b6626 100644 --- a/pp/go/storage/head/shard/writer/segment.go +++ b/pp/go/storage/head/shard/wal/writer/segment.go @@ -15,7 +15,7 @@ type EncodedSegment interface { } // WriteSegment encode [EncodedSegment] to slice byte and write to [io.Writer]. -func WriteSegment[Segment EncodedSegment](writer io.Writer, segment Segment) (n int, err error) { +func WriteSegment[TSegment EncodedSegment](writer io.Writer, segment TSegment) (n int, err error) { var buf [binary.MaxVarintLen32]byte var size int var bytesWritten int diff --git a/pp/go/storage/querier/dedup.go b/pp/go/storage/querier/dedup.go new file mode 100644 index 0000000000..d010dd4ebc --- /dev/null +++ b/pp/go/storage/querier/dedup.go @@ -0,0 +1,46 @@ +package querier + +import ( + "runtime" + "strings" + "sync/atomic" + + "github.com/prometheus/prometheus/pp/go/cppbridge" +) + +// NoOpShardedDeduplicator container for collecting values without deduplication. +type NoOpShardedDeduplicator struct { + shardedValues [][]string + // TODO snapshots any + snapshots []*cppbridge.LabelSetSnapshot + count uint64 +} + +// NewNoOpShardedDeduplicator int new [NoOpShardedDeduplicator]. +func NewNoOpShardedDeduplicator(numberOfShards uint16) *NoOpShardedDeduplicator { + return &NoOpShardedDeduplicator{ + shardedValues: make([][]string, numberOfShards), + snapshots: make([]*cppbridge.LabelSetSnapshot, numberOfShards), + } +} + +// Add values to deduplicator by shard ID. +func (d *NoOpShardedDeduplicator) Add(shardID uint16, snapshot *cppbridge.LabelSetSnapshot, values []string) { + d.shardedValues[shardID] = make([]string, len(values)) + n := copy(d.shardedValues[shardID], values) + atomic.AddUint64(&d.count, uint64(n)) // #nosec G115 // no overflow + d.snapshots[shardID] = snapshot +} + +// Values returns collected values. +func (d *NoOpShardedDeduplicator) Values() []string { + values := make([]string, 0, d.count) + for _, shardedValues := range d.shardedValues { + for _, v := range shardedValues { + values = append(values, strings.Clone(v)) + } + } + runtime.KeepAlive(d.snapshots) + + return values +} From a4f4640397ed0ccc08537a3b080119c3a0d5b9c6 Mon Sep 17 00:00:00 2001 From: Alexandr Yudin Date: Thu, 7 Aug 2025 10:16:10 +0000 Subject: [PATCH 07/96] for save --- pp/go/storage/head/head/head.go | 35 +++++- pp/go/storage/head/head/head_test.go | 18 +++ pp/go/storage/head/task/task.go | 102 +++++++++++++++ pp/go/storage/querier/dedup.go | 4 +- pp/go/storage/querier/metrics.go | 65 ++++++++++ pp/go/storage/querier/querier.go | 182 +++++++++++++++++++++++++++ 6 files changed, 401 insertions(+), 5 deletions(-) create mode 100644 pp/go/storage/head/head/head_test.go create mode 100644 pp/go/storage/head/task/task.go create mode 100644 pp/go/storage/querier/metrics.go create mode 100644 pp/go/storage/querier/querier.go diff --git a/pp/go/storage/head/head/head.go b/pp/go/storage/head/head/head.go index 20aaed207c..3bef8232d1 100644 --- a/pp/go/storage/head/head/head.go +++ b/pp/go/storage/head/head/head.go @@ -1,13 +1,21 @@ package head import ( + "context" "sync" - "github.com/prometheus/prometheus/pp/go/relabeler" + "github.com/prometheus/prometheus/pp/go/cppbridge" + "github.com/prometheus/prometheus/pp/go/model" + "github.com/prometheus/prometheus/pp/go/storage" "github.com/prometheus/prometheus/pp/go/util/locker" ) type Shard interface { + QueryLabelValues( + name string, + matchers []model.LabelMatcher, + dedupAdd func(shardID uint16, snapshot *cppbridge.LabelSetSnapshot, values []string), + ) error ShardID() uint16 } @@ -17,8 +25,8 @@ type Head struct { readOnly bool shards []Shard - lssTaskChs []chan *relabeler.GenericTask - dataStorageTaskChs []chan *relabeler.GenericTask + lssTaskChs []chan *storage.GenericTask + dataStorageTaskChs []chan *storage.GenericTask queryLocker *locker.Weighted numberOfShards uint16 @@ -39,3 +47,24 @@ type Head struct { // tasksLive *prometheus.CounterVec // tasksExecute *prometheus.CounterVec } + +func NewHead(shards []Shard) *Head { + return &Head{ + shards: shards, + } +} + +// CreateTask create a task for operations on the head shards. +func (h *Head) CreateTask(taskName string, fn func(shard TShard) error, isLss bool) TGenericTask { +} + +// Enqueue the task to be executed on head. +func (h *Head) Enqueue(t TGenericTask) + +// NumberOfShards returns current number of shards. +func (h *Head) NumberOfShards() uint16 { + return h.numberOfShards +} + +// RLockQuery locks for query to [Head]. +func (h *Head) RLockQuery(ctx context.Context) (runlock func(), err error) diff --git a/pp/go/storage/head/head/head_test.go b/pp/go/storage/head/head/head_test.go new file mode 100644 index 0000000000..b03d3e0887 --- /dev/null +++ b/pp/go/storage/head/head/head_test.go @@ -0,0 +1,18 @@ +package head_test + +import ( + "testing" + + "github.com/prometheus/prometheus/pp/go/storage/head/head" + "github.com/prometheus/prometheus/pp/go/storage/head/shard" +) + +func TestXxx(t *testing.T) { + lss := &shard.LSS{} + ds := shard.NewDataStorage() + wl := struct{}{} + sd := shard.NewShard(lss, ds, wl, 0) + + h := head.NewHead([]head.Shard{sd}) + _ = h +} diff --git a/pp/go/storage/head/task/task.go b/pp/go/storage/head/task/task.go new file mode 100644 index 0000000000..52303cba29 --- /dev/null +++ b/pp/go/storage/head/task/task.go @@ -0,0 +1,102 @@ +package task + +import ( + "errors" + "sync" + "sync/atomic" + "time" + + "github.com/prometheus/client_golang/prometheus" + "github.com/prometheus/prometheus/pp/go/cppbridge" + "github.com/prometheus/prometheus/pp/go/model" +) + +// +// Shard +// + +// Shard the minimum required head Shard implementation. +type Shard interface { + QueryLabelValues( + name string, + matchers []model.LabelMatcher, + dedupAdd func(shardID uint16, snapshot *cppbridge.LabelSetSnapshot, values []string), + ) error +} + +// +// GenericTask +// + +// Generic generic task, will be executed on each shard. +type Generic[TShard Shard] struct { + errs []error + shardFn func(shard TShard) error + wg sync.WaitGroup + createdTS int64 + executeTS int64 + created prometheus.Counter + done prometheus.Counter + live prometheus.Counter + execute prometheus.Counter + forLSS bool +} + +// NewGenericTask init new [Generic]. +func NewGenericTask( + shardFn func(shard TShard) error, + created, done, live, execute prometheus.Counter, + forLSS bool, +) *Generic { + t := &Generic{ + shardFn: shardFn, + wg: sync.WaitGroup{}, + createdTS: time.Now().UnixMicro(), + created: created, + done: done, + live: live, + execute: execute, + forLSS: forLSS, + } + t.created.Inc() + + return t +} + +// NewReadOnlyGenericTask init new GenericTask for read only head. +func NewReadOnlyGenericTask(shardFn func(shard TShard) error) *Generic { + t := &Generic{ + shardFn: shardFn, + wg: sync.WaitGroup{}, + } + + return t +} + +// SetShardsNumber set shards number +func (t *Generic) SetShardsNumber(number uint16) { + t.errs = make([]error, number) + t.wg.Add(int(number)) +} + +// ExecuteOnShard execute task on shard. +func (t *Generic) ExecuteOnShard(shard Shard) { + atomic.CompareAndSwapInt64(&t.executeTS, 0, time.Now().UnixMicro()) + t.errs[shard.ShardID()] = t.shardFn(shard) + t.wg.Done() +} + +// Wait for the task to complete on all shards. +func (t *Generic) Wait() error { + t.wg.Wait() + if t.done == nil { + return errors.Join(t.errs...) + } + + now := time.Now().UnixMicro() + t.done.Inc() + t.execute.Add(float64(now - t.executeTS)) + t.live.Add(float64(now - t.createdTS)) + + return errors.Join(t.errs...) +} diff --git a/pp/go/storage/querier/dedup.go b/pp/go/storage/querier/dedup.go index d010dd4ebc..9ef6666de8 100644 --- a/pp/go/storage/querier/dedup.go +++ b/pp/go/storage/querier/dedup.go @@ -8,7 +8,7 @@ import ( "github.com/prometheus/prometheus/pp/go/cppbridge" ) -// NoOpShardedDeduplicator container for collecting values without deduplication. +// NoOpShardedDeduplicator container for accumulating values without deduplication. type NoOpShardedDeduplicator struct { shardedValues [][]string // TODO snapshots any @@ -17,7 +17,7 @@ type NoOpShardedDeduplicator struct { } // NewNoOpShardedDeduplicator int new [NoOpShardedDeduplicator]. -func NewNoOpShardedDeduplicator(numberOfShards uint16) *NoOpShardedDeduplicator { +func NewNoOpShardedDeduplicator(numberOfShards uint16) Deduplicator { return &NoOpShardedDeduplicator{ shardedValues: make([][]string, numberOfShards), snapshots: make([]*cppbridge.LabelSetSnapshot, numberOfShards), diff --git a/pp/go/storage/querier/metrics.go b/pp/go/storage/querier/metrics.go new file mode 100644 index 0000000000..530ee752ca --- /dev/null +++ b/pp/go/storage/querier/metrics.go @@ -0,0 +1,65 @@ +package querier + +import ( + "github.com/prometheus/client_golang/prometheus" + "github.com/prometheus/prometheus/pp/go/util" +) + +const ( + // QueryableAppenderSource metrics source for Appender. + QueryableAppenderSource = "queryable_appender" + // QueryableStorageSource metrics source for Storage. + QueryableStorageSource = "queryable_storage" +) + +type Metrics struct { + LabelNamesDuration prometheus.Histogram + LabelValuesDuration prometheus.Histogram + SelectDuration *prometheus.HistogramVec +} + +func NewMetrics(registerer prometheus.Registerer, source string) *Metrics { + factory := util.NewUnconflictRegisterer(registerer) + return &Metrics{ + LabelNamesDuration: factory.NewHistogram( + prometheus.HistogramOpts{ + Name: "prompp_head_query_label_names_duration", + Help: "Label names query from head duration in microseconds", + Buckets: []float64{ + 50, 100, 250, 500, 750, + 1000, 2500, 5000, 7500, + 10000, 25000, 50000, 75000, + 100000, 500000, + }, + ConstLabels: prometheus.Labels{"source": source}, + }, + ), + LabelValuesDuration: factory.NewHistogram( + prometheus.HistogramOpts{ + Name: "prompp_head_query_label_values_duration", + Help: "Label values query from head duration in microseconds", + Buckets: []float64{ + 50, 100, 250, 500, 750, + 1000, 2500, 5000, 7500, + 10000, 25000, 50000, 75000, + 100000, 500000, + }, + ConstLabels: prometheus.Labels{"source": source}, + }, + ), + SelectDuration: factory.NewHistogramVec( + prometheus.HistogramOpts{ + Name: "prompp_head_query_select_duration", + Help: "Select query from head duration in microseconds", + Buckets: []float64{ + 50, 100, 250, 500, 750, + 1000, 2500, 5000, 7500, + 10000, 25000, 50000, 75000, + 100000, 500000, + }, + ConstLabels: prometheus.Labels{"source": source}, + }, + []string{"query_type"}, + ), + } +} diff --git a/pp/go/storage/querier/querier.go b/pp/go/storage/querier/querier.go new file mode 100644 index 0000000000..9fe6c87085 --- /dev/null +++ b/pp/go/storage/querier/querier.go @@ -0,0 +1,182 @@ +package querier + +import ( + "context" + "sort" + "time" + + "github.com/prometheus/prometheus/model/labels" + "github.com/prometheus/prometheus/pp/go/cppbridge" + "github.com/prometheus/prometheus/pp/go/model" + "github.com/prometheus/prometheus/pp/go/storage" + "github.com/prometheus/prometheus/pp/go/storage/logger" + "github.com/prometheus/prometheus/util/annotations" +) + +// +// Deduplicator +// + +// Deduplicator accumulates and deduplicates incoming values. +type Deduplicator interface { + Add(shard uint16, snapshot *cppbridge.LabelSetSnapshot, values []string) + Values() []string +} + +// deduplicatorCtor constructor [Deduplicator]. +type deduplicatorCtor func(numberOfShards uint16) Deduplicator + +// +// GenericTask +// + +// GenericTask the minimum required GenericTask implementation. +type GenericTask interface { + Wait() error +} + +// +// Shard +// + +// Shard the minimum required head Shard implementation. +type Shard interface { + QueryLabelValues( + name string, + matchers []model.LabelMatcher, + dedupAdd func(shardID uint16, snapshot *cppbridge.LabelSetSnapshot, values []string), + ) error +} + +// +// Head +// + +// Head the minimum required Head implementation. +type Head[ + TGenericTask GenericTask, + TShard Shard, +] interface { + CreateTask(taskName string, fn func(shard TShard) error, isLss bool) TGenericTask + Enqueue(t TGenericTask) + NumberOfShards() uint16 + RLockQuery(ctx context.Context) (runlock func(), err error) +} + +// +// Querier +// + +// Querier provides querying access over time series data of a fixed time range. +type Querier[ + TGenericTask GenericTask, + TShard Shard, + THead Head[TGenericTask, TShard], +] struct { + mint int64 + maxt int64 + head THead + deduplicatorCtor deduplicatorCtor + closer func() error + metrics *Metrics +} + +// Close Querier if need. +func (q *Querier[TGenericTask, TShard, THead]) Close() error { + if q.closer != nil { + return q.closer() + } + + return nil +} + +// LabelValues returns label values present in the head for the specific label name +// that are within the time range mint to maxt. If matchers are specified the returned +// result set is reduced to label values of metrics matching the matchers. +func (q *Querier[TGenericTask, TShard, THead]) LabelValues( + ctx context.Context, + name string, + matchers ...*labels.Matcher, +) ([]string, annotations.Annotations, error) { + return queryLabelValues( + ctx, + name, + q.head, + q.deduplicatorCtor, + q.metrics, + storage.LSSLabelValuesQuerier, + matchers..., + ) +} + +// convertPrometheusMatchersToPPMatchers converts prometheus matchers to pp matchers. +func convertPrometheusMatchersToPPMatchers(matchers ...*labels.Matcher) []model.LabelMatcher { + promppMatchers := make([]model.LabelMatcher, 0, len(matchers)) + for _, matcher := range matchers { + promppMatchers = append(promppMatchers, model.LabelMatcher{ + Name: matcher.Name, + Value: matcher.Value, + MatcherType: uint8(matcher.Type), // #nosec G115 // no overflow + }) + } + + return promppMatchers +} + +// queryLabelValues returns label values present in the head for the specific label name. +func queryLabelValues[ + TGenericTask GenericTask, + TShard Shard, + THead Head[TGenericTask, TShard], +]( + ctx context.Context, + name string, + head THead, + deduplicatorCtor deduplicatorCtor, + metrics *Metrics, + taskName string, + matchers ...*labels.Matcher, +) ([]string, annotations.Annotations, error) { + start := time.Now() + + anns := *annotations.New() + runlock, err := head.RLockQuery(ctx) + if err != nil { + logger.Warnf("[QUERIER]: label values failed on the capture of the read lock query: %s", err) + return nil, anns, err + } + defer runlock() + + defer func() { + if metrics != nil { + metrics.LabelValuesDuration.Observe(float64(time.Since(start).Microseconds())) + } + }() + + dedup := deduplicatorCtor(head.NumberOfShards()) + convertedMatchers := convertPrometheusMatchersToPPMatchers(matchers...) + + t := head.CreateTask( + taskName, + func(shard TShard) error { + return shard.QueryLabelValues(name, convertedMatchers, dedup.Add) + }, + storage.ForLSSTask, + ) + head.Enqueue(t) + + if err := t.Wait(); err != nil { + anns.Add(err) + } + + select { + case <-ctx.Done(): + return nil, anns, context.Cause(ctx) + default: + } + + lvs := dedup.Values() + sort.Strings(lvs) + + return lvs, anns, nil +} From 1c6b5652fc6340569791b84578b75086284133ba Mon Sep 17 00:00:00 2001 From: Alexandr Yudin Date: Thu, 7 Aug 2025 11:19:55 +0000 Subject: [PATCH 08/96] for save --- pp/go/storage/head/head/head.go | 50 ++++++++++++++++---------- pp/go/storage/head/head/head_test.go | 2 +- pp/go/storage/head/task/task.go | 31 ++++++---------- pp/go/storage/head/task/waiter.go | 43 ++++++++++++++++++++++ pp/go/storage/head/task/waiter_test.go | 22 ++++++++++++ pp/go/storage/querier/querier.go | 39 +++++++++++++++++--- pp/go/storage/querier/querier_test.go | 27 ++++++++++++++ 7 files changed, 171 insertions(+), 43 deletions(-) create mode 100644 pp/go/storage/head/task/waiter.go create mode 100644 pp/go/storage/head/task/waiter_test.go create mode 100644 pp/go/storage/querier/querier_test.go diff --git a/pp/go/storage/head/head/head.go b/pp/go/storage/head/head/head.go index 3bef8232d1..8988e6c5d6 100644 --- a/pp/go/storage/head/head/head.go +++ b/pp/go/storage/head/head/head.go @@ -4,29 +4,36 @@ import ( "context" "sync" - "github.com/prometheus/prometheus/pp/go/cppbridge" - "github.com/prometheus/prometheus/pp/go/model" - "github.com/prometheus/prometheus/pp/go/storage" + "github.com/prometheus/prometheus/pp/go/storage/head/task" "github.com/prometheus/prometheus/pp/go/util/locker" ) +// +// Shard +// + +// Shard the minimum required head Shard implementation. type Shard interface { - QueryLabelValues( - name string, - matchers []model.LabelMatcher, - dedupAdd func(shardID uint16, snapshot *cppbridge.LabelSetSnapshot, values []string), - ) error + // QueryLabelValues( + // name string, + // matchers []model.LabelMatcher, + // dedupAdd func(shardID uint16, snapshot *cppbridge.LabelSetSnapshot, values []string), + // ) error ShardID() uint16 } -type Head struct { +// +// Head +// + +type Head[TShard Shard] struct { id string generation uint64 readOnly bool - shards []Shard - lssTaskChs []chan *storage.GenericTask - dataStorageTaskChs []chan *storage.GenericTask + shards []TShard + lssTaskChs []chan *task.Generic[TShard] + dataStorageTaskChs []chan *task.Generic[TShard] queryLocker *locker.Weighted numberOfShards uint16 @@ -48,23 +55,30 @@ type Head struct { // tasksExecute *prometheus.CounterVec } -func NewHead(shards []Shard) *Head { - return &Head{ +func NewHead[TShard Shard](shards []TShard) *Head[TShard] { + return &Head[TShard]{ shards: shards, } } // CreateTask create a task for operations on the head shards. -func (h *Head) CreateTask(taskName string, fn func(shard TShard) error, isLss bool) TGenericTask { +func (h *Head[TShard]) CreateTask(taskName string, fn func(shard TShard) error) *task.Generic[TShard] { + // TODO + return nil } // Enqueue the task to be executed on head. -func (h *Head) Enqueue(t TGenericTask) +func (h *Head[TShard]) Enqueue(t *task.Generic[TShard]) { + // TODO +} // NumberOfShards returns current number of shards. -func (h *Head) NumberOfShards() uint16 { +func (h *Head[TShard]) NumberOfShards() uint16 { return h.numberOfShards } // RLockQuery locks for query to [Head]. -func (h *Head) RLockQuery(ctx context.Context) (runlock func(), err error) +func (h *Head[TShard]) RLockQuery(ctx context.Context) (runlock func(), err error) { + // TODO + return func() {}, nil +} diff --git a/pp/go/storage/head/head/head_test.go b/pp/go/storage/head/head/head_test.go index b03d3e0887..941795ed1b 100644 --- a/pp/go/storage/head/head/head_test.go +++ b/pp/go/storage/head/head/head_test.go @@ -13,6 +13,6 @@ func TestXxx(t *testing.T) { wl := struct{}{} sd := shard.NewShard(lss, ds, wl, 0) - h := head.NewHead([]head.Shard{sd}) + h := head.NewHead([]*shard.Shard[struct{}]{sd}) _ = h } diff --git a/pp/go/storage/head/task/task.go b/pp/go/storage/head/task/task.go index 52303cba29..cc7d8645df 100644 --- a/pp/go/storage/head/task/task.go +++ b/pp/go/storage/head/task/task.go @@ -7,8 +7,6 @@ import ( "time" "github.com/prometheus/client_golang/prometheus" - "github.com/prometheus/prometheus/pp/go/cppbridge" - "github.com/prometheus/prometheus/pp/go/model" ) // @@ -17,11 +15,7 @@ import ( // Shard the minimum required head Shard implementation. type Shard interface { - QueryLabelValues( - name string, - matchers []model.LabelMatcher, - dedupAdd func(shardID uint16, snapshot *cppbridge.LabelSetSnapshot, values []string), - ) error + ShardID() uint16 } // @@ -39,16 +33,14 @@ type Generic[TShard Shard] struct { done prometheus.Counter live prometheus.Counter execute prometheus.Counter - forLSS bool } -// NewGenericTask init new [Generic]. -func NewGenericTask( +// NewGeneric init new [Generic]. +func NewGeneric[TShard Shard]( shardFn func(shard TShard) error, created, done, live, execute prometheus.Counter, - forLSS bool, -) *Generic { - t := &Generic{ +) *Generic[TShard] { + t := &Generic[TShard]{ shardFn: shardFn, wg: sync.WaitGroup{}, createdTS: time.Now().UnixMicro(), @@ -56,16 +48,15 @@ func NewGenericTask( done: done, live: live, execute: execute, - forLSS: forLSS, } t.created.Inc() return t } -// NewReadOnlyGenericTask init new GenericTask for read only head. -func NewReadOnlyGenericTask(shardFn func(shard TShard) error) *Generic { - t := &Generic{ +// NewReadOnlyGeneric init new GenericTask for read only head. +func NewReadOnlyGeneric[TShard Shard](shardFn func(shard TShard) error) *Generic[TShard] { + t := &Generic[TShard]{ shardFn: shardFn, wg: sync.WaitGroup{}, } @@ -74,20 +65,20 @@ func NewReadOnlyGenericTask(shardFn func(shard TShard) error) *Generic { } // SetShardsNumber set shards number -func (t *Generic) SetShardsNumber(number uint16) { +func (t *Generic[TShard]) SetShardsNumber(number uint16) { t.errs = make([]error, number) t.wg.Add(int(number)) } // ExecuteOnShard execute task on shard. -func (t *Generic) ExecuteOnShard(shard Shard) { +func (t *Generic[TShard]) ExecuteOnShard(shard TShard) { atomic.CompareAndSwapInt64(&t.executeTS, 0, time.Now().UnixMicro()) t.errs[shard.ShardID()] = t.shardFn(shard) t.wg.Done() } // Wait for the task to complete on all shards. -func (t *Generic) Wait() error { +func (t *Generic[TShard]) Wait() error { t.wg.Wait() if t.done == nil { return errors.Join(t.errs...) diff --git a/pp/go/storage/head/task/waiter.go b/pp/go/storage/head/task/waiter.go new file mode 100644 index 0000000000..245707e0ac --- /dev/null +++ b/pp/go/storage/head/task/waiter.go @@ -0,0 +1,43 @@ +package task + +import "errors" + +// +// Task +// + +// Task the minimum required Task implementation. +type Task interface { + Wait() error +} + +// +// TaskWaiter +// + +// Waiter aggregates the wait for tasks to be completed. +type Waiter[TTask Task] struct { + tasks []TTask +} + +// NewTaskWaiter init new TaskWaiter for n task. +func NewTaskWaiter[TTask Task](n int) *Waiter[TTask] { + return &Waiter[TTask]{ + tasks: make([]TTask, 0, n), + } +} + +// Add task to waiter. +func (tw *Waiter[TTask]) Add(t TTask) { + tw.tasks = append(tw.tasks, t) +} + +// Wait for tasks to be completed. +func (tw *Waiter[TTask]) Wait() error { + errs := make([]error, len(tw.tasks)) + for _, t := range tw.tasks { + errs = append(errs, t.Wait()) + } + + return errors.Join(errs...) +} diff --git a/pp/go/storage/head/task/waiter_test.go b/pp/go/storage/head/task/waiter_test.go new file mode 100644 index 0000000000..77be3acc65 --- /dev/null +++ b/pp/go/storage/head/task/waiter_test.go @@ -0,0 +1,22 @@ +package task_test + +import ( + "testing" + + "github.com/prometheus/prometheus/pp/go/storage/head/task" + "github.com/stretchr/testify/require" +) + +func TestXxx(t *testing.T) { + tw := task.NewTaskWaiter[*task.Generic[*testShard]](5) + err := tw.Wait() + require.NoError(t, err) +} + +// testHead implementation [Shard]. +type testShard struct{} + +// ShardID implementation [Shard]. +func (*testShard) ShardID() uint16 { + return 0 +} diff --git a/pp/go/storage/querier/querier.go b/pp/go/storage/querier/querier.go index 9fe6c87085..15b7fc298b 100644 --- a/pp/go/storage/querier/querier.go +++ b/pp/go/storage/querier/querier.go @@ -8,11 +8,21 @@ import ( "github.com/prometheus/prometheus/model/labels" "github.com/prometheus/prometheus/pp/go/cppbridge" "github.com/prometheus/prometheus/pp/go/model" - "github.com/prometheus/prometheus/pp/go/storage" "github.com/prometheus/prometheus/pp/go/storage/logger" "github.com/prometheus/prometheus/util/annotations" ) +const ( + // LSSQueryInstantQuerySelector name of task. + LSSQueryInstantQuerySelector = "lss_query_instant_query_selector" + // LSSQueryRangeQuerySelector name of task. + LSSQueryRangeQuerySelector = "lss_query_range_query_selector" + // LSSLabelValuesQuerier name of task. + LSSLabelValuesQuerier = "lss_label_values_querier" + // LSSLabelNamesQuerier name of task. + LSSLabelNamesQuerier = "lss_label_names_querier" +) + // // Deduplicator // @@ -57,7 +67,7 @@ type Head[ TGenericTask GenericTask, TShard Shard, ] interface { - CreateTask(taskName string, fn func(shard TShard) error, isLss bool) TGenericTask + CreateTask(taskName string, fn func(shard TShard) error) TGenericTask Enqueue(t TGenericTask) NumberOfShards() uint16 RLockQuery(ctx context.Context) (runlock func(), err error) @@ -81,6 +91,28 @@ type Querier[ metrics *Metrics } +// NewQuerier init new [Querier]. +func NewQuerier[ + TGenericTask GenericTask, + TShard Shard, + THead Head[TGenericTask, TShard], +]( + head THead, + deduplicatorCtor deduplicatorCtor, + mint, maxt int64, + closer func() error, + metrics *Metrics, +) *Querier[TGenericTask, TShard, THead] { + return &Querier[TGenericTask, TShard, THead]{ + mint: mint, + maxt: maxt, + head: head, + deduplicatorCtor: deduplicatorCtor, + closer: closer, + metrics: metrics, + } +} + // Close Querier if need. func (q *Querier[TGenericTask, TShard, THead]) Close() error { if q.closer != nil { @@ -104,7 +136,7 @@ func (q *Querier[TGenericTask, TShard, THead]) LabelValues( q.head, q.deduplicatorCtor, q.metrics, - storage.LSSLabelValuesQuerier, + LSSLabelValuesQuerier, matchers..., ) } @@ -161,7 +193,6 @@ func queryLabelValues[ func(shard TShard) error { return shard.QueryLabelValues(name, convertedMatchers, dedup.Add) }, - storage.ForLSSTask, ) head.Enqueue(t) diff --git a/pp/go/storage/querier/querier_test.go b/pp/go/storage/querier/querier_test.go new file mode 100644 index 0000000000..ee9645b0f8 --- /dev/null +++ b/pp/go/storage/querier/querier_test.go @@ -0,0 +1,27 @@ +package querier_test + +import ( + "testing" + + "github.com/prometheus/prometheus/pp/go/storage/head/head" + "github.com/prometheus/prometheus/pp/go/storage/head/shard" + "github.com/prometheus/prometheus/pp/go/storage/querier" +) + +func TestXxx(t *testing.T) { + lss := &shard.LSS{} + ds := shard.NewDataStorage() + wl := struct{}{} + sd := shard.NewShard(lss, ds, wl, 0) + + h := head.NewHead([]*shard.Shard[struct{}]{sd}) + + querier.NewQuerier( + h, + querier.NewNoOpShardedDeduplicator, + 0, + 0, + nil, + querier.NewMetrics(nil, "test"), + ) +} From 9c4fa48d65c76e1d2d9812fde0e9d157f4460633 Mon Sep 17 00:00:00 2001 From: Alexandr Yudin Date: Thu, 7 Aug 2025 11:22:13 +0000 Subject: [PATCH 09/96] fix after merge --- pp/go/relabeler/querier/multi.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/pp/go/relabeler/querier/multi.go b/pp/go/relabeler/querier/multi.go index bc1971af34..169b6f1180 100644 --- a/pp/go/relabeler/querier/multi.go +++ b/pp/go/relabeler/querier/multi.go @@ -40,7 +40,7 @@ func (q *MultiQuerier) LabelValues( matchers ...*labels.Matcher, ) ([]string, annotations.Annotations, error) { if len(q.queriers) == 1 { - return q.queriers[0].LabelValues(ctx, name, matchers...) + return q.queriers[0].LabelValues(ctx, name, hints, matchers...) } labelValuesResults := make([][]string, len(q.queriers)) @@ -73,7 +73,7 @@ func (q *MultiQuerier) LabelNames( matchers ...*labels.Matcher, ) ([]string, annotations.Annotations, error) { if len(q.queriers) == 1 { - return q.queriers[0].LabelNames(ctx, matchers...) + return q.queriers[0].LabelNames(ctx, hints, matchers...) } labelNamesResults := make([][]string, len(q.queriers)) From e36f2eb2c35672b63e4046c9c7679f54284fb7b4 Mon Sep 17 00:00:00 2001 From: Alexandr Yudin Date: Thu, 7 Aug 2025 16:18:25 +0000 Subject: [PATCH 10/96] rebuild querier, shard --- pp/go/storage/head/shard/lss.go | 5 + pp/go/storage/head/shard/shard.go | 297 +++++++++++++++- pp/go/storage/head/shard/wal/wal.go | 60 ++-- pp/go/storage/querier/dedup.go | 5 +- pp/go/storage/querier/metrics.go | 2 + pp/go/storage/querier/querier.go | 343 ++++++++++++++++++ pp/go/storage/querier/querier_test.go | 23 +- pp/go/storage/querier/series.go | 483 ++++++++++++++++++++++++++ 8 files changed, 1176 insertions(+), 42 deletions(-) create mode 100644 pp/go/storage/querier/series.go diff --git a/pp/go/storage/head/shard/lss.go b/pp/go/storage/head/shard/lss.go index 6906fb9800..f62bd0c65d 100644 --- a/pp/go/storage/head/shard/lss.go +++ b/pp/go/storage/head/shard/lss.go @@ -20,6 +20,11 @@ func (w *LSS) AllocatedMemory() uint64 { return w.input.AllocatedMemory() + w.target.AllocatedMemory() } +// CopyAddedSeries copy label sets which were added via FindOrEmplace to destination. +func (w *LSS) CopyAddedSeries(destination *cppbridge.LabelSetStorage) { + w.target.CopyAddedSeries(destination) +} + // GetSnapshot return the actual snapshot. func (w *LSS) GetSnapshot() *cppbridge.LabelSetSnapshot { w.once.Do(func() { diff --git a/pp/go/storage/head/shard/shard.go b/pp/go/storage/head/shard/shard.go index 913054aa31..49a09016b6 100644 --- a/pp/go/storage/head/shard/shard.go +++ b/pp/go/storage/head/shard/shard.go @@ -3,6 +3,7 @@ package shard import ( + "context" "fmt" "runtime" "sync" @@ -11,26 +12,33 @@ import ( "github.com/prometheus/prometheus/pp/go/model" ) +// Wal the minimum required Wal implementation for a [Shard]. +type Wal interface { + // Commit finalize segment from encoder and write to wal. + Commit() error + // WalFlush flush all contetnt into wal. + Flush() error + // WalWrite append the incoming inner series to wal encoder. + Write(innerSeriesSlice []*cppbridge.InnerSeries) (bool, error) +} + // // Shard // // Shard bridge to labelset storage, data storage and wal. -type Shard[TWal any] struct { +type Shard[TWal Wal] struct { lss *LSS dataStorage *DataStorage wal TWal lssLocker sync.RWMutex dataStorageLocker sync.RWMutex - // write -> append samples walLocker.Lock - // commit -> lssLocker.Rlock walLocker.Lock - // flush -> walLocker.Lock - walLocker sync.Mutex - id uint16 + walLocker sync.Mutex + id uint16 } // NewShard init new [Shard]. -func NewShard[TWal any]( +func NewShard[TWal Wal]( lss *LSS, dataStorage *DataStorage, wal TWal, @@ -47,6 +55,226 @@ func NewShard[TWal any]( } } +// AppendInnerSeriesSlice add InnerSeries to [DataStorage]. +func (s *Shard[TWal]) AppendInnerSeriesSlice(innerSeriesSlice []*cppbridge.InnerSeries) { + s.dataStorageLocker.Lock() + s.dataStorage.AppendInnerSeriesSlice(innerSeriesSlice) + s.dataStorageLocker.Unlock() +} + +// AppendRelabelerSeries add relabeled ls to lss, add to result and add to cache update(second stage). +func (s *Shard[TWal]) AppendRelabelerSeries( + ctx context.Context, + relabeler *cppbridge.InputPerShardRelabeler, + shardsInnerSeries []*cppbridge.InnerSeries, + shardsRelabeledSeries []*cppbridge.RelabeledSeries, + shardsRelabelerStateUpdate []*cppbridge.RelabelerStateUpdate, +) (bool, error) { + s.lssLocker.Lock() + defer s.lssLocker.Unlock() + + return relabeler.AppendRelabelerSeries( + ctx, + s.lss.Target(), + shardsInnerSeries, + shardsRelabeledSeries, + shardsRelabelerStateUpdate, + ) +} + +// CopyAddedSeries copy label sets which were added via FindOrEmplace to destination. +func (s *Shard[TWal]) CopyAddedSeries(destination *Shard[TWal]) { + s.lssLocker.RLock() + s.lss.CopyAddedSeries(destination.lss.Target()) + s.lssLocker.RUnlock() +} + +// DataStorageAllocatedMemory return size of allocated memory for [DataStorage]. +func (s *Shard[TWal]) DataStorageAllocatedMemory() uint64 { + s.dataStorageLocker.RLock() + am := s.dataStorage.AllocatedMemory() + s.dataStorageLocker.RUnlock() + + return am +} + +// DataStorageInstantQuery returns samples for instant query from [DataStorage]. +func (s *Shard[TWal]) DataStorageInstantQuery( + maxt, valueNotFoundTimestampValue int64, + ids []uint32, +) []cppbridge.Sample { + s.dataStorageLocker.RLock() + samples := s.dataStorage.InstantQuery(maxt, valueNotFoundTimestampValue, ids) + s.dataStorageLocker.RUnlock() + + return samples +} + +// DataStorageQuery returns serialized chunks from data storage. +func (s *Shard[TWal]) DataStorageQuery( + query cppbridge.HeadDataStorageQuery, +) *cppbridge.HeadDataStorageSerializedChunks { + s.dataStorageLocker.RLock() + serializedChunks := s.dataStorage.Query(query) + s.dataStorageLocker.RUnlock() + + return serializedChunks +} + +// DataStorageQueryStatus get head status from [DataStorage]. +func (s *Shard[TWal]) DataStorageQueryStatus(status *cppbridge.HeadStatus) { + s.dataStorageLocker.RLock() + status.FromDataStorage(s.dataStorage.Raw()) + s.dataStorageLocker.RUnlock() +} + +// InputRelabeling relabeling incoming hashdex(first stage). +func (s *Shard[TWal]) InputRelabeling( + ctx context.Context, + relabeler *cppbridge.InputPerShardRelabeler, + cache *cppbridge.Cache, + options cppbridge.RelabelerOptions, + shardedData cppbridge.ShardedData, + shardsInnerSeries []*cppbridge.InnerSeries, + shardsRelabeledSeries []*cppbridge.RelabeledSeries, +) (cppbridge.RelabelerStats, bool, error) { + s.lssLocker.Lock() + defer s.lssLocker.Unlock() + + return relabeler.InputRelabeling( + ctx, + s.lss.Input(), + s.lss.Target(), + cache, + options, + shardedData, + shardsInnerSeries, + shardsRelabeledSeries, + ) +} + +// InputRelabelingFromCache relabeling incoming hashdex(first stage) from cache. +func (s *Shard[TWal]) InputRelabelingFromCache( + ctx context.Context, + relabeler *cppbridge.InputPerShardRelabeler, + cache *cppbridge.Cache, + options cppbridge.RelabelerOptions, + shardedData cppbridge.ShardedData, + shardsInnerSeries []*cppbridge.InnerSeries, +) (cppbridge.RelabelerStats, bool, error) { + s.lssLocker.RLock() + defer s.lssLocker.RUnlock() + + return relabeler.InputRelabelingFromCache( + ctx, + s.lss.Input(), + s.lss.Target(), + cache, + options, + shardedData, + shardsInnerSeries, + ) +} + +// InputRelabelingWithStalenans relabeling incoming hashdex(first stage) with state stalenans. +func (s *Shard[TWal]) InputRelabelingWithStalenans( + ctx context.Context, + relabeler *cppbridge.InputPerShardRelabeler, + cache *cppbridge.Cache, + options cppbridge.RelabelerOptions, + staleNansState *cppbridge.StaleNansState, + defTimestamp int64, + shardedData cppbridge.ShardedData, + shardsInnerSeries []*cppbridge.InnerSeries, + shardsRelabeledSeries []*cppbridge.RelabeledSeries, +) (cppbridge.RelabelerStats, bool, error) { + s.lssLocker.Lock() + defer s.lssLocker.Unlock() + + return relabeler.InputRelabelingWithStalenans( + ctx, + s.lss.Input(), + s.lss.Target(), + cache, + options, + staleNansState, + defTimestamp, + shardedData, + shardsInnerSeries, + shardsRelabeledSeries, + ) +} + +// InputRelabelingWithStalenansFromCache relabeling incoming hashdex(first stage) from cache with state stalenans. +func (s *Shard[TWal]) InputRelabelingWithStalenansFromCache( + ctx context.Context, + relabeler *cppbridge.InputPerShardRelabeler, + cache *cppbridge.Cache, + options cppbridge.RelabelerOptions, + staleNansState *cppbridge.StaleNansState, + defTimestamp int64, + shardedData cppbridge.ShardedData, + shardsInnerSeries []*cppbridge.InnerSeries, +) (cppbridge.RelabelerStats, bool, error) { + s.lssLocker.RLock() + defer s.lssLocker.RUnlock() + + return relabeler.InputRelabelingWithStalenansFromCache( + ctx, + s.lss.Input(), + s.lss.Target(), + cache, + options, + staleNansState, + defTimestamp, + shardedData, + shardsInnerSeries, + ) +} + +// LSSAllocatedMemory return size of allocated memory for labelset storages. +func (s *Shard[TWal]) LSSAllocatedMemory() uint64 { + s.lssLocker.RLock() + am := s.lss.AllocatedMemory() + s.lssLocker.RUnlock() + + return am +} + +// LSSQueryStatus get head status from lss. +func (s *Shard[TWal]) LSSQueryStatus(status *cppbridge.HeadStatus, limit int) { + s.lssLocker.RLock() + status.FromLSS(s.lss.Target(), limit) + s.lssLocker.RUnlock() +} + +// MergeOutOfOrderChunks merge chunks with out of order data chunks in [DataStorage]. +func (s *Shard[TWal]) MergeOutOfOrderChunks() { + s.dataStorageLocker.Lock() + s.dataStorage.MergeOutOfOrderChunks() + s.dataStorageLocker.Unlock() +} + +// QueryLabelNames returns all the unique label names present in lss in sorted order. +func (s *Shard[TWal]) QueryLabelNames( + matchers []model.LabelMatcher, + dedupAdd func(shardID uint16, snapshot *cppbridge.LabelSetSnapshot, values []string), +) error { + s.lssLocker.RLock() + queryLabelNamesResult := s.lss.QueryLabelNames(matchers) + snapshot := s.lss.GetSnapshot() + s.lssLocker.RUnlock() + + if queryLabelNamesResult.Status() != cppbridge.LSSQueryStatusMatch { + return fmt.Errorf("no matches on shard: %d", s.id) + } + + dedupAdd(s.id, snapshot, queryLabelNamesResult.Names()) + runtime.KeepAlive(queryLabelNamesResult) + + return nil +} + // QueryLabelValues query labels values to lss and add values to // the dedup-container that matches the given label matchers. func (s *Shard[TWal]) QueryLabelValues( @@ -69,7 +297,62 @@ func (s *Shard[TWal]) QueryLabelValues( return nil } +// QuerySelector returns a created selector that matches the given label matchers. +func (s *Shard[TWal]) QuerySelector(matchers []model.LabelMatcher) (uintptr, *cppbridge.LabelSetSnapshot, error) { + s.lssLocker.RLock() + defer s.lssLocker.RUnlock() + + selector, status := s.lss.QuerySelector(matchers) + switch status { + case cppbridge.LSSQueryStatusMatch: + return selector, s.lss.GetSnapshot(), nil + + case cppbridge.LSSQueryStatusNoMatch: + return 0, nil, nil + + default: + return 0, nil, fmt.Errorf( + "failed to query selector from shard: %d, query status: %d", s.id, status, + ) + } +} + // ShardID returns the shard ID. func (s *Shard[TWal]) ShardID() uint16 { return s.id } + +// WalCommit finalize segment from encoder and write to wal. +func (s *Shard[TWal]) WalCommit() error { + s.lssLocker.RLock() + s.walLocker.Lock() + + err := s.wal.Commit() + + s.walLocker.Unlock() + s.lssLocker.RUnlock() + + return err +} + +// WalFlush flush all contetnt into wal. +func (s *Shard[TWal]) WalFlush() error { + s.walLocker.Lock() + + err := s.wal.Flush() + + s.walLocker.Unlock() + + return err +} + +// WalWrite append the incoming inner series to wal encoder. +func (s *Shard[TWal]) WalWrite(innerSeriesSlice []*cppbridge.InnerSeries) (bool, error) { + s.walLocker.Lock() + + limitExhausted, err := s.wal.Write(innerSeriesSlice) + + s.walLocker.Unlock() + + return limitExhausted, err +} diff --git a/pp/go/storage/head/shard/wal/wal.go b/pp/go/storage/head/shard/wal/wal.go index 3474b6e479..15c1b5e4fd 100644 --- a/pp/go/storage/head/shard/wal/wal.go +++ b/pp/go/storage/head/shard/wal/wal.go @@ -76,33 +76,9 @@ func NewCorruptedWal[ } } -// CurrentSize returns current wal size. -func (w *Wal[TSegment, TStats, TWriter]) CurrentSize() int64 { - return w.segmentWriter.CurrentSize() -} - -// Write the incoming inner series to wal encoder. -func (w *Wal[TSegment, TStats, TWriter]) Write(innerSeriesSlice []*cppbridge.InnerSeries) (bool, error) { - if w.corrupted { - return false, fmt.Errorf("writing in corrupted wal") - } - - stats, err := w.encoder.Encode(innerSeriesSlice) - if err != nil { - return false, fmt.Errorf("failed to encode inner series: %w", err) - } - - if w.maxSegmentSize == 0 { - return false, nil - } - - // memoize reaching of limits to deduplicate triggers - if !w.limitExhausted && stats.Samples() >= w.maxSegmentSize { - w.limitExhausted = true - return true, nil - } - - return false, nil +// Close closes the wal segmentWriter. +func (w *Wal[TSegment, TStats, TWriter]) Close() error { + return w.segmentWriter.Close() } // Commit finalize segment from encoder and write to [SegmentWriter]. @@ -128,12 +104,36 @@ func (w *Wal[TSegment, TStats, TWriter]) Commit() error { return nil } +// CurrentSize returns current wal size. +func (w *Wal[TSegment, TStats, TWriter]) CurrentSize() int64 { + return w.segmentWriter.CurrentSize() +} + // Flush wal [SegmentWriter]. func (w *Wal[TSegment, TStats, TWriter]) Flush() error { return w.segmentWriter.Flush() } -// Close closes the wal segmentWriter. -func (w *Wal[TSegment, TStats, TWriter]) Close() error { - return w.segmentWriter.Close() +// Write the incoming inner series to wal encoder. +func (w *Wal[TSegment, TStats, TWriter]) Write(innerSeriesSlice []*cppbridge.InnerSeries) (bool, error) { + if w.corrupted { + return false, fmt.Errorf("writing in corrupted wal") + } + + stats, err := w.encoder.Encode(innerSeriesSlice) + if err != nil { + return false, fmt.Errorf("failed to encode inner series: %w", err) + } + + if w.maxSegmentSize == 0 { + return false, nil + } + + // memoize reaching of limits to deduplicate triggers + if !w.limitExhausted && stats.Samples() >= w.maxSegmentSize { + w.limitExhausted = true + return true, nil + } + + return false, nil } diff --git a/pp/go/storage/querier/dedup.go b/pp/go/storage/querier/dedup.go index 9ef6666de8..3ba2714864 100644 --- a/pp/go/storage/querier/dedup.go +++ b/pp/go/storage/querier/dedup.go @@ -11,9 +11,8 @@ import ( // NoOpShardedDeduplicator container for accumulating values without deduplication. type NoOpShardedDeduplicator struct { shardedValues [][]string - // TODO snapshots any - snapshots []*cppbridge.LabelSetSnapshot - count uint64 + snapshots []*cppbridge.LabelSetSnapshot + count uint64 } // NewNoOpShardedDeduplicator int new [NoOpShardedDeduplicator]. diff --git a/pp/go/storage/querier/metrics.go b/pp/go/storage/querier/metrics.go index 530ee752ca..c31abaee9c 100644 --- a/pp/go/storage/querier/metrics.go +++ b/pp/go/storage/querier/metrics.go @@ -12,12 +12,14 @@ const ( QueryableStorageSource = "queryable_storage" ) +// Metrics for [Querier]. type Metrics struct { LabelNamesDuration prometheus.Histogram LabelValuesDuration prometheus.Histogram SelectDuration *prometheus.HistogramVec } +// NewMetrics init new [Metrics]. func NewMetrics(registerer prometheus.Registerer, source string) *Metrics { factory := util.NewUnconflictRegisterer(registerer) return &Metrics{ diff --git a/pp/go/storage/querier/querier.go b/pp/go/storage/querier/querier.go index 15b7fc298b..87f8bc9426 100644 --- a/pp/go/storage/querier/querier.go +++ b/pp/go/storage/querier/querier.go @@ -2,13 +2,17 @@ package querier import ( "context" + "errors" + "fmt" "sort" "time" + "github.com/prometheus/client_golang/prometheus" "github.com/prometheus/prometheus/model/labels" "github.com/prometheus/prometheus/pp/go/cppbridge" "github.com/prometheus/prometheus/pp/go/model" "github.com/prometheus/prometheus/pp/go/storage/logger" + "github.com/prometheus/prometheus/storage" "github.com/prometheus/prometheus/util/annotations" ) @@ -21,6 +25,14 @@ const ( LSSLabelValuesQuerier = "lss_label_values_querier" // LSSLabelNamesQuerier name of task. LSSLabelNamesQuerier = "lss_label_names_querier" + + // DSQueryInstantQuerier name of task. + DSQueryInstantQuerier = "data_storage_query_instant_querier" + // DSQueryRangeQuerier name of task. + DSQueryRangeQuerier = "data_storage_query_range_querier" + + // DefaultInstantQueryValueNotFoundTimestampValue default value for not found timestamp value. + DefaultInstantQueryValueNotFoundTimestampValue int64 = 0 ) // @@ -29,7 +41,9 @@ const ( // Deduplicator accumulates and deduplicates incoming values. type Deduplicator interface { + // Add values to deduplicator by shard ID. Add(shard uint16, snapshot *cppbridge.LabelSetSnapshot, values []string) + // Values returns collected values. Values() []string } @@ -42,6 +56,7 @@ type deduplicatorCtor func(numberOfShards uint16) Deduplicator // GenericTask the minimum required GenericTask implementation. type GenericTask interface { + // Wait for the task to complete on all shards. Wait() error } @@ -51,11 +66,31 @@ type GenericTask interface { // Shard the minimum required head Shard implementation. type Shard interface { + // DataStorageInstantQuery returns samples for instant query from data storage. + DataStorageInstantQuery( + maxt, valueNotFoundTimestampValue int64, + ids []uint32, + ) []cppbridge.Sample + // QueryDataStorage returns serialized chunks from data storage. + DataStorageQuery( + query cppbridge.HeadDataStorageQuery, + ) *cppbridge.HeadDataStorageSerializedChunks + // QueryLabelNames returns all the unique label names present in lss in sorted order. + QueryLabelNames( + matchers []model.LabelMatcher, + dedupAdd func(shardID uint16, snapshot *cppbridge.LabelSetSnapshot, values []string), + ) error + // QueryLabelValues query labels values to lss and add values to + // the dedup-container that matches the given label matchers. QueryLabelValues( name string, matchers []model.LabelMatcher, dedupAdd func(shardID uint16, snapshot *cppbridge.LabelSetSnapshot, values []string), ) error + // QuerySelector returns a created selector that matches the given label matchers. + QuerySelector(matchers []model.LabelMatcher) (uintptr, *cppbridge.LabelSetSnapshot, error) + // ShardID returns the shard ID. + ShardID() uint16 } // @@ -122,6 +157,23 @@ func (q *Querier[TGenericTask, TShard, THead]) Close() error { return nil } +// LabelNames returns label values present in the head for the specific label name. +func (q *Querier[TGenericTask, TShard, THead]) LabelNames( + ctx context.Context, + hints *storage.LabelHints, + matchers ...*labels.Matcher, +) ([]string, annotations.Annotations, error) { + return queryLabelNames( + ctx, + q.head, + q.deduplicatorCtor, + q.metrics, + LSSLabelNamesQuerier, + hints, + matchers..., + ) +} + // LabelValues returns label values present in the head for the specific label name // that are within the time range mint to maxt. If matchers are specified the returned // result set is reduced to label values of metrics matching the matchers. @@ -141,6 +193,136 @@ func (q *Querier[TGenericTask, TShard, THead]) LabelValues( ) } +// Select returns a set of series that matches the given label matchers. +func (q *Querier[TGenericTask, TShard, THead]) Select( + ctx context.Context, + sortSeries bool, + hints *storage.SelectHints, + matchers ...*labels.Matcher, +) storage.SeriesSet { + if q.mint == q.maxt { + return q.selectInstant(ctx, sortSeries, hints, matchers...) + } + return q.selectRange(ctx, sortSeries, hints, matchers...) +} + +// selectInstant returns a instant set of series that matches the given label matchers. +// +//revive:disable-next-line:function-length long but readable. +func (q *Querier[TGenericTask, TShard, THead]) selectInstant( + ctx context.Context, + _ bool, + _ *storage.SelectHints, + matchers ...*labels.Matcher, +) storage.SeriesSet { + start := time.Now() + + runlock, err := q.head.RLockQuery(ctx) + if err != nil { + logger.Warnf("[QUERIER]: select instant failed on the capture of the read lock query: %s", err) + return storage.ErrSeriesSet(err) + } + defer runlock() + + defer func() { + if q.metrics == nil { + q.metrics.SelectDuration.With( + prometheus.Labels{"query_type": "instant"}, + ).Observe(float64(time.Since(start).Microseconds())) + } + }() + + lssQueryResults, snapshots, err := queryLss(LSSQueryInstantQuerySelector, q.head, matchers) + if err != nil { + logger.Warnf("[QUERIER]: failed to instant: %s", err) + return storage.ErrSeriesSet(err) + } + + valueNotFoundTimestampValue := DefaultInstantQueryValueNotFoundTimestampValue + if q.mint <= valueNotFoundTimestampValue { + valueNotFoundTimestampValue = q.mint - 1 + } + + numberOfShards := q.head.NumberOfShards() + seriesSets := make([]storage.SeriesSet, numberOfShards) + tDataStorageQuery := q.head.CreateTask( + DSQueryInstantQuerier, + func(shard TShard) error { + shardID := shard.ShardID() + lssQueryResult := lssQueryResults[shardID] + if lssQueryResult == nil { + seriesSets[shardID] = &SeriesSet{} + return nil + } + + seriesSets[shardID] = NewInstantSeriesSet( + lssQueryResult, + snapshots[shardID], + valueNotFoundTimestampValue, + shard.DataStorageInstantQuery(q.maxt, valueNotFoundTimestampValue, lssQueryResult.IDs()), + ) + + return nil + }, + ) + q.head.Enqueue(tDataStorageQuery) + _ = tDataStorageQuery.Wait() + + return storage.NewMergeSeriesSet(seriesSets, storage.ChainedSeriesMerge) +} + +// selectRange returns a range set of series that matches the given label matchers. +func (q *Querier[TGenericTask, TShard, THead]) selectRange( + ctx context.Context, + _ bool, + _ *storage.SelectHints, + matchers ...*labels.Matcher, +) storage.SeriesSet { + start := time.Now() + + runlock, err := q.head.RLockQuery(ctx) + if err != nil { + logger.Warnf("[QUERIER]: select range failed on the capture of the read lock query: %s", err) + return storage.ErrSeriesSet(err) + } + defer runlock() + + defer func() { + if q.metrics != nil { + q.metrics.SelectDuration.With( + prometheus.Labels{"query_type": "range"}, + ).Observe(float64(time.Since(start).Microseconds())) + } + }() + + lssQueryResults, snapshots, err := queryLss(LSSQueryRangeQuerySelector, q.head, matchers) + if err != nil { + logger.Warnf("[QUERIER]: failed to range: %s", err) + return storage.ErrSeriesSet(err) + } + + serializedChunksShards := queryDataStorage(DSQueryRangeQuerier, q.head, lssQueryResults, q.mint, q.maxt) + seriesSets := make([]storage.SeriesSet, q.head.NumberOfShards()) + for shardID, serializedChunksShard := range serializedChunksShards { + if serializedChunksShard == nil { + seriesSets[shardID] = &SeriesSet{} + continue + } + + seriesSets[shardID] = &SeriesSet{ + mint: q.mint, + maxt: q.maxt, + deserializer: cppbridge.NewHeadDataStorageDeserializer(serializedChunksShard), + chunksIndex: serializedChunksShard.MakeIndex(), + serializedChunks: serializedChunksShard, + lssQueryResult: lssQueryResults[shardID], + labelSetSnapshot: snapshots[shardID], + } + } + + return storage.NewMergeSeriesSet(seriesSets, storage.ChainedSeriesMerge) +} + // convertPrometheusMatchersToPPMatchers converts prometheus matchers to pp matchers. func convertPrometheusMatchersToPPMatchers(matchers ...*labels.Matcher) []model.LabelMatcher { promppMatchers := make([]model.LabelMatcher, 0, len(matchers)) @@ -155,6 +337,108 @@ func convertPrometheusMatchersToPPMatchers(matchers ...*labels.Matcher) []model. return promppMatchers } +// queryDataStorage returns serialized chunks from data storage for each shard. +func queryDataStorage[ + TGenericTask GenericTask, + TShard Shard, + THead Head[TGenericTask, TShard], +]( + taskName string, + head THead, + lssQueryResults []*cppbridge.LSSQueryResult, + mint, maxt int64, +) []*cppbridge.HeadDataStorageSerializedChunks { + serializedChunksShards := make([]*cppbridge.HeadDataStorageSerializedChunks, head.NumberOfShards()) + tDataStorageQuery := head.CreateTask( + taskName, + func(shard TShard) error { + shardID := shard.ShardID() + lssQueryResult := lssQueryResults[shardID] + if lssQueryResult == nil { + return nil + } + + serializedChunks := shard.DataStorageQuery(cppbridge.HeadDataStorageQuery{ + StartTimestampMs: mint, + EndTimestampMs: maxt, + LabelSetIDs: lssQueryResult.IDs(), + }) + + if serializedChunks.NumberOfChunks() == 0 { + return nil + } + + serializedChunksShards[shardID] = serializedChunks + + return nil + }, + ) + head.Enqueue(tDataStorageQuery) + _ = tDataStorageQuery.Wait() + + return serializedChunksShards +} + +// queryLabelValues returns label values present in the head for the specific label name. +func queryLabelNames[ + TGenericTask GenericTask, + TShard Shard, + THead Head[TGenericTask, TShard], +]( + ctx context.Context, + head THead, + deduplicatorCtor deduplicatorCtor, + metrics *Metrics, + taskName string, + hints *storage.LabelHints, + matchers ...*labels.Matcher, +) ([]string, annotations.Annotations, error) { + start := time.Now() + + anns := *annotations.New() + runlock, err := head.RLockQuery(ctx) + if err != nil { + logger.Warnf("[QUERIER]: label names failed on the capture of the read lock query: %s", err) + return nil, anns, err + } + defer runlock() + + defer func() { + if metrics != nil { + metrics.LabelNamesDuration.Observe(float64(time.Since(start).Microseconds())) + } + }() + + dedup := deduplicatorCtor(head.NumberOfShards()) + convertedMatchers := convertPrometheusMatchersToPPMatchers(matchers...) + + t := head.CreateTask( + taskName, + func(shard TShard) error { + return shard.QueryLabelNames(convertedMatchers, dedup.Add) + }, + ) + head.Enqueue(t) + + if err := t.Wait(); err != nil { + anns.Add(err) + } + + select { + case <-ctx.Done(): + return nil, anns, context.Cause(ctx) + default: + } + + lns := dedup.Values() + sort.Strings(lns) + + if hints.Limit > 0 && hints.Limit < len(lns) { + return lns[:hints.Limit], anns, nil + } + return lns, anns, nil +} + // queryLabelValues returns label values present in the head for the specific label name. func queryLabelValues[ TGenericTask GenericTask, @@ -211,3 +495,62 @@ func queryLabelValues[ return lvs, anns, nil } + +// lssQuery returns query results and snapshots. +// +//revive:disable-next-line:cyclomatic but readable. +func queryLss[ + TGenericTask GenericTask, + TShard Shard, + THead Head[TGenericTask, TShard], +]( + taskName string, + head THead, + matchers []*labels.Matcher, +) ( + []*cppbridge.LSSQueryResult, + []*cppbridge.LabelSetSnapshot, + error, +) { + numberOfShards := head.NumberOfShards() + selectors := make([]uintptr, numberOfShards) + snapshots := make([]*cppbridge.LabelSetSnapshot, numberOfShards) + convertedMatchers := convertPrometheusMatchersToPPMatchers(matchers...) + + tLSSQuerySelector := head.CreateTask( + taskName, + func(shard TShard) (err error) { + selectors[shard.ShardID()], snapshots[shard.ShardID()], err = shard.QuerySelector(convertedMatchers) + + return err + }, + ) + head.Enqueue(tLSSQuerySelector) + if err := tLSSQuerySelector.Wait(); err != nil { + return nil, nil, err + } + + lssQueryResults := make([]*cppbridge.LSSQueryResult, numberOfShards) + errs := make([]error, numberOfShards) + for shardID, selector := range selectors { + if selector == 0 { + continue + } + + lssQueryResult := snapshots[shardID].Query(selector) + switch lssQueryResult.Status() { + case cppbridge.LSSQueryStatusMatch: + lssQueryResults[shardID] = lssQueryResult + case cppbridge.LSSQueryStatusNoMatch: + default: + errs[shardID] = fmt.Errorf( + "failed to query from shard: %d, query status: %d", shardID, lssQueryResult.Status(), + ) + } + } + if err := errors.Join(errs...); err != nil { + return nil, nil, err + } + + return lssQueryResults, snapshots, nil +} diff --git a/pp/go/storage/querier/querier_test.go b/pp/go/storage/querier/querier_test.go index ee9645b0f8..d84f084182 100644 --- a/pp/go/storage/querier/querier_test.go +++ b/pp/go/storage/querier/querier_test.go @@ -3,6 +3,7 @@ package querier_test import ( "testing" + "github.com/prometheus/prometheus/pp/go/cppbridge" "github.com/prometheus/prometheus/pp/go/storage/head/head" "github.com/prometheus/prometheus/pp/go/storage/head/shard" "github.com/prometheus/prometheus/pp/go/storage/querier" @@ -11,10 +12,10 @@ import ( func TestXxx(t *testing.T) { lss := &shard.LSS{} ds := shard.NewDataStorage() - wl := struct{}{} + wl := &testWal{} sd := shard.NewShard(lss, ds, wl, 0) - h := head.NewHead([]*shard.Shard[struct{}]{sd}) + h := head.NewHead([]*shard.Shard[*testWal]{sd}) querier.NewQuerier( h, @@ -25,3 +26,21 @@ func TestXxx(t *testing.T) { querier.NewMetrics(nil, "test"), ) } + +// testWal test implementation wal. +type testWal struct{} + +// Commit test implementation wal. +func (*testWal) Commit() error { + return nil +} + +// Flush test implementation wal. +func (*testWal) Flush() error { + return nil +} + +// Write test implementation wal. +func (*testWal) Write(_ []*cppbridge.InnerSeries) (bool, error) { + return false, nil +} diff --git a/pp/go/storage/querier/series.go b/pp/go/storage/querier/series.go new file mode 100644 index 0000000000..3956ce84d8 --- /dev/null +++ b/pp/go/storage/querier/series.go @@ -0,0 +1,483 @@ +package querier + +import ( + "github.com/prometheus/prometheus/model/histogram" + "github.com/prometheus/prometheus/model/labels" + "github.com/prometheus/prometheus/pp/go/cppbridge" + "github.com/prometheus/prometheus/storage" + "github.com/prometheus/prometheus/tsdb/chunkenc" + "github.com/prometheus/prometheus/util/annotations" +) + +// SampleProvider create chunk iterator over samples. +type SampleProvider interface { + Samples(minT, maxtT int64) chunkenc.Iterator +} + +// +// SeriesSet +// + +// SeriesSet contains a set of series, allows to iterate over sorted, populated series. +type SeriesSet struct { + mint int64 + maxt int64 + deserializer *cppbridge.HeadDataStorageDeserializer + chunksIndex cppbridge.HeadDataStorageSerializedChunkIndex + serializedChunks *cppbridge.HeadDataStorageSerializedChunks + lssQueryResult *cppbridge.LSSQueryResult + labelSetSnapshot *cppbridge.LabelSetSnapshot + + index int + currentSeries *Series +} + +// At returns full series. Returned series should be iterable even after Next is called. +func (ss *SeriesSet) At() storage.Series { + return ss.currentSeries +} + +// Err the error that iteration as failed with. +func (*SeriesSet) Err() error { + return nil +} + +// Next return true if exist there is a next series and false otherwise. +func (ss *SeriesSet) Next() bool { + if ss.lssQueryResult == nil { + return false + } + + var ( + lsID uint32 + lsLength uint16 + chunksMetadata []cppbridge.HeadDataStorageSerializedChunkMetadata + ) + + for { + if ss.index >= ss.lssQueryResult.Len() { + return false + } + + lsID, lsLength = ss.lssQueryResult.GetByIndex(ss.index) + + chunksMetadata = ss.chunksIndex.Chunks(ss.serializedChunks, lsID) + ss.index++ + if len(chunksMetadata) != 0 { + break + } + } + + ss.currentSeries = &Series{ + mint: ss.mint, + maxt: ss.maxt, + labelSet: labels.NewLabelsWithLSS(ss.labelSetSnapshot, lsID, lsLength), + sampleProvider: &DefaultSampleProvider{ + deserializer: ss.deserializer, + chunksMetadata: chunksMetadata, + }, + } + + return true +} + +// Warnings a collection of warnings for the whole set. +func (*SeriesSet) Warnings() annotations.Annotations { + return nil +} + +// +// Series +// + +// Series is a stream of data points belonging to a metric. +type Series struct { + mint, maxt int64 + labelSet labels.Labels + sampleProvider SampleProvider +} + +// Iterator returns an iterator of the data of the series. +func (s *Series) Iterator(_ chunkenc.Iterator) chunkenc.Iterator { + return s.sampleProvider.Samples(s.mint, s.maxt) +} + +// Labels returns the complete set of labels. +func (s *Series) Labels() labels.Labels { + return s.labelSet +} + +// +// DefaultSampleProvider +// + +// DefaultSampleProvider create default chunk iterator over samples. +type DefaultSampleProvider struct { + deserializer *cppbridge.HeadDataStorageDeserializer + chunksMetadata []cppbridge.HeadDataStorageSerializedChunkMetadata +} + +// Samples reurns chunk iterator over samples. +func (sp *DefaultSampleProvider) Samples(mint, maxt int64) chunkenc.Iterator { + return NewLimitedChunkIterator( + NewChunkIterator(sp.deserializer, sp.chunksMetadata), + mint, + maxt, + ) +} + +// +// ChunkIterator +// + +// ChunkIterator iterates over the samples of a time series, that can only get the next value. +type ChunkIterator struct { + deserializer *cppbridge.HeadDataStorageDeserializer + chunksMetadata []cppbridge.HeadDataStorageSerializedChunkMetadata + decodeIterator *cppbridge.HeadDataStorageDecodeIterator + ts int64 + v float64 +} + +// NewChunkIterator init new [ChunkIterator]. +func NewChunkIterator( + deserializer *cppbridge.HeadDataStorageDeserializer, + chunksMetadata []cppbridge.HeadDataStorageSerializedChunkMetadata, +) *ChunkIterator { + return &ChunkIterator{ + deserializer: deserializer, + chunksMetadata: chunksMetadata, + } +} + +// At returns the current timestamp/value pair if the value is a float. +// +//nolint:gocritic // unnamedResult not need +func (i *ChunkIterator) At() (int64, float64) { + return i.ts, i.v +} + +// AtFloatHistogram returns the current timestamp/value pair if the value is a histogram with floating-point counts. +func (*ChunkIterator) AtFloatHistogram(_ *histogram.FloatHistogram) (int64, *histogram.FloatHistogram) { + return 0, nil +} + +// AtHistogram returns the current timestamp/value pair if the value is a histogram with integer counts. +func (*ChunkIterator) AtHistogram(*histogram.Histogram) (int64, *histogram.Histogram) { + return 0, nil +} + +// AtT returns the current timestamp. +func (i *ChunkIterator) AtT() int64 { + return i.ts +} + +// Err returns the current error. +func (*ChunkIterator) Err() error { + return nil +} + +// Next advances the iterator by one and returns the type of the value. +func (i *ChunkIterator) Next() chunkenc.ValueType { + if i.decodeIterator == nil { + if len(i.chunksMetadata) == 0 { + return chunkenc.ValNone + } + + i.decodeIterator = i.deserializer.CreateDecodeIterator(i.chunksMetadata[0]) + i.chunksMetadata = i.chunksMetadata[1:] + } + + if !i.decodeIterator.Next() { + i.decodeIterator = nil + return i.Next() + } + + i.ts, i.v = i.decodeIterator.Sample() + return chunkenc.ValFloat +} + +// Seek advances the iterator forward to the first sample with a timestamp equal or greater than t. +func (i *ChunkIterator) Seek(t int64) chunkenc.ValueType { + if i.decodeIterator == nil { + if i.Next() == chunkenc.ValNone { + return chunkenc.ValNone + } + } + + for i.ts < t { + if i.Next() == chunkenc.ValNone { + return chunkenc.ValNone + } + } + + return chunkenc.ValFloat +} + +// +// LimitedChunkIterator +// + +// LimitedChunkIterator iterates over the samples of a time series, that can only get the next value with limit. +type LimitedChunkIterator struct { + chunkIterator chunkenc.Iterator + mint int64 + maxt int64 +} + +// NewLimitedChunkIterator init new [LimitedChunkIterator]. +func NewLimitedChunkIterator(iterator chunkenc.Iterator, mint, maxt int64) *LimitedChunkIterator { + return &LimitedChunkIterator{ + chunkIterator: iterator, + mint: mint, + maxt: maxt, + } +} + +// At returns the current timestamp/value pair if the value is a float. +// +//nolint:gocritic // unnamedResult not need +func (i *LimitedChunkIterator) At() (int64, float64) { + return i.chunkIterator.At() +} + +// AtFloatHistogram returns the current timestamp/value pair if the value is a histogram with floating-point counts. +func (i *LimitedChunkIterator) AtFloatHistogram(h *histogram.FloatHistogram) (int64, *histogram.FloatHistogram) { + return i.chunkIterator.AtFloatHistogram(h) +} + +// AtHistogram returns the current timestamp/value pair if the value is a histogram with integer counts. +func (i *LimitedChunkIterator) AtHistogram(h *histogram.Histogram) (int64, *histogram.Histogram) { + return i.chunkIterator.AtHistogram(h) +} + +// AtT returns the current timestamp. +func (i *LimitedChunkIterator) AtT() int64 { + return i.chunkIterator.AtT() +} + +// Err returns the current error. +func (i *LimitedChunkIterator) Err() error { + return i.chunkIterator.Err() +} + +// Next advances the iterator by one and returns the type of the value. +func (i *LimitedChunkIterator) Next() chunkenc.ValueType { + if i.chunkIterator.Next() == chunkenc.ValNone { + return chunkenc.ValNone + } + + if i.Seek(i.mint) == chunkenc.ValNone { + return chunkenc.ValNone + } + + if i.chunkIterator.AtT() > i.maxt { + return chunkenc.ValNone + } + + return chunkenc.ValFloat +} + +// Seek advances the iterator forward to the first sample with a timestamp equal or greater than t. +func (i *LimitedChunkIterator) Seek(t int64) chunkenc.ValueType { + if t < i.mint { + t = i.mint + } + + if t > i.maxt { + t = i.maxt + } + + if i.chunkIterator.Seek(t) == chunkenc.ValNone { + return chunkenc.ValNone + } + + if i.chunkIterator.AtT() > i.maxt { + return chunkenc.ValNone + } + + return chunkenc.ValFloat +} + +// +// InstantSeriesSet +// + +// InstantSeriesSet contains a instatnt set of series, allows to iterate over sorted, populated series. +type InstantSeriesSet struct { + lssQueryResult *cppbridge.LSSQueryResult + labelSetSnapshot *cppbridge.LabelSetSnapshot + valueNotFoundTimestampValue int64 + samples []cppbridge.Sample + + index int + currentSeries *InstantSeries +} + +// NewInstantSeriesSet init new [InstantSeriesSet]. +func NewInstantSeriesSet( + lssQueryResult *cppbridge.LSSQueryResult, + labelSetSnapshot *cppbridge.LabelSetSnapshot, + valueNotFoundTimestampValue int64, + samples []cppbridge.Sample, +) *InstantSeriesSet { + return &InstantSeriesSet{ + lssQueryResult: lssQueryResult, + labelSetSnapshot: labelSetSnapshot, + valueNotFoundTimestampValue: valueNotFoundTimestampValue, + samples: samples, + index: -1, + } +} + +// At returns full series. Returned series should be iterable even after Next is called. +func (ss *InstantSeriesSet) At() storage.Series { + return ss.currentSeries +} + +// Err the error that iteration as failed with. +func (*InstantSeriesSet) Err() error { + return nil +} + +// Next return true if exist there is a next series and false otherwise. +func (ss *InstantSeriesSet) Next() bool { + for { + if ss.index+1 >= ss.lssQueryResult.Len() { + return false + } + + ss.index++ + if ss.samples[ss.index].Timestamp != ss.valueNotFoundTimestampValue { + break + } + } + + lsID, lsLength := ss.lssQueryResult.GetByIndex(ss.index) + ss.currentSeries = &InstantSeries{ + labelSet: labels.NewLabelsWithLSS( + ss.labelSetSnapshot, + lsID, + lsLength, + ), + sample: ss.samples[ss.index], + } + + return true +} + +// Warnings a collection of warnings for the whole set. +func (*InstantSeriesSet) Warnings() annotations.Annotations { + return nil +} + +// +// InstantSeries +// + +// InstantSeries is a instant stream of data points belonging to a metric. +type InstantSeries struct { + labelSet labels.Labels + sample cppbridge.Sample +} + +// Iterator is storage.Series interface implementation. +func (s *InstantSeries) Iterator(iterator chunkenc.Iterator) chunkenc.Iterator { + if i, ok := iterator.(*InstantSeriesChunkIterator); ok { + i.ResetTo(s.sample.Timestamp, s.sample.Value) + return i + } + return NewInstantSeriesChunkIterator(s.sample.Timestamp, s.sample.Value) +} + +// Labels is storage.Series interface implementation. +func (s *InstantSeries) Labels() labels.Labels { + return s.labelSet +} + +// +// InstantSeriesChunkIterator +// + +// InstantSeriesChunkIterator iterates over the samples of a instant time series, that can only get the next value. +type InstantSeriesChunkIterator struct { + i int + t int64 + v float64 +} + +// NewInstantSeriesChunkIterator init new [InstantSeriesChunkIterator]. +func NewInstantSeriesChunkIterator(t int64, v float64) *InstantSeriesChunkIterator { + return &InstantSeriesChunkIterator{ + i: -1, + t: t, + v: v, + } +} + +// At returns the current timestamp/value pair if the value is a float. +// +//nolint:gocritic // unnamedResult not need +func (i *InstantSeriesChunkIterator) At() (int64, float64) { + return i.t, i.v +} + +// AtFloatHistogram returns the current timestamp/value pair if the value is a histogram with floating-point counts. +func (*InstantSeriesChunkIterator) AtFloatHistogram(_ *histogram.FloatHistogram) (int64, *histogram.FloatHistogram) { + return 0, nil +} + +// AtHistogram returns the current timestamp/value pair if the value is a histogram with integer counts. +func (*InstantSeriesChunkIterator) AtHistogram(_ *histogram.Histogram) (int64, *histogram.Histogram) { + return 0, nil +} + +// AtT returns the current timestamp. +func (i *InstantSeriesChunkIterator) AtT() int64 { + return i.t +} + +// Err returns the current error. +func (*InstantSeriesChunkIterator) Err() error { + return nil +} + +// Next advances the iterator by one and returns the type of the value. +func (i *InstantSeriesChunkIterator) Next() chunkenc.ValueType { + if i.i < 1 { + i.i++ + } + return i.valueType() +} + +// ResetTo reset state to timestamp and value. +func (i *InstantSeriesChunkIterator) ResetTo(t int64, v float64) { + i.i = -1 + i.t = t + i.v = v +} + +// Seek advances the iterator forward to the first sample with a timestamp equal or greater than t. +func (i *InstantSeriesChunkIterator) Seek(t int64) chunkenc.ValueType { + if i.valueType() == chunkenc.ValFloat && i.t >= t { + return chunkenc.ValFloat + } + + for { + if i.Next() == chunkenc.ValNone { + return chunkenc.ValNone + } + + if i.t >= t { + return chunkenc.ValFloat + } + } +} + +func (i *InstantSeriesChunkIterator) valueType() chunkenc.ValueType { + if i.i == 0 { + return chunkenc.ValFloat + } + + return chunkenc.ValNone +} From 641e852fe4a8d9f12b028fd068205cf75689a6f1 Mon Sep 17 00:00:00 2001 From: Alexandr Yudin Date: Fri, 8 Aug 2025 14:17:01 +0000 Subject: [PATCH 11/96] for save --- pp/go/storage/head/head/head.go | 49 ++- pp/go/storage/head/head/head_test.go | 23 +- pp/go/storage/head/shard/data_storage.go | 59 +++- pp/go/storage/head/shard/lss.go | 138 ++++++-- pp/go/storage/head/shard/shard.go | 426 +++++++---------------- pp/go/storage/head/shard/wal/wal.go | 13 + pp/go/storage/querier/querier.go | 157 ++++++--- 7 files changed, 473 insertions(+), 392 deletions(-) diff --git a/pp/go/storage/head/head/head.go b/pp/go/storage/head/head/head.go index 8988e6c5d6..e74618a37e 100644 --- a/pp/go/storage/head/head/head.go +++ b/pp/go/storage/head/head/head.go @@ -2,23 +2,23 @@ package head import ( "context" + "fmt" "sync" "github.com/prometheus/prometheus/pp/go/storage/head/task" "github.com/prometheus/prometheus/pp/go/util/locker" ) +// ExtraWorkers number of extra workers for operation on shards. +var ExtraConcurrency = 0 + // // Shard // // Shard the minimum required head Shard implementation. type Shard interface { - // QueryLabelValues( - // name string, - // matchers []model.LabelMatcher, - // dedupAdd func(shardID uint16, snapshot *cppbridge.LabelSetSnapshot, values []string), - // ) error + // ShardID returns the shard ID. ShardID() uint16 } @@ -34,7 +34,7 @@ type Head[TShard Shard] struct { shards []TShard lssTaskChs []chan *task.Generic[TShard] dataStorageTaskChs []chan *task.Generic[TShard] - queryLocker *locker.Weighted + querysemaphore *locker.Weighted numberOfShards uint16 stopc chan struct{} @@ -55,30 +55,53 @@ type Head[TShard Shard] struct { // tasksExecute *prometheus.CounterVec } +// NewHead init new [Head]. func NewHead[TShard Shard](shards []TShard) *Head[TShard] { return &Head[TShard]{ shards: shards, } } -// CreateTask create a task for operations on the head shards. +// AcquireQuery acquires the [Head] semaphore with a weight of 1, +// blocking until resources are available or ctx is done. +// On success, returns nil. On failure, returns ctx.Err() and leaves the semaphore unchanged. +func (h *Head[TShard]) AcquireQuery(ctx context.Context) (release func(), err error) { + return h.querysemaphore.RLock(ctx) +} + +// Concurrency return current head workers concurrency. +func (h *Head[TShard]) Concurrency() int64 { + return calculateHeadConcurrency(h.numberOfShards) +} + +// CreateTask create a task for operations on the [Head] shards. func (h *Head[TShard]) CreateTask(taskName string, fn func(shard TShard) error) *task.Generic[TShard] { // TODO return nil } -// Enqueue the task to be executed on head. +// Enqueue the task to be executed on shards [Head]. func (h *Head[TShard]) Enqueue(t *task.Generic[TShard]) { // TODO } -// NumberOfShards returns current number of shards. +// ID returns id [Head]. +func (h *Head[TShard]) ID() string { + return h.id +} + +// NumberOfShards returns current number of shards in to [Head]. func (h *Head[TShard]) NumberOfShards() uint16 { return h.numberOfShards } -// RLockQuery locks for query to [Head]. -func (h *Head[TShard]) RLockQuery(ctx context.Context) (runlock func(), err error) { - // TODO - return func() {}, nil +// String serialize as string. +func (h *Head[TShard]) String() string { + return fmt.Sprintf("{id: %s, generation: %d}", h.id, h.generation) +} + +// calculateHeadConcurrency calculate current head workers concurrency. +func calculateHeadConcurrency(numberOfShards uint16) int64 { + //revive:disable-next-line:add-constant 2 - default run workers + return 2 * int64(1+ExtraConcurrency) * int64(numberOfShards) } diff --git a/pp/go/storage/head/head/head_test.go b/pp/go/storage/head/head/head_test.go index 941795ed1b..6c002563d7 100644 --- a/pp/go/storage/head/head/head_test.go +++ b/pp/go/storage/head/head/head_test.go @@ -3,6 +3,7 @@ package head_test import ( "testing" + "github.com/prometheus/prometheus/pp/go/cppbridge" "github.com/prometheus/prometheus/pp/go/storage/head/head" "github.com/prometheus/prometheus/pp/go/storage/head/shard" ) @@ -10,9 +11,27 @@ import ( func TestXxx(t *testing.T) { lss := &shard.LSS{} ds := shard.NewDataStorage() - wl := struct{}{} + wl := &testWal{} sd := shard.NewShard(lss, ds, wl, 0) - h := head.NewHead([]*shard.Shard[struct{}]{sd}) + h := head.NewHead([]*shard.Shard[*testWal]{sd}) _ = h } + +// testWal test implementation wal. +type testWal struct{} + +// Commit test implementation wal. +func (*testWal) Commit() error { + return nil +} + +// Flush test implementation wal. +func (*testWal) Flush() error { + return nil +} + +// Write test implementation wal. +func (*testWal) Write(_ []*cppbridge.InnerSeries) (bool, error) { + return false, nil +} diff --git a/pp/go/storage/head/shard/data_storage.go b/pp/go/storage/head/shard/data_storage.go index dcf10700ea..a3dccae1a3 100644 --- a/pp/go/storage/head/shard/data_storage.go +++ b/pp/go/storage/head/shard/data_storage.go @@ -1,11 +1,16 @@ package shard -import "github.com/prometheus/prometheus/pp/go/cppbridge" +import ( + "sync" + + "github.com/prometheus/prometheus/pp/go/cppbridge" +) // DataStorage samles storage with labels IDs. type DataStorage struct { dataStorage *cppbridge.HeadDataStorage encoder *cppbridge.HeadEncoder + locker sync.RWMutex } // NewDataStorage int new [DataStorage]. @@ -14,37 +19,79 @@ func NewDataStorage() *DataStorage { return &DataStorage{ dataStorage: dataStorage, encoder: cppbridge.NewHeadEncoderWithDataStorage(dataStorage), + locker: sync.RWMutex{}, } } -// AllocatedMemory return size of allocated memory for DataStorage. +// AllocatedMemory return size of allocated memory for [DataStorage]. func (ds *DataStorage) AllocatedMemory() uint64 { - return ds.dataStorage.AllocatedMemory() + ds.locker.RLock() + am := ds.dataStorage.AllocatedMemory() + ds.locker.RUnlock() + + return am } // AppendInnerSeriesSlice add InnerSeries to storage. func (ds *DataStorage) AppendInnerSeriesSlice(innerSeriesSlice []*cppbridge.InnerSeries) { + ds.locker.Lock() ds.encoder.EncodeInnerSeriesSlice(innerSeriesSlice) + ds.locker.Unlock() } -// InstantQuery make instant query to data storage and returns serialazed chunks. +// InstantQuery make instant query to data storage and returns samples. func (ds *DataStorage) InstantQuery( targetTimestamp, notFoundValueTimestampValue int64, seriesIDs []uint32, ) []cppbridge.Sample { - return ds.dataStorage.InstantQuery(targetTimestamp, notFoundValueTimestampValue, seriesIDs) + ds.locker.RLock() + samples := ds.dataStorage.InstantQuery(targetTimestamp, notFoundValueTimestampValue, seriesIDs) + ds.locker.RUnlock() + + return samples } // MergeOutOfOrderChunks merge chunks with out of order data chunks. func (ds *DataStorage) MergeOutOfOrderChunks() { + ds.locker.Lock() ds.encoder.MergeOutOfOrderChunks() + ds.locker.Unlock() } // Query make query to data storage and returns serialazed chunks. func (ds *DataStorage) Query(query cppbridge.HeadDataStorageQuery) *cppbridge.HeadDataStorageSerializedChunks { - return ds.dataStorage.Query(query) + ds.locker.RLock() + serializedChunks := ds.dataStorage.Query(query) + ds.locker.RUnlock() + + return serializedChunks +} + +// QueryStatus get head status from [DataStorage]. +func (ds *DataStorage) QueryStatus(status *cppbridge.HeadStatus) { + ds.locker.RLock() + status.FromDataStorage(ds.dataStorage) + ds.locker.RUnlock() } // Raw returns raw [cppbridge.HeadDataStorage]. func (ds *DataStorage) Raw() *cppbridge.HeadDataStorage { return ds.dataStorage } + +// WithLock calls fn on raw [cppbridge.HeadDataStorage] with write lock. +func (ds *DataStorage) WithLock(fn func(ds *cppbridge.HeadDataStorage) error) error { + ds.locker.Lock() + err := fn(ds.dataStorage) + ds.locker.Unlock() + + return err +} + +// WithRLock calls fn on raw [cppbridge.HeadDataStorage] with read lock. +func (ds *DataStorage) WithRLock(fn func(ds *cppbridge.HeadDataStorage) error) error { + ds.locker.RLock() + err := fn(ds.dataStorage) + ds.locker.RUnlock() + + return err +} diff --git a/pp/go/storage/head/shard/lss.go b/pp/go/storage/head/shard/lss.go index f62bd0c65d..34f245fd81 100644 --- a/pp/go/storage/head/shard/lss.go +++ b/pp/go/storage/head/shard/lss.go @@ -1,6 +1,8 @@ package shard import ( + "fmt" + "runtime" "sync" "github.com/prometheus/prometheus/pp/go/cppbridge" @@ -12,58 +14,142 @@ type LSS struct { input *cppbridge.LabelSetStorage target *cppbridge.LabelSetStorage snapshot *cppbridge.LabelSetSnapshot + locker sync.RWMutex once sync.Once } // AllocatedMemory return size of allocated memory for labelset storages. -func (w *LSS) AllocatedMemory() uint64 { - return w.input.AllocatedMemory() + w.target.AllocatedMemory() +func (l *LSS) AllocatedMemory() uint64 { + l.locker.RLock() + am := l.input.AllocatedMemory() + l.target.AllocatedMemory() + l.locker.RUnlock() + + return am } // CopyAddedSeries copy label sets which were added via FindOrEmplace to destination. -func (w *LSS) CopyAddedSeries(destination *cppbridge.LabelSetStorage) { - w.target.CopyAddedSeries(destination) +func (l *LSS) CopyAddedSeries(destination *cppbridge.LabelSetStorage) { + l.locker.RLock() + l.target.CopyAddedSeries(destination) + l.locker.RUnlock() } // GetSnapshot return the actual snapshot. -func (w *LSS) GetSnapshot() *cppbridge.LabelSetSnapshot { - w.once.Do(func() { - w.snapshot = w.target.CreateLabelSetSnapshot() +func (l *LSS) GetSnapshot() *cppbridge.LabelSetSnapshot { + // TODO + l.once.Do(func() { + l.snapshot = l.target.CreateLabelSetSnapshot() }) - return w.snapshot + return l.snapshot } // Input returns input lss. -func (w *LSS) Input() *cppbridge.LabelSetStorage { - return w.input +func (l *LSS) Input() *cppbridge.LabelSetStorage { + return l.input } -// QueryLabelNames returns a LSSQueryLabelNamesResult that matches the given label matchers. -func (w *LSS) QueryLabelNames(matchers []model.LabelMatcher) *cppbridge.LSSQueryLabelNamesResult { - return w.target.QueryLabelNames(matchers) +// QueryLabelNames add to dedup all the unique label names present in lss in sorted order. +func (l *LSS) QueryLabelNames( + shardID uint16, + matchers []model.LabelMatcher, + dedupAdd func(shardID uint16, snapshot *cppbridge.LabelSetSnapshot, values []string), +) error { + l.locker.RLock() + queryLabelNamesResult := l.target.QueryLabelNames(matchers) + snapshot := l.GetSnapshot() + l.locker.RUnlock() + + if queryLabelNamesResult.Status() != cppbridge.LSSQueryStatusMatch { + return fmt.Errorf("no matches on shard: %d", shardID) + } + + dedupAdd(shardID, snapshot, queryLabelNamesResult.Names()) + runtime.KeepAlive(queryLabelNamesResult) + + return nil } -// QueryLabelValues returns a LSSQueryLabelValuesResult that matches the given label matchers. -func (w *LSS) QueryLabelValues( - label_name string, +// QueryLabelValues query labels values to [LSS] and add values to +// the dedup-container that matches the given label matchers. +func (l *LSS) QueryLabelValues( + shardID uint16, + name string, matchers []model.LabelMatcher, -) *cppbridge.LSSQueryLabelValuesResult { - return w.target.QueryLabelValues(label_name, matchers) + dedupAdd func(shardID uint16, snapshot *cppbridge.LabelSetSnapshot, values []string), +) error { + l.locker.RLock() + queryLabelValuesResult := l.target.QueryLabelValues(name, matchers) + snapshot := l.GetSnapshot() + l.locker.RUnlock() + + if queryLabelValuesResult.Status() != cppbridge.LSSQueryStatusMatch { + return fmt.Errorf("no matches on shard: %d", shardID) + } + + dedupAdd(shardID, snapshot, queryLabelValuesResult.Values()) + runtime.KeepAlive(queryLabelValuesResult) + + return nil } // QuerySelector returns a created selector that matches the given label matchers. -func (w *LSS) QuerySelector(matchers []model.LabelMatcher) (selector uintptr, status uint32) { - return w.target.QuerySelector(matchers) +func (l *LSS) QuerySelector(shardID uint16, matchers []model.LabelMatcher) ( + uintptr, + *cppbridge.LabelSetSnapshot, + error, +) { + l.locker.RLock() + defer l.locker.RUnlock() + + selector, status := l.target.QuerySelector(matchers) + switch status { + case cppbridge.LSSQueryStatusMatch: + return selector, l.GetSnapshot(), nil + + case cppbridge.LSSQueryStatusNoMatch: + return 0, nil, nil + + default: + return 0, nil, fmt.Errorf( + "failed to query selector from shard: %d, query status: %d", shardID, status, + ) + } +} + +// QueryStatus get head status from [LSS]. +func (l *LSS) QueryStatus(status *cppbridge.HeadStatus, limit int) { + l.locker.RLock() + status.FromLSS(l.target, limit) + l.locker.RUnlock() } // ResetSnapshot resets the current snapshot. -func (w *LSS) ResetSnapshot() { - w.snapshot = nil - w.once = sync.Once{} +func (l *LSS) ResetSnapshot() { + // TODO + l.snapshot = nil + l.once = sync.Once{} +} + +// Target returns main [LSS]. +func (l *LSS) Target() *cppbridge.LabelSetStorage { + return l.target } -// Target returns main lss. -func (w *LSS) Target() *cppbridge.LabelSetStorage { - return w.target +// WithLock calls fn on raws [cppbridge.LabelSetStorage] with write lock. +func (l *LSS) WithLock(fn func(target, input *cppbridge.LabelSetStorage) error) error { + l.locker.Lock() + err := fn(l.target, l.input) + l.locker.Unlock() + + return err +} + +// WithRLock calls fn on raws [cppbridge.LabelSetStorage] with read lock. +func (l *LSS) WithRLock(fn func(target, input *cppbridge.LabelSetStorage) error) error { + l.locker.RLock() + err := fn(l.target, l.input) + l.locker.RUnlock() + + return err } diff --git a/pp/go/storage/head/shard/shard.go b/pp/go/storage/head/shard/shard.go index 49a09016b6..99a4337c89 100644 --- a/pp/go/storage/head/shard/shard.go +++ b/pp/go/storage/head/shard/shard.go @@ -3,13 +3,7 @@ package shard import ( - "context" - "fmt" - "runtime" - "sync" - "github.com/prometheus/prometheus/pp/go/cppbridge" - "github.com/prometheus/prometheus/pp/go/model" ) // Wal the minimum required Wal implementation for a [Shard]. @@ -28,13 +22,10 @@ type Wal interface { // Shard bridge to labelset storage, data storage and wal. type Shard[TWal Wal] struct { - lss *LSS - dataStorage *DataStorage - wal TWal - lssLocker sync.RWMutex - dataStorageLocker sync.RWMutex - walLocker sync.Mutex - id uint16 + lss *LSS + dataStorage *DataStorage + wal TWal + id uint16 } // NewShard init new [Shard]. @@ -45,276 +36,21 @@ func NewShard[TWal Wal]( shardID uint16, ) *Shard[TWal] { return &Shard[TWal]{ - id: shardID, - lss: lss, - dataStorage: dataStorage, - wal: wal, - lssLocker: sync.RWMutex{}, - dataStorageLocker: sync.RWMutex{}, - walLocker: sync.Mutex{}, - } -} - -// AppendInnerSeriesSlice add InnerSeries to [DataStorage]. -func (s *Shard[TWal]) AppendInnerSeriesSlice(innerSeriesSlice []*cppbridge.InnerSeries) { - s.dataStorageLocker.Lock() - s.dataStorage.AppendInnerSeriesSlice(innerSeriesSlice) - s.dataStorageLocker.Unlock() -} - -// AppendRelabelerSeries add relabeled ls to lss, add to result and add to cache update(second stage). -func (s *Shard[TWal]) AppendRelabelerSeries( - ctx context.Context, - relabeler *cppbridge.InputPerShardRelabeler, - shardsInnerSeries []*cppbridge.InnerSeries, - shardsRelabeledSeries []*cppbridge.RelabeledSeries, - shardsRelabelerStateUpdate []*cppbridge.RelabelerStateUpdate, -) (bool, error) { - s.lssLocker.Lock() - defer s.lssLocker.Unlock() - - return relabeler.AppendRelabelerSeries( - ctx, - s.lss.Target(), - shardsInnerSeries, - shardsRelabeledSeries, - shardsRelabelerStateUpdate, - ) -} - -// CopyAddedSeries copy label sets which were added via FindOrEmplace to destination. -func (s *Shard[TWal]) CopyAddedSeries(destination *Shard[TWal]) { - s.lssLocker.RLock() - s.lss.CopyAddedSeries(destination.lss.Target()) - s.lssLocker.RUnlock() -} - -// DataStorageAllocatedMemory return size of allocated memory for [DataStorage]. -func (s *Shard[TWal]) DataStorageAllocatedMemory() uint64 { - s.dataStorageLocker.RLock() - am := s.dataStorage.AllocatedMemory() - s.dataStorageLocker.RUnlock() - - return am -} - -// DataStorageInstantQuery returns samples for instant query from [DataStorage]. -func (s *Shard[TWal]) DataStorageInstantQuery( - maxt, valueNotFoundTimestampValue int64, - ids []uint32, -) []cppbridge.Sample { - s.dataStorageLocker.RLock() - samples := s.dataStorage.InstantQuery(maxt, valueNotFoundTimestampValue, ids) - s.dataStorageLocker.RUnlock() - - return samples -} - -// DataStorageQuery returns serialized chunks from data storage. -func (s *Shard[TWal]) DataStorageQuery( - query cppbridge.HeadDataStorageQuery, -) *cppbridge.HeadDataStorageSerializedChunks { - s.dataStorageLocker.RLock() - serializedChunks := s.dataStorage.Query(query) - s.dataStorageLocker.RUnlock() - - return serializedChunks -} - -// DataStorageQueryStatus get head status from [DataStorage]. -func (s *Shard[TWal]) DataStorageQueryStatus(status *cppbridge.HeadStatus) { - s.dataStorageLocker.RLock() - status.FromDataStorage(s.dataStorage.Raw()) - s.dataStorageLocker.RUnlock() -} - -// InputRelabeling relabeling incoming hashdex(first stage). -func (s *Shard[TWal]) InputRelabeling( - ctx context.Context, - relabeler *cppbridge.InputPerShardRelabeler, - cache *cppbridge.Cache, - options cppbridge.RelabelerOptions, - shardedData cppbridge.ShardedData, - shardsInnerSeries []*cppbridge.InnerSeries, - shardsRelabeledSeries []*cppbridge.RelabeledSeries, -) (cppbridge.RelabelerStats, bool, error) { - s.lssLocker.Lock() - defer s.lssLocker.Unlock() - - return relabeler.InputRelabeling( - ctx, - s.lss.Input(), - s.lss.Target(), - cache, - options, - shardedData, - shardsInnerSeries, - shardsRelabeledSeries, - ) -} - -// InputRelabelingFromCache relabeling incoming hashdex(first stage) from cache. -func (s *Shard[TWal]) InputRelabelingFromCache( - ctx context.Context, - relabeler *cppbridge.InputPerShardRelabeler, - cache *cppbridge.Cache, - options cppbridge.RelabelerOptions, - shardedData cppbridge.ShardedData, - shardsInnerSeries []*cppbridge.InnerSeries, -) (cppbridge.RelabelerStats, bool, error) { - s.lssLocker.RLock() - defer s.lssLocker.RUnlock() - - return relabeler.InputRelabelingFromCache( - ctx, - s.lss.Input(), - s.lss.Target(), - cache, - options, - shardedData, - shardsInnerSeries, - ) -} - -// InputRelabelingWithStalenans relabeling incoming hashdex(first stage) with state stalenans. -func (s *Shard[TWal]) InputRelabelingWithStalenans( - ctx context.Context, - relabeler *cppbridge.InputPerShardRelabeler, - cache *cppbridge.Cache, - options cppbridge.RelabelerOptions, - staleNansState *cppbridge.StaleNansState, - defTimestamp int64, - shardedData cppbridge.ShardedData, - shardsInnerSeries []*cppbridge.InnerSeries, - shardsRelabeledSeries []*cppbridge.RelabeledSeries, -) (cppbridge.RelabelerStats, bool, error) { - s.lssLocker.Lock() - defer s.lssLocker.Unlock() - - return relabeler.InputRelabelingWithStalenans( - ctx, - s.lss.Input(), - s.lss.Target(), - cache, - options, - staleNansState, - defTimestamp, - shardedData, - shardsInnerSeries, - shardsRelabeledSeries, - ) -} - -// InputRelabelingWithStalenansFromCache relabeling incoming hashdex(first stage) from cache with state stalenans. -func (s *Shard[TWal]) InputRelabelingWithStalenansFromCache( - ctx context.Context, - relabeler *cppbridge.InputPerShardRelabeler, - cache *cppbridge.Cache, - options cppbridge.RelabelerOptions, - staleNansState *cppbridge.StaleNansState, - defTimestamp int64, - shardedData cppbridge.ShardedData, - shardsInnerSeries []*cppbridge.InnerSeries, -) (cppbridge.RelabelerStats, bool, error) { - s.lssLocker.RLock() - defer s.lssLocker.RUnlock() - - return relabeler.InputRelabelingWithStalenansFromCache( - ctx, - s.lss.Input(), - s.lss.Target(), - cache, - options, - staleNansState, - defTimestamp, - shardedData, - shardsInnerSeries, - ) -} - -// LSSAllocatedMemory return size of allocated memory for labelset storages. -func (s *Shard[TWal]) LSSAllocatedMemory() uint64 { - s.lssLocker.RLock() - am := s.lss.AllocatedMemory() - s.lssLocker.RUnlock() - - return am -} - -// LSSQueryStatus get head status from lss. -func (s *Shard[TWal]) LSSQueryStatus(status *cppbridge.HeadStatus, limit int) { - s.lssLocker.RLock() - status.FromLSS(s.lss.Target(), limit) - s.lssLocker.RUnlock() -} - -// MergeOutOfOrderChunks merge chunks with out of order data chunks in [DataStorage]. -func (s *Shard[TWal]) MergeOutOfOrderChunks() { - s.dataStorageLocker.Lock() - s.dataStorage.MergeOutOfOrderChunks() - s.dataStorageLocker.Unlock() -} - -// QueryLabelNames returns all the unique label names present in lss in sorted order. -func (s *Shard[TWal]) QueryLabelNames( - matchers []model.LabelMatcher, - dedupAdd func(shardID uint16, snapshot *cppbridge.LabelSetSnapshot, values []string), -) error { - s.lssLocker.RLock() - queryLabelNamesResult := s.lss.QueryLabelNames(matchers) - snapshot := s.lss.GetSnapshot() - s.lssLocker.RUnlock() - - if queryLabelNamesResult.Status() != cppbridge.LSSQueryStatusMatch { - return fmt.Errorf("no matches on shard: %d", s.id) + id: shardID, + lss: lss, + dataStorage: dataStorage, + wal: wal, } - - dedupAdd(s.id, snapshot, queryLabelNamesResult.Names()) - runtime.KeepAlive(queryLabelNamesResult) - - return nil } -// QueryLabelValues query labels values to lss and add values to -// the dedup-container that matches the given label matchers. -func (s *Shard[TWal]) QueryLabelValues( - name string, - matchers []model.LabelMatcher, - dedupAdd func(shardID uint16, snapshot *cppbridge.LabelSetSnapshot, values []string), -) error { - s.lssLocker.RLock() - queryLabelValuesResult := s.lss.QueryLabelValues(name, matchers) - snapshot := s.lss.GetSnapshot() - s.lssLocker.RUnlock() - - if queryLabelValuesResult.Status() != cppbridge.LSSQueryStatusMatch { - return fmt.Errorf("no matches on shard: %d", s.id) - } - - dedupAdd(s.id, snapshot, queryLabelValuesResult.Values()) - runtime.KeepAlive(queryLabelValuesResult) - - return nil +// DataStorage returns shard [DataStorage]. +func (s *Shard[TWal]) DataStorage() *DataStorage { + return s.dataStorage } -// QuerySelector returns a created selector that matches the given label matchers. -func (s *Shard[TWal]) QuerySelector(matchers []model.LabelMatcher) (uintptr, *cppbridge.LabelSetSnapshot, error) { - s.lssLocker.RLock() - defer s.lssLocker.RUnlock() - - selector, status := s.lss.QuerySelector(matchers) - switch status { - case cppbridge.LSSQueryStatusMatch: - return selector, s.lss.GetSnapshot(), nil - - case cppbridge.LSSQueryStatusNoMatch: - return 0, nil, nil - - default: - return 0, nil, fmt.Errorf( - "failed to query selector from shard: %d, query status: %d", s.id, status, - ) - } +// LSS returns shard labelset storage [LSS]. +func (s *Shard[TWal]) LSS() *LSS { + return s.lss } // ShardID returns the shard ID. @@ -324,35 +60,121 @@ func (s *Shard[TWal]) ShardID() uint16 { // WalCommit finalize segment from encoder and write to wal. func (s *Shard[TWal]) WalCommit() error { - s.lssLocker.RLock() - s.walLocker.Lock() - - err := s.wal.Commit() - - s.walLocker.Unlock() - s.lssLocker.RUnlock() - - return err + return s.lss.WithRLock(func(_, _ *cppbridge.LabelSetStorage) error { + return s.wal.Commit() + }) } // WalFlush flush all contetnt into wal. func (s *Shard[TWal]) WalFlush() error { - s.walLocker.Lock() - - err := s.wal.Flush() - - s.walLocker.Unlock() - - return err + return s.wal.Flush() } // WalWrite append the incoming inner series to wal encoder. func (s *Shard[TWal]) WalWrite(innerSeriesSlice []*cppbridge.InnerSeries) (bool, error) { - s.walLocker.Lock() - - limitExhausted, err := s.wal.Write(innerSeriesSlice) - - s.walLocker.Unlock() - - return limitExhausted, err -} + return s.wal.Write(innerSeriesSlice) +} + +// // InputRelabeling relabeling incoming hashdex(first stage). +// func (s *Shard[TWal]) InputRelabeling( +// ctx context.Context, +// relabeler *cppbridge.InputPerShardRelabeler, +// cache *cppbridge.Cache, +// options cppbridge.RelabelerOptions, +// shardedData cppbridge.ShardedData, +// shardsInnerSeries []*cppbridge.InnerSeries, +// shardsRelabeledSeries []*cppbridge.RelabeledSeries, +// ) (cppbridge.RelabelerStats, bool, error) { +// s.lssLocker.Lock() +// defer s.lssLocker.Unlock() + +// return relabeler.InputRelabeling( +// ctx, +// s.lss.Input(), +// s.lss.Target(), +// cache, +// options, +// shardedData, +// shardsInnerSeries, +// shardsRelabeledSeries, +// ) +// } + +// // InputRelabelingFromCache relabeling incoming hashdex(first stage) from cache. +// func (s *Shard[TWal]) InputRelabelingFromCache( +// ctx context.Context, +// relabeler *cppbridge.InputPerShardRelabeler, +// cache *cppbridge.Cache, +// options cppbridge.RelabelerOptions, +// shardedData cppbridge.ShardedData, +// shardsInnerSeries []*cppbridge.InnerSeries, +// ) (cppbridge.RelabelerStats, bool, error) { +// s.lssLocker.RLock() +// defer s.lssLocker.RUnlock() + +// return relabeler.InputRelabelingFromCache( +// ctx, +// s.lss.Input(), +// s.lss.Target(), +// cache, +// options, +// shardedData, +// shardsInnerSeries, +// ) +// } + +// // InputRelabelingWithStalenans relabeling incoming hashdex(first stage) with state stalenans. +// func (s *Shard[TWal]) InputRelabelingWithStalenans( +// ctx context.Context, +// relabeler *cppbridge.InputPerShardRelabeler, +// cache *cppbridge.Cache, +// options cppbridge.RelabelerOptions, +// staleNansState *cppbridge.StaleNansState, +// defTimestamp int64, +// shardedData cppbridge.ShardedData, +// shardsInnerSeries []*cppbridge.InnerSeries, +// shardsRelabeledSeries []*cppbridge.RelabeledSeries, +// ) (cppbridge.RelabelerStats, bool, error) { +// s.lssLocker.Lock() +// defer s.lssLocker.Unlock() + +// return relabeler.InputRelabelingWithStalenans( +// ctx, +// s.lss.Input(), +// s.lss.Target(), +// cache, +// options, +// staleNansState, +// defTimestamp, +// shardedData, +// shardsInnerSeries, +// shardsRelabeledSeries, +// ) +// } + +// // InputRelabelingWithStalenansFromCache relabeling incoming hashdex(first stage) from cache with state stalenans. +// func (s *Shard[TWal]) InputRelabelingWithStalenansFromCache( +// ctx context.Context, +// relabeler *cppbridge.InputPerShardRelabeler, +// cache *cppbridge.Cache, +// options cppbridge.RelabelerOptions, +// staleNansState *cppbridge.StaleNansState, +// defTimestamp int64, +// shardedData cppbridge.ShardedData, +// shardsInnerSeries []*cppbridge.InnerSeries, +// ) (cppbridge.RelabelerStats, bool, error) { +// s.lssLocker.RLock() +// defer s.lssLocker.RUnlock() + +// return relabeler.InputRelabelingWithStalenansFromCache( +// ctx, +// s.lss.Input(), +// s.lss.Target(), +// cache, +// options, +// staleNansState, +// defTimestamp, +// shardedData, +// shardsInnerSeries, +// ) +// } diff --git a/pp/go/storage/head/shard/wal/wal.go b/pp/go/storage/head/shard/wal/wal.go index 15c1b5e4fd..6daf59d379 100644 --- a/pp/go/storage/head/shard/wal/wal.go +++ b/pp/go/storage/head/shard/wal/wal.go @@ -3,6 +3,7 @@ package wal import ( "fmt" "io" + "sync" "github.com/prometheus/prometheus/pp/go/cppbridge" ) @@ -47,6 +48,7 @@ type EncodedSegment interface { type Wal[TSegment EncodedSegment, TStats StatsSegment, TWriter SegmentWriter[TSegment]] struct { encoder Encoder[TSegment, TStats] // *cppbridge.HeadWalEncoder segmentWriter TWriter + locker sync.Mutex maxSegmentSize uint32 corrupted bool limitExhausted bool @@ -61,6 +63,7 @@ func NewWal[TSegment EncodedSegment, TStats StatsSegment, TWriter SegmentWriter[ return &Wal[TSegment, TStats, TWriter]{ encoder: encoder, segmentWriter: segmentWriter, + locker: sync.Mutex{}, maxSegmentSize: maxSegmentSize, } } @@ -72,6 +75,7 @@ func NewCorruptedWal[ TWriter SegmentWriter[TSegment], ]() *Wal[TSegment, TStats, TWriter] { return &Wal[TSegment, TStats, TWriter]{ + locker: sync.Mutex{}, corrupted: true, } } @@ -87,6 +91,9 @@ func (w *Wal[TSegment, TStats, TWriter]) Commit() error { return fmt.Errorf("committing corrupted wal") } + w.locker.Lock() + defer w.locker.Unlock() + segment, err := w.encoder.Finalize() if err != nil { return fmt.Errorf("failed to finalize segment: %w", err) @@ -111,6 +118,9 @@ func (w *Wal[TSegment, TStats, TWriter]) CurrentSize() int64 { // Flush wal [SegmentWriter]. func (w *Wal[TSegment, TStats, TWriter]) Flush() error { + w.locker.Lock() + defer w.locker.Unlock() + return w.segmentWriter.Flush() } @@ -120,6 +130,9 @@ func (w *Wal[TSegment, TStats, TWriter]) Write(innerSeriesSlice []*cppbridge.Inn return false, fmt.Errorf("writing in corrupted wal") } + w.locker.Lock() + defer w.locker.Unlock() + stats, err := w.encoder.Encode(innerSeriesSlice) if err != nil { return false, fmt.Errorf("failed to encode inner series: %w", err) diff --git a/pp/go/storage/querier/querier.go b/pp/go/storage/querier/querier.go index 87f8bc9426..8de658f8e0 100644 --- a/pp/go/storage/querier/querier.go +++ b/pp/go/storage/querier/querier.go @@ -61,34 +61,65 @@ type GenericTask interface { } // -// Shard +// DataStorage // -// Shard the minimum required head Shard implementation. -type Shard interface { - // DataStorageInstantQuery returns samples for instant query from data storage. - DataStorageInstantQuery( +// DataStorage the minimum required DataStorage implementation. +type DataStorage interface { + // InstantQuery returns samples for instant query from data storage. + InstantQuery( maxt, valueNotFoundTimestampValue int64, ids []uint32, ) []cppbridge.Sample + // QueryDataStorage returns serialized chunks from data storage. - DataStorageQuery( + Query( query cppbridge.HeadDataStorageQuery, ) *cppbridge.HeadDataStorageSerializedChunks +} + +// +// LSS +// + +// LSS the minimum required LSS implementation. +type LSS interface { // QueryLabelNames returns all the unique label names present in lss in sorted order. QueryLabelNames( + shardID uint16, matchers []model.LabelMatcher, dedupAdd func(shardID uint16, snapshot *cppbridge.LabelSetSnapshot, values []string), ) error + // QueryLabelValues query labels values to lss and add values to // the dedup-container that matches the given label matchers. QueryLabelValues( + shardID uint16, name string, matchers []model.LabelMatcher, dedupAdd func(shardID uint16, snapshot *cppbridge.LabelSetSnapshot, values []string), ) error + // QuerySelector returns a created selector that matches the given label matchers. - QuerySelector(matchers []model.LabelMatcher) (uintptr, *cppbridge.LabelSetSnapshot, error) + QuerySelector(shardID uint16, matchers []model.LabelMatcher) (uintptr, *cppbridge.LabelSetSnapshot, error) + + GetSnapshot() *cppbridge.LabelSetSnapshot + + WithRLock(fn func(target, input *cppbridge.LabelSetStorage) error) error +} + +// +// Shard +// + +// Shard the minimum required head Shard implementation. +type Shard[TDataStorage DataStorage, TLSS LSS] interface { + // DataStorage returns shard [DataStorage]. + DataStorage() TDataStorage + + // LSS returns shard labelset storage [LSS]. + LSS() TLSS + // ShardID returns the shard ID. ShardID() uint16 } @@ -100,12 +131,23 @@ type Shard interface { // Head the minimum required Head implementation. type Head[ TGenericTask GenericTask, - TShard Shard, + TDataStorage DataStorage, + TLSS LSS, + TShard Shard[TDataStorage, TLSS], ] interface { + // AcquireQuery acquires the [Head] semaphore with a weight of 1, + // blocking until resources are available or ctx is done. + // On success, returns nil. On failure, returns ctx.Err() and leaves the semaphore unchanged. + AcquireQuery(ctx context.Context) (release func(), err error) + + // CreateTask create a task for operations on the [Head] shards. CreateTask(taskName string, fn func(shard TShard) error) TGenericTask + + // Enqueue the task to be executed on shards [Head]. Enqueue(t TGenericTask) + + // NumberOfShards returns current number of shards in to [Head]. NumberOfShards() uint16 - RLockQuery(ctx context.Context) (runlock func(), err error) } // @@ -115,8 +157,10 @@ type Head[ // Querier provides querying access over time series data of a fixed time range. type Querier[ TGenericTask GenericTask, - TShard Shard, - THead Head[TGenericTask, TShard], + TDataStorage DataStorage, + TLSS LSS, + TShard Shard[TDataStorage, TLSS], + THead Head[TGenericTask, TDataStorage, TLSS, TShard], ] struct { mint int64 maxt int64 @@ -129,16 +173,18 @@ type Querier[ // NewQuerier init new [Querier]. func NewQuerier[ TGenericTask GenericTask, - TShard Shard, - THead Head[TGenericTask, TShard], + TDataStorage DataStorage, + TLSS LSS, + TShard Shard[TDataStorage, TLSS], + THead Head[TGenericTask, TDataStorage, TLSS, TShard], ]( head THead, deduplicatorCtor deduplicatorCtor, mint, maxt int64, closer func() error, metrics *Metrics, -) *Querier[TGenericTask, TShard, THead] { - return &Querier[TGenericTask, TShard, THead]{ +) *Querier[TGenericTask, TDataStorage, TLSS, TShard, THead] { + return &Querier[TGenericTask, TDataStorage, TLSS, TShard, THead]{ mint: mint, maxt: maxt, head: head, @@ -149,7 +195,7 @@ func NewQuerier[ } // Close Querier if need. -func (q *Querier[TGenericTask, TShard, THead]) Close() error { +func (q *Querier[TGenericTask, TDataStorage, TLSS, TShard, THead]) Close() error { if q.closer != nil { return q.closer() } @@ -158,7 +204,7 @@ func (q *Querier[TGenericTask, TShard, THead]) Close() error { } // LabelNames returns label values present in the head for the specific label name. -func (q *Querier[TGenericTask, TShard, THead]) LabelNames( +func (q *Querier[TGenericTask, TDataStorage, TLSS, TShard, THead]) LabelNames( ctx context.Context, hints *storage.LabelHints, matchers ...*labels.Matcher, @@ -177,7 +223,7 @@ func (q *Querier[TGenericTask, TShard, THead]) LabelNames( // LabelValues returns label values present in the head for the specific label name // that are within the time range mint to maxt. If matchers are specified the returned // result set is reduced to label values of metrics matching the matchers. -func (q *Querier[TGenericTask, TShard, THead]) LabelValues( +func (q *Querier[TGenericTask, TDataStorage, TLSS, TShard, THead]) LabelValues( ctx context.Context, name string, matchers ...*labels.Matcher, @@ -194,7 +240,7 @@ func (q *Querier[TGenericTask, TShard, THead]) LabelValues( } // Select returns a set of series that matches the given label matchers. -func (q *Querier[TGenericTask, TShard, THead]) Select( +func (q *Querier[TGenericTask, TDataStorage, TLSS, TShard, THead]) Select( ctx context.Context, sortSeries bool, hints *storage.SelectHints, @@ -209,7 +255,7 @@ func (q *Querier[TGenericTask, TShard, THead]) Select( // selectInstant returns a instant set of series that matches the given label matchers. // //revive:disable-next-line:function-length long but readable. -func (q *Querier[TGenericTask, TShard, THead]) selectInstant( +func (q *Querier[TGenericTask, TDataStorage, TLSS, TShard, THead]) selectInstant( ctx context.Context, _ bool, _ *storage.SelectHints, @@ -217,12 +263,12 @@ func (q *Querier[TGenericTask, TShard, THead]) selectInstant( ) storage.SeriesSet { start := time.Now() - runlock, err := q.head.RLockQuery(ctx) + release, err := q.head.AcquireQuery(ctx) if err != nil { logger.Warnf("[QUERIER]: select instant failed on the capture of the read lock query: %s", err) return storage.ErrSeriesSet(err) } - defer runlock() + defer release() defer func() { if q.metrics == nil { @@ -259,7 +305,7 @@ func (q *Querier[TGenericTask, TShard, THead]) selectInstant( lssQueryResult, snapshots[shardID], valueNotFoundTimestampValue, - shard.DataStorageInstantQuery(q.maxt, valueNotFoundTimestampValue, lssQueryResult.IDs()), + shard.DataStorage().InstantQuery(q.maxt, valueNotFoundTimestampValue, lssQueryResult.IDs()), ) return nil @@ -272,7 +318,7 @@ func (q *Querier[TGenericTask, TShard, THead]) selectInstant( } // selectRange returns a range set of series that matches the given label matchers. -func (q *Querier[TGenericTask, TShard, THead]) selectRange( +func (q *Querier[TGenericTask, TDataStorage, TLSS, TShard, THead]) selectRange( ctx context.Context, _ bool, _ *storage.SelectHints, @@ -280,12 +326,12 @@ func (q *Querier[TGenericTask, TShard, THead]) selectRange( ) storage.SeriesSet { start := time.Now() - runlock, err := q.head.RLockQuery(ctx) + release, err := q.head.AcquireQuery(ctx) if err != nil { logger.Warnf("[QUERIER]: select range failed on the capture of the read lock query: %s", err) return storage.ErrSeriesSet(err) } - defer runlock() + defer release() defer func() { if q.metrics != nil { @@ -340,8 +386,10 @@ func convertPrometheusMatchersToPPMatchers(matchers ...*labels.Matcher) []model. // queryDataStorage returns serialized chunks from data storage for each shard. func queryDataStorage[ TGenericTask GenericTask, - TShard Shard, - THead Head[TGenericTask, TShard], + TDataStorage DataStorage, + TLSS LSS, + TShard Shard[TDataStorage, TLSS], + THead Head[TGenericTask, TDataStorage, TLSS, TShard], ]( taskName string, head THead, @@ -358,7 +406,7 @@ func queryDataStorage[ return nil } - serializedChunks := shard.DataStorageQuery(cppbridge.HeadDataStorageQuery{ + serializedChunks := shard.DataStorage().Query(cppbridge.HeadDataStorageQuery{ StartTimestampMs: mint, EndTimestampMs: maxt, LabelSetIDs: lssQueryResult.IDs(), @@ -382,8 +430,10 @@ func queryDataStorage[ // queryLabelValues returns label values present in the head for the specific label name. func queryLabelNames[ TGenericTask GenericTask, - TShard Shard, - THead Head[TGenericTask, TShard], + TDataStorage DataStorage, + TLSS LSS, + TShard Shard[TDataStorage, TLSS], + THead Head[TGenericTask, TDataStorage, TLSS, TShard], ]( ctx context.Context, head THead, @@ -396,12 +446,12 @@ func queryLabelNames[ start := time.Now() anns := *annotations.New() - runlock, err := head.RLockQuery(ctx) + release, err := head.AcquireQuery(ctx) if err != nil { logger.Warnf("[QUERIER]: label names failed on the capture of the read lock query: %s", err) return nil, anns, err } - defer runlock() + defer release() defer func() { if metrics != nil { @@ -415,7 +465,7 @@ func queryLabelNames[ t := head.CreateTask( taskName, func(shard TShard) error { - return shard.QueryLabelNames(convertedMatchers, dedup.Add) + return shard.LSS().QueryLabelNames(shard.ShardID(), convertedMatchers, dedup.Add) }, ) head.Enqueue(t) @@ -442,8 +492,10 @@ func queryLabelNames[ // queryLabelValues returns label values present in the head for the specific label name. func queryLabelValues[ TGenericTask GenericTask, - TShard Shard, - THead Head[TGenericTask, TShard], + TDataStorage DataStorage, + TLSS LSS, + TShard Shard[TDataStorage, TLSS], + THead Head[TGenericTask, TDataStorage, TLSS, TShard], ]( ctx context.Context, name string, @@ -456,12 +508,12 @@ func queryLabelValues[ start := time.Now() anns := *annotations.New() - runlock, err := head.RLockQuery(ctx) + release, err := head.AcquireQuery(ctx) if err != nil { logger.Warnf("[QUERIER]: label values failed on the capture of the read lock query: %s", err) return nil, anns, err } - defer runlock() + defer release() defer func() { if metrics != nil { @@ -475,7 +527,7 @@ func queryLabelValues[ t := head.CreateTask( taskName, func(shard TShard) error { - return shard.QueryLabelValues(name, convertedMatchers, dedup.Add) + return shard.LSS().QueryLabelValues(shard.ShardID(), name, convertedMatchers, dedup.Add) }, ) head.Enqueue(t) @@ -501,8 +553,10 @@ func queryLabelValues[ //revive:disable-next-line:cyclomatic but readable. func queryLss[ TGenericTask GenericTask, - TShard Shard, - THead Head[TGenericTask, TShard], + TDataStorage DataStorage, + TLSS LSS, + TShard Shard[TDataStorage, TLSS], + THead Head[TGenericTask, TDataStorage, TLSS, TShard], ]( taskName string, head THead, @@ -520,9 +574,26 @@ func queryLss[ tLSSQuerySelector := head.CreateTask( taskName, func(shard TShard) (err error) { - selectors[shard.ShardID()], snapshots[shard.ShardID()], err = shard.QuerySelector(convertedMatchers) + shardID := shard.ShardID() + lss := shard.LSS() + + return lss.WithRLock(func(target, _ *cppbridge.LabelSetStorage) error { + selector, status := target.QuerySelector(convertedMatchers) + switch status { + case cppbridge.LSSQueryStatusMatch: + selectors[shardID] = selector + snapshots[shardID] = lss.GetSnapshot() + + case cppbridge.LSSQueryStatusNoMatch: - return err + default: + return fmt.Errorf( + "failed to query selector from shard: %d, query status: %d", shardID, status, + ) + } + + return nil + }) }, ) head.Enqueue(tLSSQuerySelector) From 97c26b2b5126d325a27c3dff3d0092eabadb194b Mon Sep 17 00:00:00 2001 From: Alexandr Yudin Date: Mon, 11 Aug 2025 14:34:56 +0000 Subject: [PATCH 12/96] fix head --- pp/go/storage/head/head/head.go | 457 +++++++++++++++++++++++--- pp/go/storage/head/head/head_test.go | 17 +- pp/go/storage/head/shard/lss.go | 37 +-- pp/go/storage/head/shard/shard.go | 40 +++ pp/go/storage/querier/querier.go | 44 +-- pp/go/storage/querier/querier_test.go | 20 +- 6 files changed, 520 insertions(+), 95 deletions(-) diff --git a/pp/go/storage/head/head/head.go b/pp/go/storage/head/head/head.go index e74618a37e..33616d3b44 100644 --- a/pp/go/storage/head/head/head.go +++ b/pp/go/storage/head/head/head.go @@ -2,15 +2,39 @@ package head import ( "context" + "errors" "fmt" + "runtime" + "strconv" "sync" + "sync/atomic" + "github.com/prometheus/client_golang/prometheus" "github.com/prometheus/prometheus/pp/go/storage/head/task" + "github.com/prometheus/prometheus/pp/go/storage/logger" + "github.com/prometheus/prometheus/pp/go/util" "github.com/prometheus/prometheus/pp/go/util/locker" ) // ExtraWorkers number of extra workers for operation on shards. -var ExtraConcurrency = 0 +var ExtraWorkers = 0 + +// defaultNumberOfWorkers default number of workers. +const defaultNumberOfWorkers = 2 + +// const ( +// // DSAllocatedMemory name of task. +// DSAllocatedMemory = "data_storage_allocated_memory" + +// // DSHeadStatus name of task. +// DSHeadStatus = "data_storage_head_status" + +// // LSSAllocatedMemory name of task. +// LSSAllocatedMemory = "lss_allocated_memory" + +// // LSSHeadStatus name of task. +// LSSHeadStatus = "lss_head_status" +// ) // // Shard @@ -18,90 +42,449 @@ var ExtraConcurrency = 0 // Shard the minimum required head Shard implementation. type Shard interface { + // LSS() *LSS + // ShardID returns the shard ID. ShardID() uint16 + + // Close closes the wal segmentWriter. + Close() error } // // Head // -type Head[TShard Shard] struct { +type Head[TShard Shard, TGorutineShard Shard] struct { id string generation uint64 - readOnly bool - shards []TShard - lssTaskChs []chan *task.Generic[TShard] - dataStorageTaskChs []chan *task.Generic[TShard] - querysemaphore *locker.Weighted + gshardCtor func(TShard) TGorutineShard + shards []TShard + taskChs []chan *task.Generic[TGorutineShard] + querySemaphore *locker.Weighted + + stopc chan struct{} + wg sync.WaitGroup + readOnly uint32 numberOfShards uint16 - stopc chan struct{} - wg sync.WaitGroup - // // stat - // appendedSegmentCount prometheus.Counter - // memoryInUse *prometheus.GaugeVec - // series prometheus.Gauge - // walSize *prometheus.GaugeVec - // // TODO refactoring - // queueLSS *prometheus.GaugeVec - // queueDataStorage *prometheus.GaugeVec + // stat + memoryInUse *prometheus.GaugeVec + series prometheus.Gauge + walSize *prometheus.GaugeVec + queueSize *prometheus.GaugeVec - // tasksCreated *prometheus.CounterVec - // tasksDone *prometheus.CounterVec - // tasksLive *prometheus.CounterVec - // tasksExecute *prometheus.CounterVec + tasksCreated *prometheus.CounterVec + tasksDone *prometheus.CounterVec + tasksLive *prometheus.CounterVec + tasksExecute *prometheus.CounterVec } // NewHead init new [Head]. -func NewHead[TShard Shard](shards []TShard) *Head[TShard] { - return &Head[TShard]{ - shards: shards, +func NewHead[TShard Shard, TGoroutineShard Shard]( + id string, + shards []TShard, + gshardCtor func(TShard) TGoroutineShard, + numberOfShards uint16, + registerer prometheus.Registerer, +) *Head[TShard, TGoroutineShard] { + taskChs := make([]chan *task.Generic[TGoroutineShard], numberOfShards) + concurrency := calculateHeadConcurrency(numberOfShards) // current head workers concurrency + + for shardID := uint16(0); shardID < numberOfShards; shardID++ { + taskChs[shardID] = make(chan *task.Generic[TGoroutineShard], 4*concurrency) // x4 for back pressure } + + factory := util.NewUnconflictRegisterer(registerer) + h := &Head[TShard, TGoroutineShard]{ + id: id, + gshardCtor: gshardCtor, + shards: shards, + taskChs: taskChs, + numberOfShards: uint16(len(shards)), // #nosec G115 // no overflow + memoryInUse: factory.NewGaugeVec( + prometheus.GaugeOpts{ + Name: "prompp_head_cgo_memory_bytes", + Help: "Current value memory in use in bytes.", + }, + // TODO generation -> h.id + []string{"generation", "allocator", "id"}, + ), + series: factory.NewGauge(prometheus.GaugeOpts{ + Name: "prompp_head_series", + Help: "Total number of series in the heads block.", + }), + walSize: factory.NewGaugeVec( + prometheus.GaugeOpts{ + Name: "prompp_head_current_wal_size", + Help: "The size of the wall of the current head.", + }, + []string{"shard_id"}, + ), + + queueSize: factory.NewGaugeVec( + prometheus.GaugeOpts{ + Name: "prompp_head_queue_tasks_size", + Help: "The size of the queue of tasks of the current head.", + }, + []string{"shard_id"}, + ), + + tasksCreated: factory.NewCounterVec( + prometheus.CounterOpts{ + Name: "prompp_head_task_created_count", + Help: "Number of created tasks.", + }, + []string{"type_task"}, + ), + tasksDone: factory.NewCounterVec( + prometheus.CounterOpts{ + Name: "prompp_head_task_done_count", + Help: "Number of done tasks.", + }, + []string{"type_task"}, + ), + tasksLive: factory.NewCounterVec( + prometheus.CounterOpts{ + Name: "prompp_head_task_live_duration_microseconds_sum", + Help: "The duration of the live task in microseconds.", + }, + []string{"type_task"}, + ), + tasksExecute: factory.NewCounterVec( + prometheus.CounterOpts{ + Name: "prompp_head_task_execute_duration_microseconds_sum", + Help: "The duration of the task execution in microseconds.", + }, + []string{"type_task"}, + ), + } + + h.run() + + runtime.SetFinalizer(h, func(h *Head[TShard, TGoroutineShard]) { + logger.Debugf("head %s destroyed", h.String()) + }) + + return h } // AcquireQuery acquires the [Head] semaphore with a weight of 1, // blocking until resources are available or ctx is done. // On success, returns nil. On failure, returns ctx.Err() and leaves the semaphore unchanged. -func (h *Head[TShard]) AcquireQuery(ctx context.Context) (release func(), err error) { - return h.querysemaphore.RLock(ctx) +func (h *Head[TShard, TGorutineShard]) AcquireQuery(ctx context.Context) (release func(), err error) { + return h.querySemaphore.RLock(ctx) +} + +// Close wals and clear metrics. +func (h *Head[TShard, TGorutineShard]) Close() error { + h.memoryInUse.DeletePartialMatch(prometheus.Labels{"generation": strconv.FormatUint(h.generation, 10)}) + + close(h.stopc) + h.wg.Wait() + + var err error + for _, s := range h.shards { + err = errors.Join(err, s.Close()) + } + + return err } // Concurrency return current head workers concurrency. -func (h *Head[TShard]) Concurrency() int64 { +func (h *Head[TShard, TGorutineShard]) Concurrency() int64 { return calculateHeadConcurrency(h.numberOfShards) } // CreateTask create a task for operations on the [Head] shards. -func (h *Head[TShard]) CreateTask(taskName string, fn func(shard TShard) error) *task.Generic[TShard] { - // TODO - return nil +func (h *Head[TShard, TGorutineShard]) CreateTask( + taskName string, + shardFn func(shard TGorutineShard) error, +) *task.Generic[TGorutineShard] { + ls := prometheus.Labels{"type_task": taskName} + + return task.NewGeneric( + shardFn, + h.tasksCreated.With(ls), + h.tasksDone.With(ls), + h.tasksLive.With(ls), + h.tasksExecute.With(ls), + ) } // Enqueue the task to be executed on shards [Head]. -func (h *Head[TShard]) Enqueue(t *task.Generic[TShard]) { - // TODO +func (h *Head[TShard, TGorutineShard]) Enqueue(t *task.Generic[TGorutineShard]) { + t.SetShardsNumber(h.numberOfShards) + + for _, taskCh := range h.taskChs { + taskCh <- t + } +} + +// EnqueueOnShard the task to be executed on head on specific shard. +func (h *Head[TShard, TGorutineShard]) EnqueueOnShard(t *task.Generic[TGorutineShard], shardID uint16) { + t.SetShardsNumber(1) + + h.taskChs[shardID] <- t } // ID returns id [Head]. -func (h *Head[TShard]) ID() string { +func (h *Head[TShard, TGorutineShard]) ID() string { return h.id } +// IsReadOnly returns true if the [Head] has switched to read-only. +func (h *Head[TShard, TGorutineShard]) IsReadOnly() bool { + return atomic.LoadUint32(&h.readOnly) > 0 +} + // NumberOfShards returns current number of shards in to [Head]. -func (h *Head[TShard]) NumberOfShards() uint16 { +func (h *Head[TShard, TGorutineShard]) NumberOfShards() uint16 { return h.numberOfShards } +// SetReadOnly sets the read-only flag for the [Head]. +func (h *Head[TShard, TGorutineShard]) SetReadOnly() { + atomic.StoreUint32(&h.readOnly, 1) +} + // String serialize as string. -func (h *Head[TShard]) String() string { +func (h *Head[TShard, TGorutineShard]) String() string { return fmt.Sprintf("{id: %s, generation: %d}", h.id, h.generation) } +// run loop for each shard. +func (h *Head[TShard, TGorutineShard]) run() { + workers := defaultNumberOfWorkers + ExtraWorkers + h.wg.Add(workers * int(h.numberOfShards)) + for shardID := uint16(0); shardID < h.numberOfShards; shardID++ { + for i := 0; i < workers; i++ { + go func(sid uint16) { + defer h.wg.Done() + h.shardLoop(h.taskChs[sid], h.stopc, h.shards[sid]) + }(shardID) + } + } +} + +// shardLoop run shard loop for operation. +func (h *Head[TShard, TGorutineShard]) shardLoop( + taskCH chan *task.Generic[TGorutineShard], + stopc chan struct{}, + s TShard, +) { + // TODO PerGoroutineRelabeler + pgs := h.gshardCtor(s) + + for { + select { + case <-stopc: + return + + case t := <-taskCH: + t.ExecuteOnShard(pgs) + } + } +} + // calculateHeadConcurrency calculate current head workers concurrency. func calculateHeadConcurrency(numberOfShards uint16) int64 { - //revive:disable-next-line:add-constant 2 - default run workers - return 2 * int64(1+ExtraConcurrency) * int64(numberOfShards) + return int64(defaultNumberOfWorkers+ExtraWorkers) * int64(numberOfShards) } + +// TODO Who? +// // getSortedStats returns sorted statistics for the [Head]. +// func getSortedStats(stats map[string]uint64, limit int) []storage.HeadStat { +// result := make([]storage.HeadStat, 0, len(stats)) +// for k, v := range stats { +// result = append(result, storage.HeadStat{ +// Name: k, +// Value: v, +// }) +// } + +// sort.Slice(result, func(i, j int) bool { +// return result[i].Value > result[j].Value +// }) + +// if len(result) > limit { +// return result[:limit] +// } + +// return result +// } + +// func (h *Head[TShard, TGorutineShard]) Status(limit int) storage.HeadStatus { +// shardStatuses := make([]*cppbridge.HeadStatus, h.NumberOfShards()) +// for i := range shardStatuses { +// shardStatuses[i] = cppbridge.NewHeadStatus() +// } + +// tw := task.NewTaskWaiter[*task.Generic[TGorutineShard]](2) + +// tLSSHeadStatus := h.CreateTask( +// LSSHeadStatus, +// func(shard TGorutineShard) error { +// shard.LSSRLock() +// shardStatuses[shard.ShardID()].FromLSS(shard.LSS().Raw(), limit) +// shard.LSSRUnlock() + +// return nil +// }, +// ) +// h.Enqueue(tLSSHeadStatus) + +// if limit != 0 { +// tDataStorageHeadStatus := h.CreateTask( +// DSHeadStatus, +// func(shard TGorutineShard) error { +// shard.DataStorageRLock() +// shardStatuses[shard.ShardID()].FromDataStorage(shard.DataStorage().Raw()) +// shard.DataStorageRUnlock() + +// return nil +// }, +// ) +// h.Enqueue(tDataStorageHeadStatus) +// tw.Add(tDataStorageHeadStatus) +// } + +// tw.Add(tLSSHeadStatus) +// _ = tw.Wait() + +// headStatus := storage.HeadStatus{ +// HeadStats: storage.HeadStats{ +// MinTime: math.MaxInt64, +// MaxTime: math.MinInt64, +// }, +// } + +// seriesStats := make(map[string]uint64) +// labelsStats := make(map[string]uint64) +// memoryStats := make(map[string]uint64) +// countStats := make(map[string]uint64) + +// for _, shardStatus := range shardStatuses { +// headStatus.HeadStats.NumSeries += uint64(shardStatus.NumSeries) +// if limit == 0 { +// continue +// } + +// headStatus.HeadStats.ChunkCount += int64(shardStatus.ChunkCount) +// if headStatus.HeadStats.MaxTime < shardStatus.TimeInterval.Max { +// headStatus.HeadStats.MaxTime = shardStatus.TimeInterval.Max +// } +// if headStatus.HeadStats.MinTime > shardStatus.TimeInterval.Min { +// headStatus.HeadStats.MinTime = shardStatus.TimeInterval.Min +// } + +// headStatus.HeadStats.NumLabelPairs += int(shardStatus.NumLabelPairs) + +// for _, stat := range shardStatus.SeriesCountByMetricName { +// seriesStats[stat.Name] += uint64(stat.Count) +// } +// for _, stat := range shardStatus.LabelValueCountByLabelName { +// labelsStats[stat.Name] += uint64(stat.Count) +// } +// for _, stat := range shardStatus.MemoryInBytesByLabelName { +// memoryStats[stat.Name] += uint64(stat.Size) +// } +// for _, stat := range shardStatus.SeriesCountByLabelValuePair { +// countStats[stat.Name+"="+stat.Value] += uint64(stat.Count) +// } +// } + +// if limit == 0 { +// return headStatus +// } + +// headStatus.SeriesCountByMetricName = getSortedStats(seriesStats, limit) +// headStatus.LabelValueCountByLabelName = getSortedStats(labelsStats, limit) +// headStatus.MemoryInBytesByLabelName = getSortedStats(memoryStats, limit) +// headStatus.SeriesCountByLabelValuePair = getSortedStats(countStats, limit) + +// return headStatus +// } + +// func (h *Head[TShard, TGorutineShard]) WriteMetrics(ctx context.Context) { +// if ctx.Err() != nil { +// return +// } + +// status := h.Status(0) +// h.series.Set(float64(status.HeadStats.NumSeries)) + +// if ctx.Err() != nil { +// return +// } + +// generationStr := strconv.FormatUint(h.generation, 10) +// tw := task.NewTaskWaiter[*task.Generic[TGorutineShard]](2) + +// tDataStorageHeadAllocatedMemory := h.CreateTask( +// DSAllocatedMemory, +// func(shard TGorutineShard) error { +// shard.DataStorageRLock() +// am := shard.DataStorage().AllocatedMemory() +// shard.DataStorageRUnlock() + +// h.memoryInUse.With( +// prometheus.Labels{ +// "generation": generationStr, +// "allocator": "data_storage", +// "id": strconv.FormatUint(uint64(shard.ShardID()), 10), +// }, +// ).Set(float64(am)) + +// return nil +// }, +// ) +// h.Enqueue(tDataStorageHeadAllocatedMemory) + +// tLSSHeadAllocatedMemory := h.CreateTask( +// LSSAllocatedMemory, +// func(shard TGorutineShard) error { +// shard.LSSRLock() +// am := shard.LSS().AllocatedMemory() +// shard.LSSRUnlock() + +// h.memoryInUse.With( +// prometheus.Labels{ +// "generation": generationStr, +// "allocator": "main_lss", +// "id": strconv.FormatUint(uint64(shard.ShardID()), 10), +// }, +// ).Set(float64(am)) + +// return nil +// }, +// ) +// h.Enqueue(tLSSHeadAllocatedMemory) + +// tw.Add(tLSSHeadAllocatedMemory) +// tw.Add(tDataStorageHeadAllocatedMemory) +// _ = tw.Wait() + +// if h.readOnly { +// return +// } + +// if ctx.Err() != nil { +// return +// } + +// // do not write metrics if the head is read-only. +// for shardID := uint16(0); shardID < h.numberOfShards; shardID++ { +// shardIDStr := strconv.FormatUint(uint64(shardID), 10) + +// h.walSize.With( +// prometheus.Labels{"shard_id": shardIDStr}, +// ).Set(float64(h.shards[shardID].wal.CurrentSize())) + +// h.queueSize.With(prometheus.Labels{ +// "shard_id": shardIDStr, +// }).Set(float64(len(h.taskChs[shardID]))) +// } +// } diff --git a/pp/go/storage/head/head/head_test.go b/pp/go/storage/head/head/head_test.go index 6c002563d7..cd0cd38401 100644 --- a/pp/go/storage/head/head/head_test.go +++ b/pp/go/storage/head/head/head_test.go @@ -13,14 +13,27 @@ func TestXxx(t *testing.T) { ds := shard.NewDataStorage() wl := &testWal{} sd := shard.NewShard(lss, ds, wl, 0) - - h := head.NewHead([]*shard.Shard[*testWal]{sd}) + id := "test-head-id" + numberOfShards := uint16(2) + + h := head.NewHead( + id, + []*shard.Shard[*testWal]{sd}, + shard.NewPerGoroutineShard[*testWal], + numberOfShards, + nil, + ) _ = h } // testWal test implementation wal. type testWal struct{} +// Close test implementation wal. +func (*testWal) Close() error { + return nil +} + // Commit test implementation wal. func (*testWal) Commit() error { return nil diff --git a/pp/go/storage/head/shard/lss.go b/pp/go/storage/head/shard/lss.go index 34f245fd81..022047de75 100644 --- a/pp/go/storage/head/shard/lss.go +++ b/pp/go/storage/head/shard/lss.go @@ -28,22 +28,12 @@ func (l *LSS) AllocatedMemory() uint64 { } // CopyAddedSeries copy label sets which were added via FindOrEmplace to destination. -func (l *LSS) CopyAddedSeries(destination *cppbridge.LabelSetStorage) { +func (l *LSS) CopyAddedSeries(destination *LSS) { l.locker.RLock() - l.target.CopyAddedSeries(destination) + l.target.CopyAddedSeries(destination.target) l.locker.RUnlock() } -// GetSnapshot return the actual snapshot. -func (l *LSS) GetSnapshot() *cppbridge.LabelSetSnapshot { - // TODO - l.once.Do(func() { - l.snapshot = l.target.CreateLabelSetSnapshot() - }) - - return l.snapshot -} - // Input returns input lss. func (l *LSS) Input() *cppbridge.LabelSetStorage { return l.input @@ -56,15 +46,15 @@ func (l *LSS) QueryLabelNames( dedupAdd func(shardID uint16, snapshot *cppbridge.LabelSetSnapshot, values []string), ) error { l.locker.RLock() + defer l.locker.RUnlock() + queryLabelNamesResult := l.target.QueryLabelNames(matchers) - snapshot := l.GetSnapshot() - l.locker.RUnlock() if queryLabelNamesResult.Status() != cppbridge.LSSQueryStatusMatch { return fmt.Errorf("no matches on shard: %d", shardID) } - dedupAdd(shardID, snapshot, queryLabelNamesResult.Names()) + dedupAdd(shardID, l.getSnapshot(), queryLabelNamesResult.Names()) runtime.KeepAlive(queryLabelNamesResult) return nil @@ -79,15 +69,15 @@ func (l *LSS) QueryLabelValues( dedupAdd func(shardID uint16, snapshot *cppbridge.LabelSetSnapshot, values []string), ) error { l.locker.RLock() + defer l.locker.RUnlock() + queryLabelValuesResult := l.target.QueryLabelValues(name, matchers) - snapshot := l.GetSnapshot() - l.locker.RUnlock() if queryLabelValuesResult.Status() != cppbridge.LSSQueryStatusMatch { return fmt.Errorf("no matches on shard: %d", shardID) } - dedupAdd(shardID, snapshot, queryLabelValuesResult.Values()) + dedupAdd(shardID, l.getSnapshot(), queryLabelValuesResult.Values()) runtime.KeepAlive(queryLabelValuesResult) return nil @@ -105,7 +95,7 @@ func (l *LSS) QuerySelector(shardID uint16, matchers []model.LabelMatcher) ( selector, status := l.target.QuerySelector(matchers) switch status { case cppbridge.LSSQueryStatusMatch: - return selector, l.GetSnapshot(), nil + return selector, l.getSnapshot(), nil case cppbridge.LSSQueryStatusNoMatch: return 0, nil, nil @@ -153,3 +143,12 @@ func (l *LSS) WithRLock(fn func(target, input *cppbridge.LabelSetStorage) error) return err } + +// getSnapshot return the actual snapshot. +func (l *LSS) getSnapshot() *cppbridge.LabelSetSnapshot { + l.once.Do(func() { + l.snapshot = l.target.CreateLabelSetSnapshot() + }) + + return l.snapshot +} diff --git a/pp/go/storage/head/shard/shard.go b/pp/go/storage/head/shard/shard.go index 99a4337c89..a568e41814 100644 --- a/pp/go/storage/head/shard/shard.go +++ b/pp/go/storage/head/shard/shard.go @@ -10,10 +10,15 @@ import ( type Wal interface { // Commit finalize segment from encoder and write to wal. Commit() error + // WalFlush flush all contetnt into wal. Flush() error + // WalWrite append the incoming inner series to wal encoder. Write(innerSeriesSlice []*cppbridge.InnerSeries) (bool, error) + + // Close closes the wal segmentWriter. + Close() error } // @@ -43,6 +48,11 @@ func NewShard[TWal Wal]( } } +// Close closes the wal segmentWriter. +func (s *Shard[TWal]) Close() error { + return s.wal.Close() +} + // DataStorage returns shard [DataStorage]. func (s *Shard[TWal]) DataStorage() *DataStorage { return s.dataStorage @@ -58,6 +68,11 @@ func (s *Shard[TWal]) ShardID() uint16 { return s.id } +// Wal returns write-ahead log. +func (s *Shard[TWal]) Wal() TWal { + return s.wal +} + // WalCommit finalize segment from encoder and write to wal. func (s *Shard[TWal]) WalCommit() error { return s.lss.WithRLock(func(_, _ *cppbridge.LabelSetStorage) error { @@ -75,6 +90,31 @@ func (s *Shard[TWal]) WalWrite(innerSeriesSlice []*cppbridge.InnerSeries) (bool, return s.wal.Write(innerSeriesSlice) } +// +// PerGoroutineShard +// + +// PerGoroutineShard wrapper of shard with [PerGoroutineRelabeler] for goroutines. +type PerGoroutineShard[TWal Wal] struct { + // TODO PerGoroutineRelabeler + relabeler string + *Shard[TWal] +} + +// NewPerGoroutineShard init new [PerGoroutineShard]. +func NewPerGoroutineShard[TWal Wal](s *Shard[TWal]) *PerGoroutineShard[TWal] { + return &PerGoroutineShard[TWal]{ + Shard: s, + } +} + +// TODO implementation. +func (s *PerGoroutineShard[TWal]) SetRelabeler() { +} + +func (s *PerGoroutineShard[TWal]) GetRelabeler() { +} + // // InputRelabeling relabeling incoming hashdex(first stage). // func (s *Shard[TWal]) InputRelabeling( // ctx context.Context, diff --git a/pp/go/storage/querier/querier.go b/pp/go/storage/querier/querier.go index 8de658f8e0..ecd296c084 100644 --- a/pp/go/storage/querier/querier.go +++ b/pp/go/storage/querier/querier.go @@ -64,25 +64,20 @@ type GenericTask interface { // DataStorage // -// DataStorage the minimum required DataStorage implementation. +// DataStorage the minimum required [DataStorage] implementation. type DataStorage interface { // InstantQuery returns samples for instant query from data storage. - InstantQuery( - maxt, valueNotFoundTimestampValue int64, - ids []uint32, - ) []cppbridge.Sample + InstantQuery(maxt, valueNotFoundTimestampValue int64, ids []uint32) []cppbridge.Sample // QueryDataStorage returns serialized chunks from data storage. - Query( - query cppbridge.HeadDataStorageQuery, - ) *cppbridge.HeadDataStorageSerializedChunks + Query(query cppbridge.HeadDataStorageQuery) *cppbridge.HeadDataStorageSerializedChunks } // // LSS // -// LSS the minimum required LSS implementation. +// LSS the minimum required [LSS] implementation. type LSS interface { // QueryLabelNames returns all the unique label names present in lss in sorted order. QueryLabelNames( @@ -102,17 +97,13 @@ type LSS interface { // QuerySelector returns a created selector that matches the given label matchers. QuerySelector(shardID uint16, matchers []model.LabelMatcher) (uintptr, *cppbridge.LabelSetSnapshot, error) - - GetSnapshot() *cppbridge.LabelSetSnapshot - - WithRLock(fn func(target, input *cppbridge.LabelSetStorage) error) error } // // Shard // -// Shard the minimum required head Shard implementation. +// Shard the minimum required head [Shard] implementation. type Shard[TDataStorage DataStorage, TLSS LSS] interface { // DataStorage returns shard [DataStorage]. DataStorage() TDataStorage @@ -128,7 +119,7 @@ type Shard[TDataStorage DataStorage, TLSS LSS] interface { // Head // -// Head the minimum required Head implementation. +// Head the minimum required [Head] implementation. type Head[ TGenericTask GenericTask, TDataStorage DataStorage, @@ -141,7 +132,7 @@ type Head[ AcquireQuery(ctx context.Context) (release func(), err error) // CreateTask create a task for operations on the [Head] shards. - CreateTask(taskName string, fn func(shard TShard) error) TGenericTask + CreateTask(taskName string, shardFn func(shard TShard) error) TGenericTask // Enqueue the task to be executed on shards [Head]. Enqueue(t TGenericTask) @@ -551,6 +542,7 @@ func queryLabelValues[ // lssQuery returns query results and snapshots. // //revive:disable-next-line:cyclomatic but readable. +//revive:disable-next-line:function-length long but readable. func queryLss[ TGenericTask GenericTask, TDataStorage DataStorage, @@ -575,25 +567,9 @@ func queryLss[ taskName, func(shard TShard) (err error) { shardID := shard.ShardID() - lss := shard.LSS() - - return lss.WithRLock(func(target, _ *cppbridge.LabelSetStorage) error { - selector, status := target.QuerySelector(convertedMatchers) - switch status { - case cppbridge.LSSQueryStatusMatch: - selectors[shardID] = selector - snapshots[shardID] = lss.GetSnapshot() + selectors[shardID], snapshots[shardID], err = shard.LSS().QuerySelector(shardID, convertedMatchers) - case cppbridge.LSSQueryStatusNoMatch: - - default: - return fmt.Errorf( - "failed to query selector from shard: %d, query status: %d", shardID, status, - ) - } - - return nil - }) + return err }, ) head.Enqueue(tLSSQuerySelector) diff --git a/pp/go/storage/querier/querier_test.go b/pp/go/storage/querier/querier_test.go index d84f084182..3a32e05b68 100644 --- a/pp/go/storage/querier/querier_test.go +++ b/pp/go/storage/querier/querier_test.go @@ -14,10 +14,18 @@ func TestXxx(t *testing.T) { ds := shard.NewDataStorage() wl := &testWal{} sd := shard.NewShard(lss, ds, wl, 0) + id := "test-head-id" + numberOfShards := uint16(2) + + h := head.NewHead( + id, + []*shard.Shard[*testWal]{sd}, + shard.NewPerGoroutineShard[*testWal], + numberOfShards, + nil, + ) - h := head.NewHead([]*shard.Shard[*testWal]{sd}) - - querier.NewQuerier( + q := querier.NewQuerier( h, querier.NewNoOpShardedDeduplicator, 0, @@ -25,11 +33,17 @@ func TestXxx(t *testing.T) { nil, querier.NewMetrics(nil, "test"), ) + _ = q } // testWal test implementation wal. type testWal struct{} +// Close test implementation wal. +func (*testWal) Close() error { + return nil +} + // Commit test implementation wal. func (*testWal) Commit() error { return nil From d0c43ce781c5a7daf08f4319a7b64c1386e355a9 Mon Sep 17 00:00:00 2001 From: Alexandr Yudin Date: Tue, 12 Aug 2025 16:36:23 +0000 Subject: [PATCH 13/96] add PerGoroutineRelabeler --- pp-pkg/storage/appender.go | 84 +++- pp/entrypoint/prometheus_relabeler.cpp | 257 ++++++++++ pp/entrypoint/prometheus_relabeler.h | 161 +++++++ pp/go/cppbridge/entrypoint.go | 276 +++++++++++ pp/go/cppbridge/entrypoint.h | 161 +++++++ pp/go/cppbridge/prometheus_relabeler.go | 238 ++++++++++ pp/go/cppbridge/prometheus_relabeler_test.go | 68 +++ pp/go/storage/appender/appender.go | 341 ++++++++++++++ pp/go/storage/appender/models.go | 187 ++++++++ pp/go/storage/head/head/head.go | 2 + pp/go/storage/head/keeper/keeper.go | 147 ++++-- pp/go/storage/head/manager/manager.go | 79 ++-- pp/go/storage/models.go | 2 +- pp/go/storage/querier/querier.go | 2 +- pp/prometheus/relabeler.h | 472 +++++++++++++++++++ pp/prometheus/stateless_relabeler.h | 214 ++++++++- 16 files changed, 2632 insertions(+), 59 deletions(-) create mode 100644 pp/go/storage/appender/appender.go create mode 100644 pp/go/storage/appender/models.go diff --git a/pp-pkg/storage/appender.go b/pp-pkg/storage/appender.go index e89550a871..6ab74d2d72 100644 --- a/pp-pkg/storage/appender.go +++ b/pp-pkg/storage/appender.go @@ -3,7 +3,12 @@ package storage import ( "context" + "github.com/prometheus/prometheus/model/exemplar" + "github.com/prometheus/prometheus/model/histogram" + "github.com/prometheus/prometheus/model/labels" + "github.com/prometheus/prometheus/model/metadata" "github.com/prometheus/prometheus/pp/go/model" + "github.com/prometheus/prometheus/storage" ) // timeSeriesBatch implementation buffer of [ppstorage.TimeSeriesData]. @@ -21,10 +26,85 @@ func (d *timeSeriesBatch) Destroy() { d.timeSeries = nil } -// TimeSeriesAppender +// TimeSeriesAppender appender for rules, aggregates the [model.TimeSeries] batch and append to head, +// implementation [storage.Appender]. type TimeSeriesAppender struct { ctx context.Context // receiver *Receiver relabelerID string - data *timeSeriesBatch + batch *timeSeriesBatch +} + +// Append adds a sample pair for the given series, implementation [storage.Appender]. +func (a *TimeSeriesAppender) Append( + _ storage.SeriesRef, + l labels.Labels, + t int64, + v float64, +) (storage.SeriesRef, error) { + lsb := model.NewLabelSetBuilder() + l.Range(func(label labels.Label) { + lsb.Add(label.Name, label.Value) + }) + + a.batch.timeSeries = append(a.batch.timeSeries, model.TimeSeries{ + LabelSet: lsb.Build(), + Timestamp: uint64(t), // #nosec G115 // no overflow + Value: v, + }) + return 0, nil +} + +// AppendCTZeroSample do nothing, implementation [storage.Appender]. +func (*TimeSeriesAppender) AppendCTZeroSample( + _ storage.SeriesRef, + _ labels.Labels, + _, _ int64, +) (storage.SeriesRef, error) { + return 0, nil +} + +// AppendExemplar do nothing, implementation [storage.Appender]. +func (*TimeSeriesAppender) AppendExemplar( + _ storage.SeriesRef, + _ labels.Labels, + _ exemplar.Exemplar, +) (storage.SeriesRef, error) { + return 0, nil +} + +// AppendHistogram do nothing, implementation [storage.Appender]. +func (*TimeSeriesAppender) AppendHistogram( + _ storage.SeriesRef, + _ labels.Labels, + _ int64, + _ *histogram.Histogram, + _ *histogram.FloatHistogram, +) (storage.SeriesRef, error) { + return 0, nil +} + +// Commit adds aggregated series to the head, implementation [storage.Appender]. +func (a *TimeSeriesAppender) Commit() error { + if len(a.batch.timeSeries) == 0 { + return nil + } + + // _, err := a.receiver.AppendTimeSeries(a.ctx, a.batch, nil, a.relabelerID, false) + // return err + return nil +} + +// Rollback do nothing, implementation [storage.Appender]. +func (*TimeSeriesAppender) Rollback() error { + return nil +} + +// UpdateMetadata do nothing, implementation [storage.Appender]. +func (*TimeSeriesAppender) UpdateMetadata( + _ storage.SeriesRef, + _ labels.Labels, + _ metadata.Metadata, +) (storage.SeriesRef, error) { + return 0, nil } diff --git a/pp/entrypoint/prometheus_relabeler.cpp b/pp/entrypoint/prometheus_relabeler.cpp index d1a546e78d..538a3c6eea 100644 --- a/pp/entrypoint/prometheus_relabeler.cpp +++ b/pp/entrypoint/prometheus_relabeler.cpp @@ -544,3 +544,260 @@ extern "C" void prompp_prometheus_cache_reset_to(void* args) { static_cast(args)->cache->reset(); } + +// +// PerGoroutineRelabeler +// + +using PerGoroutineRelabeler = PromPP::Prometheus::Relabel::PerGoroutineRelabeler; +using PerGoroutineRelabelerPtr = std::unique_ptr; + +extern "C" void prompp_prometheus_per_goroutine_relabeler_ctor(void* args, void* res) { + struct Arguments { + uint16_t number_of_shards; + uint16_t shard_id; + }; + struct Result { + PerGoroutineRelabelerPtr per_goroutine_relabeler; + }; + + auto* in = static_cast(args); + auto* out = new (res) Result(); + + out->per_goroutine_relabeler = std::make_unique(in->number_of_shards, in->shard_id); +} + +extern "C" void prompp_prometheus_per_goroutine_relabeler_dtor(void* args) { + struct Arguments { + PerGoroutineRelabelerPtr per_goroutine_relabeler; + }; + + static_cast(args)->~Arguments(); +} + +extern "C" void prompp_prometheus_per_goroutine_relabeler_input_relabeling(void* args, void* res) { + struct Arguments { + PromPP::Primitives::Go::SliceView shards_inner_series; + PromPP::Primitives::Go::SliceView shards_relabeled_series; + PromPP::Prometheus::Relabel::RelabelerOptions options; + PerGoroutineRelabelerPtr per_goroutine_relabeler; + PromPP::Prometheus::Relabel::StatelessRelabeler* stateless_relabeler; + HashdexVariant* hashdex; + CachePtr cache; + LssVariantPtr input_lss; + LssVariantPtr target_lss; + }; + struct Result { + uint32_t samples_added{0}; + uint32_t series_added{0}; + uint32_t series_drop{0}; + PromPP::Primitives::Go::Slice error; + bool target_lss_has_reallocations{}; + }; + + auto in = static_cast(args); + auto out = new (res) Result(); + + try { + std::visit( + [in, out](auto& hashdex) { + auto& input_lss = std::get(*in->input_lss); + auto& target_lss = std::get(*in->target_lss); + + const entrypoint::head::ReallocationsDetector reallocation_detector(target_lss); + in->per_goroutine_relabeler->input_relabeling(input_lss, target_lss, *in->cache, hashdex, in->options, *in->stateless_relabeler, *out, + in->shards_inner_series, in->shards_relabeled_series); + target_lss.build_deferred_indexes(); + out->target_lss_has_reallocations = reallocation_detector.has_reallocations(); + }, + *in->hashdex); + } catch (...) { + auto err_stream = PromPP::Primitives::Go::BytesStream(&out->error); + entrypoint::handle_current_exception(err_stream); + } +} + +extern "C" void prompp_prometheus_per_goroutine_relabeler_input_relabeling_from_cache(void* args, void* res) { + struct Arguments { + PromPP::Primitives::Go::SliceView shards_inner_series; + PromPP::Prometheus::Relabel::RelabelerOptions options; + PerGoroutineRelabelerPtr per_goroutine_relabeler; + HashdexVariant* hashdex; + CachePtr cache; + LssVariantPtr input_lss; + LssVariantPtr target_lss; + }; + struct Result { + uint32_t samples_added{0}; + uint32_t series_added{0}; + uint32_t series_drop{0}; + bool ok{}; + PromPP::Primitives::Go::Slice error; + }; + + auto in = static_cast(args); + auto out = new (res) Result(); + + try { + std::visit( + [in, out](auto& hashdex) { + auto& input_lss = std::get(*in->input_lss); + auto& target_lss = std::get(*in->target_lss); + + out->ok = + in->per_goroutine_relabeler->input_relabeling_from_cache(input_lss, target_lss, *in->cache, hashdex, in->options, *out, in->shards_inner_series); + }, + *in->hashdex); + } catch (...) { + auto err_stream = PromPP::Primitives::Go::BytesStream(&out->error); + entrypoint::handle_current_exception(err_stream); + } +} + +extern "C" void prompp_prometheus_per_goroutine_relabeler_input_relabeling_with_stalenans(void* args, void* res) { + struct Arguments { + PromPP::Primitives::Go::SliceView shards_inner_series; + PromPP::Primitives::Go::SliceView shards_relabeled_series; + PromPP::Prometheus::Relabel::RelabelerOptions options; + PerGoroutineRelabelerPtr per_goroutine_relabeler; + PromPP::Prometheus::Relabel::StatelessRelabeler* stateless_relabeler; + HashdexVariant* hashdex; + CachePtr cache; + LssVariantPtr input_lss; + LssVariantPtr target_lss; + StaleNaNsStatePtr state; + PromPP::Primitives::Timestamp def_timestamp; + }; + struct Result { + uint32_t samples_added{0}; + uint32_t series_added{0}; + uint32_t series_drop{0}; + PromPP::Primitives::Go::Slice error; + bool target_lss_has_reallocations{}; + }; + + auto in = static_cast(args); + auto out = new (res) Result(); + + try { + std::visit( + [in, out](auto& hashdex) { + auto& input_lss = std::get(*in->input_lss); + auto& target_lss = std::get(*in->target_lss); + + const entrypoint::head::ReallocationsDetector reallocation_detector(target_lss); + in->per_goroutine_relabeler->input_relabeling_with_stalenans(input_lss, target_lss, *in->cache, hashdex, in->options, *in->stateless_relabeler, *out, + in->shards_inner_series, in->shards_relabeled_series, *in->state, in->def_timestamp); + target_lss.build_deferred_indexes(); + out->target_lss_has_reallocations = reallocation_detector.has_reallocations(); + }, + *in->hashdex); + } catch (...) { + auto err_stream = PromPP::Primitives::Go::BytesStream(&out->error); + entrypoint::handle_current_exception(err_stream); + } +} + +extern "C" void prompp_prometheus_per_goroutine_relabeler_input_relabeling_with_stalenans_from_cache(void* args, void* res) { + struct Arguments { + PromPP::Primitives::Go::SliceView shards_inner_series; + PromPP::Prometheus::Relabel::RelabelerOptions options; + PerGoroutineRelabelerPtr per_goroutine_relabeler; + HashdexVariant* hashdex; + CachePtr cache; + LssVariantPtr input_lss; + LssVariantPtr target_lss; + StaleNaNsStatePtr state; + PromPP::Primitives::Timestamp def_timestamp; + }; + struct Result { + uint32_t samples_added{0}; + uint32_t series_added{0}; + uint32_t series_drop{0}; + bool ok{}; + PromPP::Primitives::Go::Slice error; + }; + + auto in = static_cast(args); + auto out = new (res) Result(); + + try { + std::visit( + [in, out](auto& hashdex) { + auto& input_lss = std::get(*in->input_lss); + auto& target_lss = std::get(*in->target_lss); + + out->ok = in->per_goroutine_relabeler->input_relabeling_with_stalenans_from_cache(input_lss, target_lss, *in->cache, hashdex, in->options, *out, + in->shards_inner_series, *in->state, in->def_timestamp); + }, + *in->hashdex); + } catch (...) { + auto err_stream = PromPP::Primitives::Go::BytesStream(&out->error); + entrypoint::handle_current_exception(err_stream); + } +} + +extern "C" void prompp_prometheus_per_goroutine_relabeler_append_relabeler_series(void* args, void* res) { + struct Arguments { + PromPP::Primitives::Go::SliceView shards_inner_series; + PromPP::Primitives::Go::SliceView shards_relabeled_series; + PromPP::Primitives::Go::SliceView shards_relabeler_state_update; + PerGoroutineRelabelerPtr per_goroutine_relabeler; + LssVariantPtr target_lss; + }; + struct Result { + PromPP::Primitives::Go::Slice error; + bool target_lss_has_reallocations{}; + }; + + const auto in = static_cast(args); + const auto out = new (res) Result(); + + try { + auto& lss = std::get(*in->target_lss); + const entrypoint::head::ReallocationsDetector reallocation_detector(lss); + + for (size_t id = 0; id != in->shards_relabeled_series.size(); ++id) { + if (in->shards_relabeled_series[id] == nullptr || in->shards_relabeled_series[id]->size() == 0) { + continue; + } + + in->per_goroutine_relabeler->append_relabeler_series(lss, in->shards_inner_series[id], in->shards_relabeled_series[id], + in->shards_relabeler_state_update[id]); + } + + lss.build_deferred_indexes(); + out->target_lss_has_reallocations = reallocation_detector.has_reallocations(); + } catch (...) { + auto err_stream = PromPP::Primitives::Go::BytesStream(&out->error); + entrypoint::handle_current_exception(err_stream); + } +} + +extern "C" void prompp_prometheus_per_goroutine_relabeler_update_relabeler_state(void* args, void* res) { + struct Arguments { + PromPP::Primitives::Go::SliceView shards_relabeler_state_update; + PerGoroutineRelabelerPtr per_goroutine_relabeler; + CachePtr cache; + uint16_t relabeled_shard_id; + }; + struct Result { + PromPP::Primitives::Go::Slice error; + }; + + const auto* in = static_cast(args); + + try { + for (size_t id = 0; id != in->shards_relabeler_state_update.size(); ++id) { + if (in->shards_relabeler_state_update[id] == nullptr || in->shards_relabeler_state_update[id]->size() == 0) { + continue; + } + + in->per_goroutine_relabeler->update_relabeler_state(*in->cache, in->shards_relabeler_state_update[id], id); + } + } catch (...) { + auto* out = new (res) Result(); + auto err_stream = PromPP::Primitives::Go::BytesStream(&out->error); + entrypoint::handle_current_exception(err_stream); + } +} diff --git a/pp/entrypoint/prometheus_relabeler.h b/pp/entrypoint/prometheus_relabeler.h index e0f3cc84bf..3e71838121 100644 --- a/pp/entrypoint/prometheus_relabeler.h +++ b/pp/entrypoint/prometheus_relabeler.h @@ -407,6 +407,167 @@ void prompp_prometheus_cache_allocated_memory(void* args, void* res); */ void prompp_prometheus_cache_reset_to(void* args); +// +// PerGoroutineRelabeler +// + +/** + * @brief Construct a new PerGoroutineRelabeler. + * + * @param args { + * number_of_shards uint16 // total shards count; + * shard_id uint16 // current shard id; + * } + * + * @param res { + * per_goroutine_relabeler uintptr // pointer to constructed PerGoroutineRelabeler; + * } + */ +void prompp_prometheus_per_goroutine_relabeler_ctor(void* args, void* res); + +/** + * @brief Destroy PerGoroutineRelabeler. + * + * @param args { + * per_goroutine_relabeler uintptr // pointer of PerGoroutineRelabeler; + * } + */ +void prompp_prometheus_per_goroutine_relabeler_dtor(void* args); + +/** + * @brief relabeling incomig hashdex(first stage). + * + * @param args { + * shards_inner_series []*InnerSeries // go slice with InnerSeries; + * shards_relabeled_series []*RelabeledSeries // go slice with RelabeledSeries; + * options RelabelerOptions // object RelabelerOptions; + * per_goroutine_relabeler uintptr // pointer to constructed per goroutine relabeler; + * stateless_relabeler uintptr // pointer to constructed stateless relabeler; + * hashdex uintptr // pointer to filled hashdex; + * cache uintptr // pointer to constructed Cache; + * input_lss uintptr // pointer to constructed input label sets; + * target_lss uintptr // pointer to constructed target label sets; + * } + * + * @param res { + * samples_added uint32 // number of added samples; + * series_added uint32 // number of added series; + * series_drop uint32 // number of dropped series; + * error []byte // error string if thrown; + * target_lss_has_reallocations bool // true if target lss has reallocations + * } + */ +void prompp_prometheus_per_goroutine_relabeler_input_relabeling(void* args, void* res); + +/** + * @brief relabeling incomig hashdex(first stage) from cache. + * + * @param args { + * shards_inner_series []*InnerSeries // go slice with InnerSeries; + * options RelabelerOptions // object RelabelerOptions; + * per_goroutine_relabeler uintptr // pointer to constructed per goroutine relabeler; + * hashdex uintptr // pointer to filled hashdex; + * cache uintptr // pointer to constructed Cache; + * input_lss uintptr // pointer to constructed input label sets; + * target_lss uintptr // pointer to constructed target label sets; + * } + * + * @param res { + * samples_added uint32 // number of added samples; + * series_added uint32 // number of added series; + * series_drop uint32 // number of dropped series; + * ok bool // true if all label set find in cache; + * error []byte // error string if thrown; + * } + */ +void prompp_prometheus_per_goroutine_relabeler_input_relabeling_from_cache(void* args, void* res); + +/** + * @brief relabeling incomig hashdex(first stage) with state stalenans. + * + * @param args { + * shards_inner_series []*InnerSeries // go slice with InnerSeries; + * shards_relabeled_series []*RelabeledSeries // go slice with RelabeledSeries; + * options RelabelerOptions // object RelabelerOptions; + * per_goroutine_relabeler uintptr // pointer to constructed per goroutine relabeler; + * stateless_relabeler uintptr // pointer to constructed stateless relabeler; + * hashdex uintptr // pointer to filled hashdex; + * cache uintptr // pointer to constructed Cache; + * input_lss uintptr // pointer to constructed input label sets; + * target_lss uintptr // pointer to constructed target label sets; + * state uintptr // pointer to source state + * def_timestamp int64 // timestamp for metrics and StaleNaNs + * } + * + * @param res { + * samples_added uint32 // number of added samples; + * series_added uint32 // number of added series; + * series_drop uint32 // number of dropped series; + * error []byte // error string if thrown; + * target_lss_has_reallocations bool // true if target lss has reallocations + * } + */ +void prompp_prometheus_per_goroutine_relabeler_input_relabeling_with_stalenans(void* args, void* res); + +/** + * @brief relabeling incomig hashdex(first stage) from cache with state stalenans. + * + * @param args { + * shards_inner_series []*InnerSeries // go slice with InnerSeries; + * options RelabelerOptions // object RelabelerOptions; + * per_goroutine_relabeler uintptr // pointer to constructed per goroutine relabeler; + * hashdex uintptr // pointer to filled hashdex; + * cache uintptr // pointer to constructed Cache; + * input_lss uintptr // pointer to constructed input label sets; + * target_lss uintptr // pointer to constructed target label sets; + * state uintptr // pointer to source state + * def_timestamp int64 // timestamp for metrics and StaleNaNs + * } + * + * @param res { + * samples_added uint32 // number of added samples; + * series_added uint32 // number of added series; + * series_drop uint32 // number of dropped series; + * ok bool // true if all label set find in cache; + * error []byte // error string if thrown; + * } + */ +void prompp_prometheus_per_goroutine_relabeler_input_relabeling_with_stalenans_from_cache(void* args, void* res); + +/** + * @brief add relabeled ls to lss, add to result and add to cache update(second stage). + * + * @param args { + * shards_inner_series []*InnerSeries // go InnerSeries per source shard; + * shards_relabeled_series []*RelabeledSeries // go RelabeledSeries per source shard; + * shards_relabeler_state_update []*RelabelerStateUpdate // pointer to RelabelerStateUpdate per source shard; + * per_goroutine_relabeler uintptr // pointer to constructed per goroutine relabeler; + * target_lss uintptr // pointer to constructed label sets; + * } + * + * @param res { + * error []byte // error string if thrown + * target_lss_has_reallocations bool // true if target lss has reallocations + * } + */ +void prompp_prometheus_per_goroutine_relabeler_append_relabeler_series(void* args, void* res); + +/** + * @brief add to cache relabled data(third stage). + * + * @param args { + * shards_relabeler_state_update []*RelabelerStateUpdate // pointer to RelabelerStateUpdate per source shard; + * per_shard_relabeler uintptr // pointer to constructed per goroutine relabeler; + * cache uintptr // pointer to constructed Cache; + * relabeled_shard_id uint16 // relabeled shard id; + * } + * + * @param res { + * error []byte // error string if thrown; + * } + */ +void prompp_prometheus_per_goroutine_relabeler_update_relabeler_state(void* args, void* res); + #ifdef __cplusplus } // extern "C" #endif diff --git a/pp/go/cppbridge/entrypoint.go b/pp/go/cppbridge/entrypoint.go index a8b74013f8..4c85712f6a 100644 --- a/pp/go/cppbridge/entrypoint.go +++ b/pp/go/cppbridge/entrypoint.go @@ -2934,3 +2934,279 @@ func LabelSetBytesWithLabels(lss uintptr, labelSetID uint32, bytes []byte, names func LabelSetBytesWithoutLabels(lss uintptr, labelSetID uint32, bytes []byte, names ...string) []byte { return labelSetBytesWithFilteredNames(C.prompp_label_set_bytes_without_labels, lss, labelSetID, bytes, names...) } + +// +// PerGoroutineRelabeler +// + +// prometheusPerGoroutineRelabelerCtor wrapper for constructor C-PerGoroutineRelabeler. +func prometheusPerGoroutineRelabelerCtor( + numberOfShards, shardID uint16, +) uintptr { + args := struct { + numberOfShards uint16 + shardID uint16 + }{numberOfShards, shardID} + var res struct { + perGoroutineRelabeler uintptr + } + + testGC() + fastcgo.UnsafeCall2( + C.prompp_prometheus_per_goroutine_relabeler_ctor, + uintptr(unsafe.Pointer(&args)), + uintptr(unsafe.Pointer(&res)), + ) + + return res.perGoroutineRelabeler +} + +// prometheusPerGoroutineRelabelerDtor wrapper for destructor C-PerGoroutineRelabeler. +func prometheusPerGoroutineRelabelerDtor(perGoroutineRelabeler uintptr) { + args := struct { + perGoroutineRelabeler uintptr + }{perGoroutineRelabeler} + + testGC() + fastcgo.UnsafeCall1( + C.prompp_prometheus_per_goroutine_relabeler_dtor, + uintptr(unsafe.Pointer(&args)), + ) +} + +// prometheusPerGoroutineRelabelerInputRelabeling wrapper for relabeling incoming hashdex(first stage). +func prometheusPerGoroutineRelabelerInputRelabeling( + perGoroutineRelabeler, statelessRelabeler, inputLss, targetLss, cache, hashdex uintptr, + options RelabelerOptions, + shardsInnerSeries []*InnerSeries, + shardsRelabeledSeries []*RelabeledSeries, +) (stats RelabelerStats, exception []byte, targetLssHasReallocations bool) { + args := struct { + shardsInnerSeries []*InnerSeries + shardsRelabeledSeries []*RelabeledSeries + options RelabelerOptions + perGoroutineRelabeler uintptr + statelessRelabeler uintptr + hashdex uintptr + cache uintptr + inputLss uintptr + targetLss uintptr + }{ + shardsInnerSeries, + shardsRelabeledSeries, + options, + perGoroutineRelabeler, + statelessRelabeler, + hashdex, + cache, + inputLss, + targetLss, + } + var res struct { + RelabelerStats + exception []byte + targetLssHasReallocations bool + } + start := time.Now().UnixNano() + testGC() + fastcgo.UnsafeCall2( + C.prompp_prometheus_per_goroutine_relabeler_input_relabeling, + uintptr(unsafe.Pointer(&args)), + uintptr(unsafe.Pointer(&res)), + ) + inputRelabelerInputRelabelingSum.Add(float64(time.Now().UnixNano() - start)) + inputRelabelerInputRelabelingCount.Inc() + + return res.RelabelerStats, res.exception, res.targetLssHasReallocations +} + +// prometheusPerGoroutineRelabelerInputRelabelingFromCache wrapper for relabeling +// incoming hashdex(first stage) from cache. +func prometheusPerGoroutineRelabelerInputRelabelingFromCache( + perGoroutineRelabeler, inputLss, targetLss, cache, hashdex uintptr, + options RelabelerOptions, + shardsInnerSeries []*InnerSeries, +) (stats RelabelerStats, exception []byte, ok bool) { + args := struct { + shardsInnerSeries []*InnerSeries + options RelabelerOptions + perGoroutineRelabeler uintptr + hashdex uintptr + cache uintptr + inputLss uintptr + targetLss uintptr + }{shardsInnerSeries, options, perGoroutineRelabeler, hashdex, cache, inputLss, targetLss} + var res struct { + RelabelerStats + ok bool + exception []byte + } + start := time.Now().UnixNano() + testGC() + fastcgo.UnsafeCall2( + C.prompp_prometheus_per_goroutine_relabeler_input_relabeling_from_cache, + uintptr(unsafe.Pointer(&args)), + uintptr(unsafe.Pointer(&res)), + ) + inputRelabelerInputRelabelingFromCacheSum.Add(float64(time.Now().UnixNano() - start)) + inputRelabelerInputRelabelingFromCacheCount.Inc() + + return res.RelabelerStats, res.exception, res.ok +} + +// prometheusPerGoroutineRelabelerInputRelabelingWithStalenans wrapper for relabeling incoming +// hashdex(first stage) with state stalenans. +func prometheusPerGoroutineRelabelerInputRelabelingWithStalenans( + perGoroutineRelabeler, statelessRelabeler, inputLss, targetLss, cache, hashdex, sourceState uintptr, + defTimestamp int64, + options RelabelerOptions, + shardsInnerSeries []*InnerSeries, + shardsRelabeledSeries []*RelabeledSeries, +) (stats RelabelerStats, exception []byte, targetLssHasReallocations bool) { + args := struct { + shardsInnerSeries []*InnerSeries + shardsRelabeledSeries []*RelabeledSeries + options RelabelerOptions + perGoroutineRelabeler uintptr + statelessRelabeler uintptr + hashdex uintptr + cache uintptr + inputLss uintptr + targetLss uintptr + state uintptr + defTimestamp int64 + }{ + shardsInnerSeries, + shardsRelabeledSeries, + options, + perGoroutineRelabeler, + statelessRelabeler, + hashdex, + cache, + inputLss, + targetLss, + sourceState, + defTimestamp, + } + var res struct { + RelabelerStats + exception []byte + targetLssHasReallocations bool + } + start := time.Now().UnixNano() + testGC() + fastcgo.UnsafeCall2( + C.prompp_prometheus_per_goroutine_relabeler_input_relabeling_with_stalenans, + uintptr(unsafe.Pointer(&args)), + uintptr(unsafe.Pointer(&res)), + ) + inputRelabelerRelabelingWithStalenansSum.Add(float64(time.Now().UnixNano() - start)) + inputRelabelerRelabelingWithStalenansCount.Inc() + + return res.RelabelerStats, res.exception, res.targetLssHasReallocations +} + +// prometheusPerGoroutineRelabelerInputRelabelingWithStalenansFromCache wrapper for relabeling incoming from cache +// hashdex(first stage) with state stalenans. +func prometheusPerGoroutineRelabelerInputRelabelingWithStalenansFromCache( + perGoroutineRelabeler, inputLss, targetLss, cache, hashdex, sourceState uintptr, + defTimestamp int64, + options RelabelerOptions, + shardsInnerSeries []*InnerSeries, +) (stats RelabelerStats, exception []byte, targetLssHasReallocations bool) { + args := struct { + shardsInnerSeries []*InnerSeries + options RelabelerOptions + perGoroutineRelabeler uintptr + hashdex uintptr + cache uintptr + inputLss uintptr + targetLss uintptr + state uintptr + defTimestamp int64 + }{ + shardsInnerSeries, + options, + perGoroutineRelabeler, + hashdex, + cache, + inputLss, + targetLss, + sourceState, + defTimestamp, + } + var res struct { + RelabelerStats + ok bool + exception []byte + } + start := time.Now().UnixNano() + testGC() + fastcgo.UnsafeCall2( + C.prompp_prometheus_per_goroutine_relabeler_input_relabeling_with_stalenans_from_cache, + uintptr(unsafe.Pointer(&args)), + uintptr(unsafe.Pointer(&res)), + ) + inputRelabelerRelabelingWithStalenansFromCacheSum.Add(float64(time.Now().UnixNano() - start)) + inputRelabelerRelabelingWithStalenansFromCacheCount.Inc() + + return res.RelabelerStats, res.exception, res.ok +} + +// prometheusPerGoroutineRelabelerAppendRelabelerSeries wrapper for add relabeled ls to lss, +// add to result and add to cache update(second stage). +func prometheusPerGoroutineRelabelerAppendRelabelerSeries( + perGoroutineRelabeler, targetLss uintptr, + shardsInnerSeries []*InnerSeries, + shardsRelabeledSeries []*RelabeledSeries, + shardsRelabelerStateUpdate []*RelabelerStateUpdate, +) (exception []byte, targetLssHasReallocations bool) { + args := struct { + shardsInnerSeries []*InnerSeries + shardsRelabeledSeries []*RelabeledSeries + shardsRelabelerStateUpdate []*RelabelerStateUpdate + perGoroutineRelabeler uintptr + targetLss uintptr + }{shardsInnerSeries, shardsRelabeledSeries, shardsRelabelerStateUpdate, perGoroutineRelabeler, targetLss} + var res struct { + exception []byte + targetLssHasReallocations bool + } + start := time.Now().UnixNano() + testGC() + fastcgo.UnsafeCall2( + C.prompp_prometheus_per_goroutine_relabeler_append_relabeler_series, + uintptr(unsafe.Pointer(&args)), + uintptr(unsafe.Pointer(&res)), + ) + inputRelabelerAppendRelabelerSeriesSum.Add(float64(time.Now().UnixNano() - start)) + inputRelabelerAppendRelabelerSeriesCount.Inc() + + return res.exception, res.targetLssHasReallocations +} + +// prometheusPerGoroutineRelabelerUpdateRelabelerState wrapper for add to cache relabled data(third stage). +func prometheusPerGoroutineRelabelerUpdateRelabelerState( + shardsRelabelerStateUpdate []*RelabelerStateUpdate, + perGoroutineRelabeler, cache uintptr, +) []byte { + args := struct { + relabelerStateUpdates []*RelabelerStateUpdate + perGoroutineRelabeler uintptr + cache uintptr + }{shardsRelabelerStateUpdate, perGoroutineRelabeler, cache} + var res struct { + exception []byte + } + start := time.Now().UnixNano() + testGC() + fastcgo.UnsafeCall2( + C.prompp_prometheus_per_goroutine_relabeler_update_relabeler_state, + uintptr(unsafe.Pointer(&args)), + uintptr(unsafe.Pointer(&res)), + ) + inputRelabelerUpdateRelabelerStateSum.Add(float64(time.Now().UnixNano() - start)) + inputRelabelerUpdateRelabelerStateCount.Inc() + + return res.exception +} diff --git a/pp/go/cppbridge/entrypoint.h b/pp/go/cppbridge/entrypoint.h index 3ae2d626fe..7f9f2a2507 100755 --- a/pp/go/cppbridge/entrypoint.h +++ b/pp/go/cppbridge/entrypoint.h @@ -1085,6 +1085,167 @@ void prompp_prometheus_cache_allocated_memory(void* args, void* res); */ void prompp_prometheus_cache_reset_to(void* args); +// +// PerGoroutineRelabeler +// + +/** + * @brief Construct a new PerGoroutineRelabeler. + * + * @param args { + * number_of_shards uint16 // total shards count; + * shard_id uint16 // current shard id; + * } + * + * @param res { + * per_goroutine_relabeler uintptr // pointer to constructed PerGoroutineRelabeler; + * } + */ +void prompp_prometheus_per_goroutine_relabeler_ctor(void* args, void* res); + +/** + * @brief Destroy PerGoroutineRelabeler. + * + * @param args { + * per_goroutine_relabeler uintptr // pointer of PerGoroutineRelabeler; + * } + */ +void prompp_prometheus_per_goroutine_relabeler_dtor(void* args); + +/** + * @brief relabeling incomig hashdex(first stage). + * + * @param args { + * shards_inner_series []*InnerSeries // go slice with InnerSeries; + * shards_relabeled_series []*RelabeledSeries // go slice with RelabeledSeries; + * options RelabelerOptions // object RelabelerOptions; + * per_goroutine_relabeler uintptr // pointer to constructed per goroutine relabeler; + * stateless_relabeler uintptr // pointer to constructed stateless relabeler; + * hashdex uintptr // pointer to filled hashdex; + * cache uintptr // pointer to constructed Cache; + * input_lss uintptr // pointer to constructed input label sets; + * target_lss uintptr // pointer to constructed target label sets; + * } + * + * @param res { + * samples_added uint32 // number of added samples; + * series_added uint32 // number of added series; + * series_drop uint32 // number of dropped series; + * error []byte // error string if thrown; + * target_lss_has_reallocations bool // true if target lss has reallocations + * } + */ +void prompp_prometheus_per_goroutine_relabeler_input_relabeling(void* args, void* res); + +/** + * @brief relabeling incomig hashdex(first stage) from cache. + * + * @param args { + * shards_inner_series []*InnerSeries // go slice with InnerSeries; + * options RelabelerOptions // object RelabelerOptions; + * per_goroutine_relabeler uintptr // pointer to constructed per goroutine relabeler; + * hashdex uintptr // pointer to filled hashdex; + * cache uintptr // pointer to constructed Cache; + * input_lss uintptr // pointer to constructed input label sets; + * target_lss uintptr // pointer to constructed target label sets; + * } + * + * @param res { + * samples_added uint32 // number of added samples; + * series_added uint32 // number of added series; + * series_drop uint32 // number of dropped series; + * ok bool // true if all label set find in cache; + * error []byte // error string if thrown; + * } + */ +void prompp_prometheus_per_goroutine_relabeler_input_relabeling_from_cache(void* args, void* res); + +/** + * @brief relabeling incomig hashdex(first stage) with state stalenans. + * + * @param args { + * shards_inner_series []*InnerSeries // go slice with InnerSeries; + * shards_relabeled_series []*RelabeledSeries // go slice with RelabeledSeries; + * options RelabelerOptions // object RelabelerOptions; + * per_goroutine_relabeler uintptr // pointer to constructed per goroutine relabeler; + * stateless_relabeler uintptr // pointer to constructed stateless relabeler; + * hashdex uintptr // pointer to filled hashdex; + * cache uintptr // pointer to constructed Cache; + * input_lss uintptr // pointer to constructed input label sets; + * target_lss uintptr // pointer to constructed target label sets; + * state uintptr // pointer to source state + * def_timestamp int64 // timestamp for metrics and StaleNaNs + * } + * + * @param res { + * samples_added uint32 // number of added samples; + * series_added uint32 // number of added series; + * series_drop uint32 // number of dropped series; + * error []byte // error string if thrown; + * target_lss_has_reallocations bool // true if target lss has reallocations + * } + */ +void prompp_prometheus_per_goroutine_relabeler_input_relabeling_with_stalenans(void* args, void* res); + +/** + * @brief relabeling incomig hashdex(first stage) from cache with state stalenans. + * + * @param args { + * shards_inner_series []*InnerSeries // go slice with InnerSeries; + * options RelabelerOptions // object RelabelerOptions; + * per_goroutine_relabeler uintptr // pointer to constructed per goroutine relabeler; + * hashdex uintptr // pointer to filled hashdex; + * cache uintptr // pointer to constructed Cache; + * input_lss uintptr // pointer to constructed input label sets; + * target_lss uintptr // pointer to constructed target label sets; + * state uintptr // pointer to source state + * def_timestamp int64 // timestamp for metrics and StaleNaNs + * } + * + * @param res { + * samples_added uint32 // number of added samples; + * series_added uint32 // number of added series; + * series_drop uint32 // number of dropped series; + * ok bool // true if all label set find in cache; + * error []byte // error string if thrown; + * } + */ +void prompp_prometheus_per_goroutine_relabeler_input_relabeling_with_stalenans_from_cache(void* args, void* res); + +/** + * @brief add relabeled ls to lss, add to result and add to cache update(second stage). + * + * @param args { + * shards_inner_series []*InnerSeries // go InnerSeries per source shard; + * shards_relabeled_series []*RelabeledSeries // go RelabeledSeries per source shard; + * shards_relabeler_state_update []*RelabelerStateUpdate // pointer to RelabelerStateUpdate per source shard; + * per_goroutine_relabeler uintptr // pointer to constructed per goroutine relabeler; + * target_lss uintptr // pointer to constructed label sets; + * } + * + * @param res { + * error []byte // error string if thrown + * target_lss_has_reallocations bool // true if target lss has reallocations + * } + */ +void prompp_prometheus_per_goroutine_relabeler_append_relabeler_series(void* args, void* res); + +/** + * @brief add to cache relabled data(third stage). + * + * @param args { + * shards_relabeler_state_update []*RelabelerStateUpdate // pointer to RelabelerStateUpdate per source shard; + * per_shard_relabeler uintptr // pointer to constructed per goroutine relabeler; + * cache uintptr // pointer to constructed Cache; + * relabeled_shard_id uint16 // relabeled shard id; + * } + * + * @param res { + * error []byte // error string if thrown; + * } + */ +void prompp_prometheus_per_goroutine_relabeler_update_relabeler_state(void* args, void* res); + #ifdef __cplusplus } // extern "C" #endif diff --git a/pp/go/cppbridge/prometheus_relabeler.go b/pp/go/cppbridge/prometheus_relabeler.go index 1572b445b7..daed303587 100644 --- a/pp/go/cppbridge/prometheus_relabeler.go +++ b/pp/go/cppbridge/prometheus_relabeler.go @@ -1119,3 +1119,241 @@ func (s *State) resetStaleNansStates(numberOfShards uint16, equaledGeneration bo ) } } + +// +// PerGoroutineRelabeler +// + +// PerGoroutineRelabeler go wrapper for C-PerGoroutineRelabeler, relabeler for shard goroutines. +type PerGoroutineRelabeler struct { + cptr uintptr + gcDestroyDetector *uint64 +} + +// NewPerGoroutineRelabeler init new [PerGoroutineRelabeler]. +func NewPerGoroutineRelabeler( + numberOfShards, shardID uint16, +) *PerGoroutineRelabeler { + pgr := &PerGoroutineRelabeler{ + cptr: prometheusPerGoroutineRelabelerCtor(numberOfShards, shardID), + gcDestroyDetector: &gcDestroyDetector, + } + runtime.SetFinalizer(pgr, func(r *PerGoroutineRelabeler) { + prometheusPerShardRelabelerDtor(r.cptr) + }) + + return pgr +} + +// AppendRelabelerSeries add relabeled ls to lss, add to result and add to cache update(second stage). +func (pgr *PerGoroutineRelabeler) AppendRelabelerSeries( + ctx context.Context, + targetLss *LabelSetStorage, + shardsInnerSeries []*InnerSeries, + shardsRelabeledSeries []*RelabeledSeries, + shardsRelabelerStateUpdate []*RelabelerStateUpdate, +) (bool, error) { + if ctx.Err() != nil { + return false, ctx.Err() + } + + exception, hasReallocations := prometheusPerGoroutineRelabelerAppendRelabelerSeries( + pgr.cptr, + targetLss.Pointer(), + shardsInnerSeries, + shardsRelabeledSeries, + shardsRelabelerStateUpdate, + ) + + return hasReallocations, handleException(exception) +} + +// InputRelabeling relabeling incoming hashdex(first stage). +func (pgr *PerGoroutineRelabeler) InputRelabeling( + ctx context.Context, + statelessRelabeler *StatelessRelabeler, + inputLss *LabelSetStorage, + targetLss *LabelSetStorage, + cache *Cache, + options RelabelerOptions, + shardedData ShardedData, + shardsInnerSeries []*InnerSeries, + shardsRelabeledSeries []*RelabeledSeries, +) (RelabelerStats, bool, error) { + if ctx.Err() != nil { + return RelabelerStats{}, false, ctx.Err() + } + + cptrContainer, ok := shardedData.(cptrable) + if !ok { + return RelabelerStats{}, false, ErrMustImplementCptrable + } + + stats, exception, hasReallocations := prometheusPerGoroutineRelabelerInputRelabeling( + pgr.cptr, + statelessRelabeler.Pointer(), + inputLss.Pointer(), + targetLss.Pointer(), + cache.cPointer, + cptrContainer.cptr(), + options, + shardsInnerSeries, + shardsRelabeledSeries, + ) + runtime.KeepAlive(pgr) + runtime.KeepAlive(statelessRelabeler) + runtime.KeepAlive(inputLss) + runtime.KeepAlive(targetLss) + runtime.KeepAlive(cache) + runtime.KeepAlive(cptrContainer) + + return stats, hasReallocations, handleException(exception) +} + +// InputRelabelingFromCache relabeling incoming hashdex(first stage) from cache. +func (pgr *PerGoroutineRelabeler) InputRelabelingFromCache( + ctx context.Context, + inputLss *LabelSetStorage, + targetLss *LabelSetStorage, + cache *Cache, + options RelabelerOptions, + shardedData ShardedData, + shardsInnerSeries []*InnerSeries, +) (RelabelerStats, bool, error) { + if ctx.Err() != nil { + return RelabelerStats{}, false, ctx.Err() + } + + cptrContainer, ok := shardedData.(cptrable) + if !ok { + return RelabelerStats{}, false, ErrMustImplementCptrable + } + + stats, exception, ok := prometheusPerGoroutineRelabelerInputRelabelingFromCache( + pgr.cptr, + inputLss.Pointer(), + targetLss.Pointer(), + cache.cPointer, + cptrContainer.cptr(), + options, + shardsInnerSeries, + ) + runtime.KeepAlive(pgr) + runtime.KeepAlive(inputLss) + runtime.KeepAlive(targetLss) + runtime.KeepAlive(cache) + runtime.KeepAlive(cptrContainer) + + return stats, ok, handleException(exception) +} + +// InputRelabelingWithStalenans relabeling incoming hashdex(first stage) with state stalenans. +func (pgr *PerGoroutineRelabeler) InputRelabelingWithStalenans( + ctx context.Context, + statelessRelabeler *StatelessRelabeler, + inputLss *LabelSetStorage, + targetLss *LabelSetStorage, + cache *Cache, + options RelabelerOptions, + staleNansState *StaleNansState, + defTimestamp int64, + shardedData ShardedData, + shardsInnerSeries []*InnerSeries, + shardsRelabeledSeries []*RelabeledSeries, +) (RelabelerStats, bool, error) { + if ctx.Err() != nil { + return RelabelerStats{}, false, ctx.Err() + } + + cptrContainer, ok := shardedData.(cptrable) + if !ok { + return RelabelerStats{}, false, ErrMustImplementCptrable + } + stats, exception, hasReallocations := prometheusPerGoroutineRelabelerInputRelabelingWithStalenans( + pgr.cptr, + statelessRelabeler.Pointer(), + inputLss.Pointer(), + targetLss.Pointer(), + cache.cPointer, + cptrContainer.cptr(), + staleNansState.state, + defTimestamp, + options, + shardsInnerSeries, + shardsRelabeledSeries, + ) + runtime.KeepAlive(pgr) + runtime.KeepAlive(statelessRelabeler) + runtime.KeepAlive(inputLss) + runtime.KeepAlive(targetLss) + runtime.KeepAlive(cache) + runtime.KeepAlive(cptrContainer) + runtime.KeepAlive(staleNansState) + + return stats, hasReallocations, handleException(exception) +} + +// InputRelabelingWithStalenansFromCache relabeling incoming hashdex(first stage) from cache with state stalenans. +func (pgr *PerGoroutineRelabeler) InputRelabelingWithStalenansFromCache( + ctx context.Context, + inputLss *LabelSetStorage, + targetLss *LabelSetStorage, + cache *Cache, + options RelabelerOptions, + staleNansState *StaleNansState, + defTimestamp int64, + shardedData ShardedData, + shardsInnerSeries []*InnerSeries, +) (RelabelerStats, bool, error) { + if ctx.Err() != nil { + return RelabelerStats{}, false, ctx.Err() + } + + cptrContainer, ok := shardedData.(cptrable) + if !ok { + return RelabelerStats{}, false, ErrMustImplementCptrable + } + + stats, exception, ok := prometheusPerGoroutineRelabelerInputRelabelingWithStalenansFromCache( + pgr.cptr, + inputLss.Pointer(), + targetLss.Pointer(), + cache.cPointer, + cptrContainer.cptr(), + staleNansState.state, + defTimestamp, + options, + shardsInnerSeries, + ) + + runtime.KeepAlive(pgr) + runtime.KeepAlive(inputLss) + runtime.KeepAlive(targetLss) + runtime.KeepAlive(cache) + runtime.KeepAlive(cptrContainer) + runtime.KeepAlive(staleNansState) + + return stats, ok, handleException(exception) +} + +// UpdateRelabelerState add to cache relabled data(third stage). +func (pgr *PerGoroutineRelabeler) UpdateRelabelerState( + ctx context.Context, + cache *Cache, + shardsRelabelerStateUpdate []*RelabelerStateUpdate, +) error { + if ctx.Err() != nil { + return ctx.Err() + } + + exception := prometheusPerGoroutineRelabelerUpdateRelabelerState( + shardsRelabelerStateUpdate, + pgr.cptr, + cache.cPointer, + ) + + runtime.KeepAlive(pgr) + runtime.KeepAlive(cache) + + return handleException(exception) +} diff --git a/pp/go/cppbridge/prometheus_relabeler_test.go b/pp/go/cppbridge/prometheus_relabeler_test.go index b195ded8f3..3e4d51c72b 100644 --- a/pp/go/cppbridge/prometheus_relabeler_test.go +++ b/pp/go/cppbridge/prometheus_relabeler_test.go @@ -600,3 +600,71 @@ func (s *RelabelerSuite) TestToHash_EmptyConfig() { s.Require().Equal(xxhash.Sum64String("0"+a.String()), cppbridge.ToHash(rCfgs)) } + +func (s *RelabelerSuite) TestInputPerGoroutineRelabeler() { + wr := prompb.WriteRequest{ + Timeseries: []prompb.TimeSeries{ + { + Labels: []prompb.Label{ + {Name: "__name__", Value: "value"}, + {Name: "job", Value: "abc"}, + {Name: "instance", Value: "value1"}, + }, + Samples: []prompb.Sample{ + {Value: 0.1, Timestamp: time.Now().UnixMilli()}, + }, + }, + { + Labels: []prompb.Label{ + {Name: "__name__", Value: "value"}, + {Name: "instance", Value: "value1"}, + }, + Samples: []prompb.Sample{ + {Value: 0.1, Timestamp: time.Now().UnixMilli()}, + }, + }, + }, + } + data, err := wr.Marshal() + s.Require().NoError(err) + + rCfgs := []*cppbridge.RelabelConfig{ + { + SourceLabels: []string{"job"}, + Regex: "abc", + Action: cppbridge.Keep, + }, + } + + inputLss := cppbridge.NewLssStorage() + targetLss := cppbridge.NewQueryableLssStorage() + + statelessRelabeler, err := cppbridge.NewStatelessRelabeler(rCfgs) + s.Require().NoError(err) + + var numberOfShards uint16 = 1 + pgr := cppbridge.NewPerGoroutineRelabeler(numberOfShards, 0) + + hlimits := cppbridge.DefaultWALHashdexLimits() + h, err := cppbridge.NewWALSnappyProtobufHashdex(snappy.Encode(nil, data), hlimits) + s.Require().NoError(err) + + shardsInnerSeries := cppbridge.NewShardsInnerSeries(numberOfShards) + shardsRelabeledSeries := cppbridge.NewShardsRelabeledSeries(numberOfShards) + cache := cppbridge.NewCache() + + stats, hasReallocations, err := pgr.InputRelabeling( + s.baseCtx, + statelessRelabeler, + inputLss, + targetLss, + cache, + s.options, + h, + shardsInnerSeries, + shardsRelabeledSeries, + ) + s.Require().NoError(err) + s.Equal(cppbridge.RelabelerStats{1, 1, 1}, stats) + s.True(hasReallocations) +} diff --git a/pp/go/storage/appender/appender.go b/pp/go/storage/appender/appender.go new file mode 100644 index 0000000000..1987cf0bdb --- /dev/null +++ b/pp/go/storage/appender/appender.go @@ -0,0 +1,341 @@ +package appender + +import ( + "context" + "fmt" + "sync/atomic" + + "github.com/prometheus/prometheus/pp/go/cppbridge" + "github.com/prometheus/prometheus/pp/go/relabeler" + "github.com/prometheus/prometheus/pp/go/storage" + "github.com/prometheus/prometheus/pp/go/storage/logger" +) + +const ( + // DSAppendInnerSeries name of task. + DSAppendInnerSeries = "data_storage_append_inner_series" + + // LSSInputRelabeling name of task. + LSSInputRelabeling = "lss_input_relabeling" + + // WalWrite name of task. + WalWrite = "wal_write" + + // WalCommit name of task. + WalCommit = "wal_commit" +) + +// +// GenericTask +// + +// GenericTask the minimum required task [Generic] implementation. +type GenericTask interface { + // Wait for the task to complete on all shards. + Wait() error +} + +// +// DataStorage +// + +// DataStorage the minimum required [DataStorage] implementation. +type DataStorage interface { + // TODO +} + +// +// LSS +// + +// LSS the minimum required [LSS] implementation. +type LSS interface { + // WithLock calls fn on raws [cppbridge.LabelSetStorage] with write lock. + WithLock(fn func(target, input *cppbridge.LabelSetStorage) error) error + + // WithRLock calls fn on raws [cppbridge.LabelSetStorage] with read lock. + WithRLock(fn func(target, input *cppbridge.LabelSetStorage) error) error +} + +// +// Shard +// + +// Shard the minimum required head [Shard] implementation. +type Shard[TDataStorage DataStorage, TLSS LSS] interface { + // DataStorage returns shard [DataStorage]. + DataStorage() TDataStorage + + // LSS returns shard labelset storage [LSS]. + LSS() TLSS + + // ShardID returns the shard ID. + ShardID() uint16 +} + +// +// Head +// + +// Head the minimum required [Head] implementation. +type Head[ + TGenericTask GenericTask, + TDataStorage DataStorage, + TLSS LSS, + TShard Shard[TDataStorage, TLSS], +] interface { + // CreateTask create a task for operations on the [Head] shards. + CreateTask(taskName string, shardFn func(shard TShard) error) TGenericTask + + // Enqueue the task to be executed on shards [Head]. + Enqueue(t TGenericTask) + + // NumberOfShards returns current number of shards in to [Head]. + NumberOfShards() uint16 +} + +type Appender[ + TGenericTask GenericTask, + TDataStorage DataStorage, + TLSS LSS, + TShard Shard[TDataStorage, TLSS], + THead Head[TGenericTask, TDataStorage, TLSS, TShard], +] struct { + head THead +} + +// Append incoming data to head. +func (a *Appender[TGenericTask, TDataStorage, TLSS, TShard, THead]) Append( + ctx context.Context, + incomingData *storage.IncomingData, + incomingState *cppbridge.State, + relabelerID string, + commitToWal bool, +) ([][]*cppbridge.InnerSeries, cppbridge.RelabelerStats, error) { + // rd, state, err := h.resolveRelabelersData(incomingState, relabelerID) + // if err != nil { + // return nil, cppbridge.RelabelerStats{}, err + // } + + shardedInnerSeries := NewShardedInnerSeries(a.head.NumberOfShards()) + shardedRelabeledSeries := NewShardedRelabeledSeries(a.head.NumberOfShards()) + + stats, err := a.inputRelabelingStage( + ctx, + state, + rd, + NewDestructibleIncomingData(incomingData, int(a.head.NumberOfShards())), + shardedInnerSeries, + shardedRelabeledSeries, + ) + if err != nil { + // reset msr.rotateWG on error + return nil, stats, fmt.Errorf("failed input relabeling stage: %w", err) + } + + if !shardedRelabeledSeries.IsEmpty() { + shardedStateUpdates := NewShardedStateUpdates(a.head.NumberOfShards()) + if err = h.appendRelabelerSeriesStage( + ctx, + rd, + shardedInnerSeries, + shardedRelabeledSeries, + shardedStateUpdates, + ); err != nil { + return nil, stats, fmt.Errorf("failed append relabeler series stage: %w", err) + } + + if err = h.updateRelabelerStateStage( + ctx, + state, + rd, + shardedStateUpdates, + ); err != nil { + return nil, stats, fmt.Errorf("failed update relabeler stage: %w", err) + } + } + + tw := relabeler.NewTaskWaiter(2) + + tAppend := h.CreateTask( + DSAppendInnerSeries, + func(shard relabeler.Shard) error { + shard.DataStorageLock() + shard.DataStorage().AppendInnerSeriesSlice(shardedInnerSeries.DataByShard(shard.ShardID())) + shard.DataStorageUnlock() + + return nil + }, + relabeler.ForDataStorageTask, + ) + h.Enqueue(tAppend) + + var atomiclimitExhausted uint32 + tWalWrite := h.CreateTask( + WalWrite, + func(shard relabeler.Shard) error { + shard.LSSLock() + limitExhausted, errWrite := shard.Wal().Write(shardedInnerSeries.DataByShard(shard.ShardID())) + shard.LSSUnlock() + if errWrite != nil { + return fmt.Errorf("shard %d: %w", shard.ShardID(), errWrite) + } + + if limitExhausted { + atomic.AddUint32(&atomiclimitExhausted, 1) + } + + return nil + }, + relabeler.ForLSSTask, + ) + h.Enqueue(tWalWrite) + + tw.Add(tAppend) + tw.Add(tWalWrite) + + if err := tw.Wait(); err != nil { + logger.Errorf("failed to write wal: %v", err) + } + + if commitToWal || atomiclimitExhausted > 0 { + t := h.CreateTask( + WalCommit, + func(shard relabeler.Shard) error { + shard.LSSLock() + defer shard.LSSUnlock() + + return shard.Wal().Commit() + }, + relabeler.ForLSSTask, + ) + h.Enqueue(t) + + if err := t.Wait(); err != nil { + logger.Errorf("failed to commit wal: %v", err) + } + } + + return shardedInnerSeries.Data(), stats, nil +} + +// inputRelabelingStage first stage - relabeling. +func (a *Appender[TGenericTask, TDataStorage, TLSS, TShard, THead]) inputRelabelingStage( + ctx context.Context, + state *cppbridge.State, + rd *RelabelerData, + incomingData *DestructibleIncomingData, + shardedInnerSeries *ShardedInnerSeries, + shardedRelabeledSeries *ShardedRelabeledSeries, +) (cppbridge.RelabelerStats, error) { + stats := make([]cppbridge.RelabelerStats, a.head.NumberOfShards()) + t := a.head.CreateTask( + LSSInputRelabeling, + func(shard TShard) error { + var ( + lss = shard.LSS() + shardID = shard.ShardID() + hasReallocations bool + ok bool + ) + + if err := lss.WithRLock(func(target, input *cppbridge.LabelSetStorage) error { + var rErr error + if state.TrackStaleness() { + stats[shardID], ok, rErr = rd.InputRelabelerByShard( + shardID, + ).InputRelabelingWithStalenansFromCache( + ctx, + input, + target, + state.CacheByShard(shardID), + state.RelabelerOptions(), + state.StaleNansStateByShard(shardID), + state.DefTimestamp(), + incomingData.ShardedData(), + shardedInnerSeries.DataBySourceShard(shardID), + ) + } else { + stats[shardID], ok, rErr = rd.InputRelabelerByShard(shardID).InputRelabelingFromCache( + ctx, + input, + target, + state.CacheByShard(shardID), + state.RelabelerOptions(), + incomingData.ShardedData(), + shardedInnerSeries.DataBySourceShard(shardID), + ) + } + + return rErr + }); err != nil { + incomingData.Destroy() + return fmt.Errorf("shard %d: %w", shardID, err) + } + + if ok { + incomingData.Destroy() + return nil + } + + shard.LSSLock() + defer shard.LSSUnlock() + rstats := cppbridge.RelabelerStats{} + + if state.TrackStaleness() { + rstats, hasReallocations, err = rd.InputRelabelerByShard(shardID).InputRelabelingWithStalenans( + ctx, + shard.LSS().Input(), + shard.LSS().Target(), + state.CacheByShard(shardID), + state.RelabelerOptions(), + state.StaleNansStateByShard(shardID), + state.DefTimestamp(), + incomingData.ShardedData(), + shardedInnerSeries.DataBySourceShard(shardID), + shardedRelabeledSeries.DataByShard(shardID), + ) + } else { + rstats, hasReallocations, err = rd.InputRelabelerByShard(shardID).InputRelabeling( + ctx, + shard.LSS().Input(), + shard.LSS().Target(), + state.CacheByShard(shardID), + state.RelabelerOptions(), + incomingData.ShardedData(), + shardedInnerSeries.DataBySourceShard(shardID), + shardedRelabeledSeries.DataByShard(shardID), + ) + } + + incomingData.Destroy() + if err != nil { + return fmt.Errorf("shard %d: %w", shardID, err) + } + + stats[shardID].SamplesAdded += rstats.SamplesAdded + stats[shardID].SeriesAdded += rstats.SeriesAdded + stats[shardID].SeriesDrop += rstats.SeriesDrop + + if hasReallocations { + shard.LSS().ResetSnapshot() + } + + return nil + }, + ) + a.head.Enqueue(t) + + resStats := cppbridge.RelabelerStats{} + if err := t.Wait(); err != nil { + return resStats, err + } + + for _, s := range stats { + resStats.SamplesAdded += s.SamplesAdded + resStats.SeriesAdded += s.SeriesAdded + resStats.SeriesDrop += s.SeriesDrop + } + + return resStats, nil +} diff --git a/pp/go/storage/appender/models.go b/pp/go/storage/appender/models.go new file mode 100644 index 0000000000..d70fe1c461 --- /dev/null +++ b/pp/go/storage/appender/models.go @@ -0,0 +1,187 @@ +package appender + +import ( + "sync/atomic" + + "github.com/prometheus/prometheus/pp/go/cppbridge" + "github.com/prometheus/prometheus/pp/go/storage" +) + +// +// ShardedInnerSeries +// + +// ShardedInnerSeries conteiner for InnerSeries for each shard. +type ShardedInnerSeries struct { + // id slice - shard id, data[shard_id] - amount of data = x2 numberOfShards + data [][]*cppbridge.InnerSeries +} + +// NewShardedInnerSeries init new ShardedInnerSeries. +func NewShardedInnerSeries(numberOfShards uint16) *ShardedInnerSeries { + // id slice - shard id + data := make([][]*cppbridge.InnerSeries, numberOfShards) + for i := range data { + // amount of data = x2 numberOfShards + data[i] = cppbridge.NewShardsInnerSeries(numberOfShards) + } + + return &ShardedInnerSeries{ + data: data, + } +} + +// Data return slice of elemets for each shard. +func (sis *ShardedInnerSeries) Data() [][]*cppbridge.InnerSeries { + return sis.data +} + +// DataByShard return slice with the results per shard. +func (sis *ShardedInnerSeries) DataByShard(shardID uint16) []*cppbridge.InnerSeries { + return sis.data[shardID] +} + +// DataBySourceShard return slice with the results per source shard. +func (sis *ShardedInnerSeries) DataBySourceShard(sourceShardID uint16) []*cppbridge.InnerSeries { + data := make([]*cppbridge.InnerSeries, len(sis.data)) + for i, iss := range sis.data { + data[i] = iss[sourceShardID] + } + + return data +} + +// +// ShardedRelabeledSeries +// + +// ShardedRelabeledSeries conteiner for RelabeledSeries for each shard. +type ShardedRelabeledSeries struct { + // id slice - shard id, data[shard_id] id slice - source shard id + // data[shard_id][source_shard_id] - amount of data = numberOfShards + data [][]*cppbridge.RelabeledSeries +} + +// NewShardedRelabeledSeries init new ShardedRelabeledSeries. +func NewShardedRelabeledSeries(numberOfShards uint16) *ShardedRelabeledSeries { + // id slice - shard id + data := make([][]*cppbridge.RelabeledSeries, numberOfShards) + for i := range data { + // data[shard_id] id slice - source shard id + // data[shard_id][source_shard_id] - amount of data = numberOfShards + data[i] = cppbridge.NewShardsRelabeledSeries(numberOfShards) + } + return &ShardedRelabeledSeries{ + data: data, + } +} + +// DataByShard return slice with the results per shard. +func (srs *ShardedRelabeledSeries) DataByShard(shardID uint16) []*cppbridge.RelabeledSeries { + return srs.data[shardID] +} + +// DataBySourceShard return slice with the results per source shard. +func (srs *ShardedRelabeledSeries) DataBySourceShard(sourceShardID uint16) ([]*cppbridge.RelabeledSeries, bool) { + ok := false + data := make([]*cppbridge.RelabeledSeries, len(srs.data)) + for i, rss := range srs.data { + data[i] = rss[sourceShardID] + if data[i].Size() != 0 { + ok = true + } + } + + return data, ok +} + +// IsEmpty return false if there are no elements. +func (srs *ShardedRelabeledSeries) IsEmpty() bool { + for _, rss := range srs.data { + for _, rs := range rss { + if rs.Size() != 0 { + return false + } + } + } + + return true +} + +// +// ShardedStateUpdates +// + +// ShardedStateUpdates conteiner for RelabelerStateUpdate for each shard. +type ShardedStateUpdates struct { + // id slice - shard id, data[shard_id] id slice - source shard id + // data[shard_id][source_shard_id] - amount of data = numberOfShards + data [][]*cppbridge.RelabelerStateUpdate +} + +// NewShardedStateUpdates init new ShardedStateUpdates. +func NewShardedStateUpdates(numberOfShards uint16) *ShardedStateUpdates { + // id slice - shard id + data := make([][]*cppbridge.RelabelerStateUpdate, numberOfShards) + for i := range data { + // data[shard_id] id slice - source shard id + // data[shard_id][source_shard_id] - amount of data = numberOfShards + data[i] = cppbridge.NewShardsRelabelerStateUpdate(numberOfShards) + } + return &ShardedStateUpdates{ + data: data, + } +} + +// DataByShard return slice with the results per shard. +func (sru *ShardedStateUpdates) DataByShard(shardID uint16) []*cppbridge.RelabelerStateUpdate { + return sru.data[shardID] +} + +// DataBySourceShard return slice with the results per source shard. +func (sru *ShardedStateUpdates) DataBySourceShard(sourceShardID uint16) ([]*cppbridge.RelabelerStateUpdate, bool) { + ok := false + data := make([]*cppbridge.RelabelerStateUpdate, len(sru.data)) + for i, rsu := range sru.data { + data[i] = rsu[sourceShardID] + if !data[i].IsEmpty() { + ok = true + } + } + + return data, ok +} + +// +// DestructibleIncomingData +// + +// DestructibleIncomingData wrapeer over [storage.IncomingData] with detroy-counter. +type DestructibleIncomingData struct { + data *storage.IncomingData + destructCount atomic.Int64 +} + +// NewDestructibleIncomingData init new [DestructibleIncomingData]. +func NewDestructibleIncomingData(data *storage.IncomingData, destructCount int) *DestructibleIncomingData { + d := &DestructibleIncomingData{ + data: data, + } + d.destructCount.Store(int64(destructCount)) + + return d +} + +// ShardedData return hashdex. +func (d *DestructibleIncomingData) ShardedData() cppbridge.ShardedData { + return d.data.ShardedData() +} + +// Destroy decrement count or destroy IncomingData. +func (d *DestructibleIncomingData) Destroy() { + if d.destructCount.Add(-1) != 0 { + return + } + + d.data.Destroy() +} diff --git a/pp/go/storage/head/head/head.go b/pp/go/storage/head/head/head.go index 33616d3b44..a151ff17ea 100644 --- a/pp/go/storage/head/head/head.go +++ b/pp/go/storage/head/head/head.go @@ -294,6 +294,8 @@ func calculateHeadConcurrency(numberOfShards uint16) int64 { return int64(defaultNumberOfWorkers+ExtraWorkers) * int64(numberOfShards) } +// TODO Flush CommitToWal ? + // TODO Who? // // getSortedStats returns sorted statistics for the [Head]. // func getSortedStats(stats map[string]uint64, limit int) []storage.HeadStat { diff --git a/pp/go/storage/head/keeper/keeper.go b/pp/go/storage/head/keeper/keeper.go index 7c55267844..0f01510350 100644 --- a/pp/go/storage/head/keeper/keeper.go +++ b/pp/go/storage/head/keeper/keeper.go @@ -2,11 +2,11 @@ package keeper import ( "sync" + "sync/atomic" "time" "github.com/jonboulle/clockwork" "github.com/prometheus/client_golang/prometheus" - "github.com/prometheus/prometheus/pp/go/relabeler" "github.com/prometheus/prometheus/pp/go/relabeler/querier" "github.com/prometheus/prometheus/pp/go/storage/logger" "github.com/prometheus/prometheus/pp/go/util" @@ -18,6 +18,16 @@ import ( // // IndexWriter() IndexWriter // } +const ( + writeRetryTimeout = 5 * time.Minute + maxAddIter uint32 = 5 +) + +const ( + // BlockWrite name of task. + BlockWrite = "block_write" +) + // HeadBlockWriter writes block on disk from [Head]. type HeadBlockWriter[TBlock any] interface { Write(block TBlock) error @@ -27,15 +37,83 @@ type WriteNotifier interface { NotifyWritten() } +// GenericTask the minimum required task [Generic] implementation. +type GenericTask interface { + // Wait for the task to complete on all shards. + Wait() error +} + +// +// DataStorage +// + +// DataStorage the minimum required [DataStorage] implementation. +type DataStorage interface { + // TODO +} + +// +// LSS +// + +// LSS the minimum required [LSS] implementation. +type LSS interface { + // TODO +} + +// +// Shard +// + +// Shard the minimum required head [Shard] implementation. +type Shard[TDataStorage DataStorage, TLSS LSS] interface { + // DataStorage returns shard [DataStorage]. + DataStorage() TDataStorage + + // LSS returns shard labelset storage [LSS]. + LSS() TLSS +} + +// +// Head +// + +// Head the minimum required [Head] implementation. +type Head[ + TGenericTask GenericTask, + TDataStorage DataStorage, + TLSS LSS, + TShard Shard[TDataStorage, TLSS], +] interface { + // CreateTask create a task for operations on the [Head] shards. + CreateTask(taskName string, shardFn func(shard TShard) error) TGenericTask + + // Enqueue the task to be executed on shards [Head]. + Enqueue(t TGenericTask) + + // ID returns id [Head]. + ID() string + + // String serialize as string. + String() string +} + type HeadBlockBuilder[TBlock any] func() TBlock -type Keeper[TBlock any] struct { +type Keeper[ + TGenericTask GenericTask, + TDataStorage DataStorage, + TLSS LSS, + TShard Shard[TDataStorage, TLSS], + THead Head[TGenericTask, TDataStorage, TLSS, TShard], + TBlock any, +] struct { hbWriter HeadBlockWriter[TBlock] headBlockBuilder HeadBlockBuilder[TBlock] writeNotifier WriteNotifier mtx sync.Mutex - heads []relabeler.Head + heads []THead headRetentionTimeout time.Duration writeTimer clockwork.Timer @@ -49,7 +127,18 @@ type Keeper[TBlock any] struct { querierMetrics *querier.Metrics } -func (k *Keeper[TBlock]) write() bool { +func (k *Keeper[TGenericTask, TDataStorage, TLSS, TShard, THead, TBlock]) Add(head THead) { + k.mtx.Lock() + k.heads = append(k.heads, head) + logger.Infof("QUERYABLE STORAGE: head %s added", head.String()) + k.mtx.Unlock() + + if atomic.AddUint32(&k.addCount, 1) < maxAddIter { + k.writeTimer.Reset(k.writeTimeout) + } +} + +func (k *Keeper[TGenericTask, TDataStorage, TLSS, TShard, THead, TBlock]) write() bool { k.mtx.Lock() lenHeads := len(k.heads) if lenHeads == 0 { @@ -57,7 +146,7 @@ func (k *Keeper[TBlock]) write() bool { k.mtx.Unlock() return true } - heads := make([]relabeler.Head, lenHeads) + heads := make([]THead, lenHeads) copy(heads, k.heads) k.mtx.Unlock() @@ -71,27 +160,27 @@ func (k *Keeper[TBlock]) write() bool { shouldNotify = true continue } - if err := head.Flush(); err != nil { - logger.Errorf("QUERYABLE STORAGE: failed to flush head %s: %s", head.String(), err.Error()) - successful = false - continue - } - if err := head.Rotate(); err != nil { - logger.Errorf("QUERYABLE STORAGE: failed to rotate head %s: %s", head.String(), err.Error()) - successful = false - continue - } + // TODO + // if err := head.Flush(); err != nil { + // logger.Errorf("QUERYABLE STORAGE: failed to flush head %s: %s", head.String(), err.Error()) + // successful = false + // continue + // } + // if err := head.Rotate(); err != nil { + // logger.Errorf("QUERYABLE STORAGE: failed to rotate head %s: %s", head.String(), err.Error()) + // successful = false + // continue + // } tBlockWrite := head.CreateTask( - relabeler.BlockWrite, - func(shard relabeler.Shard) error { - shard.LSSLock() - defer shard.LSSUnlock() + BlockWrite, + func(shard TShard) error { + // shard.LSSLock() + // defer shard.LSSUnlock() bl := k.headBlockBuilder() // relabeler.NewBlock(shard.LSS().Raw(), shard.DataStorage().Raw()) return k.hbWriter.Write(bl) }, - relabeler.ForLSSTask, ) head.Enqueue(tBlockWrite) if err := tBlockWrite.Wait(); err != nil { @@ -122,12 +211,15 @@ func (k *Keeper[TBlock]) write() bool { return successful } -func (k *Keeper[TBlock]) headIsOutdated(head relabeler.Head) bool { - headMaxTimestampMs := head.Status(1).HeadStats.MaxTime - return k.clock.Now().Sub(time.Unix(headMaxTimestampMs/1000, 0)) > k.maxRetentionDuration +func (k *Keeper[TGenericTask, TDataStorage, TLSS, TShard, THead, TBlock]) headIsOutdated(head THead) bool { + // TODO + // headMaxTimestampMs := head.Status(1).HeadStats.MaxTime + // return k.clock.Now().Sub(time.Unix(headMaxTimestampMs/1000, 0)) > k.maxRetentionDuration + + return false } -func (k *Keeper[TBlock]) shrink(persisted ...string) { +func (k *Keeper[TGenericTask, TDataStorage, TLSS, TShard, THead, TBlock]) shrink(persisted ...string) { k.mtx.Lock() defer k.mtx.Unlock() @@ -136,11 +228,12 @@ func (k *Keeper[TBlock]) shrink(persisted ...string) { persistedMap[headID] = struct{}{} } - var heads []relabeler.Head + heads := make([]THead, len(k.heads)) for _, head := range k.heads { if _, ok := persistedMap[head.ID()]; ok { - _ = head.Close() - _ = head.Discard() + // TODO + // _ = head.Close() + // _ = head.Discard() logger.Infof("QUERYABLE STORAGE: head %s persisted, closed and discarded", head.String()) continue } diff --git a/pp/go/storage/head/manager/manager.go b/pp/go/storage/head/manager/manager.go index bbdb28fdb6..65c81d63ae 100644 --- a/pp/go/storage/head/manager/manager.go +++ b/pp/go/storage/head/manager/manager.go @@ -7,7 +7,6 @@ import ( "github.com/prometheus/client_golang/prometheus" "github.com/prometheus/prometheus/pp/go/relabeler/config" - "github.com/prometheus/prometheus/pp/go/storage" "github.com/prometheus/prometheus/pp/go/storage/logger" "github.com/prometheus/prometheus/pp/go/util" ) @@ -18,10 +17,18 @@ type Timer interface { Stop() } -type ActiveHeadContainer interface { - Get(ctx context.Context) (storage.Head, error) - Replace(ctx context.Context, newHead storage.Head) error - With(ctx context.Context, fn func(h storage.Head) error) error +type Head interface { + // TODO ? +} + +type ActiveHeadContainer[THead Head] interface { + Get(ctx context.Context) (THead, error) + Replace(ctx context.Context, newHead THead) error + With(ctx context.Context, fn func(h THead) error) error +} + +type Keeper[THead Head] interface { + Add(head THead) } // type ActiveHeadContainer[T any] interface { @@ -32,16 +39,17 @@ type ActiveHeadContainer interface { // var _ ActiveHeadContainer[testHead] = (*container.Weighted[testHead, *testHead])(nil) -// HeadBuilder builder for the [storage.Head]. -type HeadBuilder interface { +// HeadBuilder builder for the [Head]. +type HeadBuilder[THead Head] interface { // inputRelabelerConfigs []*config.InputRelabelerConfig, - BuildWithConfig(numberOfShards uint16) (storage.Head, error) + BuildWithConfig(numberOfShards uint16) (THead, error) } -type Manager struct { +type Manager[THead Head] struct { // - activeHead ActiveHeadContainer - headBuilder HeadBuilder + activeHead ActiveHeadContainer[THead] + headBuilder HeadBuilder[THead] + keeper Keeper[THead] rotateTimer Timer commitTimer Timer mergeTimer Timer @@ -55,7 +63,7 @@ type Manager struct { } // ApplyConfig update config. -func (m *Manager) ApplyConfig( +func (m *Manager[THead]) ApplyConfig( ctx context.Context, inputRelabelerConfigs []*config.InputRelabelerConfig, numberOfShards uint16, @@ -69,35 +77,40 @@ func (m *Manager) ApplyConfig( } // MergeOutOfOrderChunks merge chunks with out of order data chunks. -func (m *Manager) MergeOutOfOrderChunks(ctx context.Context) error { - return m.activeHead.With(ctx, func(h storage.Head) error { - h.MergeOutOfOrderChunks() +func (m *Manager[THead]) MergeOutOfOrderChunks(ctx context.Context) error { + // TODO ? + // return m.activeHead.With(ctx, func(h storage.Head) error { + // h.MergeOutOfOrderChunks() - return nil - }) + // return nil + // }) + + return nil } // Run starts processing of the [Manager]. // TODO implementation. -func (m *Manager) Run(ctx context.Context) error { +func (m *Manager[THead]) Run(ctx context.Context) error { go m.loop(ctx) return nil } // Shutdown safe shutdown [Manager]. -func (m *Manager) Shutdown(ctx context.Context) error { +func (m *Manager[THead]) Shutdown(ctx context.Context) error { return nil } // commitToWal commit the accumulated data into the wal. -func (m *Manager) commitToWal(ctx context.Context) error { - return m.activeHead.With(ctx, func(h storage.Head) error { - return h.CommitToWal() - }) +func (m *Manager[THead]) commitToWal(ctx context.Context) error { + // TODO ? + // return m.activeHead.With(ctx, func(h storage.Head) error { + // return h.CommitToWal() + // }) + return nil } // TODO implementation. -func (m *Manager) loop(ctx context.Context) { +func (m *Manager[THead]) loop(ctx context.Context) { defer m.closer.Done() for { @@ -132,7 +145,7 @@ func (m *Manager) loop(ctx context.Context) { } } -func (m *Manager) rotate(ctx context.Context) error { +func (m *Manager[THead]) rotate(ctx context.Context) error { newHead, err := m.headBuilder.BuildWithConfig(m.numberOfShards) if err != nil { return fmt.Errorf("failed to build a new head: %w", err) @@ -143,10 +156,22 @@ func (m *Manager) rotate(ctx context.Context) error { return fmt.Errorf("getting active head failed: %w", err) } - newHead.CopySeriesFrom(oldHead) + // TODO + // newHead.CopySeriesFrom(oldHead) - // storgae.Add(oldHead) + m.keeper.Add(oldHead) // TODO if replace error? return m.activeHead.Replace(ctx, newHead) } + +// WithAppendableHead +// TODO implementation. +func (m *Manager[THead]) WithAppendableHead(ctx context.Context, fn func(h THead) error) error { + return m.activeHead.With(ctx, fn) +} + +// RangeQueriableHeads +// TODO implementation. +func (m *Manager[THead]) RangeQueriableHeads() { +} diff --git a/pp/go/storage/models.go b/pp/go/storage/models.go index 5dc72e0b37..abb1bb97f2 100644 --- a/pp/go/storage/models.go +++ b/pp/go/storage/models.go @@ -17,7 +17,7 @@ func (i *IncomingData) ShardedData() cppbridge.ShardedData { return i.Hashdex } -// Destroy increment or destroy IncomingData. +// Destroy IncomingData. func (i *IncomingData) Destroy() { i.Hashdex = nil if i.Data != nil { diff --git a/pp/go/storage/querier/querier.go b/pp/go/storage/querier/querier.go index ecd296c084..50bff850df 100644 --- a/pp/go/storage/querier/querier.go +++ b/pp/go/storage/querier/querier.go @@ -54,7 +54,7 @@ type deduplicatorCtor func(numberOfShards uint16) Deduplicator // GenericTask // -// GenericTask the minimum required GenericTask implementation. +// GenericTask the minimum required task [Generic] implementation. type GenericTask interface { // Wait for the task to complete on all shards. Wait() error diff --git a/pp/prometheus/relabeler.h b/pp/prometheus/relabeler.h index a57a08438e..e3c50e12d8 100644 --- a/pp/prometheus/relabeler.h +++ b/pp/prometheus/relabeler.h @@ -900,4 +900,476 @@ class PerShardRelabeler { PROMPP_ALWAYS_INLINE ~PerShardRelabeler() = default; }; +// +// PerGoroutineRelabeler +// + +// PerGoroutineRelabeler stateful relabeler for shard goroutines. +class PerGoroutineRelabeler { + std::stringstream buf_; + PromPP::Primitives::LabelsBuilderStateMap builder_state_; + PromPP::Primitives::TimeseriesSemiview timeseries_buf_; + uint16_t number_of_shards_; + uint16_t shard_id_; + + public: + // PerShardRelabeler constructor. + PROMPP_ALWAYS_INLINE PerGoroutineRelabeler(const uint16_t number_of_shards, const uint16_t shard_id) + : number_of_shards_(number_of_shards), shard_id_(shard_id) {} + + private: + // calculate_samples counts the number of samples excluding stale_nan. + PROMPP_ALWAYS_INLINE size_t calculate_samples(const BareBones::Vector& samples) noexcept { + size_t samples_count{0}; + for (const auto smpl : samples) { + if (is_stale_nan(smpl.value())) { + continue; + } + + ++samples_count; + } + + return samples_count; + } + + // inject_target_labels add labels from target to builder. + template + PROMPP_ALWAYS_INLINE bool inject_target_labels(LabelsBuilder& target_builder, const RelabelerOptions& options) { + if (options.target_labels.empty()) { + return false; + } + + bool changed{false}; + + if (options.honor_labels) { + for (const auto& [lname, lvalue] : options.target_labels) { + if (target_builder.contains(static_cast(lname))) [[unlikely]] { + continue; + } + + target_builder.set(static_cast(lname), static_cast(lvalue)); + changed = true; + } + + return changed; + } + + std::vector conflicting_exposed_labels; + for (const auto& [lname, lvalue] : options.target_labels) { + PromPP::Primitives::Label existing_label = target_builder.extract(static_cast(lname)); + if (!existing_label.second.empty()) [[likely]] { + conflicting_exposed_labels.emplace_back(std::move(existing_label)); + } + + // It is now safe to set the target label. + target_builder.set(static_cast(lname), static_cast(lvalue)); + changed = true; + } + + // resolve conflict + if (!conflicting_exposed_labels.empty()) { + resolve_conflicting_exposed_labels(target_builder, conflicting_exposed_labels); + } + + return changed; + } + + template + PROMPP_ALWAYS_INLINE bool input_relabeling_from_cache_internal(InputLSS& input_lss, + TargetLSS& target_lss, + Cache& cache, + const Hashdex& hashdex, + const RelabelerOptions& options, + Stats& stats, + PromPP::Primitives::Go::SliceView& shards_inner_series, + StNaNsState& stale_nan_state, + PromPP::Primitives::Timestamp def_timestamp) { + assert(number_of_shards_ > 0); + + size_t n = std::min(static_cast(hashdex.size()), static_cast((hashdex.size() * (1 - cache.part_of_drops()) * 1.1) / number_of_shards_)); + for (auto i = 0; i < number_of_shards_; ++i) { + shards_inner_series[i]->reserve(n); + } + + size_t samples_count{0}; + + for (const auto& item : hashdex) { + if ((item.hash() % number_of_shards_) != shard_id_) { + continue; + } + + timeseries_buf_.clear(); + item.read(timeseries_buf_); + Cache::CheckResult check_result = cache.check(input_lss, target_lss, timeseries_buf_.label_set(), item.hash()); + switch (check_result.status) { + case Cache::CheckResult::kNotFound: { + return false; + }; + case Cache::CheckResult::kKeep: { + auto& samples = timeseries_buf_.samples(); + bool all_samples_reseted_to_scrape_ts = resolve_timestamps(def_timestamp, samples, options); + if (options.track_timestamps_staleness || all_samples_reseted_to_scrape_ts) { + stale_nan_state.add_target(check_result.ls_id); + } + for (const PromPP::Primitives::Sample& sample : samples) { + shards_inner_series[shard_id_]->emplace_back(sample, check_result.ls_id); + } + + break; + } + case Cache::CheckResult::kRelabel: { + auto& samples = timeseries_buf_.samples(); + bool all_samples_reseted_to_scrape_ts = resolve_timestamps(def_timestamp, samples, options); + if (options.track_timestamps_staleness || all_samples_reseted_to_scrape_ts) { + stale_nan_state.add_input(check_result.source_ls_id); + } + for (const PromPP::Primitives::Sample& sample : samples) { + shards_inner_series[check_result.shard_id]->emplace_back(sample, check_result.ls_id); + } + + break; + } + default: + continue; + } + + stats.samples_added += static_cast(timeseries_buf_.samples().size()); + + if (options.metric_limits == nullptr) { + continue; + } + + samples_count += calculate_samples(timeseries_buf_.samples()); + if (options.metric_limits->samples_limit_exceeded(samples_count)) { + break; + } + } + + PromPP::Primitives::Sample smpl{def_timestamp, kStaleNan}; + stale_nan_state.swap( + [&](uint32_t ls_id) { + if (auto res = cache.check_input(ls_id); res.status == Cache::CheckResult::kRelabel) { + shards_inner_series[res.shard_id]->emplace_back(smpl, res.ls_id); + } + }, + [&](uint32_t ls_id) { + if (auto res = cache.check_target(ls_id); res.status == Cache::CheckResult::kKeep) { + shards_inner_series[shard_id_]->emplace_back(smpl, res.ls_id); + } + }); + + return true; + } + + template + PROMPP_ALWAYS_INLINE void input_relabeling_internal(InputLSS& input_lss, + TargetLSS& target_lss, + Cache& cache, + const Hashdex& hashdex, + const RelabelerOptions& options, + const StatelessRelabeler& stateless_relabeler, + Stats& stats, + PromPP::Primitives::Go::SliceView& shards_inner_series, + PromPP::Primitives::Go::SliceView& shards_relabeled_series, + StNaNsState& stale_nan_state, + PromPP::Primitives::Timestamp def_timestamp) { + assert(number_of_shards_ > 0); + + size_t n = std::min(static_cast(hashdex.size()), static_cast((hashdex.size() * (1 - cache.part_of_drops()) * 1.1) / number_of_shards_)); + for (auto i = 0; i < number_of_shards_; ++i) { + if (shards_inner_series[i]->size() >= n) { + continue; + } + + shards_inner_series[i]->reserve(n); + } + + PromPP::Primitives::LabelsBuilder builder{builder_state_}; + size_t samples_count{0}; + + for (auto it = skip_shard_inner_series(hashdex, shards_inner_series[shard_id_]->size()); it != hashdex.end(); ++it) { + if ((it->hash() % number_of_shards_) != shard_id_) { + continue; + } + + timeseries_buf_.clear(); + it->read(timeseries_buf_); + + Cache::CheckResult check_result = cache.check(input_lss, target_lss, timeseries_buf_.label_set(), it->hash()); + switch (check_result.status) { + case Cache::CheckResult::kNotFound: { + builder.reset(timeseries_buf_.label_set()); + auto rstatus = relabel(options, stateless_relabeler, builder); + switch (rstatus) { + case rsDrop: { + cache.add_drop(input_lss.find_or_emplace(timeseries_buf_.label_set(), it->hash())); + ++stats.series_drop; + + continue; + } + case rsInvalid: { + cache.add_drop(input_lss.find_or_emplace(timeseries_buf_.label_set(), it->hash())); + ++stats.series_drop; + + continue; + } + case rsKeep: { + auto ls_id = target_lss.find_or_emplace(timeseries_buf_.label_set(), it->hash()); + cache.add_keep(ls_id); + auto& samples = timeseries_buf_.samples(); + bool all_samples_reseted_to_scrape_ts = resolve_timestamps(def_timestamp, samples, options); + if (options.track_timestamps_staleness || all_samples_reseted_to_scrape_ts) { + stale_nan_state.add_target(ls_id); + } + + for (const PromPP::Primitives::Sample& sample : samples) { + shards_inner_series[shard_id_]->emplace_back(sample, ls_id); + } + + ++stats.series_added; + + break; + } + case rsRelabel: { + auto ls_id = input_lss.find_or_emplace(timeseries_buf_.label_set(), it->hash()); + PromPP::Primitives::LabelSet new_label_set = builder.label_set(); + size_t new_hash = hash_value(new_label_set); + size_t new_shard_id = new_hash % number_of_shards_; + auto& samples = timeseries_buf_.samples(); + bool all_samples_reseted_to_scrape_ts = resolve_timestamps(def_timestamp, samples, options); + if (options.track_timestamps_staleness || all_samples_reseted_to_scrape_ts) { + stale_nan_state.add_input(ls_id); + } + + shards_relabeled_series[new_shard_id]->emplace_back(new_label_set, samples, new_hash, ls_id); + ++stats.series_added; + + break; + } + } + + break; + } + case Cache::CheckResult::kKeep: { + auto& samples = timeseries_buf_.samples(); + bool all_samples_reseted_to_scrape_ts = resolve_timestamps(def_timestamp, samples, options); + if (options.track_timestamps_staleness || all_samples_reseted_to_scrape_ts) { + stale_nan_state.add_target(check_result.ls_id); + } + + for (const PromPP::Primitives::Sample& sample : samples) { + shards_inner_series[shard_id_]->emplace_back(sample, check_result.ls_id); + } + + break; + } + case Cache::CheckResult::kRelabel: { + auto& samples = timeseries_buf_.samples(); + bool all_samples_reseted_to_scrape_ts = resolve_timestamps(def_timestamp, samples, options); + if (options.track_timestamps_staleness || all_samples_reseted_to_scrape_ts) { + stale_nan_state.add_input(check_result.source_ls_id); + } + for (const PromPP::Primitives::Sample& sample : samples) { + shards_inner_series[check_result.shard_id]->emplace_back(sample, check_result.ls_id); + } + + break; + } + default: + continue; + } + + stats.samples_added += static_cast(timeseries_buf_.samples().size()); + + if (options.metric_limits == nullptr) { + continue; + } + + samples_count += calculate_samples(timeseries_buf_.samples()); + if (options.metric_limits->samples_limit_exceeded(samples_count)) { + break; + } + } + + PromPP::Primitives::Sample smpl{def_timestamp, kStaleNan}; + stale_nan_state.swap( + [&](uint32_t ls_id) { + if (auto res = cache.check_input(ls_id); res.status == Cache::CheckResult::kRelabel) { + shards_inner_series[res.shard_id]->emplace_back(smpl, res.ls_id); + } + }, + [&](uint32_t ls_id) { + if (auto res = cache.check_target(ls_id); res.status == Cache::CheckResult::kKeep) { + shards_inner_series[shard_id_]->emplace_back(smpl, res.ls_id); + } + }); + cache.optimize(); + } + + template + PROMPP_ALWAYS_INLINE relabelStatus relabel(const RelabelerOptions& options, const StatelessRelabeler& stateless_relabeler, LabelsBuilder& builder) { + bool changed = inject_target_labels(builder, options); + + relabelStatus rstatus = stateless_relabeler.relabeling_process(buf_, builder); + hard_validate(rstatus, builder, options.metric_limits); + if (changed && rstatus == rsKeep) { + rstatus = rsRelabel; + } + + return rstatus; + } + + // resolve_conflicting_exposed_labels add prefix to conflicting label name. + template + PROMPP_ALWAYS_INLINE void resolve_conflicting_exposed_labels(LabelsBuilder& builder, std::vector& conflicting_exposed_labels) { + std::stable_sort(conflicting_exposed_labels.begin(), conflicting_exposed_labels.end(), + [](PromPP::Primitives::LabelView a, PromPP::Primitives::LabelView b) { return a.first.size() < b.first.size(); }); + + for (auto& [ln, lv] : conflicting_exposed_labels) { + while (true) { + ln.insert(0, "exported_"); + if (builder.get(ln).empty()) { + builder.set(ln, lv); + break; + } + } + } + } + + PROMPP_ALWAYS_INLINE bool resolve_timestamps(PromPP::Primitives::Timestamp def_timestamp, + BareBones::Vector& samples, + const RelabelerOptions& options) { + // skip resolve without stalenans + if (def_timestamp == PromPP::Primitives::kNullTimestamp) { + return false; + } + + bool track_staleness{true}; + for (auto& sample : samples) { + // replace null timestamp on def timestamp + if (sample.timestamp() == PromPP::Primitives::kNullTimestamp) { + sample.timestamp() = def_timestamp; + continue; + } + + // replace incoming timestamp on def timestamp + if (!options.honor_timestamps) { + sample.timestamp() = def_timestamp; + continue; + } + + track_staleness = false; + } + + return track_staleness; + } + + template + [[nodiscard]] PROMPP_ALWAYS_INLINE auto skip_shard_inner_series(const Hashdex& hashdex, size_t i) { + auto it = hashdex.begin(); + for (; it != hashdex.end() && i > 0; ++it) { + if ((it->hash() % number_of_shards_) != shard_id_) { + continue; + } + --i; + } + + return it; + } + + public: + // first stage + template + PROMPP_ALWAYS_INLINE void input_relabeling(InputLSS& input_lss, + TargetLSS& target_lss, + Cache& cache, + const Hashdex& hashdex, + const RelabelerOptions& options, + const StatelessRelabeler& stateless_relabeler, + Stats& stats, + PromPP::Primitives::Go::SliceView& shards_inner_series, + PromPP::Primitives::Go::SliceView& shards_relabeled_series) { + NoOpStaleNaNsState state{}; + input_relabeling_internal(input_lss, target_lss, cache, hashdex, options, stateless_relabeler, stats, shards_inner_series, shards_relabeled_series, state, + PromPP::Primitives::kNullTimestamp); + } + + template + PROMPP_ALWAYS_INLINE bool input_relabeling_from_cache(InputLSS& input_lss, + TargetLSS& target_lss, + Cache& cache, + const Hashdex& hashdex, + const RelabelerOptions& options, + Stats& stats, + PromPP::Primitives::Go::SliceView& shards_inner_series) { + NoOpStaleNaNsState state{}; + return input_relabeling_from_cache_internal(input_lss, target_lss, cache, hashdex, options, stats, shards_inner_series, state, + PromPP::Primitives::kNullTimestamp); + } + + template + PROMPP_ALWAYS_INLINE void input_relabeling_with_stalenans(InputLSS& input_lss, + TargetLSS& target_lss, + Cache& cache, + const Hashdex& hashdex, + const RelabelerOptions& options, + const StatelessRelabeler& stateless_relabeler, + Stats& stats, + PromPP::Primitives::Go::SliceView& shards_inner_series, + PromPP::Primitives::Go::SliceView& shards_relabeled_series, + StaleNaNsState& state, + PromPP::Primitives::Timestamp def_timestamp) { + input_relabeling_internal(input_lss, target_lss, cache, hashdex, options, stateless_relabeler, stats, shards_inner_series, shards_relabeled_series, state, + def_timestamp); + } + + template + PROMPP_ALWAYS_INLINE bool input_relabeling_with_stalenans_from_cache(InputLSS& input_lss, + TargetLSS& target_lss, + Cache& cache, + const Hashdex& hashdex, + const RelabelerOptions& options, + Stats& stats, + PromPP::Primitives::Go::SliceView& shards_inner_series, + StaleNaNsState& state, + PromPP::Primitives::Timestamp def_timestamp) { + return input_relabeling_from_cache_internal(input_lss, target_lss, cache, hashdex, options, stats, shards_inner_series, state, def_timestamp); + } + + // second stage + // append_relabeler_series add relabeled ls to lss, add to result and add to cache update. + template + PROMPP_ALWAYS_INLINE void append_relabeler_series(LSS& target_lss, + InnerSeries* inner_series, + const RelabeledSeries* relabeled_series, + RelabelerStateUpdate* relabeler_state_update) { + relabeler_state_update->reserve(relabeler_state_update->size() + relabeled_series->size()); + inner_series->reserve(inner_series->size() + relabeled_series->size()); + if constexpr (BareBones::concepts::has_reserve) { + target_lss.reserve(target_lss.size() + relabeled_series->size()); + } + + for (const auto& relabeled_serie : relabeled_series->data()) { + uint32_t ls_id = target_lss.find_or_emplace(relabeled_serie.ls, relabeled_serie.hash); + + for (const Primitives::Sample& sample : relabeled_serie.samples) { + inner_series->emplace_back(sample, ls_id); + } + + relabeler_state_update->emplace_back(relabeled_serie.ls_id, ls_id); + } + } + + // third stage + // update_relabeler_state add to cache relabled data. + PROMPP_ALWAYS_INLINE void update_relabeler_state(Cache& cache, const RelabelerStateUpdate* relabeler_state_update, const uint16_t relabeled_shard_id) { + for (const auto& update : *relabeler_state_update) { + cache.add_relabel(update.incoming_ls_id, update.relabeled_ls_id, relabeled_shard_id); + } + } + + // destructor. + PROMPP_ALWAYS_INLINE ~PerGoroutineRelabeler() = default; +}; + } // namespace PromPP::Prometheus::Relabel diff --git a/pp/prometheus/stateless_relabeler.h b/pp/prometheus/stateless_relabeler.h index 33a3a8c7db..fa7bdb17c2 100644 --- a/pp/prometheus/stateless_relabeler.h +++ b/pp/prometheus/stateless_relabeler.h @@ -93,6 +93,14 @@ class PatternPart { } } + PROMPP_ALWAYS_INLINE void write(std::ostream& out, std::vector& groups) const { + if (type_ == pGroup) { + out << groups[data_.group_]; + } else { + out << data_.string_; + } + } + ~PatternPart() = default; }; @@ -112,6 +120,8 @@ class Regexp { // count. Use in test. PROMPP_ALWAYS_INLINE int number_of_capturing_groups() { return re_->NumberOfCapturingGroups(); } + PROMPP_ALWAYS_INLINE int number_of_capturing_groups() const { return re_->NumberOfCapturingGroups(); } + // groups - get named capturing groups and number groups. PROMPP_ALWAYS_INLINE std::map groups() { // get named capturing groups @@ -143,6 +153,24 @@ class Regexp { return ok; } + PROMPP_ALWAYS_INLINE bool match_to_args(std::string_view src, std::vector& res) const { + int n = number_of_capturing_groups(); + + // search full match to args, where size - number of capturing groups + res.resize(n + 1); + res[0] = src; + std::vector re_args; + re_args.reserve(n); + std::vector re_args_ptr; + re_args_ptr.reserve(n); + for (int i = 1; i <= n; ++i) { + re_args.emplace_back(&res[i]); + re_args_ptr.emplace_back(&re_args[i - 1]); + } + bool ok = RE2::FullMatchN(src, *re_.get(), &re_args_ptr[0], n); + return ok; + } + // replace_with_args - replace in template with incoming args. PROMPP_ALWAYS_INLINE std::string replace_with_args(std::stringstream& buf, std::vector& args, std::vector& tmpl) { if (tmpl.size() == 0) [[unlikely]] { @@ -158,6 +186,20 @@ class Regexp { return buf.str(); } + PROMPP_ALWAYS_INLINE std::string replace_with_args(std::stringstream& buf, std::vector& args, const std::vector& tmpl) const { + if (tmpl.size() == 0) [[unlikely]] { + // no template or source data + return ""; + } + + buf.str(""); + for (auto& val : tmpl) { + val.write(buf, args); + } + + return buf.str(); + } + // replace_full - find match for source and replace in template. PROMPP_ALWAYS_INLINE std::string replace_full(std::stringstream& out, std::string_view src, std::vector& tmpl) { if (src.size() == 0 || tmpl.size() == 0) [[unlikely]] { @@ -175,8 +217,24 @@ class Regexp { return replace_with_args(out, res_args, tmpl); } + PROMPP_ALWAYS_INLINE std::string replace_full(std::stringstream& out, std::string_view src, const std::vector& tmpl) const { + if (src.size() == 0 || tmpl.size() == 0) [[unlikely]] { + // no template or source data + return ""; + } + + std::vector res_args; + bool ok = match_to_args(src, res_args); + if (!ok) { + // no entries in regexp + return ""; + } + + return replace_with_args(out, res_args, tmpl); + } + // full_match - check text for full match regexp. - PROMPP_ALWAYS_INLINE bool full_match(std::string_view str) { return RE2::FullMatch(str, *re_.get()); } + PROMPP_ALWAYS_INLINE bool full_match(std::string_view str) const { return RE2::FullMatch(str, *re_.get()); } }; struct GORelabelConfig { @@ -562,6 +620,144 @@ class RelabelConfig { return rsKeep; } + template + PROMPP_ALWAYS_INLINE relabelStatus relabel(std::stringstream& buf, LabelsBuilder& builder) const { + std::string value; + for (size_t i = 0; i < source_labels_.size(); ++i) { + std::string_view lv = builder.get(source_labels_[i]); + if (i == 0) { + value += std::string(lv); + continue; + } + value += separator_; + value += lv; + } + + switch (action_) { + case rDrop: { + if (regexp_.full_match(value)) { + return rsDrop; + } + break; + } + + case rKeep: { + if (!regexp_.full_match(value)) { + return rsDrop; + } + break; + } + + case rDropEqual: { + if (builder.get(target_label_) == value) { + return rsDrop; + } + break; + } + + case rKeepEqual: { + if (builder.get(target_label_) != value) { + return rsDrop; + } + break; + } + + case rReplace: { + std::vector res_args; + bool ok = regexp_.match_to_args(value, res_args); + if (!ok) { + break; + } + + std::string lname = regexp_.replace_with_args(buf, res_args, target_label_parts_); + if (!label_name_is_valid(lname)) { + break; + } + std::string lvalue = regexp_.replace_with_args(buf, res_args, replacement_parts_); + if (lvalue.size() == 0) { + if (builder.contains(lname)) { + builder.del(lname); + return rsRelabel; + } + break; + } + builder.set(lname, lvalue); + return rsRelabel; + } + + case rLowercase: { + std::string lvalue{value}; + std::ranges::transform(lvalue, lvalue.begin(), [](unsigned char c) { return std::tolower(c); }); + builder.set(target_label_, lvalue); + return rsRelabel; + } + + case rUppercase: { + std::string lvalue{value}; + std::ranges::transform(lvalue, lvalue.begin(), [](unsigned char c) { return std::toupper(c); }); + builder.set(target_label_, lvalue); + return rsRelabel; + } + + case rHashMod: { + std::string lvalue{std::to_string(make_hash_uint64(value) % modulus_)}; + builder.set(target_label_, lvalue); + return rsRelabel; + } + + case rLabelMap: { + bool change{false}; + builder.range([&](LNameType& lname, LValueType& lvalue) PROMPP_LAMBDA_INLINE -> bool { + if (regexp_.full_match(lname)) { + std::string rlname = regexp_.replace_full(buf, lname, replacement_parts_); + builder.set(rlname, lvalue); + change = true; + } + return true; + }); + if (change) { + return rsRelabel; + } + break; + } + + case rLabelDrop: { + bool change{false}; + builder.range([&](LNameType& lname, [[maybe_unused]] LValueType& lvalue) PROMPP_LAMBDA_INLINE -> bool { + if (regexp_.full_match(lname)) { + builder.del(lname); + change = true; + } + return true; + }); + if (change) { + return rsRelabel; + } + break; + } + case rLabelKeep: { + bool change{false}; + builder.range([&](LNameType& lname, [[maybe_unused]] LValueType& lvalue) PROMPP_LAMBDA_INLINE -> bool { + if (!regexp_.full_match(lname)) { + builder.del(lname); + change = true; + } + return true; + }); + if (change) { + return rsRelabel; + } + break; + } + + default: { + throw BareBones::Exception(0x481dea53751b85c3, "unknown relabel action"); + } + } + + return rsKeep; + } + // ~RelabelConfig - destructor for RelabelConfig from go-config. PROMPP_ALWAYS_INLINE ~RelabelConfig() = default; }; @@ -599,6 +795,22 @@ class StatelessRelabeler { return rstatus; } + template + PROMPP_ALWAYS_INLINE relabelStatus relabeling_process(std::stringstream& buf, LabelsBuilder& builder) const { + relabelStatus rstatus{rsKeep}; + for (auto& rcfg : configs_) { + relabelStatus status = rcfg.relabel(buf, builder); + if (status == rsDrop) { + return rsDrop; + } + if (status == rsRelabel) { + rstatus = rsRelabel; + } + } + + return rstatus; + } + // relabeling_process_with_soft_validate caller passes a LabelsBuilder containing the initial set of labels, which is mutated by the rules with soft(on empty) // validate label set. template From 7c7896bbc9c67eb1c4740e32c730f7857486250a Mon Sep 17 00:00:00 2001 From: Alexandr Yudin Date: Thu, 14 Aug 2025 07:34:12 +0000 Subject: [PATCH 14/96] refactoring PerGoroutineRelabeler --- pp/go/cppbridge/prometheus_relabeler.go | 52 ++- pp/go/cppbridge/prometheus_relabeler_test.go | 346 ++++++++++++++++++- pp/go/storage/appender/appender.go | 97 +++--- pp/go/storage/head/head/head.go | 7 +- pp/go/storage/head/shard/lss.go | 3 +- pp/go/storage/head/shard/shard.go | 16 +- 6 files changed, 423 insertions(+), 98 deletions(-) diff --git a/pp/go/cppbridge/prometheus_relabeler.go b/pp/go/cppbridge/prometheus_relabeler.go index daed303587..7cf1f15411 100644 --- a/pp/go/cppbridge/prometheus_relabeler.go +++ b/pp/go/cppbridge/prometheus_relabeler.go @@ -1128,6 +1128,7 @@ func (s *State) resetStaleNansStates(numberOfShards uint16, equaledGeneration bo type PerGoroutineRelabeler struct { cptr uintptr gcDestroyDetector *uint64 + shardID uint16 } // NewPerGoroutineRelabeler init new [PerGoroutineRelabeler]. @@ -1137,6 +1138,7 @@ func NewPerGoroutineRelabeler( pgr := &PerGoroutineRelabeler{ cptr: prometheusPerGoroutineRelabelerCtor(numberOfShards, shardID), gcDestroyDetector: &gcDestroyDetector, + shardID: shardID, } runtime.SetFinalizer(pgr, func(r *PerGoroutineRelabeler) { prometheusPerShardRelabelerDtor(r.cptr) @@ -1174,8 +1176,7 @@ func (pgr *PerGoroutineRelabeler) InputRelabeling( statelessRelabeler *StatelessRelabeler, inputLss *LabelSetStorage, targetLss *LabelSetStorage, - cache *Cache, - options RelabelerOptions, + state *State, shardedData ShardedData, shardsInnerSeries []*InnerSeries, shardsRelabeledSeries []*RelabeledSeries, @@ -1194,9 +1195,9 @@ func (pgr *PerGoroutineRelabeler) InputRelabeling( statelessRelabeler.Pointer(), inputLss.Pointer(), targetLss.Pointer(), - cache.cPointer, + state.CacheByShard(pgr.shardID).cPointer, cptrContainer.cptr(), - options, + state.RelabelerOptions(), shardsInnerSeries, shardsRelabeledSeries, ) @@ -1204,7 +1205,7 @@ func (pgr *PerGoroutineRelabeler) InputRelabeling( runtime.KeepAlive(statelessRelabeler) runtime.KeepAlive(inputLss) runtime.KeepAlive(targetLss) - runtime.KeepAlive(cache) + runtime.KeepAlive(state) runtime.KeepAlive(cptrContainer) return stats, hasReallocations, handleException(exception) @@ -1215,8 +1216,7 @@ func (pgr *PerGoroutineRelabeler) InputRelabelingFromCache( ctx context.Context, inputLss *LabelSetStorage, targetLss *LabelSetStorage, - cache *Cache, - options RelabelerOptions, + state *State, shardedData ShardedData, shardsInnerSeries []*InnerSeries, ) (RelabelerStats, bool, error) { @@ -1233,15 +1233,15 @@ func (pgr *PerGoroutineRelabeler) InputRelabelingFromCache( pgr.cptr, inputLss.Pointer(), targetLss.Pointer(), - cache.cPointer, + state.CacheByShard(pgr.shardID).cPointer, cptrContainer.cptr(), - options, + state.RelabelerOptions(), shardsInnerSeries, ) runtime.KeepAlive(pgr) runtime.KeepAlive(inputLss) runtime.KeepAlive(targetLss) - runtime.KeepAlive(cache) + runtime.KeepAlive(state) runtime.KeepAlive(cptrContainer) return stats, ok, handleException(exception) @@ -1253,10 +1253,7 @@ func (pgr *PerGoroutineRelabeler) InputRelabelingWithStalenans( statelessRelabeler *StatelessRelabeler, inputLss *LabelSetStorage, targetLss *LabelSetStorage, - cache *Cache, - options RelabelerOptions, - staleNansState *StaleNansState, - defTimestamp int64, + state *State, shardedData ShardedData, shardsInnerSeries []*InnerSeries, shardsRelabeledSeries []*RelabeledSeries, @@ -1274,11 +1271,11 @@ func (pgr *PerGoroutineRelabeler) InputRelabelingWithStalenans( statelessRelabeler.Pointer(), inputLss.Pointer(), targetLss.Pointer(), - cache.cPointer, + state.CacheByShard(pgr.shardID).cPointer, cptrContainer.cptr(), - staleNansState.state, - defTimestamp, - options, + state.StaleNansStateByShard(pgr.shardID).state, + state.DefTimestamp(), + state.RelabelerOptions(), shardsInnerSeries, shardsRelabeledSeries, ) @@ -1286,9 +1283,8 @@ func (pgr *PerGoroutineRelabeler) InputRelabelingWithStalenans( runtime.KeepAlive(statelessRelabeler) runtime.KeepAlive(inputLss) runtime.KeepAlive(targetLss) - runtime.KeepAlive(cache) + runtime.KeepAlive(state) runtime.KeepAlive(cptrContainer) - runtime.KeepAlive(staleNansState) return stats, hasReallocations, handleException(exception) } @@ -1298,10 +1294,7 @@ func (pgr *PerGoroutineRelabeler) InputRelabelingWithStalenansFromCache( ctx context.Context, inputLss *LabelSetStorage, targetLss *LabelSetStorage, - cache *Cache, - options RelabelerOptions, - staleNansState *StaleNansState, - defTimestamp int64, + state *State, shardedData ShardedData, shardsInnerSeries []*InnerSeries, ) (RelabelerStats, bool, error) { @@ -1318,20 +1311,19 @@ func (pgr *PerGoroutineRelabeler) InputRelabelingWithStalenansFromCache( pgr.cptr, inputLss.Pointer(), targetLss.Pointer(), - cache.cPointer, + state.CacheByShard(pgr.shardID).cPointer, cptrContainer.cptr(), - staleNansState.state, - defTimestamp, - options, + state.StaleNansStateByShard(pgr.shardID).state, + state.DefTimestamp(), + state.RelabelerOptions(), shardsInnerSeries, ) runtime.KeepAlive(pgr) runtime.KeepAlive(inputLss) runtime.KeepAlive(targetLss) - runtime.KeepAlive(cache) + runtime.KeepAlive(state) runtime.KeepAlive(cptrContainer) - runtime.KeepAlive(staleNansState) return stats, ok, handleException(exception) } diff --git a/pp/go/cppbridge/prometheus_relabeler_test.go b/pp/go/cppbridge/prometheus_relabeler_test.go index 3e4d51c72b..a98e3abd6b 100644 --- a/pp/go/cppbridge/prometheus_relabeler_test.go +++ b/pp/go/cppbridge/prometheus_relabeler_test.go @@ -643,7 +643,6 @@ func (s *RelabelerSuite) TestInputPerGoroutineRelabeler() { s.Require().NoError(err) var numberOfShards uint16 = 1 - pgr := cppbridge.NewPerGoroutineRelabeler(numberOfShards, 0) hlimits := cppbridge.DefaultWALHashdexLimits() h, err := cppbridge.NewWALSnappyProtobufHashdex(snappy.Encode(nil, data), hlimits) @@ -651,15 +650,16 @@ func (s *RelabelerSuite) TestInputPerGoroutineRelabeler() { shardsInnerSeries := cppbridge.NewShardsInnerSeries(numberOfShards) shardsRelabeledSeries := cppbridge.NewShardsRelabeledSeries(numberOfShards) - cache := cppbridge.NewCache() + state := cppbridge.NewState(numberOfShards) + state.SetRelabelerOptions(&s.options) + pgr := cppbridge.NewPerGoroutineRelabeler(numberOfShards, 0) stats, hasReallocations, err := pgr.InputRelabeling( s.baseCtx, statelessRelabeler, inputLss, targetLss, - cache, - s.options, + state, h, shardsInnerSeries, shardsRelabeledSeries, @@ -668,3 +668,341 @@ func (s *RelabelerSuite) TestInputPerGoroutineRelabeler() { s.Equal(cppbridge.RelabelerStats{1, 1, 1}, stats) s.True(hasReallocations) } + +func (s *RelabelerSuite) TestInputPerGoroutineRelabelerFromCacheTrue() { + wr := prompb.WriteRequest{ + Timeseries: []prompb.TimeSeries{ + { + Labels: []prompb.Label{ + {Name: "__name__", Value: "value"}, + {Name: "job", Value: "abc"}, + {Name: "instance", Value: "value1"}, + }, + Samples: []prompb.Sample{ + {Value: 0.1, Timestamp: time.Now().UnixMilli()}, + }, + }, + }, + } + data, err := wr.Marshal() + s.Require().NoError(err) + + rCfgs := []*cppbridge.RelabelConfig{ + { + SourceLabels: []string{"job"}, + Regex: "abc", + Action: cppbridge.Keep, + }, + } + + inputLss := cppbridge.NewLssStorage() + targetLss := cppbridge.NewQueryableLssStorage() + + statelessRelabeler, err := cppbridge.NewStatelessRelabeler(rCfgs) + s.Require().NoError(err) + + var numberOfShards uint16 = 1 + + hlimits := cppbridge.DefaultWALHashdexLimits() + h, err := cppbridge.NewWALSnappyProtobufHashdex(snappy.Encode(nil, data), hlimits) + s.Require().NoError(err) + + shardsInnerSeries := cppbridge.NewShardsInnerSeries(numberOfShards) + shardsRelabeledSeries := cppbridge.NewShardsRelabeledSeries(numberOfShards) + state := cppbridge.NewState(numberOfShards) + state.SetRelabelerOptions(&s.options) + + pgr := cppbridge.NewPerGoroutineRelabeler(numberOfShards, 0) + stats, hasReallocations, err := pgr.InputRelabeling( + s.baseCtx, + statelessRelabeler, + inputLss, + targetLss, + state, + h, + shardsInnerSeries, + shardsRelabeledSeries, + ) + s.Require().NoError(err) + s.Equal(cppbridge.RelabelerStats{1, 1, 0}, stats) + s.True(hasReallocations) + + stats, ok, err := pgr.InputRelabelingFromCache( + s.baseCtx, + inputLss, + targetLss, + state, + h, + shardsInnerSeries, + ) + s.Require().NoError(err) + s.Equal(cppbridge.RelabelerStats{1, 0, 0}, stats) + s.True(ok) +} + +func (s *RelabelerSuite) TestInputPerGoroutineRelabelerFromCacheFalse() { + wr := prompb.WriteRequest{ + Timeseries: []prompb.TimeSeries{ + { + Labels: []prompb.Label{ + {Name: "__name__", Value: "value"}, + {Name: "job", Value: "abc"}, + {Name: "instance", Value: "value1"}, + }, + Samples: []prompb.Sample{ + {Value: 0.1, Timestamp: time.Now().UnixMilli()}, + }, + }, + { + Labels: []prompb.Label{ + {Name: "__name__", Value: "value"}, + {Name: "instance", Value: "value1"}, + }, + Samples: []prompb.Sample{ + {Value: 0.1, Timestamp: time.Now().UnixMilli()}, + }, + }, + }, + } + data, err := wr.Marshal() + s.Require().NoError(err) + + inputLss := cppbridge.NewLssStorage() + targetLss := cppbridge.NewQueryableLssStorage() + + var numberOfShards uint16 = 1 + + hlimits := cppbridge.DefaultWALHashdexLimits() + h, err := cppbridge.NewWALSnappyProtobufHashdex(snappy.Encode(nil, data), hlimits) + s.Require().NoError(err) + + shardsInnerSeries := cppbridge.NewShardsInnerSeries(numberOfShards) + state := cppbridge.NewState(numberOfShards) + state.SetRelabelerOptions(&s.options) + + pgr := cppbridge.NewPerGoroutineRelabeler(numberOfShards, 0) + stats, ok, err := pgr.InputRelabelingFromCache( + s.baseCtx, + inputLss, + targetLss, + state, + h, + shardsInnerSeries, + ) + s.Require().NoError(err) + s.Equal(cppbridge.RelabelerStats{0, 0, 0}, stats) + s.False(ok) +} + +func (s *RelabelerSuite) TestInputPerGoroutineRelabelerFromCachePartially() { + ts := time.Now().UnixMilli() + wr1 := prompb.WriteRequest{ + Timeseries: []prompb.TimeSeries{ + { + Labels: []prompb.Label{ + {Name: "__name__", Value: "value0"}, + {Name: "job", Value: "abc"}, + {Name: "instance", Value: "value0"}, + }, + Samples: []prompb.Sample{ + {Value: 0.1, Timestamp: ts}, + }, + }, + { + Labels: []prompb.Label{ + {Name: "__name__", Value: "value1"}, + {Name: "job", Value: "abc"}, + {Name: "instance", Value: "value1"}, + }, + Samples: []prompb.Sample{ + {Value: 0.1, Timestamp: ts}, + }, + }, + { + Labels: []prompb.Label{ + {Name: "__name__", Value: "value2"}, + {Name: "job", Value: "abc"}, + {Name: "instance", Value: "value2"}, + }, + Samples: []prompb.Sample{ + {Value: 0.1, Timestamp: ts}, + }, + }, + { + Labels: []prompb.Label{ + {Name: "__name__", Value: "value3"}, + {Name: "job", Value: "abc"}, + {Name: "instance", Value: "value3"}, + }, + Samples: []prompb.Sample{ + {Value: 0.1, Timestamp: ts}, + }, + }, + { + Labels: []prompb.Label{ + {Name: "__name__", Value: "value3"}, + {Name: "instance", Value: "value3"}, + }, + Samples: []prompb.Sample{ + {Value: 0.1, Timestamp: ts}, + }, + }, + { + Labels: []prompb.Label{ + {Name: "__name__", Value: "value4"}, + {Name: "instance", Value: "value4"}, + }, + Samples: []prompb.Sample{ + {Value: 0.1, Timestamp: ts}, + }, + }, + { + Labels: []prompb.Label{ + {Name: "__name__", Value: "value5"}, + {Name: "instance", Value: "value5"}, + }, + Samples: []prompb.Sample{ + {Value: 0.1, Timestamp: ts}, + }, + }, + { + Labels: []prompb.Label{ + {Name: "__name__", Value: "value6"}, + {Name: "instance", Value: "value6"}, + }, + Samples: []prompb.Sample{ + {Value: 0.1, Timestamp: ts}, + }, + }, + }, + } + data1, err := wr1.Marshal() + s.Require().NoError(err) + hlimits := cppbridge.DefaultWALHashdexLimits() + h1, err := cppbridge.NewWALSnappyProtobufHashdex(snappy.Encode(nil, data1), hlimits) + s.Require().NoError(err) + + ts += 6000 + wr2 := prompb.WriteRequest{ + Timeseries: []prompb.TimeSeries{ + { + Labels: []prompb.Label{ + {Name: "__name__", Value: "value0"}, + {Name: "job", Value: "abc"}, + {Name: "instance", Value: "value0"}, + }, + Samples: []prompb.Sample{ + {Value: 0.1, Timestamp: ts}, + }, + }, + { + Labels: []prompb.Label{ + {Name: "__name__", Value: "value1"}, + {Name: "job", Value: "abc"}, + {Name: "instance", Value: "value1"}, + }, + Samples: []prompb.Sample{ + {Value: 0.1, Timestamp: ts}, + }, + }, + { + Labels: []prompb.Label{ + {Name: "__name__", Value: "value2"}, + {Name: "job", Value: "abc"}, + {Name: "instance", Value: "value2"}, + }, + Samples: []prompb.Sample{ + {Value: 0.1, Timestamp: ts}, + }, + }, + { + Labels: []prompb.Label{ + {Name: "__name__", Value: "value3"}, + {Name: "job", Value: "abc"}, + {Name: "instance", Value: "value3"}, + }, + Samples: []prompb.Sample{ + {Value: 0.1, Timestamp: ts}, + }, + }, + { + Labels: []prompb.Label{ + {Name: "__name__", Value: "value4"}, + {Name: "job", Value: "abc"}, + {Name: "instance", Value: "value4"}, + }, + Samples: []prompb.Sample{ + {Value: 0.1, Timestamp: ts}, + }, + }, + }, + } + data2, err := wr2.Marshal() + s.Require().NoError(err) + h2, err := cppbridge.NewWALSnappyProtobufHashdex(snappy.Encode(nil, data2), hlimits) + s.Require().NoError(err) + + rCfgs := []*cppbridge.RelabelConfig{ + { + SourceLabels: []string{"job"}, + Regex: "abc", + Action: cppbridge.Keep, + }, + } + + inputLss := cppbridge.NewLssStorage() + targetLss := cppbridge.NewQueryableLssStorage() + + statelessRelabeler, err := cppbridge.NewStatelessRelabeler(rCfgs) + s.Require().NoError(err) + + var numberOfShards uint16 = 1 + + shardsInnerSeries := cppbridge.NewShardsInnerSeries(numberOfShards) + shardsRelabeledSeries := cppbridge.NewShardsRelabeledSeries(numberOfShards) + state := cppbridge.NewState(numberOfShards) + state.SetRelabelerOptions(&s.options) + + pgr := cppbridge.NewPerGoroutineRelabeler(numberOfShards, 0) + stats, hasReallocations, err := pgr.InputRelabeling( + s.baseCtx, + statelessRelabeler, + inputLss, + targetLss, + state, + h1, + shardsInnerSeries, + shardsRelabeledSeries, + ) + s.Require().NoError(err) + s.Equal(cppbridge.RelabelerStats{4, 4, 4}, stats) + s.True(hasReallocations) + + shardsInnerSeries = cppbridge.NewShardsInnerSeries(numberOfShards) + stats, ok, err := pgr.InputRelabelingFromCache( + s.baseCtx, + inputLss, + targetLss, + state, + h2, + shardsInnerSeries, + ) + s.Require().NoError(err) + s.Equal(cppbridge.RelabelerStats{4, 0, 0}, stats) + s.False(ok) + s.Equal(uint64(4), shardsInnerSeries[0].Size()) + + stats, _, err = pgr.InputRelabeling( + s.baseCtx, + statelessRelabeler, + inputLss, + targetLss, + state, + h2, + shardsInnerSeries, + shardsRelabeledSeries, + ) + s.Require().NoError(err) + s.Equal(cppbridge.RelabelerStats{1, 1, 0}, stats) + s.Equal(uint64(5), shardsInnerSeries[0].Size()) +} diff --git a/pp/go/storage/appender/appender.go b/pp/go/storage/appender/appender.go index 1987cf0bdb..8d06acc229 100644 --- a/pp/go/storage/appender/appender.go +++ b/pp/go/storage/appender/appender.go @@ -55,6 +55,9 @@ type LSS interface { // WithRLock calls fn on raws [cppbridge.LabelSetStorage] with read lock. WithRLock(fn func(target, input *cppbridge.LabelSetStorage) error) error + + // ResetSnapshot resets the current snapshot. Use only WithLock. + ResetSnapshot() } // @@ -69,6 +72,9 @@ type Shard[TDataStorage DataStorage, TLSS LSS] interface { // LSS returns shard labelset storage [LSS]. LSS() TLSS + // Relabeler returns relabeler for shard goroutines. + Relabeler() *cppbridge.PerGoroutineRelabeler + // ShardID returns the shard ID. ShardID() uint16 } @@ -233,35 +239,28 @@ func (a *Appender[TGenericTask, TDataStorage, TLSS, TShard, THead]) inputRelabel LSSInputRelabeling, func(shard TShard) error { var ( - lss = shard.LSS() - shardID = shard.ShardID() - hasReallocations bool - ok bool + lss = shard.LSS() + relabeler = shard.Relabeler() + shardID = shard.ShardID() + ok bool ) - if err := lss.WithRLock(func(target, input *cppbridge.LabelSetStorage) error { - var rErr error + if err := lss.WithRLock(func(target, input *cppbridge.LabelSetStorage) (rErr error) { if state.TrackStaleness() { - stats[shardID], ok, rErr = rd.InputRelabelerByShard( - shardID, - ).InputRelabelingWithStalenansFromCache( + stats[shardID], ok, rErr = relabeler.InputRelabelingWithStalenansFromCache( ctx, input, target, - state.CacheByShard(shardID), - state.RelabelerOptions(), - state.StaleNansStateByShard(shardID), - state.DefTimestamp(), + state, incomingData.ShardedData(), shardedInnerSeries.DataBySourceShard(shardID), ) } else { - stats[shardID], ok, rErr = rd.InputRelabelerByShard(shardID).InputRelabelingFromCache( + stats[shardID], ok, rErr = relabeler.InputRelabelingFromCache( ctx, input, target, - state.CacheByShard(shardID), - state.RelabelerOptions(), + state, incomingData.ShardedData(), shardedInnerSeries.DataBySourceShard(shardID), ) @@ -278,35 +277,39 @@ func (a *Appender[TGenericTask, TDataStorage, TLSS, TShard, THead]) inputRelabel return nil } - shard.LSSLock() - defer shard.LSSUnlock() - rstats := cppbridge.RelabelerStats{} - - if state.TrackStaleness() { - rstats, hasReallocations, err = rd.InputRelabelerByShard(shardID).InputRelabelingWithStalenans( - ctx, - shard.LSS().Input(), - shard.LSS().Target(), - state.CacheByShard(shardID), - state.RelabelerOptions(), - state.StaleNansStateByShard(shardID), - state.DefTimestamp(), - incomingData.ShardedData(), - shardedInnerSeries.DataBySourceShard(shardID), - shardedRelabeledSeries.DataByShard(shardID), - ) - } else { - rstats, hasReallocations, err = rd.InputRelabelerByShard(shardID).InputRelabeling( - ctx, - shard.LSS().Input(), - shard.LSS().Target(), - state.CacheByShard(shardID), - state.RelabelerOptions(), - incomingData.ShardedData(), - shardedInnerSeries.DataBySourceShard(shardID), - shardedRelabeledSeries.DataByShard(shardID), - ) - } + var ( + hasReallocations bool + rstats = cppbridge.RelabelerStats{} + ) + err := lss.WithLock(func(target, input *cppbridge.LabelSetStorage) (rErr error) { + if state.TrackStaleness() { + rstats, hasReallocations, rErr = relabeler.InputRelabelingWithStalenans( + ctx, + input, + target, + state, + incomingData.ShardedData(), + shardedInnerSeries.DataBySourceShard(shardID), + shardedRelabeledSeries.DataByShard(shardID), + ) + } else { + rstats, hasReallocations, rErr = relabeler.InputRelabeling( + ctx, + input, + target, + state, + incomingData.ShardedData(), + shardedInnerSeries.DataBySourceShard(shardID), + shardedRelabeledSeries.DataByShard(shardID), + ) + } + + if hasReallocations { + shard.LSS().ResetSnapshot() + } + + return rErr + }) incomingData.Destroy() if err != nil { @@ -317,10 +320,6 @@ func (a *Appender[TGenericTask, TDataStorage, TLSS, TShard, THead]) inputRelabel stats[shardID].SeriesAdded += rstats.SeriesAdded stats[shardID].SeriesDrop += rstats.SeriesDrop - if hasReallocations { - shard.LSS().ResetSnapshot() - } - return nil }, ) diff --git a/pp/go/storage/head/head/head.go b/pp/go/storage/head/head/head.go index a151ff17ea..1f3156cc32 100644 --- a/pp/go/storage/head/head/head.go +++ b/pp/go/storage/head/head/head.go @@ -59,7 +59,7 @@ type Head[TShard Shard, TGorutineShard Shard] struct { id string generation uint64 - gshardCtor func(TShard) TGorutineShard + gshardCtor func(s TShard, numberOfShards uint16) TGorutineShard shards []TShard taskChs []chan *task.Generic[TGorutineShard] querySemaphore *locker.Weighted @@ -86,7 +86,7 @@ type Head[TShard Shard, TGorutineShard Shard] struct { func NewHead[TShard Shard, TGoroutineShard Shard]( id string, shards []TShard, - gshardCtor func(TShard) TGoroutineShard, + gshardCtor func(TShard, uint16) TGoroutineShard, numberOfShards uint16, registerer prometheus.Registerer, ) *Head[TShard, TGoroutineShard] { @@ -275,8 +275,7 @@ func (h *Head[TShard, TGorutineShard]) shardLoop( stopc chan struct{}, s TShard, ) { - // TODO PerGoroutineRelabeler - pgs := h.gshardCtor(s) + pgs := h.gshardCtor(s, h.numberOfShards) for { select { diff --git a/pp/go/storage/head/shard/lss.go b/pp/go/storage/head/shard/lss.go index 022047de75..c5e5b085e4 100644 --- a/pp/go/storage/head/shard/lss.go +++ b/pp/go/storage/head/shard/lss.go @@ -114,9 +114,8 @@ func (l *LSS) QueryStatus(status *cppbridge.HeadStatus, limit int) { l.locker.RUnlock() } -// ResetSnapshot resets the current snapshot. +// ResetSnapshot resets the current snapshot. Use only WithLock. func (l *LSS) ResetSnapshot() { - // TODO l.snapshot = nil l.once = sync.Once{} } diff --git a/pp/go/storage/head/shard/shard.go b/pp/go/storage/head/shard/shard.go index a568e41814..456098ea59 100644 --- a/pp/go/storage/head/shard/shard.go +++ b/pp/go/storage/head/shard/shard.go @@ -96,23 +96,21 @@ func (s *Shard[TWal]) WalWrite(innerSeriesSlice []*cppbridge.InnerSeries) (bool, // PerGoroutineShard wrapper of shard with [PerGoroutineRelabeler] for goroutines. type PerGoroutineShard[TWal Wal] struct { - // TODO PerGoroutineRelabeler - relabeler string + relabeler *cppbridge.PerGoroutineRelabeler *Shard[TWal] } // NewPerGoroutineShard init new [PerGoroutineShard]. -func NewPerGoroutineShard[TWal Wal](s *Shard[TWal]) *PerGoroutineShard[TWal] { +func NewPerGoroutineShard[TWal Wal](s *Shard[TWal], numberOfShards uint16) *PerGoroutineShard[TWal] { return &PerGoroutineShard[TWal]{ - Shard: s, + relabeler: cppbridge.NewPerGoroutineRelabeler(numberOfShards, s.ShardID()), + Shard: s, } } -// TODO implementation. -func (s *PerGoroutineShard[TWal]) SetRelabeler() { -} - -func (s *PerGoroutineShard[TWal]) GetRelabeler() { +// Relabeler returns relabeler for shard goroutines. +func (s *PerGoroutineShard[TWal]) Relabeler() *cppbridge.PerGoroutineRelabeler { + return s.relabeler } // // InputRelabeling relabeling incoming hashdex(first stage). From 4eb55a1007aa5462af37a8a1c98941c2aae4b158 Mon Sep 17 00:00:00 2001 From: Alexandr Yudin Date: Thu, 14 Aug 2025 09:02:56 +0000 Subject: [PATCH 15/96] state and StatelessRelabeler --- pp/go/cppbridge/prometheus_relabeler.go | 104 +++++++++++-------- pp/go/cppbridge/prometheus_relabeler_test.go | 7 +- pp/go/storage/appender/appender.go | 1 - 3 files changed, 61 insertions(+), 51 deletions(-) diff --git a/pp/go/cppbridge/prometheus_relabeler.go b/pp/go/cppbridge/prometheus_relabeler.go index 7cf1f15411..88b6104130 100644 --- a/pp/go/cppbridge/prometheus_relabeler.go +++ b/pp/go/cppbridge/prometheus_relabeler.go @@ -950,10 +950,15 @@ func (c *Cache) ResetTo() { runtime.KeepAlive(c) } +// +// State +// + // State state of relabelers per shard. type State struct { caches []*Cache staleNansStates []*StaleNansState + statelessRelabeler *StatelessRelabeler defTimestamp int64 generationRelabeler uint64 generationHead uint64 @@ -991,23 +996,6 @@ func (s *State) CacheByShard(shardID uint16) *Cache { return s.caches[shardID] } -// StaleNansStateByShard return SourceStaleNansState for shard. -func (s *State) StaleNansStateByShard(shardID uint16) *StaleNansState { - if int(shardID) >= len(s.staleNansStates) { - panic(fmt.Sprintf( - "shardID(%d) out of range in staleNansStates(%d)", - shardID, - len(s.caches), - )) - } - - if s.staleNansStates[shardID] == nil { - s.staleNansStates[shardID] = NewStaleNansState() - } - - return s.staleNansStates[shardID] -} - // DefTimestamp return timestamp for scrape time and stalenan. func (s *State) DefTimestamp() int64 { if s.defTimestamp == 0 { @@ -1017,34 +1005,14 @@ func (s *State) DefTimestamp() int64 { return s.defTimestamp } -// SetDefTimestamp set timestamp for scrape time and stalenan. -func (s *State) SetDefTimestamp(ts int64) { - s.defTimestamp = ts -} - -// EnableTrackStaleness enable track stalenans. -func (s *State) EnableTrackStaleness() { - s.trackStaleness = true -} - // DisableTrackStaleness disable track stalenans. func (s *State) DisableTrackStaleness() { s.trackStaleness = false } -// TrackStaleness return state track stalenans. -func (s *State) TrackStaleness() bool { - return s.trackStaleness -} - -// RelabelerOptions return Options for relabeler. -func (s *State) RelabelerOptions() RelabelerOptions { - return s.options -} - -// SetRelabelerOptions set Options for relabeler. -func (s *State) SetRelabelerOptions(options *RelabelerOptions) { - s.options = *options +// EnableTrackStaleness enable track stalenans. +func (s *State) EnableTrackStaleness() { + s.trackStaleness = true } // Reconfigure recreate caches and stalenans states if need and set new generations. @@ -1061,6 +1029,54 @@ func (s *State) Reconfigure( s.generationHead = generationHead } +// RelabelerOptions return Options for relabeler. +func (s *State) RelabelerOptions() RelabelerOptions { + return s.options +} + +// SetDefTimestamp set timestamp for scrape time and stalenan. +func (s *State) SetDefTimestamp(ts int64) { + s.defTimestamp = ts +} + +// SetRelabelerOptions set Options for relabeler. +func (s *State) SetRelabelerOptions(options *RelabelerOptions) { + s.options = *options +} + +// SetStatelessRelabeler set [StatelessRelabeler] for [PerGoroutineRelabeler]. +func (s *State) SetStatelessRelabeler(relabeler *StatelessRelabeler) { + s.statelessRelabeler = relabeler +} + +// StaleNansStateByShard return SourceStaleNansState for shard. +func (s *State) StaleNansStateByShard(shardID uint16) *StaleNansState { + if int(shardID) >= len(s.staleNansStates) { + panic(fmt.Sprintf( + "shardID(%d) out of range in staleNansStates(%d)", + shardID, + len(s.caches), + )) + } + + if s.staleNansStates[shardID] == nil { + s.staleNansStates[shardID] = NewStaleNansState() + } + + return s.staleNansStates[shardID] +} + +// StatelessRelabeler returns [StatelessRelabeler] for [PerGoroutineRelabeler]. +func (s *State) StatelessRelabeler() *StatelessRelabeler { + // TODO validate nil and reconfigure + return s.statelessRelabeler +} + +// TrackStaleness return state track stalenans. +func (s *State) TrackStaleness() bool { + return s.trackStaleness +} + // resetCaches recreate Caches. // //revive:disable-next-line:flag-parameter this is a flag, but it's more convenient this way @@ -1173,7 +1189,6 @@ func (pgr *PerGoroutineRelabeler) AppendRelabelerSeries( // InputRelabeling relabeling incoming hashdex(first stage). func (pgr *PerGoroutineRelabeler) InputRelabeling( ctx context.Context, - statelessRelabeler *StatelessRelabeler, inputLss *LabelSetStorage, targetLss *LabelSetStorage, state *State, @@ -1192,7 +1207,7 @@ func (pgr *PerGoroutineRelabeler) InputRelabeling( stats, exception, hasReallocations := prometheusPerGoroutineRelabelerInputRelabeling( pgr.cptr, - statelessRelabeler.Pointer(), + state.StatelessRelabeler().Pointer(), inputLss.Pointer(), targetLss.Pointer(), state.CacheByShard(pgr.shardID).cPointer, @@ -1202,7 +1217,6 @@ func (pgr *PerGoroutineRelabeler) InputRelabeling( shardsRelabeledSeries, ) runtime.KeepAlive(pgr) - runtime.KeepAlive(statelessRelabeler) runtime.KeepAlive(inputLss) runtime.KeepAlive(targetLss) runtime.KeepAlive(state) @@ -1250,7 +1264,6 @@ func (pgr *PerGoroutineRelabeler) InputRelabelingFromCache( // InputRelabelingWithStalenans relabeling incoming hashdex(first stage) with state stalenans. func (pgr *PerGoroutineRelabeler) InputRelabelingWithStalenans( ctx context.Context, - statelessRelabeler *StatelessRelabeler, inputLss *LabelSetStorage, targetLss *LabelSetStorage, state *State, @@ -1268,7 +1281,7 @@ func (pgr *PerGoroutineRelabeler) InputRelabelingWithStalenans( } stats, exception, hasReallocations := prometheusPerGoroutineRelabelerInputRelabelingWithStalenans( pgr.cptr, - statelessRelabeler.Pointer(), + state.StatelessRelabeler().Pointer(), inputLss.Pointer(), targetLss.Pointer(), state.CacheByShard(pgr.shardID).cPointer, @@ -1280,7 +1293,6 @@ func (pgr *PerGoroutineRelabeler) InputRelabelingWithStalenans( shardsRelabeledSeries, ) runtime.KeepAlive(pgr) - runtime.KeepAlive(statelessRelabeler) runtime.KeepAlive(inputLss) runtime.KeepAlive(targetLss) runtime.KeepAlive(state) diff --git a/pp/go/cppbridge/prometheus_relabeler_test.go b/pp/go/cppbridge/prometheus_relabeler_test.go index a98e3abd6b..ee56df001b 100644 --- a/pp/go/cppbridge/prometheus_relabeler_test.go +++ b/pp/go/cppbridge/prometheus_relabeler_test.go @@ -652,11 +652,11 @@ func (s *RelabelerSuite) TestInputPerGoroutineRelabeler() { shardsRelabeledSeries := cppbridge.NewShardsRelabeledSeries(numberOfShards) state := cppbridge.NewState(numberOfShards) state.SetRelabelerOptions(&s.options) + state.SetStatelessRelabeler(statelessRelabeler) pgr := cppbridge.NewPerGoroutineRelabeler(numberOfShards, 0) stats, hasReallocations, err := pgr.InputRelabeling( s.baseCtx, - statelessRelabeler, inputLss, targetLss, state, @@ -711,11 +711,11 @@ func (s *RelabelerSuite) TestInputPerGoroutineRelabelerFromCacheTrue() { shardsRelabeledSeries := cppbridge.NewShardsRelabeledSeries(numberOfShards) state := cppbridge.NewState(numberOfShards) state.SetRelabelerOptions(&s.options) + state.SetStatelessRelabeler(statelessRelabeler) pgr := cppbridge.NewPerGoroutineRelabeler(numberOfShards, 0) stats, hasReallocations, err := pgr.InputRelabeling( s.baseCtx, - statelessRelabeler, inputLss, targetLss, state, @@ -962,11 +962,11 @@ func (s *RelabelerSuite) TestInputPerGoroutineRelabelerFromCachePartially() { shardsRelabeledSeries := cppbridge.NewShardsRelabeledSeries(numberOfShards) state := cppbridge.NewState(numberOfShards) state.SetRelabelerOptions(&s.options) + state.SetStatelessRelabeler(statelessRelabeler) pgr := cppbridge.NewPerGoroutineRelabeler(numberOfShards, 0) stats, hasReallocations, err := pgr.InputRelabeling( s.baseCtx, - statelessRelabeler, inputLss, targetLss, state, @@ -994,7 +994,6 @@ func (s *RelabelerSuite) TestInputPerGoroutineRelabelerFromCachePartially() { stats, _, err = pgr.InputRelabeling( s.baseCtx, - statelessRelabeler, inputLss, targetLss, state, diff --git a/pp/go/storage/appender/appender.go b/pp/go/storage/appender/appender.go index 8d06acc229..23e26d47d3 100644 --- a/pp/go/storage/appender/appender.go +++ b/pp/go/storage/appender/appender.go @@ -229,7 +229,6 @@ func (a *Appender[TGenericTask, TDataStorage, TLSS, TShard, THead]) Append( func (a *Appender[TGenericTask, TDataStorage, TLSS, TShard, THead]) inputRelabelingStage( ctx context.Context, state *cppbridge.State, - rd *RelabelerData, incomingData *DestructibleIncomingData, shardedInnerSeries *ShardedInnerSeries, shardedRelabeledSeries *ShardedRelabeledSeries, From 85b2b0d73f4a6b2aea3eebd9a292092da880213c Mon Sep 17 00:00:00 2001 From: Alexandr Yudin Date: Tue, 26 Aug 2025 08:23:42 +0000 Subject: [PATCH 16/96] add catalog appender --- pp/entrypoint/prometheus_relabeler.cpp | 23 +- pp/entrypoint/prometheus_relabeler.h | 12 +- pp/go/cppbridge/entrypoint.go | 26 +- pp/go/cppbridge/entrypoint.h | 12 +- pp/go/cppbridge/prometheus_relabeler.go | 121 +++++++- pp/go/storage/appender/appender.go | 222 ++++++++------ pp/go/storage/catalog/catalog.go | 337 ++++++++++++++++++++++ pp/go/storage/catalog/catalog_test.go | 160 ++++++++++ pp/go/storage/catalog/decoder.go | 314 ++++++++++++++++++++ pp/go/storage/catalog/decoder_test.go | 80 +++++ pp/go/storage/catalog/encoder.go | 249 ++++++++++++++++ pp/go/storage/catalog/gc.go | 107 +++++++ pp/go/storage/catalog/log.go | 296 +++++++++++++++++++ pp/go/storage/catalog/log_file_handler.go | 102 +++++++ pp/go/storage/catalog/log_migrate.go | 286 ++++++++++++++++++ pp/go/storage/catalog/log_test.go | 102 +++++++ pp/go/storage/catalog/record.go | 210 ++++++++++++++ pp/go/storage/catalog/record_test.go | 27 ++ pp/go/storage/catalog/testdata/headv1.log | Bin 0 -> 353 bytes pp/go/storage/catalog/testdata/headv2.log | Bin 0 -> 146 bytes pp/go/storage/catalog/testdata/headv3.log | Bin 0 -> 208 bytes pp/go/storage/head/builder/builder.go | 2 + pp/go/storage/head/shard/shard.go | 20 +- pp/go/storage/head/shard/wal/wal.go | 1 + pp/go/storage/querier/interface.go | 114 ++++++++ pp/go/storage/querier/querier.go | 106 ------- pp/prometheus/relabeler.h | 14 +- 27 files changed, 2706 insertions(+), 237 deletions(-) create mode 100644 pp/go/storage/catalog/catalog_test.go create mode 100644 pp/go/storage/catalog/decoder.go create mode 100644 pp/go/storage/catalog/decoder_test.go create mode 100644 pp/go/storage/catalog/encoder.go create mode 100644 pp/go/storage/catalog/gc.go create mode 100644 pp/go/storage/catalog/log.go create mode 100644 pp/go/storage/catalog/log_file_handler.go create mode 100644 pp/go/storage/catalog/log_migrate.go create mode 100644 pp/go/storage/catalog/log_test.go create mode 100644 pp/go/storage/catalog/record.go create mode 100644 pp/go/storage/catalog/record_test.go create mode 100644 pp/go/storage/catalog/testdata/headv1.log create mode 100644 pp/go/storage/catalog/testdata/headv2.log create mode 100644 pp/go/storage/catalog/testdata/headv3.log create mode 100644 pp/go/storage/querier/interface.go diff --git a/pp/entrypoint/prometheus_relabeler.cpp b/pp/entrypoint/prometheus_relabeler.cpp index 538a3c6eea..aaf88a408a 100644 --- a/pp/entrypoint/prometheus_relabeler.cpp +++ b/pp/entrypoint/prometheus_relabeler.cpp @@ -537,12 +537,31 @@ extern "C" void prompp_prometheus_cache_allocated_memory(void* args, void* res) new (res) Result{.allocated_memory = in->cache->allocated_memory()}; } -extern "C" void prompp_prometheus_cache_reset_to(void* args) { +extern "C" void prompp_prometheus_cache_update(void* args, void* res) { struct Arguments { + PromPP::Primitives::Go::SliceView shards_relabeler_state_update; CachePtr cache; + uint16_t relabeled_shard_id; + }; + struct Result { + PromPP::Primitives::Go::Slice error; }; - static_cast(args)->cache->reset(); + const auto* in = static_cast(args); + + try { + for (size_t id = 0; id != in->shards_relabeler_state_update.size(); ++id) { + if (in->shards_relabeler_state_update[id] == nullptr || in->shards_relabeler_state_update[id]->size() == 0) { + continue; + } + + in->cache->update(in->shards_relabeler_state_update[id], id); + } + } catch (...) { + auto* out = new (res) Result(); + auto err_stream = PromPP::Primitives::Go::BytesStream(&out->error); + entrypoint::handle_current_exception(err_stream); + } } // diff --git a/pp/entrypoint/prometheus_relabeler.h b/pp/entrypoint/prometheus_relabeler.h index 3e71838121..a075ca4a0f 100644 --- a/pp/entrypoint/prometheus_relabeler.h +++ b/pp/entrypoint/prometheus_relabeler.h @@ -399,13 +399,19 @@ void prompp_prometheus_cache_dtor(void* args); void prompp_prometheus_cache_allocated_memory(void* args, void* res); /** - * @brief reset cache and store lss generation. + * @brief add to cache relabled data(third stage). * * @param args { - * cache uintptr // pointer to constructed Cache; + * shards_relabeler_state_update []*RelabelerStateUpdate // pointer to RelabelerStateUpdate per source shard; + * cache uintptr // pointer to constructed Cache; + * relabeled_shard_id uint16 // relabeled shard id; + * } + * + * @param res { + * error []byte // error string if thrown; * } */ -void prompp_prometheus_cache_reset_to(void* args); +void prompp_prometheus_cache_update(void* args, void* res); // // PerGoroutineRelabeler diff --git a/pp/go/cppbridge/entrypoint.go b/pp/go/cppbridge/entrypoint.go index 4c85712f6a..77fdd3a1e8 100644 --- a/pp/go/cppbridge/entrypoint.go +++ b/pp/go/cppbridge/entrypoint.go @@ -2621,17 +2621,29 @@ func prometheusCacheAllocatedMemory(cache uintptr) uint64 { return res.cacheAllocatedMemory } -// prometheusCacheResetTo reset cache. -func prometheusCacheResetTo(cache uintptr) { +// prometheusCacheUpdate add to cache relabled data(third stage). +func prometheusCacheUpdate( + shardsRelabelerStateUpdate []*RelabelerStateUpdate, + cache uintptr, +) []byte { args := struct { - cache uintptr - }{cache} - + relabelerStateUpdates []*RelabelerStateUpdate + cache uintptr + }{shardsRelabelerStateUpdate, cache} + var res struct { + exception []byte + } + start := time.Now().UnixNano() testGC() - fastcgo.UnsafeCall1( - C.prompp_prometheus_cache_reset_to, + fastcgo.UnsafeCall2( + C.prompp_prometheus_cache_update, uintptr(unsafe.Pointer(&args)), + uintptr(unsafe.Pointer(&res)), ) + inputRelabelerUpdateRelabelerStateSum.Add(float64(time.Now().UnixNano() - start)) + inputRelabelerUpdateRelabelerStateCount.Inc() + + return res.exception } func headWalEncoderCtor(shardID uint16, logShards uint8, lss uintptr) uintptr { diff --git a/pp/go/cppbridge/entrypoint.h b/pp/go/cppbridge/entrypoint.h index 7f9f2a2507..0a9cbc9538 100755 --- a/pp/go/cppbridge/entrypoint.h +++ b/pp/go/cppbridge/entrypoint.h @@ -1077,13 +1077,19 @@ void prompp_prometheus_cache_dtor(void* args); void prompp_prometheus_cache_allocated_memory(void* args, void* res); /** - * @brief reset cache and store lss generation. + * @brief add to cache relabled data(third stage). * * @param args { - * cache uintptr // pointer to constructed Cache; + * shards_relabeler_state_update []*RelabelerStateUpdate // pointer to RelabelerStateUpdate per source shard; + * cache uintptr // pointer to constructed Cache; + * relabeled_shard_id uint16 // relabeled shard id; + * } + * + * @param res { + * error []byte // error string if thrown; * } */ -void prompp_prometheus_cache_reset_to(void* args); +void prompp_prometheus_cache_update(void* args, void* res); // // PerGoroutineRelabeler diff --git a/pp/go/cppbridge/prometheus_relabeler.go b/pp/go/cppbridge/prometheus_relabeler.go index 88b6104130..a4425c8592 100644 --- a/pp/go/cppbridge/prometheus_relabeler.go +++ b/pp/go/cppbridge/prometheus_relabeler.go @@ -9,6 +9,7 @@ import ( "runtime" "strconv" "strings" + "sync" "time" "github.com/cespare/xxhash/v2" @@ -536,6 +537,22 @@ type RelabelerStats struct { SeriesDrop uint32 } +// Add another stats. +func (rs *RelabelerStats) Add(stat RelabelerStats) { + rs.SamplesAdded += stat.SamplesAdded + rs.SeriesAdded += stat.SeriesAdded + rs.SeriesDrop += stat.SeriesDrop +} + +// Adds another slice stats. +func (rs *RelabelerStats) Adds(stats []RelabelerStats) { + for _, s := range stats { + rs.SamplesAdded += s.SamplesAdded + rs.SeriesAdded += s.SeriesAdded + rs.SeriesDrop += s.SeriesDrop + } +} + // String serialize to string. func (rs RelabelerStats) String() string { return fmt.Sprintf( @@ -924,6 +941,7 @@ func (opsr *OutputPerShardRelabeler) UpdateRelabelerState( // cPointer - pointer to C-Cache; type Cache struct { cPointer uintptr + lock sync.RWMutex } // NewCache init new Cache. @@ -939,15 +957,25 @@ func NewCache() *Cache { // AllocatedMemory return size of allocated memory for caches. func (c *Cache) AllocatedMemory() uint64 { + c.lock.RLock() res := prometheusCacheAllocatedMemory(c.cPointer) + c.lock.RUnlock() runtime.KeepAlive(c) return res } -// ResetTo reset cache. -func (c *Cache) ResetTo() { - prometheusCacheResetTo(c.cPointer) +// Update add to cache relabled data(third stage). +func (c *Cache) Update(ctx context.Context, shardsRelabelerStateUpdate []*RelabelerStateUpdate) error { + if ctx.Err() != nil { + return ctx.Err() + } + + c.lock.Lock() + exception := prometheusCacheUpdate(shardsRelabelerStateUpdate, c.cPointer) + c.lock.Unlock() runtime.KeepAlive(c) + + return handleException(exception) } // @@ -1205,17 +1233,21 @@ func (pgr *PerGoroutineRelabeler) InputRelabeling( return RelabelerStats{}, false, ErrMustImplementCptrable } + cache := state.CacheByShard(pgr.shardID) + cache.lock.Lock() stats, exception, hasReallocations := prometheusPerGoroutineRelabelerInputRelabeling( pgr.cptr, state.StatelessRelabeler().Pointer(), inputLss.Pointer(), targetLss.Pointer(), - state.CacheByShard(pgr.shardID).cPointer, + cache.cPointer, cptrContainer.cptr(), state.RelabelerOptions(), shardsInnerSeries, shardsRelabeledSeries, ) + cache.lock.Unlock() + runtime.KeepAlive(pgr) runtime.KeepAlive(inputLss) runtime.KeepAlive(targetLss) @@ -1243,15 +1275,19 @@ func (pgr *PerGoroutineRelabeler) InputRelabelingFromCache( return RelabelerStats{}, false, ErrMustImplementCptrable } + cache := state.CacheByShard(pgr.shardID) + cache.lock.RLock() stats, exception, ok := prometheusPerGoroutineRelabelerInputRelabelingFromCache( pgr.cptr, inputLss.Pointer(), targetLss.Pointer(), - state.CacheByShard(pgr.shardID).cPointer, + cache.cPointer, cptrContainer.cptr(), state.RelabelerOptions(), shardsInnerSeries, ) + cache.lock.RUnlock() + runtime.KeepAlive(pgr) runtime.KeepAlive(inputLss) runtime.KeepAlive(targetLss) @@ -1279,12 +1315,15 @@ func (pgr *PerGoroutineRelabeler) InputRelabelingWithStalenans( if !ok { return RelabelerStats{}, false, ErrMustImplementCptrable } + + cache := state.CacheByShard(pgr.shardID) + cache.lock.Lock() stats, exception, hasReallocations := prometheusPerGoroutineRelabelerInputRelabelingWithStalenans( pgr.cptr, state.StatelessRelabeler().Pointer(), inputLss.Pointer(), targetLss.Pointer(), - state.CacheByShard(pgr.shardID).cPointer, + cache.cPointer, cptrContainer.cptr(), state.StaleNansStateByShard(pgr.shardID).state, state.DefTimestamp(), @@ -1292,6 +1331,8 @@ func (pgr *PerGoroutineRelabeler) InputRelabelingWithStalenans( shardsInnerSeries, shardsRelabeledSeries, ) + cache.lock.Unlock() + runtime.KeepAlive(pgr) runtime.KeepAlive(inputLss) runtime.KeepAlive(targetLss) @@ -1319,17 +1360,20 @@ func (pgr *PerGoroutineRelabeler) InputRelabelingWithStalenansFromCache( return RelabelerStats{}, false, ErrMustImplementCptrable } + cache := state.CacheByShard(pgr.shardID) + cache.lock.RLock() stats, exception, ok := prometheusPerGoroutineRelabelerInputRelabelingWithStalenansFromCache( pgr.cptr, inputLss.Pointer(), targetLss.Pointer(), - state.CacheByShard(pgr.shardID).cPointer, + cache.cPointer, cptrContainer.cptr(), state.StaleNansStateByShard(pgr.shardID).state, state.DefTimestamp(), state.RelabelerOptions(), shardsInnerSeries, ) + cache.lock.RUnlock() runtime.KeepAlive(pgr) runtime.KeepAlive(inputLss) @@ -1340,6 +1384,69 @@ func (pgr *PerGoroutineRelabeler) InputRelabelingWithStalenansFromCache( return stats, ok, handleException(exception) } +// Relabeling relabeling incoming hashdex(first stage). +func (pgr *PerGoroutineRelabeler) Relabeling( + ctx context.Context, + inputLss *LabelSetStorage, + targetLss *LabelSetStorage, + state *State, + shardedData ShardedData, + shardsInnerSeries []*InnerSeries, + shardsRelabeledSeries []*RelabeledSeries, +) (RelabelerStats, bool, error) { + if state.TrackStaleness() { + return pgr.InputRelabelingWithStalenans( + ctx, + inputLss, + targetLss, + state, + shardedData, + shardsInnerSeries, + shardsRelabeledSeries, + ) + } + + return pgr.InputRelabeling( + ctx, + inputLss, + targetLss, + state, + shardedData, + shardsInnerSeries, + shardsRelabeledSeries, + ) +} + +// RelabelingFromCache relabeling incoming hashdex(first stage) from cache. +func (pgr *PerGoroutineRelabeler) RelabelingFromCache( + ctx context.Context, + inputLss *LabelSetStorage, + targetLss *LabelSetStorage, + state *State, + shardedData ShardedData, + shardsInnerSeries []*InnerSeries, +) (RelabelerStats, bool, error) { + if state.TrackStaleness() { + return pgr.InputRelabelingWithStalenansFromCache( + ctx, + inputLss, + targetLss, + state, + shardedData, + shardsInnerSeries, + ) + } + + return pgr.InputRelabelingFromCache( + ctx, + inputLss, + targetLss, + state, + shardedData, + shardsInnerSeries, + ) +} + // UpdateRelabelerState add to cache relabled data(third stage). func (pgr *PerGoroutineRelabeler) UpdateRelabelerState( ctx context.Context, diff --git a/pp/go/storage/appender/appender.go b/pp/go/storage/appender/appender.go index 23e26d47d3..1e72c9f7ce 100644 --- a/pp/go/storage/appender/appender.go +++ b/pp/go/storage/appender/appender.go @@ -6,8 +6,8 @@ import ( "sync/atomic" "github.com/prometheus/prometheus/pp/go/cppbridge" - "github.com/prometheus/prometheus/pp/go/relabeler" "github.com/prometheus/prometheus/pp/go/storage" + "github.com/prometheus/prometheus/pp/go/storage/head/task" "github.com/prometheus/prometheus/pp/go/storage/logger" ) @@ -17,6 +17,8 @@ const ( // LSSInputRelabeling name of task. LSSInputRelabeling = "lss_input_relabeling" + // LSSAppendRelabelerSeries name of task. + LSSAppendRelabelerSeries = "lss_append_relabeler_series" // WalWrite name of task. WalWrite = "wal_write" @@ -41,7 +43,8 @@ type GenericTask interface { // DataStorage the minimum required [DataStorage] implementation. type DataStorage interface { - // TODO + // AppendInnerSeriesSlice add InnerSeries to storage. + AppendInnerSeriesSlice(innerSeriesSlice []*cppbridge.InnerSeries) } // @@ -60,12 +63,26 @@ type LSS interface { ResetSnapshot() } +// +// LSS +// + +// Wal the minimum required Wal implementation for a [Shard]. +type Wal interface { + // Commit finalize segment from encoder and write to wal. + // It is necessary to lock the LSS for reading for the commit. + Commit() error + + // Write append the incoming inner series to wal encoder. + Write(innerSeriesSlice []*cppbridge.InnerSeries) (bool, error) +} + // // Shard // // Shard the minimum required head [Shard] implementation. -type Shard[TDataStorage DataStorage, TLSS LSS] interface { +type Shard[TDataStorage DataStorage, TLSS LSS, TWal Wal] interface { // DataStorage returns shard [DataStorage]. DataStorage() TDataStorage @@ -77,6 +94,9 @@ type Shard[TDataStorage DataStorage, TLSS LSS] interface { // ShardID returns the shard ID. ShardID() uint16 + + // Wal returns write-ahead log. + Wal() TWal } // @@ -88,7 +108,8 @@ type Head[ TGenericTask GenericTask, TDataStorage DataStorage, TLSS LSS, - TShard Shard[TDataStorage, TLSS], + TWal Wal, + TShard Shard[TDataStorage, TLSS, TWal], ] interface { // CreateTask create a task for operations on the [Head] shards. CreateTask(taskName string, shardFn func(shard TShard) error) TGenericTask @@ -100,18 +121,23 @@ type Head[ NumberOfShards() uint16 } +// +// Appender +// + type Appender[ TGenericTask GenericTask, TDataStorage DataStorage, TLSS LSS, - TShard Shard[TDataStorage, TLSS], - THead Head[TGenericTask, TDataStorage, TLSS, TShard], + TWal Wal, + TShard Shard[TDataStorage, TLSS, TWal], + THead Head[TGenericTask, TDataStorage, TLSS, TWal, TShard], ] struct { head THead } // Append incoming data to head. -func (a *Appender[TGenericTask, TDataStorage, TLSS, TShard, THead]) Append( +func (a *Appender[TGenericTask, TDataStorage, TLSS, TWal, TShard, THead]) Append( ctx context.Context, incomingData *storage.IncomingData, incomingState *cppbridge.State, @@ -123,27 +149,27 @@ func (a *Appender[TGenericTask, TDataStorage, TLSS, TShard, THead]) Append( // return nil, cppbridge.RelabelerStats{}, err // } + // TODO ? + var state *cppbridge.State + shardedInnerSeries := NewShardedInnerSeries(a.head.NumberOfShards()) shardedRelabeledSeries := NewShardedRelabeledSeries(a.head.NumberOfShards()) stats, err := a.inputRelabelingStage( ctx, state, - rd, NewDestructibleIncomingData(incomingData, int(a.head.NumberOfShards())), shardedInnerSeries, shardedRelabeledSeries, ) if err != nil { - // reset msr.rotateWG on error return nil, stats, fmt.Errorf("failed input relabeling stage: %w", err) } if !shardedRelabeledSeries.IsEmpty() { shardedStateUpdates := NewShardedStateUpdates(a.head.NumberOfShards()) - if err = h.appendRelabelerSeriesStage( + if err = a.appendRelabelerSeriesStage( ctx, - rd, shardedInnerSeries, shardedRelabeledSeries, shardedStateUpdates, @@ -151,38 +177,32 @@ func (a *Appender[TGenericTask, TDataStorage, TLSS, TShard, THead]) Append( return nil, stats, fmt.Errorf("failed append relabeler series stage: %w", err) } - if err = h.updateRelabelerStateStage( + if err = a.updateRelabelerStateStage( ctx, state, - rd, shardedStateUpdates, ); err != nil { return nil, stats, fmt.Errorf("failed update relabeler stage: %w", err) } } - tw := relabeler.NewTaskWaiter(2) + tw := task.NewTaskWaiter[TGenericTask](2) - tAppend := h.CreateTask( + tAppend := a.head.CreateTask( DSAppendInnerSeries, - func(shard relabeler.Shard) error { - shard.DataStorageLock() + func(shard TShard) error { shard.DataStorage().AppendInnerSeriesSlice(shardedInnerSeries.DataByShard(shard.ShardID())) - shard.DataStorageUnlock() return nil }, - relabeler.ForDataStorageTask, ) - h.Enqueue(tAppend) + a.head.Enqueue(tAppend) var atomiclimitExhausted uint32 - tWalWrite := h.CreateTask( + tWalWrite := a.head.CreateTask( WalWrite, - func(shard relabeler.Shard) error { - shard.LSSLock() + func(shard TShard) error { limitExhausted, errWrite := shard.Wal().Write(shardedInnerSeries.DataByShard(shard.ShardID())) - shard.LSSUnlock() if errWrite != nil { return fmt.Errorf("shard %d: %w", shard.ShardID(), errWrite) } @@ -193,9 +213,8 @@ func (a *Appender[TGenericTask, TDataStorage, TLSS, TShard, THead]) Append( return nil }, - relabeler.ForLSSTask, ) - h.Enqueue(tWalWrite) + a.head.Enqueue(tWalWrite) tw.Add(tAppend) tw.Add(tWalWrite) @@ -205,17 +224,16 @@ func (a *Appender[TGenericTask, TDataStorage, TLSS, TShard, THead]) Append( } if commitToWal || atomiclimitExhausted > 0 { - t := h.CreateTask( + t := a.head.CreateTask( WalCommit, - func(shard relabeler.Shard) error { - shard.LSSLock() - defer shard.LSSUnlock() - - return shard.Wal().Commit() + func(shard TShard) error { + // It is necessary to lock the LSS for reading for the commit. + return shard.LSS().WithRLock(func(_, _ *cppbridge.LabelSetStorage) error { + return shard.Wal().Commit() + }) }, - relabeler.ForLSSTask, ) - h.Enqueue(t) + a.head.Enqueue(t) if err := t.Wait(); err != nil { logger.Errorf("failed to commit wal: %v", err) @@ -226,7 +244,7 @@ func (a *Appender[TGenericTask, TDataStorage, TLSS, TShard, THead]) Append( } // inputRelabelingStage first stage - relabeling. -func (a *Appender[TGenericTask, TDataStorage, TLSS, TShard, THead]) inputRelabelingStage( +func (a *Appender[TGenericTask, TDataStorage, TLSS, TWal, TShard, THead]) inputRelabelingStage( ctx context.Context, state *cppbridge.State, incomingData *DestructibleIncomingData, @@ -245,25 +263,14 @@ func (a *Appender[TGenericTask, TDataStorage, TLSS, TShard, THead]) inputRelabel ) if err := lss.WithRLock(func(target, input *cppbridge.LabelSetStorage) (rErr error) { - if state.TrackStaleness() { - stats[shardID], ok, rErr = relabeler.InputRelabelingWithStalenansFromCache( - ctx, - input, - target, - state, - incomingData.ShardedData(), - shardedInnerSeries.DataBySourceShard(shardID), - ) - } else { - stats[shardID], ok, rErr = relabeler.InputRelabelingFromCache( - ctx, - input, - target, - state, - incomingData.ShardedData(), - shardedInnerSeries.DataBySourceShard(shardID), - ) - } + stats[shardID], ok, rErr = relabeler.RelabelingFromCache( + ctx, + input, + target, + state, + incomingData.ShardedData(), + shardedInnerSeries.DataBySourceShard(shardID), + ) return rErr }); err != nil { @@ -281,30 +288,18 @@ func (a *Appender[TGenericTask, TDataStorage, TLSS, TShard, THead]) inputRelabel rstats = cppbridge.RelabelerStats{} ) err := lss.WithLock(func(target, input *cppbridge.LabelSetStorage) (rErr error) { - if state.TrackStaleness() { - rstats, hasReallocations, rErr = relabeler.InputRelabelingWithStalenans( - ctx, - input, - target, - state, - incomingData.ShardedData(), - shardedInnerSeries.DataBySourceShard(shardID), - shardedRelabeledSeries.DataByShard(shardID), - ) - } else { - rstats, hasReallocations, rErr = relabeler.InputRelabeling( - ctx, - input, - target, - state, - incomingData.ShardedData(), - shardedInnerSeries.DataBySourceShard(shardID), - shardedRelabeledSeries.DataByShard(shardID), - ) - } + rstats, hasReallocations, rErr = relabeler.Relabeling( + ctx, + input, + target, + state, + incomingData.ShardedData(), + shardedInnerSeries.DataBySourceShard(shardID), + shardedRelabeledSeries.DataByShard(shardID), + ) if hasReallocations { - shard.LSS().ResetSnapshot() + lss.ResetSnapshot() } return rErr @@ -315,9 +310,7 @@ func (a *Appender[TGenericTask, TDataStorage, TLSS, TShard, THead]) inputRelabel return fmt.Errorf("shard %d: %w", shardID, err) } - stats[shardID].SamplesAdded += rstats.SamplesAdded - stats[shardID].SeriesAdded += rstats.SeriesAdded - stats[shardID].SeriesDrop += rstats.SeriesDrop + stats[shardID].Add(rstats) return nil }, @@ -329,11 +322,72 @@ func (a *Appender[TGenericTask, TDataStorage, TLSS, TShard, THead]) inputRelabel return resStats, err } - for _, s := range stats { - resStats.SamplesAdded += s.SamplesAdded - resStats.SeriesAdded += s.SeriesAdded - resStats.SeriesDrop += s.SeriesDrop - } + resStats.Adds(stats) return resStats, nil } + +// appendRelabelerSeriesStage second stage - append to lss relabeling ls. +func (a *Appender[TGenericTask, TDataStorage, TLSS, TWal, TShard, THead]) appendRelabelerSeriesStage( + ctx context.Context, + shardedInnerSeries *ShardedInnerSeries, + shardedRelabeledSeries *ShardedRelabeledSeries, + shardedStateUpdates *ShardedStateUpdates, +) error { + t := a.head.CreateTask( + LSSAppendRelabelerSeries, + func(shard TShard) error { + shardID := shard.ShardID() + + relabeledSeries, ok := shardedRelabeledSeries.DataBySourceShard(shardID) + if !ok { + return nil + } + + lss := shard.LSS() + + return lss.WithLock(func(target, _ *cppbridge.LabelSetStorage) error { + hasReallocations, err := shard.Relabeler().AppendRelabelerSeries( + ctx, + target, + shardedInnerSeries.DataByShard(shardID), + relabeledSeries, + shardedStateUpdates.DataByShard(shardID), + ) + if err != nil { + return fmt.Errorf("shard %d: %w", shardID, err) + } + + if hasReallocations { + lss.ResetSnapshot() + } + + return nil + }) + }, + ) + a.head.Enqueue(t) + + return t.Wait() +} + +// updateRelabelerStateStage third stage - update state cache. +func (a *Appender[TGenericTask, TDataStorage, TLSS, TWal, TShard, THead]) updateRelabelerStateStage( + ctx context.Context, + state *cppbridge.State, + shardedStateUpdates *ShardedStateUpdates, +) error { + numberOfShards := a.head.NumberOfShards() + for shardID := uint16(0); shardID < numberOfShards; shardID++ { + updates, ok := shardedStateUpdates.DataBySourceShard(shardID) + if !ok { + continue + } + + if err := state.CacheByShard(shardID).Update(ctx, updates); err != nil { + return fmt.Errorf("shard %d: %w", shardID, err) + } + } + + return nil +} diff --git a/pp/go/storage/catalog/catalog.go b/pp/go/storage/catalog/catalog.go index e571e24c61..99570c3e6b 100644 --- a/pp/go/storage/catalog/catalog.go +++ b/pp/go/storage/catalog/catalog.go @@ -1 +1,338 @@ package catalog + +import ( + "errors" + "fmt" + "io" + "sort" + "sync" + + "github.com/google/uuid" + "github.com/jonboulle/clockwork" + "github.com/prometheus/client_golang/prometheus" + "github.com/prometheus/prometheus/pp/go/storage/logger" + "github.com/prometheus/prometheus/pp/go/util" +) + +const ( + // DefaultMaxLogFileSize default size of log file. + DefaultMaxLogFileSize = 4 << 20 + + // compactErr format string for compact error. + compactErr = "compact: %w" + // logWriteErr format string for log write error. + logWriteErr = "log write: %w" + // notFoundErr format string for not found id error. + notFoundErr = "not found: %s" +) + +// +// Log +// + +// Log head-log file, contains [Record]s of heads. +type Log interface { + // ReWrite rewrite [FileLog] with [Record]s. + ReWrite(records ...*Record) error + + // Read [Record] from [FileLog]. + Read(record *Record) error + + // Size return current size of [FileHandler]. + Size() int + + // Write [Record] to [FileLog]. + Write(record *Record) error +} + +// +// IDGenerator +// + +// IDGenerator generator UUID. +type IDGenerator interface { + // Generate UUID. + Generate() uuid.UUID +} + +// DefaultIDGenerator default generator UUID. +type DefaultIDGenerator struct{} + +// Generate UUID. +func (DefaultIDGenerator) Generate() uuid.UUID { + return uuid.New() +} + +// +// Catalog +// + +// Catalog of current head records. +type Catalog struct { + mtx sync.Mutex + clock clockwork.Clock + log Log + idGenerator IDGenerator + records map[string]*Record + maxLogFileSize int + corruptedHead prometheus.Counter + activeHeadCreatedAt prometheus.Gauge +} + +// New init new [Catalog]. +func New( + clock clockwork.Clock, + log Log, + idGenerator IDGenerator, + maxLogFileSize int, + registerer prometheus.Registerer, +) (*Catalog, error) { + factory := util.NewUnconflictRegisterer(registerer) + catalog := &Catalog{ + clock: clock, + log: log, + idGenerator: idGenerator, + records: make(map[string]*Record), + maxLogFileSize: maxLogFileSize, + corruptedHead: factory.NewCounter( + prometheus.CounterOpts{ + Name: "prompp_head_catalog_corrupted_head_total", + Help: "Total number of corrupted heads.", + }, + ), + activeHeadCreatedAt: factory.NewGauge( + prometheus.GaugeOpts{ + Name: "prompp_head_catalog_active_head_created_at", + Help: "The time when the active head was created.", + }, + ), + } + + if err := catalog.sync(); err != nil { + return nil, fmt.Errorf("failed to sync catalog: %w", err) + } + + return catalog, nil +} + +// Compact catalog. +func (c *Catalog) Compact() error { + c.mtx.Lock() + defer c.mtx.Unlock() + return c.compactLog() +} + +// Create creates new [Record] and write to [Log]. +func (c *Catalog) Create(numberOfShards uint16) (r *Record, err error) { + c.mtx.Lock() + defer c.mtx.Unlock() + + if err = c.compactIfNeeded(); err != nil { + return nil, fmt.Errorf(compactErr, err) + } + + id := c.idGenerator.Generate() + now := c.clock.Now().UnixMilli() + r = &Record{ + id: id, + numberOfShards: numberOfShards, + createdAt: now, + updatedAt: now, + deletedAt: 0, + referenceCount: 0, + status: StatusNew, + } + + if err = c.log.Write(r); err != nil { + return r, fmt.Errorf(logWriteErr, err) + } + c.records[id.String()] = r + + return r, nil +} + +// Delete record by ID. +func (c *Catalog) Delete(id string) (err error) { + c.mtx.Lock() + defer c.mtx.Unlock() + + if err = c.compactIfNeeded(); err != nil { + return fmt.Errorf(compactErr, err) + } + + r, ok := c.records[id] + if !ok || r.deletedAt > 0 { + return nil + } + + changed := createRecordCopy(r) + changed.deletedAt = c.clock.Now().UnixMilli() + changed.updatedAt = r.deletedAt + + if err = c.log.Write(changed); err != nil { + return fmt.Errorf(logWriteErr, err) + } + + applyRecordChanges(r, changed) + delete(c.records, r.id.String()) + + return nil +} + +// Get returns [Record] if exist. +func (c *Catalog) Get(id string) (*Record, error) { + c.mtx.Lock() + defer c.mtx.Unlock() + + r, ok := c.records[id] + if !ok { + return nil, fmt.Errorf(notFoundErr, id) + } + + return r, nil +} + +// List returns slice of records with filter and sort. +func (c *Catalog) List( + filterFn func(record *Record) bool, + sortLess func(lhs, rhs *Record) bool, +) (records []*Record, err error) { + c.mtx.Lock() + defer c.mtx.Unlock() + records = make([]*Record, 0, len(c.records)) + for _, record := range c.records { + if filterFn != nil && !filterFn(record) { + continue + } + records = append(records, record) + } + + if sortLess != nil { + sort.Slice(records, func(i, j int) bool { + return sortLess(records[i], records[j]) + }) + } + + return records, nil +} + +// OnDiskSize size of [Log] file on disk. +func (c *Catalog) OnDiskSize() int64 { + return int64(c.log.Size()) +} + +// SetCorrupted set corrupted flag for ID and returns [Record] if exist. +func (c *Catalog) SetCorrupted(id string) (_ *Record, err error) { + c.mtx.Lock() + defer c.mtx.Unlock() + + if err = c.compactIfNeeded(); err != nil { + return nil, fmt.Errorf(compactErr, err) + } + + r, ok := c.records[id] + if !ok { + return nil, fmt.Errorf(notFoundErr, id) + } + + if r.corrupted { + return r, nil + } + + changed := createRecordCopy(r) + changed.corrupted = true + changed.updatedAt = c.clock.Now().UnixMilli() + + if err = c.log.Write(changed); err != nil { + return r, fmt.Errorf(logWriteErr, err) + } + + applyRecordChanges(r, changed) + c.records[id] = r + + c.corruptedHead.Inc() + + return r, nil +} + +// SetStatus set status for ID and returns [Record] if exist. +func (c *Catalog) SetStatus(id string, status Status) (_ *Record, err error) { + c.mtx.Lock() + defer c.mtx.Unlock() + + if err = c.compactIfNeeded(); err != nil { + return nil, fmt.Errorf(compactErr, err) + } + + r, ok := c.records[id] + if !ok { + return nil, fmt.Errorf(notFoundErr, id) + } + + if r.status == status { + if status == StatusActive { + c.activeHeadCreatedAt.Set(float64(r.createdAt)) + } + + return r, nil + } + + changed := createRecordCopy(r) + changed.status = status + changed.updatedAt = c.clock.Now().UnixMilli() + + if err = c.log.Write(changed); err != nil { + return r, fmt.Errorf(logWriteErr, err) + } + + applyRecordChanges(r, changed) + c.records[id] = r + + if status == StatusActive { + c.activeHeadCreatedAt.Set(float64(r.createdAt)) + } + + return r, nil +} + +// compactIfNeeded compact [Catalog] if necessary. +func (c *Catalog) compactIfNeeded() error { + if c.log.Size() < c.maxLogFileSize { + return nil + } + + return c.compactLog() +} + +// compactLog delete old(deleted [Record]s). +func (c *Catalog) compactLog() error { + records := make([]*Record, 0, len(c.records)) + for _, record := range c.records { + if record.deletedAt == 0 { + records = append(records, record) + } + } + + sort.Slice(records, func(i, j int) bool { + return records[i].createdAt < records[j].createdAt + }) + + return c.log.ReWrite(records...) +} + +// sync catalog with [Log]. +func (c *Catalog) sync() error { + for { + r := NewEmptyRecord() + if err := c.log.Read(r); err != nil { + if errors.Is(err, io.EOF) { + return nil + } + // this could happen if log file is corrupted + logger.Errorf("catalog is corrupted: %v", err) + + return c.compactLog() + } + c.records[r.id.String()] = r + } +} diff --git a/pp/go/storage/catalog/catalog_test.go b/pp/go/storage/catalog/catalog_test.go new file mode 100644 index 0000000000..d2cf77b555 --- /dev/null +++ b/pp/go/storage/catalog/catalog_test.go @@ -0,0 +1,160 @@ +package catalog_test + +import ( + "os" + "sort" + "testing" + "time" + + "github.com/jonboulle/clockwork" + "github.com/prometheus/client_golang/prometheus" + "github.com/stretchr/testify/suite" + + "github.com/prometheus/prometheus/pp/go/storage/catalog" +) + +type CatalogSuite struct { + suite.Suite +} + +func TestCatalogSuite(t *testing.T) { + suite.Run(t, new(CatalogSuite)) +} + +func (s *CatalogSuite) TestHappyPath() { + tmpFile, err := os.CreateTemp("", "log_file") + s.Require().NoError(err) + + logFileName := tmpFile.Name() + s.Require().NoError(tmpFile.Close()) + + l, err := catalog.NewFileLogV2(logFileName) + s.Require().NoError(err) + + clock := clockwork.NewFakeClockAt(time.Now()) + + c, err := catalog.New( + clock, + l, + catalog.DefaultIDGenerator{}, + catalog.DefaultMaxLogFileSize, + nil, + ) + s.Require().NoError(err) + + now := clock.Now().UnixMilli() + + var nos1 uint16 = 2 + var nos2 uint16 = 4 + + r1, err := c.Create(nos1) + s.Require().NoError(err) + id1 := r1.ID() + + s.Require().Equal(id1, r1.ID()) + s.Require().Equal(id1, r1.Dir()) + s.Require().Equal(nos1, r1.NumberOfShards()) + s.Require().Equal(now, r1.CreatedAt()) + s.Require().Equal(now, r1.UpdatedAt()) + s.Require().Equal(int64(0), r1.DeletedAt()) + s.Require().Equal(catalog.StatusNew, r1.Status()) + + clock.Advance(time.Second) + now = clock.Now().UnixMilli() + + r2, err := c.Create(nos2) + s.Require().NoError(err) + id2 := r2.ID() + + s.Require().Equal(id2, r2.ID()) + s.Require().Equal(id2, r2.Dir()) + s.Require().Equal(nos2, r2.NumberOfShards()) + s.Require().Equal(now, r2.CreatedAt()) + s.Require().Equal(now, r2.UpdatedAt()) + s.Require().Equal(int64(0), r2.DeletedAt()) + s.Require().Equal(catalog.StatusNew, r2.Status()) + + _, err = c.SetStatus(r1.ID(), catalog.StatusPersisted) + s.Require().NoError(err) + + c = nil + s.Require().NoError(l.Close()) + + l, err = catalog.NewFileLogV2(logFileName) + s.Require().NoError(err) + c, err = catalog.New( + clock, + l, + catalog.DefaultIDGenerator{}, + catalog.DefaultMaxLogFileSize, + nil, + ) + s.Require().NoError(err) + + records, err := c.List(nil, nil) + s.Require().NoError(err) + sort.Slice(records, func(i, j int) bool { + return records[i].CreatedAt() < records[j].CreatedAt() + }) +} + +func (s *CatalogSuite) TestCatalogSyncFail() { + tmpFile, err := os.CreateTemp("", "log_file") + s.Require().NoError(err) + + logFileName := tmpFile.Name() + s.Require().NoError(tmpFile.Close()) + + l, err := catalog.NewFileLogV2(logFileName) + s.Require().NoError(err) + + clock := clockwork.NewFakeClockAt(time.Now()) + + c, err := catalog.New( + clock, + l, + catalog.DefaultIDGenerator{}, + catalog.DefaultMaxLogFileSize, + prometheus.DefaultRegisterer, + ) + s.Require().NoError(err) + + var nos1 uint16 = 2 + var nos2 uint16 = 4 + + r1, err := c.Create(nos1) + s.Require().NoError(err) + + r2, err := c.Create(nos2) + s.Require().NoError(err) + + fileInfo, err := os.Stat(logFileName) + s.Require().NoError(err) + s.Require().NoError(os.Truncate(logFileName, fileInfo.Size()-1)) + + l, err = catalog.NewFileLogV2(logFileName) + s.Require().NoError(err) + + c, err = catalog.New( + clock, + l, + catalog.DefaultIDGenerator{}, + catalog.DefaultMaxLogFileSize, + nil, + ) + s.Require().NoError(err) + + restoredR1, err := c.Get(r1.ID()) + s.Require().NoError(err) + + _, err = c.Get(r2.ID()) + s.Require().Error(err) + + s.Require().Equal(r1.ID(), restoredR1.ID()) + s.Require().Equal(r1.Dir(), restoredR1.Dir()) + s.Require().Equal(r1.NumberOfShards(), restoredR1.NumberOfShards()) + s.Require().Equal(r1.CreatedAt(), restoredR1.CreatedAt()) + s.Require().Equal(r1.UpdatedAt(), restoredR1.UpdatedAt()) + s.Require().Equal(r1.DeletedAt(), restoredR1.DeletedAt()) + s.Require().Equal(r1.Status(), restoredR1.Status()) +} diff --git a/pp/go/storage/catalog/decoder.go b/pp/go/storage/catalog/decoder.go new file mode 100644 index 0000000000..07b0252f6b --- /dev/null +++ b/pp/go/storage/catalog/decoder.go @@ -0,0 +1,314 @@ +package catalog + +import ( + "encoding/binary" + "errors" + "fmt" + "hash" + "hash/crc32" + "io" + + "github.com/google/uuid" + "github.com/prometheus/prometheus/pp/go/util/optional" +) + +const ( + // size of uint32. + sizeOfUint32 = 4 + // size of int64 or uint64. + sizeOf64 = 8 +) + +// +// DecoderV1 +// + +// DecoderV1 decodes [Record], version 1. +// +// Deprecated: For backward compatibility. +type DecoderV1 struct{} + +// DecodeFrom decode [Record] from [io.Reader]. +// +//revive:disable-next-line:cyclomatic this is decode. +func (DecoderV1) DecodeFrom(reader io.Reader, r *Record) (err error) { + var size uint64 + if err = binary.Read(reader, binary.LittleEndian, &size); err != nil { + return fmt.Errorf("read id size: %w", err) + } + + defer func() { + if err != nil && errors.Is(err, io.EOF) { + err = fmt.Errorf("%s: %w", err.Error(), io.ErrUnexpectedEOF) + } + }() + + buf := make([]byte, size) + if _, err = reader.Read(buf); err != nil { + return fmt.Errorf("read id: %w", err) + } + r.id = uuid.MustParse(string(buf)) + + if err = binary.Read(reader, binary.LittleEndian, &size); err != nil { + return fmt.Errorf("read dir size: %w", err) + } + + buf = make([]byte, size) + if _, err = reader.Read(buf); err != nil { + return fmt.Errorf("read dir: %w", err) + } + + if err = binary.Read(reader, binary.LittleEndian, &r.numberOfShards); err != nil { + return fmt.Errorf("read number of shards: %w", err) + } + + if err = binary.Read(reader, binary.LittleEndian, &r.createdAt); err != nil { + return fmt.Errorf("read created at: %w", err) + } + + if err = binary.Read(reader, binary.LittleEndian, &r.updatedAt); err != nil { + return fmt.Errorf("read updated at: %w", err) + } + + if err = binary.Read(reader, binary.LittleEndian, &r.deletedAt); err != nil { + return fmt.Errorf("read deleted at: %w", err) + } + + if err = binary.Read(reader, binary.LittleEndian, &r.status); err != nil { + return fmt.Errorf("read status: %w", err) + } + + return nil +} + +// +// DecoderV2 +// + +// DecoderV2 decodes [Record], version 2. +type DecoderV2 struct{} + +// DecodeFrom decode [Record] from [io.Reader]. +// +//revive:disable-next-line:cyclomatic this is decode. +//revive:disable-next-line:function-length long but this is decode. +func (DecoderV2) DecodeFrom(reader io.Reader, r *Record) (err error) { + var size uint8 + if err = binary.Read(reader, binary.LittleEndian, &size); err != nil { + return fmt.Errorf("read record size: %w", err) + } + + rReader := newReaderWithCounter(reader) + + defer func() { + if err != nil && errors.Is(err, io.EOF) || int(size) != rReader.BytesRead() { + if err == nil { + err = fmt.Errorf("bytes read: %d, bytes expected: %d", rReader.BytesRead(), size) + } + err = fmt.Errorf("%s: %w", err.Error(), io.ErrUnexpectedEOF) + } + }() + + if err = binary.Read(rReader, binary.LittleEndian, &r.id); err != nil { + return fmt.Errorf("read record id: %w", err) + } + + if err = binary.Read(rReader, binary.LittleEndian, &r.numberOfShards); err != nil { + return fmt.Errorf("read number of shards: %w", err) + } + + if err = binary.Read(rReader, binary.LittleEndian, &r.createdAt); err != nil { + return fmt.Errorf("read created at: %w", err) + } + + if err = binary.Read(rReader, binary.LittleEndian, &r.updatedAt); err != nil { + return fmt.Errorf("read updated at: %w", err) + } + + if err = binary.Read(rReader, binary.LittleEndian, &r.deletedAt); err != nil { + return fmt.Errorf("read deleted at: %w", err) + } + + if err = binary.Read(rReader, binary.LittleEndian, &r.corrupted); err != nil { + return fmt.Errorf("read currupted: %w", err) + } + + if err = binary.Read(rReader, binary.LittleEndian, &r.status); err != nil { + return fmt.Errorf("read status: %w", err) + } + + if err = decodeOptionalValue(rReader, binary.LittleEndian, &r.lastAppendedSegmentID); err != nil { + return fmt.Errorf("read last written segment id: %w", err) + } + + return nil +} + +// readerWithCounter reader with a counter of read bytes. +type readerWithCounter struct { + reader io.Reader + n int +} + +// newReaderWithCounter init new [readerWithCounter]. +func newReaderWithCounter(reader io.Reader) *readerWithCounter { + return &readerWithCounter{reader: reader} +} + +// Read reads up to len(p) bytes into p. +func (r *readerWithCounter) Read(p []byte) (n int, err error) { + n, err = r.reader.Read(p) + r.n += n + return n, err +} + +// BytesRead return a counter of read bytes. +func (r *readerWithCounter) BytesRead() int { + return r.n +} + +// decodeOptionalValue decode [optional.Optional[T]] from [io.Reader]. +func decodeOptionalValue[T any]( + reader io.Reader, + byteOrder binary.ByteOrder, + valueRef *optional.Optional[T], +) (err error) { + var nilIndicator uint8 + if err = binary.Read(reader, byteOrder, &nilIndicator); err != nil { + return err + } + if nilIndicator == 0 { + return nil + } + + var value T + if err = binary.Read(reader, byteOrder, &value); err != nil { + return err + } + valueRef.Set(value) + return nil +} + +// +// DecoderV3 +// + +// DecoderV3 decodes [Record], version 3. +type DecoderV3 struct { + offset int + size uint8 + buffer [RecordFrameSizeV3]byte + hasher hash.Hash32 +} + +// NewDecoderV3 init new [DecoderV3]. +func NewDecoderV3() *DecoderV3 { + return &DecoderV3{ + hasher: crc32.NewIEEE(), + } +} + +// DecodeFrom decode [Record] from [io.Reader]. +// +//revive:disable-next-line:cyclomatic this is decode. +//revive:disable-next-line:function-length long but this is decode. +func (d *DecoderV3) DecodeFrom(reader io.Reader, r *Record) (err error) { + d.reset() + + if err = d.readSize(reader); err != nil { + return err + } + + defer func() { + if err != nil && errors.Is(err, io.EOF) { + err = fmt.Errorf("%s: %w", err.Error(), io.ErrUnexpectedEOF) + } + }() + + if err = d.readRecord(reader); err != nil { + return err + } + + if err = d.validateCRC32(); err != nil { + return fmt.Errorf("read crc32: %w", err) + } + + targetOffset := d.offset + 16 //revive:disable-line:add-constant it's size of UUID + r.id = uuid.UUID(d.buffer[d.offset:targetOffset]) + d.offset = targetOffset + + r.numberOfShards = binary.LittleEndian.Uint16(d.buffer[d.offset:]) + d.offset += 2 //revive:disable-line:add-constant it's size of uint16 + + r.createdAt = int64(binary.LittleEndian.Uint64(d.buffer[d.offset:])) // #nosec G115 // no overflow + d.offset += sizeOf64 + + r.updatedAt = int64(binary.LittleEndian.Uint64(d.buffer[d.offset:])) // #nosec G115 // no overflow + d.offset += sizeOf64 + + r.deletedAt = int64(binary.LittleEndian.Uint64(d.buffer[d.offset:])) // #nosec G115 // no overflow + d.offset += sizeOf64 + + r.corrupted = d.buffer[d.offset] > 0 + d.offset++ + + r.status = Status(d.buffer[d.offset]) + d.offset++ + + r.numberOfSegments = binary.LittleEndian.Uint32(d.buffer[d.offset:]) + d.offset += sizeOfUint32 + + r.mint = int64(binary.LittleEndian.Uint64(d.buffer[d.offset:])) // #nosec G115 // no overflow + d.offset += sizeOf64 + + r.maxt = int64(binary.LittleEndian.Uint64(d.buffer[d.offset:])) // #nosec G115 // no overflow + d.offset += sizeOf64 + + return nil +} + +// reset state of decoder. +func (d *DecoderV3) reset() { + d.offset = 0 + d.size = 0 + d.hasher.Reset() +} + +// readSize read size of buffer from [io.Reader]. +func (d *DecoderV3) readSize(reader io.Reader) error { + if _, err := reader.Read(d.buffer[:1]); err != nil { + return fmt.Errorf("read record size: %w", err) + } + d.size = d.buffer[0] + + if int(d.size) != len(d.buffer) { + return fmt.Errorf("invalid size: %d", d.size) + } + + return nil +} + +// readRecord read [Record] from [io.Reader]. +func (d *DecoderV3) readRecord(reader io.Reader) error { + if _, err := reader.Read(d.buffer[:d.size]); err != nil { + return fmt.Errorf("read whole record: %w", err) + } + return nil +} + +// validateCRC32 validate [Record] on CRC32. +func (d *DecoderV3) validateCRC32() (err error) { + expectedCRC32Hash := binary.LittleEndian.Uint32(d.buffer[d.offset:]) + d.offset += sizeOfUint32 + + if _, err = d.hasher.Write(d.buffer[d.offset:]); err != nil { + return fmt.Errorf("write to crc32 hasher: %w", err) + } + + actualCRC32Hash := d.hasher.Sum32() + if expectedCRC32Hash != actualCRC32Hash { + return fmt.Errorf("invalid crc32: expected: %d, actual: %d", expectedCRC32Hash, actualCRC32Hash) + } + + return nil +} diff --git a/pp/go/storage/catalog/decoder_test.go b/pp/go/storage/catalog/decoder_test.go new file mode 100644 index 0000000000..82c994037b --- /dev/null +++ b/pp/go/storage/catalog/decoder_test.go @@ -0,0 +1,80 @@ +package catalog_test + +import ( + "bytes" + "testing" + + "github.com/google/uuid" + "github.com/stretchr/testify/require" + "github.com/stretchr/testify/suite" + + "github.com/prometheus/prometheus/pp/go/storage/catalog" +) + +type DecoderSuite struct { + suite.Suite +} + +func TestDecoderSuite(t *testing.T) { + suite.Run(t, new(DecoderSuite)) +} + +func (s *DecoderSuite) TestDecoderV3Decode() { + buffer := bytes.NewBuffer(nil) + record := catalog.NewRecordWithDataV3(uuid.New(), 5, 25, 26, 27, true, catalog.StatusActive, 25, 2, 3) + + encoder := catalog.NewEncoderV3() + s.Require().NoError(encoder.EncodeTo(buffer, record)) + + decoder := catalog.NewDecoderV3() + decodedRecord := &catalog.Record{} + s.Require().NoError(decoder.DecodeFrom(buffer, decodedRecord)) + + s.Require().Equal(record.ID(), decodedRecord.ID()) + s.Require().Equal(record.NumberOfShards(), decodedRecord.NumberOfShards()) + s.Require().Equal(record.CreatedAt(), decodedRecord.CreatedAt()) + s.Require().Equal(record.UpdatedAt(), decodedRecord.UpdatedAt()) + s.Require().Equal(record.DeletedAt(), decodedRecord.DeletedAt()) + s.Require().Equal(record.Corrupted(), decodedRecord.Corrupted()) + s.Require().Equal(record.Status(), decodedRecord.Status()) + s.Require().Equal(record.NumberOfSegments(), decodedRecord.NumberOfSegments()) + s.Require().Equal(record.Maxt(), decodedRecord.Maxt()) + s.Require().Equal(record.Mint(), decodedRecord.Mint()) +} + +func BenchmarkDecodeV3(b *testing.B) { + buffer := bytes.NewBuffer(nil) + record := catalog.NewRecordWithDataV3(uuid.New(), 5, 25, 26, 27, true, catalog.StatusActive, 25, 2, 3) + var encoder catalog.Encoder + decodedRecord := &catalog.Record{} + b.StopTimer() + b.ResetTimer() + + for i := 0; i < b.N; i++ { + buffer.Reset() + encoder = catalog.NewEncoderV3() + require.NoError(b, encoder.EncodeTo(buffer, record)) + decoder := catalog.NewDecoderV3() + b.StartTimer() + require.NoError(b, decoder.DecodeFrom(buffer, decodedRecord)) + b.StopTimer() + } +} + +func BenchmarkDecodeV3_State(b *testing.B) { + buffer := bytes.NewBuffer(nil) + record := catalog.NewRecordWithDataV3(uuid.New(), 5, 25, 26, 27, true, catalog.StatusActive, 25, 2, 3) + encoder := catalog.NewEncoderV3() + decodedRecord := &catalog.Record{} + decoder := catalog.NewDecoderV3() + b.StopTimer() + b.ResetTimer() + + for i := 0; i < b.N; i++ { + buffer.Reset() + require.NoError(b, encoder.EncodeTo(buffer, record)) + b.StartTimer() + require.NoError(b, decoder.DecodeFrom(buffer, decodedRecord)) + b.StopTimer() + } +} diff --git a/pp/go/storage/catalog/encoder.go b/pp/go/storage/catalog/encoder.go new file mode 100644 index 0000000000..0fc4a73dbc --- /dev/null +++ b/pp/go/storage/catalog/encoder.go @@ -0,0 +1,249 @@ +package catalog + +import ( + "bytes" + "encoding/binary" + "fmt" + "hash" + "hash/crc32" + "io" + + "github.com/prometheus/prometheus/pp/go/util/optional" +) + +const ( + // RecordStructMaxSizeV2 max size of [Record] for [EncoderV2]. + RecordStructMaxSizeV2 = 50 + // RecordFrameSizeV3 size of frame [Record] for [EncoderV3]. + RecordFrameSizeV3 = 68 +) + +// +// EncoderV1 +// + +// EncoderV1 encodes [Record], version 1. +// +// Deprecated. +type EncoderV1 struct{} + +// EncodeTo encode [Record] to [io.Writer]. +func (EncoderV1) EncodeTo(writer io.Writer, r *Record) (err error) { + if err = encodeString(writer, r.id.String()); err != nil { + return fmt.Errorf("v1: encode id: %w", err) + } + + if err = encodeString(writer, r.id.String()); err != nil { + return fmt.Errorf("v1: encode dir: %w", err) + } + + if err = binary.Write(writer, binary.LittleEndian, &r.numberOfShards); err != nil { + return fmt.Errorf("v1: write number of shards: %w", err) + } + + if err = binary.Write(writer, binary.LittleEndian, &r.createdAt); err != nil { + return fmt.Errorf("v1: write created at: %w", err) + } + + if err = binary.Write(writer, binary.LittleEndian, &r.updatedAt); err != nil { + return fmt.Errorf("v1: write updated at: %w", err) + } + + if err = binary.Write(writer, binary.LittleEndian, &r.deletedAt); err != nil { + return fmt.Errorf("v1: write deleted at: %w", err) + } + + if err = binary.Write(writer, binary.LittleEndian, &r.status); err != nil { + return fmt.Errorf("v1: write status: %w", err) + } + + return nil +} + +// encodeString encode string to [io.Writer]. +func encodeString(writer io.Writer, value string) (err error) { + if err = binary.Write(writer, binary.LittleEndian, uint64(len(value))); err != nil { + return fmt.Errorf("write string length: %w", err) + } + + if _, err = writer.Write([]byte(value)); err != nil { + return fmt.Errorf("write string: %w", err) + } + + return nil +} + +// +// EncoderV2 +// + +// EncoderV2 encodes [Record], version 2. +type EncoderV2 struct { + buffer *bytes.Buffer +} + +// NewEncoderV2 init new [EncoderV2]. +func NewEncoderV2() *EncoderV2 { + return &EncoderV2{ + buffer: bytes.NewBuffer(make([]byte, 0, RecordStructMaxSizeV2)), + } +} + +// EncodeTo encode [Record] to [io.Writer]. +// +//revive:disable-next-line:cyclomatic this is encode. +//revive:disable-next-line:function-length long but this is encode. +func (e *EncoderV2) EncodeTo(writer io.Writer, r *Record) (err error) { + e.buffer.Reset() + + if err = binary.Write(e.buffer, binary.LittleEndian, uint8(0)); err != nil { + return fmt.Errorf("v2: encode size filler: %w", err) + } + + if err = binary.Write(e.buffer, binary.LittleEndian, r.id); err != nil { + return fmt.Errorf("v2: encode id: %w", err) + } + + if err = binary.Write(e.buffer, binary.LittleEndian, &r.numberOfShards); err != nil { + return fmt.Errorf("v2: write number of shards: %w", err) + } + + if err = binary.Write(e.buffer, binary.LittleEndian, &r.createdAt); err != nil { + return fmt.Errorf("v2: write created at: %w", err) + } + + if err = binary.Write(e.buffer, binary.LittleEndian, &r.updatedAt); err != nil { + return fmt.Errorf("v2: write updated at: %w", err) + } + + if err = binary.Write(e.buffer, binary.LittleEndian, &r.deletedAt); err != nil { + return fmt.Errorf("v2: write deleted at: %w", err) + } + + if err = binary.Write(e.buffer, binary.LittleEndian, &r.corrupted); err != nil { + return fmt.Errorf("v2: write corrupted: %w", err) + } + + if err = binary.Write(e.buffer, binary.LittleEndian, &r.status); err != nil { + return fmt.Errorf("v2: write status: %w", err) + } + + if err = encodeOptionalValue(e.buffer, binary.LittleEndian, r.lastAppendedSegmentID); err != nil { + return fmt.Errorf("v2: write last written segment id: %w", err) + } + + e.buffer.Bytes()[0] = uint8(len(e.buffer.Bytes()) - 1) // #nosec G115 // no overflow + + if _, err = e.buffer.WriteTo(writer); err != nil { + return fmt.Errorf("v2: write record: %w", err) + } + + return nil +} + +// encodeOptionalValue encode [optional.Optional[T]] to [io.Writer]. +func encodeOptionalValue[T any](writer io.Writer, byteOrder binary.ByteOrder, value optional.Optional[T]) (err error) { + var nilIndicator uint8 + if value.IsNil() { + return binary.Write(writer, byteOrder, nilIndicator) + } + + nilIndicator = 1 + if err = binary.Write(writer, byteOrder, nilIndicator); err != nil { + return err + } + + return binary.Write(writer, byteOrder, value.Value()) +} + +// +// EncoderV3 +// + +// EncoderV3 encodes [Record], version 3. +type EncoderV3 struct { + buffer *bytes.Buffer + crc32Hasher hash.Hash32 +} + +// NewEncoderV3 init new [EncoderV3]. +func NewEncoderV3() *EncoderV3 { + return &EncoderV3{ + buffer: bytes.NewBuffer(make([]byte, 0, RecordFrameSizeV3+1)), // +1 is for size byte + crc32Hasher: crc32.NewIEEE(), + } +} + +// EncodeTo encode [Record] to [io.Writer]. +// +//revive:disable-next-line:cyclomatic this is encode. +//revive:disable-next-line:function-length long but this is encode. +func (e *EncoderV3) EncodeTo(writer io.Writer, r *Record) (err error) { + e.buffer.Reset() + + if err = binary.Write(e.buffer, binary.LittleEndian, uint8(0)); err != nil { + return fmt.Errorf("v3: encode size filler: %w", err) + } + + if err = binary.Write(e.buffer, binary.LittleEndian, uint32(0)); err != nil { + return fmt.Errorf("v3: encode crc32 filler: %w", err) + } + + if err = binary.Write(e.buffer, binary.LittleEndian, r.id); err != nil { + return fmt.Errorf("v3: encode id: %w", err) + } + + if err = binary.Write(e.buffer, binary.LittleEndian, &r.numberOfShards); err != nil { + return fmt.Errorf("v3: write number of shards: %w", err) + } + + if err = binary.Write(e.buffer, binary.LittleEndian, &r.createdAt); err != nil { + return fmt.Errorf("v3: write created at: %w", err) + } + + if err = binary.Write(e.buffer, binary.LittleEndian, &r.updatedAt); err != nil { + return fmt.Errorf("v3: write updated at: %w", err) + } + + if err = binary.Write(e.buffer, binary.LittleEndian, &r.deletedAt); err != nil { + return fmt.Errorf("v3: write deleted at: %w", err) + } + + if err = binary.Write(e.buffer, binary.LittleEndian, &r.corrupted); err != nil { + return fmt.Errorf("v3: write corrupted: %w", err) + } + + if err = binary.Write(e.buffer, binary.LittleEndian, &r.status); err != nil { + return fmt.Errorf("v3: write status: %w", err) + } + + if err = binary.Write(e.buffer, binary.LittleEndian, &r.numberOfSegments); err != nil { + return fmt.Errorf("v3: write number of segments: %w", err) + } + + if err = binary.Write(e.buffer, binary.LittleEndian, &r.mint); err != nil { + return fmt.Errorf("v3: write min timestamp: %w", err) + } + + if err = binary.Write(e.buffer, binary.LittleEndian, &r.maxt); err != nil { + return fmt.Errorf("v3: write max timestamp: %w", err) + } + + e.buffer.Bytes()[0] = uint8(len(e.buffer.Bytes()) - 1) // #nosec G115 // no overflow + + e.crc32Hasher.Reset() + _, err = e.crc32Hasher.Write(e.buffer.Bytes()[5:]) + if err != nil { + return fmt.Errorf("v3: write hash: %w", err) + } + + var binaryCRC32 [4]byte + binary.LittleEndian.PutUint32(binaryCRC32[:], e.crc32Hasher.Sum32()) + copy(e.buffer.Bytes()[1:5], binaryCRC32[:]) + + if _, err = e.buffer.WriteTo(writer); err != nil { + return fmt.Errorf("v3: write record: %w", err) + } + + return nil +} diff --git a/pp/go/storage/catalog/gc.go b/pp/go/storage/catalog/gc.go new file mode 100644 index 0000000000..cd0488feb3 --- /dev/null +++ b/pp/go/storage/catalog/gc.go @@ -0,0 +1,107 @@ +package catalog + +import ( + "context" + "errors" + "os" + "path/filepath" + "time" + + "github.com/prometheus/prometheus/pp/go/relabeler/head/ready" + "github.com/prometheus/prometheus/pp/go/relabeler/logger" +) + +// GC garbage collector for old [Head]. +type GC struct { + dataDir string + catalog *Catalog + readyNotifiable ready.Notifiable + stop chan struct{} + stopped chan struct{} +} + +// NewGC init new [GC]. +func NewGC(dataDir string, catalog *Catalog, readyNotifiable ready.Notifiable) *GC { + return &GC{ + dataDir: dataDir, + catalog: catalog, + readyNotifiable: readyNotifiable, + stop: make(chan struct{}), + stopped: make(chan struct{}), + } +} + +// Iterate over the [Catalog] list and remove old [Head]s. +func (gc *GC) Iterate() { + logger.Debugf("catalog gc iteration: head started") + defer logger.Debugf("catalog gc iteration: head ended") + + records, err := gc.catalog.List( + func(record *Record) bool { + return record.DeletedAt() == 0 + }, + func(lhs, rhs *Record) bool { + return lhs.CreatedAt() < rhs.CreatedAt() + }, + ) + if err != nil { + logger.Debugf("catalog gc failed", err) + return + } + + for _, record := range records { + if record.deletedAt != 0 { + continue + } + logger.Debugf("catalog gc iteration: head: %s", record.ID()) + if record.ReferenceCount() > 0 { + return + } + + if record.Corrupted() { + logger.Debugf("catalog gc iteration: head: %s: %s", record.ID(), "corrupted") + continue + } + + if err = os.RemoveAll(filepath.Join(gc.dataDir, record.Dir())); err != nil { + logger.Errorf("failed to remote head dir: %w", err) + return + } + + if err = gc.catalog.Delete(record.ID()); err != nil { + logger.Errorf("failed to delete head record: %w", err) + return + } + + logger.Debugf("catalog gc iteration: head: %s: %s", record.ID(), "removed") + } +} + +// Run main loop [GC]. +func (gc *GC) Run(ctx context.Context) error { + defer close(gc.stopped) + + for { + select { + case <-ctx.Done(): + return ctx.Err() + case <-gc.readyNotifiable.ReadyChan(): + case <-gc.stop: + return errors.New("stopped") + } + select { + case <-ctx.Done(): + return ctx.Err() + case <-time.After(time.Minute): + gc.Iterate() + case <-gc.stop: + return errors.New("stopped") + } + } +} + +// Stop the garbage collection loop. +func (gc *GC) Stop() { + close(gc.stop) + <-gc.stopped +} diff --git a/pp/go/storage/catalog/log.go b/pp/go/storage/catalog/log.go new file mode 100644 index 0000000000..7e71033a88 --- /dev/null +++ b/pp/go/storage/catalog/log.go @@ -0,0 +1,296 @@ +package catalog + +import ( + "encoding/binary" + "errors" + "fmt" + "io" + "os" + "strings" + + "github.com/prometheus/prometheus/pp/go/storage/logger" +) + +const ( + // LogFileVersionV1 version 1 of log-file. + LogFileVersionV1 uint64 = 1 + // LogFileVersionV2 version 2 of log-file. + LogFileVersionV2 uint64 = 2 + // LogFileVersionV3 version 3 of log-file. + LogFileVersionV3 uint64 = 3 + + // logFilePerm log-file permissions. + logFilePerm = 0o600 +) + +// +// Encoder +// + +// Encoder encodes [Record]. +type Encoder interface { + // EncodeTo encode [Record] to [io.Writer]. + EncodeTo(writer io.Writer, r *Record) error +} + +// +// Decoder +// + +// Decoder decodes [Record]. +type Decoder interface { + // DecodeFrom decode [Record] from [io.Reader]. + DecodeFrom(reader io.Reader, r *Record) error +} + +// +// FileLog +// + +// FileLog head-log file, contains [Record]s of heads. +type FileLog struct { + version uint64 + file *FileHandler + filePath string + encoder Encoder + decoder Decoder +} + +// NewFileLogV1 init new [FileLog] with [EncoderV1], [DecoderV1], version 1. +// +// Deprecated. +func NewFileLogV1(fileName string) (*FileLog, error) { + file, err := NewFileHandler(fileName) + if err != nil { + return nil, err + } + + fl := &FileLog{ + version: LogFileVersionV1, + file: file, + encoder: EncoderV1{}, + decoder: DecoderV1{}, + } + + defer func() { + if err != nil { + _ = fl.Close() + } + }() + + if file.Size() == 0 { + if err = binary.Write(file, binary.LittleEndian, fl.version); err != nil { + return nil, errors.Join(fmt.Errorf("failed to write log file version: %w", err), fl.Close()) + } + } else { + var version uint64 + if err = binary.Read(file, binary.LittleEndian, &version); err != nil { + return nil, errors.Join(fmt.Errorf("failed to read log file version: %w", err), fl.Close()) + } + if version != fl.version { + return nil, errors.Join(fmt.Errorf("invalid log file version: %d", version), fl.Close()) + } + } + + return fl, nil +} + +// NewFileLogV2 init new [FileLog] with [EncoderV2], [DecoderV2], version 2. +func NewFileLogV2(filePath string) (*FileLog, error) { + return NewFileLog(filePath, LogFileVersionV2) +} + +// NewFileLogV3 init new [FileLog] with [EncoderV3], [DecoderV3], version 3. +func NewFileLogV3(filePath string) (*FileLog, error) { + return NewFileLog(filePath, LogFileVersionV3) +} + +// NewFileLog init new [FileLog] with migrate to target version encoder and decoder. +func NewFileLog(filePath string, targetVersion uint64) (*FileLog, error) { + sourceFilePath := filePath + fl, err := openFileLog(filePath, sourceFilePath, targetVersion) + if err == nil { + return fl, nil + } + + if !errors.Is(err, ErrUnreadableLogFile) { + return nil, err + } + + logger.Errorf("unreadable log file: filepath: %s, error: %v", sourceFilePath, err) + + sourceFilePath = fmt.Sprintf("%s.compacted", filePath) + fl, err = openFileLog(filePath, sourceFilePath, targetVersion) + if err == nil { + return fl, nil + } + + if !errors.Is(err, ErrUnreadableLogFile) { + return nil, err + } + + logger.Errorf("unreadable log file: filepath: %s, error: %v", sourceFilePath, err) + + return newFileLogByVersion(filePath, targetVersion) +} + +// openFileLog open [FileLog] with migrate to version. +func openFileLog(filePath, sourceFilePath string, version uint64) (*FileLog, error) { + file, encoder, decoder, err := migrate(filePath, sourceFilePath, version) + if err != nil { + return nil, err + } + + return &FileLog{ + version: version, + file: file, + filePath: filePath, + encoder: encoder, + decoder: decoder, + }, nil +} + +// newFileLogByVersion init new [FileLog] by version. +func newFileLogByVersion(filePath string, version uint64) (*FileLog, error) { + encoder, decoder, err := codecsByVersion(version) + if err != nil { + return nil, fmt.Errorf("create encoder/decoder: %w", err) + } + + file, err := createFileHandlerByVersion(filePath, version) + if err != nil { + return nil, fmt.Errorf("create file handler: %w", err) + } + + return &FileLog{ + version: version, + file: file, + filePath: filePath, + encoder: encoder, + decoder: decoder, + }, nil +} + +// Close closes the [FileHandler], rendering it unusable for I/O. +func (fl *FileLog) Close() error { + return fl.file.Close() +} + +// ReWrite rewrite [FileLog] with [Record]s. +func (fl *FileLog) ReWrite(records ...*Record) error { + oldFile := fl.file + swapFilePath := fmt.Sprintf("%s.compacted", strings.TrimSuffix(fl.filePath, ".compacted")) + newFile, err := writeSwapAndSwitchAtFilePath(fl.filePath, swapFilePath, fl.version, fl.encoder, records...) + if err != nil { + return fmt.Errorf("write log file: %w", err) + } + + fl.file = newFile + if err = oldFile.Close(); err != nil { + logger.Errorf("failed to close old file: %v", err) + } + + return nil +} + +// Read [Record] from [FileLog]. +func (fl *FileLog) Read(r *Record) error { + return fl.decoder.DecodeFrom(fl.file, r) +} + +// Size return current size of [FileHandler]. +func (fl *FileLog) Size() int { + return fl.file.Size() +} + +// Write [Record] to [FileLog]. +func (fl *FileLog) Write(r *Record) error { + return fl.encoder.EncodeTo(fl.file, r) +} + +func writeSwapAndSwitchAtFilePath( + targetFilePath, swapFilePath string, + version uint64, + encoder Encoder, + records ...*Record, +) (*FileHandler, error) { + swapFile, err := createSwapFile(swapFilePath, version, encoder, records...) + if err != nil { + return nil, fmt.Errorf("create swap file: %w", err) + } + + defer func() { + if err != nil { + err = errors.Join(err, swapFile.Close(), os.RemoveAll(swapFilePath)) + } + }() + + if err = os.Rename(swapFilePath, targetFilePath); err != nil { + return nil, fmt.Errorf("rename swap file: %w", err) + } + + return swapFile, nil +} + +// creates swap file, writes records & sets read offset at first record. +func createSwapFile(fileName string, version uint64, encoder Encoder, records ...*Record) (*FileHandler, error) { + swapFile, err := NewFileHandlerWithOpts(fileName, os.O_CREATE|os.O_WRONLY|os.O_TRUNC, logFilePerm) + if err != nil { + return nil, fmt.Errorf("new file handler: %w", err) + } + + defer func() { + if err != nil { + err = errors.Join(err, swapFile.Close(), os.RemoveAll(fileName)) + } + }() + + offset, err := writeLogFileVersion(swapFile, version) + if err != nil { + return nil, fmt.Errorf("write log file version: %w", err) + } + + for _, record := range records { + if err = encoder.EncodeTo(swapFile, record); err != nil { + return nil, fmt.Errorf("encode record: %w", err) + } + } + + if err = swapFile.Sync(); err != nil { + return nil, fmt.Errorf("sync swap file: %w", err) + } + + swapFile.SetReadOffset(int64(offset)) + + return swapFile, nil +} + +// readLogFileVersion reads log file version respecting on disk version size. +func readLogFileVersion(reader io.Reader) (version uint64, err error) { + var v [8]byte + _, err = reader.Read(v[:1]) + if err != nil { + return 0, err + } + + version = binary.LittleEndian.Uint64(v[:8]) + if version <= LogFileVersionV2 { + // skip next 7 bytes + _, err = reader.Read(v[1:8]) + return version, err + } + + return version, nil +} + +// writeLogFileVersion writes log file version respecting on disk version size. +func writeLogFileVersion(writer io.Writer, version uint64) (int, error) { + var v [8]byte + binary.LittleEndian.PutUint64(v[:8], version) + numberOfBytesToWrite := len(v) + if version >= LogFileVersionV3 { + numberOfBytesToWrite = 1 + } + bytesWritten, err := writer.Write(v[:numberOfBytesToWrite]) + return bytesWritten, err +} diff --git a/pp/go/storage/catalog/log_file_handler.go b/pp/go/storage/catalog/log_file_handler.go new file mode 100644 index 0000000000..f487fd0dcb --- /dev/null +++ b/pp/go/storage/catalog/log_file_handler.go @@ -0,0 +1,102 @@ +package catalog + +import ( + "errors" + "fmt" + "io" + "os" +) + +// +// FileHandler +// + +// FileHandler handler for work with [os.File]. +type FileHandler struct { + file *os.File + size int + readOffset int64 + writeOffset int64 +} + +// NewFileHandler init new [FileHandler]. +func NewFileHandler(filePath string) (*FileHandler, error) { + return NewFileHandlerWithOpts(filePath, os.O_CREATE|os.O_RDWR, logFilePerm) +} + +// NewFileHandlerWithOpts init new [FileHandler] with opts. +func NewFileHandlerWithOpts(filePath string, flag int, perm os.FileMode) (*FileHandler, error) { + file, err := os.OpenFile(filePath, flag, perm) // #nosec G304 // it's meant to be that way + if err != nil { + return nil, fmt.Errorf("open file: %w", err) + } + defer func() { + if err != nil { + err = errors.Join(err, file.Close()) + } + }() + + fileInfo, err := file.Stat() + if err != nil { + return nil, fmt.Errorf("read file info: %w", err) + } + + return &FileHandler{ + file: file, + size: int(fileInfo.Size()), + writeOffset: fileInfo.Size(), + }, nil +} + +// Close closes the [os.File], rendering it unusable for I/O. +func (fh *FileHandler) Close() error { + return fh.file.Close() +} + +// FileName returns the current name of the file. +func (fh *FileHandler) FileName() string { + return fh.file.Name() +} + +// Read reads len(b) bytes from the [os.File]. +func (fh *FileHandler) Read(p []byte) (n int, err error) { + n, err = fh.file.ReadAt(p, fh.readOffset) + if err != nil { + if !errors.Is(err, io.EOF) { + return 0, fmt.Errorf("read file: %w", err) + } + } + fh.readOffset += int64(n) + return n, err +} + +// SetReadOffset set offset for read file. +func (fh *FileHandler) SetReadOffset(offset int64) { + fh.readOffset = offset +} + +// Size returns current size of file. +func (fh *FileHandler) Size() int { + return fh.size +} + +// Sync commits the current contents of the file to stable storage. +func (fh *FileHandler) Sync() error { + return fh.file.Sync() +} + +// Write writes len(b) bytes to the [os.File]. +func (fh *FileHandler) Write(p []byte) (n int, err error) { + n, err = fh.file.WriteAt(p, fh.writeOffset) + if err != nil { + return 0, fmt.Errorf("write file: %w", err) + } + + if err = fh.file.Sync(); err != nil { + return 0, fmt.Errorf("sync file: %w", err) + } + + fh.size += n + fh.writeOffset += int64(n) + return n, nil +} diff --git a/pp/go/storage/catalog/log_migrate.go b/pp/go/storage/catalog/log_migrate.go new file mode 100644 index 0000000000..82cce04fc0 --- /dev/null +++ b/pp/go/storage/catalog/log_migrate.go @@ -0,0 +1,286 @@ +package catalog + +import ( + "errors" + "fmt" + "io" + "os" + + "github.com/prometheus/prometheus/pp/go/storage/logger" + "github.com/prometheus/prometheus/pp/go/util/optional" +) + +var ( + // ErrUnsupportedVersion unsupported version error. + ErrUnsupportedVersion = errors.New("unsupported version") + // ErrUnreadableLogFile unreadable log file error. + ErrUnreadableLogFile = errors.New("unreadable log file") +) + +// migrate source file to target file on target version. +// +//revive:disable-next-line:function-result-limit there is no point in packing it into a structure. +func migrate( + targetFilePath, sourceFilePath string, + targetVersion uint64, +) (_ *FileHandler, _ Encoder, _ Decoder, _ error) { + sourceFile, sourceVersion, err := loadFile(sourceFilePath) + if err != nil { + return nil, nil, nil, err + } + + sourceEncoder, sourceDecoder, err := codecsByVersion(sourceVersion) + if err != nil { + return nil, nil, nil, errors.Join(ErrUnreadableLogFile, sourceFile.Close()) + } + + if sourceVersion != targetVersion { + return migrateTo(sourceFile, sourceDecoder, targetFilePath, sourceFilePath, targetVersion, sourceVersion) + } + + if sourceFilePath == targetFilePath { + return sourceFile, sourceEncoder, sourceDecoder, nil + } + + err = os.Rename(sourceFilePath, targetFilePath) + if err != nil { + return nil, nil, nil, errors.Join(err, sourceFile.Close()) + } + + return sourceFile, sourceEncoder, sourceDecoder, nil +} + +// migrateTo source file to target file on target version. +// +//revive:disable-next-line:function-result-limit there is no point in packing it into a structure. +func migrateTo( + sourceFile *FileHandler, + sourceDecoder Decoder, + targetFilePath, sourceFilePath string, + targetVersion, sourceVersion uint64, +) (_ *FileHandler, _ Encoder, _ Decoder, _ error) { + targetEncoder, targetDecoder, err := codecsByVersion(targetVersion) + if err != nil { + return nil, nil, nil, errors.Join(err, sourceFile.Close()) + } + + records := make([]*Record, 0, 10) //revive:disable-line:add-constant it's average value of records + for { + record := &Record{} + if err = sourceDecoder.DecodeFrom(sourceFile, record); err != nil { + if errors.Is(err, io.EOF) { + break + } + logger.Errorf("failed to decode record: %v", err) + break + } + records = append(records, record) + } + + migration := getMigration(sourceVersion, targetVersion) + migratedRecords := make([]*Record, 0, len(records)) + for _, record := range records { + migratedRecords = append(migratedRecords, migration.Migrate(record)) + } + + swapFilePath := fmt.Sprintf("%s.swap", sourceFilePath) + targetFile, err := writeSwapAndSwitchAtFilePath( + targetFilePath, + swapFilePath, + targetVersion, + targetEncoder, + migratedRecords..., + ) + if err != nil { + return nil, nil, nil, errors.Join(err, sourceFile.Close()) + } + + if err = sourceFile.Close(); err != nil { + logger.Errorf("failed to close file: %v", err) + } + + return targetFile, targetEncoder, targetDecoder, nil +} + +// loadFile load [FileHandler] from file. +func loadFile(filePath string) (_ *FileHandler, _ uint64, err error) { + fileInfo, err := os.Stat(filePath) + if err != nil { + if os.IsNotExist(err) { + err = ErrUnreadableLogFile + } + return nil, 0, err + } + + if fileInfo.Size() == 0 { + return nil, 0, ErrUnreadableLogFile + } + + fh, err := NewFileHandlerWithOpts(filePath, os.O_RDWR, logFilePerm) + if err != nil { + return nil, 0, err + } + + version, err := readLogFileVersion(fh) + if err != nil { + return nil, 0, errors.Join(fmt.Errorf("read log file version: %w", err), fh.Close()) + } + + return fh, version, nil +} + +// createFileHandlerByVersion create [FileHandler] by version. +func createFileHandlerByVersion(filePath string, version uint64) (*FileHandler, error) { + fh, err := NewFileHandlerWithOpts(filePath, os.O_CREATE|os.O_RDWR|os.O_TRUNC, logFilePerm) + if err != nil { + return nil, err + } + + offset, err := writeLogFileVersion(fh, version) + if err != nil { + return nil, errors.Join(err, fh.Close()) + } + + fh.SetReadOffset(int64(offset)) + + return fh, nil +} + +// codecsByVersion select codec by version. +func codecsByVersion(version uint64) (e Encoder, d Decoder, err error) { + switch version { + case LogFileVersionV1: + return EncoderV1{}, DecoderV1{}, nil + case LogFileVersionV2: + return NewEncoderV2(), DecoderV2{}, nil + case LogFileVersionV3: + return NewEncoderV3(), NewDecoderV3(), nil + default: + return nil, nil, ErrUnsupportedVersion + } +} + +// +// Migration +// + +// Migration migrates record interface. +type Migration interface { + Migrate(record *Record) *Record +} + +// MigrationFunc is Migration interface function wrapper. +type MigrationFunc func(record *Record) *Record + +// Migrate reacord version. +func (fn MigrationFunc) Migrate(record *Record) *Record { + return fn(record) +} + +// +// MigrationV2 +// + +// MigrationV2 migrates record from v1 to v2 and vice versa. +type MigrationV2 struct{} + +// Up migrates from v1 to v2. +func (MigrationV2) Up(record *Record) *Record { + if record.status == StatusCorrupted { + record.corrupted = true + record.status = StatusRotated + } + return record +} + +// Down migrates from v2 to v1. +func (MigrationV2) Down(record *Record) *Record { + if record.status == StatusRotated && record.corrupted { + record.status = StatusCorrupted + } + return record +} + +// +// MigrationV3 +// + +// MigrationV3 migrates record from v2 to v3 and vice versa. +type MigrationV3 struct{} + +// Up migrates from v2 to v3. +func (MigrationV3) Up(record *Record) *Record { + record.numberOfSegments = 0 + if !record.lastAppendedSegmentID.IsNil() { + record.numberOfSegments = record.lastAppendedSegmentID.Value() + 1 + } + return record +} + +// Down migrates from v3 to v2. +func (MigrationV3) Down(record *Record) *Record { + if record.numberOfSegments > 0 { + record.lastAppendedSegmentID.Set(record.numberOfSegments - 1) + } else { + record.lastAppendedSegmentID = optional.WithRawValue[uint32](nil) + } + return record +} + +// +// ChainedMigration +// + +// ChainedMigration combines migrations to provide multiple migrations. +type ChainedMigration struct { + migrations []Migration +} + +// NewChainedMigration constructor. +func NewChainedMigration(migrations ...Migration) *ChainedMigration { + return &ChainedMigration{migrations: migrations} +} + +// Migrate is an Migration interface implementation. +func (c *ChainedMigration) Migrate(record *Record) *Record { + for _, migration := range c.migrations { + record = migration.Migrate(record) + } + return record +} + +// getMigration create [Migration] from version to version. +func getMigration(from, to uint64) Migration { + up := true + if from > to { + up = false + from, to = to, from + } + + var migrations []Migration + for i := from + 1; i <= to; i++ { + migrations = append(migrations, migrationByVersion(i, up)) + } + + return NewChainedMigration(migrations...) +} + +// migrationByVersion create [Migration] by version. +// +//revive:disable-next-line:flag-parameter this is a flag, but it's more convenient this way +func migrationByVersion(version uint64, up bool) Migration { + switch version { + case LogFileVersionV2: + if up { + return MigrationFunc(MigrationV2{}.Up) + } + return MigrationFunc(MigrationV2{}.Down) + case LogFileVersionV3: + if up { + return MigrationFunc(MigrationV3{}.Up) + } + return MigrationFunc(MigrationV3{}.Down) + default: + panic(fmt.Sprintf("invalid version: %d", version)) + } +} diff --git a/pp/go/storage/catalog/log_test.go b/pp/go/storage/catalog/log_test.go new file mode 100644 index 0000000000..2a23f6ef61 --- /dev/null +++ b/pp/go/storage/catalog/log_test.go @@ -0,0 +1,102 @@ +package catalog_test + +import ( + "encoding/hex" + "os" + "path/filepath" + "testing" + + "github.com/google/uuid" + "github.com/stretchr/testify/suite" + + "github.com/prometheus/prometheus/pp/go/storage/catalog" +) + +type FileLogTestSuite struct { + suite.Suite + records []*catalog.Record +} + +func TestFileLog(t *testing.T) { + suite.Run(t, new(FileLogTestSuite)) +} + +func (s *FileLogTestSuite) SetupSuite() { + id1 := uuid.MustParse("fe52d991-fe22-41d9-9642-7e4d66d81a0c") + var lastWrittenSegmentIDForID1 uint32 = 5 + id2 := uuid.MustParse("2d89cb33-9daa-4aea-9855-f844add5e3e4") + id3 := uuid.MustParse("ec0c2898-9c42-449c-9a58-74bea665481c") + + s.records = []*catalog.Record{ + catalog.NewRecordWithData(id1, 1, 4, 5, 0, false, 0, catalog.StatusNew, &lastWrittenSegmentIDForID1), + catalog.NewRecordWithData(id2, 2, 34, 420, 0, false, 0, catalog.StatusCorrupted, nil), + catalog.NewRecordWithData(id3, 3, 25, 256, 0, false, 0, catalog.StatusPersisted, nil), + } +} + +func (s *FileLogTestSuite) TestMigrateV1ToV2() { + logFileV1Name := filepath.Join(s.T().TempDir(), "v1.log") + logFileV1, err := catalog.NewFileLogV1(logFileV1Name) + s.Require().NoError(err) + + for _, record := range s.records { + s.Require().NoError(logFileV1.Write(record)) + } + s.Require().NoError(logFileV1.Close()) + + fileContentIsEqual(s, logFileV1Name, "testdata/headv1.log") + + logFile, err := catalog.NewFileLogV2(logFileV1Name) + s.Require().NoError(err) + s.Require().NoError(logFile.Close()) + + fileContentIsEqual(s, logFileV1Name, "testdata/headv2.log") +} + +func (s *FileLogTestSuite) TestMigrateV2ToV3() { + tmpDir := filepath.Join(s.T().TempDir(), "logtest") + s.Require().NoError(os.CopyFS(tmpDir, os.DirFS("testdata"))) + + logFilePath := filepath.Join(tmpDir, "headv2.log") + logFile, err := catalog.NewFileLogV3(logFilePath) + s.Require().NoError(err) + s.Require().NoError(logFile.Close()) + + fileContentIsEqual(s, logFilePath, "testdata/headv3.log") +} + +func (s *FileLogTestSuite) TestMigrateV3ToV2() { + tmpDir := filepath.Join(s.T().TempDir(), "logtest") + s.Require().NoError(os.CopyFS(tmpDir, os.DirFS("testdata"))) + + logFilePath := filepath.Join(tmpDir, "headv3.log") + logFile, err := catalog.NewFileLogV2(logFilePath) + s.Require().NoError(err) + s.Require().NoError(logFile.Close()) + + fileContentIsEqual(s, logFilePath, "testdata/headv2.log") +} + +func (s *FileLogTestSuite) TestMigrateV1ToV3() { + tmpDir := filepath.Join(s.T().TempDir(), "logtest") + s.Require().NoError(os.CopyFS(tmpDir, os.DirFS("testdata"))) + + logFilePath := filepath.Join(tmpDir, "headv1.log") + logFile, err := catalog.NewFileLogV3(logFilePath) + s.Require().NoError(err) + s.Require().NoError(logFile.Close()) + + fileContentIsEqual(s, logFilePath, "testdata/headv3.log") +} + +func fileContentIsEqual(s *FileLogTestSuite, filePath1, filePath2 string) { + data1, err := os.ReadFile(filePath1) + s.Require().NoError(err) + s.T().Log(hex.EncodeToString(data1)) + + data2, err := os.ReadFile(filePath2) + s.Require().NoError(err) + s.T().Log(hex.EncodeToString(data1)) + + s.Require().Equal(data1, data2) +} diff --git a/pp/go/storage/catalog/record.go b/pp/go/storage/catalog/record.go new file mode 100644 index 0000000000..6958343e66 --- /dev/null +++ b/pp/go/storage/catalog/record.go @@ -0,0 +1,210 @@ +package catalog + +import ( + "sync" + "sync/atomic" + + "github.com/google/uuid" + "github.com/prometheus/prometheus/pp/go/util/optional" +) + +// +// Status +// + +// Status of [Head] for record in catalog. +type Status uint8 + +const ( + // StatusNew status for new [Head]. + StatusNew Status = iota + // StatusRotated status for rotated [Head]. + StatusRotated + // StatusCorrupted status for corrupted [Head]. Deprecated. + StatusCorrupted + // StatusPersisted status for persisted [Head]. + StatusPersisted + // StatusActive status for active [Head]. + StatusActive +) + +// +// Record +// + +// Record information about the [Head] in the catalog. +type Record struct { + id uuid.UUID // uuid + numberOfShards uint16 // number of shards + createdAt int64 // time of record creation + updatedAt int64 + deletedAt int64 + corrupted bool + lastAppendedSegmentID optional.Optional[uint32] + referenceCount int64 + status Status // status + numberOfSegments uint32 + mint int64 + maxt int64 +} + +// NewEmptyRecord init new empty [Record]. +func NewEmptyRecord() *Record { + return &Record{} +} + +// NewRecordWithData init new [Record] with parameters. +func NewRecordWithData( + id uuid.UUID, + numberOfShards uint16, + createdAt int64, + updatedAt int64, + deletedAt int64, + corrupted bool, + referenceCount int64, + status Status, + lastAppendedSegmentID *uint32, +) *Record { + return &Record{ + id: id, + numberOfShards: numberOfShards, + createdAt: createdAt, + updatedAt: updatedAt, + deletedAt: deletedAt, + corrupted: corrupted, + referenceCount: referenceCount, + status: status, + lastAppendedSegmentID: optional.WithRawValue(lastAppendedSegmentID), + } +} + +// NewRecordWithDataV3 init new [Record] version 3 with parameters. +func NewRecordWithDataV3( + id uuid.UUID, + numberOfShards uint16, + createdAt int64, + updatedAt int64, + deletedAt int64, + corrupted bool, + status Status, + numberOfSegments uint32, + mint int64, + maxt int64, +) *Record { + return &Record{ + id: id, + numberOfShards: numberOfShards, + createdAt: createdAt, + updatedAt: updatedAt, + deletedAt: deletedAt, + corrupted: corrupted, + status: status, + numberOfSegments: numberOfSegments, + mint: mint, + maxt: maxt, + } +} + +// Acquire increase reference count to [Head]. Returns func decrease reference count. +func (r *Record) Acquire() func() { + atomic.AddInt64(&r.referenceCount, 1) + var onceRelease sync.Once + return func() { + onceRelease.Do(func() { + atomic.AddInt64(&r.referenceCount, -1) + }) + } +} + +// Corrupted returns true if [Head] is corrupted. +func (r *Record) Corrupted() bool { + return r.corrupted +} + +// CreatedAt returns the timestamp when the [Record]([Head]) was created. +func (r *Record) CreatedAt() int64 { + return r.createdAt +} + +// DeletedAt returns the timestamp when the [Record]([Head]) was deleted. +func (r *Record) DeletedAt() int64 { + return r.deletedAt +} + +// Dir returns dir of [Head]. +func (r *Record) Dir() string { + return r.id.String() +} + +// ID returns id of [Head]. +func (r *Record) ID() string { + return r.id.String() +} + +// LastAppendedSegmentID returns last appended segment id if exist, else nil. +func (r *Record) LastAppendedSegmentID() *uint32 { + return r.lastAppendedSegmentID.RawValue() +} + +// Maxt returns max timestamp in [Head]. +func (r *Record) Maxt() int64 { + return r.maxt +} + +// Mint returns min timestamp in [Head]. +func (r *Record) Mint() int64 { + return r.mint +} + +// NumberOfSegments returns number of segments in [Head]. +func (r *Record) NumberOfSegments() uint32 { + return r.numberOfSegments +} + +// NumberOfShards returns number of shards of [Head]. +func (r *Record) NumberOfShards() uint16 { + return r.numberOfShards +} + +// ReferenceCount returns current of reference count. +func (r *Record) ReferenceCount() int64 { + return atomic.LoadInt64(&r.referenceCount) +} + +// SetLastAppendedSegmentID set last appended segment id. +func (r *Record) SetLastAppendedSegmentID(segmentID uint32) { + r.lastAppendedSegmentID.Set(segmentID) +} + +// SetNumberOfSegments number of segments in [Head]. +func (r *Record) SetNumberOfSegments(numberOfSegments uint32) { + r.numberOfSegments = numberOfSegments +} + +// Status returns current status of [Head]. +func (r *Record) Status() Status { + return r.status +} + +// UpdatedAt returns the timestamp when the [Record]([Head]) was updated. +func (r *Record) UpdatedAt() int64 { + return r.updatedAt +} + +// createRecordCopy create a copy of the [Record]. +func createRecordCopy(r *Record) *Record { + c := *r + return &c +} + +// applyRecordChanges apply changes to current [Record]. +func applyRecordChanges(r, changed *Record) { + r.createdAt = changed.createdAt + r.updatedAt = changed.updatedAt + r.deletedAt = changed.deletedAt + r.corrupted = changed.corrupted + r.status = changed.status + r.numberOfShards = changed.numberOfShards + r.mint = changed.mint + r.maxt = changed.maxt +} diff --git a/pp/go/storage/catalog/record_test.go b/pp/go/storage/catalog/record_test.go new file mode 100644 index 0000000000..254443b595 --- /dev/null +++ b/pp/go/storage/catalog/record_test.go @@ -0,0 +1,27 @@ +package catalog_test + +import ( + "testing" + + "github.com/prometheus/prometheus/pp/go/storage/catalog" + "github.com/stretchr/testify/suite" +) + +type RecordSuite struct { + suite.Suite +} + +func TestRecordSuite(t *testing.T) { + suite.Run(t, new(RecordSuite)) +} + +func (s *RecordSuite) TestReferenceCounterIncDecValue() { + r := catalog.NewEmptyRecord() + s.Require().Equal(int64(0), r.ReferenceCount()) + release := r.Acquire() + s.Require().Equal(int64(1), r.ReferenceCount()) + release() + s.Require().Equal(int64(0), r.ReferenceCount()) + release() + s.Require().Equal(int64(0), r.ReferenceCount()) +} diff --git a/pp/go/storage/catalog/testdata/headv1.log b/pp/go/storage/catalog/testdata/headv1.log new file mode 100644 index 0000000000000000000000000000000000000000..ba6115a7b13724c6c17cae474bc2a576bbef0abb GIT binary patch literal 353 zcmb7g9WQ`cL1x?>%}p%TbvFG>Z{1 zC<7Oaup4SAN?@h~YRBZay0A9i)OVfs<7FA0)5?EnA( literal 0 HcmV?d00001 diff --git a/pp/go/storage/catalog/testdata/headv2.log b/pp/go/storage/catalog/testdata/headv2.log new file mode 100644 index 0000000000000000000000000000000000000000..54d0f38eb51c7b32fe1e1fc4e17b67bdc80aefa5 GIT binary patch literal 146 zcmZQ#fB@ZpK{qGQ&BLfRmiWN#DQM$UFr;X>X@_IER^oPsZtB;>B tF(^T$moOr!12P#I8Fb(9Xv~=7Ww(`KH~F;$dW9VSoTuD2<5CT5rf GG7SJeKOUg~ literal 0 HcmV?d00001 diff --git a/pp/go/storage/head/builder/builder.go b/pp/go/storage/head/builder/builder.go index e647fb226b..bcb526b0c0 100644 --- a/pp/go/storage/head/builder/builder.go +++ b/pp/go/storage/head/builder/builder.go @@ -1 +1,3 @@ package builder + +// TODO ? diff --git a/pp/go/storage/head/shard/shard.go b/pp/go/storage/head/shard/shard.go index 456098ea59..04c894b929 100644 --- a/pp/go/storage/head/shard/shard.go +++ b/pp/go/storage/head/shard/shard.go @@ -8,15 +8,9 @@ import ( // Wal the minimum required Wal implementation for a [Shard]. type Wal interface { - // Commit finalize segment from encoder and write to wal. - Commit() error - - // WalFlush flush all contetnt into wal. + // Flush flush all contetnt into wal. Flush() error - // WalWrite append the incoming inner series to wal encoder. - Write(innerSeriesSlice []*cppbridge.InnerSeries) (bool, error) - // Close closes the wal segmentWriter. Close() error } @@ -73,23 +67,11 @@ func (s *Shard[TWal]) Wal() TWal { return s.wal } -// WalCommit finalize segment from encoder and write to wal. -func (s *Shard[TWal]) WalCommit() error { - return s.lss.WithRLock(func(_, _ *cppbridge.LabelSetStorage) error { - return s.wal.Commit() - }) -} - // WalFlush flush all contetnt into wal. func (s *Shard[TWal]) WalFlush() error { return s.wal.Flush() } -// WalWrite append the incoming inner series to wal encoder. -func (s *Shard[TWal]) WalWrite(innerSeriesSlice []*cppbridge.InnerSeries) (bool, error) { - return s.wal.Write(innerSeriesSlice) -} - // // PerGoroutineShard // diff --git a/pp/go/storage/head/shard/wal/wal.go b/pp/go/storage/head/shard/wal/wal.go index 6daf59d379..2147635f5b 100644 --- a/pp/go/storage/head/shard/wal/wal.go +++ b/pp/go/storage/head/shard/wal/wal.go @@ -86,6 +86,7 @@ func (w *Wal[TSegment, TStats, TWriter]) Close() error { } // Commit finalize segment from encoder and write to [SegmentWriter]. +// It is necessary to lock the LSS for reading for the commit. func (w *Wal[TSegment, TStats, TWriter]) Commit() error { if w.corrupted { return fmt.Errorf("committing corrupted wal") diff --git a/pp/go/storage/querier/interface.go b/pp/go/storage/querier/interface.go new file mode 100644 index 0000000000..766422d6df --- /dev/null +++ b/pp/go/storage/querier/interface.go @@ -0,0 +1,114 @@ +package querier + +import ( + "context" + + "github.com/prometheus/prometheus/pp/go/cppbridge" + "github.com/prometheus/prometheus/pp/go/model" +) + +// +// Deduplicator +// + +// Deduplicator accumulates and deduplicates incoming values. +type Deduplicator interface { + // Add values to deduplicator by shard ID. + Add(shard uint16, snapshot *cppbridge.LabelSetSnapshot, values []string) + // Values returns collected values. + Values() []string +} + +// deduplicatorCtor constructor [Deduplicator]. +type deduplicatorCtor func(numberOfShards uint16) Deduplicator + +// +// GenericTask +// + +// GenericTask the minimum required task [Generic] implementation. +type GenericTask interface { + // Wait for the task to complete on all shards. + Wait() error +} + +// +// DataStorage +// + +// DataStorage the minimum required [DataStorage] implementation. +type DataStorage interface { + // InstantQuery returns samples for instant query from data storage. + InstantQuery(maxt, valueNotFoundTimestampValue int64, ids []uint32) []cppbridge.Sample + + // QueryDataStorage returns serialized chunks from data storage. + Query(query cppbridge.HeadDataStorageQuery) *cppbridge.HeadDataStorageSerializedChunks +} + +// +// LSS +// + +// LSS the minimum required [LSS] implementation. +type LSS interface { + // QueryLabelNames returns all the unique label names present in lss in sorted order. + QueryLabelNames( + shardID uint16, + matchers []model.LabelMatcher, + dedupAdd func(shardID uint16, snapshot *cppbridge.LabelSetSnapshot, values []string), + ) error + + // QueryLabelValues query labels values to lss and add values to + // the dedup-container that matches the given label matchers. + QueryLabelValues( + shardID uint16, + name string, + matchers []model.LabelMatcher, + dedupAdd func(shardID uint16, snapshot *cppbridge.LabelSetSnapshot, values []string), + ) error + + // QuerySelector returns a created selector that matches the given label matchers. + QuerySelector(shardID uint16, matchers []model.LabelMatcher) (uintptr, *cppbridge.LabelSetSnapshot, error) +} + +// +// Shard +// + +// Shard the minimum required head [Shard] implementation. +type Shard[TDataStorage DataStorage, TLSS LSS] interface { + // DataStorage returns shard [DataStorage]. + DataStorage() TDataStorage + + // LSS returns shard labelset storage [LSS]. + LSS() TLSS + + // ShardID returns the shard ID. + ShardID() uint16 +} + +// +// Head +// + +// Head the minimum required [Head] implementation. +type Head[ + TGenericTask GenericTask, + TDataStorage DataStorage, + TLSS LSS, + TShard Shard[TDataStorage, TLSS], +] interface { + // AcquireQuery acquires the [Head] semaphore with a weight of 1, + // blocking until resources are available or ctx is done. + // On success, returns nil. On failure, returns ctx.Err() and leaves the semaphore unchanged. + AcquireQuery(ctx context.Context) (release func(), err error) + + // CreateTask create a task for operations on the [Head] shards. + CreateTask(taskName string, shardFn func(shard TShard) error) TGenericTask + + // Enqueue the task to be executed on shards [Head]. + Enqueue(t TGenericTask) + + // NumberOfShards returns current number of shards in to [Head]. + NumberOfShards() uint16 +} diff --git a/pp/go/storage/querier/querier.go b/pp/go/storage/querier/querier.go index 50bff850df..47a083f022 100644 --- a/pp/go/storage/querier/querier.go +++ b/pp/go/storage/querier/querier.go @@ -35,112 +35,6 @@ const ( DefaultInstantQueryValueNotFoundTimestampValue int64 = 0 ) -// -// Deduplicator -// - -// Deduplicator accumulates and deduplicates incoming values. -type Deduplicator interface { - // Add values to deduplicator by shard ID. - Add(shard uint16, snapshot *cppbridge.LabelSetSnapshot, values []string) - // Values returns collected values. - Values() []string -} - -// deduplicatorCtor constructor [Deduplicator]. -type deduplicatorCtor func(numberOfShards uint16) Deduplicator - -// -// GenericTask -// - -// GenericTask the minimum required task [Generic] implementation. -type GenericTask interface { - // Wait for the task to complete on all shards. - Wait() error -} - -// -// DataStorage -// - -// DataStorage the minimum required [DataStorage] implementation. -type DataStorage interface { - // InstantQuery returns samples for instant query from data storage. - InstantQuery(maxt, valueNotFoundTimestampValue int64, ids []uint32) []cppbridge.Sample - - // QueryDataStorage returns serialized chunks from data storage. - Query(query cppbridge.HeadDataStorageQuery) *cppbridge.HeadDataStorageSerializedChunks -} - -// -// LSS -// - -// LSS the minimum required [LSS] implementation. -type LSS interface { - // QueryLabelNames returns all the unique label names present in lss in sorted order. - QueryLabelNames( - shardID uint16, - matchers []model.LabelMatcher, - dedupAdd func(shardID uint16, snapshot *cppbridge.LabelSetSnapshot, values []string), - ) error - - // QueryLabelValues query labels values to lss and add values to - // the dedup-container that matches the given label matchers. - QueryLabelValues( - shardID uint16, - name string, - matchers []model.LabelMatcher, - dedupAdd func(shardID uint16, snapshot *cppbridge.LabelSetSnapshot, values []string), - ) error - - // QuerySelector returns a created selector that matches the given label matchers. - QuerySelector(shardID uint16, matchers []model.LabelMatcher) (uintptr, *cppbridge.LabelSetSnapshot, error) -} - -// -// Shard -// - -// Shard the minimum required head [Shard] implementation. -type Shard[TDataStorage DataStorage, TLSS LSS] interface { - // DataStorage returns shard [DataStorage]. - DataStorage() TDataStorage - - // LSS returns shard labelset storage [LSS]. - LSS() TLSS - - // ShardID returns the shard ID. - ShardID() uint16 -} - -// -// Head -// - -// Head the minimum required [Head] implementation. -type Head[ - TGenericTask GenericTask, - TDataStorage DataStorage, - TLSS LSS, - TShard Shard[TDataStorage, TLSS], -] interface { - // AcquireQuery acquires the [Head] semaphore with a weight of 1, - // blocking until resources are available or ctx is done. - // On success, returns nil. On failure, returns ctx.Err() and leaves the semaphore unchanged. - AcquireQuery(ctx context.Context) (release func(), err error) - - // CreateTask create a task for operations on the [Head] shards. - CreateTask(taskName string, shardFn func(shard TShard) error) TGenericTask - - // Enqueue the task to be executed on shards [Head]. - Enqueue(t TGenericTask) - - // NumberOfShards returns current number of shards in to [Head]. - NumberOfShards() uint16 -} - // // Querier // diff --git a/pp/prometheus/relabeler.h b/pp/prometheus/relabeler.h index e3c50e12d8..aa4ef12b00 100644 --- a/pp/prometheus/relabeler.h +++ b/pp/prometheus/relabeler.h @@ -276,12 +276,6 @@ class Cache { cache_drop_.runOptimize(); } - PROMPP_ALWAYS_INLINE void reset() { - cache_relabel_.clear(); - cache_keep_ = roaring::Roaring{}; - cache_drop_ = roaring::Roaring{}; - } - PROMPP_ALWAYS_INLINE double part_of_drops() { if (cache_drop_.cardinality() == 0) { return 0; @@ -337,6 +331,14 @@ class Cache { return {}; } + + // third stage + // update add to cache relabled data. + PROMPP_ALWAYS_INLINE void update(const RelabelerStateUpdate* relabeler_state_update, const uint16_t relabeled_shard_id) { + for (const auto& update : *relabeler_state_update) { + add_relabel(update.incoming_ls_id, update.relabeled_ls_id, relabeled_shard_id); + } + } }; struct RelabelerOptions { From 2df6e105deb51e8f2a307d69794804c9b24c75a0 Mon Sep 17 00:00:00 2001 From: Alexandr Yudin Date: Tue, 26 Aug 2025 09:55:52 +0000 Subject: [PATCH 17/96] add ready notifier --- pp/go/storage/catalog/catalog.go | 1 + pp/go/storage/catalog/catalog_test.go | 37 +++++-- pp/go/storage/catalog/decoder.go | 1 + pp/go/storage/catalog/gc.go | 22 +++- pp/go/storage/catalog/record.go | 1 + pp/go/storage/catalog/record_test.go | 3 +- pp/go/storage/ready/ready.go | 146 ++++++++++++++++++++++++++ 7 files changed, 197 insertions(+), 14 deletions(-) create mode 100644 pp/go/storage/ready/ready.go diff --git a/pp/go/storage/catalog/catalog.go b/pp/go/storage/catalog/catalog.go index 99570c3e6b..7da1f40594 100644 --- a/pp/go/storage/catalog/catalog.go +++ b/pp/go/storage/catalog/catalog.go @@ -10,6 +10,7 @@ import ( "github.com/google/uuid" "github.com/jonboulle/clockwork" "github.com/prometheus/client_golang/prometheus" + "github.com/prometheus/prometheus/pp/go/storage/logger" "github.com/prometheus/prometheus/pp/go/util" ) diff --git a/pp/go/storage/catalog/catalog_test.go b/pp/go/storage/catalog/catalog_test.go index d2cf77b555..d2e9a02b6a 100644 --- a/pp/go/storage/catalog/catalog_test.go +++ b/pp/go/storage/catalog/catalog_test.go @@ -6,6 +6,7 @@ import ( "testing" "time" + "github.com/google/uuid" "github.com/jonboulle/clockwork" "github.com/prometheus/client_golang/prometheus" "github.com/stretchr/testify/suite" @@ -33,10 +34,12 @@ func (s *CatalogSuite) TestHappyPath() { clock := clockwork.NewFakeClockAt(time.Now()) + idGenerator := &testIDGenerator{} + c, err := catalog.New( clock, l, - catalog.DefaultIDGenerator{}, + idGenerator, catalog.DefaultMaxLogFileSize, nil, ) @@ -49,10 +52,9 @@ func (s *CatalogSuite) TestHappyPath() { r1, err := c.Create(nos1) s.Require().NoError(err) - id1 := r1.ID() - s.Require().Equal(id1, r1.ID()) - s.Require().Equal(id1, r1.Dir()) + s.Require().Equal(idGenerator.last(), r1.ID()) + s.Require().Equal(idGenerator.last(), r1.Dir()) s.Require().Equal(nos1, r1.NumberOfShards()) s.Require().Equal(now, r1.CreatedAt()) s.Require().Equal(now, r1.UpdatedAt()) @@ -64,10 +66,9 @@ func (s *CatalogSuite) TestHappyPath() { r2, err := c.Create(nos2) s.Require().NoError(err) - id2 := r2.ID() - s.Require().Equal(id2, r2.ID()) - s.Require().Equal(id2, r2.Dir()) + s.Require().Equal(idGenerator.last(), r2.ID()) + s.Require().Equal(idGenerator.last(), r2.Dir()) s.Require().Equal(nos2, r2.NumberOfShards()) s.Require().Equal(now, r2.CreatedAt()) s.Require().Equal(now, r2.UpdatedAt()) @@ -110,10 +111,12 @@ func (s *CatalogSuite) TestCatalogSyncFail() { clock := clockwork.NewFakeClockAt(time.Now()) + idGenerator := &testIDGenerator{} + c, err := catalog.New( clock, l, - catalog.DefaultIDGenerator{}, + idGenerator, catalog.DefaultMaxLogFileSize, prometheus.DefaultRegisterer, ) @@ -138,7 +141,7 @@ func (s *CatalogSuite) TestCatalogSyncFail() { c, err = catalog.New( clock, l, - catalog.DefaultIDGenerator{}, + idGenerator, catalog.DefaultMaxLogFileSize, nil, ) @@ -158,3 +161,19 @@ func (s *CatalogSuite) TestCatalogSyncFail() { s.Require().Equal(r1.DeletedAt(), restoredR1.DeletedAt()) s.Require().Equal(r1.Status(), restoredR1.Status()) } + +// testIDGenerator generator UUID for test. +type testIDGenerator struct { + lastUUID uuid.UUID +} + +// Generate UUID. Implementation [catalog.IDGenerator]. +func (g *testIDGenerator) Generate() uuid.UUID { + g.lastUUID = uuid.New() + return g.lastUUID +} + +// last returns last UUID as string. +func (g *testIDGenerator) last() string { + return g.lastUUID.String() +} diff --git a/pp/go/storage/catalog/decoder.go b/pp/go/storage/catalog/decoder.go index 07b0252f6b..d38b95cdd4 100644 --- a/pp/go/storage/catalog/decoder.go +++ b/pp/go/storage/catalog/decoder.go @@ -9,6 +9,7 @@ import ( "io" "github.com/google/uuid" + "github.com/prometheus/prometheus/pp/go/util/optional" ) diff --git a/pp/go/storage/catalog/gc.go b/pp/go/storage/catalog/gc.go index cd0488feb3..c14565c939 100644 --- a/pp/go/storage/catalog/gc.go +++ b/pp/go/storage/catalog/gc.go @@ -7,21 +7,34 @@ import ( "path/filepath" "time" - "github.com/prometheus/prometheus/pp/go/relabeler/head/ready" - "github.com/prometheus/prometheus/pp/go/relabeler/logger" + "github.com/prometheus/prometheus/pp/go/storage/logger" ) +// +// Notifiable +// + +// Notifiable notifies the recipient that it is ready to work. +type Notifiable interface { + // ReadyChan notifies the recipient that it is ready to work. + ReadyChan() <-chan struct{} +} + +// +// GC +// + // GC garbage collector for old [Head]. type GC struct { dataDir string catalog *Catalog - readyNotifiable ready.Notifiable + readyNotifiable Notifiable stop chan struct{} stopped chan struct{} } // NewGC init new [GC]. -func NewGC(dataDir string, catalog *Catalog, readyNotifiable ready.Notifiable) *GC { +func NewGC(dataDir string, catalog *Catalog, readyNotifiable Notifiable) *GC { return &GC{ dataDir: dataDir, catalog: catalog, @@ -89,6 +102,7 @@ func (gc *GC) Run(ctx context.Context) error { case <-gc.stop: return errors.New("stopped") } + select { case <-ctx.Done(): return ctx.Err() diff --git a/pp/go/storage/catalog/record.go b/pp/go/storage/catalog/record.go index 6958343e66..e2ac07a984 100644 --- a/pp/go/storage/catalog/record.go +++ b/pp/go/storage/catalog/record.go @@ -5,6 +5,7 @@ import ( "sync/atomic" "github.com/google/uuid" + "github.com/prometheus/prometheus/pp/go/util/optional" ) diff --git a/pp/go/storage/catalog/record_test.go b/pp/go/storage/catalog/record_test.go index 254443b595..75ffa9edd7 100644 --- a/pp/go/storage/catalog/record_test.go +++ b/pp/go/storage/catalog/record_test.go @@ -3,8 +3,9 @@ package catalog_test import ( "testing" - "github.com/prometheus/prometheus/pp/go/storage/catalog" "github.com/stretchr/testify/suite" + + "github.com/prometheus/prometheus/pp/go/storage/catalog" ) type RecordSuite struct { diff --git a/pp/go/storage/ready/ready.go b/pp/go/storage/ready/ready.go new file mode 100644 index 0000000000..3dd69ae0db --- /dev/null +++ b/pp/go/storage/ready/ready.go @@ -0,0 +1,146 @@ +package ready + +import ( + "sync" + "sync/atomic" +) + +// +// Notifier +// + +// Notifier the sender notifies about readiness for work. +type Notifier interface { + // NotifyReady the sender notifies about readiness for work. + NotifyReady() +} + +// +// Notifiable +// + +// Notifiable notifies the recipient that it is ready to work. +type Notifiable interface { + // ReadyChan notifies the recipient that it is ready to work. + ReadyChan() <-chan struct{} +} + +// +// Builder +// + +// Builder for creating [MultiNotifiable]. +type Builder struct { + input []Notifiable +} + +// NewMultiNotifiableBuilder init new [Builder]. +func NewMultiNotifiableBuilder() *Builder { + return &Builder{} +} + +// Add [Notifiable] to list. +func (b *Builder) Add(notifiable Notifiable) *Builder { + b.input = append(b.input, notifiable) + return b +} + +// Build creating [MultiNotifiable] from [Notifiable]s. +func (b *Builder) Build() *MultiNotifiable { + mn := &MultiNotifiable{ + ready: make(chan struct{}), + closed: make(chan struct{}), + } + + mn.counter.Add(int64(len(b.input))) + for _, notifiable := range b.input { + go func(notifiable Notifiable) { + select { + case <-notifiable.ReadyChan(): + if mn.counter.Add(-1) == 0 { + mn.setReady() + } + case <-mn.closed: + } + }(notifiable) + } + + return mn +} + +// +// MultiNotifiable +// + +// MultiNotifiable aggregates multiple [Notifiable]s. +type MultiNotifiable struct { + readyOnce sync.Once + ready chan struct{} + closedOnce sync.Once + closed chan struct{} + counter atomic.Int64 +} + +// Close stop [MultiNotifiable]. +func (mn *MultiNotifiable) Close() error { + mn.setClosed() + return nil +} + +// ReadyChan notifies the recipient that it is ready to work. +func (mn *MultiNotifiable) ReadyChan() <-chan struct{} { + return mn.ready +} + +// setClosed set once [MultiNotifiable] is closed. +func (mn *MultiNotifiable) setClosed() { + mn.closedOnce.Do(func() { + close(mn.closed) + }) +} + +// setReady set once [MultiNotifiable] is ready. +func (mn *MultiNotifiable) setReady() { + mn.readyOnce.Do(func() { + close(mn.ready) + }) +} + +// +// NotifiableNotifier +// + +// NotifiableNotifier the sender notifies about readiness for work, notifies the recipient that it is ready to work. +type NotifiableNotifier struct { + once sync.Once + c chan struct{} +} + +// NewNotifiableNotifier init new [NotifiableNotifier]. +func NewNotifiableNotifier() *NotifiableNotifier { + return &NotifiableNotifier{ + c: make(chan struct{}), + } +} + +// NotifyReady the sender notifies about readiness for work. +func (nn *NotifiableNotifier) NotifyReady() { + nn.once.Do(func() { + close(nn.c) + }) +} + +// ReadyChan notifies the recipient that it is ready to work. +func (nn *NotifiableNotifier) ReadyChan() <-chan struct{} { + return nn.c +} + +// +// NoOpNotifier +// + +// NoOpNotifier do nothing notifier. +type NoOpNotifier struct{} + +// NotifyReady implementation [Notifier], do nothing. +func (NoOpNotifier) NotifyReady() {} From 40b0e441902b8be250d68cd12501db28550b3dd0 Mon Sep 17 00:00:00 2001 From: Alexandr Yudin Date: Tue, 26 Aug 2025 15:30:10 +0000 Subject: [PATCH 18/96] add builder --- pp/go/storage/catalog/catalog.go | 24 +++--- pp/go/storage/catalog/catalog_test.go | 3 +- pp/go/storage/catalog/gc.go | 27 ++++-- pp/go/storage/head/builder/builder.go | 85 ++++++++++++++++++- pp/go/storage/head/container/weighted.go | 12 +-- pp/go/storage/head/head/head.go | 12 ++- pp/go/storage/head/head/head_test.go | 3 + pp/go/storage/head/shard/lss.go | 8 ++ pp/go/storage/head/shard/shard.go | 13 +++ .../storage/head/shard/wal/writer/notifier.go | 42 +++++++++ pp/go/storage/querier/querier_test.go | 3 + 11 files changed, 200 insertions(+), 32 deletions(-) create mode 100644 pp/go/storage/head/shard/wal/writer/notifier.go diff --git a/pp/go/storage/catalog/catalog.go b/pp/go/storage/catalog/catalog.go index 7da1f40594..8ded7b67b9 100644 --- a/pp/go/storage/catalog/catalog.go +++ b/pp/go/storage/catalog/catalog.go @@ -124,17 +124,17 @@ func (c *Catalog) Compact() error { } // Create creates new [Record] and write to [Log]. -func (c *Catalog) Create(numberOfShards uint16) (r *Record, err error) { +func (c *Catalog) Create(numberOfShards uint16) (*Record, error) { c.mtx.Lock() defer c.mtx.Unlock() - if err = c.compactIfNeeded(); err != nil { + if err := c.compactIfNeeded(); err != nil { return nil, fmt.Errorf(compactErr, err) } id := c.idGenerator.Generate() now := c.clock.Now().UnixMilli() - r = &Record{ + r := &Record{ id: id, numberOfShards: numberOfShards, createdAt: now, @@ -144,7 +144,7 @@ func (c *Catalog) Create(numberOfShards uint16) (r *Record, err error) { status: StatusNew, } - if err = c.log.Write(r); err != nil { + if err := c.log.Write(r); err != nil { return r, fmt.Errorf(logWriteErr, err) } c.records[id.String()] = r @@ -153,11 +153,11 @@ func (c *Catalog) Create(numberOfShards uint16) (r *Record, err error) { } // Delete record by ID. -func (c *Catalog) Delete(id string) (err error) { +func (c *Catalog) Delete(id string) error { c.mtx.Lock() defer c.mtx.Unlock() - if err = c.compactIfNeeded(); err != nil { + if err := c.compactIfNeeded(); err != nil { return fmt.Errorf(compactErr, err) } @@ -170,7 +170,7 @@ func (c *Catalog) Delete(id string) (err error) { changed.deletedAt = c.clock.Now().UnixMilli() changed.updatedAt = r.deletedAt - if err = c.log.Write(changed); err != nil { + if err := c.log.Write(changed); err != nil { return fmt.Errorf(logWriteErr, err) } @@ -194,13 +194,11 @@ func (c *Catalog) Get(id string) (*Record, error) { } // List returns slice of records with filter and sort. -func (c *Catalog) List( - filterFn func(record *Record) bool, - sortLess func(lhs, rhs *Record) bool, -) (records []*Record, err error) { +func (c *Catalog) List(filterFn func(record *Record) bool, sortLess func(lhs, rhs *Record) bool) []*Record { c.mtx.Lock() defer c.mtx.Unlock() - records = make([]*Record, 0, len(c.records)) + + records := make([]*Record, 0, len(c.records)) for _, record := range c.records { if filterFn != nil && !filterFn(record) { continue @@ -214,7 +212,7 @@ func (c *Catalog) List( }) } - return records, nil + return records } // OnDiskSize size of [Log] file on disk. diff --git a/pp/go/storage/catalog/catalog_test.go b/pp/go/storage/catalog/catalog_test.go index d2e9a02b6a..5718b7a81f 100644 --- a/pp/go/storage/catalog/catalog_test.go +++ b/pp/go/storage/catalog/catalog_test.go @@ -92,8 +92,7 @@ func (s *CatalogSuite) TestHappyPath() { ) s.Require().NoError(err) - records, err := c.List(nil, nil) - s.Require().NoError(err) + records := c.List(nil, nil) sort.Slice(records, func(i, j int) bool { return records[i].CreatedAt() < records[j].CreatedAt() }) diff --git a/pp/go/storage/catalog/gc.go b/pp/go/storage/catalog/gc.go index c14565c939..6b5192d9f0 100644 --- a/pp/go/storage/catalog/gc.go +++ b/pp/go/storage/catalog/gc.go @@ -10,6 +10,19 @@ import ( "github.com/prometheus/prometheus/pp/go/storage/logger" ) +// +// HeadsCatalog +// + +// HeadsCatalog of current head records. +type HeadsCatalog interface { + // Delete record by ID. + Delete(id string) error + + // List returns slice of records with filter and sort. + List(filterFn func(record *Record) bool, sortLess func(lhs, rhs *Record) bool) []*Record +} + // // Notifiable // @@ -27,14 +40,14 @@ type Notifiable interface { // GC garbage collector for old [Head]. type GC struct { dataDir string - catalog *Catalog + catalog HeadsCatalog readyNotifiable Notifiable stop chan struct{} stopped chan struct{} } // NewGC init new [GC]. -func NewGC(dataDir string, catalog *Catalog, readyNotifiable Notifiable) *GC { +func NewGC(dataDir string, catalog HeadsCatalog, readyNotifiable Notifiable) *GC { return &GC{ dataDir: dataDir, catalog: catalog, @@ -49,7 +62,7 @@ func (gc *GC) Iterate() { logger.Debugf("catalog gc iteration: head started") defer logger.Debugf("catalog gc iteration: head ended") - records, err := gc.catalog.List( + records := gc.catalog.List( func(record *Record) bool { return record.DeletedAt() == 0 }, @@ -57,10 +70,6 @@ func (gc *GC) Iterate() { return lhs.CreatedAt() < rhs.CreatedAt() }, ) - if err != nil { - logger.Debugf("catalog gc failed", err) - return - } for _, record := range records { if record.deletedAt != 0 { @@ -76,12 +85,12 @@ func (gc *GC) Iterate() { continue } - if err = os.RemoveAll(filepath.Join(gc.dataDir, record.Dir())); err != nil { + if err := os.RemoveAll(filepath.Join(gc.dataDir, record.Dir())); err != nil { logger.Errorf("failed to remote head dir: %w", err) return } - if err = gc.catalog.Delete(record.ID()); err != nil { + if err := gc.catalog.Delete(record.ID()); err != nil { logger.Errorf("failed to delete head record: %w", err) return } diff --git a/pp/go/storage/head/builder/builder.go b/pp/go/storage/head/builder/builder.go index bcb526b0c0..0563391d36 100644 --- a/pp/go/storage/head/builder/builder.go +++ b/pp/go/storage/head/builder/builder.go @@ -1,3 +1,86 @@ package builder -// TODO ? +import ( + "errors" + "fmt" + "os" + "path/filepath" + + "github.com/prometheus/client_golang/prometheus" + "github.com/prometheus/prometheus/pp/go/storage/catalog" +) + +// +// HeadsCatalog +// + +// HeadsCatalog of current head records. +type HeadsCatalog interface { + Create(numberOfShards uint16) (*catalog.Record, error) + + // Delete record by ID. + Delete(id string) error + + // List returns slice of records with filter and sort. + List(filterFn func(record *catalog.Record) bool, sortLess func(lhs, rhs *catalog.Record) bool) []*catalog.Record +} + +// +// Head +// + +// Head the minimum required Head implementation for a container. +type Head[T any] interface { + *T +} + +// +// Builder +// + +type Builder[TShard any, TGoroutineShard any, T any, THead Head[T]] struct { + catalog HeadsCatalog + dir string + generation uint64 + headFactory func( + id string, + releaseHeadFn func(), + generation uint64, + numberOfShards uint16, + registerer prometheus.Registerer, + ) (THead, error) + registerer prometheus.Registerer +} + +func (b *Builder[TShard, TGoroutineShard, T, THead]) Build(numberOfShards uint16) (THead, error) { + headRecord, err := b.catalog.Create(numberOfShards) + if err != nil { + return nil, err + } + + headDir := filepath.Join(b.dir, headRecord.ID()) + //revive:disable-next-line:add-constant // this is already a constant + if err = os.Mkdir(headDir, 0o777); err != nil { //nolint:gosec // need this permissions + return nil, err + } + defer func() { + if err != nil { + err = errors.Join(err, os.RemoveAll(headDir)) + } + }() + + h, err := b.headFactory( + headRecord.ID(), + headRecord.Acquire(), + b.generation, + numberOfShards, + b.registerer, + ) + if err != nil { + return nil, fmt.Errorf("failed to create head: %w", err) + } + + b.generation++ + + return h, nil +} diff --git a/pp/go/storage/head/container/weighted.go b/pp/go/storage/head/container/weighted.go index f9efff564d..7388f4ab28 100644 --- a/pp/go/storage/head/container/weighted.go +++ b/pp/go/storage/head/container/weighted.go @@ -16,28 +16,28 @@ type Head[T any] interface { } // Weighted container for [Head] with weighted locker. -type Weighted[T any, H Head[T]] struct { +type Weighted[T any, THead Head[T]] struct { wlocker *locker.Weighted head *T } // NewWeighted init new [Weighted]. -func NewWeighted[T any, H Head[T]](head H) *Weighted[T, H] { - return &Weighted[T, H]{ +func NewWeighted[T any, THead Head[T]](head THead) *Weighted[T, THead] { + return &Weighted[T, THead]{ wlocker: locker.NewWeighted(2 * head.Concurrency()), // x2 for back pressure head: head, } } // Get the active head [Head] under the non-exlusive lock and return. -func (c *Weighted[T, H]) Get() H { +func (c *Weighted[T, THead]) Get() THead { return (*T)(atomic.LoadPointer( (*unsafe.Pointer)(unsafe.Pointer(&c.head))), // #nosec G103 // it's meant to be that way ) } // Replace the active head [Head] with a new head under the exlusive priority lock. -func (c *Weighted[T, H]) Replace(ctx context.Context, newHead H) error { +func (c *Weighted[T, THead]) Replace(ctx context.Context, newHead THead) error { unlock, err := c.wlocker.LockWithPriority(ctx) if err != nil { return fmt.Errorf("weighted lock with priority: %w", err) @@ -54,7 +54,7 @@ func (c *Weighted[T, H]) Replace(ctx context.Context, newHead H) error { } // With calls fn(h Head) under the non-exlusive lock. -func (c *Weighted[T, H]) With(ctx context.Context, fn func(h H) error) error { +func (c *Weighted[T, THead]) With(ctx context.Context, fn func(h THead) error) error { runlock, err := c.wlocker.RLock(ctx) if err != nil { return fmt.Errorf("weighted rlock: %w", err) diff --git a/pp/go/storage/head/head/head.go b/pp/go/storage/head/head/head.go index 1f3156cc32..0927b99b14 100644 --- a/pp/go/storage/head/head/head.go +++ b/pp/go/storage/head/head/head.go @@ -59,7 +59,9 @@ type Head[TShard Shard, TGorutineShard Shard] struct { id string generation uint64 - gshardCtor func(s TShard, numberOfShards uint16) TGorutineShard + gshardCtor func(s TShard, numberOfShards uint16) TGorutineShard + releaseHeadFn func() + shards []TShard taskChs []chan *task.Generic[TGorutineShard] querySemaphore *locker.Weighted @@ -87,6 +89,8 @@ func NewHead[TShard Shard, TGoroutineShard Shard]( id string, shards []TShard, gshardCtor func(TShard, uint16) TGoroutineShard, + releaseHeadFn func(), + generation uint64, numberOfShards uint16, registerer prometheus.Registerer, ) *Head[TShard, TGoroutineShard] { @@ -100,7 +104,9 @@ func NewHead[TShard Shard, TGoroutineShard Shard]( factory := util.NewUnconflictRegisterer(registerer) h := &Head[TShard, TGoroutineShard]{ id: id, + generation: generation, gshardCtor: gshardCtor, + releaseHeadFn: releaseHeadFn, shards: shards, taskChs: taskChs, numberOfShards: uint16(len(shards)), // #nosec G115 // no overflow @@ -190,6 +196,10 @@ func (h *Head[TShard, TGorutineShard]) Close() error { err = errors.Join(err, s.Close()) } + if h.releaseHeadFn != nil { + h.releaseHeadFn() + } + return err } diff --git a/pp/go/storage/head/head/head_test.go b/pp/go/storage/head/head/head_test.go index cd0cd38401..b7f9ffc9fb 100644 --- a/pp/go/storage/head/head/head_test.go +++ b/pp/go/storage/head/head/head_test.go @@ -15,11 +15,14 @@ func TestXxx(t *testing.T) { sd := shard.NewShard(lss, ds, wl, 0) id := "test-head-id" numberOfShards := uint16(2) + generation := uint64(0) h := head.NewHead( id, []*shard.Shard[*testWal]{sd}, shard.NewPerGoroutineShard[*testWal], + nil, + generation, numberOfShards, nil, ) diff --git a/pp/go/storage/head/shard/lss.go b/pp/go/storage/head/shard/lss.go index c5e5b085e4..de0e6dda42 100644 --- a/pp/go/storage/head/shard/lss.go +++ b/pp/go/storage/head/shard/lss.go @@ -18,6 +18,14 @@ type LSS struct { once sync.Once } +// NewLSS init new [LSS]. +func NewLSS() *LSS { + return &LSS{ + input: cppbridge.NewLssStorage(), + target: cppbridge.NewQueryableLssStorage(), + } +} + // AllocatedMemory return size of allocated memory for labelset storages. func (l *LSS) AllocatedMemory() uint64 { l.locker.RLock() diff --git a/pp/go/storage/head/shard/shard.go b/pp/go/storage/head/shard/shard.go index 04c894b929..a85bbe786c 100644 --- a/pp/go/storage/head/shard/shard.go +++ b/pp/go/storage/head/shard/shard.go @@ -42,6 +42,19 @@ func NewShard[TWal Wal]( } } +// NewShardWithWal init new [Shard] with [Wal]. +func NewShardWithWal[TWal Wal]( + wal TWal, + shardID uint16, +) *Shard[TWal] { + return &Shard[TWal]{ + id: shardID, + lss: NewLSS(), + dataStorage: NewDataStorage(), + wal: wal, + } +} + // Close closes the wal segmentWriter. func (s *Shard[TWal]) Close() error { return s.wal.Close() diff --git a/pp/go/storage/head/shard/wal/writer/notifier.go b/pp/go/storage/head/shard/wal/writer/notifier.go new file mode 100644 index 0000000000..f8a1ecd39b --- /dev/null +++ b/pp/go/storage/head/shard/wal/writer/notifier.go @@ -0,0 +1,42 @@ +package writer + +import ( + "slices" + "sync" +) + +// LastAppendedSegmentIDSetter the setter of the last added segment ID. +type LastAppendedSegmentIDSetter interface { + SetLastAppendedSegmentID(segmentID uint32) +} + +// SegmentWriteNotifier notifies that the segment has been written. +type SegmentWriteNotifier struct { + locker sync.Mutex + shards []uint32 + setter LastAppendedSegmentIDSetter +} + +// NewSegmentWriteNotifier init new [SegmentWriteNotifier]. +func NewSegmentWriteNotifier(numberOfShards uint16, setter LastAppendedSegmentIDSetter) *SegmentWriteNotifier { + return &SegmentWriteNotifier{ + shards: make([]uint32, numberOfShards), + setter: setter, + } +} + +// NotifySegmentIsWritten notify that the segment has been written for shard. +func (swn *SegmentWriteNotifier) NotifySegmentIsWritten(shardID uint16) { + swn.locker.Lock() + defer swn.locker.Unlock() + swn.shards[shardID]++ + minNumberOfSegments := slices.Min(swn.shards) + if minNumberOfSegments > 0 { + swn.setter.SetLastAppendedSegmentID(minNumberOfSegments - 1) + } +} + +// Set for shard number of segments. +func (swn *SegmentWriteNotifier) Set(shardID uint16, numberOfSegments uint32) { + swn.shards[shardID] = numberOfSegments +} diff --git a/pp/go/storage/querier/querier_test.go b/pp/go/storage/querier/querier_test.go index 3a32e05b68..74cf19feb8 100644 --- a/pp/go/storage/querier/querier_test.go +++ b/pp/go/storage/querier/querier_test.go @@ -16,11 +16,14 @@ func TestXxx(t *testing.T) { sd := shard.NewShard(lss, ds, wl, 0) id := "test-head-id" numberOfShards := uint16(2) + generation := uint64(0) h := head.NewHead( id, []*shard.Shard[*testWal]{sd}, shard.NewPerGoroutineShard[*testWal], + nil, + generation, numberOfShards, nil, ) From 250828e7d3ccf9693cdbfef814a79eece00099ef Mon Sep 17 00:00:00 2001 From: Alexandr Yudin Date: Tue, 26 Aug 2025 16:04:13 +0000 Subject: [PATCH 19/96] for save --- pp/go/storage/head/builder/builder.go | 29 +++++++++++++++++++++++++-- pp/go/storage/head/loader/loader.go | 5 +++++ pp/go/storage/head/manager/manager.go | 5 ++--- pp/go/storage/head/manager/timer.go | 17 ++++++++++++---- 4 files changed, 47 insertions(+), 9 deletions(-) create mode 100644 pp/go/storage/head/loader/loader.go diff --git a/pp/go/storage/head/builder/builder.go b/pp/go/storage/head/builder/builder.go index 0563391d36..314bea1784 100644 --- a/pp/go/storage/head/builder/builder.go +++ b/pp/go/storage/head/builder/builder.go @@ -38,7 +38,8 @@ type Head[T any] interface { // Builder // -type Builder[TShard any, TGoroutineShard any, T any, THead Head[T]] struct { +// Builder building new [Head] from factory with parameters. +type Builder[T any, THead Head[T]] struct { catalog HeadsCatalog dir string generation uint64 @@ -52,7 +53,31 @@ type Builder[TShard any, TGoroutineShard any, T any, THead Head[T]] struct { registerer prometheus.Registerer } -func (b *Builder[TShard, TGoroutineShard, T, THead]) Build(numberOfShards uint16) (THead, error) { +// NewBuilder init new [Builder]. +func NewBuilder[T any, THead Head[T]]( + hcatalog HeadsCatalog, + dir string, + generation uint64, + headFactory func( + id string, + releaseHeadFn func(), + generation uint64, + numberOfShards uint16, + registerer prometheus.Registerer, + ) (THead, error), + registerer prometheus.Registerer, +) *Builder[T, THead] { + return &Builder[T, THead]{ + catalog: hcatalog, + dir: dir, + generation: generation, + headFactory: headFactory, + registerer: registerer, + } +} + +// Build new [Head]. +func (b *Builder[T, THead]) Build(numberOfShards uint16) (THead, error) { headRecord, err := b.catalog.Create(numberOfShards) if err != nil { return nil, err diff --git a/pp/go/storage/head/loader/loader.go b/pp/go/storage/head/loader/loader.go new file mode 100644 index 0000000000..43536fb0ca --- /dev/null +++ b/pp/go/storage/head/loader/loader.go @@ -0,0 +1,5 @@ +package loader + +type Loader struct { + // +} diff --git a/pp/go/storage/head/manager/manager.go b/pp/go/storage/head/manager/manager.go index 65c81d63ae..335a60bf2d 100644 --- a/pp/go/storage/head/manager/manager.go +++ b/pp/go/storage/head/manager/manager.go @@ -41,8 +41,7 @@ type Keeper[THead Head] interface { // HeadBuilder builder for the [Head]. type HeadBuilder[THead Head] interface { - // inputRelabelerConfigs []*config.InputRelabelerConfig, - BuildWithConfig(numberOfShards uint16) (THead, error) + Build(numberOfShards uint16) (THead, error) } type Manager[THead Head] struct { @@ -146,7 +145,7 @@ func (m *Manager[THead]) loop(ctx context.Context) { } func (m *Manager[THead]) rotate(ctx context.Context) error { - newHead, err := m.headBuilder.BuildWithConfig(m.numberOfShards) + newHead, err := m.headBuilder.Build(m.numberOfShards) if err != nil { return fmt.Errorf("failed to build a new head: %w", err) } diff --git a/pp/go/storage/head/manager/timer.go b/pp/go/storage/head/manager/timer.go index a35889a84b..d60b86ac38 100644 --- a/pp/go/storage/head/manager/timer.go +++ b/pp/go/storage/head/manager/timer.go @@ -23,14 +23,18 @@ type RotateTimer struct { rndDurationBlock int64 } -// NewRotateTimer - init new RotateTimer. The duration durationBlock and delayAfterNotify must be greater than zero; +// NewRotateTimer init new [RotateTimer]. The duration durationBlock and delayAfterNotify must be greater than zero; // if not, Ticker will panic. Stop the ticker to release associated resources. func NewRotateTimer(clock clockwork.Clock, desiredBlockFormationDuration time.Duration) *RotateTimer { - return NewRotateTimerWithSeed(clock, desiredBlockFormationDuration, uint64(clock.Now().UnixNano())) + return NewRotateTimerWithSeed( + clock, + desiredBlockFormationDuration, + uint64(clock.Now().UnixNano()), // #nosec G115 // no overflow + ) } -// NewRotateTimerWithSeed - init new RotateTimer. The duration durationBlock and delayAfterNotify must be greater than zero; -// if not, Ticker will panic. Stop the ticker to release associated resources. +// NewRotateTimerWithSeed init new [RotateTimer]. The duration durationBlock and delayAfterNotify +// must be greater than zero; if not, Ticker will panic. Stop the ticker to release associated resources. func NewRotateTimerWithSeed( clock clockwork.Clock, desiredBlockFormationDuration time.Duration, @@ -96,11 +100,13 @@ func (rt *RotateTimer) Stop() { // ConstantIntervalTimer // +// ConstantIntervalTimer timer with contatnt interval duration. type ConstantIntervalTimer struct { timer clockwork.Timer interval time.Duration } +// NewConstantIntervalTimer init new [ConstantIntervalTimer]. func NewConstantIntervalTimer(clock clockwork.Clock, interval time.Duration) *ConstantIntervalTimer { return &ConstantIntervalTimer{ timer: clock.NewTimer(interval), @@ -108,14 +114,17 @@ func NewConstantIntervalTimer(clock clockwork.Clock, interval time.Duration) *Co } } +// Chan returns chan with tick time. func (t *ConstantIntervalTimer) Chan() <-chan time.Time { return t.timer.Chan() } +// Reset to a constant interval duration. func (t *ConstantIntervalTimer) Reset() { t.timer.Reset(t.interval) } +// Stop timer. func (t *ConstantIntervalTimer) Stop() { t.timer.Stop() } From da66d5d36d863ab6b33cf4e4b485b1ffde534761 Mon Sep 17 00:00:00 2001 From: Alexandr Yudin Date: Wed, 27 Aug 2025 08:15:48 +0000 Subject: [PATCH 20/96] for save --- pp-pkg/configadapter/headkeeper.go | 7 +- pp/go/storage/factory.go | 104 +++++ pp/go/storage/head/builder/builder.go | 38 +- pp/go/storage/head/loader/shard_loader.go | 103 +++++ pp/go/storage/head/manager/manager.go | 3 - pp/go/storage/head/shard/shard.go | 13 - .../storage/head/shard/wal/writer/notifier.go | 22 +- pp/go/storage/interface.go | 296 ++++++------- pp/go/storage/models.go | 35 +- pp/go/storage/task.go | 396 +++++++++--------- 10 files changed, 626 insertions(+), 391 deletions(-) create mode 100644 pp/go/storage/factory.go create mode 100644 pp/go/storage/head/loader/shard_loader.go diff --git a/pp-pkg/configadapter/headkeeper.go b/pp-pkg/configadapter/headkeeper.go index 0867f9f29e..b2059bf4c7 100644 --- a/pp-pkg/configadapter/headkeeper.go +++ b/pp-pkg/configadapter/headkeeper.go @@ -11,7 +11,10 @@ import ( var DefaultNumberOfShards uint16 = 2 // HeadKeeperApplyConfig returns func-adapter for apply config on [headkeeper.HeadKeeper]. -func HeadKeeperApplyConfig(ctx context.Context, hk *manager.Manager) func(cfg *prom_config.Config) error { +func HeadKeeperApplyConfig[THead any]( + ctx context.Context, + hk *manager.Manager[THead], +) func(cfg *prom_config.Config) error { return func(cfg *prom_config.Config) error { rCfg, err := cfg.GetReceiverConfig() if err != nil { @@ -23,6 +26,6 @@ func HeadKeeperApplyConfig(ctx context.Context, hk *manager.Manager) func(cfg *p numberOfShards = DefaultNumberOfShards } - return hk.ApplyConfig(ctx, rCfg.Configs, numberOfShards) + return hk.ApplyConfig(ctx, numberOfShards) } } diff --git a/pp/go/storage/factory.go b/pp/go/storage/factory.go new file mode 100644 index 0000000000..4f2aacc5f0 --- /dev/null +++ b/pp/go/storage/factory.go @@ -0,0 +1,104 @@ +package storage + +import ( + "fmt" + "os" + "path/filepath" + + "github.com/prometheus/client_golang/prometheus" + + "github.com/prometheus/prometheus/pp/go/cppbridge" + "github.com/prometheus/prometheus/pp/go/storage/head/head" + "github.com/prometheus/prometheus/pp/go/storage/head/shard" + "github.com/prometheus/prometheus/pp/go/storage/head/shard/wal" + "github.com/prometheus/prometheus/pp/go/storage/head/shard/wal/writer" +) + +// WalOnDisk wal on disk. +type WalOnDisk = wal.Wal[ + *cppbridge.EncodedSegment, + cppbridge.WALEncoderStats, + *writer.Buffered[*cppbridge.EncodedSegment], +] + +// ShardOnDisk [shard.Shard] with [WalOnDisk]. +type ShardOnDisk = shard.Shard[*WalOnDisk] + +// HeadOnDisk [head.Head] with [ShardOnDisk]. +type HeadOnDisk = head.Head[*ShardOnDisk, *shard.PerGoroutineShard[*WalOnDisk]] + +// ShardOnDiskConstructor create [shard.Shard] with [wal.Wal] which is written to disk. +func ShardOnDiskConstructor( + dir string, + setLastAppendedSegmentID func(segmentID uint32), + maxSegmentSize uint32, + numberOfShards, shardID uint16, +) (*ShardOnDisk, error) { + shardFile, err := os.Create(filepath.Join(filepath.Clean(dir), fmt.Sprintf("shard_%d.wal", shardID))) + if err != nil { + return nil, fmt.Errorf("failed to create shard wal file id %d: %w", shardID, err) + } + + defer func() { + if err == nil { + return + } + _ = shardFile.Close() + }() + + swn := writer.NewSegmentWriteNotifier(numberOfShards, setLastAppendedSegmentID) + + sw, err := writer.NewBuffered(shardID, shardFile, writer.WriteSegment[*cppbridge.EncodedSegment], swn) + if err != nil { + return nil, fmt.Errorf("failed to create buffered writer shard id %d: %w", shardID, err) + } + + lss := shard.NewLSS() + + // logShards is 0 for single encoder + shardWalEncoder := cppbridge.NewHeadWalEncoder(shardID, 0, lss.Target()) + + return shard.NewShard( + lss, + shard.NewDataStorage(), + wal.NewWal(shardWalEncoder, sw, maxSegmentSize), + shardID, + ), nil +} + +// HeadConstructor create [head.Head] with [shard.Shard] with [wal.Wal] which is written to disk. +func HeadConstructor( + id, headDir string, + releaseHeadFn func(), + setLastAppendedSegmentID func(segmentID uint32), + generation uint64, + maxSegmentSize uint32, + numberOfShards uint16, + registerer prometheus.Registerer, +) (*HeadOnDisk, error) { + shards := make([]*ShardOnDisk, numberOfShards) + for shardID := range numberOfShards { + s, err := ShardOnDiskConstructor( + headDir, + setLastAppendedSegmentID, + maxSegmentSize, + numberOfShards, + shardID, + ) + if err != nil { + return nil, err + } + + shards[shardID] = s + } + + return head.NewHead( + id, + shards, + shard.NewPerGoroutineShard[*WalOnDisk], + releaseHeadFn, + generation, + numberOfShards, + registerer, + ), nil +} diff --git a/pp/go/storage/head/builder/builder.go b/pp/go/storage/head/builder/builder.go index 314bea1784..62786ec4a1 100644 --- a/pp/go/storage/head/builder/builder.go +++ b/pp/go/storage/head/builder/builder.go @@ -40,17 +40,20 @@ type Head[T any] interface { // Builder building new [Head] from factory with parameters. type Builder[T any, THead Head[T]] struct { - catalog HeadsCatalog - dir string - generation uint64 - headFactory func( - id string, + catalog HeadsCatalog + dir string + generation uint64 + headCtor func( + id, headDir string, releaseHeadFn func(), + setLastAppendedSegmentID func(segmentID uint32), generation uint64, + maxSegmentSize uint32, numberOfShards uint16, registerer prometheus.Registerer, ) (THead, error) - registerer prometheus.Registerer + maxSegmentSize uint32 + registerer prometheus.Registerer } // NewBuilder init new [Builder]. @@ -58,21 +61,25 @@ func NewBuilder[T any, THead Head[T]]( hcatalog HeadsCatalog, dir string, generation uint64, - headFactory func( - id string, + headCtor func( + id, headDir string, releaseHeadFn func(), + setLastAppendedSegmentID func(segmentID uint32), generation uint64, + maxSegmentSize uint32, numberOfShards uint16, registerer prometheus.Registerer, ) (THead, error), + maxSegmentSize uint32, registerer prometheus.Registerer, ) *Builder[T, THead] { return &Builder[T, THead]{ - catalog: hcatalog, - dir: dir, - generation: generation, - headFactory: headFactory, - registerer: registerer, + catalog: hcatalog, + dir: dir, + generation: generation, + headCtor: headCtor, + maxSegmentSize: maxSegmentSize, + registerer: registerer, } } @@ -94,10 +101,13 @@ func (b *Builder[T, THead]) Build(numberOfShards uint16) (THead, error) { } }() - h, err := b.headFactory( + h, err := b.headCtor( headRecord.ID(), + headDir, headRecord.Acquire(), + headRecord.SetLastAppendedSegmentID, b.generation, + b.maxSegmentSize, numberOfShards, b.registerer, ) diff --git a/pp/go/storage/head/loader/shard_loader.go b/pp/go/storage/head/loader/shard_loader.go new file mode 100644 index 0000000000..56183eb081 --- /dev/null +++ b/pp/go/storage/head/loader/shard_loader.go @@ -0,0 +1,103 @@ +package loader + +import ( + "bufio" + "errors" + "fmt" + "io" + "os" + + "github.com/prometheus/prometheus/pp/go/cppbridge" +) + +type SegmentWriteNotifier interface { + Set(shardID uint16, numberOfSegments uint32) +} + +type ShardLoader struct { + notifier SegmentWriteNotifier + shardFilePath string + maxSegmentSize uint32 + shardID uint16 +} + +func (l *ShardLoader) Load() (result ShardLoadResult) { + targetLss := cppbridge.NewQueryableLssStorage() + dataStorage := NewDataStorage() + + result.Lss = &LSS{ + input: cppbridge.NewLssStorage(), + target: targetLss, + } + result.DataStorage = dataStorage + result.Wal = newCorruptedShardWal() + result.Corrupted = true + + shardWalFile, err := os.OpenFile(l.shardFilePath, os.O_RDWR, 0o600) + if err != nil { + result.Err = err + return + } + + defer func() { + if result.Corrupted { + _ = shardWalFile.Close() + } + }() + + reader := bufio.NewReaderSize(shardWalFile, 1024*1024*4) + _, encoderVersion, offset, err := ReadHeader(reader) + if err != nil { + result.Err = fmt.Errorf("failed to read wal header: %w", err) + return + } + + decoder := cppbridge.NewHeadWalDecoder(targetLss, encoderVersion) + lastReadSegmentID := -1 + + var bytesRead int + for { + var segment DecodedSegment + segment, bytesRead, err = ReadSegment(reader) + if err != nil { + if errors.Is(err, io.EOF) { + break + } + result.Err = fmt.Errorf("failed to read segment: %w", err) + break + } + + err = decoder.DecodeToDataStorage(segment.data, dataStorage.encoder) + if err != nil { + result.Err = fmt.Errorf("failed to decode segment: %w", err) + break + } + + offset += bytesRead + lastReadSegmentID++ + } + + numberOfSegments := lastReadSegmentID + 1 + result.NumberOfSegments = uint32(numberOfSegments) // #nosec G115 // no overflow + sw, err := newSegmentWriter(l.shardID, shardWalFile, l.notifier) + if err != nil { + result.Err = err + return + } + + l.notifier.Set(l.shardID, uint32(numberOfSegments)) // #nosec G115 // no overflow + result.Wal = newShardWal(decoder.CreateEncoder(), l.maxSegmentSize, sw) + if result.Err == nil { + result.Corrupted = false + } + return result +} + +type ShardLoadResult struct { + Lss *LSS + DataStorage *DataStorage + Wal *ShardWal + NumberOfSegments uint32 + Corrupted bool + Err error +} diff --git a/pp/go/storage/head/manager/manager.go b/pp/go/storage/head/manager/manager.go index 335a60bf2d..43b38b21ae 100644 --- a/pp/go/storage/head/manager/manager.go +++ b/pp/go/storage/head/manager/manager.go @@ -6,7 +6,6 @@ import ( "time" "github.com/prometheus/client_golang/prometheus" - "github.com/prometheus/prometheus/pp/go/relabeler/config" "github.com/prometheus/prometheus/pp/go/storage/logger" "github.com/prometheus/prometheus/pp/go/util" ) @@ -45,7 +44,6 @@ type HeadBuilder[THead Head] interface { } type Manager[THead Head] struct { - // activeHead ActiveHeadContainer[THead] headBuilder HeadBuilder[THead] keeper Keeper[THead] @@ -64,7 +62,6 @@ type Manager[THead Head] struct { // ApplyConfig update config. func (m *Manager[THead]) ApplyConfig( ctx context.Context, - inputRelabelerConfigs []*config.InputRelabelerConfig, numberOfShards uint16, ) error { logger.Infof("reconfiguration start") diff --git a/pp/go/storage/head/shard/shard.go b/pp/go/storage/head/shard/shard.go index a85bbe786c..04c894b929 100644 --- a/pp/go/storage/head/shard/shard.go +++ b/pp/go/storage/head/shard/shard.go @@ -42,19 +42,6 @@ func NewShard[TWal Wal]( } } -// NewShardWithWal init new [Shard] with [Wal]. -func NewShardWithWal[TWal Wal]( - wal TWal, - shardID uint16, -) *Shard[TWal] { - return &Shard[TWal]{ - id: shardID, - lss: NewLSS(), - dataStorage: NewDataStorage(), - wal: wal, - } -} - // Close closes the wal segmentWriter. func (s *Shard[TWal]) Close() error { return s.wal.Close() diff --git a/pp/go/storage/head/shard/wal/writer/notifier.go b/pp/go/storage/head/shard/wal/writer/notifier.go index f8a1ecd39b..d1d7b11592 100644 --- a/pp/go/storage/head/shard/wal/writer/notifier.go +++ b/pp/go/storage/head/shard/wal/writer/notifier.go @@ -5,23 +5,21 @@ import ( "sync" ) -// LastAppendedSegmentIDSetter the setter of the last added segment ID. -type LastAppendedSegmentIDSetter interface { - SetLastAppendedSegmentID(segmentID uint32) -} - // SegmentWriteNotifier notifies that the segment has been written. type SegmentWriteNotifier struct { - locker sync.Mutex - shards []uint32 - setter LastAppendedSegmentIDSetter + locker sync.Mutex + shards []uint32 + setLastAppendedSegmentID func(segmentID uint32) } // NewSegmentWriteNotifier init new [SegmentWriteNotifier]. -func NewSegmentWriteNotifier(numberOfShards uint16, setter LastAppendedSegmentIDSetter) *SegmentWriteNotifier { +func NewSegmentWriteNotifier( + numberOfShards uint16, + setLastAppendedSegmentID func(segmentID uint32), +) *SegmentWriteNotifier { return &SegmentWriteNotifier{ - shards: make([]uint32, numberOfShards), - setter: setter, + shards: make([]uint32, numberOfShards), + setLastAppendedSegmentID: setLastAppendedSegmentID, } } @@ -32,7 +30,7 @@ func (swn *SegmentWriteNotifier) NotifySegmentIsWritten(shardID uint16) { swn.shards[shardID]++ minNumberOfSegments := slices.Min(swn.shards) if minNumberOfSegments > 0 { - swn.setter.SetLastAppendedSegmentID(minNumberOfSegments - 1) + swn.setLastAppendedSegmentID(minNumberOfSegments - 1) } } diff --git a/pp/go/storage/interface.go b/pp/go/storage/interface.go index fd01b37cd6..5fcdbb86d4 100644 --- a/pp/go/storage/interface.go +++ b/pp/go/storage/interface.go @@ -1,150 +1,150 @@ package storage -import ( - "context" - - "github.com/prometheus/prometheus/pp/go/cppbridge" - "github.com/prometheus/prometheus/pp/go/model" - "github.com/prometheus/prometheus/pp/go/relabeler/config" -) - -// -// Head -// - -// Head implementation of the head with added metrics. -type Head interface { - ID() string - Generation() uint64 - Append( - ctx context.Context, - incomingData *IncomingData, - state *cppbridge.State, - relabelerID string, - commitToWal bool, - ) ([][]*cppbridge.InnerSeries, cppbridge.RelabelerStats, error) - CommitToWal() error - // MergeOutOfOrderChunks merge chunks with out of order data chunks. - MergeOutOfOrderChunks() - NumberOfShards() uint16 - Stop() - Flush() error - Reconfigure(ctx context.Context, inputRelabelerConfigs []*config.InputRelabelerConfig, numberOfShards uint16) error - WriteMetrics(ctx context.Context) - Status(limit int) HeadStatus - Rotate() error - Close() error - Discard() error - String() string - CopySeriesFrom(other Head) - Enqueue(t *GenericTask) - EnqueueOnShard(t *GenericTask, shardID uint16) - CreateTask(taskName string, fn ShardFn, isLss bool) *GenericTask - Concurrency() int64 - RLockQuery(ctx context.Context) (runlock func(), err error) - Raw() Head -} - -// -// Shard -// - -// Shard interface for shards [Head]. -type Shard interface { - // DataStorage returns [DataStorage] shard. - DataStorage() DataStorage - // lock for DataStorage - DataStorageLock() - DataStorageRLock() - DataStorageRUnlock() - DataStorageUnlock() - // LSS returns [LSS] shard. - LSS() LSS - // lock for LSS - LSSLock() - LSSRLock() - LSSRUnlock() - LSSUnlock() - // ShardID returns ID shard. - ShardID() uint16 - // Wal returns [Wal] shard. - Wal() Wal -} - -// ShardFn function executing on a [Shard]. -type ShardFn func(shard Shard) error - -// -// DataStorage -// - -// DataStorage sample storage interface. -type DataStorage interface { - AllocatedMemory() uint64 - // AppendInnerSeriesSlice append slice of [cppbridge.InnerSeries](samples with label IDs) to the storage. - AppendInnerSeriesSlice(innerSeriesSlice []*cppbridge.InnerSeries) - InstantQuery(targetTimestamp, notFoundValueTimestampValue int64, seriesIDs []uint32) []cppbridge.Sample - MergeOutOfOrderChunks() - Query(query cppbridge.HeadDataStorageQuery) *cppbridge.HeadDataStorageSerializedChunks - Raw() *cppbridge.HeadDataStorage -} - -// -// LSS -// - -// LSS labelset storage interface. -type LSS interface { - AllocatedMemory() uint64 - GetLabelSets(labelSetIDs []uint32) *cppbridge.LabelSetStorageGetLabelSetsResult - GetSnapshot() *cppbridge.LabelSetSnapshot - Input() *cppbridge.LabelSetStorage - QueryLabelNames(matchers []model.LabelMatcher) *cppbridge.LSSQueryLabelNamesResult - QueryLabelValues(label_name string, matchers []model.LabelMatcher) *cppbridge.LSSQueryLabelValuesResult - QuerySelector(matchers []model.LabelMatcher) (selector uintptr, status uint32) - Raw() *cppbridge.LabelSetStorage - ResetSnapshot() - Target() *cppbridge.LabelSetStorage -} - -// -// Wal -// - -// Wal write-ahead log for [Shard]. -type Wal interface { - // DO NOT USE in public interfaces like ForEachShard - Commit() error - Flush() error - Write(innerSeriesSlice []*cppbridge.InnerSeries) (bool, error) -} - -// -// MetricData -// - -// MetricData is an universal interface for blob protobuf data or batch [model.TimeSeries]. -type MetricData interface { - // Destroy incoming data. - Destroy() -} - -// -// ProtobufData -// - -// ProtobufData is an universal interface for blob protobuf data. -type ProtobufData interface { - Bytes() []byte - Destroy() -} - -// -// TimeSeriesData -// - -// TimeSeriesBatch is an universal interface for batch [model.TimeSeries]. -type TimeSeriesBatch interface { - TimeSeries() []model.TimeSeries - Destroy() -} +// import ( +// "context" + +// "github.com/prometheus/prometheus/pp/go/cppbridge" +// "github.com/prometheus/prometheus/pp/go/model" +// "github.com/prometheus/prometheus/pp/go/relabeler/config" +// ) + +// // +// // Head +// // + +// // Head implementation of the head with added metrics. +// type Head interface { +// ID() string +// Generation() uint64 +// Append( +// ctx context.Context, +// incomingData *IncomingData, +// state *cppbridge.State, +// relabelerID string, +// commitToWal bool, +// ) ([][]*cppbridge.InnerSeries, cppbridge.RelabelerStats, error) +// CommitToWal() error +// // MergeOutOfOrderChunks merge chunks with out of order data chunks. +// MergeOutOfOrderChunks() +// NumberOfShards() uint16 +// Stop() +// Flush() error +// Reconfigure(ctx context.Context, inputRelabelerConfigs []*config.InputRelabelerConfig, numberOfShards uint16) error +// WriteMetrics(ctx context.Context) +// Status(limit int) HeadStatus +// Rotate() error +// Close() error +// Discard() error +// String() string +// CopySeriesFrom(other Head) +// Enqueue(t *GenericTask) +// EnqueueOnShard(t *GenericTask, shardID uint16) +// CreateTask(taskName string, fn ShardFn, isLss bool) *GenericTask +// Concurrency() int64 +// RLockQuery(ctx context.Context) (runlock func(), err error) +// Raw() Head +// } + +// // +// // Shard +// // + +// // Shard interface for shards [Head]. +// type Shard interface { +// // DataStorage returns [DataStorage] shard. +// DataStorage() DataStorage +// // lock for DataStorage +// DataStorageLock() +// DataStorageRLock() +// DataStorageRUnlock() +// DataStorageUnlock() +// // LSS returns [LSS] shard. +// LSS() LSS +// // lock for LSS +// LSSLock() +// LSSRLock() +// LSSRUnlock() +// LSSUnlock() +// // ShardID returns ID shard. +// ShardID() uint16 +// // Wal returns [Wal] shard. +// Wal() Wal +// } + +// // ShardFn function executing on a [Shard]. +// type ShardFn func(shard Shard) error + +// // +// // DataStorage +// // + +// // DataStorage sample storage interface. +// type DataStorage interface { +// AllocatedMemory() uint64 +// // AppendInnerSeriesSlice append slice of [cppbridge.InnerSeries](samples with label IDs) to the storage. +// AppendInnerSeriesSlice(innerSeriesSlice []*cppbridge.InnerSeries) +// InstantQuery(targetTimestamp, notFoundValueTimestampValue int64, seriesIDs []uint32) []cppbridge.Sample +// MergeOutOfOrderChunks() +// Query(query cppbridge.HeadDataStorageQuery) *cppbridge.HeadDataStorageSerializedChunks +// Raw() *cppbridge.HeadDataStorage +// } + +// // +// // LSS +// // + +// // LSS labelset storage interface. +// type LSS interface { +// AllocatedMemory() uint64 +// GetLabelSets(labelSetIDs []uint32) *cppbridge.LabelSetStorageGetLabelSetsResult +// GetSnapshot() *cppbridge.LabelSetSnapshot +// Input() *cppbridge.LabelSetStorage +// QueryLabelNames(matchers []model.LabelMatcher) *cppbridge.LSSQueryLabelNamesResult +// QueryLabelValues(label_name string, matchers []model.LabelMatcher) *cppbridge.LSSQueryLabelValuesResult +// QuerySelector(matchers []model.LabelMatcher) (selector uintptr, status uint32) +// Raw() *cppbridge.LabelSetStorage +// ResetSnapshot() +// Target() *cppbridge.LabelSetStorage +// } + +// // +// // Wal +// // + +// // Wal write-ahead log for [Shard]. +// type Wal interface { +// // DO NOT USE in public interfaces like ForEachShard +// Commit() error +// Flush() error +// Write(innerSeriesSlice []*cppbridge.InnerSeries) (bool, error) +// } + +// // +// // MetricData +// // + +// // MetricData is an universal interface for blob protobuf data or batch [model.TimeSeries]. +// type MetricData interface { +// // Destroy incoming data. +// Destroy() +// } + +// // +// // ProtobufData +// // + +// // ProtobufData is an universal interface for blob protobuf data. +// type ProtobufData interface { +// Bytes() []byte +// Destroy() +// } + +// // +// // TimeSeriesData +// // + +// // TimeSeriesBatch is an universal interface for batch [model.TimeSeries]. +// type TimeSeriesBatch interface { +// TimeSeries() []model.TimeSeries +// Destroy() +// } diff --git a/pp/go/storage/models.go b/pp/go/storage/models.go index abb1bb97f2..9df3d60b24 100644 --- a/pp/go/storage/models.go +++ b/pp/go/storage/models.go @@ -1,6 +1,39 @@ package storage -import "github.com/prometheus/prometheus/pp/go/cppbridge" +import ( + "github.com/prometheus/prometheus/pp/go/cppbridge" + "github.com/prometheus/prometheus/pp/go/model" +) + +// +// MetricData +// + +// MetricData is an universal interface for blob protobuf data or batch [model.TimeSeries]. +type MetricData interface { + // Destroy incoming data. + Destroy() +} + +// +// ProtobufData +// + +// ProtobufData is an universal interface for blob protobuf data. +type ProtobufData interface { + Bytes() []byte + Destroy() +} + +// +// TimeSeriesData +// + +// TimeSeriesBatch is an universal interface for batch [model.TimeSeries]. +type TimeSeriesBatch interface { + TimeSeries() []model.TimeSeries + Destroy() +} // // IncomingData diff --git a/pp/go/storage/task.go b/pp/go/storage/task.go index a3c9a2d282..086fff8df5 100644 --- a/pp/go/storage/task.go +++ b/pp/go/storage/task.go @@ -1,200 +1,200 @@ package storage -import ( - "errors" - "sync" - "sync/atomic" - "time" - - "github.com/prometheus/client_golang/prometheus" -) - -const ( - // ForLSSTask task for LSS. - ForLSSTask = true - // ForDataStorageTask task for DataStorage. - ForDataStorageTask = false - // ExclusiveTask exclusive task(write). - ExclusiveTask = true - // NonExclusiveTask non-exclusive task(read). - NonExclusiveTask = false -) - -const ( - // LSSInputRelabeling name of task. - LSSInputRelabeling = "lss_input_relabeling" - // LSSAppendRelabelerSeries name of task. - LSSAppendRelabelerSeries = "lss_append_relabeler_series" - - // LSSWalCommit name of task. - LSSWalCommit = "lss_wal_commit" - // LSSWalFlush name of task. - LSSWalFlush = "lss_wal_flush" - // LSSWalWrite name of task. - LSSWalWrite = "lss_wal_write" - - // LSSCopyAddedSeries name of task. - LSSCopyAddedSeries = "lss_copy_added_series" - - // LSSOutputRelabeling name of task. - LSSOutputRelabeling = "lss_output_relabeling" - - // LSSAllocatedMemory name of task. - LSSAllocatedMemory = "lss_allocated_memory" - - // LSSHeadStatus name of task. - LSSHeadStatus = "lss_head_status" - - // LSSQueryChunkQuerySelector name of task. - LSSQueryChunkQuerySelector = "lss_query_chunk_query_selector" - // LSSLabelValuesChunkQuerier name of task. - LSSLabelValuesChunkQuerier = "lss_label_values_chunk_querier" - // LSSLabelNamesChunkQuerier name of task. - LSSLabelNamesChunkQuerier = "lss_label_names_chunk_querier" - - // LSSQueryInstantQuerySelector name of task. - LSSQueryInstantQuerySelector = "lss_query_instant_query_selector" - // LSSQueryRangeQuerySelector name of task. - LSSQueryRangeQuerySelector = "lss_query_range_query_selector" - // LSSLabelValuesQuerier name of task. - LSSLabelValuesQuerier = "lss_label_values_querier" - // LSSLabelNamesQuerier name of task. - LSSLabelNamesQuerier = "lss_label_names_querier" - - // DSAppendInnerSeries name of task. - DSAppendInnerSeries = "data_storage_append_inner_series" - // DSMergeOutOfOrderChunks name of task. - DSMergeOutOfOrderChunks = "data_storage_merge_out_of_order_chunks" - - // DSAllocatedMemory name of task. - DSAllocatedMemory = "data_storage_allocated_memory" - - // DSHeadStatus name of task. - DSHeadStatus = "data_storage_head_status" - - // DSQueryChunkQuerier name of task. - DSQueryChunkQuerier = "data_storage_query_chunk_querier" - - // DSQueryInstantQuerier name of task. - DSQueryInstantQuerier = "data_storage_query_instant_querier" - // DSQueryRangeQuerier name of task. - DSQueryRangeQuerier = "data_storage_query_range_querier" - - // Read Only - - // BlockWrite name of task. - BlockWrite = "block_write" -) - -// -// GenericTask -// - -// GenericTask generic task, will be executed on each shard. -type GenericTask struct { - errs []error - shardFn ShardFn - wg sync.WaitGroup - createdTS int64 - executeTS int64 - created prometheus.Counter - done prometheus.Counter - live prometheus.Counter - execute prometheus.Counter - forLSS bool -} - -// NewGenericTask init new [GenericTask]. -func NewGenericTask( - shardFn ShardFn, - created, done, live, execute prometheus.Counter, - forLSS bool, -) *GenericTask { - t := &GenericTask{ - shardFn: shardFn, - wg: sync.WaitGroup{}, - createdTS: time.Now().UnixMicro(), - created: created, - done: done, - live: live, - execute: execute, - forLSS: forLSS, - } - t.created.Inc() - - return t -} - -// NewReadOnlyGenericTask init new GenericTask for read only head. -func NewReadOnlyGenericTask(shardFn ShardFn) *GenericTask { - t := &GenericTask{ - shardFn: shardFn, - wg: sync.WaitGroup{}, - } - - return t -} - -// SetShardsNumber set shards number -func (t *GenericTask) SetShardsNumber(number uint16) { - t.errs = make([]error, number) - t.wg.Add(int(number)) -} - -// ExecuteOnShard execute task on shard. -func (t *GenericTask) ExecuteOnShard(shard Shard) { - atomic.CompareAndSwapInt64(&t.executeTS, 0, time.Now().UnixMicro()) - t.errs[shard.ShardID()] = t.shardFn(shard) - t.wg.Done() -} - -// ForLSS indicates that the task is for operation on lss. -func (t *GenericTask) ForLSS() bool { - return t.forLSS -} - -// Wait for the task to complete on all shards. -func (t *GenericTask) Wait() error { - t.wg.Wait() - if t.done == nil { - return errors.Join(t.errs...) - } - - now := time.Now().UnixMicro() - t.done.Inc() - t.execute.Add(float64(now - t.executeTS)) - t.live.Add(float64(now - t.createdTS)) - - return errors.Join(t.errs...) -} - -// -// TaskWaiter -// - -// TaskWaiter aggregates the wait for tasks to be completed. -type TaskWaiter struct { - tasks []*GenericTask -} - -// NewTaskWaiter init new TaskWaiter for n task. -func NewTaskWaiter(n int) *TaskWaiter { - return &TaskWaiter{ - tasks: make([]*GenericTask, 0, n), - } -} - -// Add task to waiter. -func (tw *TaskWaiter) Add(t *GenericTask) { - tw.tasks = append(tw.tasks, t) -} - -// Wait for tasks to be completed. -func (tw *TaskWaiter) Wait() error { - errs := make([]error, len(tw.tasks)) - for _, t := range tw.tasks { - errs = append(errs, t.Wait()) - } - - return errors.Join(errs...) -} +// import ( +// "errors" +// "sync" +// "sync/atomic" +// "time" + +// "github.com/prometheus/client_golang/prometheus" +// ) + +// const ( +// // ForLSSTask task for LSS. +// ForLSSTask = true +// // ForDataStorageTask task for DataStorage. +// ForDataStorageTask = false +// // ExclusiveTask exclusive task(write). +// ExclusiveTask = true +// // NonExclusiveTask non-exclusive task(read). +// NonExclusiveTask = false +// ) + +// const ( +// // LSSInputRelabeling name of task. +// LSSInputRelabeling = "lss_input_relabeling" +// // LSSAppendRelabelerSeries name of task. +// LSSAppendRelabelerSeries = "lss_append_relabeler_series" + +// // LSSWalCommit name of task. +// LSSWalCommit = "lss_wal_commit" +// // LSSWalFlush name of task. +// LSSWalFlush = "lss_wal_flush" +// // LSSWalWrite name of task. +// LSSWalWrite = "lss_wal_write" + +// // LSSCopyAddedSeries name of task. +// LSSCopyAddedSeries = "lss_copy_added_series" + +// // LSSOutputRelabeling name of task. +// LSSOutputRelabeling = "lss_output_relabeling" + +// // LSSAllocatedMemory name of task. +// LSSAllocatedMemory = "lss_allocated_memory" + +// // LSSHeadStatus name of task. +// LSSHeadStatus = "lss_head_status" + +// // LSSQueryChunkQuerySelector name of task. +// LSSQueryChunkQuerySelector = "lss_query_chunk_query_selector" +// // LSSLabelValuesChunkQuerier name of task. +// LSSLabelValuesChunkQuerier = "lss_label_values_chunk_querier" +// // LSSLabelNamesChunkQuerier name of task. +// LSSLabelNamesChunkQuerier = "lss_label_names_chunk_querier" + +// // LSSQueryInstantQuerySelector name of task. +// LSSQueryInstantQuerySelector = "lss_query_instant_query_selector" +// // LSSQueryRangeQuerySelector name of task. +// LSSQueryRangeQuerySelector = "lss_query_range_query_selector" +// // LSSLabelValuesQuerier name of task. +// LSSLabelValuesQuerier = "lss_label_values_querier" +// // LSSLabelNamesQuerier name of task. +// LSSLabelNamesQuerier = "lss_label_names_querier" + +// // DSAppendInnerSeries name of task. +// DSAppendInnerSeries = "data_storage_append_inner_series" +// // DSMergeOutOfOrderChunks name of task. +// DSMergeOutOfOrderChunks = "data_storage_merge_out_of_order_chunks" + +// // DSAllocatedMemory name of task. +// DSAllocatedMemory = "data_storage_allocated_memory" + +// // DSHeadStatus name of task. +// DSHeadStatus = "data_storage_head_status" + +// // DSQueryChunkQuerier name of task. +// DSQueryChunkQuerier = "data_storage_query_chunk_querier" + +// // DSQueryInstantQuerier name of task. +// DSQueryInstantQuerier = "data_storage_query_instant_querier" +// // DSQueryRangeQuerier name of task. +// DSQueryRangeQuerier = "data_storage_query_range_querier" + +// // Read Only + +// // BlockWrite name of task. +// BlockWrite = "block_write" +// ) + +// // +// // GenericTask +// // + +// // GenericTask generic task, will be executed on each shard. +// type GenericTask struct { +// errs []error +// shardFn ShardFn +// wg sync.WaitGroup +// createdTS int64 +// executeTS int64 +// created prometheus.Counter +// done prometheus.Counter +// live prometheus.Counter +// execute prometheus.Counter +// forLSS bool +// } + +// // NewGenericTask init new [GenericTask]. +// func NewGenericTask( +// shardFn ShardFn, +// created, done, live, execute prometheus.Counter, +// forLSS bool, +// ) *GenericTask { +// t := &GenericTask{ +// shardFn: shardFn, +// wg: sync.WaitGroup{}, +// createdTS: time.Now().UnixMicro(), +// created: created, +// done: done, +// live: live, +// execute: execute, +// forLSS: forLSS, +// } +// t.created.Inc() + +// return t +// } + +// // NewReadOnlyGenericTask init new GenericTask for read only head. +// func NewReadOnlyGenericTask(shardFn ShardFn) *GenericTask { +// t := &GenericTask{ +// shardFn: shardFn, +// wg: sync.WaitGroup{}, +// } + +// return t +// } + +// // SetShardsNumber set shards number +// func (t *GenericTask) SetShardsNumber(number uint16) { +// t.errs = make([]error, number) +// t.wg.Add(int(number)) +// } + +// // ExecuteOnShard execute task on shard. +// func (t *GenericTask) ExecuteOnShard(shard Shard) { +// atomic.CompareAndSwapInt64(&t.executeTS, 0, time.Now().UnixMicro()) +// t.errs[shard.ShardID()] = t.shardFn(shard) +// t.wg.Done() +// } + +// // ForLSS indicates that the task is for operation on lss. +// func (t *GenericTask) ForLSS() bool { +// return t.forLSS +// } + +// // Wait for the task to complete on all shards. +// func (t *GenericTask) Wait() error { +// t.wg.Wait() +// if t.done == nil { +// return errors.Join(t.errs...) +// } + +// now := time.Now().UnixMicro() +// t.done.Inc() +// t.execute.Add(float64(now - t.executeTS)) +// t.live.Add(float64(now - t.createdTS)) + +// return errors.Join(t.errs...) +// } + +// // +// // TaskWaiter +// // + +// // TaskWaiter aggregates the wait for tasks to be completed. +// type TaskWaiter struct { +// tasks []*GenericTask +// } + +// // NewTaskWaiter init new TaskWaiter for n task. +// func NewTaskWaiter(n int) *TaskWaiter { +// return &TaskWaiter{ +// tasks: make([]*GenericTask, 0, n), +// } +// } + +// // Add task to waiter. +// func (tw *TaskWaiter) Add(t *GenericTask) { +// tw.tasks = append(tw.tasks, t) +// } + +// // Wait for tasks to be completed. +// func (tw *TaskWaiter) Wait() error { +// errs := make([]error, len(tw.tasks)) +// for _, t := range tw.tasks { +// errs = append(errs, t.Wait()) +// } + +// return errors.Join(errs...) +// } From 7f37a19975e537b1b895ad61aa6ad3ddbdf4b154 Mon Sep 17 00:00:00 2001 From: Alexandr Yudin Date: Thu, 28 Aug 2025 08:13:59 +0000 Subject: [PATCH 21/96] for save --- pp/go/storage/appender/appender.go | 2 +- pp/go/storage/factory.go | 161 +++++++++++++++-- pp/go/storage/head/builder/builder.go | 2 + pp/go/storage/head/container/weighted.go | 3 + pp/go/storage/head/head/head.go | 12 ++ pp/go/storage/head/loader/loader.go | 53 +++++- pp/go/storage/head/loader/shard_loader.go | 103 ----------- pp/go/storage/head/manager/manager.go | 19 +- pp/go/storage/head/shard/data_storage.go | 6 + pp/go/storage/head/shard/shard.go | 105 +---------- .../storage/head/shard/wal/reader/segment.go | 33 ++-- pp/go/storage/head/shard/wal/wal_reader.go | 55 ++++++ pp/go/storage/querier/multi.go | 169 ++++++++++++++++++ pp/go/storage/querier/querier.go | 2 +- 14 files changed, 483 insertions(+), 242 deletions(-) delete mode 100644 pp/go/storage/head/loader/shard_loader.go create mode 100644 pp/go/storage/head/shard/wal/wal_reader.go create mode 100644 pp/go/storage/querier/multi.go diff --git a/pp/go/storage/appender/appender.go b/pp/go/storage/appender/appender.go index 1e72c9f7ce..12b67cc20a 100644 --- a/pp/go/storage/appender/appender.go +++ b/pp/go/storage/appender/appender.go @@ -227,7 +227,7 @@ func (a *Appender[TGenericTask, TDataStorage, TLSS, TWal, TShard, THead]) Append t := a.head.CreateTask( WalCommit, func(shard TShard) error { - // It is necessary to lock the LSS for reading for the commit. + // wal contains LSS and it is necessary to lock the LSS for reading for the commit. return shard.LSS().WithRLock(func(_, _ *cppbridge.LabelSetStorage) error { return shard.Wal().Commit() }) diff --git a/pp/go/storage/factory.go b/pp/go/storage/factory.go index 4f2aacc5f0..6cd480e9fa 100644 --- a/pp/go/storage/factory.go +++ b/pp/go/storage/factory.go @@ -4,6 +4,7 @@ import ( "fmt" "os" "path/filepath" + "sync" "github.com/prometheus/client_golang/prometheus" @@ -11,7 +12,10 @@ import ( "github.com/prometheus/prometheus/pp/go/storage/head/head" "github.com/prometheus/prometheus/pp/go/storage/head/shard" "github.com/prometheus/prometheus/pp/go/storage/head/shard/wal" + "github.com/prometheus/prometheus/pp/go/storage/head/shard/wal/reader" "github.com/prometheus/prometheus/pp/go/storage/head/shard/wal/writer" + "github.com/prometheus/prometheus/pp/go/storage/logger" + "github.com/prometheus/prometheus/pp/go/util/optional" ) // WalOnDisk wal on disk. @@ -30,9 +34,9 @@ type HeadOnDisk = head.Head[*ShardOnDisk, *shard.PerGoroutineShard[*WalOnDisk]] // ShardOnDiskConstructor create [shard.Shard] with [wal.Wal] which is written to disk. func ShardOnDiskConstructor( dir string, - setLastAppendedSegmentID func(segmentID uint32), + swn *writer.SegmentWriteNotifier, maxSegmentSize uint32, - numberOfShards, shardID uint16, + shardID uint16, ) (*ShardOnDisk, error) { shardFile, err := os.Create(filepath.Join(filepath.Clean(dir), fmt.Sprintf("shard_%d.wal", shardID))) if err != nil { @@ -46,18 +50,20 @@ func ShardOnDiskConstructor( _ = shardFile.Close() }() - swn := writer.NewSegmentWriteNotifier(numberOfShards, setLastAppendedSegmentID) + lss := shard.NewLSS() + // logShards is 0 for single encoder + shardWalEncoder := cppbridge.NewHeadWalEncoder(shardID, 0, lss.Target()) + + _, err = writer.WriteHeader(shardFile, wal.FileFormatVersion, shardWalEncoder.Version()) + if err != nil { + return nil, fmt.Errorf("failed to write header: %w", err) + } sw, err := writer.NewBuffered(shardID, shardFile, writer.WriteSegment[*cppbridge.EncodedSegment], swn) if err != nil { return nil, fmt.Errorf("failed to create buffered writer shard id %d: %w", shardID, err) } - lss := shard.NewLSS() - - // logShards is 0 for single encoder - shardWalEncoder := cppbridge.NewHeadWalEncoder(shardID, 0, lss.Target()) - return shard.NewShard( lss, shard.NewDataStorage(), @@ -77,12 +83,12 @@ func HeadConstructor( registerer prometheus.Registerer, ) (*HeadOnDisk, error) { shards := make([]*ShardOnDisk, numberOfShards) + swn := writer.NewSegmentWriteNotifier(numberOfShards, setLastAppendedSegmentID) for shardID := range numberOfShards { s, err := ShardOnDiskConstructor( headDir, - setLastAppendedSegmentID, + swn, maxSegmentSize, - numberOfShards, shardID, ) if err != nil { @@ -102,3 +108,138 @@ func HeadConstructor( registerer, ), nil } + +// UploadHead upload [HeadOnDisk] from [WalOnDisk]. +func UploadHead( + id, headDir string, + releaseHeadFn func(), + setLastAppendedSegmentID func(segmentID uint32), + generation uint64, + maxSegmentSize uint32, + numberOfShards uint16, + registerer prometheus.Registerer, +) (_ *HeadOnDisk, _ uint32, corrupted bool) { + shardLoadResults := make([]ShardLoadResult, numberOfShards) + wg := &sync.WaitGroup{} + swn := writer.NewSegmentWriteNotifier(numberOfShards, setLastAppendedSegmentID) + for shardID := range numberOfShards { + wg.Add(1) + go func(shardID uint16, shardWalFilePath string) { + defer wg.Done() + shardLoadResults[shardID] = UploadShard(shardWalFilePath, swn, maxSegmentSize, shardID) + }(shardID, filepath.Join(headDir, fmt.Sprintf("shard_%d.wal", shardID))) + } + wg.Wait() + + shards := make([]*ShardOnDisk, numberOfShards) + numberOfSegmentsRead := optional.Optional[uint32]{} + + for shardID, res := range shardLoadResults { + shards[shardID] = res.Shard() + if res.Corrupted() { + corrupted = true + } + + if numberOfSegmentsRead.IsNil() { + numberOfSegmentsRead.Set(res.NumberOfSegments()) + } else if numberOfSegmentsRead.Value() != res.NumberOfSegments() { + corrupted = true + // calculating maximum number of segments (critical for remote write). + if numberOfSegmentsRead.Value() < res.NumberOfSegments() { + numberOfSegmentsRead.Set(res.NumberOfSegments()) + } + } + } + + // h.MergeOutOfOrderChunks() + return head.NewHead( + id, + shards, + shard.NewPerGoroutineShard[*WalOnDisk], + releaseHeadFn, + generation, + numberOfShards, + registerer, + ), + numberOfSegmentsRead.Value(), + corrupted +} + +// UploadShard upload [ShardOnDisk] from [WalOnDisk]. +func UploadShard( + shardFilePath string, + swn *writer.SegmentWriteNotifier, + maxSegmentSize uint32, + shardID uint16, +) ShardLoadResult { + res := ShardLoadResult{corrupted: true} + + //revive:disable-next-line:add-constant file permissions simple readable as octa-number + shardFile, err := os.OpenFile(shardFilePath, os.O_RDWR, 0o600) // #nosec G304 // it's meant to be that way + if err != nil { + logger.Debugf("failed to open file shard id %d: %w", shardID, err) + return res + } + defer func() { + if res.corrupted { + _ = shardFile.Close() + } + }() + + _, encoderVersion, _, err := reader.ReadHeader(shardFile) + if err != nil { + logger.Debugf("failed to read wal header: %w", err) + return res + } + + lss := shard.NewLSS() + decoder := cppbridge.NewHeadWalDecoder(lss.Target(), encoderVersion) + dataStorage := shard.NewDataStorage() + + if err = wal.NewSegmentWalReader[reader.Segment](shardFile).ForEachSegment(func(s *reader.Segment) error { + if decodeErr := dataStorage.DecodeSegment(decoder, s.Bytes()); decodeErr != nil { + return fmt.Errorf("failed to decode segment: %w", decodeErr) + } + + res.numberOfSegments++ + + return nil + }); err != nil { + logger.Debugf(err.Error()) + return res + } + + sw, err := writer.NewBuffered(shardID, shardFile, writer.WriteSegment[*cppbridge.EncodedSegment], swn) + if err != nil { + logger.Debugf("failed to create buffered writer shard id %d: %w", shardID, err) + return res + } + + swn.Set(shardID, res.numberOfSegments) + res.corrupted = false + res.shard = shard.NewShard(lss, dataStorage, wal.NewWal(decoder.CreateEncoder(), sw, maxSegmentSize), shardID) + + return res +} + +// ShardLoadResult the result of uploading the [ShardOnDisk] from the [WalOnDisk]. +type ShardLoadResult struct { + shard *ShardOnDisk + numberOfSegments uint32 + corrupted bool +} + +// Corrupted returns true if [ShardOnDisk] is corrupted. +func (sr *ShardLoadResult) Corrupted() bool { + return sr.corrupted +} + +// NumberOfSegments returns number of segments in [ShardOnDisk]s. +func (sr *ShardLoadResult) NumberOfSegments() uint32 { + return sr.numberOfSegments +} + +// Shard returns [*ShardOnDisk] or nil. +func (sr *ShardLoadResult) Shard() *ShardOnDisk { + return sr.shard +} diff --git a/pp/go/storage/head/builder/builder.go b/pp/go/storage/head/builder/builder.go index 62786ec4a1..535745cf44 100644 --- a/pp/go/storage/head/builder/builder.go +++ b/pp/go/storage/head/builder/builder.go @@ -16,6 +16,7 @@ import ( // HeadsCatalog of current head records. type HeadsCatalog interface { + // Create creates new [Record] and write to [Log]. Create(numberOfShards uint16) (*catalog.Record, error) // Delete record by ID. @@ -31,6 +32,7 @@ type HeadsCatalog interface { // Head the minimum required Head implementation for a container. type Head[T any] interface { + // for use as a pointer *T } diff --git a/pp/go/storage/head/container/weighted.go b/pp/go/storage/head/container/weighted.go index 7388f4ab28..03c0ff2404 100644 --- a/pp/go/storage/head/container/weighted.go +++ b/pp/go/storage/head/container/weighted.go @@ -11,7 +11,10 @@ import ( // Head the minimum required Head implementation for a container. type Head[T any] interface { + // Concurrency return current head workers concurrency. Concurrency() int64 + + // for use as a pointer *T } diff --git a/pp/go/storage/head/head/head.go b/pp/go/storage/head/head/head.go index 0927b99b14..9f04dfc7a4 100644 --- a/pp/go/storage/head/head/head.go +++ b/pp/go/storage/head/head/head.go @@ -110,6 +110,7 @@ func NewHead[TShard Shard, TGoroutineShard Shard]( shards: shards, taskChs: taskChs, numberOfShards: uint16(len(shards)), // #nosec G115 // no overflow + // TODO metrics memoryInUse: factory.NewGaugeVec( prometheus.GaugeOpts{ Name: "prompp_head_cgo_memory_bytes", @@ -255,6 +256,17 @@ func (h *Head[TShard, TGorutineShard]) NumberOfShards() uint16 { return h.numberOfShards } +// RangeShards returns an iterator over the [Head] [Shard]s, through which the shard can be directly accessed. +func (h *Head[TShard, TGorutineShard]) RangeShards() func(func(TShard) bool) { + return func(yield func(s TShard) bool) { + for _, shard := range h.shards { + if !yield(shard) { + return + } + } + } +} + // SetReadOnly sets the read-only flag for the [Head]. func (h *Head[TShard, TGorutineShard]) SetReadOnly() { atomic.StoreUint32(&h.readOnly, 1) diff --git a/pp/go/storage/head/loader/loader.go b/pp/go/storage/head/loader/loader.go index 43536fb0ca..5cdb599e43 100644 --- a/pp/go/storage/head/loader/loader.go +++ b/pp/go/storage/head/loader/loader.go @@ -1,5 +1,54 @@ package loader -type Loader struct { - // +import ( + "github.com/prometheus/client_golang/prometheus" + "github.com/prometheus/prometheus/pp/go/storage/catalog" +) + +// +// HeadsCatalog +// + +// HeadsCatalog of current head records. +type HeadsCatalog interface { + // List returns slice of records with filter and sort. + List(filterFn func(record *catalog.Record) bool, sortLess func(lhs, rhs *catalog.Record) bool) []*catalog.Record +} + +// +// Head +// + +// Head the minimum required Head implementation for a container. +type Head[T any] interface { + // for use as a pointer + *T +} + +type Loader[T any, THead Head[T]] struct { + catalog HeadsCatalog + dir string + headCtor func( + id, headDir string, + releaseHeadFn func(), + setLastAppendedSegmentID func(segmentID uint32), + generation uint64, + maxSegmentSize uint32, + numberOfShards uint16, + registerer prometheus.Registerer, + ) (THead, error) + uploadHead func( + id, headDir string, + releaseHeadFn func(), + setLastAppendedSegmentID func(segmentID uint32), + generation uint64, + maxSegmentSize uint32, + numberOfShards uint16, + registerer prometheus.Registerer, + ) (THead, uint32, bool) + maxSegmentSize uint32 + registerer prometheus.Registerer + + // TODO ? + // generation uint64 } diff --git a/pp/go/storage/head/loader/shard_loader.go b/pp/go/storage/head/loader/shard_loader.go deleted file mode 100644 index 56183eb081..0000000000 --- a/pp/go/storage/head/loader/shard_loader.go +++ /dev/null @@ -1,103 +0,0 @@ -package loader - -import ( - "bufio" - "errors" - "fmt" - "io" - "os" - - "github.com/prometheus/prometheus/pp/go/cppbridge" -) - -type SegmentWriteNotifier interface { - Set(shardID uint16, numberOfSegments uint32) -} - -type ShardLoader struct { - notifier SegmentWriteNotifier - shardFilePath string - maxSegmentSize uint32 - shardID uint16 -} - -func (l *ShardLoader) Load() (result ShardLoadResult) { - targetLss := cppbridge.NewQueryableLssStorage() - dataStorage := NewDataStorage() - - result.Lss = &LSS{ - input: cppbridge.NewLssStorage(), - target: targetLss, - } - result.DataStorage = dataStorage - result.Wal = newCorruptedShardWal() - result.Corrupted = true - - shardWalFile, err := os.OpenFile(l.shardFilePath, os.O_RDWR, 0o600) - if err != nil { - result.Err = err - return - } - - defer func() { - if result.Corrupted { - _ = shardWalFile.Close() - } - }() - - reader := bufio.NewReaderSize(shardWalFile, 1024*1024*4) - _, encoderVersion, offset, err := ReadHeader(reader) - if err != nil { - result.Err = fmt.Errorf("failed to read wal header: %w", err) - return - } - - decoder := cppbridge.NewHeadWalDecoder(targetLss, encoderVersion) - lastReadSegmentID := -1 - - var bytesRead int - for { - var segment DecodedSegment - segment, bytesRead, err = ReadSegment(reader) - if err != nil { - if errors.Is(err, io.EOF) { - break - } - result.Err = fmt.Errorf("failed to read segment: %w", err) - break - } - - err = decoder.DecodeToDataStorage(segment.data, dataStorage.encoder) - if err != nil { - result.Err = fmt.Errorf("failed to decode segment: %w", err) - break - } - - offset += bytesRead - lastReadSegmentID++ - } - - numberOfSegments := lastReadSegmentID + 1 - result.NumberOfSegments = uint32(numberOfSegments) // #nosec G115 // no overflow - sw, err := newSegmentWriter(l.shardID, shardWalFile, l.notifier) - if err != nil { - result.Err = err - return - } - - l.notifier.Set(l.shardID, uint32(numberOfSegments)) // #nosec G115 // no overflow - result.Wal = newShardWal(decoder.CreateEncoder(), l.maxSegmentSize, sw) - if result.Err == nil { - result.Corrupted = false - } - return result -} - -type ShardLoadResult struct { - Lss *LSS - DataStorage *DataStorage - Wal *ShardWal - NumberOfSegments uint32 - Corrupted bool - Err error -} diff --git a/pp/go/storage/head/manager/manager.go b/pp/go/storage/head/manager/manager.go index 43b38b21ae..b718fb2c00 100644 --- a/pp/go/storage/head/manager/manager.go +++ b/pp/go/storage/head/manager/manager.go @@ -21,13 +21,14 @@ type Head interface { } type ActiveHeadContainer[THead Head] interface { - Get(ctx context.Context) (THead, error) + Get() THead Replace(ctx context.Context, newHead THead) error With(ctx context.Context, fn func(h THead) error) error } type Keeper[THead Head] interface { Add(head THead) + RangeQueriableHeads(mint, maxt int64) func(func(THead) bool) } // type ActiveHeadContainer[T any] interface { @@ -147,10 +148,7 @@ func (m *Manager[THead]) rotate(ctx context.Context) error { return fmt.Errorf("failed to build a new head: %w", err) } - oldHead, err := m.activeHead.Get(ctx) - if err != nil { - return fmt.Errorf("getting active head failed: %w", err) - } + oldHead := m.activeHead.Get() // TODO // newHead.CopySeriesFrom(oldHead) @@ -169,5 +167,14 @@ func (m *Manager[THead]) WithAppendableHead(ctx context.Context, fn func(h THead // RangeQueriableHeads // TODO implementation. -func (m *Manager[THead]) RangeQueriableHeads() { +func (m *Manager[THead]) RangeQueriableHeads(mint, maxt int64) func(func(THead) bool) { + // ahead := m.activeHead.Get() + // for h := range m.keeper.RangeQueriableHeads(mint, maxt) { + // TODO + // if h == ahead { + // continue + // } + // } + + return nil } diff --git a/pp/go/storage/head/shard/data_storage.go b/pp/go/storage/head/shard/data_storage.go index a3dccae1a3..27ea8d83d9 100644 --- a/pp/go/storage/head/shard/data_storage.go +++ b/pp/go/storage/head/shard/data_storage.go @@ -39,6 +39,12 @@ func (ds *DataStorage) AppendInnerSeriesSlice(innerSeriesSlice []*cppbridge.Inne ds.locker.Unlock() } +// DecodeSegment decode segment data from decoder [cppbridge.HeadWalDecoder] +// and add to encoder [cppbridge.HeadEncoder]. +func (ds *DataStorage) DecodeSegment(decoder *cppbridge.HeadWalDecoder, data []byte) error { + return decoder.DecodeToDataStorage(data, ds.encoder) +} + // InstantQuery make instant query to data storage and returns samples. func (ds *DataStorage) InstantQuery( targetTimestamp, notFoundValueTimestampValue int64, seriesIDs []uint32, diff --git a/pp/go/storage/head/shard/shard.go b/pp/go/storage/head/shard/shard.go index 04c894b929..3afad97cd6 100644 --- a/pp/go/storage/head/shard/shard.go +++ b/pp/go/storage/head/shard/shard.go @@ -69,6 +69,7 @@ func (s *Shard[TWal]) Wal() TWal { // WalFlush flush all contetnt into wal. func (s *Shard[TWal]) WalFlush() error { + // TODO return s.wal.Flush() } @@ -94,107 +95,3 @@ func NewPerGoroutineShard[TWal Wal](s *Shard[TWal], numberOfShards uint16) *PerG func (s *PerGoroutineShard[TWal]) Relabeler() *cppbridge.PerGoroutineRelabeler { return s.relabeler } - -// // InputRelabeling relabeling incoming hashdex(first stage). -// func (s *Shard[TWal]) InputRelabeling( -// ctx context.Context, -// relabeler *cppbridge.InputPerShardRelabeler, -// cache *cppbridge.Cache, -// options cppbridge.RelabelerOptions, -// shardedData cppbridge.ShardedData, -// shardsInnerSeries []*cppbridge.InnerSeries, -// shardsRelabeledSeries []*cppbridge.RelabeledSeries, -// ) (cppbridge.RelabelerStats, bool, error) { -// s.lssLocker.Lock() -// defer s.lssLocker.Unlock() - -// return relabeler.InputRelabeling( -// ctx, -// s.lss.Input(), -// s.lss.Target(), -// cache, -// options, -// shardedData, -// shardsInnerSeries, -// shardsRelabeledSeries, -// ) -// } - -// // InputRelabelingFromCache relabeling incoming hashdex(first stage) from cache. -// func (s *Shard[TWal]) InputRelabelingFromCache( -// ctx context.Context, -// relabeler *cppbridge.InputPerShardRelabeler, -// cache *cppbridge.Cache, -// options cppbridge.RelabelerOptions, -// shardedData cppbridge.ShardedData, -// shardsInnerSeries []*cppbridge.InnerSeries, -// ) (cppbridge.RelabelerStats, bool, error) { -// s.lssLocker.RLock() -// defer s.lssLocker.RUnlock() - -// return relabeler.InputRelabelingFromCache( -// ctx, -// s.lss.Input(), -// s.lss.Target(), -// cache, -// options, -// shardedData, -// shardsInnerSeries, -// ) -// } - -// // InputRelabelingWithStalenans relabeling incoming hashdex(first stage) with state stalenans. -// func (s *Shard[TWal]) InputRelabelingWithStalenans( -// ctx context.Context, -// relabeler *cppbridge.InputPerShardRelabeler, -// cache *cppbridge.Cache, -// options cppbridge.RelabelerOptions, -// staleNansState *cppbridge.StaleNansState, -// defTimestamp int64, -// shardedData cppbridge.ShardedData, -// shardsInnerSeries []*cppbridge.InnerSeries, -// shardsRelabeledSeries []*cppbridge.RelabeledSeries, -// ) (cppbridge.RelabelerStats, bool, error) { -// s.lssLocker.Lock() -// defer s.lssLocker.Unlock() - -// return relabeler.InputRelabelingWithStalenans( -// ctx, -// s.lss.Input(), -// s.lss.Target(), -// cache, -// options, -// staleNansState, -// defTimestamp, -// shardedData, -// shardsInnerSeries, -// shardsRelabeledSeries, -// ) -// } - -// // InputRelabelingWithStalenansFromCache relabeling incoming hashdex(first stage) from cache with state stalenans. -// func (s *Shard[TWal]) InputRelabelingWithStalenansFromCache( -// ctx context.Context, -// relabeler *cppbridge.InputPerShardRelabeler, -// cache *cppbridge.Cache, -// options cppbridge.RelabelerOptions, -// staleNansState *cppbridge.StaleNansState, -// defTimestamp int64, -// shardedData cppbridge.ShardedData, -// shardsInnerSeries []*cppbridge.InnerSeries, -// ) (cppbridge.RelabelerStats, bool, error) { -// s.lssLocker.RLock() -// defer s.lssLocker.RUnlock() - -// return relabeler.InputRelabelingWithStalenansFromCache( -// ctx, -// s.lss.Input(), -// s.lss.Target(), -// cache, -// options, -// staleNansState, -// defTimestamp, -// shardedData, -// shardsInnerSeries, -// ) -// } diff --git a/pp/go/storage/head/shard/wal/reader/segment.go b/pp/go/storage/head/shard/wal/reader/segment.go index 94734ddc7d..c46baddb93 100644 --- a/pp/go/storage/head/shard/wal/reader/segment.go +++ b/pp/go/storage/head/shard/wal/reader/segment.go @@ -7,8 +7,6 @@ import ( "io" ) -// TODO rebuild ReadFrom - // Segment encoded segment from wal. type Segment struct { data []byte @@ -20,18 +18,24 @@ func (s *Segment) Bytes() []byte { return s.data } -// Reset segment data. +// ReadFrom reads [Segment] data from r [io.Reader]. The return value n is the number of bytes read. +// Any error encountered during the read is also returned. +func (s *Segment) ReadFrom(r io.Reader) (int64, error) { + return ReadSegment(r, s) +} + +// Reset [Segment] data. func (s *Segment) Reset() { s.data = s.data[:0] s.sampleCount = 0 } -// Samples returns count of samples in segment. +// Samples returns count of samples in [Segment]. func (s *Segment) Samples() uint32 { return s.sampleCount } -// resize segment data. +// resize [Segment] data. func (s *Segment) resize(size int) { if cap(s.data) < size { s.data = make([]byte, size) @@ -41,38 +45,37 @@ func (s *Segment) resize(size int) { } // ReadSegment read and decode [Segment] from [io.Reader] and returns. -func ReadSegment(reader io.Reader, segment *Segment) (n int, err error) { +func ReadSegment(reader io.Reader, segment *Segment) (int64, error) { br := newByteReader(reader) - var size uint64 - size, err = binary.ReadUvarint(br) + size, err := binary.ReadUvarint(br) if err != nil { - return br.n, fmt.Errorf("failed to read segment size: %w", err) + return int64(br.n), fmt.Errorf("failed to read segment size: %w", err) } crc32HashU64, err := binary.ReadUvarint(br) if err != nil { - return br.n, fmt.Errorf("failed to read segment crc32 hash: %w", err) + return int64(br.n), fmt.Errorf("failed to read segment crc32 hash: %w", err) } crc32Hash := uint32(crc32HashU64) // #nosec G115 // no overflow sampleCountU64, err := binary.ReadUvarint(br) if err != nil { - return br.n, fmt.Errorf("failed to read segment sample count: %w", err) + return int64(br.n), fmt.Errorf("failed to read segment sample count: %w", err) } segment.sampleCount = uint32(sampleCountU64) // #nosec G115 // no overflow segment.resize(int(size)) // #nosec G115 // no overflow - n, err = io.ReadFull(reader, segment.data) + n, err := io.ReadFull(reader, segment.data) if err != nil { - return br.n, fmt.Errorf("failed to read segment data: %w", err) + return int64(br.n), fmt.Errorf("failed to read segment data: %w", err) } n += br.n if crc32Hash != crc32.ChecksumIEEE(segment.data) { - return n, fmt.Errorf( + return int64(n), fmt.Errorf( "crc32 did not match, want: %d, have: %d", crc32Hash, crc32.ChecksumIEEE(segment.data), ) } - return n, nil + return int64(n), nil } diff --git a/pp/go/storage/head/shard/wal/wal_reader.go b/pp/go/storage/head/shard/wal/wal_reader.go new file mode 100644 index 0000000000..658072a946 --- /dev/null +++ b/pp/go/storage/head/shard/wal/wal_reader.go @@ -0,0 +1,55 @@ +package wal + +import ( + "bufio" + "errors" + "fmt" + "io" +) + +// ReadSegment the minimum required [Segment] implementation for a [Wal]. +type ReadSegment[T any] interface { + // ReadFrom reads [ReadSegment] data from r [io.Reader]. The return value n is the number of bytes read. + // Any error encountered during the read is also returned. + ReadFrom(r io.Reader) (int64, error) + + // Reset [ReadSegment] data. + Reset() + + // for use as a pointer + *T +} + +// SegmentWalReader buffered reader [ReadSegment]s from wal. +type SegmentWalReader[T any, TReadSegment ReadSegment[T]] struct { + reader *bufio.Reader +} + +// NewSegmentWalReader init new [SegmentWalReader]. +func NewSegmentWalReader[T any, TReadSegment ReadSegment[T]](r io.Reader) *SegmentWalReader[T, TReadSegment] { + return &SegmentWalReader[T, TReadSegment]{ + reader: bufio.NewReaderSize(r, 1024*1024*4), + } +} + +// ForEachSegment reads [ReadSegment]s from the reader and for each [ReadSegment] a [do] is called for each, +// if an error occurs during reading it will return and reading will stop. +func (r *SegmentWalReader[T, TReadSegment]) ForEachSegment(do func(TReadSegment) error) error { + var segment TReadSegment + for { + segment.Reset() + + if _, err := segment.ReadFrom(r.reader); err != nil { + if errors.Is(err, io.EOF) { + break + } + return fmt.Errorf("failed to read segment: %w", err) + } + + if err := do(segment); err != nil { + return err + } + } + + return nil +} diff --git a/pp/go/storage/querier/multi.go b/pp/go/storage/querier/multi.go new file mode 100644 index 0000000000..840a1297e0 --- /dev/null +++ b/pp/go/storage/querier/multi.go @@ -0,0 +1,169 @@ +package querier + +import ( + "context" + "errors" + "sort" + "sync" + + "github.com/prometheus/prometheus/model/labels" + "github.com/prometheus/prometheus/storage" + "github.com/prometheus/prometheus/util/annotations" +) + +// +// MultiQuerier +// + +type MultiQuerier struct { + queriers []storage.Querier + closer func() error +} + +// NewMultiQuerier init new [MultiQuerier]. +func NewMultiQuerier(queriers []storage.Querier, closer func() error) *MultiQuerier { + qs := make([]storage.Querier, 0, len(queriers)) + for _, q := range queriers { + if rawQ, ok := q.(*MultiQuerier); ok { + qs = append(qs, rawQ.queriers...) + continue + } + + qs = append(qs, q) + } + + return &MultiQuerier{ + queriers: qs, + closer: closer, + } +} + +// Close closes all [storage.Querier]s if need. +func (q *MultiQuerier) Close() (err error) { + for _, querier := range q.queriers { + err = errors.Join(err, querier.Close()) + } + + if q.closer != nil { + err = errors.Join(err, q.closer()) + } + + return err +} + +// LabelNames returns label values present in the head for the specific label name from all [storage.Querier]s. +func (q *MultiQuerier) LabelNames( + ctx context.Context, + hints *storage.LabelHints, + matchers ...*labels.Matcher, +) ([]string, annotations.Annotations, error) { + if len(q.queriers) == 1 { + return q.queriers[0].LabelNames(ctx, hints, matchers...) + } + + labelNamesResults := make([][]string, len(q.queriers)) + annotationResults := make([]annotations.Annotations, len(q.queriers)) + errs := make([]error, len(q.queriers)) + + wg := &sync.WaitGroup{} + for index, querier := range q.queriers { + wg.Add(1) + go func(index int, querier storage.Querier) { + defer wg.Done() + labelNamesResults[index], annotationResults[index], errs[index] = querier.LabelNames( + ctx, + hints, + matchers..., + ) + }(index, querier) + } + + wg.Wait() + + labelNames := DeduplicateAndSortStringSlices(labelNamesResults...) + + return labelNames, nil, errors.Join(errs...) +} + +// LabelValues returns label values present in the head for the specific label name +// that are within the time range mint to maxt from all [storage.Querier]s. If matchers are specified the returned +// result set is reduced to label values of metrics matching the matchers. +func (q *MultiQuerier) LabelValues( + ctx context.Context, + name string, + hints *storage.LabelHints, + matchers ...*labels.Matcher, +) ([]string, annotations.Annotations, error) { + if len(q.queriers) == 1 { + return q.queriers[0].LabelValues(ctx, name, hints, matchers...) + } + + labelValuesResults := make([][]string, len(q.queriers)) + annotationResults := make([]annotations.Annotations, len(q.queriers)) + errs := make([]error, len(q.queriers)) + + wg := &sync.WaitGroup{} + for index, querier := range q.queriers { + wg.Add(1) + go func(index int, querier storage.Querier) { + defer wg.Done() + labelValuesResults[index], annotationResults[index], errs[index] = querier.LabelValues( + ctx, + name, + hints, + matchers..., + ) + }(index, querier) + } + + wg.Wait() + + labelValues := DeduplicateAndSortStringSlices(labelValuesResults...) + return labelValues, nil, errors.Join(errs...) +} + +// Select returns a set of series that matches the given label matchers from all [storage.Querier]s. +func (q *MultiQuerier) Select( + ctx context.Context, + sortSeries bool, + hints *storage.SelectHints, + matchers ...*labels.Matcher, +) storage.SeriesSet { + if len(q.queriers) == 1 { + return q.queriers[0].Select(ctx, sortSeries, hints, matchers...) + } + + seriesSets := make([]storage.SeriesSet, len(q.queriers)) + wg := &sync.WaitGroup{} + + for index, querier := range q.queriers { + wg.Add(1) + go func(index int, querier storage.Querier) { + defer wg.Done() + seriesSets[index] = querier.Select(ctx, sortSeries, hints, matchers...) + }(index, querier) + } + + wg.Wait() + + return storage.NewMergeSeriesSet(seriesSets, storage.ChainedSeriesMerge) +} + +// DeduplicateAndSortStringSlices merge, deduplicate, and sort rows in slices +// and return a single sorted slice of unique rows. +func DeduplicateAndSortStringSlices(stringSlices ...[]string) []string { + dedup := make(map[string]struct{}) + for _, stringSlice := range stringSlices { + for _, value := range stringSlice { + dedup[value] = struct{}{} + } + } + + result := make([]string, 0, len(dedup)) + for value := range dedup { + result = append(result, value) + } + + sort.Strings(result) + return result +} diff --git a/pp/go/storage/querier/querier.go b/pp/go/storage/querier/querier.go index 47a083f022..0568cd423e 100644 --- a/pp/go/storage/querier/querier.go +++ b/pp/go/storage/querier/querier.go @@ -79,7 +79,7 @@ func NewQuerier[ } } -// Close Querier if need. +// Close [Querier] if need. func (q *Querier[TGenericTask, TDataStorage, TLSS, TShard, THead]) Close() error { if q.closer != nil { return q.closer() From 3f99fe3c2a54cd9d04bab41c366901059b1c6cec Mon Sep 17 00:00:00 2001 From: Alexandr Yudin Date: Thu, 28 Aug 2025 11:16:56 +0000 Subject: [PATCH 22/96] add chunkqurier --- pp/go/storage/{head/builder => }/builder.go | 2 +- pp/go/storage/{head/loader => }/loader.go | 23 +- pp/go/storage/querier/chunk_querier.go | 155 ++++++++++++++ pp/go/storage/querier/chunk_series.go | 225 ++++++++++++++++++++ pp/go/storage/querier/multi.go | 1 + pp/go/storage/querier/querier.go | 11 + pp/go/storage/querier/querier_test.go | 2 + 7 files changed, 396 insertions(+), 23 deletions(-) rename pp/go/storage/{head/builder => }/builder.go (99%) rename pp/go/storage/{head/loader => }/loader.go (60%) create mode 100644 pp/go/storage/querier/chunk_querier.go create mode 100644 pp/go/storage/querier/chunk_series.go diff --git a/pp/go/storage/head/builder/builder.go b/pp/go/storage/builder.go similarity index 99% rename from pp/go/storage/head/builder/builder.go rename to pp/go/storage/builder.go index 535745cf44..9a967a9015 100644 --- a/pp/go/storage/head/builder/builder.go +++ b/pp/go/storage/builder.go @@ -1,4 +1,4 @@ -package builder +package storage import ( "errors" diff --git a/pp/go/storage/head/loader/loader.go b/pp/go/storage/loader.go similarity index 60% rename from pp/go/storage/head/loader/loader.go rename to pp/go/storage/loader.go index 5cdb599e43..385bacd934 100644 --- a/pp/go/storage/head/loader/loader.go +++ b/pp/go/storage/loader.go @@ -1,30 +1,9 @@ -package loader +package storage import ( "github.com/prometheus/client_golang/prometheus" - "github.com/prometheus/prometheus/pp/go/storage/catalog" ) -// -// HeadsCatalog -// - -// HeadsCatalog of current head records. -type HeadsCatalog interface { - // List returns slice of records with filter and sort. - List(filterFn func(record *catalog.Record) bool, sortLess func(lhs, rhs *catalog.Record) bool) []*catalog.Record -} - -// -// Head -// - -// Head the minimum required Head implementation for a container. -type Head[T any] interface { - // for use as a pointer - *T -} - type Loader[T any, THead Head[T]] struct { catalog HeadsCatalog dir string diff --git a/pp/go/storage/querier/chunk_querier.go b/pp/go/storage/querier/chunk_querier.go new file mode 100644 index 0000000000..e75fa114c6 --- /dev/null +++ b/pp/go/storage/querier/chunk_querier.go @@ -0,0 +1,155 @@ +package querier + +import ( + "context" + + "github.com/prometheus/prometheus/model/labels" + "github.com/prometheus/prometheus/pp/go/cppbridge" + "github.com/prometheus/prometheus/pp/go/storage/logger" + "github.com/prometheus/prometheus/storage" + "github.com/prometheus/prometheus/util/annotations" +) + +const ( + // LSSQueryChunkQuerySelector name of task. + LSSQueryChunkQuerySelector = "lss_query_chunk_query_selector" + // LSSLabelValuesChunkQuerier name of task. + LSSLabelValuesChunkQuerier = "lss_label_values_chunk_querier" + // LSSLabelNamesChunkQuerier name of task. + LSSLabelNamesChunkQuerier = "lss_label_names_chunk_querier" + + // DSQueryChunkQuerier name of task. + DSQueryChunkQuerier = "data_storage_query_chunk_querier" +) + +// ChunkQuerier provides querying access over time series data of a fixed time range. +type ChunkQuerier[ + TGenericTask GenericTask, + TDataStorage DataStorage, + TLSS LSS, + TShard Shard[TDataStorage, TLSS], + THead Head[TGenericTask, TDataStorage, TLSS, TShard], +] struct { + head THead + deduplicatorCtor deduplicatorCtor + mint int64 + maxt int64 + closer func() error +} + +// NewChunkQuerier init new [ChunkQuerier]. +func NewChunkQuerier[ + TGenericTask GenericTask, + TDataStorage DataStorage, + TLSS LSS, + TShard Shard[TDataStorage, TLSS], + THead Head[TGenericTask, TDataStorage, TLSS, TShard], +]( + head THead, + deduplicatorCtor deduplicatorCtor, + mint, maxt int64, + closer func() error, +) *ChunkQuerier[TGenericTask, TDataStorage, TLSS, TShard, THead] { + return &ChunkQuerier[TGenericTask, TDataStorage, TLSS, TShard, THead]{ + head: head, + deduplicatorCtor: deduplicatorCtor, + mint: mint, + maxt: maxt, + closer: closer, + } +} + +// Close [ChunkQuerier] if need. +// +//revive:disable-next-line:confusing-naming // other type of querier. +func (q *ChunkQuerier[TGenericTask, TDataStorage, TLSS, TShard, THead]) Close() error { + if q.closer != nil { + err := q.closer() + q.closer = nil + return err + } + + return nil +} + +// LabelNames returns label values present in the head for the specific label name. +// +//revive:disable-next-line:confusing-naming // other type of querier. +func (q *ChunkQuerier[TGenericTask, TDataStorage, TLSS, TShard, THead]) LabelNames( + ctx context.Context, + hints *storage.LabelHints, + matchers ...*labels.Matcher, +) ([]string, annotations.Annotations, error) { + return queryLabelNames( + ctx, + q.head, + q.deduplicatorCtor, + nil, + LSSLabelNamesChunkQuerier, + hints, + matchers..., + ) +} + +// LabelValues returns label values present in the head for the specific label name +// that are within the time range mint to maxt. If matchers are specified the returned +// result set is reduced to label values of metrics matching the matchers. +// +//revive:disable:confusing-naming // other type of querier. +func (q *ChunkQuerier[TGenericTask, TDataStorage, TLSS, TShard, THead]) LabelValues( + ctx context.Context, + name string, + hints *storage.LabelHints, + matchers ...*labels.Matcher, +) ([]string, annotations.Annotations, error) { + return queryLabelValues( + ctx, + name, + q.head, + q.deduplicatorCtor, + nil, + LSSLabelValuesChunkQuerier, + hints, + matchers..., + ) +} + +// Select returns a chunk set of series that matches the given label matchers. +// +//revive:disable-next-line:confusing-naming // other type of querier. +func (q *ChunkQuerier[TGenericTask, TDataStorage, TLSS, TShard, THead]) Select( + ctx context.Context, + _ bool, + _ *storage.SelectHints, + matchers ...*labels.Matcher, +) storage.ChunkSeriesSet { + release, err := q.head.AcquireQuery(ctx) + if err != nil { + logger.Warnf("[ChunkQuerier]: Select failed: %s", err) + return storage.ErrChunkSeriesSet(err) + } + defer release() + + lssQueryResults, snapshots, err := queryLss(LSSQueryChunkQuerySelector, q.head, matchers) + if err != nil { + logger.Warnf("[ChunkQuerier]: failed: %s", err) + return storage.ErrChunkSeriesSet(err) + } + + serializedChunksShards := queryDataStorage(DSQueryChunkQuerier, q.head, lssQueryResults, q.mint, q.maxt) + chunkSeriesSets := make([]storage.ChunkSeriesSet, q.head.NumberOfShards()) + for shardID, serializedChunks := range serializedChunksShards { + if serializedChunks == nil { + chunkSeriesSets[shardID] = &EmptyChunkSeriesSet{} + continue + } + + chunkSeriesSets[shardID] = NewChunkSeriesSet( + lssQueryResults[shardID], + snapshots[shardID], + cppbridge.NewSerializedChunkRecoder(serializedChunks, cppbridge.TimeInterval{MinT: q.mint, MaxT: q.maxt}), + ) + } + + return storage.NewMergeChunkSeriesSet(chunkSeriesSets, storage.NewConcatenatingChunkSeriesMerger()) +} diff --git a/pp/go/storage/querier/chunk_series.go b/pp/go/storage/querier/chunk_series.go new file mode 100644 index 0000000000..231da34ec9 --- /dev/null +++ b/pp/go/storage/querier/chunk_series.go @@ -0,0 +1,225 @@ +package querier + +import ( + "github.com/prometheus/prometheus/model/labels" + "github.com/prometheus/prometheus/pp/go/cppbridge" + "github.com/prometheus/prometheus/storage" + "github.com/prometheus/prometheus/tsdb/chunkenc" + "github.com/prometheus/prometheus/tsdb/chunks" + "github.com/prometheus/prometheus/util/annotations" +) + +// +// ChunkSeriesSet +// + +// ChunkSeriesSet contains a set of chunked series. +type ChunkSeriesSet struct { + lssQueryResult *cppbridge.LSSQueryResult + labelSetSnapshot *cppbridge.LabelSetSnapshot + chunkRecoder *cppbridge.ChunkRecoder + + index int + lastRecodedChunk *cppbridge.RecodedChunk + chunkSeries *ChunkSeries + + recoderIsExhausted bool +} + +// NewChunkSeriesSet init new [ChunkSeriesSet]. +func NewChunkSeriesSet( + lssQueryResult *cppbridge.LSSQueryResult, + labelSetSnapshot *cppbridge.LabelSetSnapshot, + chunkRecoder *cppbridge.ChunkRecoder, +) *ChunkSeriesSet { + return &ChunkSeriesSet{ + lssQueryResult: lssQueryResult, + labelSetSnapshot: labelSetSnapshot, + chunkRecoder: chunkRecoder, + } +} + +// At returns full chunk series. Returned series should be iterable even after Next is called. +func (css *ChunkSeriesSet) At() storage.ChunkSeries { + return css.chunkSeries +} + +// Err returns the current error - always nil. +func (*ChunkSeriesSet) Err() error { + return nil +} + +// Next advances the iterator by one and returns false if there are no more values. +func (css *ChunkSeriesSet) Next() bool { + if css.lastRecodedChunk == nil && !css.nextChunk() { + return false + } + + seriesID := css.lastRecodedChunk.SeriesId + recodedChunks := make([]cppbridge.RecodedChunk, 1) + recodedChunks[0] = *css.lastRecodedChunk + + nextSeriesIDFound := false + for css.nextChunk() { + if css.lastRecodedChunk.SeriesId != seriesID { + nextSeriesIDFound = true + break + } + recodedChunks = append(recodedChunks, *css.lastRecodedChunk) + } + + if !nextSeriesIDFound && css.recoderIsExhausted { + css.lastRecodedChunk = nil + } + + var ( + lsID uint32 + lsLength uint16 + ) + + for { + if css.index >= css.lssQueryResult.Len() { + return false + } + + lsID, lsLength = css.lssQueryResult.GetByIndex(css.index) + + if lsID == seriesID { + break + } + + css.index++ + } + + css.chunkSeries = &ChunkSeries{ + labelSet: labels.NewLabelsWithLSS(css.labelSetSnapshot, lsID, lsLength), + recodedChunks: recodedChunks, + } + + return true +} + +// Warnings a collection of warnings for the whole set - always nil. +func (*ChunkSeriesSet) Warnings() annotations.Annotations { + return nil +} + +// nextChunk advances the iterator by one and returns false if there are no more values. +func (css *ChunkSeriesSet) nextChunk() bool { + if css.recoderIsExhausted { + return false + } + + lastRecodedChunk := css.chunkRecoder.RecodeNextChunk() + css.recoderIsExhausted = !lastRecodedChunk.HasMoreData + chunkData := make([]byte, len(lastRecodedChunk.ChunkData)) + copy(chunkData, lastRecodedChunk.ChunkData) + lastRecodedChunk.ChunkData = chunkData + css.lastRecodedChunk = &lastRecodedChunk + + return true +} + +// +// ChunkSeries +// + +// ChunkSeries exposes a single time series and allows iterating over chunks. +type ChunkSeries struct { + labelSet labels.Labels + recodedChunks []cppbridge.RecodedChunk +} + +// Iterator returns an iterator that iterates over potentially overlapping +// chunks of the series, sorted by min time. +func (cs *ChunkSeries) Iterator(iterator chunks.Iterator) chunks.Iterator { + if ci, ok := iterator.(*ChunkSeriesChunksIterator); ok { + ci.ResetTo(cs.recodedChunks) + return ci + } + + return NewChunkSeriesChunksIterator(cs.recodedChunks) +} + +// Labels returns the complete set of labels. For series it means all labels identifying the series. +func (cs *ChunkSeries) Labels() labels.Labels { + return cs.labelSet +} + +// +// ChunkSeriesChunksIterator +// + +// ChunkSeriesChunksIterator iterator that iterates over chunks of the series, sorted by min time. +type ChunkSeriesChunksIterator struct { + idx int + recodedChunks []cppbridge.RecodedChunk + xorChunk *chunkenc.XORChunk + meta chunks.Meta +} + +// NewChunkSeriesChunksIterator init new [ChunkSeriesChunksIterator]. +func NewChunkSeriesChunksIterator(recodedChunks []cppbridge.RecodedChunk) *ChunkSeriesChunksIterator { + return &ChunkSeriesChunksIterator{ + recodedChunks: recodedChunks, + xorChunk: chunkenc.NewXORChunk(), + } +} + +// At returns the current meta. +func (ci *ChunkSeriesChunksIterator) At() chunks.Meta { + return ci.meta +} + +// Err returns the current error - always nil. +func (*ChunkSeriesChunksIterator) Err() error { + return nil +} + +// Next advances the iterator by one. +func (ci *ChunkSeriesChunksIterator) Next() bool { + if ci.idx >= len(ci.recodedChunks) { + return false + } + + ci.meta.MinTime = ci.recodedChunks[ci.idx].MinT + ci.meta.MaxTime = ci.recodedChunks[ci.idx].MaxT + ci.xorChunk.Reset(ci.recodedChunks[ci.idx].ChunkData) + ci.meta.Chunk = ci.xorChunk + ci.idx++ + + return true +} + +// ResetTo reset [ChunkSeriesChunksIterator] to recodedChunks. +func (ci *ChunkSeriesChunksIterator) ResetTo(recodedChunks []cppbridge.RecodedChunk) { + ci.idx = 0 + ci.recodedChunks = recodedChunks +} + +// +// EmptyChunkSeriesSet +// + +// EmptyChunkSeriesSet implementation [ChunkSeriesSet], do nothing. +type EmptyChunkSeriesSet struct{} + +// At implementation [ChunkSeriesSet], do nothing. +func (EmptyChunkSeriesSet) At() storage.ChunkSeries { + return nil +} + +// Err implementation [ChunkSeriesSet], do nothing. +func (EmptyChunkSeriesSet) Err() error { + return nil +} + +// Next implementation [ChunkSeriesSet], do nothing. +func (EmptyChunkSeriesSet) Next() bool { + return false +} + +// Warnings implementation [ChunkSeriesSet], do nothing. +func (EmptyChunkSeriesSet) Warnings() annotations.Annotations { + return nil +} diff --git a/pp/go/storage/querier/multi.go b/pp/go/storage/querier/multi.go index 840a1297e0..db879598b9 100644 --- a/pp/go/storage/querier/multi.go +++ b/pp/go/storage/querier/multi.go @@ -15,6 +15,7 @@ import ( // MultiQuerier // +// MultiQuerier querier which makes requests to all queriers from the created list and merges the received results. type MultiQuerier struct { queriers []storage.Querier closer func() error diff --git a/pp/go/storage/querier/querier.go b/pp/go/storage/querier/querier.go index 0568cd423e..35cbb9bc1c 100644 --- a/pp/go/storage/querier/querier.go +++ b/pp/go/storage/querier/querier.go @@ -80,6 +80,8 @@ func NewQuerier[ } // Close [Querier] if need. +// +//revive:disable-next-line:confusing-naming // other type of querier. func (q *Querier[TGenericTask, TDataStorage, TLSS, TShard, THead]) Close() error { if q.closer != nil { return q.closer() @@ -89,6 +91,8 @@ func (q *Querier[TGenericTask, TDataStorage, TLSS, TShard, THead]) Close() error } // LabelNames returns label values present in the head for the specific label name. +// +//revive:disable-next-line:confusing-naming // other type of querier. func (q *Querier[TGenericTask, TDataStorage, TLSS, TShard, THead]) LabelNames( ctx context.Context, hints *storage.LabelHints, @@ -108,9 +112,12 @@ func (q *Querier[TGenericTask, TDataStorage, TLSS, TShard, THead]) LabelNames( // LabelValues returns label values present in the head for the specific label name // that are within the time range mint to maxt. If matchers are specified the returned // result set is reduced to label values of metrics matching the matchers. +// +//revive:disable-next-line:confusing-naming // other type of querier. func (q *Querier[TGenericTask, TDataStorage, TLSS, TShard, THead]) LabelValues( ctx context.Context, name string, + hints *storage.LabelHints, matchers ...*labels.Matcher, ) ([]string, annotations.Annotations, error) { return queryLabelValues( @@ -120,11 +127,14 @@ func (q *Querier[TGenericTask, TDataStorage, TLSS, TShard, THead]) LabelValues( q.deduplicatorCtor, q.metrics, LSSLabelValuesQuerier, + hints, matchers..., ) } // Select returns a set of series that matches the given label matchers. +// +//revive:disable-next-line:confusing-naming // other type of querier. func (q *Querier[TGenericTask, TDataStorage, TLSS, TShard, THead]) Select( ctx context.Context, sortSeries bool, @@ -388,6 +398,7 @@ func queryLabelValues[ deduplicatorCtor deduplicatorCtor, metrics *Metrics, taskName string, + _ *storage.LabelHints, matchers ...*labels.Matcher, ) ([]string, annotations.Annotations, error) { start := time.Now() diff --git a/pp/go/storage/querier/querier_test.go b/pp/go/storage/querier/querier_test.go index 74cf19feb8..f6eb3477f8 100644 --- a/pp/go/storage/querier/querier_test.go +++ b/pp/go/storage/querier/querier_test.go @@ -37,6 +37,8 @@ func TestXxx(t *testing.T) { querier.NewMetrics(nil, "test"), ) _ = q + + t.Log("end") } // testWal test implementation wal. From 1c3c8db7107e97e67851bcbcb936aa2c5b42d3cd Mon Sep 17 00:00:00 2001 From: Alexandr Yudin Date: Thu, 28 Aug 2025 11:59:38 +0000 Subject: [PATCH 23/96] rebuild --- pp/go/storage/builder.go | 134 ++++++++++---------- pp/go/storage/factory.go | 245 ------------------------------------- pp/go/storage/interface.go | 150 ----------------------- pp/go/storage/loader.go | 173 ++++++++++++++++++++++---- pp/go/storage/task.go | 200 ------------------------------ pp/go/storage/types.go | 22 ++++ 6 files changed, 239 insertions(+), 685 deletions(-) delete mode 100644 pp/go/storage/factory.go delete mode 100644 pp/go/storage/interface.go delete mode 100644 pp/go/storage/task.go create mode 100644 pp/go/storage/types.go diff --git a/pp/go/storage/builder.go b/pp/go/storage/builder.go index 9a967a9015..10918f425f 100644 --- a/pp/go/storage/builder.go +++ b/pp/go/storage/builder.go @@ -7,86 +7,43 @@ import ( "path/filepath" "github.com/prometheus/client_golang/prometheus" + "github.com/prometheus/prometheus/pp/go/cppbridge" "github.com/prometheus/prometheus/pp/go/storage/catalog" + "github.com/prometheus/prometheus/pp/go/storage/head/head" + "github.com/prometheus/prometheus/pp/go/storage/head/shard" + "github.com/prometheus/prometheus/pp/go/storage/head/shard/wal" + "github.com/prometheus/prometheus/pp/go/storage/head/shard/wal/writer" ) -// -// HeadsCatalog -// - -// HeadsCatalog of current head records. -type HeadsCatalog interface { - // Create creates new [Record] and write to [Log]. - Create(numberOfShards uint16) (*catalog.Record, error) - - // Delete record by ID. - Delete(id string) error - - // List returns slice of records with filter and sort. - List(filterFn func(record *catalog.Record) bool, sortLess func(lhs, rhs *catalog.Record) bool) []*catalog.Record -} - -// -// Head -// - -// Head the minimum required Head implementation for a container. -type Head[T any] interface { - // for use as a pointer - *T -} - // // Builder // -// Builder building new [Head] from factory with parameters. -type Builder[T any, THead Head[T]] struct { - catalog HeadsCatalog - dir string - generation uint64 - headCtor func( - id, headDir string, - releaseHeadFn func(), - setLastAppendedSegmentID func(segmentID uint32), - generation uint64, - maxSegmentSize uint32, - numberOfShards uint16, - registerer prometheus.Registerer, - ) (THead, error) +// Builder building new [HeadOnDisk] with parameters. +type Builder struct { + catalog *catalog.Catalog + dir string maxSegmentSize uint32 registerer prometheus.Registerer } // NewBuilder init new [Builder]. -func NewBuilder[T any, THead Head[T]]( - hcatalog HeadsCatalog, +func NewBuilder( + hcatalog *catalog.Catalog, dir string, - generation uint64, - headCtor func( - id, headDir string, - releaseHeadFn func(), - setLastAppendedSegmentID func(segmentID uint32), - generation uint64, - maxSegmentSize uint32, - numberOfShards uint16, - registerer prometheus.Registerer, - ) (THead, error), maxSegmentSize uint32, registerer prometheus.Registerer, -) *Builder[T, THead] { - return &Builder[T, THead]{ +) *Builder { + return &Builder{ catalog: hcatalog, dir: dir, - generation: generation, - headCtor: headCtor, maxSegmentSize: maxSegmentSize, registerer: registerer, } } -// Build new [Head]. -func (b *Builder[T, THead]) Build(numberOfShards uint16) (THead, error) { +// Build new [HeadOnDisk] - [head.Head] with [shard.Shard] with [wal.Wal] which is written to disk. +func (b *Builder) Build(generation uint64, numberOfShards uint16) (*HeadOnDisk, error) { headRecord, err := b.catalog.Create(numberOfShards) if err != nil { return nil, err @@ -103,21 +60,64 @@ func (b *Builder[T, THead]) Build(numberOfShards uint16) (THead, error) { } }() - h, err := b.headCtor( + shards := make([]*ShardOnDisk, numberOfShards) + swn := writer.NewSegmentWriteNotifier(numberOfShards, headRecord.SetLastAppendedSegmentID) + for shardID := range numberOfShards { + s, err := b.createShardOnDisk(headDir, swn, shardID) + if err != nil { + return nil, err + } + + shards[shardID] = s + } + + return head.NewHead( headRecord.ID(), - headDir, + shards, + shard.NewPerGoroutineShard[*WalOnDisk], headRecord.Acquire(), - headRecord.SetLastAppendedSegmentID, - b.generation, - b.maxSegmentSize, + generation, numberOfShards, b.registerer, - ) + ), nil +} + +// createShardOnDisk create [shard.Shard] with [wal.Wal] which is written to disk. +func (b *Builder) createShardOnDisk( + headDir string, + swn *writer.SegmentWriteNotifier, + shardID uint16, +) (*ShardOnDisk, error) { + shardFile, err := os.Create(filepath.Join(filepath.Clean(headDir), fmt.Sprintf("shard_%d.wal", shardID))) + if err != nil { + return nil, fmt.Errorf("failed to create shard wal file id %d: %w", shardID, err) + } + + defer func() { + if err == nil { + return + } + _ = shardFile.Close() + }() + + lss := shard.NewLSS() + // logShards is 0 for single encoder + shardWalEncoder := cppbridge.NewHeadWalEncoder(shardID, 0, lss.Target()) + + _, err = writer.WriteHeader(shardFile, wal.FileFormatVersion, shardWalEncoder.Version()) if err != nil { - return nil, fmt.Errorf("failed to create head: %w", err) + return nil, fmt.Errorf("failed to write header: %w", err) } - b.generation++ + sw, err := writer.NewBuffered(shardID, shardFile, writer.WriteSegment[*cppbridge.EncodedSegment], swn) + if err != nil { + return nil, fmt.Errorf("failed to create buffered writer shard id %d: %w", shardID, err) + } - return h, nil + return shard.NewShard( + lss, + shard.NewDataStorage(), + wal.NewWal(shardWalEncoder, sw, b.maxSegmentSize), + shardID, + ), nil } diff --git a/pp/go/storage/factory.go b/pp/go/storage/factory.go deleted file mode 100644 index 6cd480e9fa..0000000000 --- a/pp/go/storage/factory.go +++ /dev/null @@ -1,245 +0,0 @@ -package storage - -import ( - "fmt" - "os" - "path/filepath" - "sync" - - "github.com/prometheus/client_golang/prometheus" - - "github.com/prometheus/prometheus/pp/go/cppbridge" - "github.com/prometheus/prometheus/pp/go/storage/head/head" - "github.com/prometheus/prometheus/pp/go/storage/head/shard" - "github.com/prometheus/prometheus/pp/go/storage/head/shard/wal" - "github.com/prometheus/prometheus/pp/go/storage/head/shard/wal/reader" - "github.com/prometheus/prometheus/pp/go/storage/head/shard/wal/writer" - "github.com/prometheus/prometheus/pp/go/storage/logger" - "github.com/prometheus/prometheus/pp/go/util/optional" -) - -// WalOnDisk wal on disk. -type WalOnDisk = wal.Wal[ - *cppbridge.EncodedSegment, - cppbridge.WALEncoderStats, - *writer.Buffered[*cppbridge.EncodedSegment], -] - -// ShardOnDisk [shard.Shard] with [WalOnDisk]. -type ShardOnDisk = shard.Shard[*WalOnDisk] - -// HeadOnDisk [head.Head] with [ShardOnDisk]. -type HeadOnDisk = head.Head[*ShardOnDisk, *shard.PerGoroutineShard[*WalOnDisk]] - -// ShardOnDiskConstructor create [shard.Shard] with [wal.Wal] which is written to disk. -func ShardOnDiskConstructor( - dir string, - swn *writer.SegmentWriteNotifier, - maxSegmentSize uint32, - shardID uint16, -) (*ShardOnDisk, error) { - shardFile, err := os.Create(filepath.Join(filepath.Clean(dir), fmt.Sprintf("shard_%d.wal", shardID))) - if err != nil { - return nil, fmt.Errorf("failed to create shard wal file id %d: %w", shardID, err) - } - - defer func() { - if err == nil { - return - } - _ = shardFile.Close() - }() - - lss := shard.NewLSS() - // logShards is 0 for single encoder - shardWalEncoder := cppbridge.NewHeadWalEncoder(shardID, 0, lss.Target()) - - _, err = writer.WriteHeader(shardFile, wal.FileFormatVersion, shardWalEncoder.Version()) - if err != nil { - return nil, fmt.Errorf("failed to write header: %w", err) - } - - sw, err := writer.NewBuffered(shardID, shardFile, writer.WriteSegment[*cppbridge.EncodedSegment], swn) - if err != nil { - return nil, fmt.Errorf("failed to create buffered writer shard id %d: %w", shardID, err) - } - - return shard.NewShard( - lss, - shard.NewDataStorage(), - wal.NewWal(shardWalEncoder, sw, maxSegmentSize), - shardID, - ), nil -} - -// HeadConstructor create [head.Head] with [shard.Shard] with [wal.Wal] which is written to disk. -func HeadConstructor( - id, headDir string, - releaseHeadFn func(), - setLastAppendedSegmentID func(segmentID uint32), - generation uint64, - maxSegmentSize uint32, - numberOfShards uint16, - registerer prometheus.Registerer, -) (*HeadOnDisk, error) { - shards := make([]*ShardOnDisk, numberOfShards) - swn := writer.NewSegmentWriteNotifier(numberOfShards, setLastAppendedSegmentID) - for shardID := range numberOfShards { - s, err := ShardOnDiskConstructor( - headDir, - swn, - maxSegmentSize, - shardID, - ) - if err != nil { - return nil, err - } - - shards[shardID] = s - } - - return head.NewHead( - id, - shards, - shard.NewPerGoroutineShard[*WalOnDisk], - releaseHeadFn, - generation, - numberOfShards, - registerer, - ), nil -} - -// UploadHead upload [HeadOnDisk] from [WalOnDisk]. -func UploadHead( - id, headDir string, - releaseHeadFn func(), - setLastAppendedSegmentID func(segmentID uint32), - generation uint64, - maxSegmentSize uint32, - numberOfShards uint16, - registerer prometheus.Registerer, -) (_ *HeadOnDisk, _ uint32, corrupted bool) { - shardLoadResults := make([]ShardLoadResult, numberOfShards) - wg := &sync.WaitGroup{} - swn := writer.NewSegmentWriteNotifier(numberOfShards, setLastAppendedSegmentID) - for shardID := range numberOfShards { - wg.Add(1) - go func(shardID uint16, shardWalFilePath string) { - defer wg.Done() - shardLoadResults[shardID] = UploadShard(shardWalFilePath, swn, maxSegmentSize, shardID) - }(shardID, filepath.Join(headDir, fmt.Sprintf("shard_%d.wal", shardID))) - } - wg.Wait() - - shards := make([]*ShardOnDisk, numberOfShards) - numberOfSegmentsRead := optional.Optional[uint32]{} - - for shardID, res := range shardLoadResults { - shards[shardID] = res.Shard() - if res.Corrupted() { - corrupted = true - } - - if numberOfSegmentsRead.IsNil() { - numberOfSegmentsRead.Set(res.NumberOfSegments()) - } else if numberOfSegmentsRead.Value() != res.NumberOfSegments() { - corrupted = true - // calculating maximum number of segments (critical for remote write). - if numberOfSegmentsRead.Value() < res.NumberOfSegments() { - numberOfSegmentsRead.Set(res.NumberOfSegments()) - } - } - } - - // h.MergeOutOfOrderChunks() - return head.NewHead( - id, - shards, - shard.NewPerGoroutineShard[*WalOnDisk], - releaseHeadFn, - generation, - numberOfShards, - registerer, - ), - numberOfSegmentsRead.Value(), - corrupted -} - -// UploadShard upload [ShardOnDisk] from [WalOnDisk]. -func UploadShard( - shardFilePath string, - swn *writer.SegmentWriteNotifier, - maxSegmentSize uint32, - shardID uint16, -) ShardLoadResult { - res := ShardLoadResult{corrupted: true} - - //revive:disable-next-line:add-constant file permissions simple readable as octa-number - shardFile, err := os.OpenFile(shardFilePath, os.O_RDWR, 0o600) // #nosec G304 // it's meant to be that way - if err != nil { - logger.Debugf("failed to open file shard id %d: %w", shardID, err) - return res - } - defer func() { - if res.corrupted { - _ = shardFile.Close() - } - }() - - _, encoderVersion, _, err := reader.ReadHeader(shardFile) - if err != nil { - logger.Debugf("failed to read wal header: %w", err) - return res - } - - lss := shard.NewLSS() - decoder := cppbridge.NewHeadWalDecoder(lss.Target(), encoderVersion) - dataStorage := shard.NewDataStorage() - - if err = wal.NewSegmentWalReader[reader.Segment](shardFile).ForEachSegment(func(s *reader.Segment) error { - if decodeErr := dataStorage.DecodeSegment(decoder, s.Bytes()); decodeErr != nil { - return fmt.Errorf("failed to decode segment: %w", decodeErr) - } - - res.numberOfSegments++ - - return nil - }); err != nil { - logger.Debugf(err.Error()) - return res - } - - sw, err := writer.NewBuffered(shardID, shardFile, writer.WriteSegment[*cppbridge.EncodedSegment], swn) - if err != nil { - logger.Debugf("failed to create buffered writer shard id %d: %w", shardID, err) - return res - } - - swn.Set(shardID, res.numberOfSegments) - res.corrupted = false - res.shard = shard.NewShard(lss, dataStorage, wal.NewWal(decoder.CreateEncoder(), sw, maxSegmentSize), shardID) - - return res -} - -// ShardLoadResult the result of uploading the [ShardOnDisk] from the [WalOnDisk]. -type ShardLoadResult struct { - shard *ShardOnDisk - numberOfSegments uint32 - corrupted bool -} - -// Corrupted returns true if [ShardOnDisk] is corrupted. -func (sr *ShardLoadResult) Corrupted() bool { - return sr.corrupted -} - -// NumberOfSegments returns number of segments in [ShardOnDisk]s. -func (sr *ShardLoadResult) NumberOfSegments() uint32 { - return sr.numberOfSegments -} - -// Shard returns [*ShardOnDisk] or nil. -func (sr *ShardLoadResult) Shard() *ShardOnDisk { - return sr.shard -} diff --git a/pp/go/storage/interface.go b/pp/go/storage/interface.go deleted file mode 100644 index 5fcdbb86d4..0000000000 --- a/pp/go/storage/interface.go +++ /dev/null @@ -1,150 +0,0 @@ -package storage - -// import ( -// "context" - -// "github.com/prometheus/prometheus/pp/go/cppbridge" -// "github.com/prometheus/prometheus/pp/go/model" -// "github.com/prometheus/prometheus/pp/go/relabeler/config" -// ) - -// // -// // Head -// // - -// // Head implementation of the head with added metrics. -// type Head interface { -// ID() string -// Generation() uint64 -// Append( -// ctx context.Context, -// incomingData *IncomingData, -// state *cppbridge.State, -// relabelerID string, -// commitToWal bool, -// ) ([][]*cppbridge.InnerSeries, cppbridge.RelabelerStats, error) -// CommitToWal() error -// // MergeOutOfOrderChunks merge chunks with out of order data chunks. -// MergeOutOfOrderChunks() -// NumberOfShards() uint16 -// Stop() -// Flush() error -// Reconfigure(ctx context.Context, inputRelabelerConfigs []*config.InputRelabelerConfig, numberOfShards uint16) error -// WriteMetrics(ctx context.Context) -// Status(limit int) HeadStatus -// Rotate() error -// Close() error -// Discard() error -// String() string -// CopySeriesFrom(other Head) -// Enqueue(t *GenericTask) -// EnqueueOnShard(t *GenericTask, shardID uint16) -// CreateTask(taskName string, fn ShardFn, isLss bool) *GenericTask -// Concurrency() int64 -// RLockQuery(ctx context.Context) (runlock func(), err error) -// Raw() Head -// } - -// // -// // Shard -// // - -// // Shard interface for shards [Head]. -// type Shard interface { -// // DataStorage returns [DataStorage] shard. -// DataStorage() DataStorage -// // lock for DataStorage -// DataStorageLock() -// DataStorageRLock() -// DataStorageRUnlock() -// DataStorageUnlock() -// // LSS returns [LSS] shard. -// LSS() LSS -// // lock for LSS -// LSSLock() -// LSSRLock() -// LSSRUnlock() -// LSSUnlock() -// // ShardID returns ID shard. -// ShardID() uint16 -// // Wal returns [Wal] shard. -// Wal() Wal -// } - -// // ShardFn function executing on a [Shard]. -// type ShardFn func(shard Shard) error - -// // -// // DataStorage -// // - -// // DataStorage sample storage interface. -// type DataStorage interface { -// AllocatedMemory() uint64 -// // AppendInnerSeriesSlice append slice of [cppbridge.InnerSeries](samples with label IDs) to the storage. -// AppendInnerSeriesSlice(innerSeriesSlice []*cppbridge.InnerSeries) -// InstantQuery(targetTimestamp, notFoundValueTimestampValue int64, seriesIDs []uint32) []cppbridge.Sample -// MergeOutOfOrderChunks() -// Query(query cppbridge.HeadDataStorageQuery) *cppbridge.HeadDataStorageSerializedChunks -// Raw() *cppbridge.HeadDataStorage -// } - -// // -// // LSS -// // - -// // LSS labelset storage interface. -// type LSS interface { -// AllocatedMemory() uint64 -// GetLabelSets(labelSetIDs []uint32) *cppbridge.LabelSetStorageGetLabelSetsResult -// GetSnapshot() *cppbridge.LabelSetSnapshot -// Input() *cppbridge.LabelSetStorage -// QueryLabelNames(matchers []model.LabelMatcher) *cppbridge.LSSQueryLabelNamesResult -// QueryLabelValues(label_name string, matchers []model.LabelMatcher) *cppbridge.LSSQueryLabelValuesResult -// QuerySelector(matchers []model.LabelMatcher) (selector uintptr, status uint32) -// Raw() *cppbridge.LabelSetStorage -// ResetSnapshot() -// Target() *cppbridge.LabelSetStorage -// } - -// // -// // Wal -// // - -// // Wal write-ahead log for [Shard]. -// type Wal interface { -// // DO NOT USE in public interfaces like ForEachShard -// Commit() error -// Flush() error -// Write(innerSeriesSlice []*cppbridge.InnerSeries) (bool, error) -// } - -// // -// // MetricData -// // - -// // MetricData is an universal interface for blob protobuf data or batch [model.TimeSeries]. -// type MetricData interface { -// // Destroy incoming data. -// Destroy() -// } - -// // -// // ProtobufData -// // - -// // ProtobufData is an universal interface for blob protobuf data. -// type ProtobufData interface { -// Bytes() []byte -// Destroy() -// } - -// // -// // TimeSeriesData -// // - -// // TimeSeriesBatch is an universal interface for batch [model.TimeSeries]. -// type TimeSeriesBatch interface { -// TimeSeries() []model.TimeSeries -// Destroy() -// } diff --git a/pp/go/storage/loader.go b/pp/go/storage/loader.go index 385bacd934..2cc939464f 100644 --- a/pp/go/storage/loader.go +++ b/pp/go/storage/loader.go @@ -1,33 +1,160 @@ package storage import ( + "fmt" + "os" + "path/filepath" + "sync" + "github.com/prometheus/client_golang/prometheus" + "github.com/prometheus/prometheus/pp/go/cppbridge" + "github.com/prometheus/prometheus/pp/go/storage/catalog" + "github.com/prometheus/prometheus/pp/go/storage/head/head" + "github.com/prometheus/prometheus/pp/go/storage/head/shard" + "github.com/prometheus/prometheus/pp/go/storage/head/shard/wal" + "github.com/prometheus/prometheus/pp/go/storage/head/shard/wal/reader" + "github.com/prometheus/prometheus/pp/go/storage/head/shard/wal/writer" + "github.com/prometheus/prometheus/pp/go/storage/logger" + "github.com/prometheus/prometheus/pp/go/util/optional" ) -type Loader[T any, THead Head[T]] struct { - catalog HeadsCatalog - dir string - headCtor func( - id, headDir string, - releaseHeadFn func(), - setLastAppendedSegmentID func(segmentID uint32), - generation uint64, - maxSegmentSize uint32, - numberOfShards uint16, - registerer prometheus.Registerer, - ) (THead, error) - uploadHead func( - id, headDir string, - releaseHeadFn func(), - setLastAppendedSegmentID func(segmentID uint32), - generation uint64, - maxSegmentSize uint32, - numberOfShards uint16, - registerer prometheus.Registerer, - ) (THead, uint32, bool) +type Loader struct { + dir string maxSegmentSize uint32 registerer prometheus.Registerer +} + +// UploadHead upload [HeadOnDisk] from [WalOnDisk] by head ID. +func (l *Loader) UploadHead( + headRecord *catalog.Record, + generation uint64, +) (_ *HeadOnDisk, _ uint32, corrupted bool) { + headID := headRecord.ID() + headDir := filepath.Join(l.dir, headID) + numberOfShards := headRecord.NumberOfShards() + shardLoadResults := make([]ShardLoadResult, numberOfShards) + wg := &sync.WaitGroup{} + swn := writer.NewSegmentWriteNotifier(numberOfShards, headRecord.SetLastAppendedSegmentID) + for shardID := range numberOfShards { + wg.Add(1) + go func(shardID uint16, shardWalFilePath string) { + defer wg.Done() + shardLoadResults[shardID] = l.UploadShard(shardWalFilePath, swn, shardID) + }(shardID, filepath.Join(headDir, fmt.Sprintf("shard_%d.wal", shardID))) + } + wg.Wait() + + shards := make([]*ShardOnDisk, numberOfShards) + numberOfSegmentsRead := optional.Optional[uint32]{} + for shardID, res := range shardLoadResults { + shards[shardID] = res.Shard() + if res.Corrupted() { + corrupted = true + } + + if numberOfSegmentsRead.IsNil() { + numberOfSegmentsRead.Set(res.NumberOfSegments()) + } else if numberOfSegmentsRead.Value() != res.NumberOfSegments() { + corrupted = true + // calculating maximum number of segments (critical for remote write). + if numberOfSegmentsRead.Value() < res.NumberOfSegments() { + numberOfSegmentsRead.Set(res.NumberOfSegments()) + } + } + } + + // h.MergeOutOfOrderChunks() + return head.NewHead( + headID, + shards, + shard.NewPerGoroutineShard[*WalOnDisk], + headRecord.Acquire(), + generation, + numberOfShards, + l.registerer, + ), + numberOfSegmentsRead.Value(), + corrupted +} + +// UploadShard upload [ShardOnDisk] from [WalOnDisk]. +func (l *Loader) UploadShard( + shardFilePath string, + swn *writer.SegmentWriteNotifier, + shardID uint16, +) ShardLoadResult { + res := ShardLoadResult{corrupted: true} + + //revive:disable-next-line:add-constant file permissions simple readable as octa-number + shardFile, err := os.OpenFile(shardFilePath, os.O_RDWR, 0o600) // #nosec G304 // it's meant to be that way + if err != nil { + logger.Debugf("failed to open file shard id %d: %w", shardID, err) + return res + } + defer func() { + if res.corrupted { + _ = shardFile.Close() + } + }() + + _, encoderVersion, _, err := reader.ReadHeader(shardFile) + if err != nil { + logger.Debugf("failed to read wal header: %w", err) + return res + } + + lss := shard.NewLSS() + decoder := cppbridge.NewHeadWalDecoder(lss.Target(), encoderVersion) + dataStorage := shard.NewDataStorage() + + if err = wal.NewSegmentWalReader[reader.Segment](shardFile).ForEachSegment(func(s *reader.Segment) error { + if decodeErr := dataStorage.DecodeSegment(decoder, s.Bytes()); decodeErr != nil { + return fmt.Errorf("failed to decode segment: %w", decodeErr) + } + + res.numberOfSegments++ + + return nil + }); err != nil { + logger.Debugf(err.Error()) + return res + } + + sw, err := writer.NewBuffered(shardID, shardFile, writer.WriteSegment[*cppbridge.EncodedSegment], swn) + if err != nil { + logger.Debugf("failed to create buffered writer shard id %d: %w", shardID, err) + return res + } + + swn.Set(shardID, res.numberOfSegments) + res.corrupted = false + res.shard = shard.NewShard(lss, dataStorage, wal.NewWal(decoder.CreateEncoder(), sw, l.maxSegmentSize), shardID) + + return res +} + +// +// ShardLoadResult +// + +// ShardLoadResult the result of uploading the [ShardOnDisk] from the [WalOnDisk]. +type ShardLoadResult struct { + shard *ShardOnDisk + numberOfSegments uint32 + corrupted bool +} + +// Corrupted returns true if [ShardOnDisk] is corrupted. +func (sr *ShardLoadResult) Corrupted() bool { + return sr.corrupted +} + +// NumberOfSegments returns number of segments in [ShardOnDisk]s. +func (sr *ShardLoadResult) NumberOfSegments() uint32 { + return sr.numberOfSegments +} - // TODO ? - // generation uint64 +// Shard returns [*ShardOnDisk] or nil. +func (sr *ShardLoadResult) Shard() *ShardOnDisk { + return sr.shard } diff --git a/pp/go/storage/task.go b/pp/go/storage/task.go deleted file mode 100644 index 086fff8df5..0000000000 --- a/pp/go/storage/task.go +++ /dev/null @@ -1,200 +0,0 @@ -package storage - -// import ( -// "errors" -// "sync" -// "sync/atomic" -// "time" - -// "github.com/prometheus/client_golang/prometheus" -// ) - -// const ( -// // ForLSSTask task for LSS. -// ForLSSTask = true -// // ForDataStorageTask task for DataStorage. -// ForDataStorageTask = false -// // ExclusiveTask exclusive task(write). -// ExclusiveTask = true -// // NonExclusiveTask non-exclusive task(read). -// NonExclusiveTask = false -// ) - -// const ( -// // LSSInputRelabeling name of task. -// LSSInputRelabeling = "lss_input_relabeling" -// // LSSAppendRelabelerSeries name of task. -// LSSAppendRelabelerSeries = "lss_append_relabeler_series" - -// // LSSWalCommit name of task. -// LSSWalCommit = "lss_wal_commit" -// // LSSWalFlush name of task. -// LSSWalFlush = "lss_wal_flush" -// // LSSWalWrite name of task. -// LSSWalWrite = "lss_wal_write" - -// // LSSCopyAddedSeries name of task. -// LSSCopyAddedSeries = "lss_copy_added_series" - -// // LSSOutputRelabeling name of task. -// LSSOutputRelabeling = "lss_output_relabeling" - -// // LSSAllocatedMemory name of task. -// LSSAllocatedMemory = "lss_allocated_memory" - -// // LSSHeadStatus name of task. -// LSSHeadStatus = "lss_head_status" - -// // LSSQueryChunkQuerySelector name of task. -// LSSQueryChunkQuerySelector = "lss_query_chunk_query_selector" -// // LSSLabelValuesChunkQuerier name of task. -// LSSLabelValuesChunkQuerier = "lss_label_values_chunk_querier" -// // LSSLabelNamesChunkQuerier name of task. -// LSSLabelNamesChunkQuerier = "lss_label_names_chunk_querier" - -// // LSSQueryInstantQuerySelector name of task. -// LSSQueryInstantQuerySelector = "lss_query_instant_query_selector" -// // LSSQueryRangeQuerySelector name of task. -// LSSQueryRangeQuerySelector = "lss_query_range_query_selector" -// // LSSLabelValuesQuerier name of task. -// LSSLabelValuesQuerier = "lss_label_values_querier" -// // LSSLabelNamesQuerier name of task. -// LSSLabelNamesQuerier = "lss_label_names_querier" - -// // DSAppendInnerSeries name of task. -// DSAppendInnerSeries = "data_storage_append_inner_series" -// // DSMergeOutOfOrderChunks name of task. -// DSMergeOutOfOrderChunks = "data_storage_merge_out_of_order_chunks" - -// // DSAllocatedMemory name of task. -// DSAllocatedMemory = "data_storage_allocated_memory" - -// // DSHeadStatus name of task. -// DSHeadStatus = "data_storage_head_status" - -// // DSQueryChunkQuerier name of task. -// DSQueryChunkQuerier = "data_storage_query_chunk_querier" - -// // DSQueryInstantQuerier name of task. -// DSQueryInstantQuerier = "data_storage_query_instant_querier" -// // DSQueryRangeQuerier name of task. -// DSQueryRangeQuerier = "data_storage_query_range_querier" - -// // Read Only - -// // BlockWrite name of task. -// BlockWrite = "block_write" -// ) - -// // -// // GenericTask -// // - -// // GenericTask generic task, will be executed on each shard. -// type GenericTask struct { -// errs []error -// shardFn ShardFn -// wg sync.WaitGroup -// createdTS int64 -// executeTS int64 -// created prometheus.Counter -// done prometheus.Counter -// live prometheus.Counter -// execute prometheus.Counter -// forLSS bool -// } - -// // NewGenericTask init new [GenericTask]. -// func NewGenericTask( -// shardFn ShardFn, -// created, done, live, execute prometheus.Counter, -// forLSS bool, -// ) *GenericTask { -// t := &GenericTask{ -// shardFn: shardFn, -// wg: sync.WaitGroup{}, -// createdTS: time.Now().UnixMicro(), -// created: created, -// done: done, -// live: live, -// execute: execute, -// forLSS: forLSS, -// } -// t.created.Inc() - -// return t -// } - -// // NewReadOnlyGenericTask init new GenericTask for read only head. -// func NewReadOnlyGenericTask(shardFn ShardFn) *GenericTask { -// t := &GenericTask{ -// shardFn: shardFn, -// wg: sync.WaitGroup{}, -// } - -// return t -// } - -// // SetShardsNumber set shards number -// func (t *GenericTask) SetShardsNumber(number uint16) { -// t.errs = make([]error, number) -// t.wg.Add(int(number)) -// } - -// // ExecuteOnShard execute task on shard. -// func (t *GenericTask) ExecuteOnShard(shard Shard) { -// atomic.CompareAndSwapInt64(&t.executeTS, 0, time.Now().UnixMicro()) -// t.errs[shard.ShardID()] = t.shardFn(shard) -// t.wg.Done() -// } - -// // ForLSS indicates that the task is for operation on lss. -// func (t *GenericTask) ForLSS() bool { -// return t.forLSS -// } - -// // Wait for the task to complete on all shards. -// func (t *GenericTask) Wait() error { -// t.wg.Wait() -// if t.done == nil { -// return errors.Join(t.errs...) -// } - -// now := time.Now().UnixMicro() -// t.done.Inc() -// t.execute.Add(float64(now - t.executeTS)) -// t.live.Add(float64(now - t.createdTS)) - -// return errors.Join(t.errs...) -// } - -// // -// // TaskWaiter -// // - -// // TaskWaiter aggregates the wait for tasks to be completed. -// type TaskWaiter struct { -// tasks []*GenericTask -// } - -// // NewTaskWaiter init new TaskWaiter for n task. -// func NewTaskWaiter(n int) *TaskWaiter { -// return &TaskWaiter{ -// tasks: make([]*GenericTask, 0, n), -// } -// } - -// // Add task to waiter. -// func (tw *TaskWaiter) Add(t *GenericTask) { -// tw.tasks = append(tw.tasks, t) -// } - -// // Wait for tasks to be completed. -// func (tw *TaskWaiter) Wait() error { -// errs := make([]error, len(tw.tasks)) -// for _, t := range tw.tasks { -// errs = append(errs, t.Wait()) -// } - -// return errors.Join(errs...) -// } diff --git a/pp/go/storage/types.go b/pp/go/storage/types.go new file mode 100644 index 0000000000..862fcedb61 --- /dev/null +++ b/pp/go/storage/types.go @@ -0,0 +1,22 @@ +package storage + +import ( + "github.com/prometheus/prometheus/pp/go/cppbridge" + "github.com/prometheus/prometheus/pp/go/storage/head/head" + "github.com/prometheus/prometheus/pp/go/storage/head/shard" + "github.com/prometheus/prometheus/pp/go/storage/head/shard/wal" + "github.com/prometheus/prometheus/pp/go/storage/head/shard/wal/writer" +) + +// WalOnDisk wal on disk. +type WalOnDisk = wal.Wal[ + *cppbridge.EncodedSegment, + cppbridge.WALEncoderStats, + *writer.Buffered[*cppbridge.EncodedSegment], +] + +// ShardOnDisk [shard.Shard] with [WalOnDisk]. +type ShardOnDisk = shard.Shard[*WalOnDisk] + +// HeadOnDisk [head.Head] with [ShardOnDisk]. +type HeadOnDisk = head.Head[*ShardOnDisk, *shard.PerGoroutineShard[*WalOnDisk]] From 3fa04d50ecce553c9d8401e9670f4daf92fdb063 Mon Sep 17 00:00:00 2001 From: Alexandr Yudin Date: Thu, 28 Aug 2025 15:09:41 +0000 Subject: [PATCH 24/96] fix fatal --- pp/go/cppbridge/prometheus_relabeler.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pp/go/cppbridge/prometheus_relabeler.go b/pp/go/cppbridge/prometheus_relabeler.go index a4425c8592..309fce2422 100644 --- a/pp/go/cppbridge/prometheus_relabeler.go +++ b/pp/go/cppbridge/prometheus_relabeler.go @@ -1185,7 +1185,7 @@ func NewPerGoroutineRelabeler( shardID: shardID, } runtime.SetFinalizer(pgr, func(r *PerGoroutineRelabeler) { - prometheusPerShardRelabelerDtor(r.cptr) + prometheusPerGoroutineRelabelerDtor(r.cptr) }) return pgr From f6c21c47942fa451e616c33bec0b1804ed56c054 Mon Sep 17 00:00:00 2001 From: Alexandr Yudin Date: Fri, 29 Aug 2025 12:07:16 +0000 Subject: [PATCH 25/96] for save --- pp/go/storage/builder.go | 8 +- pp/go/storage/constructor.go | 89 +++ pp/go/storage/head/container/weighted.go | 1 + pp/go/storage/head/head/head.go | 1 + pp/go/storage/head/head/head.md | 19 + pp/go/storage/head/manager/manager.go | 65 +- pp/go/storage/loader.go | 16 +- pp/go/storage/remotewriter/README.md | 19 + pp/go/storage/remotewriter/cursor.go | 159 +++++ pp/go/storage/remotewriter/cursor_test.go | 160 +++++ pp/go/storage/remotewriter/datasource.go | 594 ++++++++++++++++++ pp/go/storage/remotewriter/decoder.go | 86 +++ pp/go/storage/remotewriter/destination.go | 468 ++++++++++++++ pp/go/storage/remotewriter/errors.go | 26 + pp/go/storage/remotewriter/iterator.go | 492 +++++++++++++++ pp/go/storage/remotewriter/remotewriter.go | 137 ++++ .../storage/remotewriter/remotewriter_test.go | 15 + pp/go/storage/remotewriter/walreader.go | 60 ++ pp/go/storage/remotewriter/writeloop.go | 394 ++++++++++++ pp/go/storage/remotewriter/writeloop_test.go | 263 ++++++++ pp/go/storage/remotewriter/writer.go | 35 ++ pp/go/storage/types.go | 4 + 22 files changed, 3095 insertions(+), 16 deletions(-) create mode 100644 pp/go/storage/constructor.go create mode 100644 pp/go/storage/head/head/head.md create mode 100644 pp/go/storage/remotewriter/README.md create mode 100644 pp/go/storage/remotewriter/cursor.go create mode 100644 pp/go/storage/remotewriter/cursor_test.go create mode 100644 pp/go/storage/remotewriter/datasource.go create mode 100644 pp/go/storage/remotewriter/decoder.go create mode 100644 pp/go/storage/remotewriter/destination.go create mode 100644 pp/go/storage/remotewriter/errors.go create mode 100644 pp/go/storage/remotewriter/iterator.go create mode 100644 pp/go/storage/remotewriter/remotewriter.go create mode 100644 pp/go/storage/remotewriter/remotewriter_test.go create mode 100644 pp/go/storage/remotewriter/walreader.go create mode 100644 pp/go/storage/remotewriter/writeloop.go create mode 100644 pp/go/storage/remotewriter/writeloop_test.go create mode 100644 pp/go/storage/remotewriter/writer.go diff --git a/pp/go/storage/builder.go b/pp/go/storage/builder.go index 10918f425f..e36521fa37 100644 --- a/pp/go/storage/builder.go +++ b/pp/go/storage/builder.go @@ -22,7 +22,7 @@ import ( // Builder building new [HeadOnDisk] with parameters. type Builder struct { catalog *catalog.Catalog - dir string + dataDir string maxSegmentSize uint32 registerer prometheus.Registerer } @@ -30,13 +30,13 @@ type Builder struct { // NewBuilder init new [Builder]. func NewBuilder( hcatalog *catalog.Catalog, - dir string, + dataDir string, maxSegmentSize uint32, registerer prometheus.Registerer, ) *Builder { return &Builder{ catalog: hcatalog, - dir: dir, + dataDir: dataDir, maxSegmentSize: maxSegmentSize, registerer: registerer, } @@ -49,7 +49,7 @@ func (b *Builder) Build(generation uint64, numberOfShards uint16) (*HeadOnDisk, return nil, err } - headDir := filepath.Join(b.dir, headRecord.ID()) + headDir := filepath.Join(b.dataDir, headRecord.ID()) //revive:disable-next-line:add-constant // this is already a constant if err = os.Mkdir(headDir, 0o777); err != nil { //nolint:gosec // need this permissions return nil, err diff --git a/pp/go/storage/constructor.go b/pp/go/storage/constructor.go new file mode 100644 index 0000000000..c4bc466e18 --- /dev/null +++ b/pp/go/storage/constructor.go @@ -0,0 +1,89 @@ +package storage + +import ( + "fmt" + "os" + + "github.com/go-kit/log" + "github.com/go-kit/log/level" + "github.com/prometheus/client_golang/prometheus" + + "github.com/prometheus/prometheus/pp/go/storage/catalog" + "github.com/prometheus/prometheus/pp/go/storage/head/container" + "github.com/prometheus/prometheus/pp/go/storage/head/manager" + "github.com/prometheus/prometheus/pp/go/storage/logger" +) + +func HeadManagerCtor( + l log.Logger, + dataDir string, + hcatalog *catalog.Catalog, + maxSegmentSize uint32, + registerer prometheus.Registerer, +) (*HeadManager, error) { + dirStat, err := os.Stat(dataDir) + if err != nil { + return nil, fmt.Errorf("failed to stat dir: %w", err) + } + + if !dirStat.IsDir() { + return nil, fmt.Errorf("%s is not directory", dataDir) + } + + initLogHandler(l) + + headRecords := hcatalog.List( + func(record *catalog.Record) bool { + return record.DeletedAt() == 0 && record.Status() != catalog.StatusPersisted + }, + func(lhs, rhs *catalog.Record) bool { + return lhs.CreatedAt() < rhs.CreatedAt() + }, + ) + + loader := NewLoader( + dataDir, + maxSegmentSize, + registerer, + ) + + builder := NewBuilder( + hcatalog, + dataDir, + maxSegmentSize, + registerer, + ) + + // + activeHead := container.NewWeighted(expectedHead) + + m := manager.NewManager( + activeHead, + builder, + loader, + registerer, + ) + + return m, nil +} + +// initLogHandler init log handler for pp. +func initLogHandler(l log.Logger) { + l = log.With(l, "pp_caller", log.Caller(4)) + + logger.Debugf = func(template string, args ...any) { + level.Debug(l).Log("msg", fmt.Sprintf(template, args...)) + } + + logger.Infof = func(template string, args ...any) { + level.Info(l).Log("msg", fmt.Sprintf(template, args...)) + } + + logger.Warnf = func(template string, args ...any) { + level.Warn(l).Log("msg", fmt.Sprintf(template, args...)) + } + + logger.Errorf = func(template string, args ...any) { + level.Error(l).Log("msg", fmt.Sprintf(template, args...)) + } +} diff --git a/pp/go/storage/head/container/weighted.go b/pp/go/storage/head/container/weighted.go index 03c0ff2404..9537381ca7 100644 --- a/pp/go/storage/head/container/weighted.go +++ b/pp/go/storage/head/container/weighted.go @@ -50,6 +50,7 @@ func (c *Weighted[T, THead]) Replace(ctx context.Context, newHead THead) error { (*unsafe.Pointer)(unsafe.Pointer(&c.head)), // #nosec G103 // it's meant to be that way unsafe.Pointer(newHead), // #nosec G103 // it's meant to be that way ) + c.wlocker = locker.NewWeighted(2 * newHead.Concurrency()) // x2 for back pressure unlock() diff --git a/pp/go/storage/head/head/head.go b/pp/go/storage/head/head/head.go index 9f04dfc7a4..1e3823fea9 100644 --- a/pp/go/storage/head/head/head.go +++ b/pp/go/storage/head/head/head.go @@ -188,6 +188,7 @@ func (h *Head[TShard, TGorutineShard]) AcquireQuery(ctx context.Context) (releas // Close wals and clear metrics. func (h *Head[TShard, TGorutineShard]) Close() error { h.memoryInUse.DeletePartialMatch(prometheus.Labels{"generation": strconv.FormatUint(h.generation, 10)}) + // TODO Close ? close(h.stopc) h.wg.Wait() diff --git a/pp/go/storage/head/head/head.md b/pp/go/storage/head/head/head.md new file mode 100644 index 0000000000..968dd6671a --- /dev/null +++ b/pp/go/storage/head/head/head.md @@ -0,0 +1,19 @@ +# Head + +- Create; + - create shard(lss, datastorage, wal); + - run goroutine; +- Active: + - Append: + - LSS - write; + - DataStorage - write; + - Wal - commit(write to wal); + - Query: + - LSS - read; + - DataStorage - read; + - +- Rotated; +- Close; +- Shutdown: + - Wal - commit(write to wal); + - Wal - flush; diff --git a/pp/go/storage/head/manager/manager.go b/pp/go/storage/head/manager/manager.go index b718fb2c00..3550c268be 100644 --- a/pp/go/storage/head/manager/manager.go +++ b/pp/go/storage/head/manager/manager.go @@ -6,6 +6,7 @@ import ( "time" "github.com/prometheus/client_golang/prometheus" + "github.com/prometheus/prometheus/pp/go/storage/catalog" "github.com/prometheus/prometheus/pp/go/storage/logger" "github.com/prometheus/prometheus/pp/go/util" ) @@ -17,7 +18,7 @@ type Timer interface { } type Head interface { - // TODO ? + SetReadOnly() } type ActiveHeadContainer[THead Head] interface { @@ -31,6 +32,21 @@ type Keeper[THead Head] interface { RangeQueriableHeads(mint, maxt int64) func(func(THead) bool) } +// Loader loads [Head] from wal. +type Loader[THead Head] interface { + // UploadHead upload [THead] from wal by head ID. + UploadHead( + headRecord *catalog.Record, + generation uint64, + ) (head THead, numberOfSegments uint32, corrupted bool) +} + +// HeadBuilder building new [Head] with parameters. +type HeadBuilder[THead Head] interface { + // Build new [Head]. + Build(generation uint64, numberOfShards uint16) (THead, error) +} + // type ActiveHeadContainer[T any] interface { // Get() *T // Replace(ctx context.Context, newHead *T) error @@ -39,27 +55,48 @@ type Keeper[THead Head] interface { // var _ ActiveHeadContainer[testHead] = (*container.Weighted[testHead, *testHead])(nil) -// HeadBuilder builder for the [Head]. -type HeadBuilder[THead Head] interface { - Build(numberOfShards uint16) (THead, error) -} - type Manager[THead Head] struct { - activeHead ActiveHeadContainer[THead] + // TODO logger headBuilder HeadBuilder[THead] + headLoader Loader[THead] keeper Keeper[THead] + activeHead ActiveHeadContainer[THead] rotateTimer Timer commitTimer Timer mergeTimer Timer + generation uint64 // TODO closer vs shutdowner closer *util.Closer shutdowner *util.GracefulShutdowner rotateCounter prometheus.Counter + counter *prometheus.CounterVec numberOfShards uint16 } +// NewManager init new [Manager] of [Head]s. +func NewManager[THead Head]( + activeHead ActiveHeadContainer[THead], + headBuilder HeadBuilder[THead], + headLoader Loader[THead], + registerer prometheus.Registerer, +) *Manager[THead] { + factory := util.NewUnconflictRegisterer(registerer) + return &Manager[THead]{ + headBuilder: headBuilder, + headLoader: headLoader, + + counter: factory.NewCounterVec( + prometheus.CounterOpts{ + Name: "prompp_head_event_count", + Help: "Number of head events", + }, + []string{"type"}, + ), + } +} + // ApplyConfig update config. func (m *Manager[THead]) ApplyConfig( ctx context.Context, @@ -143,11 +180,14 @@ func (m *Manager[THead]) loop(ctx context.Context) { } func (m *Manager[THead]) rotate(ctx context.Context) error { - newHead, err := m.headBuilder.Build(m.numberOfShards) + newHead, err := m.headBuilder.Build(m.generation, m.numberOfShards) if err != nil { return fmt.Errorf("failed to build a new head: %w", err) } + // TODO oldHead.Generation() + m.generation++ + oldHead := m.activeHead.Get() // TODO @@ -156,7 +196,14 @@ func (m *Manager[THead]) rotate(ctx context.Context) error { m.keeper.Add(oldHead) // TODO if replace error? - return m.activeHead.Replace(ctx, newHead) + err = m.activeHead.Replace(ctx, newHead) + if err != nil { + return fmt.Errorf("failed to replace old to new head: %w", err) + } + + oldHead.SetReadOnly() + + return nil } // WithAppendableHead diff --git a/pp/go/storage/loader.go b/pp/go/storage/loader.go index 2cc939464f..3550df2bd5 100644 --- a/pp/go/storage/loader.go +++ b/pp/go/storage/loader.go @@ -18,19 +18,29 @@ import ( "github.com/prometheus/prometheus/pp/go/util/optional" ) +// Loader loads [HeadOnDisk] or [ShardOnDisk] from [WalOnDisk]. type Loader struct { - dir string + dataDir string maxSegmentSize uint32 registerer prometheus.Registerer } +// NewLoader init new [Loader]. +func NewLoader(dataDir string, maxSegmentSize uint32, registerer prometheus.Registerer) *Loader { + return &Loader{ + dataDir: dataDir, + maxSegmentSize: maxSegmentSize, + registerer: registerer, + } +} + // UploadHead upload [HeadOnDisk] from [WalOnDisk] by head ID. func (l *Loader) UploadHead( headRecord *catalog.Record, generation uint64, ) (_ *HeadOnDisk, _ uint32, corrupted bool) { headID := headRecord.ID() - headDir := filepath.Join(l.dir, headID) + headDir := filepath.Join(l.dataDir, headID) numberOfShards := headRecord.NumberOfShards() shardLoadResults := make([]ShardLoadResult, numberOfShards) wg := &sync.WaitGroup{} @@ -63,7 +73,7 @@ func (l *Loader) UploadHead( } } - // h.MergeOutOfOrderChunks() + // TODO h.MergeOutOfOrderChunks() return head.NewHead( headID, shards, diff --git a/pp/go/storage/remotewriter/README.md b/pp/go/storage/remotewriter/README.md new file mode 100644 index 0000000000..b20e450f3a --- /dev/null +++ b/pp/go/storage/remotewriter/README.md @@ -0,0 +1,19 @@ +1. Create cursor. +2. Create client. +3. Loop: + 1. create batch. (readTimeout) + 1. read next segment. + 1. if permanent error - batch completed + end of block is reached. + 2. if not permanent error - if batch is fulfilled or deadline reached - batch is completed. + 1. recalculate number of output shards. + 3. if no error, and batch is not full, wait for 5 sec and repeat + 2. try go (write cache). (retry+backoff) + 3. encode protobuf. + 4. send. (retry+backoff) + 1. if outdated + 1. return permanent error + 2. on error -> non permanent error + 3. on success -> nil + 5. try ack. + 6. check end of block + \ No newline at end of file diff --git a/pp/go/storage/remotewriter/cursor.go b/pp/go/storage/remotewriter/cursor.go new file mode 100644 index 0000000000..441504a8b7 --- /dev/null +++ b/pp/go/storage/remotewriter/cursor.go @@ -0,0 +1,159 @@ +package remotewriter + +import ( + "errors" + "fmt" + "os" + "unsafe" + + "github.com/edsrzf/mmap-go" + + "github.com/prometheus/prometheus/tsdb/fileutil" +) + +// +// MMapFile +// + +// MMapFile a wrapper over the mmap file. +type MMapFile struct { + file *os.File + mmap mmap.MMap +} + +// NewMMapFile init new [MMapFile]. +func NewMMapFile(fileName string, flag int, perm os.FileMode, targetSize int64) (*MMapFile, error) { + file, err := os.OpenFile(fileName, flag, perm) // #nosec G304 // it's meant to be that way + if err != nil { + return nil, fmt.Errorf("open file: %w", err) + } + + fileInfo, err := file.Stat() + if err != nil { + return nil, errors.Join(fmt.Errorf("stat file: %w", err), file.Close()) + } + + if fileInfo.Size() < targetSize { + if err = fileutil.Preallocate(file, targetSize, true); err != nil { + return nil, errors.Join(fmt.Errorf("preallocate file: %w", err), file.Close()) + } + if err = file.Sync(); err != nil { + return nil, errors.Join(fmt.Errorf("sync file: %w", err), file.Close()) + } + } + + mapped, err := mmap.Map(file, mmap.RDWR, 0) + if err != nil { + return nil, errors.Join(fmt.Errorf("map file: %w", err), file.Close()) + } + + if err = mapped.Lock(); err != nil { + return nil, errors.Join(fmt.Errorf("lock mapped file: %w", err), mapped.Unmap(), file.Close()) + } + + return &MMapFile{ + file: file, + mmap: mapped, + }, nil +} + +// Bytes returns mapped into memory data. +func (f *MMapFile) Bytes() []byte { + return f.mmap +} + +// Close closes the [os.File], rendering it unusable for I/O. +// Unmap deletes the memory mapped region, flushes any remaining changes, and sets m to nil. +func (f *MMapFile) Close() error { + return errors.Join(f.mmap.Unlock(), f.mmap.Unmap(), f.file.Close()) +} + +// Sync synchronizes the mapping's contents to the file's contents on disk. +func (f *MMapFile) Sync() error { + return f.mmap.Flush() +} + +// +// Cursor +// + +// Cursor to the required ID segment. +type Cursor struct { + targetSegmentID uint32 + configCRC32 uint32 +} + +// CursorReadWriter reader and writer [Cursor]s from mmaped [MMapFile]. +type CursorReadWriter struct { + cursor *Cursor + failedShards []byte + file *MMapFile +} + +// NewCursorReadWriter init new [CursorReadWriter]. +func NewCursorReadWriter(fileName string, numberOfShards uint16) (*CursorReadWriter, error) { + cursorSize := int64(unsafe.Sizeof(Cursor{})) + fileSize := cursorSize + int64(numberOfShards) + //revive:disable-next-line:add-constant file permissions simple readable as octa-number + file, err := NewMMapFile(fileName, os.O_CREATE|os.O_RDWR, 0o600, fileSize) + if err != nil { + return nil, fmt.Errorf("create file: %w", err) + } + + crw := &CursorReadWriter{ + cursor: (*Cursor)(unsafe.Pointer(&file.Bytes()[0])), // #nosec G103 // cast to Cursor + failedShards: unsafe.Slice( // #nosec G103 // it's meant to be that way + unsafe.SliceData(file.Bytes()[cursorSize:]), // #nosec G103 // it's meant to be that way + numberOfShards, + ), + file: file, + } + + return crw, nil +} + +// Close closes the [MMapFile]. +func (crw *CursorReadWriter) Close() error { + if crw.file != nil { + err := crw.file.Close() + if err == nil { + crw.file = nil + } + return err + } + + return nil +} + +// GetConfigCRC32 returns CRC32 for config. +func (crw *CursorReadWriter) GetConfigCRC32() uint32 { + return crw.cursor.configCRC32 +} + +// GetTargetSegmentID returns target segment ID. +func (crw *CursorReadWriter) GetTargetSegmentID() uint32 { + return crw.cursor.targetSegmentID +} + +// SetConfigCRC32 set CRC32 for config. +func (crw *CursorReadWriter) SetConfigCRC32(configCRC32 uint32) error { + crw.cursor.configCRC32 = configCRC32 + return crw.file.Sync() +} + +// SetShardCorrupted adds a flag that is a shard corrupted by shard ID. +func (crw *CursorReadWriter) SetShardCorrupted(shardID uint16) error { + crw.failedShards[shardID] = 1 + return crw.file.Sync() +} + +// SetTargetSegmentID set target segment ID. +func (crw *CursorReadWriter) SetTargetSegmentID(segmentID uint32) error { + crw.cursor.targetSegmentID = segmentID + return crw.file.Sync() +} + +// ShardIsCorrupted returns a flag that is a shard corrupted by shard ID. +func (crw *CursorReadWriter) ShardIsCorrupted(shardID uint16) bool { + return crw.failedShards[shardID] > 0 +} diff --git a/pp/go/storage/remotewriter/cursor_test.go b/pp/go/storage/remotewriter/cursor_test.go new file mode 100644 index 0000000000..d71b01d2be --- /dev/null +++ b/pp/go/storage/remotewriter/cursor_test.go @@ -0,0 +1,160 @@ +package remotewriter + +import ( + "bytes" + "os" + "testing" + "unsafe" + + "github.com/edsrzf/mmap-go" + "github.com/stretchr/testify/require" + + "github.com/prometheus/prometheus/tsdb/fileutil" +) + +func TestNewCursor(t *testing.T) { + tempFile, err := os.CreateTemp("", "mmaptest") + require.NoError(t, err) + tempFileName := tempFile.Name() + defer func() { + _ = os.RemoveAll(tempFileName) + }() + require.NoError(t, tempFile.Close()) + + var numberOfShards uint16 = 2 + func(t *testing.T) { + var crw *CursorReadWriter + crw, err = NewCursorReadWriter(tempFileName, numberOfShards) + require.NoError(t, err) + defer func() { _ = crw.Close() }() + + require.Equal(t, uint32(0), crw.GetTargetSegmentID()) + require.Equal(t, uint32(0), crw.GetConfigCRC32()) + require.False(t, crw.ShardIsCorrupted(0)) + require.False(t, crw.ShardIsCorrupted(1)) + require.NoError(t, crw.SetTargetSegmentID(42)) + require.NoError(t, crw.SetShardCorrupted(1)) + }(t) + + func(t *testing.T) { + var crw *CursorReadWriter + crw, err = NewCursorReadWriter(tempFileName, numberOfShards) + require.NoError(t, err) + defer func() { _ = crw.Close() }() + + require.Equal(t, uint32(42), crw.GetTargetSegmentID()) + require.Equal(t, uint32(0), crw.GetConfigCRC32()) + require.False(t, crw.ShardIsCorrupted(0)) + require.True(t, crw.ShardIsCorrupted(1)) + }(t) +} + +type CursorV2 struct { + TargetSegmentID uint32 + ConfigCRC32 uint32 +} + +func TestMMapFile_Bytes(t *testing.T) { + tempFile, err := os.CreateTemp("", "mmaptest") + require.NoError(t, err) + tempFileName := tempFile.Name() + defer func() { + _ = os.RemoveAll(tempFileName) + }() + require.NoError(t, tempFile.Close()) + + targetSize := unsafe.Sizeof(Cursor{}) + + func(t *testing.T) { + mfile, err := NewMMapFile(tempFileName, os.O_CREATE|os.O_RDWR, 0o600, int64(targetSize)) + require.NoError(t, err) + defer func() { + _ = mfile.Close() + }() + t.Log(mfile.mmap) + c := (*CursorV2)(unsafe.Pointer(&mfile.mmap[0])) + c.TargetSegmentID = 1 + require.NoError(t, mfile.Sync()) + }(t) + + func(t *testing.T) { + mfile, err := NewMMapFile(tempFileName, os.O_CREATE|os.O_RDWR, 0o600, int64(targetSize)) + require.NoError(t, err) + defer func() { + _ = mfile.Close() + }() + c := (*CursorV2)(unsafe.Pointer(&mfile.mmap[0])) + + require.Equal(t, uint32(1), c.TargetSegmentID) + }(t) +} + +func TestMMap(t *testing.T) { + tempFile, err := os.CreateTemp("", "mmaptest") + require.NoError(t, err) + tempFileName := tempFile.Name() + defer func() { + _ = os.RemoveAll(tempFileName) + }() + require.NoError(t, tempFile.Close()) + + testString := "lol + kek = chebureck" + + func(t *testing.T) { + var file *os.File + file, err = os.OpenFile(tempFileName, os.O_CREATE|os.O_RDWR, 0o755) + require.NoError(t, err) + defer func() { + require.NoError(t, file.Close()) + }() + + require.NoError(t, fileutil.Preallocate(file, int64(len(testString)), true)) + require.NoError(t, file.Sync()) + + var mappedFile mmap.MMap + mappedFile, err = mmap.MapRegion(file, len(testString), mmap.RDWR, 0, 0) + require.NoError(t, err) + defer func() { + require.NoError(t, mappedFile.Unmap()) + }() + copy(mappedFile, testString) + require.NoError(t, mappedFile.Flush()) + }(t) + + func(t *testing.T) { + testFile, err := os.OpenFile(tempFileName, os.O_CREATE|os.O_RDWR, 0o755) + require.NoError(t, err) + defer func() { + require.NoError(t, testFile.Close()) + }() + + mappedFile, err := mmap.Map(testFile, mmap.RDWR, 0) + require.NoError(t, err) + defer func() { + require.NoError(t, mappedFile.Unmap()) + }() + require.True(t, bytes.Equal(mappedFile, []byte(testString))) + }(t) + + func(t *testing.T) { + testFile, err := os.OpenFile(tempFileName, os.O_CREATE|os.O_RDWR|os.O_TRUNC, 0o755) + require.NoError(t, err) + defer func() { + require.NoError(t, testFile.Close()) + }() + + _, err = testFile.WriteString(testString) + require.NoError(t, err) + + mappedFile, err := mmap.Map(testFile, mmap.RDWR, 0) + require.NoError(t, err) + defer func() { + require.NoError(t, mappedFile.Unmap()) + }() + + require.True(t, bytes.Equal(mappedFile, []byte("lol + kek = chebureck"))) + copy(mappedFile, "lol + lol") + require.NoError(t, mappedFile.Flush()) + require.True(t, bytes.Equal(mappedFile, []byte("lol + lol = chebureck"))) + }(t) +} diff --git a/pp/go/storage/remotewriter/datasource.go b/pp/go/storage/remotewriter/datasource.go new file mode 100644 index 0000000000..ac1e14f6be --- /dev/null +++ b/pp/go/storage/remotewriter/datasource.go @@ -0,0 +1,594 @@ +package remotewriter + +import ( + "bytes" + "context" + "errors" + "fmt" + "io" + "os" + "path/filepath" + "sync" + + "github.com/prometheus/client_golang/prometheus" + + "github.com/prometheus/prometheus/model/labels" + "github.com/prometheus/prometheus/model/relabel" + "github.com/prometheus/prometheus/pp/go/cppbridge" + "github.com/prometheus/prometheus/pp/go/storage/catalog" + "github.com/prometheus/prometheus/pp/go/storage/logger" + "github.com/prometheus/prometheus/pp/go/util/optional" +) + +// +// CorruptMarker +// + +// CorruptMarker mark head as corrupted by ID. +type CorruptMarker interface { + // MarkCorrupted mark head as corrupted by ID. + MarkCorrupted(headID string) error +} + +// +// ShardError +// + +// ShardError error reading the shard. +type ShardError struct { + shardID int + processable bool + err error +} + +// NewShardError init new [ShardError]. +func NewShardError(shardID int, processable bool, err error) ShardError { + return ShardError{ + shardID: shardID, + processable: processable, + err: err, + } +} + +// Error returns error as string, implementation error. +func (e ShardError) Error() string { + return e.err.Error() +} + +// ShardID returns shard ID. +func (e ShardError) ShardID() int { + return e.shardID +} + +// Unwrap retruns source error. +func (e ShardError) Unwrap() error { + return e.err +} + +// +// ShardWalReader +// + +// ShardWalReader a shard wall reader. +type ShardWalReader interface { + // Close wal file. + Close() error + + // Read [Segment] from wal and return. + Read() (segment Segment, err error) +} + +// NoOpShardWalReader a shard wall reader, do nothing. +type NoOpShardWalReader struct{} + +// Close implementation [ShardWalReader], do nothing. +func (NoOpShardWalReader) Close() error { return nil } + +// Read implementation [ShardWalReader], do nothing. +func (NoOpShardWalReader) Read() (segment Segment, err error) { return segment, io.EOF } + +// +// shard +// + +type shard struct { + headID string + shardID uint16 + corrupted bool + lastReadSegmentID optional.Optional[uint32] + walReader ShardWalReader + decoder *Decoder + decoderStateFile io.WriteCloser + unexpectedEOFCount prometheus.Counter + segmentSize prometheus.Histogram +} + +// newShard init new [shard]. +func newShard( + headID string, + shardID uint16, + shardFileName, decoderStateFileName string, + resetDecoderState bool, + externalLabels labels.Labels, + relabelConfigs []*cppbridge.RelabelConfig, + unexpectedEOFCount prometheus.Counter, + segmentSize prometheus.Histogram, +) (*shard, error) { + wr, encoderVersion, err := newWalReader(shardFileName) + if err != nil { + return nil, fmt.Errorf("failed to create wal file reader: %w", err) + } + + decoder, err := NewDecoder( + externalLabels, + relabelConfigs, + shardID, + encoderVersion, + ) + if err != nil { + return nil, errors.Join(fmt.Errorf("failed to create decoder: %w", err), wr.Close()) + } + + decoderStateFileFlags := os.O_CREATE | os.O_RDWR + if resetDecoderState { + decoderStateFileFlags = decoderStateFileFlags | os.O_TRUNC + } + decoderStateFile, err := os.OpenFile(decoderStateFileName, decoderStateFileFlags, 0o600) + if err != nil { + return nil, errors.Join(fmt.Errorf("failed to open cache file: %w", err), wr.Close()) + } + + s := &shard{ + headID: headID, + shardID: shardID, + walReader: wr, + decoder: decoder, + decoderStateFile: decoderStateFile, + unexpectedEOFCount: unexpectedEOFCount, + segmentSize: segmentSize, + } + + if !resetDecoderState { + if err = decoder.LoadFrom(decoderStateFile); err != nil { + return nil, errors.Join(fmt.Errorf("failed to restore from cache: %w", err), s.Close()) + } + } else { + if err = decoderStateFile.Truncate(0); err != nil { + return nil, errors.Join(fmt.Errorf("failed to truncate decoder state file: %w", err), s.Close()) + } + } + + return s, nil +} + +func (s *shard) Close() error { + return errors.Join(s.walReader.Close(), s.decoderStateFile.Close()) +} + +func (s *shard) Read(ctx context.Context, targetSegmentID uint32, minTimestamp int64) (*DecodedSegment, error) { + if s.corrupted { + return nil, ErrShardIsCorrupted + } + + if !s.lastReadSegmentID.IsNil() && s.lastReadSegmentID.Value() >= targetSegmentID { + return nil, nil + } + + for { + if ctx.Err() != nil { + return nil, ctx.Err() + } + + segment, err := s.walReader.Read() + if err != nil { + s.corrupted = true + logger.Errorf("remotewritedebug shard %s/%d is corrupted by read: %v", s.headID, s.shardID, err) + return nil, errors.Join(err, ErrShardIsCorrupted) + } + + s.segmentSize.Observe(float64(len(segment.Data()))) + + decodedSegment, err := s.decoder.Decode(segment.Data(), minTimestamp) + if err != nil { + s.corrupted = true + logger.Errorf("remotewritedebug shard %s/%d is corrupted by decode: %v", s.headID, s.shardID, err) + return nil, errors.Join(err, ErrShardIsCorrupted) + } + + s.lastReadSegmentID.Set(segment.ID) + + if segment.ID == targetSegmentID { + decodedSegment.ID = segment.ID + return decodedSegment, nil + } + } +} + +func (s *shard) SetCorrupted() { + s.corrupted = true +} + +// +// SegmentReadyChecker +// + +type SegmentReadyChecker interface { + SegmentIsReady(segmentID uint32) (ready bool, outOfRange bool) +} + +// +// segmentReadyChecker +// + +type segmentReadyChecker struct { + headRecord *catalog.Record +} + +func newSegmentReadyChecker(headRecord *catalog.Record) *segmentReadyChecker { + return &segmentReadyChecker{headRecord: headRecord} +} + +func (src *segmentReadyChecker) SegmentIsReady(segmentID uint32) (ready bool, outOfRange bool) { + ready = src.headRecord.LastAppendedSegmentID() != nil && *src.headRecord.LastAppendedSegmentID() >= segmentID + outOfRange = (src.headRecord.Status() != catalog.StatusNew && src.headRecord.Status() != catalog.StatusActive) && !ready + return ready, outOfRange +} + +// +// shardCache +// + +type shardCache struct { + shardID uint16 + cache *bytes.Buffer + written bool + writer io.Writer +} + +// +// dataSource +// + +type dataSource struct { + ID string + shards []*shard + segmentReadyChecker SegmentReadyChecker + corruptMarker CorruptMarker + closed bool + completed bool + corrupted bool + headReleaseFunc func() + + lssSlice []*cppbridge.LabelSetStorage + + cacheMtx sync.Mutex + caches []*shardCache + cacheWriteSignal chan struct{} + cacheWriteLoopClosed chan struct{} + + unexpectedEOFCount prometheus.Counter + segmentSize prometheus.Histogram +} + +func newDataSource(dataDir string, + numberOfShards uint16, + config DestinationConfig, + discardCache bool, + segmentReadyChecker SegmentReadyChecker, + corruptMarker CorruptMarker, + headRecord *catalog.Record, + unexpectedEOFCount prometheus.Counter, + segmentSize prometheus.Histogram, +) (*dataSource, error) { + var err error + var convertedRelabelConfigs []*cppbridge.RelabelConfig + convertedRelabelConfigs, err = convertRelabelConfigs(config.WriteRelabelConfigs...) + if err != nil { + return nil, fmt.Errorf("failed to convert relabel configs: %w", err) + } + + b := &dataSource{ + corruptMarker: corruptMarker, + segmentReadyChecker: segmentReadyChecker, + headReleaseFunc: headRecord.Acquire(), + unexpectedEOFCount: unexpectedEOFCount, + segmentSize: segmentSize, + cacheWriteSignal: make(chan struct{}), + cacheWriteLoopClosed: make(chan struct{}), + } + + go b.cacheWriteLoop() + + for shardID := uint16(0); shardID < numberOfShards; shardID++ { + shardFileName := filepath.Join(dataDir, fmt.Sprintf("shard_%d.wal", shardID)) + decoderStateFileName := filepath.Join(dataDir, fmt.Sprintf("%s_shard_%d.state", config.Name, shardID)) + var s *shard + s, err = createShard( + headRecord.ID(), + shardID, + shardFileName, + decoderStateFileName, + discardCache, + config.ExternalLabels, + convertedRelabelConfigs, + b.unexpectedEOFCount, + b.segmentSize, + ) + if err != nil { + return nil, errors.Join(fmt.Errorf("failed to create shard: %w", err), b.Close()) + } + b.shards = append(b.shards, s) + b.lssSlice = append(b.lssSlice, s.decoder.lss) + b.caches = append(b.caches, &shardCache{ + shardID: shardID, + cache: bytes.NewBuffer(nil), + written: true, + writer: s.decoderStateFile, + }) + } + + return b, nil +} + +func createShard( + headID string, + shardID uint16, + shardFileName, decoderStateFileName string, + resetDecoderState bool, + externalLabels labels.Labels, + relabelConfigs []*cppbridge.RelabelConfig, + unexpectedEOFCount prometheus.Counter, + segmentSize prometheus.Histogram, +) (*shard, error) { + s, err := newShard( + headID, + shardID, + shardFileName, + decoderStateFileName, + resetDecoderState, + externalLabels, + relabelConfigs, + unexpectedEOFCount, + segmentSize, + ) + if err != nil { + logger.Errorf("failed to create shard: %v", err) + return newShard( + headID, + shardID, + shardFileName, + decoderStateFileName, + true, + externalLabels, + relabelConfigs, + unexpectedEOFCount, + segmentSize, + ) + } + return s, nil +} + +func convertRelabelConfigs(relabelConfigs ...*relabel.Config) ([]*cppbridge.RelabelConfig, error) { + convertedConfigs := make([]*cppbridge.RelabelConfig, 0, len(relabelConfigs)) + for _, relabelConfig := range relabelConfigs { + var sourceLabels []string + for _, label := range relabelConfig.SourceLabels { + sourceLabels = append(sourceLabels, string(label)) + } + + convertedConfig := &cppbridge.RelabelConfig{ + SourceLabels: sourceLabels, + Separator: relabelConfig.Separator, + Regex: relabelConfig.Regex.String(), + Modulus: relabelConfig.Modulus, + TargetLabel: relabelConfig.TargetLabel, + Replacement: relabelConfig.Replacement, + Action: cppbridge.ActionNameToValueMap[string(relabelConfig.Action)], + } + + if err := convertedConfig.Validate(); err != nil { + return nil, fmt.Errorf("failed to validate config: %w", err) + } + + convertedConfigs = append(convertedConfigs, convertedConfig) + } + + return convertedConfigs, nil +} + +func (ds *dataSource) Close() error { + if ds.closed { + return nil + } + ds.closed = true + var err error + // stop cache writing first + close(ds.cacheWriteSignal) + <-ds.cacheWriteLoopClosed + + for _, s := range ds.shards { + err = errors.Join(err, s.Close()) + } + ds.headReleaseFunc() + return err +} + +func (ds *dataSource) IsCompleted() bool { + return ds.completed +} + +type readShardResult struct { + segment *DecodedSegment + err error +} + +func (ds *dataSource) Read(ctx context.Context, segmentID uint32, minTimestamp int64) ([]*DecodedSegment, error) { + if ds.completed { + return nil, ErrEndOfBlock + } + + segmentIsReady, segmentIsOutOfRange := ds.segmentReadyChecker.SegmentIsReady(segmentID) + if !segmentIsReady { + if segmentIsOutOfRange { + return nil, ErrEndOfBlock + } + + return nil, ErrEmptyReadResult + } + + wg := &sync.WaitGroup{} + readShardResults := make([]readShardResult, len(ds.shards)) + for shardID := 0; shardID < len(ds.shards); shardID++ { + if ds.shards[shardID].corrupted { + readShardResults[shardID] = readShardResult{segment: nil, err: NewShardError(shardID, false, ErrShardIsCorrupted)} + continue + } + wg.Add(1) + go func(shardID int) { + defer wg.Done() + segment, err := ds.shards[shardID].Read(ctx, segmentID, minTimestamp) + if err != nil { + err = NewShardError(shardID, true, err) + } + readShardResults[shardID] = readShardResult{segment: segment, err: err} + }(shardID) + } + wg.Wait() + + segments := make([]*DecodedSegment, 0, len(ds.shards)) + errs := make([]error, 0, len(ds.shards)) + for _, result := range readShardResults { + if result.segment != nil { + segments = append(segments, result.segment) + } + if result.err != nil { + errs = append(errs, result.err) + } + } + + return segments, ds.handleReadErrors(errs) +} + +func (ds *dataSource) handleReadErrors(errs []error) error { + if len(errs) == 0 { + return nil + } + + if len(errs) == len(ds.shards) { + ds.corrupted = true + if ds.corruptMarker != nil { + if err := ds.corruptMarker.MarkCorrupted(ds.ID); err != nil { + return fmt.Errorf("failed to mark head corrupted: %w", err) + } + ds.corruptMarker = nil + } + + return ErrEndOfBlock + } + + ds.corrupted = true + if ds.corruptMarker != nil { + if err := ds.corruptMarker.MarkCorrupted(ds.ID); err != nil { + return fmt.Errorf("failed to mark head corrupted: %w", err) + } + ds.corruptMarker = nil + } + + for _, err := range errs { + var shardErr ShardError + if errors.As(err, &shardErr) { + if shardErr.processable { + logger.Errorf("shard %s/%d is corrupted", ds.ID, shardErr.ShardID()) + } + } + } + + return nil +} + +func (ds *dataSource) LSSes() []*cppbridge.LabelSetStorage { + return ds.lssSlice +} + +func (ds *dataSource) WriteCaches() { + ds.cacheMtx.Lock() + for shardID, sc := range ds.caches { + if !sc.written { + continue + } + sc.cache.Reset() + if _, err := ds.shards[shardID].decoder.WriteTo(sc.cache); err != nil { + logger.Errorf("failed to get output decoder cache: %v", err) + continue + } + sc.written = false + } + ds.cacheMtx.Unlock() + + select { + case ds.cacheWriteSignal <- struct{}{}: + default: + } +} + +func (ds *dataSource) cacheWriteLoop() { + defer close(ds.cacheWriteLoopClosed) + var closed bool + var writeRequested bool + var writeResultc chan struct{} + + for { + if writeRequested && !closed && writeResultc == nil { + writeResultc = make(chan struct{}) + go func() { + defer close(writeResultc) + ds.writeCaches() + }() + writeRequested = false + } + + if closed && writeResultc == nil { + return + } + + select { + case _, ok := <-ds.cacheWriteSignal: + if !ok { + return + } + writeRequested = true + case <-writeResultc: + writeResultc = nil + } + } +} + +func (ds *dataSource) writeCaches() { + ds.cacheMtx.Lock() + caches := make([]*shardCache, 0, len(ds.caches)) + for _, sc := range ds.caches { + if sc.written { + continue + } + sc := sc + caches = append(caches, sc) + } + ds.cacheMtx.Unlock() + + writtenCacheShardIDs := make([]uint16, 0, len(caches)) + for _, sc := range caches { + if _, err := sc.cache.WriteTo(sc.writer); err != nil { + logger.Errorf("failed to write cache: %v", err) + continue + } + writtenCacheShardIDs = append(writtenCacheShardIDs, sc.shardID) + } + + if len(writtenCacheShardIDs) > 0 { + ds.cacheMtx.Lock() + for _, shardID := range writtenCacheShardIDs { + ds.caches[shardID].written = true + } + ds.cacheMtx.Unlock() + } +} diff --git a/pp/go/storage/remotewriter/decoder.go b/pp/go/storage/remotewriter/decoder.go new file mode 100644 index 0000000000..a889f678b7 --- /dev/null +++ b/pp/go/storage/remotewriter/decoder.go @@ -0,0 +1,86 @@ +package remotewriter + +import ( + "fmt" + "io" + + "github.com/prometheus/prometheus/model/labels" + "github.com/prometheus/prometheus/pp/go/cppbridge" +) + +type Decoder struct { + relabeler *cppbridge.StatelessRelabeler + lss *cppbridge.LabelSetStorage + outputDecoder *cppbridge.WALOutputDecoder +} + +func NewDecoder( + externalLabels labels.Labels, + relabelConfigs []*cppbridge.RelabelConfig, + shardID uint16, + encoderVersion uint8, +) (*Decoder, error) { + relabeler, err := cppbridge.NewStatelessRelabeler(relabelConfigs) + if err != nil { + return nil, fmt.Errorf("failed to create stateless relabeler: %w", err) + } + + lss := cppbridge.NewLssStorage() + outputDecoder := cppbridge.NewWALOutputDecoder(LabelsToCppBridgeLabels(externalLabels), relabeler, lss, shardID, encoderVersion) + + return &Decoder{ + relabeler: relabeler, + lss: lss, + outputDecoder: outputDecoder, + }, nil +} + +type DecodedSegment struct { + ID uint32 + Samples *cppbridge.DecodedRefSamples + MaxTimestamp int64 + OutdatedSamplesCount uint32 + DroppedSamplesCount uint32 + AddSeriesCount uint32 + DroppedSeriesCount uint32 +} + +func (d *Decoder) Decode(segment []byte, minTimestamp int64) (*DecodedSegment, error) { + samples, stats, err := d.outputDecoder.Decode(segment, minTimestamp) + if err != nil { + return nil, err + } + return &DecodedSegment{ + Samples: samples, + MaxTimestamp: stats.MaxTimestamp(), + OutdatedSamplesCount: stats.OutdatedSampleCount(), + DroppedSamplesCount: stats.DroppedSampleCount(), + AddSeriesCount: stats.AddSeriesCount(), + DroppedSeriesCount: stats.DroppedSeriesCount(), + }, nil +} + +func (d *Decoder) LoadFrom(reader io.Reader) error { + state, err := io.ReadAll(reader) + if err != nil { + return fmt.Errorf("failed to read from reader: %w", err) + } + + return d.outputDecoder.LoadFrom(state) +} + +// WriteTo writes output decoder state to io.Writer. +func (d *Decoder) WriteTo(writer io.Writer) (int64, error) { + return d.outputDecoder.WriteTo(writer) +} + +func LabelsToCppBridgeLabels(lbls labels.Labels) []cppbridge.Label { + result := make([]cppbridge.Label, 0, lbls.Len()) + lbls.Range(func(l labels.Label) { + result = append(result, cppbridge.Label{ + Name: l.Name, + Value: l.Value, + }) + }) + return result +} diff --git a/pp/go/storage/remotewriter/destination.go b/pp/go/storage/remotewriter/destination.go new file mode 100644 index 0000000000..5fd36fbcf1 --- /dev/null +++ b/pp/go/storage/remotewriter/destination.go @@ -0,0 +1,468 @@ +package remotewriter + +import ( + "bytes" + "hash/crc32" + "sync" + "time" + + "github.com/prometheus/client_golang/prometheus" + "github.com/prometheus/common/model" + "gopkg.in/yaml.v2" + + "github.com/prometheus/prometheus/config" + "github.com/prometheus/prometheus/model/labels" +) + +// String constants for instrumentation. +const ( + namespace = "prometheus" + subsystem = "remote_storage" + remoteName = "remote_name" + endpoint = "url" + + reasonTooOld = "too_old" + reasonDroppedSeries = "dropped_series" + + DefaultSampleAgeLimit = model.Duration(time.Hour * 24 * 30) +) + +type DestinationConfig struct { + config.RemoteWriteConfig + ExternalLabels labels.Labels `yaml:"external_labels"` + ReadTimeout time.Duration +} + +func (c DestinationConfig) EqualTo(other DestinationConfig) bool { + return c.ExternalLabels.Hash() == other.ExternalLabels.Hash() && + c.ReadTimeout == other.ReadTimeout && + remoteWriteConfigsAreEqual(c.RemoteWriteConfig, other.RemoteWriteConfig) +} + +func (c DestinationConfig) CRC32() (uint32, error) { + data, err := yaml.Marshal(c) + if err != nil { + return 0, err + } + + return crc32.ChecksumIEEE(data), nil +} + +type Destination struct { + config DestinationConfig + metrics *DestinationMetrics +} + +func (d *Destination) Config() DestinationConfig { + return d.config +} + +func (d *Destination) ResetConfig(config DestinationConfig) { + d.config = config +} + +func NewDestination(cfg DestinationConfig) *Destination { + constLabels := prometheus.Labels{ + remoteName: cfg.Name, + endpoint: cfg.URL.Redacted(), + } + + return &Destination{ + config: cfg, + metrics: &DestinationMetrics{ + samplesTotal: prometheus.NewCounter(prometheus.CounterOpts{ + Namespace: namespace, + Subsystem: subsystem, + Name: "samples_total", + Help: "Total number of samples sent to remote storage.", + ConstLabels: constLabels, + }), + exemplarsTotal: prometheus.NewCounter(prometheus.CounterOpts{ + Namespace: namespace, + Subsystem: subsystem, + Name: "exemplars_total", + Help: "Total number of exemplars sent to remote storage.", + ConstLabels: constLabels, + }), + histogramsTotal: prometheus.NewCounter(prometheus.CounterOpts{ + Namespace: namespace, + Subsystem: subsystem, + Name: "histograms_total", + Help: "Total number of histograms sent to remote storage.", + ConstLabels: constLabels, + }), + metadataTotal: prometheus.NewCounter(prometheus.CounterOpts{ + Namespace: namespace, + Subsystem: subsystem, + Name: "metadata_total", + Help: "Total number of metadata entries sent to remote storage.", + ConstLabels: constLabels, + }), + failedSamplesTotal: prometheus.NewCounter(prometheus.CounterOpts{ + Namespace: namespace, + Subsystem: subsystem, + Name: "samples_failed_total", + Help: "Total number of samples which failed on send to remote storage, non-recoverable errors.", + ConstLabels: constLabels, + }), + failedExemplarsTotal: prometheus.NewCounter(prometheus.CounterOpts{ + Namespace: namespace, + Subsystem: subsystem, + Name: "exemplars_failed_total", + Help: "Total number of exemplars which failed on send to remote storage, non-recoverable errors.", + ConstLabels: constLabels, + }), + failedHistogramsTotal: prometheus.NewCounter(prometheus.CounterOpts{ + Namespace: namespace, + Subsystem: subsystem, + Name: "histograms_failed_total", + Help: "Total number of histograms which failed on send to remote storage, non-recoverable errors.", + ConstLabels: constLabels, + }), + failedMetadataTotal: prometheus.NewCounter(prometheus.CounterOpts{ + Namespace: namespace, + Subsystem: subsystem, + Name: "metadata_failed_total", + Help: "Total number of metadata entries which failed on send to remote storage, non-recoverable errors.", + ConstLabels: constLabels, + }), + retriedSamplesTotal: prometheus.NewCounter(prometheus.CounterOpts{ + Namespace: namespace, + Subsystem: subsystem, + Name: "samples_retried_total", + Help: "Total number of samples which failed on send to remote storage but were retried because the send error was recoverable.", + ConstLabels: constLabels, + }), + retriedExemplarsTotal: prometheus.NewCounter(prometheus.CounterOpts{ + Namespace: namespace, + Subsystem: subsystem, + Name: "exemplars_retried_total", + Help: "Total number of exemplars which failed on send to remote storage but were retried because the send error was recoverable.", + ConstLabels: constLabels, + }), + retriedHistogramsTotal: prometheus.NewCounter(prometheus.CounterOpts{ + Namespace: namespace, + Subsystem: subsystem, + Name: "histograms_retried_total", + Help: "Total number of histograms which failed on send to remote storage but were retried because the send error was recoverable.", + ConstLabels: constLabels, + }), + retriedMetadataTotal: prometheus.NewCounter(prometheus.CounterOpts{ + Namespace: namespace, + Subsystem: subsystem, + Name: "metadata_retried_total", + Help: "Total number of metadata entries which failed on send to remote storage but were retried because the send error was recoverable.", + ConstLabels: constLabels, + }), + droppedSamplesTotal: prometheus.NewCounterVec(prometheus.CounterOpts{ + Namespace: namespace, + Subsystem: subsystem, + Name: "samples_dropped_total", + Help: "Total number of samples which were dropped after being read from the WAL before being sent via remote write, either via relabelling, due to being too old or unintentionally because of an unknown reference ID.", + ConstLabels: constLabels, + }, []string{"reason"}), + addSeriesTotal: prometheus.NewCounter(prometheus.CounterOpts{ + Namespace: namespace, + Subsystem: subsystem, + Name: "series_added_total", + Help: "Total number of series which were add after being read from the WAL before being sent via remote write, either via relabelling.", + ConstLabels: constLabels, + }), + droppedSeriesTotal: prometheus.NewCounter(prometheus.CounterOpts{ + Namespace: namespace, + Subsystem: subsystem, + Name: "series_dropped_total", + Help: "Total number of series which were dropped after being read from the WAL before being sent via remote write, either via relabelling.", + ConstLabels: constLabels, + }), + droppedExemplarsTotal: prometheus.NewCounterVec(prometheus.CounterOpts{ + Namespace: namespace, + Subsystem: subsystem, + Name: "exemplars_dropped_total", + Help: "Total number of exemplars which were dropped after being read from the WAL before being sent via remote write, either via relabelling, due to being too old or unintentionally because of an unknown reference ID.", + ConstLabels: constLabels, + }, []string{"reason"}), + droppedHistogramsTotal: prometheus.NewCounterVec(prometheus.CounterOpts{ + Namespace: namespace, + Subsystem: subsystem, + Name: "histograms_dropped_total", + Help: "Total number of histograms which were dropped after being read from the WAL before being sent via remote write, either via relabelling, due to being too old or unintentionally because of an unknown reference ID.", + ConstLabels: constLabels, + }, []string{"reason"}), + enqueueRetriesTotal: prometheus.NewCounter(prometheus.CounterOpts{ + Namespace: namespace, + Subsystem: subsystem, + Name: "enqueue_retries_total", + Help: "Total number of times enqueue has failed because a shards queue was full.", + ConstLabels: constLabels, + }), + sentBatchDuration: prometheus.NewHistogram(prometheus.HistogramOpts{ + Namespace: namespace, + Subsystem: subsystem, + Name: "sent_batch_duration_seconds", + Help: "Duration of send calls to the remote storage.", + Buckets: append(prometheus.DefBuckets, 25, 60, 120, 300), + ConstLabels: constLabels, + NativeHistogramBucketFactor: 1.1, + NativeHistogramMaxBucketNumber: 100, + NativeHistogramMinResetDuration: 1 * time.Hour, + }), + highestSentTimestamp: &maxTimestamp{ + Gauge: prometheus.NewGauge(prometheus.GaugeOpts{ + Namespace: namespace, + Subsystem: subsystem, + Name: "queue_highest_sent_timestamp_seconds", + Help: "Timestamp from a WAL sample, the highest timestamp successfully sent by this queue, in seconds since epoch.", + ConstLabels: constLabels, + }), + }, + pendingSamples: prometheus.NewGauge(prometheus.GaugeOpts{ + Namespace: namespace, + Subsystem: subsystem, + Name: "samples_pending", + Help: "The number of samples pending in the queues shards to be sent to the remote storage.", + ConstLabels: constLabels, + }), + pendingExemplars: prometheus.NewGauge(prometheus.GaugeOpts{ + Namespace: namespace, + Subsystem: subsystem, + Name: "exemplars_pending", + Help: "The number of exemplars pending in the queues shards to be sent to the remote storage.", + ConstLabels: constLabels, + }), + pendingHistograms: prometheus.NewGauge(prometheus.GaugeOpts{ + Namespace: namespace, + Subsystem: subsystem, + Name: "histograms_pending", + Help: "The number of histograms pending in the queues shards to be sent to the remote storage.", + ConstLabels: constLabels, + }), + shardCapacity: prometheus.NewGauge(prometheus.GaugeOpts{ + Namespace: namespace, + Subsystem: subsystem, + Name: "shard_capacity", + Help: "The capacity of each shard of the queue used for parallel sending to the remote storage.", + ConstLabels: constLabels, + }), + numShards: prometheus.NewGauge(prometheus.GaugeOpts{ + Namespace: namespace, + Subsystem: subsystem, + Name: "shards", + Help: "The number of shards used for parallel sending to the remote storage.", + ConstLabels: constLabels, + }), + maxNumShards: prometheus.NewGauge(prometheus.GaugeOpts{ + Namespace: namespace, + Subsystem: subsystem, + Name: "shards_max", + Help: "The maximum number of shards that the queue is allowed to run.", + ConstLabels: constLabels, + }), + minNumShards: prometheus.NewGauge(prometheus.GaugeOpts{ + Namespace: namespace, + Subsystem: subsystem, + Name: "shards_min", + Help: "The minimum number of shards that the queue is allowed to run.", + ConstLabels: constLabels, + }), + desiredNumShards: prometheus.NewGauge(prometheus.GaugeOpts{ + Namespace: namespace, + Subsystem: subsystem, + Name: "shards_desired", + Help: "The number of shards that the queues shard calculation wants to run based on the rate of samples in vs. samples out.", + ConstLabels: constLabels, + }), + bestNumShards: prometheus.NewGauge(prometheus.GaugeOpts{ + Namespace: namespace, + Subsystem: subsystem, + Name: "shards_best", + Help: "The number of shards that are calculated from the actual number of accumulated segments.", + ConstLabels: constLabels, + }), + sentBytesTotal: prometheus.NewCounter(prometheus.CounterOpts{ + Namespace: namespace, + Subsystem: subsystem, + Name: "bytes_total", + Help: "The total number of bytes of data (not metadata) sent by the queue after compression. Note that when exemplars over remote write is enabled the exemplars included in a remote write request count towards this metric.", + ConstLabels: constLabels, + }), + metadataBytesTotal: prometheus.NewCounter(prometheus.CounterOpts{ + Namespace: namespace, + Subsystem: subsystem, + Name: "metadata_bytes_total", + Help: "The total number of bytes of metadata sent by the queue after compression.", + ConstLabels: constLabels, + }), + maxSamplesPerSend: prometheus.NewGauge(prometheus.GaugeOpts{ + Namespace: namespace, + Subsystem: subsystem, + Name: "max_samples_per_send", + Help: "The maximum number of samples to be sent, in a single request, to the remote storage. Note that, when sending of exemplars over remote write is enabled, exemplars count towards this limt.", + ConstLabels: constLabels, + }), + unexpectedEOFCount: prometheus.NewCounter( + prometheus.CounterOpts{ + Namespace: namespace, + Subsystem: subsystem, + Name: "unexpected_eof_count", + Help: "Number of eof occurred during reading active head wal", + ConstLabels: constLabels, + }, + ), + segmentSizeInBytes: prometheus.NewHistogram( + prometheus.HistogramOpts{ + Namespace: namespace, + Subsystem: subsystem, + Name: "segment_size_bytes", + Help: "Size of segment.", + ConstLabels: constLabels, + Buckets: []float64{1 << 10, 1 << 11, 1 << 12, 1 << 13, 1 << 14, 1 << 15, 1 << 16, 1 << 17, 1 << 18, 1 << 19, 1 << 20}, + }, + ), + }, + } +} + +func (d *Destination) RegisterMetrics(registerer prometheus.Registerer) { + registerer.MustRegister(d.metrics.samplesTotal) + registerer.MustRegister(d.metrics.exemplarsTotal) + registerer.MustRegister(d.metrics.histogramsTotal) + registerer.MustRegister(d.metrics.metadataTotal) + registerer.MustRegister(d.metrics.failedSamplesTotal) + registerer.MustRegister(d.metrics.failedExemplarsTotal) + registerer.MustRegister(d.metrics.failedHistogramsTotal) + registerer.MustRegister(d.metrics.failedMetadataTotal) + registerer.MustRegister(d.metrics.retriedSamplesTotal) + registerer.MustRegister(d.metrics.retriedExemplarsTotal) + registerer.MustRegister(d.metrics.retriedHistogramsTotal) + registerer.MustRegister(d.metrics.retriedMetadataTotal) + registerer.MustRegister(d.metrics.droppedSamplesTotal) + registerer.MustRegister(d.metrics.addSeriesTotal) + registerer.MustRegister(d.metrics.droppedSeriesTotal) + registerer.MustRegister(d.metrics.droppedExemplarsTotal) + registerer.MustRegister(d.metrics.droppedHistogramsTotal) + registerer.MustRegister(d.metrics.enqueueRetriesTotal) + registerer.MustRegister(d.metrics.sentBatchDuration) + registerer.MustRegister(d.metrics.highestSentTimestamp) + registerer.MustRegister(d.metrics.pendingSamples) + registerer.MustRegister(d.metrics.pendingExemplars) + registerer.MustRegister(d.metrics.pendingHistograms) + registerer.MustRegister(d.metrics.shardCapacity) + registerer.MustRegister(d.metrics.numShards) + registerer.MustRegister(d.metrics.maxNumShards) + registerer.MustRegister(d.metrics.minNumShards) + registerer.MustRegister(d.metrics.desiredNumShards) + registerer.MustRegister(d.metrics.bestNumShards) + registerer.MustRegister(d.metrics.sentBytesTotal) + registerer.MustRegister(d.metrics.metadataBytesTotal) + registerer.MustRegister(d.metrics.maxSamplesPerSend) + registerer.MustRegister(d.metrics.unexpectedEOFCount) + registerer.MustRegister(d.metrics.segmentSizeInBytes) +} + +func (d *Destination) UnregisterMetrics(registerer prometheus.Registerer) { + registerer.Unregister(d.metrics.samplesTotal) + registerer.Unregister(d.metrics.exemplarsTotal) + registerer.Unregister(d.metrics.histogramsTotal) + registerer.Unregister(d.metrics.metadataTotal) + registerer.Unregister(d.metrics.failedSamplesTotal) + registerer.Unregister(d.metrics.failedExemplarsTotal) + registerer.Unregister(d.metrics.failedHistogramsTotal) + registerer.Unregister(d.metrics.failedMetadataTotal) + registerer.Unregister(d.metrics.retriedSamplesTotal) + registerer.Unregister(d.metrics.retriedExemplarsTotal) + registerer.Unregister(d.metrics.retriedHistogramsTotal) + registerer.Unregister(d.metrics.retriedMetadataTotal) + registerer.Unregister(d.metrics.droppedSamplesTotal) + registerer.Unregister(d.metrics.addSeriesTotal) + registerer.Unregister(d.metrics.droppedSeriesTotal) + registerer.Unregister(d.metrics.droppedExemplarsTotal) + registerer.Unregister(d.metrics.droppedHistogramsTotal) + registerer.Unregister(d.metrics.enqueueRetriesTotal) + registerer.Unregister(d.metrics.sentBatchDuration) + registerer.Unregister(d.metrics.highestSentTimestamp) + registerer.Unregister(d.metrics.pendingSamples) + registerer.Unregister(d.metrics.pendingExemplars) + registerer.Unregister(d.metrics.pendingHistograms) + registerer.Unregister(d.metrics.shardCapacity) + registerer.Unregister(d.metrics.numShards) + registerer.Unregister(d.metrics.maxNumShards) + registerer.Unregister(d.metrics.minNumShards) + registerer.Unregister(d.metrics.desiredNumShards) + registerer.Unregister(d.metrics.bestNumShards) + registerer.Unregister(d.metrics.sentBytesTotal) + registerer.Unregister(d.metrics.metadataBytesTotal) + registerer.Unregister(d.metrics.maxSamplesPerSend) + registerer.Unregister(d.metrics.unexpectedEOFCount) + registerer.Unregister(d.metrics.segmentSizeInBytes) +} + +func remoteWriteConfigsAreEqual(lrwc, rwrc config.RemoteWriteConfig) bool { + ldata, _ := yaml.Marshal(lrwc) + rdata, _ := yaml.Marshal(rwrc) + return bytes.Equal(ldata, rdata) +} + +type maxTimestamp struct { + mtx sync.Mutex + value float64 + prometheus.Gauge +} + +func (m *maxTimestamp) Set(value float64) { + m.mtx.Lock() + defer m.mtx.Unlock() + if value > m.value { + m.value = value + m.Gauge.Set(value) + } +} + +func (m *maxTimestamp) Get() float64 { + m.mtx.Lock() + defer m.mtx.Unlock() + return m.value +} + +func (m *maxTimestamp) Collect(c chan<- prometheus.Metric) { + if m.Get() > 0 { + m.Gauge.Collect(c) + } +} + +type DestinationMetrics struct { + samplesTotal prometheus.Counter + exemplarsTotal prometheus.Counter + histogramsTotal prometheus.Counter + metadataTotal prometheus.Counter + failedSamplesTotal prometheus.Counter + failedExemplarsTotal prometheus.Counter + failedHistogramsTotal prometheus.Counter + failedMetadataTotal prometheus.Counter + retriedSamplesTotal prometheus.Counter + retriedExemplarsTotal prometheus.Counter + retriedHistogramsTotal prometheus.Counter + retriedMetadataTotal prometheus.Counter + droppedSamplesTotal *prometheus.CounterVec + addSeriesTotal prometheus.Counter + droppedSeriesTotal prometheus.Counter + droppedExemplarsTotal *prometheus.CounterVec + droppedHistogramsTotal *prometheus.CounterVec + enqueueRetriesTotal prometheus.Counter + sentBatchDuration prometheus.Histogram + highestSentTimestamp *maxTimestamp + pendingSamples prometheus.Gauge + pendingExemplars prometheus.Gauge + pendingHistograms prometheus.Gauge + shardCapacity prometheus.Gauge + numShards prometheus.Gauge + maxNumShards prometheus.Gauge + minNumShards prometheus.Gauge + desiredNumShards prometheus.Gauge + bestNumShards prometheus.Gauge + sentBytesTotal prometheus.Counter + metadataBytesTotal prometheus.Counter + maxSamplesPerSend prometheus.Gauge + unexpectedEOFCount prometheus.Counter + segmentSizeInBytes prometheus.Histogram +} diff --git a/pp/go/storage/remotewriter/errors.go b/pp/go/storage/remotewriter/errors.go new file mode 100644 index 0000000000..7f47e172d0 --- /dev/null +++ b/pp/go/storage/remotewriter/errors.go @@ -0,0 +1,26 @@ +package remotewriter + +import ( + "errors" + "io" +) + +var ( + // ErrShardIsCorrupted error when the shard file was corrupted. + ErrShardIsCorrupted = errors.New("shard is corrupted") + // ErrEndOfBlock error indicating the end of the block. + ErrEndOfBlock = errors.New("end of block") + // ErrEmptyReadResult an error indicating an empty reading result. + ErrEmptyReadResult = errors.New("empty read result") +) + +// CloseAll closes all given closers. +func CloseAll(closers ...io.Closer) error { + var err error + for _, closer := range closers { + if closer != nil { + err = errors.Join(err, closer.Close()) + } + } + return err +} diff --git a/pp/go/storage/remotewriter/iterator.go b/pp/go/storage/remotewriter/iterator.go new file mode 100644 index 0000000000..ac9c766f8e --- /dev/null +++ b/pp/go/storage/remotewriter/iterator.go @@ -0,0 +1,492 @@ +package remotewriter + +import ( + "context" + "errors" + "fmt" + "math" + "sync" + "time" + + "github.com/cenkalti/backoff/v4" + "github.com/jonboulle/clockwork" + + "github.com/prometheus/prometheus/config" + "github.com/prometheus/prometheus/pp/go/cppbridge" + "github.com/prometheus/prometheus/pp/go/storage/logger" + "github.com/prometheus/prometheus/storage/remote" +) + +type DataSource interface { + Read(ctx context.Context, targetSegmentID uint32, minTimestamp int64) ([]*DecodedSegment, error) + LSSes() []*cppbridge.LabelSetStorage + WriteCaches() + Close() error +} + +type TargetSegmentIDSetCloser interface { + SetTargetSegmentID(segmentID uint32) error + Close() error +} + +type ProtobufWriter interface { + Write(ctx context.Context, data *cppbridge.SnappyProtobufEncodedData) error +} + +type sharder struct { + min int + max int + numberOfShards int +} + +func newSharder(min, max int) (*sharder, error) { + if min > max || min <= 0 { + return nil, fmt.Errorf("failed to create sharder, min: %d, max: %d", min, max) + } + return &sharder{ + min: min, + max: max, + numberOfShards: min, + }, nil +} + +func (s *sharder) Apply(value float64) { + newValue := int(math.Ceil(value)) + if newValue < s.min { + newValue = s.min + } else if newValue > s.max { + newValue = s.max + } + + s.numberOfShards = newValue +} + +// BestNumberOfShards clamping value between min and max. +func (s *sharder) BestNumberOfShards(value float64) int { + newValue := int(math.Ceil(value)) + if newValue < s.min { + newValue = s.min + } else if newValue > s.max { + newValue = s.max + } + + return newValue +} + +func (s *sharder) NumberOfShards() int { + return s.numberOfShards +} + +type Iterator struct { + clock clockwork.Clock + queueConfig config.QueueConfig + dataSource DataSource + protobufWriter ProtobufWriter + targetSegmentIDSetCloser TargetSegmentIDSetCloser + segmentReadyChecker SegmentReadyChecker + metrics *DestinationMetrics + targetSegmentID uint32 + targetSegmentIsPartiallyRead bool + + outputSharder *sharder + + scrapeInterval time.Duration + endOfBlockReached bool +} + +type MessageShard struct { + Protobuf *cppbridge.SnappyProtobufEncodedData + Size uint64 + SampleCount uint64 + MaxTimestamp int64 + Delivered bool + PostProcessed bool +} + +type Message struct { + MaxTimestamp int64 + Shards []*MessageShard +} + +func (m *Message) HasDataToDeliver() bool { + for _, shrd := range m.Shards { + if !shrd.Delivered { + return true + } + } + return false +} + +func (m *Message) IsObsoleted(minTimestamp int64) bool { + return m.MaxTimestamp < minTimestamp +} + +func newIterator( + clock clockwork.Clock, + queueConfig config.QueueConfig, + dataSource DataSource, + targetSegmentIDSetCloser TargetSegmentIDSetCloser, + targetSegmentID uint32, + readTimeout time.Duration, + protobufWriter ProtobufWriter, + metrics *DestinationMetrics, +) (*Iterator, error) { + outputSharder, err := newSharder(queueConfig.MinShards, queueConfig.MaxShards) + if err != nil { + return nil, err + } + + return &Iterator{ + clock: clock, + queueConfig: queueConfig, + dataSource: dataSource, + protobufWriter: protobufWriter, + targetSegmentIDSetCloser: targetSegmentIDSetCloser, + metrics: metrics, + targetSegmentID: targetSegmentID, + scrapeInterval: readTimeout, + outputSharder: outputSharder, + }, nil +} + +func (i *Iterator) wrapError(err error) error { + if err != nil { + return err + } + + if i.endOfBlockReached { + return ErrEndOfBlock + } + + return nil +} + +func (i *Iterator) Next(ctx context.Context) error { + if i.endOfBlockReached { + return i.wrapError(nil) + } + + startTime := i.clock.Now() + var deadlineReached bool + var delay time.Duration + numberOfShards := i.outputSharder.NumberOfShards() + i.metrics.numShards.Set(float64(numberOfShards)) + b := newBatch(numberOfShards, i.queueConfig.MaxSamplesPerSend) + deadline := i.clock.After(i.scrapeInterval) + +readLoop: + for { + select { + case <-ctx.Done(): + return i.wrapError(ctx.Err()) + case <-deadline: + deadlineReached = true + break readLoop + case <-i.clock.After(delay): + } + + decodedSegments, err := i.dataSource.Read(ctx, i.targetSegmentID, i.minTimestamp()) + if err != nil { + if errors.Is(err, ErrEndOfBlock) { + i.endOfBlockReached = true + break readLoop + } + + if errors.Is(err, ErrEmptyReadResult) { + delay = defaultDelay + continue + } + + logger.Errorf("datasource read failed: %v", err) + delay = defaultDelay + continue + } + + b.add(decodedSegments) + i.targetSegmentID++ + i.targetSegmentIsPartiallyRead = false + + if b.IsFilled() { + break readLoop + } + + delay = 0 + } + + readDuration := i.clock.Since(startTime) + + if b.HasDroppedSamples() { + i.metrics.droppedSamplesTotal.WithLabelValues(reasonTooOld).Add(float64(b.OutdatedSamplesCount())) + i.metrics.droppedSamplesTotal.WithLabelValues(reasonDroppedSeries).Add(float64(b.DroppedSamplesCount())) + } + + i.metrics.droppedSeriesTotal.Add(float64(b.DroppedSeriesCount())) + + if b.IsEmpty() { + return i.wrapError(nil) + } + + i.metrics.addSeriesTotal.Add(float64(b.AddSeriesCount())) + + var desiredNumberOfShards float64 + if deadlineReached { + desiredNumberOfShards = math.Ceil( + float64(b.NumberOfSamples()) / float64(b.MaxNumberOfSamplesPerShard()) * float64(numberOfShards), + ) + } else { + desiredNumberOfShards = math.Ceil(float64(i.scrapeInterval) / float64(readDuration) * float64(numberOfShards)) + } + + bestNumberOfShards := i.outputSharder.BestNumberOfShards( + float64(b.NumberOfSamples()) / float64(b.MaxNumberOfSamplesPerShard()), + ) + + i.outputSharder.Apply(desiredNumberOfShards) + i.metrics.desiredNumShards.Set(desiredNumberOfShards) + i.metrics.bestNumShards.Set(float64(bestNumberOfShards)) + + i.writeCaches() + + msg, err := i.encode(b.Data(), uint16(bestNumberOfShards)) + if err != nil { + return i.wrapError(err) + } + + numberOfSamples := b.NumberOfSamples() + + b = nil + + sendIteration := 0 + err = backoff.Retry(func() error { + defer func() { sendIteration++ }() + if msg.IsObsoleted(i.minTimestamp()) { + for _, messageShard := range msg.Shards { + if messageShard.Delivered { + continue + } + i.metrics.droppedSamplesTotal.WithLabelValues(reasonTooOld).Add(float64(messageShard.SampleCount)) + } + return nil + } + + i.metrics.samplesTotal.Add(float64(numberOfSamples)) + + wg := &sync.WaitGroup{} + for _, shrd := range msg.Shards { + if shrd.Delivered { + continue + } + wg.Add(1) + go func(shrd *MessageShard) { + defer wg.Done() + begin := i.clock.Now() + writeErr := i.protobufWriter.Write(ctx, shrd.Protobuf) + i.metrics.sentBatchDuration.Observe(i.clock.Since(begin).Seconds()) + if writeErr != nil { + logger.Errorf("failed to send protobuf: %v", writeErr) + } + + shrd.Delivered = !errors.Is(writeErr, remote.RecoverableError{}) + }(shrd) + } + wg.Wait() + + var failedSamplesTotal uint64 + var sentBytesTotal uint64 + var highestSentTimestamp int64 + var retriedSamplesTotal uint64 + for _, shrd := range msg.Shards { + if shrd.Delivered { + if shrd.PostProcessed { + continue + } + // delivered on this iteration + shrd.PostProcessed = true + retriedSamplesTotal += shrd.SampleCount + sentBytesTotal += shrd.Size + if highestSentTimestamp < shrd.MaxTimestamp { + highestSentTimestamp = shrd.MaxTimestamp + } + continue + } + // delivery failed bool + retriedSamplesTotal += shrd.SampleCount + failedSamplesTotal += shrd.SampleCount + } + + i.metrics.failedSamplesTotal.Add(float64(failedSamplesTotal)) + i.metrics.sentBytesTotal.Add(float64(sentBytesTotal)) + i.metrics.highestSentTimestamp.Set(float64(highestSentTimestamp)) + + if sendIteration > 0 { + i.metrics.retriedSamplesTotal.Add(float64(retriedSamplesTotal)) + } + + if msg.HasDataToDeliver() { + return errors.New("not all data delivered") + } + + return nil + }, + backoff.WithContext( + backoff.NewExponentialBackOff( + backoff.WithClockProvider(i.clock), + backoff.WithMaxElapsedTime(0), + backoff.WithMaxInterval(i.scrapeInterval), + ), + ctx, + ), + ) + if err != nil { + return i.wrapError(err) + } + + if err = i.tryAck(ctx); err != nil { + logger.Errorf("failed to ack segment id: %v", err) + } + + return i.wrapError(nil) +} + +func (i *Iterator) writeCaches() { + i.dataSource.WriteCaches() +} + +func (i *Iterator) encode(segments []*DecodedSegment, numberOfShards uint16) (*Message, error) { + var maxTimestamp int64 + batchToEncode := make([]*cppbridge.DecodedRefSamples, 0, len(segments)) + for _, segment := range segments { + if maxTimestamp < segment.MaxTimestamp { + maxTimestamp = segment.MaxTimestamp + } + + batchToEncode = append(batchToEncode, segment.Samples) + } + + protobufEncoder := cppbridge.NewWALProtobufEncoder(i.dataSource.LSSes()) + protobufs, err := protobufEncoder.Encode(batchToEncode, numberOfShards) + if err != nil { + return nil, fmt.Errorf("failed to encode protobuf: %w", err) + } + shards := make([]*MessageShard, 0, len(protobufs)) + for _, protobuf := range protobufs { + proto := protobuf + var size uint64 + _ = proto.Do(func(buf []byte) error { + size = uint64(len(buf)) + return nil + }) + shards = append(shards, &MessageShard{ + Protobuf: protobuf, + Size: size, + SampleCount: protobuf.SamplesCount(), + MaxTimestamp: protobuf.MaxTimestamp(), + Delivered: false, + }) + } + return &Message{ + MaxTimestamp: maxTimestamp, + Shards: shards, + }, nil +} + +func (i *Iterator) tryAck(_ context.Context) error { + if i.targetSegmentID == 0 && i.targetSegmentIsPartiallyRead { + return nil + } + + targetSegmentID := i.targetSegmentID + if i.targetSegmentIsPartiallyRead { + targetSegmentID-- + } + + if err := i.targetSegmentIDSetCloser.SetTargetSegmentID(targetSegmentID); err != nil { + return fmt.Errorf("failed to set target segment id: %w", err) + } + + return nil +} + +func (i *Iterator) minTimestamp() int64 { + sampleAgeLimit := time.Duration(i.queueConfig.SampleAgeLimit) + return i.clock.Now().Add(-sampleAgeLimit).UnixMilli() +} + +func (i *Iterator) Close() error { + return errors.Join(i.dataSource.Close(), i.targetSegmentIDSetCloser.Close()) +} + +type batch struct { + segments []*DecodedSegment + numberOfShards int + numberOfSamples int + outdatedSamplesCount uint32 + droppedSamplesCount uint32 + addSeriesCount uint32 + droppedSeriesCount uint32 + maxNumberOfSamplesPerShard int +} + +func newBatch(numberOfShards int, maxNumberOfSamplesPerShard int) *batch { + return &batch{ + numberOfShards: numberOfShards, + maxNumberOfSamplesPerShard: maxNumberOfSamplesPerShard, + } +} + +func (b *batch) add(segments []*DecodedSegment) { + for _, segment := range segments { + b.segments = append(b.segments, segment) + b.numberOfSamples += segment.Samples.Size() + b.outdatedSamplesCount += segment.OutdatedSamplesCount + b.droppedSamplesCount += segment.DroppedSamplesCount + b.addSeriesCount += segment.AddSeriesCount + b.droppedSeriesCount += segment.DroppedSeriesCount + } +} + +func (b *batch) IsFilled() bool { + return b.numberOfSamples > b.numberOfShards*b.maxNumberOfSamplesPerShard +} + +func (b *batch) IsEmpty() bool { + return b.numberOfSamples == 0 +} + +func (b *batch) HasDroppedSamples() bool { + return b.droppedSamplesCount > 0 || b.outdatedSamplesCount > 0 +} + +func (b *batch) OutdatedSamplesCount() uint32 { + return b.outdatedSamplesCount +} + +func (b *batch) DroppedSamplesCount() uint32 { + return b.droppedSamplesCount +} + +// AddSeriesCount number of add series. +func (b *batch) AddSeriesCount() uint32 { + return b.addSeriesCount +} + +// DroppedSeriesCount number of dropped series. +func (b *batch) DroppedSeriesCount() uint32 { + return b.droppedSeriesCount +} + +func (b *batch) NumberOfSamples() int { + return b.numberOfSamples +} + +func (b *batch) MaxNumberOfSamplesPerShard() int { + return b.maxNumberOfSamplesPerShard +} + +func (b *batch) NumberOfShards() int { + return b.numberOfShards +} + +func (b *batch) Data() []*DecodedSegment { + return b.segments +} diff --git a/pp/go/storage/remotewriter/remotewriter.go b/pp/go/storage/remotewriter/remotewriter.go new file mode 100644 index 0000000000..a1e375ad53 --- /dev/null +++ b/pp/go/storage/remotewriter/remotewriter.go @@ -0,0 +1,137 @@ +package remotewriter + +import ( + "context" + "fmt" + "time" + + "github.com/jonboulle/clockwork" + "github.com/prometheus/client_golang/prometheus" + + "github.com/prometheus/prometheus/pp/go/storage/catalog" + "github.com/prometheus/prometheus/pp/go/storage/ready" +) + +type Catalog interface { + List(filterFn func(record *catalog.Record) bool, sortLess func(lhs, rhs *catalog.Record) bool) (records []*catalog.Record, err error) + SetCorrupted(id string) (*catalog.Record, error) +} + +type RemoteWriter struct { + dataDir string + configQueue chan []DestinationConfig + catalog Catalog + clock clockwork.Clock + readyNotifier ready.Notifier + registerer prometheus.Registerer +} + +func New(dataDir string, catalog Catalog, clock clockwork.Clock, readyNotifier ready.Notifier, registerer prometheus.Registerer) *RemoteWriter { + return &RemoteWriter{ + dataDir: dataDir, + catalog: catalog, + clock: clock, + configQueue: make(chan []DestinationConfig), + readyNotifier: readyNotifier, + registerer: registerer, + } +} + +func (rw *RemoteWriter) Run(ctx context.Context) error { + writeLoopRunners := make(map[string]*writeLoopRunner) + defer func() { + for _, wlr := range writeLoopRunners { + wlr.stop() + } + }() + + destinations := make(map[string]*Destination) + + for { + select { + case <-ctx.Done(): + return nil + case configs := <-rw.configQueue: + destinationConfigs := make(map[string]DestinationConfig) + for _, destinationConfig := range configs { + destinationConfigs[destinationConfig.Name] = destinationConfig + } + + for _, destination := range destinations { + name := destination.Config().Name + if _, ok := destinationConfigs[name]; !ok { + wlr := writeLoopRunners[name] + wlr.stop() + destination.UnregisterMetrics(rw.registerer) + delete(destinations, name) + delete(writeLoopRunners, name) + } + } + + for _, config := range configs { + destination, ok := destinations[config.Name] + if !ok { + destination = NewDestination(config) + destination.RegisterMetrics(rw.registerer) + wl := newWriteLoop(rw.dataDir, destination, rw.catalog, rw.clock) + wlr := newWriteLoopRunner(wl) + writeLoopRunners[config.Name] = wlr + destinations[config.Name] = destination + go wlr.run(ctx) + continue + } + + if config.EqualTo(destination.Config()) { + continue + } + + wlr := writeLoopRunners[config.Name] + wlr.stop() + destination.ResetConfig(config) + wl := newWriteLoop(rw.dataDir, destination, rw.catalog, rw.clock) + wlr = newWriteLoopRunner(wl) + writeLoopRunners[config.Name] = wlr + go wlr.run(ctx) + } + rw.readyNotifier.NotifyReady() + } + } +} + +func (rw *RemoteWriter) ApplyConfig(configs ...DestinationConfig) (err error) { + select { + case rw.configQueue <- configs: + return nil + case <-time.After(time.Minute): + return fmt.Errorf("failed to apply remote write configs, timeout") + } +} + +type writeLoopRunner struct { + wl *writeLoop + stopc chan struct{} + stoppedc chan struct{} +} + +func newWriteLoopRunner(wl *writeLoop) *writeLoopRunner { + return &writeLoopRunner{ + wl: wl, + stopc: make(chan struct{}), + stoppedc: make(chan struct{}), + } +} + +func (r *writeLoopRunner) run(ctx context.Context) { + ctx, cancel := context.WithCancel(ctx) + go func() { + <-r.stopc + cancel() + }() + r.wl.run(ctx) + close(r.stoppedc) +} + +func (r *writeLoopRunner) stop() { + close(r.stopc) + <-r.stoppedc +} diff --git a/pp/go/storage/remotewriter/remotewriter_test.go b/pp/go/storage/remotewriter/remotewriter_test.go new file mode 100644 index 0000000000..24d67b2519 --- /dev/null +++ b/pp/go/storage/remotewriter/remotewriter_test.go @@ -0,0 +1,15 @@ +package remotewriter + +import ( + "testing" + + "github.com/jonboulle/clockwork" + "github.com/prometheus/client_golang/prometheus" + + "github.com/prometheus/prometheus/pp/go/storage/ready" +) + +func TestRemoteWriter_Run(t *testing.T) { + rw := New("", nil, clockwork.NewFakeClock(), ready.NoOpNotifier{}, prometheus.DefaultRegisterer) + _ = rw +} diff --git a/pp/go/storage/remotewriter/walreader.go b/pp/go/storage/remotewriter/walreader.go new file mode 100644 index 0000000000..19fca78d9f --- /dev/null +++ b/pp/go/storage/remotewriter/walreader.go @@ -0,0 +1,60 @@ +package remotewriter + +import ( + "bufio" + "errors" + "fmt" + "io" + "os" + + "github.com/prometheus/prometheus/pp/go/relabeler/head" +) + +type walReader struct { + nextSegmentID uint32 + file *os.File + reader io.Reader +} + +func newWalReader(fileName string) (*walReader, uint8, error) { + file, err := os.Open(fileName) + if err != nil { + return nil, 0, fmt.Errorf("failed to read wal file: %w", err) + } + + _, encoderVersion, _, err := head.ReadHeader(file) + if err != nil { + return nil, 0, errors.Join(fmt.Errorf("failed to read header: %w", err), file.Close()) + } + + return &walReader{ + file: file, + reader: bufio.NewReaderSize(file, 4096), + }, encoderVersion, nil +} + +// Close wal file. +func (r *walReader) Close() error { + return r.file.Close() +} + +// Read [Segment] from wal and return. +func (r *walReader) Read() (segment Segment, err error) { + decodedSegment, _, err := head.ReadSegment(r.reader) + if err != nil { + return segment, fmt.Errorf("failed to read segment: %w", err) + } + + segment.ID = r.nextSegmentID + r.nextSegmentID++ + segment.DecodedSegment = decodedSegment + + return segment, nil +} + +// Segment encoded segment from wal. +type Segment struct { + ID uint32 + encoderVersion uint8 + head.DecodedSegment +} diff --git a/pp/go/storage/remotewriter/writeloop.go b/pp/go/storage/remotewriter/writeloop.go new file mode 100644 index 0000000000..6d80af45f2 --- /dev/null +++ b/pp/go/storage/remotewriter/writeloop.go @@ -0,0 +1,394 @@ +package remotewriter + +import ( + "context" + "errors" + "fmt" + "os" + "path/filepath" + "time" + + "github.com/jonboulle/clockwork" + "github.com/prometheus/prometheus/storage/remote" + + "github.com/prometheus/prometheus/pp/go/cppbridge" + "github.com/prometheus/prometheus/pp/go/storage/catalog" + "github.com/prometheus/prometheus/pp/go/storage/logger" +) + +const defaultDelay = time.Second * 5 + +type writeLoop struct { + dataDir string + destination *Destination + currentHeadID *string + catalog Catalog + clock clockwork.Clock + client remote.WriteClient +} + +func newWriteLoop(dataDir string, destination *Destination, catalog Catalog, clock clockwork.Clock) *writeLoop { + return &writeLoop{ + dataDir: dataDir, + destination: destination, + catalog: catalog, + clock: clock, + } +} + +func (wl *writeLoop) run(ctx context.Context) { + var delay time.Duration + var err error + var i *Iterator + var nextI *Iterator + + rw := &readyProtobufWriter{} + + wl.destination.metrics.maxNumShards.Set(float64(wl.destination.Config().QueueConfig.MaxShards)) + wl.destination.metrics.minNumShards.Set(float64(wl.destination.Config().QueueConfig.MinShards)) + + defer func() { + if i != nil { + _ = i.Close() + } + if nextI != nil { + _ = nextI.Close() + } + }() + + for { + select { + case <-ctx.Done(): + return + case <-wl.clock.After(delay): + delay = 0 + } + + if i == nil { + if nextI != nil { + i = nextI + nextI = nil + } else { + i, err = wl.nextIterator(ctx, rw) + if err != nil { + logger.Errorf("get current next iterator: %v", err) + delay = defaultDelay + continue + } + } + } + + if wl.client == nil { + wl.client, err = createClient(wl.destination.Config()) + if err != nil { + logger.Errorf("create client: %v", err) + delay = defaultDelay + continue + } + + rw.SetProtobufWriter(newProtobufWriter(wl.client)) + } + + if err = wl.write(ctx, i); err != nil { + logger.Errorf("iterator write: %v", err) + delay = defaultDelay + continue + } + + if nextI == nil { + nextI, err = wl.nextIterator(ctx, rw) + if err != nil { + logger.Errorf("get next iterator: %v", err) + delay = defaultDelay + continue + } + } + + if err = i.Close(); err != nil { + logger.Errorf("close iterator: %v", err) + delay = defaultDelay + continue + } + + i = nil + } +} + +func createClient(config DestinationConfig) (client remote.WriteClient, err error) { + clientConfig := remote.ClientConfig{ + URL: config.URL, + Timeout: config.RemoteTimeout, + HTTPClientConfig: config.HTTPClientConfig, + SigV4Config: config.SigV4Config, + AzureADConfig: config.AzureADConfig, + Headers: config.Headers, + RetryOnRateLimit: true, + } + + client, err = remote.NewWriteClient(config.Name, &clientConfig) + if err != nil { + return nil, fmt.Errorf("falied to create client: %w", err) + } + + return client, nil +} + +func (wl *writeLoop) write(ctx context.Context, iterator *Iterator) error { + for { + select { + case <-ctx.Done(): + return ctx.Err() + default: + err := iterator.Next(ctx) + if err != nil { + if errors.Is(err, ErrEndOfBlock) { + return nil + } + logger.Errorf("iteration failed: %v", err) + } + } + } +} + +func (wl *writeLoop) nextIterator(ctx context.Context, protobufWriter ProtobufWriter) (*Iterator, error) { + var nextHeadRecord *catalog.Record + var err error + var cleanStart bool + if wl.currentHeadID != nil { + nextHeadRecord, err = nextHead(ctx, wl.dataDir, wl.catalog, *wl.currentHeadID) + } else { + var headFound bool + nextHeadRecord, headFound, err = scanForNextHead(ctx, wl.dataDir, wl.catalog, wl.destination.Config().Name) + cleanStart = !headFound + } + if err != nil { + return nil, fmt.Errorf("find next head: %w", err) + } + headDir := filepath.Join(wl.dataDir, nextHeadRecord.Dir()) + crw, err := NewCursorReadWriter(filepath.Join(headDir, fmt.Sprintf("%s.cursor", wl.destination.Config().Name)), nextHeadRecord.NumberOfShards()) + if err != nil { + return nil, fmt.Errorf("create cursor: %w", err) + } + + crc32, err := wl.destination.Config().CRC32() + if err != nil { + return nil, errors.Join(fmt.Errorf("calculate crc32: %w", err), crw.Close()) + } + + var discardCache bool + if crw.GetConfigCRC32() != crc32 { + if err = crw.SetConfigCRC32(crc32); err != nil { + return nil, errors.Join(fmt.Errorf("write crc32: %w", err), crw.Close()) + } + discardCache = true + } + + ds, err := newDataSource( + headDir, + nextHeadRecord.NumberOfShards(), + wl.destination.Config(), + discardCache, + newSegmentReadyChecker(nextHeadRecord), + wl.makeCorruptMarker(), + nextHeadRecord, + wl.destination.metrics.unexpectedEOFCount, + wl.destination.metrics.segmentSizeInBytes, + ) + if err != nil { + return nil, errors.Join(fmt.Errorf("create data source: %w", err), crw.Close()) + } + + headID := nextHeadRecord.ID() + ds.ID = headID + + var targetSegmentID uint32 + if cleanStart { + if nextHeadRecord.LastAppendedSegmentID() != nil { + targetSegmentID = *nextHeadRecord.LastAppendedSegmentID() + } else { + targetSegmentID = crw.GetTargetSegmentID() + } + } else { + targetSegmentID = crw.GetTargetSegmentID() + } + + i, err := newIterator( + wl.clock, + wl.destination.Config().QueueConfig, + ds, + crw, + targetSegmentID, + wl.destination.Config().ReadTimeout, + protobufWriter, + wl.destination.metrics, + ) + if err != nil { + return nil, errors.Join(fmt.Errorf("create data source: %w", err), crw.Close(), ds.Close()) + } + + wl.currentHeadID = &headID + + return i, nil +} + +// CorruptMarkerFn func for mark head as corrupted by ID. +type CorruptMarkerFn func(headID string) error + +// MarkCorrupted mark head as corrupted by ID. +func (fn CorruptMarkerFn) MarkCorrupted(headID string) error { + return fn(headID) +} + +// makeCorruptMarker set marker on head is corrupted. +func (wl *writeLoop) makeCorruptMarker() CorruptMarker { + return CorruptMarkerFn(func(headID string) error { + _, err := wl.catalog.SetCorrupted(headID) + return err + }) +} + +func nextHead(ctx context.Context, dataDir string, headCatalog Catalog, headID string) (*catalog.Record, error) { + if err := contextErr(ctx); err != nil { + return nil, err + } + + headRecords, err := headCatalog.List( + nil, + func(lhs, rhs *catalog.Record) bool { + return lhs.CreatedAt() < rhs.CreatedAt() + }, + ) + if err != nil { + return nil, fmt.Errorf("list head records: %w", err) + } + + if len(headRecords) == 0 { + return nil, fmt.Errorf("nextHead: no new heads: empty head records") + } + + currentHeadFound := false + for _, headRecord := range headRecords { + if headRecord.ID() == headID { + currentHeadFound = true + continue + } + + if !currentHeadFound { + continue + } + + if err = validateHead(ctx, filepath.Join(dataDir, headRecord.Dir())); err != nil { + if !errors.Is(err, os.ErrNotExist) { + return nil, err + } + + switch headRecord.Status() { + case catalog.StatusNew, catalog.StatusActive: + return nil, fmt.Errorf("validate active head: %w", err) + default: + continue + } + } + + return headRecord, nil + } + + // unknown head id, selecting last head + if !currentHeadFound { + return headRecords[len(headRecords)-1], nil + } + + return nil, fmt.Errorf("nextHead: no new heads: appropriate head not found") +} + +func validateHead(ctx context.Context, headDir string) error { + dir, err := os.Open(headDir) + if err != nil { + return err + } + + return dir.Close() +} + +func scanForNextHead(ctx context.Context, dataDir string, headCatalog Catalog, destinationName string) (*catalog.Record, bool, error) { + if err := contextErr(ctx); err != nil { + return nil, false, err + } + + headRecords, err := headCatalog.List( + nil, + func(lhs, rhs *catalog.Record) bool { + return lhs.CreatedAt() > rhs.CreatedAt() + }, + ) + if err != nil { + return nil, false, fmt.Errorf("list head records: %w", err) + } + + if len(headRecords) == 0 { + return nil, false, fmt.Errorf("scanForNextHead: no new heads: empty head records") + } + + var headFound bool + for _, headRecord := range headRecords { + headFound, err = scanHeadForDestination(filepath.Join(dataDir, headRecord.Dir()), destinationName) + if err != nil { + if !headRecord.Corrupted() { + logger.Errorf("head %s is corrupted: %v", headRecord.ID(), err) + if _, corruptErr := headCatalog.SetCorrupted(headRecord.ID()); corruptErr != nil { + logger.Errorf("set corrupted state: %v", corruptErr) + } + } + + continue + } + if headFound { + return headRecord, true, nil + } + } + + // track of the previous destination not found, selecting last head + return headRecords[0], false, nil +} + +func scanHeadForDestination(dirPath string, destinationName string) (bool, error) { + dir, err := os.Open(dirPath) + if err != nil { + return false, fmt.Errorf("open head dir: %w", err) + } + defer func() { _ = dir.Close() }() + + fileNames, err := dir.Readdirnames(-1) + if err != nil { + return false, fmt.Errorf("read dir names: %w", err) + } + + for _, fileName := range fileNames { + if fileName == fmt.Sprintf("%s.cursor", destinationName) { + return true, nil + } + } + + return false, nil +} + +func contextErr(ctx context.Context) error { + select { + case <-ctx.Done(): + return ctx.Err() + default: + return nil + } +} + +type readyProtobufWriter struct { + protobufWriter ProtobufWriter +} + +func (rpw *readyProtobufWriter) SetProtobufWriter(protobufWriter ProtobufWriter) { + rpw.protobufWriter = protobufWriter +} + +func (rw *readyProtobufWriter) Write(ctx context.Context, protobuf *cppbridge.SnappyProtobufEncodedData) error { + return rw.protobufWriter.Write(ctx, protobuf) +} diff --git a/pp/go/storage/remotewriter/writeloop_test.go b/pp/go/storage/remotewriter/writeloop_test.go new file mode 100644 index 0000000000..930d56d20d --- /dev/null +++ b/pp/go/storage/remotewriter/writeloop_test.go @@ -0,0 +1,263 @@ +package remotewriter + +// import ( +// "context" +// "errors" +// "net/url" +// "os" +// "path/filepath" +// "sync" +// "testing" +// "time" + +// "github.com/golang/snappy" +// "github.com/jonboulle/clockwork" +// "github.com/prometheus/client_golang/prometheus" +// config3 "github.com/prometheus/common/config" +// model2 "github.com/prometheus/common/model" +// config2 "github.com/prometheus/prometheus/config" +// "github.com/prometheus/prometheus/model/labels" +// "github.com/prometheus/prometheus/pp/go/cppbridge" +// "github.com/prometheus/prometheus/pp/go/model" +// "github.com/prometheus/prometheus/pp/go/relabeler" +// "github.com/prometheus/prometheus/pp/go/relabeler/appender" +// "github.com/prometheus/prometheus/pp/go/relabeler/config" +// "github.com/prometheus/prometheus/pp/go/relabeler/head/catalog" +// "github.com/prometheus/prometheus/pp/go/relabeler/head/manager" +// "github.com/prometheus/prometheus/prompb" +// "github.com/stretchr/testify/require" +// ) + +// const transparentRelabelerName = "transparent_relabeler" + +// type ConfigSource struct { +// inputRelabelConfigs []*config.InputRelabelerConfig +// numberOfShards uint16 +// } + +// func (s *ConfigSource) Set(inputRelabelConfigs []*config.InputRelabelerConfig, numberOfShards uint16) { +// s.inputRelabelConfigs = inputRelabelConfigs +// s.numberOfShards = numberOfShards +// } + +// func (s *ConfigSource) Get() (inputRelabelConfigs []*config.InputRelabelerConfig, numberOfShards uint16) { +// return s.inputRelabelConfigs, s.numberOfShards +// } + +// type NoOpStorage struct { +// heads []relabeler.Head +// } + +// func (s *NoOpStorage) Add(head relabeler.Head) { +// if err := head.Rotate(); err != nil { +// panic(err) +// } +// s.heads = append(s.heads, head) +// } + +// func (s *NoOpStorage) Close() error { +// var err error +// for _, head := range s.heads { +// err = errors.Join(err, head.Close()) +// } +// s.heads = nil +// return err +// } + +// type TestHeads struct { +// Dir string +// FileLog *catalog.FileLog +// Catalog *catalog.Catalog +// ConfigSource *ConfigSource +// Manager *manager.Manager +// NumberOfShards uint8 +// Storage *NoOpStorage +// Head relabeler.Head +// } + +// func NewTestHeads(dir string, inputRelabelConfigs []*config.InputRelabelerConfig, numberOfShards uint16, clock clockwork.Clock) (*TestHeads, error) { +// th := &TestHeads{ +// Dir: dir, +// } +// var err error +// th.FileLog, err = catalog.NewFileLogV2(filepath.Join(dir, "catalog")) +// if err != nil { +// return nil, err +// } + +// th.Catalog, err = catalog.New(clock, th.FileLog, catalog.DefaultIDGenerator{}, catalog.DefaultMaxLogFileSize, nil) +// if err != nil { +// return nil, errors.Join(err, th.Close()) +// } + +// th.ConfigSource = &ConfigSource{ +// inputRelabelConfigs: inputRelabelConfigs, +// numberOfShards: numberOfShards, +// } + +// th.Manager, err = manager.New(dir, clock, th.ConfigSource, th.Catalog, 0, prometheus.DefaultRegisterer) +// if err != nil { +// return nil, errors.Join(err, th.Close()) +// } + +// activeHead, _, err := th.Manager.Restore(time.Hour) +// if err != nil { +// return nil, errors.Join(err, th.Close()) +// } + +// th.Storage = &NoOpStorage{} +// th.Head = appender.NewRotatableHead(activeHead, th.Storage, th.Manager, appender.NoOpHeadActivator{}) + +// return th, nil +// } + +// func (th *TestHeads) Append(ctx context.Context, timeSeriesSlice []model.TimeSeries, relabelerID string) error { +// hx, err := cppbridge.HashdexFactory{}.GoModel(timeSeriesSlice, cppbridge.DefaultWALHashdexLimits()) +// if err != nil { +// return err +// } + +// _, _, err = th.Head.Append(ctx, &relabeler.IncomingData{Hashdex: hx}, nil, relabelerID, true) +// return err +// } + +// func (th *TestHeads) Rotate() error { +// return th.Head.Rotate() +// } + +// func (th *TestHeads) Close() error { +// return errors.Join(th.Storage.Close(), th.Head.Close(), th.FileLog.Close()) +// } + +// type remoteClient struct { +// mtx sync.Mutex +// data [][]byte +// name string +// } + +// func (c *remoteClient) Store(_ context.Context, bytes []byte, _ int) error { +// c.mtx.Lock() +// defer c.mtx.Unlock() +// c.data = append(c.data, bytes) +// return nil +// } + +// func (c *remoteClient) Name() string { +// return c.name +// } + +// func (c *remoteClient) Endpoint() string { +// return "" +// } + +// type testWriter struct { +// mtx sync.Mutex +// data []*cppbridge.SnappyProtobufEncodedData +// } + +// func (w *testWriter) Write(ctx context.Context, protobuf *cppbridge.SnappyProtobufEncodedData) error { +// w.mtx.Lock() +// defer w.mtx.Unlock() +// w.data = append(w.data, protobuf) +// return ctx.Err() +// } + +// func TestWriteLoopWrite(t *testing.T) { +// tmpDir, err := os.MkdirTemp("", "write_loop_iterate_test") +// require.NoError(t, err) +// defer func() { _ = os.RemoveAll(tmpDir) }() + +// clock := clockwork.NewRealClock() +// cfgs := []*config.InputRelabelerConfig{ +// { +// Name: transparentRelabelerName, +// RelabelConfigs: []*cppbridge.RelabelConfig{ +// { +// SourceLabels: []string{"__name__"}, +// Regex: ".*", +// Action: cppbridge.Keep, +// }, +// }, +// }, +// } +// var numberOfShards uint16 = 2 + +// testHeads, err := NewTestHeads(tmpDir, cfgs, numberOfShards, clock) +// require.NoError(t, err) + +// ctx, cancel := context.WithTimeout(context.Background(), time.Minute*5) +// defer cancel() + +// labelSets := []model.LabelSet{ +// model.NewLabelSetBuilder().Set("__name__", "test_metric_0").Build(), +// model.NewLabelSetBuilder().Set("__name__", "test_metric_1").Build(), +// model.NewLabelSetBuilder().Set("__name__", "test_metric_2").Build(), +// model.NewLabelSetBuilder().Set("__name__", "test_metric_3").Build(), +// } + +// ts := clock.Now().UnixMilli() +// batches := [][]model.TimeSeries{ +// { +// {LabelSet: labelSets[0], Timestamp: uint64(ts), Value: 0}, +// {LabelSet: labelSets[1], Timestamp: uint64(ts), Value: 1000}, +// {LabelSet: labelSets[2], Timestamp: uint64(ts), Value: 1000000}, +// {LabelSet: labelSets[3], Timestamp: uint64(ts), Value: 1000000000}, +// }, +// } + +// err = testHeads.Append(ctx, batches[0], transparentRelabelerName) +// require.NoError(t, err) + +// u, err := url.Parse("http://localhost:8080") +// require.NoError(t, err) + +// destination := NewDestination(DestinationConfig{ +// RemoteWriteConfig: config2.RemoteWriteConfig{ +// URL: &config3.URL{u}, +// RemoteTimeout: 0, +// Headers: nil, +// WriteRelabelConfigs: nil, +// Name: "remote_write_0", +// SendExemplars: false, +// SendNativeHistograms: false, +// HTTPClientConfig: config3.HTTPClientConfig{}, +// QueueConfig: config2.QueueConfig{ +// MaxSamplesPerSend: 2, +// MinShards: 3, +// MaxShards: 5, +// SampleAgeLimit: model2.Duration(time.Hour), +// }, +// MetadataConfig: config2.MetadataConfig{}, +// SigV4Config: nil, +// AzureADConfig: nil, +// }, +// ExternalLabels: labels.FromStrings("lol", "kek"), +// ReadTimeout: time.Second * 3, +// }) + +// wl := newWriteLoop(tmpDir, destination, testHeads.Catalog, clock) +// w := &testWriter{} +// i, err := wl.nextIterator(ctx, w) +// require.NoError(t, err) + +// require.NoError(t, i.Next(ctx)) +// require.NoError(t, err) + +// require.NoError(t, testHeads.Rotate()) + +// require.ErrorIs(t, i.Next(ctx), ErrEndOfBlock) + +// for _, data := range w.data { +// wr := prompb.WriteRequest{} +// err = data.Do(func(buf []byte) error { +// var decoded []byte +// decoded, err = snappy.Decode(nil, buf) +// if err != nil { +// return err +// } +// return wr.Unmarshal(decoded) +// }) +// require.NoError(t, err) +// t.Log(wr.String()) +// } +// } diff --git a/pp/go/storage/remotewriter/writer.go b/pp/go/storage/remotewriter/writer.go new file mode 100644 index 0000000000..e47e8fa764 --- /dev/null +++ b/pp/go/storage/remotewriter/writer.go @@ -0,0 +1,35 @@ +package remotewriter + +import ( + "context" + + "github.com/prometheus/prometheus/pp/go/cppbridge" + "github.com/prometheus/prometheus/storage/remote" +) + +// protobufWriter +type protobufWriter struct { + client remote.WriteClient +} + +func newProtobufWriter(client remote.WriteClient) *protobufWriter { + return &protobufWriter{ + client: client, + } +} + +func (w *protobufWriter) Write(ctx context.Context, protobuf *cppbridge.SnappyProtobufEncodedData) error { + return protobuf.Do(func(buf []byte) error { + if len(buf) == 0 { + return nil + } + + // TODO WriteResponseStats + _, err := w.client.Store(ctx, buf, 0) + return err + }) +} + +func (w *protobufWriter) Close() error { + return nil +} diff --git a/pp/go/storage/types.go b/pp/go/storage/types.go index 862fcedb61..9ed68db609 100644 --- a/pp/go/storage/types.go +++ b/pp/go/storage/types.go @@ -3,6 +3,7 @@ package storage import ( "github.com/prometheus/prometheus/pp/go/cppbridge" "github.com/prometheus/prometheus/pp/go/storage/head/head" + "github.com/prometheus/prometheus/pp/go/storage/head/manager" "github.com/prometheus/prometheus/pp/go/storage/head/shard" "github.com/prometheus/prometheus/pp/go/storage/head/shard/wal" "github.com/prometheus/prometheus/pp/go/storage/head/shard/wal/writer" @@ -20,3 +21,6 @@ type ShardOnDisk = shard.Shard[*WalOnDisk] // HeadOnDisk [head.Head] with [ShardOnDisk]. type HeadOnDisk = head.Head[*ShardOnDisk, *shard.PerGoroutineShard[*WalOnDisk]] + +// HeadManager [manager.Manager] for [HeadOnDisk]s. +type HeadManager = manager.Manager[*HeadOnDisk] From 76d142026babded2b6ed1491b192e7f383071127 Mon Sep 17 00:00:00 2001 From: Alexandr Yudin Date: Mon, 1 Sep 2025 15:26:14 +0000 Subject: [PATCH 26/96] for save --- pp/go/storage/constructor.go | 85 ++++++++++++++++++++---- pp/go/storage/head.md | 49 ++++++++++++++ pp/go/storage/head/container/weighted.go | 7 ++ pp/go/storage/head/head/head.go | 9 ++- pp/go/storage/head/head/head.md | 19 ------ pp/go/storage/head/shard/wal/wal.go | 13 +++- pp/go/storage/loader.go | 2 + pp/go/util/locker/priweighted.go | 35 ++++++++++ 8 files changed, 183 insertions(+), 36 deletions(-) create mode 100644 pp/go/storage/head.md delete mode 100644 pp/go/storage/head/head/head.md diff --git a/pp/go/storage/constructor.go b/pp/go/storage/constructor.go index c4bc466e18..72ed12ddfb 100644 --- a/pp/go/storage/constructor.go +++ b/pp/go/storage/constructor.go @@ -3,9 +3,11 @@ package storage import ( "fmt" "os" + "time" "github.com/go-kit/log" "github.com/go-kit/log/level" + "github.com/jonboulle/clockwork" "github.com/prometheus/client_golang/prometheus" "github.com/prometheus/prometheus/pp/go/storage/catalog" @@ -14,11 +16,16 @@ import ( "github.com/prometheus/prometheus/pp/go/storage/logger" ) +var DefaultNumberOfShards uint16 = 2 + func HeadManagerCtor( l log.Logger, + clock clockwork.Clock, dataDir string, hcatalog *catalog.Catalog, + blockDuration time.Duration, maxSegmentSize uint32, + numberOfShards uint16, registerer prometheus.Registerer, ) (*HeadManager, error) { dirStat, err := os.Stat(dataDir) @@ -32,13 +39,11 @@ func HeadManagerCtor( initLogHandler(l) - headRecords := hcatalog.List( - func(record *catalog.Record) bool { - return record.DeletedAt() == 0 && record.Status() != catalog.StatusPersisted - }, - func(lhs, rhs *catalog.Record) bool { - return lhs.CreatedAt() < rhs.CreatedAt() - }, + builder := NewBuilder( + hcatalog, + dataDir, + maxSegmentSize, + registerer, ) loader := NewLoader( @@ -47,15 +52,23 @@ func HeadManagerCtor( registerer, ) - builder := NewBuilder( + h, err := uploadOrBuildHead( + clock, hcatalog, - dataDir, - maxSegmentSize, - registerer, + builder, + loader, + blockDuration, + numberOfShards, ) + if err != nil { + return nil, err + } + + if _, err = hcatalog.SetStatus(h.ID(), catalog.StatusActive); err != nil { + return nil, fmt.Errorf("failed to set active status: %w", err) + } - // - activeHead := container.NewWeighted(expectedHead) + activeHead := container.NewWeighted(h) m := manager.NewManager( activeHead, @@ -67,6 +80,52 @@ func HeadManagerCtor( return m, nil } +func uploadOrBuildHead( + clock clockwork.Clock, + hcatalog *catalog.Catalog, + builder *Builder, + loader *Loader, + blockDuration time.Duration, + numberOfShards uint16, +) (*HeadOnDisk, error) { + headRecords := hcatalog.List( + func(record *catalog.Record) bool { + statusIsAppropriate := record.Status() == catalog.StatusNew || + record.Status() == catalog.StatusActive + + isInBlockTimeRange := clock.Now().Sub( + time.UnixMilli(record.CreatedAt()), + ).Milliseconds() < blockDuration.Milliseconds() + + return record.DeletedAt() == 0 && statusIsAppropriate && isInBlockTimeRange + }, + func(lhs, rhs *catalog.Record) bool { + return lhs.CreatedAt() < rhs.CreatedAt() + }, + ) + + if numberOfShards == 0 { + numberOfShards = DefaultNumberOfShards + } + + var generation uint64 + if len(headRecords) == 0 { + return builder.Build(generation, numberOfShards) + } + + h, numberOfSegments, corrupted := loader.UploadHead(headRecords[0], generation) + if corrupted { + if _, err := hcatalog.SetStatus(headRecords[0].ID(), catalog.StatusRotated); err != nil { + // TODO Warning ? + return nil, fmt.Errorf("failed to set rotated status: %w", err) + } + + // TODO loadResult.head.Stop() + + return builder.Build(generation, numberOfShards) + } +} + // initLogHandler init log handler for pp. func initLogHandler(l log.Logger) { l = log.With(l, "pp_caller", log.Caller(4)) diff --git a/pp/go/storage/head.md b/pp/go/storage/head.md new file mode 100644 index 0000000000..45d25dfcbc --- /dev/null +++ b/pp/go/storage/head.md @@ -0,0 +1,49 @@ +# Head + +## Creation + +Creates shards(*LSS*, *DataStorage*, *Wal*), run goroutines of the head, stored in the **Manager**. + +## Active + +Head is used to add and read current data, stored in the **Manager**: + +- **Appender** - add current data: + - *LSS* - write; + - *DataStorage* - write; + - *Wal*(commit) - encode(LSS read) and write; +- **Querier** - provides querying access over time series data: + - *LSS* - read; + - *DataStorage* - read; +- **Committer**(by timer): + - *Wal*(commit) - encode(LSS read) and write; +- **Merger**: + - *DataStorage*(MergeOutOfOrderChunks) - write; +- **Flusher**(on rotate): + - *Wal*(commit) - encode(LSS read) and write; +- **ActiveHeadContainer**(on shutdown) - container for active Head with weighted locker, wait all active task is finished and close semaphore with lock(on append returns error); +- **Flusher**(on shutdown): + - *Wal*(commit) - encode(LSS read) and write; + - *Wal*(flush) - write; + - *Wal* close; +- **Stopper**(on shutdown) - wait all active task is finished and close query semaphore with lock(on select returns empty series set), stop goroutine, *Wal* close. + +## Rotated + +The head that has completed its work, but has not yet been converted into blocks, is read-only, and new data is not being added, stored in the **Keeper**: + +- **Querier** - provides querying access over time series data: + - *LSS* - read; + - *DataStorage* - read; +- **BlockWriter** - converts the head into prom blocks and writes them to a storage: + - *DataStorage*(MergeOutOfOrderChunks) - write; + - **Flusher**: + - *Wal*(flush) - write; + - *Wal* close if flush operations were successful; + - *WriteBlock*: + - *LSS* - read; + - *DataStorage* - read; +- **Flusher**(on shutdown): + - *Wal*(flush) - write; + - *Wal* close; +- **Stopper**(on shutdown or persist) - wait all active task is finished and close query semaphore with lock(on select returns empty series set), stop goroutine, *Wal* close. diff --git a/pp/go/storage/head/container/weighted.go b/pp/go/storage/head/container/weighted.go index 9537381ca7..604b1be18d 100644 --- a/pp/go/storage/head/container/weighted.go +++ b/pp/go/storage/head/container/weighted.go @@ -32,6 +32,13 @@ func NewWeighted[T any, THead Head[T]](head THead) *Weighted[T, THead] { } } +// // Close closes wlocker semaphore for the inability to get query and clear metrics. +// func (c *Weighted[T, THead]) Close(ctx context.Context) error { +// if err := h.querySemaphore.Close(ctx); err != nil { +// return err +// } +// } + // Get the active head [Head] under the non-exlusive lock and return. func (c *Weighted[T, THead]) Get() THead { return (*T)(atomic.LoadPointer( diff --git a/pp/go/storage/head/head/head.go b/pp/go/storage/head/head/head.go index 1e3823fea9..add44c26bf 100644 --- a/pp/go/storage/head/head/head.go +++ b/pp/go/storage/head/head/head.go @@ -185,10 +185,13 @@ func (h *Head[TShard, TGorutineShard]) AcquireQuery(ctx context.Context) (releas return h.querySemaphore.RLock(ctx) } -// Close wals and clear metrics. -func (h *Head[TShard, TGorutineShard]) Close() error { +// Close closes wals, query semaphore for the inability to get query and clear metrics. +func (h *Head[TShard, TGorutineShard]) Close(ctx context.Context) error { + if err := h.querySemaphore.Close(ctx); err != nil { + return err + } + h.memoryInUse.DeletePartialMatch(prometheus.Labels{"generation": strconv.FormatUint(h.generation, 10)}) - // TODO Close ? close(h.stopc) h.wg.Wait() diff --git a/pp/go/storage/head/head/head.md b/pp/go/storage/head/head/head.md deleted file mode 100644 index 968dd6671a..0000000000 --- a/pp/go/storage/head/head/head.md +++ /dev/null @@ -1,19 +0,0 @@ -# Head - -- Create; - - create shard(lss, datastorage, wal); - - run goroutine; -- Active: - - Append: - - LSS - write; - - DataStorage - write; - - Wal - commit(write to wal); - - Query: - - LSS - read; - - DataStorage - read; - - -- Rotated; -- Close; -- Shutdown: - - Wal - commit(write to wal); - - Wal - flush; diff --git a/pp/go/storage/head/shard/wal/wal.go b/pp/go/storage/head/shard/wal/wal.go index 2147635f5b..f04f7b5922 100644 --- a/pp/go/storage/head/shard/wal/wal.go +++ b/pp/go/storage/head/shard/wal/wal.go @@ -52,6 +52,7 @@ type Wal[TSegment EncodedSegment, TStats StatsSegment, TWriter SegmentWriter[TSe maxSegmentSize uint32 corrupted bool limitExhausted bool + closed bool } // NewWal init new [Wal]. @@ -82,7 +83,17 @@ func NewCorruptedWal[ // Close closes the wal segmentWriter. func (w *Wal[TSegment, TStats, TWriter]) Close() error { - return w.segmentWriter.Close() + if w.closed { + return nil + } + + if err := w.segmentWriter.Close(); err != nil { + return err + } + + w.closed = true + + return nil } // Commit finalize segment from encoder and write to [SegmentWriter]. diff --git a/pp/go/storage/loader.go b/pp/go/storage/loader.go index 3550df2bd5..34585726a5 100644 --- a/pp/go/storage/loader.go +++ b/pp/go/storage/loader.go @@ -7,6 +7,7 @@ import ( "sync" "github.com/prometheus/client_golang/prometheus" + "github.com/prometheus/prometheus/pp/go/cppbridge" "github.com/prometheus/prometheus/pp/go/storage/catalog" "github.com/prometheus/prometheus/pp/go/storage/head/head" @@ -43,6 +44,7 @@ func (l *Loader) UploadHead( headDir := filepath.Join(l.dataDir, headID) numberOfShards := headRecord.NumberOfShards() shardLoadResults := make([]ShardLoadResult, numberOfShards) + wg := &sync.WaitGroup{} swn := writer.NewSegmentWriteNotifier(numberOfShards, headRecord.SetLastAppendedSegmentID) for shardID := range numberOfShards { diff --git a/pp/go/util/locker/priweighted.go b/pp/go/util/locker/priweighted.go index 974dda6ace..aa1a2f7a54 100644 --- a/pp/go/util/locker/priweighted.go +++ b/pp/go/util/locker/priweighted.go @@ -3,9 +3,13 @@ package locker // based "golang.org/x/sync/semaphore" import ( "container/list" "context" + "errors" "sync" ) +// ErrSemaphoreClosed error when the semaphore was closed. +var ErrSemaphoreClosed = errors.New("semaphore was closed") + type waiter struct { n int64 ready chan<- struct{} // Closed when semaphore acquired. @@ -27,6 +31,24 @@ type Weighted struct { waiters list.List lastPri *list.Element exclusive bool + closed bool +} + +// Close sets the flag that the semaphore is closed under the priority lock +// and after unlocking all those waiting will receive the error [ErrSemaphoreClosed]. +func (s *Weighted) Close(ctx context.Context) error { + unlock, err := s.LockWithPriority(ctx) + if err != nil { + return err + } + + s.mu.Lock() + s.closed = true + s.mu.Unlock() + + unlock() + + return nil } // Lock locks for exclusive operation with weight of full size. @@ -96,6 +118,11 @@ func (s *Weighted) acquireWithInserter(ctx context.Context, n int64, inserter fu done := ctx.Done() s.mu.Lock() + if s.closed { + s.mu.Unlock() + return ErrSemaphoreClosed + } + select { case <-done: // ctx becoming done has "happened before" acquiring the semaphore, @@ -106,6 +133,7 @@ func (s *Weighted) acquireWithInserter(ctx context.Context, n int64, inserter fu return ctx.Err() default: } + if ws := s.weightSize(n); s.size-s.cur >= ws && s.waiters.Len() == 0 { // Since we hold s.mu and haven't synchronized since checking done, if // ctx becomes done before we return here, it becoming done must have @@ -116,6 +144,7 @@ func (s *Weighted) acquireWithInserter(ctx context.Context, n int64, inserter fu s.exclusive = true } s.mu.Unlock() + return nil } @@ -154,6 +183,12 @@ func (s *Weighted) acquireWithInserter(ctx context.Context, n int64, inserter fu return ctx.Err() default: } + + if s.closed { + s.release(n) + return ErrSemaphoreClosed + } + return nil } } From 098ff7dccbe880915073cd287a51f56efb807b92 Mon Sep 17 00:00:00 2001 From: Alexandr Yudin Date: Tue, 2 Sep 2025 10:52:13 +0000 Subject: [PATCH 27/96] for save --- pp/go/storage/appender/appender.go | 15 ++++-- pp/go/storage/builder.go | 1 - pp/go/storage/constructor.go | 31 ++++++++---- pp/go/storage/head.md | 49 ++++++++++--------- pp/go/storage/head/container/weighted.go | 12 ++--- pp/go/storage/head/head/head.go | 44 +++++++++-------- pp/go/storage/head/head/head_test.go | 2 - pp/go/storage/head/manager/manager.go | 62 +++++++++++++----------- pp/go/storage/head/shard/wal/wal.go | 6 +-- pp/go/storage/loader.go | 27 +++++++++-- pp/go/storage/querier/querier_test.go | 2 - 11 files changed, 149 insertions(+), 102 deletions(-) diff --git a/pp/go/storage/appender/appender.go b/pp/go/storage/appender/appender.go index 12b67cc20a..8ac57b9bdf 100644 --- a/pp/go/storage/appender/appender.go +++ b/pp/go/storage/appender/appender.go @@ -73,6 +73,9 @@ type Wal interface { // It is necessary to lock the LSS for reading for the commit. Commit() error + // Flush wal segment writer, write all buffered data to storage. + Flush() error + // Write append the incoming inner series to wal encoder. Write(innerSeriesSlice []*cppbridge.InnerSeries) (bool, error) } @@ -227,10 +230,16 @@ func (a *Appender[TGenericTask, TDataStorage, TLSS, TWal, TShard, THead]) Append t := a.head.CreateTask( WalCommit, func(shard TShard) error { + swal := shard.Wal() + // wal contains LSS and it is necessary to lock the LSS for reading for the commit. - return shard.LSS().WithRLock(func(_, _ *cppbridge.LabelSetStorage) error { - return shard.Wal().Commit() - }) + if err := shard.LSS().WithRLock(func(_, _ *cppbridge.LabelSetStorage) error { + return swal.Commit() + }); err != nil { + return err + } + + return swal.Flush() }, ) a.head.Enqueue(t) diff --git a/pp/go/storage/builder.go b/pp/go/storage/builder.go index e36521fa37..8625df676f 100644 --- a/pp/go/storage/builder.go +++ b/pp/go/storage/builder.go @@ -77,7 +77,6 @@ func (b *Builder) Build(generation uint64, numberOfShards uint16) (*HeadOnDisk, shard.NewPerGoroutineShard[*WalOnDisk], headRecord.Acquire(), generation, - numberOfShards, b.registerer, ), nil } diff --git a/pp/go/storage/constructor.go b/pp/go/storage/constructor.go index 72ed12ddfb..9ec0c407b3 100644 --- a/pp/go/storage/constructor.go +++ b/pp/go/storage/constructor.go @@ -1,6 +1,7 @@ package storage import ( + "context" "fmt" "os" "time" @@ -16,6 +17,7 @@ import ( "github.com/prometheus/prometheus/pp/go/storage/logger" ) +// DefaultNumberOfShards default number of shards. var DefaultNumberOfShards uint16 = 2 func HeadManagerCtor( @@ -74,6 +76,7 @@ func HeadManagerCtor( activeHead, builder, loader, + numberOfShards, registerer, ) @@ -100,7 +103,7 @@ func uploadOrBuildHead( return record.DeletedAt() == 0 && statusIsAppropriate && isInBlockTimeRange }, func(lhs, rhs *catalog.Record) bool { - return lhs.CreatedAt() < rhs.CreatedAt() + return lhs.CreatedAt() > rhs.CreatedAt() }, ) @@ -110,20 +113,30 @@ func uploadOrBuildHead( var generation uint64 if len(headRecords) == 0 { + // TODO // m.counter.With(prometheus.Labels{"type": "created"}).Inc() return builder.Build(generation, numberOfShards) } - h, numberOfSegments, corrupted := loader.UploadHead(headRecords[0], generation) + h, corrupted := loader.UploadHead(headRecords[0], generation) if corrupted { + if !headRecords[0].Corrupted() { + if _, setCorruptedErr := hcatalog.SetCorrupted(headRecords[0].ID()); setCorruptedErr != nil { + logger.Errorf("failed to set corrupted state, head id: %s: %v", headRecords[0].ID(), setCorruptedErr) + } + } + // TODO // m.counter.With(prometheus.Labels{"type": "corrupted"}).Inc() + if _, err := hcatalog.SetStatus(headRecords[0].ID(), catalog.StatusRotated); err != nil { - // TODO Warning ? - return nil, fmt.Errorf("failed to set rotated status: %w", err) + logger.Warnf("failed to set rotated status for head {%s}: %s", headRecords[0].ID(), err) } - // TODO loadResult.head.Stop() + _ = h.Close(context.Background()) + // TODO // m.counter.With(prometheus.Labels{"type": "created"}).Inc() return builder.Build(generation, numberOfShards) } + + return h, nil } // initLogHandler init log handler for pp. @@ -131,18 +144,18 @@ func initLogHandler(l log.Logger) { l = log.With(l, "pp_caller", log.Caller(4)) logger.Debugf = func(template string, args ...any) { - level.Debug(l).Log("msg", fmt.Sprintf(template, args...)) + _ = level.Debug(l).Log("msg", fmt.Sprintf(template, args...)) } logger.Infof = func(template string, args ...any) { - level.Info(l).Log("msg", fmt.Sprintf(template, args...)) + _ = level.Info(l).Log("msg", fmt.Sprintf(template, args...)) } logger.Warnf = func(template string, args ...any) { - level.Warn(l).Log("msg", fmt.Sprintf(template, args...)) + _ = level.Warn(l).Log("msg", fmt.Sprintf(template, args...)) } logger.Errorf = func(template string, args ...any) { - level.Error(l).Log("msg", fmt.Sprintf(template, args...)) + _ = level.Error(l).Log("msg", fmt.Sprintf(template, args...)) } } diff --git a/pp/go/storage/head.md b/pp/go/storage/head.md index 45d25dfcbc..3bb3634261 100644 --- a/pp/go/storage/head.md +++ b/pp/go/storage/head.md @@ -2,48 +2,51 @@ ## Creation -Creates shards(*LSS*, *DataStorage*, *Wal*), run goroutines of the head, stored in the **Manager**. +Creates shards(**LSS**, **DataStorage**, **Wal**), run goroutines of the head, stored in the **Manager**. ## Active Head is used to add and read current data, stored in the **Manager**: - **Appender** - add current data: - - *LSS* - write; - - *DataStorage* - write; - - *Wal*(commit) - encode(LSS read) and write; + - **LSS** - write; + - **DataStorage** - write; + - **Wal**(commit) - encode(LSS read) segment; + - **Wal**(flush) - write; - **Querier** - provides querying access over time series data: - - *LSS* - read; - - *DataStorage* - read; + - **LSS** - read; + - **DataStorage** - read; - **Committer**(by timer): - - *Wal*(commit) - encode(LSS read) and write; + - **Wal**(commit) - encode(LSS read) segment; + - **Wal**(flush) - write; - **Merger**: - - *DataStorage*(MergeOutOfOrderChunks) - write; + - **DataStorage**(MergeOutOfOrderChunks) - write; - **Flusher**(on rotate): - - *Wal*(commit) - encode(LSS read) and write; + - **Wal**(commit) - encode(LSS read) segment; + - **Wal**(flush) - write; - **ActiveHeadContainer**(on shutdown) - container for active Head with weighted locker, wait all active task is finished and close semaphore with lock(on append returns error); - **Flusher**(on shutdown): - - *Wal*(commit) - encode(LSS read) and write; - - *Wal*(flush) - write; - - *Wal* close; -- **Stopper**(on shutdown) - wait all active task is finished and close query semaphore with lock(on select returns empty series set), stop goroutine, *Wal* close. + - **Wal**(commit) - encode(LSS read) and write; + - **Wal**(flush) - write; + - **Wal** close; +- *Close*(on shutdown) - wait all active task is finished and close query semaphore with lock(on select returns empty series set), stop goroutine, *Wal* close. ## Rotated The head that has completed its work, but has not yet been converted into blocks, is read-only, and new data is not being added, stored in the **Keeper**: - **Querier** - provides querying access over time series data: - - *LSS* - read; - - *DataStorage* - read; + - **LSS** - read; + - **DataStorage** - read; - **BlockWriter** - converts the head into prom blocks and writes them to a storage: - - *DataStorage*(MergeOutOfOrderChunks) - write; + - **DataStorage**(MergeOutOfOrderChunks) - write; - **Flusher**: - - *Wal*(flush) - write; - - *Wal* close if flush operations were successful; + - **Wal**(flush) - write; + - **Wal** close if flush operations were successful; - *WriteBlock*: - - *LSS* - read; - - *DataStorage* - read; + - **LSS** - read; + - **DataStorage** - read; - **Flusher**(on shutdown): - - *Wal*(flush) - write; - - *Wal* close; -- **Stopper**(on shutdown or persist) - wait all active task is finished and close query semaphore with lock(on select returns empty series set), stop goroutine, *Wal* close. + - **Wal**(flush) - write; + - **Wal** close; +- *Close*(on shutdown or persist) - wait all active task is finished and close query semaphore with lock(on select returns empty series set), stop goroutine, *Wal* close. diff --git a/pp/go/storage/head/container/weighted.go b/pp/go/storage/head/container/weighted.go index 604b1be18d..df1f0b7588 100644 --- a/pp/go/storage/head/container/weighted.go +++ b/pp/go/storage/head/container/weighted.go @@ -32,14 +32,12 @@ func NewWeighted[T any, THead Head[T]](head THead) *Weighted[T, THead] { } } -// // Close closes wlocker semaphore for the inability to get query and clear metrics. -// func (c *Weighted[T, THead]) Close(ctx context.Context) error { -// if err := h.querySemaphore.Close(ctx); err != nil { -// return err -// } -// } +// Close closes wlocker semaphore for the inability work with [Head]. +func (c *Weighted[T, THead]) Close(ctx context.Context) error { + return c.wlocker.Close(ctx) +} -// Get the active head [Head] under the non-exlusive lock and return. +// Get the active head [Head] without lock and return. func (c *Weighted[T, THead]) Get() THead { return (*T)(atomic.LoadPointer( (*unsafe.Pointer)(unsafe.Pointer(&c.head))), // #nosec G103 // it's meant to be that way diff --git a/pp/go/storage/head/head/head.go b/pp/go/storage/head/head/head.go index add44c26bf..f224a3c774 100644 --- a/pp/go/storage/head/head/head.go +++ b/pp/go/storage/head/head/head.go @@ -69,8 +69,7 @@ type Head[TShard Shard, TGorutineShard Shard] struct { stopc chan struct{} wg sync.WaitGroup - readOnly uint32 - numberOfShards uint16 + readOnly uint32 // stat memoryInUse *prometheus.GaugeVec @@ -91,25 +90,24 @@ func NewHead[TShard Shard, TGoroutineShard Shard]( gshardCtor func(TShard, uint16) TGoroutineShard, releaseHeadFn func(), generation uint64, - numberOfShards uint16, registerer prometheus.Registerer, ) *Head[TShard, TGoroutineShard] { + numberOfShards := len(shards) taskChs := make([]chan *task.Generic[TGoroutineShard], numberOfShards) concurrency := calculateHeadConcurrency(numberOfShards) // current head workers concurrency - for shardID := uint16(0); shardID < numberOfShards; shardID++ { + for shardID := range numberOfShards { taskChs[shardID] = make(chan *task.Generic[TGoroutineShard], 4*concurrency) // x4 for back pressure } factory := util.NewUnconflictRegisterer(registerer) h := &Head[TShard, TGoroutineShard]{ - id: id, - generation: generation, - gshardCtor: gshardCtor, - releaseHeadFn: releaseHeadFn, - shards: shards, - taskChs: taskChs, - numberOfShards: uint16(len(shards)), // #nosec G115 // no overflow + id: id, + generation: generation, + gshardCtor: gshardCtor, + releaseHeadFn: releaseHeadFn, + shards: shards, + taskChs: taskChs, // TODO metrics memoryInUse: factory.NewGaugeVec( prometheus.GaugeOpts{ @@ -210,7 +208,7 @@ func (h *Head[TShard, TGorutineShard]) Close(ctx context.Context) error { // Concurrency return current head workers concurrency. func (h *Head[TShard, TGorutineShard]) Concurrency() int64 { - return calculateHeadConcurrency(h.numberOfShards) + return calculateHeadConcurrency(len(h.shards)) } // CreateTask create a task for operations on the [Head] shards. @@ -231,7 +229,7 @@ func (h *Head[TShard, TGorutineShard]) CreateTask( // Enqueue the task to be executed on shards [Head]. func (h *Head[TShard, TGorutineShard]) Enqueue(t *task.Generic[TGorutineShard]) { - t.SetShardsNumber(h.numberOfShards) + t.SetShardsNumber(h.NumberOfShards()) for _, taskCh := range h.taskChs { taskCh <- t @@ -245,6 +243,11 @@ func (h *Head[TShard, TGorutineShard]) EnqueueOnShard(t *task.Generic[TGorutineS h.taskChs[shardID] <- t } +// Generation returns current generation of [Head]. +func (h *Head[TShard, TGorutineShard]) Generation() uint64 { + return h.generation +} + // ID returns id [Head]. func (h *Head[TShard, TGorutineShard]) ID() string { return h.id @@ -257,7 +260,7 @@ func (h *Head[TShard, TGorutineShard]) IsReadOnly() bool { // NumberOfShards returns current number of shards in to [Head]. func (h *Head[TShard, TGorutineShard]) NumberOfShards() uint16 { - return h.numberOfShards + return uint16(len(h.shards)) // #nosec G115 // no overflow } // RangeShards returns an iterator over the [Head] [Shard]s, through which the shard can be directly accessed. @@ -284,10 +287,11 @@ func (h *Head[TShard, TGorutineShard]) String() string { // run loop for each shard. func (h *Head[TShard, TGorutineShard]) run() { workers := defaultNumberOfWorkers + ExtraWorkers - h.wg.Add(workers * int(h.numberOfShards)) - for shardID := uint16(0); shardID < h.numberOfShards; shardID++ { - for i := 0; i < workers; i++ { - go func(sid uint16) { + numberOfShards := len(h.shards) + h.wg.Add(workers * numberOfShards) + for shardID := range numberOfShards { + for range workers { + go func(sid int) { defer h.wg.Done() h.shardLoop(h.taskChs[sid], h.stopc, h.shards[sid]) }(shardID) @@ -301,7 +305,7 @@ func (h *Head[TShard, TGorutineShard]) shardLoop( stopc chan struct{}, s TShard, ) { - pgs := h.gshardCtor(s, h.numberOfShards) + pgs := h.gshardCtor(s, h.NumberOfShards()) for { select { @@ -315,7 +319,7 @@ func (h *Head[TShard, TGorutineShard]) shardLoop( } // calculateHeadConcurrency calculate current head workers concurrency. -func calculateHeadConcurrency(numberOfShards uint16) int64 { +func calculateHeadConcurrency(numberOfShards int) int64 { return int64(defaultNumberOfWorkers+ExtraWorkers) * int64(numberOfShards) } diff --git a/pp/go/storage/head/head/head_test.go b/pp/go/storage/head/head/head_test.go index b7f9ffc9fb..22c3aad356 100644 --- a/pp/go/storage/head/head/head_test.go +++ b/pp/go/storage/head/head/head_test.go @@ -14,7 +14,6 @@ func TestXxx(t *testing.T) { wl := &testWal{} sd := shard.NewShard(lss, ds, wl, 0) id := "test-head-id" - numberOfShards := uint16(2) generation := uint64(0) h := head.NewHead( @@ -23,7 +22,6 @@ func TestXxx(t *testing.T) { shard.NewPerGoroutineShard[*testWal], nil, generation, - numberOfShards, nil, ) _ = h diff --git a/pp/go/storage/head/manager/manager.go b/pp/go/storage/head/manager/manager.go index 3550c268be..4a02c25b73 100644 --- a/pp/go/storage/head/manager/manager.go +++ b/pp/go/storage/head/manager/manager.go @@ -6,6 +6,7 @@ import ( "time" "github.com/prometheus/client_golang/prometheus" + "github.com/prometheus/prometheus/pp/go/storage/catalog" "github.com/prometheus/prometheus/pp/go/storage/logger" "github.com/prometheus/prometheus/pp/go/util" @@ -18,12 +19,25 @@ type Timer interface { } type Head interface { + // Generation returns current generation of [Head]. + Generation() uint64 + + // NumberOfShards returns current number of shards in to [Head]. + NumberOfShards() uint16 + + // SetReadOnly sets the read-only flag for the [Head]. SetReadOnly() } +// ActiveHeadContainer container for active [Head], the minimum required [ActiveHeadContainer] implementation. type ActiveHeadContainer[THead Head] interface { + // Get the active head [Head]. Get() THead + + // Replace the active head [Head] with a new head. Replace(ctx context.Context, newHead THead) error + + // With calls fn(h Head). With(ctx context.Context, fn func(h THead) error) error } @@ -32,47 +46,35 @@ type Keeper[THead Head] interface { RangeQueriableHeads(mint, maxt int64) func(func(THead) bool) } -// Loader loads [Head] from wal. +// Loader loads [Head] from wal, the minimum required [Loader] implementation. type Loader[THead Head] interface { // UploadHead upload [THead] from wal by head ID. - UploadHead( - headRecord *catalog.Record, - generation uint64, - ) (head THead, numberOfSegments uint32, corrupted bool) + UploadHead(headRecord *catalog.Record, generation uint64) (head THead, corrupted bool) } -// HeadBuilder building new [Head] with parameters. +// HeadBuilder building new [Head] with parameters, the minimum required [HeadBuilder] implementation. type HeadBuilder[THead Head] interface { // Build new [Head]. Build(generation uint64, numberOfShards uint16) (THead, error) } -// type ActiveHeadContainer[T any] interface { -// Get() *T -// Replace(ctx context.Context, newHead *T) error -// With(ctx context.Context, fn func(h *T) error) error -// } - -// var _ ActiveHeadContainer[testHead] = (*container.Weighted[testHead, *testHead])(nil) - type Manager[THead Head] struct { - // TODO logger + activeHead ActiveHeadContainer[THead] + keeper Keeper[THead] headBuilder HeadBuilder[THead] headLoader Loader[THead] - keeper Keeper[THead] - activeHead ActiveHeadContainer[THead] rotateTimer Timer commitTimer Timer mergeTimer Timer - generation uint64 + + numberOfShards uint16 + // TODO closer vs shutdowner closer *util.Closer shutdowner *util.GracefulShutdowner rotateCounter prometheus.Counter counter *prometheus.CounterVec - - numberOfShards uint16 } // NewManager init new [Manager] of [Head]s. @@ -80,13 +82,17 @@ func NewManager[THead Head]( activeHead ActiveHeadContainer[THead], headBuilder HeadBuilder[THead], headLoader Loader[THead], + numberOfShards uint16, registerer prometheus.Registerer, ) *Manager[THead] { factory := util.NewUnconflictRegisterer(registerer) return &Manager[THead]{ + activeHead: activeHead, headBuilder: headBuilder, headLoader: headLoader, + numberOfShards: numberOfShards, + counter: factory.NewCounterVec( prometheus.CounterOpts{ Name: "prompp_head_event_count", @@ -105,7 +111,12 @@ func (m *Manager[THead]) ApplyConfig( logger.Infof("reconfiguration start") defer logger.Infof("reconfiguration completed") - // TODO HeadConfigStorage + m.numberOfShards = numberOfShards + + h := m.activeHead.Get() + if h.NumberOfShards() != numberOfShards { + // TODO rotate + } return nil } @@ -180,16 +191,13 @@ func (m *Manager[THead]) loop(ctx context.Context) { } func (m *Manager[THead]) rotate(ctx context.Context) error { - newHead, err := m.headBuilder.Build(m.generation, m.numberOfShards) + oldHead := m.activeHead.Get() + + newHead, err := m.headBuilder.Build(oldHead.Generation()+1, m.numberOfShards) if err != nil { return fmt.Errorf("failed to build a new head: %w", err) } - // TODO oldHead.Generation() - m.generation++ - - oldHead := m.activeHead.Get() - // TODO // newHead.CopySeriesFrom(oldHead) diff --git a/pp/go/storage/head/shard/wal/wal.go b/pp/go/storage/head/shard/wal/wal.go index f04f7b5922..7fb9e63089 100644 --- a/pp/go/storage/head/shard/wal/wal.go +++ b/pp/go/storage/head/shard/wal/wal.go @@ -116,10 +116,6 @@ func (w *Wal[TSegment, TStats, TWriter]) Commit() error { return fmt.Errorf("failed to write segment: %w", err) } - if err = w.segmentWriter.Flush(); err != nil { - return fmt.Errorf("failed to flush segment writer: %w", err) - } - return nil } @@ -128,7 +124,7 @@ func (w *Wal[TSegment, TStats, TWriter]) CurrentSize() int64 { return w.segmentWriter.CurrentSize() } -// Flush wal [SegmentWriter]. +// Flush wal [SegmentWriter], write all buffered data to storage. func (w *Wal[TSegment, TStats, TWriter]) Flush() error { w.locker.Lock() defer w.locker.Unlock() diff --git a/pp/go/storage/loader.go b/pp/go/storage/loader.go index 34585726a5..5dc086cfe4 100644 --- a/pp/go/storage/loader.go +++ b/pp/go/storage/loader.go @@ -39,7 +39,7 @@ func NewLoader(dataDir string, maxSegmentSize uint32, registerer prometheus.Regi func (l *Loader) UploadHead( headRecord *catalog.Record, generation uint64, -) (_ *HeadOnDisk, _ uint32, corrupted bool) { +) (_ *HeadOnDisk, corrupted bool) { headID := headRecord.ID() headDir := filepath.Join(l.dataDir, headID) numberOfShards := headRecord.NumberOfShards() @@ -75,6 +75,21 @@ func (l *Loader) UploadHead( } } + switch { + case headRecord.Status() == catalog.StatusActive: + // numberOfSegments here is actual number of segments. + if numberOfSegmentsRead.Value() > 0 { + headRecord.SetLastAppendedSegmentID(numberOfSegmentsRead.Value() - 1) + } + case isNumberOfSegmentsMismatched(headRecord, numberOfSegmentsRead.Value()): + corrupted = true + // numberOfSegments here is actual number of segments. + if numberOfSegmentsRead.Value() > 0 { + headRecord.SetLastAppendedSegmentID(numberOfSegmentsRead.Value() - 1) + } + logger.Errorf("head: %s number of segments mismatched", headRecord.ID()) + } + // TODO h.MergeOutOfOrderChunks() return head.NewHead( headID, @@ -82,10 +97,8 @@ func (l *Loader) UploadHead( shard.NewPerGoroutineShard[*WalOnDisk], headRecord.Acquire(), generation, - numberOfShards, l.registerer, ), - numberOfSegmentsRead.Value(), corrupted } @@ -170,3 +183,11 @@ func (sr *ShardLoadResult) NumberOfSegments() uint32 { func (sr *ShardLoadResult) Shard() *ShardOnDisk { return sr.shard } + +// isNumberOfSegmentsMismatched check number of segments loaded and last appended to record. +func isNumberOfSegmentsMismatched(record *catalog.Record, loadedSegments uint32) bool { + if record.LastAppendedSegmentID() == nil { + return loadedSegments != 0 + } + return *record.LastAppendedSegmentID()+1 != loadedSegments +} diff --git a/pp/go/storage/querier/querier_test.go b/pp/go/storage/querier/querier_test.go index f6eb3477f8..4baa1425b8 100644 --- a/pp/go/storage/querier/querier_test.go +++ b/pp/go/storage/querier/querier_test.go @@ -15,7 +15,6 @@ func TestXxx(t *testing.T) { wl := &testWal{} sd := shard.NewShard(lss, ds, wl, 0) id := "test-head-id" - numberOfShards := uint16(2) generation := uint64(0) h := head.NewHead( @@ -24,7 +23,6 @@ func TestXxx(t *testing.T) { shard.NewPerGoroutineShard[*testWal], nil, generation, - numberOfShards, nil, ) From 72bb59e293c73938d229fcfa6b7279ebd8f0c499 Mon Sep 17 00:00:00 2001 From: Alexandr Yudin Date: Thu, 4 Sep 2025 11:36:48 +0000 Subject: [PATCH 28/96] for save --- pp-pkg/configadapter/headkeeper.go | 4 +- pp/go/storage/appender/appender.go | 78 +++--- pp/go/storage/appender/appender_test.go | 58 +++++ pp/go/storage/head.md | 72 +++--- pp/go/storage/head/manager/manager.go | 327 ++++++++++++++++++++---- pp/go/storage/head/services/commiter.go | 204 +++++++++++++++ pp/go/storage/head/services/rotator.go | 131 ++++++++++ pp/go/storage/head/shard/shard.go | 19 +- pp/go/storage/types.go | 16 +- 9 files changed, 802 insertions(+), 107 deletions(-) create mode 100644 pp/go/storage/appender/appender_test.go create mode 100644 pp/go/storage/head/services/commiter.go create mode 100644 pp/go/storage/head/services/rotator.go diff --git a/pp-pkg/configadapter/headkeeper.go b/pp-pkg/configadapter/headkeeper.go index b2059bf4c7..a0fb389bca 100644 --- a/pp-pkg/configadapter/headkeeper.go +++ b/pp-pkg/configadapter/headkeeper.go @@ -4,7 +4,7 @@ import ( "context" prom_config "github.com/prometheus/prometheus/config" - "github.com/prometheus/prometheus/pp/go/storage/head/manager" + "github.com/prometheus/prometheus/pp/go/storage" ) // DefaultNumberOfShards default value for number of shards [pp_storage.Head]. @@ -13,7 +13,7 @@ var DefaultNumberOfShards uint16 = 2 // HeadKeeperApplyConfig returns func-adapter for apply config on [headkeeper.HeadKeeper]. func HeadKeeperApplyConfig[THead any]( ctx context.Context, - hk *manager.Manager[THead], + hk *storage.HeadManager, ) func(cfg *prom_config.Config) error { return func(cfg *prom_config.Config) error { rCfg, err := cfg.GetReceiverConfig() diff --git a/pp/go/storage/appender/appender.go b/pp/go/storage/appender/appender.go index 8ac57b9bdf..d2ec338fed 100644 --- a/pp/go/storage/appender/appender.go +++ b/pp/go/storage/appender/appender.go @@ -64,12 +64,12 @@ type LSS interface { } // -// LSS +// Wal // // Wal the minimum required Wal implementation for a [Shard]. type Wal interface { - // Commit finalize segment from encoder and write to wal. + // Commit finalize segment from encoder and add to wal. // It is necessary to lock the LSS for reading for the commit. Commit() error @@ -136,11 +136,26 @@ type Appender[ TShard Shard[TDataStorage, TLSS, TWal], THead Head[TGenericTask, TDataStorage, TLSS, TWal, TShard], ] struct { - head THead + head THead + commitAndFlush func(h THead) error +} + +func New[ + TGenericTask GenericTask, + TDataStorage DataStorage, + TLSS LSS, + TWal Wal, + TShard Shard[TDataStorage, TLSS, TWal], + THead Head[TGenericTask, TDataStorage, TLSS, TWal, TShard], +](head THead, commitAndFlush func(h THead) error) Appender[TGenericTask, TDataStorage, TLSS, TWal, TShard, THead] { + return Appender[TGenericTask, TDataStorage, TLSS, TWal, TShard, THead]{ + head: head, + commitAndFlush: commitAndFlush, + } } // Append incoming data to head. -func (a *Appender[TGenericTask, TDataStorage, TLSS, TWal, TShard, THead]) Append( +func (a Appender[TGenericTask, TDataStorage, TLSS, TWal, TShard, THead]) Append( ctx context.Context, incomingData *storage.IncomingData, incomingState *cppbridge.State, @@ -201,7 +216,7 @@ func (a *Appender[TGenericTask, TDataStorage, TLSS, TWal, TShard, THead]) Append ) a.head.Enqueue(tAppend) - var atomiclimitExhausted uint32 + var atomicLimitExhausted uint32 tWalWrite := a.head.CreateTask( WalWrite, func(shard TShard) error { @@ -211,7 +226,7 @@ func (a *Appender[TGenericTask, TDataStorage, TLSS, TWal, TShard, THead]) Append } if limitExhausted { - atomic.AddUint32(&atomiclimitExhausted, 1) + atomic.AddUint32(&atomicLimitExhausted, 1) } return nil @@ -226,25 +241,8 @@ func (a *Appender[TGenericTask, TDataStorage, TLSS, TWal, TShard, THead]) Append logger.Errorf("failed to write wal: %v", err) } - if commitToWal || atomiclimitExhausted > 0 { - t := a.head.CreateTask( - WalCommit, - func(shard TShard) error { - swal := shard.Wal() - - // wal contains LSS and it is necessary to lock the LSS for reading for the commit. - if err := shard.LSS().WithRLock(func(_, _ *cppbridge.LabelSetStorage) error { - return swal.Commit() - }); err != nil { - return err - } - - return swal.Flush() - }, - ) - a.head.Enqueue(t) - - if err := t.Wait(); err != nil { + if commitToWal || atomicLimitExhausted > 0 { + if err := a.commitAndFlush(a.head); err != nil { logger.Errorf("failed to commit wal: %v", err) } } @@ -253,7 +251,7 @@ func (a *Appender[TGenericTask, TDataStorage, TLSS, TWal, TShard, THead]) Append } // inputRelabelingStage first stage - relabeling. -func (a *Appender[TGenericTask, TDataStorage, TLSS, TWal, TShard, THead]) inputRelabelingStage( +func (a Appender[TGenericTask, TDataStorage, TLSS, TWal, TShard, THead]) inputRelabelingStage( ctx context.Context, state *cppbridge.State, incomingData *DestructibleIncomingData, @@ -337,7 +335,7 @@ func (a *Appender[TGenericTask, TDataStorage, TLSS, TWal, TShard, THead]) inputR } // appendRelabelerSeriesStage second stage - append to lss relabeling ls. -func (a *Appender[TGenericTask, TDataStorage, TLSS, TWal, TShard, THead]) appendRelabelerSeriesStage( +func (a Appender[TGenericTask, TDataStorage, TLSS, TWal, TShard, THead]) appendRelabelerSeriesStage( ctx context.Context, shardedInnerSeries *ShardedInnerSeries, shardedRelabeledSeries *ShardedRelabeledSeries, @@ -381,13 +379,13 @@ func (a *Appender[TGenericTask, TDataStorage, TLSS, TWal, TShard, THead]) append } // updateRelabelerStateStage third stage - update state cache. -func (a *Appender[TGenericTask, TDataStorage, TLSS, TWal, TShard, THead]) updateRelabelerStateStage( +func (a Appender[TGenericTask, TDataStorage, TLSS, TWal, TShard, THead]) updateRelabelerStateStage( ctx context.Context, state *cppbridge.State, shardedStateUpdates *ShardedStateUpdates, ) error { numberOfShards := a.head.NumberOfShards() - for shardID := uint16(0); shardID < numberOfShards; shardID++ { + for shardID := range numberOfShards { updates, ok := shardedStateUpdates.DataBySourceShard(shardID) if !ok { continue @@ -400,3 +398,25 @@ func (a *Appender[TGenericTask, TDataStorage, TLSS, TWal, TShard, THead]) update return nil } + +// CommitAndFlush commit and flash [Wal]. +func (a Appender[TGenericTask, TDataStorage, TLSS, TWal, TShard, THead]) CommitAndFlush() error { + t := a.head.CreateTask( + WalCommit, + func(shard TShard) error { + swal := shard.Wal() + + // wal contains LSS and it is necessary to lock the LSS for reading for the commit. + if err := shard.LSS().WithRLock(func(_, _ *cppbridge.LabelSetStorage) error { + return swal.Commit() + }); err != nil { + return err + } + + return swal.Flush() + }, + ) + a.head.Enqueue(t) + + return t.Wait() +} diff --git a/pp/go/storage/appender/appender_test.go b/pp/go/storage/appender/appender_test.go new file mode 100644 index 0000000000..e8c82d78ff --- /dev/null +++ b/pp/go/storage/appender/appender_test.go @@ -0,0 +1,58 @@ +package appender_test + +import ( + "testing" + + "github.com/prometheus/prometheus/pp/go/cppbridge" + "github.com/prometheus/prometheus/pp/go/storage/appender" + "github.com/prometheus/prometheus/pp/go/storage/head/head" + "github.com/prometheus/prometheus/pp/go/storage/head/services" + "github.com/prometheus/prometheus/pp/go/storage/head/shard" +) + +func TestXxx(t *testing.T) { + lss := &shard.LSS{} + ds := shard.NewDataStorage() + wl := &testWal{} + sd := shard.NewShard(lss, ds, wl, 0) + id := "test-head-id" + generation := uint64(0) + + h := head.NewHead( + id, + []*shard.Shard[*testWal]{sd}, + shard.NewPerGoroutineShard[*testWal], + nil, + generation, + nil, + ) + + app := appender.New(h, services.CommitAndFlushViaRange) + + _ = app + + t.Log("end") +} + +// testWal test implementation wal. +type testWal struct{} + +// Close test implementation wal. +func (*testWal) Close() error { + return nil +} + +// Commit test implementation wal. +func (*testWal) Commit() error { + return nil +} + +// Flush test implementation wal. +func (*testWal) Flush() error { + return nil +} + +// Write test implementation wal. +func (*testWal) Write(_ []*cppbridge.InnerSeries) (bool, error) { + return false, nil +} diff --git a/pp/go/storage/head.md b/pp/go/storage/head.md index 3bb3634261..2f3f5dab5f 100644 --- a/pp/go/storage/head.md +++ b/pp/go/storage/head.md @@ -9,27 +9,37 @@ Creates shards(**LSS**, **DataStorage**, **Wal**), run goroutines of the head, s Head is used to add and read current data, stored in the **Manager**: - **Appender** - add current data: - - **LSS** - write; - - **DataStorage** - write; - - **Wal**(commit) - encode(LSS read) segment; - - **Wal**(flush) - write; + - *Append*: + - **LSS** - write; + - **DataStorage** - write; + - **Wal** via task: + - *Commit* - encode(LSS read) segment and add to segment writer(buffer); + - *Flush* - write to storage from buffer if exist; - **Querier** - provides querying access over time series data: - **LSS** - read; - **DataStorage** - read; -- **Committer**(by timer): - - **Wal**(commit) - encode(LSS read) segment; - - **Wal**(flush) - write; -- **Merger**: - - **DataStorage**(MergeOutOfOrderChunks) - write; -- **Flusher**(on rotate): - - **Wal**(commit) - encode(LSS read) segment; - - **Wal**(flush) - write; -- **ActiveHeadContainer**(on shutdown) - container for active Head with weighted locker, wait all active task is finished and close semaphore with lock(on append returns error); -- **Flusher**(on shutdown): - - **Wal**(commit) - encode(LSS read) and write; - - **Wal**(flush) - write; - - **Wal** close; -- *Close*(on shutdown) - wait all active task is finished and close query semaphore with lock(on select returns empty series set), stop goroutine, *Wal* close. +- **Manager**: + - *MergeOutOfOrderChunks*: + - **DataStorage**: + - *MergeOutOfOrderChunks* - write; + - *CommitToWal* by timer: + - **Wal** via task: + - *Commit* - encode(LSS read) segment and add to segment writer(buffer); + - *Flush* - write to storage from buffer if exist; + - *Rotate* by timer: + - **DataStorage** via task: + - *MergeOutOfOrderChunks* - write; + - **Wal** via range: + - *Commit* - encode(LSS read) segment and add to segment writer(buffer); + - *Flush* - write to storage from buffer if exist; + - *Shutdown*: + - **ActiveHeadContainer** - container for active Head with weighted locker: + - *Close* - wait all active task is finished and close semaphore with lock(on append returns error); + - **Wal** via range: + - *Commit* - encode(LSS read) segment and add to segment writer(buffer); + - *Flush* - write to storage from buffer if exist; + - **Head**: + - *Close* - wait all active task is finished and close query semaphore with lock(on select returns empty series set), stop goroutine, **Wal** close. ## Rotated @@ -38,15 +48,17 @@ The head that has completed its work, but has not yet been converted into blocks - **Querier** - provides querying access over time series data: - **LSS** - read; - **DataStorage** - read; -- **BlockWriter** - converts the head into prom blocks and writes them to a storage: - - **DataStorage**(MergeOutOfOrderChunks) - write; - - **Flusher**: - - **Wal**(flush) - write; - - **Wal** close if flush operations were successful; - - *WriteBlock*: - - **LSS** - read; - - **DataStorage** - read; -- **Flusher**(on shutdown): - - **Wal**(flush) - write; - - **Wal** close; -- *Close*(on shutdown or persist) - wait all active task is finished and close query semaphore with lock(on select returns empty series set), stop goroutine, *Wal* close. +- **Keeper**: + - *Write*: + - **Wal** via range: + - *Flush* - write to storage from buffer if exist; + - *Close* - if flush operations were successful; + - **BlockWriter** - converts the head into prom blocks and writes them to a storage: + - *WriteBlock*: + - **LSS** - read; + - **DataStorage** - read; + - *Shutdown*: + - **Wal** via range: + - *Flush* - write to storage from buffer if exist; + - **Head**: + - *Close* - wait all active task is finished and close query semaphore with lock(on select returns empty series set), stop goroutine, **Wal** close. diff --git a/pp/go/storage/head/manager/manager.go b/pp/go/storage/head/manager/manager.go index 4a02c25b73..d81c811c26 100644 --- a/pp/go/storage/head/manager/manager.go +++ b/pp/go/storage/head/manager/manager.go @@ -2,35 +2,150 @@ package manager import ( "context" + "errors" "fmt" "time" "github.com/prometheus/client_golang/prometheus" + "github.com/prometheus/prometheus/pp/go/cppbridge" "github.com/prometheus/prometheus/pp/go/storage/catalog" "github.com/prometheus/prometheus/pp/go/storage/logger" "github.com/prometheus/prometheus/pp/go/util" ) +const ( + // DSMergeOutOfOrderChunks name of task. + DSMergeOutOfOrderChunks = "data_storage_merge_out_of_order_chunks" + + // WalCommit name of task. + WalCommit = "wal_commit" +) + +// +// Timer +// + +// Timer implementation timer. type Timer interface { Chan() <-chan time.Time Reset() Stop() } -type Head interface { +// +// GenericTask +// + +// GenericTask the minimum required task [Generic] implementation. +type GenericTask interface { + // Wait for the task to complete on all shards. + Wait() error +} + +// +// DataStorage +// + +// DataStorage the minimum required [DataStorage] implementation. +type DataStorage interface { + // MergeOutOfOrderChunks merge chunks with out of order data chunks. + MergeOutOfOrderChunks() +} + +// +// LSS +// + +// LSS the minimum required [LSS] implementation. +type LSS interface { + // WithRLock calls fn on raws [cppbridge.LabelSetStorage] with read lock. + WithRLock(fn func(target, input *cppbridge.LabelSetStorage) error) error +} + +// +// Wal +// + +// Wal the minimum required Wal implementation for a [Shard]. +type Wal interface { + // Commit finalize segment from encoder and add to wal. + // It is necessary to lock the LSS for reading for the commit. + Commit() error + + // Flush wal segment writer, write all buffered data to storage. + Flush() error +} + +// +// Shard +// + +// Shard the minimum required head [Shard] implementation. +type Shard[TDataStorage DataStorage, TLSS LSS, TWal Wal] interface { + // DataStorage returns shard [DataStorage]. + DataStorage() TDataStorage + + // LSS returns shard labelset storage [LSS]. + LSS() TLSS + + // ShardID returns the shard ID. + ShardID() uint16 + + // Wal returns write-ahead log. + Wal() TWal +} + +// +// Head +// + +// Head the minimum required [Head] implementation. +type Head[ + TGenericTask GenericTask, + TDataStorage DataStorage, + TLSS LSS, + TWal Wal, + TShard, TGShard Shard[TDataStorage, TLSS, TWal], +] interface { + // Close closes wals, query semaphore for the inability to get query and clear metrics. + Close(ctx context.Context) error + + // CreateTask create a task for operations on the [Head] shards. + CreateTask(taskName string, shardFn func(shard TGShard) error) TGenericTask + + // Enqueue the task to be executed on shards [Head]. + Enqueue(t TGenericTask) + // Generation returns current generation of [Head]. Generation() uint64 // NumberOfShards returns current number of shards in to [Head]. NumberOfShards() uint16 + // RangeShards returns an iterator over the [Head] [Shard]s, through which the shard can be directly accessed. + RangeShards() func(func(TShard) bool) + // SetReadOnly sets the read-only flag for the [Head]. SetReadOnly() } +// +// ActiveHeadContainer +// + // ActiveHeadContainer container for active [Head], the minimum required [ActiveHeadContainer] implementation. -type ActiveHeadContainer[THead Head] interface { +type ActiveHeadContainer[ + TGenericTask GenericTask, + TDataStorage DataStorage, + TLSS LSS, + TWal Wal, + TShard, TGShard Shard[TDataStorage, TLSS, TWal], + THead Head[TGenericTask, TDataStorage, TLSS, TWal, TShard, TGShard], +] interface { + // Close closes [ActiveHeadContainer] for the inability work with [Head]. + Close(ctx context.Context) error + // Get the active head [Head]. Get() THead @@ -41,28 +156,64 @@ type ActiveHeadContainer[THead Head] interface { With(ctx context.Context, fn func(h THead) error) error } -type Keeper[THead Head] interface { +// +// Keeper +// + +type Keeper[ + TGenericTask GenericTask, + TDataStorage DataStorage, + TLSS LSS, + TWal Wal, + TShard, TGShard Shard[TDataStorage, TLSS, TWal], + THead Head[TGenericTask, TDataStorage, TLSS, TWal, TShard, TGShard], +] interface { Add(head THead) RangeQueriableHeads(mint, maxt int64) func(func(THead) bool) } +// +// Loader +// + // Loader loads [Head] from wal, the minimum required [Loader] implementation. -type Loader[THead Head] interface { +type Loader[ + TGenericTask GenericTask, + TDataStorage DataStorage, + TLSS LSS, + TWal Wal, + TShard, TGShard Shard[TDataStorage, TLSS, TWal], + THead Head[TGenericTask, TDataStorage, TLSS, TWal, TShard, TGShard], +] interface { // UploadHead upload [THead] from wal by head ID. UploadHead(headRecord *catalog.Record, generation uint64) (head THead, corrupted bool) } // HeadBuilder building new [Head] with parameters, the minimum required [HeadBuilder] implementation. -type HeadBuilder[THead Head] interface { +type HeadBuilder[ + TGenericTask GenericTask, + TDataStorage DataStorage, + TLSS LSS, + TWal Wal, + TShard, TGShard Shard[TDataStorage, TLSS, TWal], + THead Head[TGenericTask, TDataStorage, TLSS, TWal, TShard, TGShard], +] interface { // Build new [Head]. Build(generation uint64, numberOfShards uint16) (THead, error) } -type Manager[THead Head] struct { - activeHead ActiveHeadContainer[THead] - keeper Keeper[THead] - headBuilder HeadBuilder[THead] - headLoader Loader[THead] +type Manager[ + TGenericTask GenericTask, + TDataStorage DataStorage, + TLSS LSS, + TWal Wal, + TShard, TGShard Shard[TDataStorage, TLSS, TWal], + THead Head[TGenericTask, TDataStorage, TLSS, TWal, TShard, TGShard], +] struct { + activeHead ActiveHeadContainer[TGenericTask, TDataStorage, TLSS, TWal, TShard, TGShard, THead] + keeper Keeper[TGenericTask, TDataStorage, TLSS, TWal, TShard, TGShard, THead] + headBuilder HeadBuilder[TGenericTask, TDataStorage, TLSS, TWal, TShard, TGShard, THead] + headLoader Loader[TGenericTask, TDataStorage, TLSS, TWal, TShard, TGShard, THead] rotateTimer Timer commitTimer Timer mergeTimer Timer @@ -78,15 +229,22 @@ type Manager[THead Head] struct { } // NewManager init new [Manager] of [Head]s. -func NewManager[THead Head]( - activeHead ActiveHeadContainer[THead], - headBuilder HeadBuilder[THead], - headLoader Loader[THead], +func NewManager[ + TGenericTask GenericTask, + TDataStorage DataStorage, + TLSS LSS, + TWal Wal, + TShard, TGShard Shard[TDataStorage, TLSS, TWal], + THead Head[TGenericTask, TDataStorage, TLSS, TWal, TShard, TGShard], +]( + activeHead ActiveHeadContainer[TGenericTask, TDataStorage, TLSS, TWal, TShard, TGShard, THead], + headBuilder HeadBuilder[TGenericTask, TDataStorage, TLSS, TWal, TShard, TGShard, THead], + headLoader Loader[TGenericTask, TDataStorage, TLSS, TWal, TShard, TGShard, THead], numberOfShards uint16, registerer prometheus.Registerer, -) *Manager[THead] { +) *Manager[TGenericTask, TDataStorage, TLSS, TWal, TShard, TGShard, THead] { factory := util.NewUnconflictRegisterer(registerer) - return &Manager[THead]{ + return &Manager[TGenericTask, TDataStorage, TLSS, TWal, TShard, TGShard, THead]{ activeHead: activeHead, headBuilder: headBuilder, headLoader: headLoader, @@ -104,7 +262,7 @@ func NewManager[THead Head]( } // ApplyConfig update config. -func (m *Manager[THead]) ApplyConfig( +func (m *Manager[TGenericTask, TDataStorage, TLSS, TWal, TShard, TGShard, THead]) ApplyConfig( ctx context.Context, numberOfShards uint16, ) error { @@ -114,48 +272,74 @@ func (m *Manager[THead]) ApplyConfig( m.numberOfShards = numberOfShards h := m.activeHead.Get() - if h.NumberOfShards() != numberOfShards { - // TODO rotate + if h.NumberOfShards() == numberOfShards { + return nil } - return nil + return m.rotate(ctx) } // MergeOutOfOrderChunks merge chunks with out of order data chunks. -func (m *Manager[THead]) MergeOutOfOrderChunks(ctx context.Context) error { - // TODO ? - // return m.activeHead.With(ctx, func(h storage.Head) error { - // h.MergeOutOfOrderChunks() - - // return nil - // }) +func (m *Manager[TGenericTask, TDataStorage, TLSS, TWal, TShard, TGShard, THead]) MergeOutOfOrderChunks( + ctx context.Context, +) error { + return m.activeHead.With(ctx, func(h THead) error { + mergeOutOfOrderChunksWithHead(h) - return nil + return nil + }) } // Run starts processing of the [Manager]. // TODO implementation. -func (m *Manager[THead]) Run(ctx context.Context) error { +func (m *Manager[TGenericTask, TDataStorage, TLSS, TWal, TShard, TGShard, THead]) Run(ctx context.Context) error { go m.loop(ctx) return nil } // Shutdown safe shutdown [Manager]. -func (m *Manager[THead]) Shutdown(ctx context.Context) error { - return nil +func (m *Manager[TGenericTask, TDataStorage, TLSS, TWal, TShard, TGShard, THead]) Shutdown(ctx context.Context) error { + // TODO + // cgogcErr := rr.cgogc.Shutdown(ctx) + // err := rr.shutdowner.Shutdown(ctx) + activeHeadErr := m.activeHead.Close(ctx) + + h := m.activeHead.Get() + commitErr := commitAndFlushViaRange(h) + + headCloseErr := h.Close(ctx) + + return errors.Join(activeHeadErr, commitErr, headCloseErr) } -// commitToWal commit the accumulated data into the wal. -func (m *Manager[THead]) commitToWal(ctx context.Context) error { - // TODO ? - // return m.activeHead.With(ctx, func(h storage.Head) error { - // return h.CommitToWal() - // }) - return nil +// commitToWal commit and flush the accumulated data into the wal. +func (m *Manager[TGenericTask, TDataStorage, TLSS, TWal, TShard, TGShard, THead]) commitToWal( + ctx context.Context, +) error { + return m.activeHead.With(ctx, func(h THead) error { + t := h.CreateTask( + WalCommit, + func(shard TGShard) error { + swal := shard.Wal() + + // wal contains LSS and it is necessary to lock the LSS for reading for the commit. + if err := shard.LSS().WithRLock(func(_, _ *cppbridge.LabelSetStorage) error { + return swal.Commit() + }); err != nil { + return err + } + + return swal.Flush() + }, + ) + h.Enqueue(t) + + return t.Wait() + }) } // TODO implementation. -func (m *Manager[THead]) loop(ctx context.Context) { +func (m *Manager[TGenericTask, TDataStorage, TLSS, TWal, TShard, TGShard, THead]) loop(ctx context.Context) { defer m.closer.Done() for { @@ -190,7 +374,7 @@ func (m *Manager[THead]) loop(ctx context.Context) { } } -func (m *Manager[THead]) rotate(ctx context.Context) error { +func (m *Manager[TGenericTask, TDataStorage, TLSS, TWal, TShard, TGShard, THead]) rotate(ctx context.Context) error { oldHead := m.activeHead.Get() newHead, err := m.headBuilder.Build(oldHead.Generation()+1, m.numberOfShards) @@ -198,7 +382,7 @@ func (m *Manager[THead]) rotate(ctx context.Context) error { return fmt.Errorf("failed to build a new head: %w", err) } - // TODO + // TODO CopySeriesFrom only old nunber of shards == new // newHead.CopySeriesFrom(oldHead) m.keeper.Add(oldHead) @@ -209,6 +393,12 @@ func (m *Manager[THead]) rotate(ctx context.Context) error { return fmt.Errorf("failed to replace old to new head: %w", err) } + mergeOutOfOrderChunksWithHead(oldHead) + + if err := commitAndFlushViaRange(oldHead); err != nil { + logger.Warnf("failed commit and flush to wal: %s", err) + } + oldHead.SetReadOnly() return nil @@ -216,13 +406,18 @@ func (m *Manager[THead]) rotate(ctx context.Context) error { // WithAppendableHead // TODO implementation. -func (m *Manager[THead]) WithAppendableHead(ctx context.Context, fn func(h THead) error) error { +func (m *Manager[TGenericTask, TDataStorage, TLSS, TWal, TShard, TGShard, THead]) WithAppendableHead( + ctx context.Context, + fn func(h THead) error, +) error { return m.activeHead.With(ctx, fn) } // RangeQueriableHeads // TODO implementation. -func (m *Manager[THead]) RangeQueriableHeads(mint, maxt int64) func(func(THead) bool) { +func (m *Manager[TGenericTask, TDataStorage, TLSS, TWal, TShard, TGShard, THead]) RangeQueriableHeads( + mint, maxt int64, +) func(func(THead) bool) { // ahead := m.activeHead.Get() // for h := range m.keeper.RangeQueriableHeads(mint, maxt) { // TODO @@ -233,3 +428,49 @@ func (m *Manager[THead]) RangeQueriableHeads(mint, maxt int64) func(func(THead) return nil } + +// mergeOutOfOrderChunksWithHead merge chunks with out of order data chunks for [Head]. +func mergeOutOfOrderChunksWithHead[ + TGenericTask GenericTask, + TDataStorage DataStorage, + TLSS LSS, + TWal Wal, + TShard, TGShard Shard[TDataStorage, TLSS, TWal], + THead Head[TGenericTask, TDataStorage, TLSS, TWal, TShard, TGShard], +](h THead) { + t := h.CreateTask( + DSMergeOutOfOrderChunks, + func(shard TGShard) error { + shard.DataStorage().MergeOutOfOrderChunks() + + return nil + }, + ) + h.Enqueue(t) + + _ = t.Wait() +} + +// commitAndFlushViaRange finalize segment from encoder and add to wal +// and flush wal segment writer, write all buffered data to storage. +func commitAndFlushViaRange[ + TGenericTask GenericTask, + TDataStorage DataStorage, + TLSS LSS, + TWal Wal, + TShard, TGShard Shard[TDataStorage, TLSS, TWal], + THead Head[TGenericTask, TDataStorage, TLSS, TWal, TShard, TGShard], +](h THead) error { + errs := make([]error, 0, h.NumberOfShards()*2) + for shard := range h.RangeShards() { + if err := shard.Wal().Commit(); err != nil { + errs = append(errs, fmt.Errorf("commit shard id %d: %w", shard.ShardID(), err)) + } + + if err := shard.Wal().Flush(); err != nil { + errs = append(errs, fmt.Errorf("flush shard id %d: %w", shard.ShardID(), err)) + } + } + + return errors.Join(errs...) +} diff --git a/pp/go/storage/head/services/commiter.go b/pp/go/storage/head/services/commiter.go new file mode 100644 index 0000000000..7fda5ef2d8 --- /dev/null +++ b/pp/go/storage/head/services/commiter.go @@ -0,0 +1,204 @@ +package services + +import ( + "context" + "errors" + "fmt" + + "github.com/prometheus/prometheus/pp/go/storage/logger" +) + +// const ( +// // walCommit name of task. +// walCommit = "wal_commit" +// ) + +// +// Task +// + +// Task the minimum required task [Generic] implementation. +type Task interface { + // Wait for the task to complete on all shards. + Wait() error +} + +// +// Shard +// + +// Shard the minimum required head [Shard] implementation. +type Shard interface { + // MergeOutOfOrderChunks merge chunks with out of order data chunks in [DataStorage]. + MergeOutOfOrderChunks() + + // ShardID returns the shard ID. + ShardID() uint16 + + // WalCommit finalize segment from encoder and write to wal. + WalCommit() error + + // WalFlush flush all contetnt into wal. + WalFlush() error +} + +// +// Head +// + +// Head the minimum required [Head] implementation. +type Head[ + TTask Task, + TShard, TGoShard Shard, +] interface { + // // Close closes wals, query semaphore for the inability to get query and clear metrics. + // Close(ctx context.Context) error + + // CreateTask create a task for operations on the [Head] shards. + CreateTask(taskName string, shardFn func(shard TGoShard) error) TTask + + // Enqueue the task to be executed on shards [Head]. + Enqueue(t TTask) + + // Generation returns current generation of [Head]. + Generation() uint64 + + // NumberOfShards returns current number of shards in to [Head]. + NumberOfShards() uint16 + + // RangeShards returns an iterator over the [Head] [Shard]s, through which the shard can be directly accessed. + RangeShards() func(func(TShard) bool) + + // SetReadOnly sets the read-only flag for the [Head]. + SetReadOnly() +} + +// +// ActiveHeadContainer +// + +// ActiveHeadContainer container for active [Head], the minimum required [ActiveHeadContainer] implementation. +type ActiveHeadContainer[ + TTask Task, + TShard, TGoShard Shard, + THead Head[TTask, TShard, TGoShard], +] interface { + // Close closes [ActiveHeadContainer] for the inability work with [Head]. + Close(ctx context.Context) error + + // Get the active head [Head]. + Get() THead + + // Replace the active head [Head] with a new head. + Replace(ctx context.Context, newHead THead) error + + // With calls fn(h Head). + With(ctx context.Context, fn func(h THead) error) error +} + +// +// Mediator +// + +// Mediator notifies about events via the channel. +type Mediator interface { + // C returns channel with events. + C() <-chan struct{} + + // Close close channel and stop [Mediator]. + Close() +} + +// +// Committer +// + +// Committer finalize segment from encoder and add to wal +// and flush wal segment writer, write all buffered data to storage, do via task. +type Committer[ + TTask Task, + TShard, TGoShard Shard, + THead Head[TTask, TShard, TGoShard], +] struct { + activeHead ActiveHeadContainer[TTask, TShard, TGoShard, THead] + m Mediator +} + +// Execute starts the [Committer]. +// +//revive:disable-next-line:confusing-naming // other type of Service. +func (s *Committer[TTask, TShard, TGoShard, THead]) Execute(ctx context.Context) error { + logger.Infof("The Committer is running.") + for range s.m.C() { + if err := s.activeHead.With(ctx, CommitAndFlushViaRange); err != nil { + logger.Errorf("wal commit failed: %v", err) + } + } + + logger.Infof("The Committer stopped.") + + return nil +} + +// Interrupt interrupts the [Committer] work. +// +//revive:disable-next-line:confusing-naming // other type of Service. +func (s *Committer[TTask, TShard, TGoShard, THead]) Interrupt(_ error) { + logger.Infof("Stopping Committer...") + + s.m.Close() +} + +// +// CommitAndFlushViaRange +// + +// CommitAndFlushViaRange finalize segment from encoder and add to wal +// and flush wal segment writer, write all buffered data to storage, do via range. +func CommitAndFlushViaRange[ + TTask Task, + TShard, TGoShard Shard, + THead Head[TTask, TShard, TGoShard], +](h THead) error { + errs := make([]error, 0, h.NumberOfShards()*2) + for shard := range h.RangeShards() { + if err := shard.WalCommit(); err != nil { + errs = append(errs, fmt.Errorf("commit shard id %d: %w", shard.ShardID(), err)) + } + + if err := shard.WalFlush(); err != nil { + errs = append(errs, fmt.Errorf("flush shard id %d: %w", shard.ShardID(), err)) + } + } + + return errors.Join(errs...) +} + +// // commitAndFlushViaTask finalize segment from encoder and add to wal +// // and flush wal segment writer, write all buffered data to storage, do via task. +// func commitAndFlushViaTask[ +// TTask Task, +// TDataStorage DataStorage, +// TLSS LSS, +// TShard, TGoShard Shard[TDataStorage, TLSS], +// THead Head[TTask, TDataStorage, TLSS, TShard, TGoShard], +// ](h THead) error { +// t := h.CreateTask( +// WalCommit, +// func(shard TGoShard) error { +// swal := shard.Wal() + +// // wal contains LSS and it is necessary to lock the LSS for reading for the commit. +// if err := shard.LSS().WithRLock(func(_, _ *cppbridge.LabelSetStorage) error { +// return swal.Commit() +// }); err != nil { +// return err +// } + +// return swal.Flush() +// }, +// ) +// h.Enqueue(t) + +// return t.Wait() +// } diff --git a/pp/go/storage/head/services/rotator.go b/pp/go/storage/head/services/rotator.go new file mode 100644 index 0000000000..5c33a3e140 --- /dev/null +++ b/pp/go/storage/head/services/rotator.go @@ -0,0 +1,131 @@ +package services + +import ( + "context" + "fmt" + + "github.com/prometheus/prometheus/pp/go/storage/logger" +) + +const ( + // dsMergeOutOfOrderChunks name of task. + dsMergeOutOfOrderChunks = "data_storage_merge_out_of_order_chunks" +) + +// HeadBuilder building new [Head] with parameters, the minimum required [HeadBuilder] implementation. +type HeadBuilder[ + TTask Task, + TShard, TGoShard Shard, + THead Head[TTask, TShard, TGoShard], +] interface { + // Build new [Head]. + Build(generation uint64, numberOfShards uint16) (THead, error) +} + +// +// Keeper +// + +type Keeper[ + TTask Task, + TShard, TGShard Shard, + THead Head[TTask, TShard, TGShard], +] interface { + Add(head THead) +} + +// +// Rotator +// + +type Rotator[ + TTask Task, + TShard, TGoShard Shard, + THead Head[TTask, TShard, TGoShard], +] struct { + activeHead ActiveHeadContainer[TTask, TShard, TGoShard, THead] + headBuilder HeadBuilder[TTask, TShard, TGoShard, THead] + keeper Keeper[TTask, TShard, TGoShard, THead] + m Mediator +} + +// Execute starts the [Rotator]. +// +//revive:disable-next-line:confusing-naming // other type of Service. +func (s *Rotator[TTask, TShard, TGoShard, THead]) Execute(ctx context.Context) error { + logger.Infof("The Rotator is running.") + + // TODO + var numberOfShards uint16 + + for range s.m.C() { + if err := s.rotate(ctx, numberOfShards); err != nil { + logger.Errorf("rotation failed: %v", err) + } + } + + logger.Infof("The Rotator stopped.") + + return nil +} + +// Interrupt interrupts the [Rotator] work. +// +//revive:disable-next-line:confusing-naming // other type of Service. +func (s *Rotator[TTask, TShard, TGoShard, THead]) Interrupt(_ error) { + logger.Infof("Stopping Rotator...") + + s.m.Close() +} + +func (s *Rotator[TTask, TShard, TGoShard, THead]) rotate( + ctx context.Context, + numberOfShards uint16, +) error { + oldHead := s.activeHead.Get() + + newHead, err := s.headBuilder.Build(oldHead.Generation()+1, numberOfShards) + if err != nil { + return fmt.Errorf("failed to build a new head: %w", err) + } + + // TODO CopySeriesFrom only old nunber of shards == new + // newHead.CopySeriesFrom(oldHead) + + s.keeper.Add(oldHead) + + // TODO if replace error? + err = s.activeHead.Replace(ctx, newHead) + if err != nil { + return fmt.Errorf("failed to replace old to new head: %w", err) + } + + mergeOutOfOrderChunksWithHead(oldHead) + + if err := CommitAndFlushViaRange(oldHead); err != nil { + logger.Warnf("failed commit and flush to wal: %s", err) + } + + oldHead.SetReadOnly() + + return nil +} + +// mergeOutOfOrderChunksWithHead merge chunks with out of order data chunks for [Head]. +func mergeOutOfOrderChunksWithHead[ + TTask Task, + TShard, TGShard Shard, + THead Head[TTask, TShard, TGShard], +](h THead) { + t := h.CreateTask( + dsMergeOutOfOrderChunks, + func(shard TGShard) error { + shard.MergeOutOfOrderChunks() + + return nil + }, + ) + h.Enqueue(t) + + _ = t.Wait() +} diff --git a/pp/go/storage/head/shard/shard.go b/pp/go/storage/head/shard/shard.go index 3afad97cd6..423c5d86c1 100644 --- a/pp/go/storage/head/shard/shard.go +++ b/pp/go/storage/head/shard/shard.go @@ -8,9 +8,15 @@ import ( // Wal the minimum required Wal implementation for a [Shard]. type Wal interface { + // Commit finalize segment from encoder and write to wal. + Commit() error + // Flush flush all contetnt into wal. Flush() error + // Write append the incoming inner series to wal encoder. + Write(innerSeriesSlice []*cppbridge.InnerSeries) (bool, error) + // Close closes the wal segmentWriter. Close() error } @@ -57,6 +63,11 @@ func (s *Shard[TWal]) LSS() *LSS { return s.lss } +// MergeOutOfOrderChunks merge chunks with out of order data chunks in [DataStorage]. +func (s *Shard[TWal]) MergeOutOfOrderChunks() { + s.dataStorage.MergeOutOfOrderChunks() +} + // ShardID returns the shard ID. func (s *Shard[TWal]) ShardID() uint16 { return s.id @@ -67,9 +78,15 @@ func (s *Shard[TWal]) Wal() TWal { return s.wal } +// WalCommit finalize segment from encoder and write to wal. +func (s *Shard[TWal]) WalCommit() error { + return s.lss.WithRLock(func(_, _ *cppbridge.LabelSetStorage) error { + return s.wal.Commit() + }) +} + // WalFlush flush all contetnt into wal. func (s *Shard[TWal]) WalFlush() error { - // TODO return s.wal.Flush() } diff --git a/pp/go/storage/types.go b/pp/go/storage/types.go index 9ed68db609..9dcb11a354 100644 --- a/pp/go/storage/types.go +++ b/pp/go/storage/types.go @@ -7,6 +7,7 @@ import ( "github.com/prometheus/prometheus/pp/go/storage/head/shard" "github.com/prometheus/prometheus/pp/go/storage/head/shard/wal" "github.com/prometheus/prometheus/pp/go/storage/head/shard/wal/writer" + "github.com/prometheus/prometheus/pp/go/storage/head/task" ) // WalOnDisk wal on disk. @@ -19,8 +20,19 @@ type WalOnDisk = wal.Wal[ // ShardOnDisk [shard.Shard] with [WalOnDisk]. type ShardOnDisk = shard.Shard[*WalOnDisk] +// PerGoroutineShard [shard.PerGoroutineShard] with [WalOnDisk]. +type PerGoroutineShard = shard.PerGoroutineShard[*WalOnDisk] + // HeadOnDisk [head.Head] with [ShardOnDisk]. -type HeadOnDisk = head.Head[*ShardOnDisk, *shard.PerGoroutineShard[*WalOnDisk]] +type HeadOnDisk = head.Head[*ShardOnDisk, *PerGoroutineShard] // HeadManager [manager.Manager] for [HeadOnDisk]s. -type HeadManager = manager.Manager[*HeadOnDisk] +type HeadManager = manager.Manager[ + *task.Generic[*PerGoroutineShard], + *shard.DataStorage, + *shard.LSS, + *WalOnDisk, + *ShardOnDisk, + *PerGoroutineShard, + *HeadOnDisk, +] From 4bca70a5446f3b1119ab571bd76483d0f40b5e2d Mon Sep 17 00:00:00 2001 From: Alexandr Yudin Date: Mon, 8 Sep 2025 11:47:05 +0000 Subject: [PATCH 29/96] for save --- pp/go/storage/appender/appender.go | 185 ++++------- pp/go/storage/appender/appender_test.go | 5 + pp/go/storage/constructor.go | 25 +- pp/go/storage/head/container/weighted.go | 4 +- pp/go/storage/head/head/head.go | 306 +++--------------- pp/go/storage/head/head/head_test.go | 5 + pp/go/storage/head/manager/manager.go | 8 +- pp/go/storage/head/services/commiter.go | 184 ++--------- pp/go/storage/head/services/functions.go | 90 ++++++ pp/go/storage/head/services/interface.go | 151 +++++++++ pp/go/storage/head/services/merger.go | 53 +++ .../storage/head/services/metrics_updater.go | 153 +++++++++ pp/go/storage/head/services/rotator.go | 140 ++++---- pp/go/storage/head/shard/shard.go | 28 ++ pp/go/storage/loader.go | 25 +- pp/go/storage/manager.go | 276 ++++++++++++++++ pp/go/storage/querier/chunk_querier.go | 52 +-- pp/go/storage/querier/interface.go | 12 +- pp/go/storage/querier/querier.go | 95 +++--- pp/go/storage/querier/querier_test.go | 5 + pp/go/storage/querier/status_querier.go | 188 +++++++++++ pp/go/util/closer.go | 22 +- pp/go/util/locker/priweighted.go | 4 +- 23 files changed, 1295 insertions(+), 721 deletions(-) create mode 100644 pp/go/storage/head/services/functions.go create mode 100644 pp/go/storage/head/services/interface.go create mode 100644 pp/go/storage/head/services/merger.go create mode 100644 pp/go/storage/head/services/metrics_updater.go create mode 100644 pp/go/storage/manager.go create mode 100644 pp/go/storage/querier/status_querier.go diff --git a/pp/go/storage/appender/appender.go b/pp/go/storage/appender/appender.go index d2ec338fed..198581889d 100644 --- a/pp/go/storage/appender/appender.go +++ b/pp/go/storage/appender/appender.go @@ -12,41 +12,28 @@ import ( ) const ( - // DSAppendInnerSeries name of task. - DSAppendInnerSeries = "data_storage_append_inner_series" + // dsAppendInnerSeries name of task. + dsAppendInnerSeries = "data_storage_append_inner_series" - // LSSInputRelabeling name of task. - LSSInputRelabeling = "lss_input_relabeling" - // LSSAppendRelabelerSeries name of task. - LSSAppendRelabelerSeries = "lss_append_relabeler_series" + // lssInputRelabeling name of task. + lssInputRelabeling = "lss_input_relabeling" + // lssAppendRelabelerSeries name of task. + lssAppendRelabelerSeries = "lss_append_relabeler_series" - // WalWrite name of task. - WalWrite = "wal_write" - - // WalCommit name of task. - WalCommit = "wal_commit" + // walWrite name of task. + walWrite = "wal_write" ) // -// GenericTask +// Task // -// GenericTask the minimum required task [Generic] implementation. -type GenericTask interface { +// Task the minimum required task [Generic] implementation. +type Task interface { // Wait for the task to complete on all shards. Wait() error } -// -// DataStorage -// - -// DataStorage the minimum required [DataStorage] implementation. -type DataStorage interface { - // AppendInnerSeriesSlice add InnerSeries to storage. - AppendInnerSeriesSlice(innerSeriesSlice []*cppbridge.InnerSeries) -} - // // LSS // @@ -63,31 +50,14 @@ type LSS interface { ResetSnapshot() } -// -// Wal -// - -// Wal the minimum required Wal implementation for a [Shard]. -type Wal interface { - // Commit finalize segment from encoder and add to wal. - // It is necessary to lock the LSS for reading for the commit. - Commit() error - - // Flush wal segment writer, write all buffered data to storage. - Flush() error - - // Write append the incoming inner series to wal encoder. - Write(innerSeriesSlice []*cppbridge.InnerSeries) (bool, error) -} - // // Shard // // Shard the minimum required head [Shard] implementation. -type Shard[TDataStorage DataStorage, TLSS LSS, TWal Wal] interface { - // DataStorage returns shard [DataStorage]. - DataStorage() TDataStorage +type Shard[TLSS LSS] interface { + // AppendInnerSeriesSlice add InnerSeries to [DataStorage]. + AppendInnerSeriesSlice(innerSeriesSlice []*cppbridge.InnerSeries) // LSS returns shard labelset storage [LSS]. LSS() TLSS @@ -98,8 +68,8 @@ type Shard[TDataStorage DataStorage, TLSS LSS, TWal Wal] interface { // ShardID returns the shard ID. ShardID() uint16 - // Wal returns write-ahead log. - Wal() TWal + // WalWrite append the incoming inner series to wal encoder. + WalWrite(innerSeriesSlice []*cppbridge.InnerSeries) (bool, error) } // @@ -108,17 +78,15 @@ type Shard[TDataStorage DataStorage, TLSS LSS, TWal Wal] interface { // Head the minimum required [Head] implementation. type Head[ - TGenericTask GenericTask, - TDataStorage DataStorage, + TTask Task, TLSS LSS, - TWal Wal, - TShard Shard[TDataStorage, TLSS, TWal], + TShard Shard[TLSS], ] interface { // CreateTask create a task for operations on the [Head] shards. - CreateTask(taskName string, shardFn func(shard TShard) error) TGenericTask + CreateTask(taskName string, shardFn func(shard TShard) error) TTask // Enqueue the task to be executed on shards [Head]. - Enqueue(t TGenericTask) + Enqueue(t TTask) // NumberOfShards returns current number of shards in to [Head]. NumberOfShards() uint16 @@ -129,33 +97,30 @@ type Head[ // type Appender[ - TGenericTask GenericTask, - TDataStorage DataStorage, + TTask Task, TLSS LSS, - TWal Wal, - TShard Shard[TDataStorage, TLSS, TWal], - THead Head[TGenericTask, TDataStorage, TLSS, TWal, TShard], + TShard Shard[TLSS], + THead Head[TTask, TLSS, TShard], ] struct { head THead commitAndFlush func(h THead) error } +// New init new [Appender]. func New[ - TGenericTask GenericTask, - TDataStorage DataStorage, + TTask Task, TLSS LSS, - TWal Wal, - TShard Shard[TDataStorage, TLSS, TWal], - THead Head[TGenericTask, TDataStorage, TLSS, TWal, TShard], -](head THead, commitAndFlush func(h THead) error) Appender[TGenericTask, TDataStorage, TLSS, TWal, TShard, THead] { - return Appender[TGenericTask, TDataStorage, TLSS, TWal, TShard, THead]{ + TShard Shard[TLSS], + THead Head[TTask, TLSS, TShard], +](head THead, commitAndFlush func(h THead) error) Appender[TTask, TLSS, TShard, THead] { + return Appender[TTask, TLSS, TShard, THead]{ head: head, commitAndFlush: commitAndFlush, } } // Append incoming data to head. -func (a Appender[TGenericTask, TDataStorage, TLSS, TWal, TShard, THead]) Append( +func (a Appender[TTask, TLSS, TShard, THead]) Append( ctx context.Context, incomingData *storage.IncomingData, incomingState *cppbridge.State, @@ -204,40 +169,8 @@ func (a Appender[TGenericTask, TDataStorage, TLSS, TWal, TShard, THead]) Append( } } - tw := task.NewTaskWaiter[TGenericTask](2) - - tAppend := a.head.CreateTask( - DSAppendInnerSeries, - func(shard TShard) error { - shard.DataStorage().AppendInnerSeriesSlice(shardedInnerSeries.DataByShard(shard.ShardID())) - - return nil - }, - ) - a.head.Enqueue(tAppend) - - var atomicLimitExhausted uint32 - tWalWrite := a.head.CreateTask( - WalWrite, - func(shard TShard) error { - limitExhausted, errWrite := shard.Wal().Write(shardedInnerSeries.DataByShard(shard.ShardID())) - if errWrite != nil { - return fmt.Errorf("shard %d: %w", shard.ShardID(), errWrite) - } - - if limitExhausted { - atomic.AddUint32(&atomicLimitExhausted, 1) - } - - return nil - }, - ) - a.head.Enqueue(tWalWrite) - - tw.Add(tAppend) - tw.Add(tWalWrite) - - if err := tw.Wait(); err != nil { + atomicLimitExhausted, err := a.appendInnerSeriesAndWriteToWal(shardedInnerSeries) + if err != nil { logger.Errorf("failed to write wal: %v", err) } @@ -251,7 +184,7 @@ func (a Appender[TGenericTask, TDataStorage, TLSS, TWal, TShard, THead]) Append( } // inputRelabelingStage first stage - relabeling. -func (a Appender[TGenericTask, TDataStorage, TLSS, TWal, TShard, THead]) inputRelabelingStage( +func (a Appender[TTask, TLSS, TShard, THead]) inputRelabelingStage( ctx context.Context, state *cppbridge.State, incomingData *DestructibleIncomingData, @@ -260,7 +193,7 @@ func (a Appender[TGenericTask, TDataStorage, TLSS, TWal, TShard, THead]) inputRe ) (cppbridge.RelabelerStats, error) { stats := make([]cppbridge.RelabelerStats, a.head.NumberOfShards()) t := a.head.CreateTask( - LSSInputRelabeling, + lssInputRelabeling, func(shard TShard) error { var ( lss = shard.LSS() @@ -335,14 +268,14 @@ func (a Appender[TGenericTask, TDataStorage, TLSS, TWal, TShard, THead]) inputRe } // appendRelabelerSeriesStage second stage - append to lss relabeling ls. -func (a Appender[TGenericTask, TDataStorage, TLSS, TWal, TShard, THead]) appendRelabelerSeriesStage( +func (a Appender[TTask, TLSS, TShard, THead]) appendRelabelerSeriesStage( ctx context.Context, shardedInnerSeries *ShardedInnerSeries, shardedRelabeledSeries *ShardedRelabeledSeries, shardedStateUpdates *ShardedStateUpdates, ) error { t := a.head.CreateTask( - LSSAppendRelabelerSeries, + lssAppendRelabelerSeries, func(shard TShard) error { shardID := shard.ShardID() @@ -379,7 +312,7 @@ func (a Appender[TGenericTask, TDataStorage, TLSS, TWal, TShard, THead]) appendR } // updateRelabelerStateStage third stage - update state cache. -func (a Appender[TGenericTask, TDataStorage, TLSS, TWal, TShard, THead]) updateRelabelerStateStage( +func (a Appender[TTask, TLSS, TShard, THead]) updateRelabelerStateStage( ctx context.Context, state *cppbridge.State, shardedStateUpdates *ShardedStateUpdates, @@ -399,24 +332,42 @@ func (a Appender[TGenericTask, TDataStorage, TLSS, TWal, TShard, THead]) updateR return nil } -// CommitAndFlush commit and flash [Wal]. -func (a Appender[TGenericTask, TDataStorage, TLSS, TWal, TShard, THead]) CommitAndFlush() error { - t := a.head.CreateTask( - WalCommit, +// appendInnerSeriesAndWriteToWal append [cppbridge.InnerSeries] to [Shard]'s to [DataStorage] and write to [Wal]. +func (a Appender[TTask, TLSS, TShard, THead]) appendInnerSeriesAndWriteToWal( + shardedInnerSeries *ShardedInnerSeries, +) (uint32, error) { + tw := task.NewTaskWaiter[TTask](2) //revive:disable-line:add-constant // 2 task for wait + + tAppend := a.head.CreateTask( + dsAppendInnerSeries, func(shard TShard) error { - swal := shard.Wal() + shard.AppendInnerSeriesSlice(shardedInnerSeries.DataByShard(shard.ShardID())) - // wal contains LSS and it is necessary to lock the LSS for reading for the commit. - if err := shard.LSS().WithRLock(func(_, _ *cppbridge.LabelSetStorage) error { - return swal.Commit() - }); err != nil { - return err + return nil + }, + ) + a.head.Enqueue(tAppend) + + var atomicLimitExhausted uint32 + tWalWrite := a.head.CreateTask( + walWrite, + func(shard TShard) error { + limitExhausted, errWrite := shard.WalWrite(shardedInnerSeries.DataByShard(shard.ShardID())) + if errWrite != nil { + return fmt.Errorf("shard %d: %w", shard.ShardID(), errWrite) } - return swal.Flush() + if limitExhausted { + atomic.AddUint32(&atomicLimitExhausted, 1) + } + + return nil }, ) - a.head.Enqueue(t) + a.head.Enqueue(tWalWrite) - return t.Wait() + tw.Add(tAppend) + tw.Add(tWalWrite) + + return atomicLimitExhausted, tw.Wait() } diff --git a/pp/go/storage/appender/appender_test.go b/pp/go/storage/appender/appender_test.go index e8c82d78ff..87af145fc0 100644 --- a/pp/go/storage/appender/appender_test.go +++ b/pp/go/storage/appender/appender_test.go @@ -47,6 +47,11 @@ func (*testWal) Commit() error { return nil } +// CurrentSize test implementation wal. +func (*testWal) CurrentSize() int64 { + return 0 +} + // Flush test implementation wal. func (*testWal) Flush() error { return nil diff --git a/pp/go/storage/constructor.go b/pp/go/storage/constructor.go index 9ec0c407b3..a49da0057e 100644 --- a/pp/go/storage/constructor.go +++ b/pp/go/storage/constructor.go @@ -1,13 +1,11 @@ package storage import ( - "context" "fmt" "os" "time" "github.com/go-kit/log" - "github.com/go-kit/log/level" "github.com/jonboulle/clockwork" "github.com/prometheus/client_golang/prometheus" @@ -130,7 +128,7 @@ func uploadOrBuildHead( logger.Warnf("failed to set rotated status for head {%s}: %s", headRecords[0].ID(), err) } - _ = h.Close(context.Background()) + _ = h.Close() // TODO // m.counter.With(prometheus.Labels{"type": "created"}).Inc() return builder.Build(generation, numberOfShards) @@ -138,24 +136,3 @@ func uploadOrBuildHead( return h, nil } - -// initLogHandler init log handler for pp. -func initLogHandler(l log.Logger) { - l = log.With(l, "pp_caller", log.Caller(4)) - - logger.Debugf = func(template string, args ...any) { - _ = level.Debug(l).Log("msg", fmt.Sprintf(template, args...)) - } - - logger.Infof = func(template string, args ...any) { - _ = level.Info(l).Log("msg", fmt.Sprintf(template, args...)) - } - - logger.Warnf = func(template string, args ...any) { - _ = level.Warn(l).Log("msg", fmt.Sprintf(template, args...)) - } - - logger.Errorf = func(template string, args ...any) { - _ = level.Error(l).Log("msg", fmt.Sprintf(template, args...)) - } -} diff --git a/pp/go/storage/head/container/weighted.go b/pp/go/storage/head/container/weighted.go index df1f0b7588..2f57397c54 100644 --- a/pp/go/storage/head/container/weighted.go +++ b/pp/go/storage/head/container/weighted.go @@ -33,8 +33,8 @@ func NewWeighted[T any, THead Head[T]](head THead) *Weighted[T, THead] { } // Close closes wlocker semaphore for the inability work with [Head]. -func (c *Weighted[T, THead]) Close(ctx context.Context) error { - return c.wlocker.Close(ctx) +func (c *Weighted[T, THead]) Close() error { + return c.wlocker.Close() } // Get the active head [Head] without lock and return. diff --git a/pp/go/storage/head/head/head.go b/pp/go/storage/head/head/head.go index f224a3c774..3105035c0f 100644 --- a/pp/go/storage/head/head/head.go +++ b/pp/go/storage/head/head/head.go @@ -5,7 +5,6 @@ import ( "errors" "fmt" "runtime" - "strconv" "sync" "sync/atomic" @@ -71,12 +70,9 @@ type Head[TShard Shard, TGorutineShard Shard] struct { readOnly uint32 - // stat + // for clearing [Head] metrics memoryInUse *prometheus.GaugeVec - series prometheus.Gauge - walSize *prometheus.GaugeVec - queueSize *prometheus.GaugeVec - + // for tasks metrics tasksCreated *prometheus.CounterVec tasksDone *prometheus.CounterVec tasksLive *prometheus.CounterVec @@ -95,7 +91,6 @@ func NewHead[TShard Shard, TGoroutineShard Shard]( numberOfShards := len(shards) taskChs := make([]chan *task.Generic[TGoroutineShard], numberOfShards) concurrency := calculateHeadConcurrency(numberOfShards) // current head workers concurrency - for shardID := range numberOfShards { taskChs[shardID] = make(chan *task.Generic[TGoroutineShard], 4*concurrency) // x4 for back pressure } @@ -108,68 +103,37 @@ func NewHead[TShard Shard, TGoroutineShard Shard]( releaseHeadFn: releaseHeadFn, shards: shards, taskChs: taskChs, - // TODO metrics - memoryInUse: factory.NewGaugeVec( - prometheus.GaugeOpts{ - Name: "prompp_head_cgo_memory_bytes", - Help: "Current value memory in use in bytes.", - }, - // TODO generation -> h.id - []string{"generation", "allocator", "id"}, - ), - series: factory.NewGauge(prometheus.GaugeOpts{ - Name: "prompp_head_series", - Help: "Total number of series in the heads block.", - }), - walSize: factory.NewGaugeVec( - prometheus.GaugeOpts{ - Name: "prompp_head_current_wal_size", - Help: "The size of the wall of the current head.", - }, - []string{"shard_id"}, - ), - - queueSize: factory.NewGaugeVec( - prometheus.GaugeOpts{ - Name: "prompp_head_queue_tasks_size", - Help: "The size of the queue of tasks of the current head.", - }, - []string{"shard_id"}, - ), - - tasksCreated: factory.NewCounterVec( - prometheus.CounterOpts{ - Name: "prompp_head_task_created_count", - Help: "Number of created tasks.", - }, - []string{"type_task"}, - ), - tasksDone: factory.NewCounterVec( - prometheus.CounterOpts{ - Name: "prompp_head_task_done_count", - Help: "Number of done tasks.", - }, - []string{"type_task"}, - ), - tasksLive: factory.NewCounterVec( - prometheus.CounterOpts{ - Name: "prompp_head_task_live_duration_microseconds_sum", - Help: "The duration of the live task in microseconds.", - }, - []string{"type_task"}, - ), - tasksExecute: factory.NewCounterVec( - prometheus.CounterOpts{ - Name: "prompp_head_task_execute_duration_microseconds_sum", - Help: "The duration of the task execution in microseconds.", - }, - []string{"type_task"}, - ), + + // for clearing [Head] metrics + memoryInUse: factory.NewGaugeVec(prometheus.GaugeOpts{ + Name: "prompp_head_cgo_memory_bytes", + Help: "Current value memory in use in bytes.", + }, []string{"head_id", "allocator", "shard_id"}), + + // for tasks metrics + tasksCreated: factory.NewCounterVec(prometheus.CounterOpts{ + Name: "prompp_head_task_created_count", + Help: "Number of created tasks.", + }, []string{"type_task"}), + tasksDone: factory.NewCounterVec(prometheus.CounterOpts{ + Name: "prompp_head_task_done_count", + Help: "Number of done tasks.", + }, []string{"type_task"}), + tasksLive: factory.NewCounterVec(prometheus.CounterOpts{ + Name: "prompp_head_task_live_duration_microseconds_sum", + Help: "The duration of the live task in microseconds.", + }, []string{"type_task"}), + tasksExecute: factory.NewCounterVec(prometheus.CounterOpts{ + Name: "prompp_head_task_execute_duration_microseconds_sum", + Help: "The duration of the task execution in microseconds.", + }, []string{"type_task"}), } h.run() runtime.SetFinalizer(h, func(h *Head[TShard, TGoroutineShard]) { + h.memoryInUse.DeletePartialMatch(prometheus.Labels{"head_id": h.id}) + logger.Debugf("head %s destroyed", h.String()) }) @@ -184,13 +148,11 @@ func (h *Head[TShard, TGorutineShard]) AcquireQuery(ctx context.Context) (releas } // Close closes wals, query semaphore for the inability to get query and clear metrics. -func (h *Head[TShard, TGorutineShard]) Close(ctx context.Context) error { - if err := h.querySemaphore.Close(ctx); err != nil { +func (h *Head[TShard, TGorutineShard]) Close() error { + if err := h.querySemaphore.Close(); err != nil { return err } - h.memoryInUse.DeletePartialMatch(prometheus.Labels{"generation": strconv.FormatUint(h.generation, 10)}) - close(h.stopc) h.wg.Wait() @@ -263,6 +225,17 @@ func (h *Head[TShard, TGorutineShard]) NumberOfShards() uint16 { return uint16(len(h.shards)) // #nosec G115 // no overflow } +// RangeQueueSize returns an iterator over the [Head] task channels, to collect metrics. +func (h *Head[TShard, TGorutineShard]) RangeQueueSize() func(func(shardID, size int) bool) { + return func(yield func(shardID, size int) bool) { + for shardID, taskCh := range h.taskChs { + if !yield(shardID, len(taskCh)) { + return + } + } + } +} + // RangeShards returns an iterator over the [Head] [Shard]s, through which the shard can be directly accessed. func (h *Head[TShard, TGorutineShard]) RangeShards() func(func(TShard) bool) { return func(yield func(s TShard) bool) { @@ -322,200 +295,3 @@ func (h *Head[TShard, TGorutineShard]) shardLoop( func calculateHeadConcurrency(numberOfShards int) int64 { return int64(defaultNumberOfWorkers+ExtraWorkers) * int64(numberOfShards) } - -// TODO Flush CommitToWal ? - -// TODO Who? -// // getSortedStats returns sorted statistics for the [Head]. -// func getSortedStats(stats map[string]uint64, limit int) []storage.HeadStat { -// result := make([]storage.HeadStat, 0, len(stats)) -// for k, v := range stats { -// result = append(result, storage.HeadStat{ -// Name: k, -// Value: v, -// }) -// } - -// sort.Slice(result, func(i, j int) bool { -// return result[i].Value > result[j].Value -// }) - -// if len(result) > limit { -// return result[:limit] -// } - -// return result -// } - -// func (h *Head[TShard, TGorutineShard]) Status(limit int) storage.HeadStatus { -// shardStatuses := make([]*cppbridge.HeadStatus, h.NumberOfShards()) -// for i := range shardStatuses { -// shardStatuses[i] = cppbridge.NewHeadStatus() -// } - -// tw := task.NewTaskWaiter[*task.Generic[TGorutineShard]](2) - -// tLSSHeadStatus := h.CreateTask( -// LSSHeadStatus, -// func(shard TGorutineShard) error { -// shard.LSSRLock() -// shardStatuses[shard.ShardID()].FromLSS(shard.LSS().Raw(), limit) -// shard.LSSRUnlock() - -// return nil -// }, -// ) -// h.Enqueue(tLSSHeadStatus) - -// if limit != 0 { -// tDataStorageHeadStatus := h.CreateTask( -// DSHeadStatus, -// func(shard TGorutineShard) error { -// shard.DataStorageRLock() -// shardStatuses[shard.ShardID()].FromDataStorage(shard.DataStorage().Raw()) -// shard.DataStorageRUnlock() - -// return nil -// }, -// ) -// h.Enqueue(tDataStorageHeadStatus) -// tw.Add(tDataStorageHeadStatus) -// } - -// tw.Add(tLSSHeadStatus) -// _ = tw.Wait() - -// headStatus := storage.HeadStatus{ -// HeadStats: storage.HeadStats{ -// MinTime: math.MaxInt64, -// MaxTime: math.MinInt64, -// }, -// } - -// seriesStats := make(map[string]uint64) -// labelsStats := make(map[string]uint64) -// memoryStats := make(map[string]uint64) -// countStats := make(map[string]uint64) - -// for _, shardStatus := range shardStatuses { -// headStatus.HeadStats.NumSeries += uint64(shardStatus.NumSeries) -// if limit == 0 { -// continue -// } - -// headStatus.HeadStats.ChunkCount += int64(shardStatus.ChunkCount) -// if headStatus.HeadStats.MaxTime < shardStatus.TimeInterval.Max { -// headStatus.HeadStats.MaxTime = shardStatus.TimeInterval.Max -// } -// if headStatus.HeadStats.MinTime > shardStatus.TimeInterval.Min { -// headStatus.HeadStats.MinTime = shardStatus.TimeInterval.Min -// } - -// headStatus.HeadStats.NumLabelPairs += int(shardStatus.NumLabelPairs) - -// for _, stat := range shardStatus.SeriesCountByMetricName { -// seriesStats[stat.Name] += uint64(stat.Count) -// } -// for _, stat := range shardStatus.LabelValueCountByLabelName { -// labelsStats[stat.Name] += uint64(stat.Count) -// } -// for _, stat := range shardStatus.MemoryInBytesByLabelName { -// memoryStats[stat.Name] += uint64(stat.Size) -// } -// for _, stat := range shardStatus.SeriesCountByLabelValuePair { -// countStats[stat.Name+"="+stat.Value] += uint64(stat.Count) -// } -// } - -// if limit == 0 { -// return headStatus -// } - -// headStatus.SeriesCountByMetricName = getSortedStats(seriesStats, limit) -// headStatus.LabelValueCountByLabelName = getSortedStats(labelsStats, limit) -// headStatus.MemoryInBytesByLabelName = getSortedStats(memoryStats, limit) -// headStatus.SeriesCountByLabelValuePair = getSortedStats(countStats, limit) - -// return headStatus -// } - -// func (h *Head[TShard, TGorutineShard]) WriteMetrics(ctx context.Context) { -// if ctx.Err() != nil { -// return -// } - -// status := h.Status(0) -// h.series.Set(float64(status.HeadStats.NumSeries)) - -// if ctx.Err() != nil { -// return -// } - -// generationStr := strconv.FormatUint(h.generation, 10) -// tw := task.NewTaskWaiter[*task.Generic[TGorutineShard]](2) - -// tDataStorageHeadAllocatedMemory := h.CreateTask( -// DSAllocatedMemory, -// func(shard TGorutineShard) error { -// shard.DataStorageRLock() -// am := shard.DataStorage().AllocatedMemory() -// shard.DataStorageRUnlock() - -// h.memoryInUse.With( -// prometheus.Labels{ -// "generation": generationStr, -// "allocator": "data_storage", -// "id": strconv.FormatUint(uint64(shard.ShardID()), 10), -// }, -// ).Set(float64(am)) - -// return nil -// }, -// ) -// h.Enqueue(tDataStorageHeadAllocatedMemory) - -// tLSSHeadAllocatedMemory := h.CreateTask( -// LSSAllocatedMemory, -// func(shard TGorutineShard) error { -// shard.LSSRLock() -// am := shard.LSS().AllocatedMemory() -// shard.LSSRUnlock() - -// h.memoryInUse.With( -// prometheus.Labels{ -// "generation": generationStr, -// "allocator": "main_lss", -// "id": strconv.FormatUint(uint64(shard.ShardID()), 10), -// }, -// ).Set(float64(am)) - -// return nil -// }, -// ) -// h.Enqueue(tLSSHeadAllocatedMemory) - -// tw.Add(tLSSHeadAllocatedMemory) -// tw.Add(tDataStorageHeadAllocatedMemory) -// _ = tw.Wait() - -// if h.readOnly { -// return -// } - -// if ctx.Err() != nil { -// return -// } - -// // do not write metrics if the head is read-only. -// for shardID := uint16(0); shardID < h.numberOfShards; shardID++ { -// shardIDStr := strconv.FormatUint(uint64(shardID), 10) - -// h.walSize.With( -// prometheus.Labels{"shard_id": shardIDStr}, -// ).Set(float64(h.shards[shardID].wal.CurrentSize())) - -// h.queueSize.With(prometheus.Labels{ -// "shard_id": shardIDStr, -// }).Set(float64(len(h.taskChs[shardID]))) -// } -// } diff --git a/pp/go/storage/head/head/head_test.go b/pp/go/storage/head/head/head_test.go index 22c3aad356..304912fe9c 100644 --- a/pp/go/storage/head/head/head_test.go +++ b/pp/go/storage/head/head/head_test.go @@ -40,6 +40,11 @@ func (*testWal) Commit() error { return nil } +// CurrentSize test implementation wal. +func (*testWal) CurrentSize() int64 { + return 0 +} + // Flush test implementation wal. func (*testWal) Flush() error { return nil diff --git a/pp/go/storage/head/manager/manager.go b/pp/go/storage/head/manager/manager.go index d81c811c26..90e3dd0e33 100644 --- a/pp/go/storage/head/manager/manager.go +++ b/pp/go/storage/head/manager/manager.go @@ -109,7 +109,7 @@ type Head[ TShard, TGShard Shard[TDataStorage, TLSS, TWal], ] interface { // Close closes wals, query semaphore for the inability to get query and clear metrics. - Close(ctx context.Context) error + Close() error // CreateTask create a task for operations on the [Head] shards. CreateTask(taskName string, shardFn func(shard TGShard) error) TGenericTask @@ -144,7 +144,7 @@ type ActiveHeadContainer[ THead Head[TGenericTask, TDataStorage, TLSS, TWal, TShard, TGShard], ] interface { // Close closes [ActiveHeadContainer] for the inability work with [Head]. - Close(ctx context.Context) error + Close() error // Get the active head [Head]. Get() THead @@ -302,12 +302,12 @@ func (m *Manager[TGenericTask, TDataStorage, TLSS, TWal, TShard, TGShard, THead] // TODO // cgogcErr := rr.cgogc.Shutdown(ctx) // err := rr.shutdowner.Shutdown(ctx) - activeHeadErr := m.activeHead.Close(ctx) + activeHeadErr := m.activeHead.Close() h := m.activeHead.Get() commitErr := commitAndFlushViaRange(h) - headCloseErr := h.Close(ctx) + headCloseErr := h.Close() return errors.Join(activeHeadErr, commitErr, headCloseErr) } diff --git a/pp/go/storage/head/services/commiter.go b/pp/go/storage/head/services/commiter.go index 7fda5ef2d8..0efa664890 100644 --- a/pp/go/storage/head/services/commiter.go +++ b/pp/go/storage/head/services/commiter.go @@ -2,113 +2,10 @@ package services import ( "context" - "errors" - "fmt" "github.com/prometheus/prometheus/pp/go/storage/logger" ) -// const ( -// // walCommit name of task. -// walCommit = "wal_commit" -// ) - -// -// Task -// - -// Task the minimum required task [Generic] implementation. -type Task interface { - // Wait for the task to complete on all shards. - Wait() error -} - -// -// Shard -// - -// Shard the minimum required head [Shard] implementation. -type Shard interface { - // MergeOutOfOrderChunks merge chunks with out of order data chunks in [DataStorage]. - MergeOutOfOrderChunks() - - // ShardID returns the shard ID. - ShardID() uint16 - - // WalCommit finalize segment from encoder and write to wal. - WalCommit() error - - // WalFlush flush all contetnt into wal. - WalFlush() error -} - -// -// Head -// - -// Head the minimum required [Head] implementation. -type Head[ - TTask Task, - TShard, TGoShard Shard, -] interface { - // // Close closes wals, query semaphore for the inability to get query and clear metrics. - // Close(ctx context.Context) error - - // CreateTask create a task for operations on the [Head] shards. - CreateTask(taskName string, shardFn func(shard TGoShard) error) TTask - - // Enqueue the task to be executed on shards [Head]. - Enqueue(t TTask) - - // Generation returns current generation of [Head]. - Generation() uint64 - - // NumberOfShards returns current number of shards in to [Head]. - NumberOfShards() uint16 - - // RangeShards returns an iterator over the [Head] [Shard]s, through which the shard can be directly accessed. - RangeShards() func(func(TShard) bool) - - // SetReadOnly sets the read-only flag for the [Head]. - SetReadOnly() -} - -// -// ActiveHeadContainer -// - -// ActiveHeadContainer container for active [Head], the minimum required [ActiveHeadContainer] implementation. -type ActiveHeadContainer[ - TTask Task, - TShard, TGoShard Shard, - THead Head[TTask, TShard, TGoShard], -] interface { - // Close closes [ActiveHeadContainer] for the inability work with [Head]. - Close(ctx context.Context) error - - // Get the active head [Head]. - Get() THead - - // Replace the active head [Head] with a new head. - Replace(ctx context.Context, newHead THead) error - - // With calls fn(h Head). - With(ctx context.Context, fn func(h THead) error) error -} - -// -// Mediator -// - -// Mediator notifies about events via the channel. -type Mediator interface { - // C returns channel with events. - C() <-chan struct{} - - // Close close channel and stop [Mediator]. - Close() -} - // // Committer // @@ -124,13 +21,29 @@ type Committer[ m Mediator } +// NewCommitter init new [Committer]. +func NewCommitter[ + TTask Task, + TShard, TGoShard Shard, + THead Head[TTask, TShard, TGoShard], +]( + activeHead ActiveHeadContainer[TTask, TShard, TGoShard, THead], + m Mediator, +) *Committer[TTask, TShard, TGoShard, THead] { + return &Committer[TTask, TShard, TGoShard, THead]{ + activeHead: activeHead, + m: m, + } +} + // Execute starts the [Committer]. // //revive:disable-next-line:confusing-naming // other type of Service. func (s *Committer[TTask, TShard, TGoShard, THead]) Execute(ctx context.Context) error { logger.Infof("The Committer is running.") + for range s.m.C() { - if err := s.activeHead.With(ctx, CommitAndFlushViaRange); err != nil { + if err := s.activeHead.With(ctx, s.commitAndFlushViaRange); err != nil { logger.Errorf("wal commit failed: %v", err) } } @@ -140,65 +53,6 @@ func (s *Committer[TTask, TShard, TGoShard, THead]) Execute(ctx context.Context) return nil } -// Interrupt interrupts the [Committer] work. -// -//revive:disable-next-line:confusing-naming // other type of Service. -func (s *Committer[TTask, TShard, TGoShard, THead]) Interrupt(_ error) { - logger.Infof("Stopping Committer...") - - s.m.Close() +func (s *Committer[TTask, TShard, TGoShard, THead]) commitAndFlushViaRange(h THead) error { + return CommitAndFlushViaRange(h) } - -// -// CommitAndFlushViaRange -// - -// CommitAndFlushViaRange finalize segment from encoder and add to wal -// and flush wal segment writer, write all buffered data to storage, do via range. -func CommitAndFlushViaRange[ - TTask Task, - TShard, TGoShard Shard, - THead Head[TTask, TShard, TGoShard], -](h THead) error { - errs := make([]error, 0, h.NumberOfShards()*2) - for shard := range h.RangeShards() { - if err := shard.WalCommit(); err != nil { - errs = append(errs, fmt.Errorf("commit shard id %d: %w", shard.ShardID(), err)) - } - - if err := shard.WalFlush(); err != nil { - errs = append(errs, fmt.Errorf("flush shard id %d: %w", shard.ShardID(), err)) - } - } - - return errors.Join(errs...) -} - -// // commitAndFlushViaTask finalize segment from encoder and add to wal -// // and flush wal segment writer, write all buffered data to storage, do via task. -// func commitAndFlushViaTask[ -// TTask Task, -// TDataStorage DataStorage, -// TLSS LSS, -// TShard, TGoShard Shard[TDataStorage, TLSS], -// THead Head[TTask, TDataStorage, TLSS, TShard, TGoShard], -// ](h THead) error { -// t := h.CreateTask( -// WalCommit, -// func(shard TGoShard) error { -// swal := shard.Wal() - -// // wal contains LSS and it is necessary to lock the LSS for reading for the commit. -// if err := shard.LSS().WithRLock(func(_, _ *cppbridge.LabelSetStorage) error { -// return swal.Commit() -// }); err != nil { -// return err -// } - -// return swal.Flush() -// }, -// ) -// h.Enqueue(t) - -// return t.Wait() -// } diff --git a/pp/go/storage/head/services/functions.go b/pp/go/storage/head/services/functions.go new file mode 100644 index 0000000000..a854616a8e --- /dev/null +++ b/pp/go/storage/head/services/functions.go @@ -0,0 +1,90 @@ +package services + +import ( + "errors" + "fmt" +) + +const ( + // dsMergeOutOfOrderChunks name of task. + dsMergeOutOfOrderChunks = "data_storage_merge_out_of_order_chunks" +) + +// TODO appender Commit + +// +// CommitAndFlushViaRange +// + +// CommitAndFlushViaRange finalize segment from encoder and add to wal +// and flush wal segment writer, write all buffered data to storage, do via range. +func CommitAndFlushViaRange[ + TTask Task, + TShard, TGoShard Shard, + THead Head[TTask, TShard, TGoShard], +](h THead) error { + errs := make([]error, 0, h.NumberOfShards()*2) + for shard := range h.RangeShards() { + if err := shard.WalCommit(); err != nil { + errs = append(errs, fmt.Errorf("commit shard id %d: %w", shard.ShardID(), err)) + } + + if err := shard.WalFlush(); err != nil { + errs = append(errs, fmt.Errorf("flush shard id %d: %w", shard.ShardID(), err)) + } + } + + return errors.Join(errs...) +} + +// +// MergeOutOfOrderChunksWithHead +// + +// MergeOutOfOrderChunksWithHead merge chunks with out of order data chunks for [Head]. +func MergeOutOfOrderChunksWithHead[ + TTask Task, + TShard, TGShard Shard, + THead Head[TTask, TShard, TGShard], +](h THead) error { + t := h.CreateTask( + dsMergeOutOfOrderChunks, + func(shard TGShard) error { + shard.MergeOutOfOrderChunks() + + return nil + }, + ) + h.Enqueue(t) + + return t.Wait() +} + +// // commitAndFlushViaTask finalize segment from encoder and add to wal +// // and flush wal segment writer, write all buffered data to storage, do via task. +// func commitAndFlushViaTask[ +// TTask Task, +// TDataStorage DataStorage, +// TLSS LSS, +// TShard, TGoShard Shard[TDataStorage, TLSS], +// THead Head[TTask, TDataStorage, TLSS, TShard, TGoShard], +// ](h THead) error { +// t := h.CreateTask( +// WalCommit, +// func(shard TGoShard) error { +// swal := shard.Wal() + +// // wal contains LSS and it is necessary to lock the LSS for reading for the commit. +// if err := shard.LSS().WithRLock(func(_, _ *cppbridge.LabelSetStorage) error { +// return swal.Commit() +// }); err != nil { +// return err +// } + +// return swal.Flush() +// }, +// ) +// h.Enqueue(t) + +// return t.Wait() +// } diff --git a/pp/go/storage/head/services/interface.go b/pp/go/storage/head/services/interface.go new file mode 100644 index 0000000000..89df608104 --- /dev/null +++ b/pp/go/storage/head/services/interface.go @@ -0,0 +1,151 @@ +package services + +import "context" + +// +// ActiveHeadContainer +// + +// ActiveHeadContainer container for active [Head], the minimum required [ActiveHeadContainer] implementation. +type ActiveHeadContainer[ + TTask Task, + TShard, TGoShard Shard, + THead Head[TTask, TShard, TGoShard], +] interface { + // Close closes [ActiveHeadContainer] for the inability work with [Head]. + Close() error + + // Get the active head [Head]. + Get() THead + + // Replace the active head [Head] with a new head. + Replace(ctx context.Context, newHead THead) error + + // With calls fn(h Head). + With(ctx context.Context, fn func(h THead) error) error +} + +// +// Head +// + +// Head the minimum required [Head] implementation. +type Head[ + TTask Task, + TShard, TGoShard Shard, +] interface { + // // Close closes wals, query semaphore for the inability to get query and clear metrics. + // Close(ctx context.Context) error + + // CreateTask create a task for operations on the [Head] shards. + CreateTask(taskName string, shardFn func(shard TGoShard) error) TTask + + // Enqueue the task to be executed on shards [Head]. + Enqueue(t TTask) + + // Generation returns current generation of [Head]. + Generation() uint64 + + // ID returns id [Head]. + ID() string + + // NumberOfShards returns current number of shards in to [Head]. + NumberOfShards() uint16 + + // RangeQueueSize returns an iterator over the [Head] task channels, to collect metrics. + RangeQueueSize() func(func(shardID, size int) bool) + + // RangeShards returns an iterator over the [Head] [Shard]s, through which the shard can be directly accessed. + RangeShards() func(func(TShard) bool) + + // SetReadOnly sets the read-only flag for the [Head]. + SetReadOnly() +} + +// +// HeadBuilder +// + +// HeadBuilder building new [Head] with parameters, the minimum required [HeadBuilder] implementation. +type HeadBuilder[ + TTask Task, + TShard, TGoShard Shard, + THead Head[TTask, TShard, TGoShard], +] interface { + // Build new [Head]. + Build(generation uint64, numberOfShards uint16) (THead, error) +} + +// +// HeadStatusSetter +// + +// HeadStatusSetter sets status by headID in to catalog. +type HeadStatusSetter interface { + // SetActiveStatus sets the [catalog.StatusActive] status by headID. + SetActiveStatus(headID string) error + + // SetRotatedStatus sets the [catalog.StatusRotated] status by headID. + SetRotatedStatus(headID string) error +} + +// +// Keeper +// + +type Keeper[ + TTask Task, + TShard, TGShard Shard, + THead Head[TTask, TShard, TGShard], +] interface { + Add(head THead) + RangeQueriableHeads(mint, maxt int64) func(func(THead) bool) +} + +// +// Mediator +// + +// Mediator notifies about events via the channel. +type Mediator interface { + // C returns channel with events. + C() <-chan struct{} +} + +// +// Shard +// + +// Shard the minimum required head [Shard] implementation. +type Shard interface { + // DSAllocatedMemory return size of allocated memory for [DataStorage]. + DSAllocatedMemory() uint64 + + // LSSAllocatedMemory return size of allocated memory for labelset storages. + LSSAllocatedMemory() uint64 + + // MergeOutOfOrderChunks merge chunks with out of order data chunks in [DataStorage]. + MergeOutOfOrderChunks() + + // ShardID returns the shard ID. + ShardID() uint16 + + // WalCommit finalize segment from encoder and write to wal. + WalCommit() error + + // WalCurrentSize returns current [Wal] size. + WalCurrentSize() int64 + + // WalFlush flush all contetnt into wal. + WalFlush() error +} + +// +// Task +// + +// Task the minimum required task [Generic] implementation. +type Task interface { + // Wait for the task to complete on all shards. + Wait() error +} diff --git a/pp/go/storage/head/services/merger.go b/pp/go/storage/head/services/merger.go new file mode 100644 index 0000000000..6a6e22d514 --- /dev/null +++ b/pp/go/storage/head/services/merger.go @@ -0,0 +1,53 @@ +package services + +import ( + "context" + + "github.com/prometheus/prometheus/pp/go/storage/logger" +) + +// +// Merger +// + +// Merger a service that merge chunks with out of order data chunks for [Head]. +type Merger[ + TTask Task, + TShard, TGoShard Shard, + THead Head[TTask, TShard, TGoShard], +] struct { + activeHead ActiveHeadContainer[TTask, TShard, TGoShard, THead] + m Mediator +} + +// NewMerger init new [Merger]. +func NewMerger[ + TTask Task, + TShard, TGoShard Shard, + THead Head[TTask, TShard, TGoShard], +]( + activeHead ActiveHeadContainer[TTask, TShard, TGoShard, THead], + m Mediator, +) *Merger[TTask, TShard, TGoShard, THead] { + return &Merger[TTask, TShard, TGoShard, THead]{ + activeHead: activeHead, + m: m, + } +} + +// Execute starts the [Merger]. +// +//revive:disable-next-line:confusing-naming // other type of Service. +func (s *Merger[TTask, TShard, TGoShard, THead]) Execute(ctx context.Context) error { + logger.Infof("The Merger is running.") + + for range s.m.C() { + if err := s.activeHead.With(ctx, MergeOutOfOrderChunksWithHead); err != nil { + logger.Errorf("data storage merge failed: %v", err) + } + } + + logger.Infof("The Merger stopped.") + + return nil +} diff --git a/pp/go/storage/head/services/metrics_updater.go b/pp/go/storage/head/services/metrics_updater.go new file mode 100644 index 0000000000..8f5a02bb03 --- /dev/null +++ b/pp/go/storage/head/services/metrics_updater.go @@ -0,0 +1,153 @@ +package services + +import ( + "context" + "math" + "strconv" + + "github.com/prometheus/client_golang/prometheus" + + "github.com/prometheus/prometheus/pp/go/storage/logger" + "github.com/prometheus/prometheus/pp/go/util" +) + +// HeadStatus holds information about number of series from [Head]. +type HeadStatus interface { + // NumSeries returns number of series. + NumSeries() uint64 +} + +// +// MetricsUpdater +// + +// MetricsUpdater a service that updates [Head] metrics. +type MetricsUpdater[ + TTask Task, + TShard, TGoShard Shard, + THead Head[TTask, TShard, TGoShard], + THeadStatus HeadStatus, +] struct { + activeHead ActiveHeadContainer[TTask, TShard, TGoShard, THead] + keeper Keeper[TTask, TShard, TGoShard, THead] + m Mediator + queryHeadStatus func(ctx context.Context, head THead, limit int) (THeadStatus, error) + + // [Head] metrics for an active head. + memoryInUse *prometheus.GaugeVec + series prometheus.Gauge + walSize *prometheus.GaugeVec + queueSize *prometheus.GaugeVec +} + +// NewMetricsUpdater init new [MetricsUpdater]. +func NewMetricsUpdater[ + TTask Task, + TShard, TGoShard Shard, + THead Head[TTask, TShard, TGoShard], + THeadStatus HeadStatus, +]( + activeHead ActiveHeadContainer[TTask, TShard, TGoShard, THead], + keeper Keeper[TTask, TShard, TGoShard, THead], + m Mediator, + queryHeadStatus func(ctx context.Context, head THead, limit int) (THeadStatus, error), + r prometheus.Registerer, +) *MetricsUpdater[TTask, TShard, TGoShard, THead, THeadStatus] { + factory := util.NewUnconflictRegisterer(r) + return &MetricsUpdater[TTask, TShard, TGoShard, THead, THeadStatus]{ + activeHead: activeHead, + keeper: keeper, + m: m, + queryHeadStatus: queryHeadStatus, + + memoryInUse: factory.NewGaugeVec( + prometheus.GaugeOpts{ + Name: "prompp_head_cgo_memory_bytes", + Help: "Current value memory in use in bytes.", + }, + []string{"head_id", "allocator", "shard_id"}, + ), + series: factory.NewGauge(prometheus.GaugeOpts{ + Name: "prompp_head_series", + Help: "Total number of series in the heads block.", + }), + walSize: factory.NewGaugeVec( + prometheus.GaugeOpts{ + Name: "prompp_head_current_wal_size", + Help: "The size of the wall of the current head.", + }, + []string{"shard_id"}, + ), + queueSize: factory.NewGaugeVec( + prometheus.GaugeOpts{ + Name: "prompp_head_queue_tasks_size", + Help: "The size of the queue of tasks of the current head.", + }, + []string{"shard_id"}, + ), + } +} + +// Execute starts the [MetricsUpdater]. +// +//revive:disable-next-line:confusing-naming // other type of Service. +func (s *MetricsUpdater[TTask, TShard, TGoShard, THead, THeadStatus]) Execute(ctx context.Context) error { + logger.Infof("The Rotator is running.") + + for range s.m.C() { + s.collect(ctx) + } + + logger.Infof("The Rotator stopped.") + + return nil +} + +// collect metrics from the head. +func (s *MetricsUpdater[TTask, TShard, TGoShard, THead, THeadStatus]) collect(ctx context.Context) { + ahead := s.activeHead.Get() + + status, err := s.queryHeadStatus(ctx, ahead, 0) + if err == nil { + // error may be only head is rotated, skip + return + } + + s.series.Set(float64(status.NumSeries())) + + for shardID, size := range ahead.RangeQueueSize() { + s.queueSize.With(prometheus.Labels{"shard_id": strconv.Itoa(shardID)}).Set(float64(size)) + } + + s.collectFromShards(ahead, true) + + for head := range s.keeper.RangeQueriableHeads(0, math.MaxInt64) { + if head.ID() == ahead.ID() { + continue + } + + s.collectFromShards(head, false) + } +} + +// fromShards collects metrics from the head's shards. +// +//revive:disable-next-line:flag-parameter this is a flag, but it's more convenient this way +func (s *MetricsUpdater[TTask, TShard, TGoShard, THead, THeadStatus]) collectFromShards(head THead, active bool) { + headID := head.ID() + for shard := range head.RangeShards() { + ls := make(prometheus.Labels, 3) //revive:disable-line:add-constant it's labels count + + ls["shard_id"] = strconv.FormatUint(uint64(shard.ShardID()), 10) //revive:disable-line:add-constant it's base 10 + if active { + s.walSize.With(ls).Set(float64(shard.WalCurrentSize())) + } + + ls["head_id"] = headID + ls["allocator"] = "data_storage" + s.memoryInUse.With(ls).Set(float64(shard.DSAllocatedMemory())) + + ls["allocator"] = "main_lss" + s.memoryInUse.With(ls).Set(float64(shard.LSSAllocatedMemory())) + } +} diff --git a/pp/go/storage/head/services/rotator.go b/pp/go/storage/head/services/rotator.go index 5c33a3e140..803afaff9a 100644 --- a/pp/go/storage/head/services/rotator.go +++ b/pp/go/storage/head/services/rotator.go @@ -3,50 +3,88 @@ package services import ( "context" "fmt" + "sync/atomic" + "github.com/prometheus/client_golang/prometheus" "github.com/prometheus/prometheus/pp/go/storage/logger" + "github.com/prometheus/prometheus/pp/go/util" ) -const ( - // dsMergeOutOfOrderChunks name of task. - dsMergeOutOfOrderChunks = "data_storage_merge_out_of_order_chunks" -) +// +// RotatorConfig +// -// HeadBuilder building new [Head] with parameters, the minimum required [HeadBuilder] implementation. -type HeadBuilder[ - TTask Task, - TShard, TGoShard Shard, - THead Head[TTask, TShard, TGoShard], -] interface { - // Build new [Head]. - Build(generation uint64, numberOfShards uint16) (THead, error) +// RotatorConfig config for [Rotator]. +type RotatorConfig struct { + numberOfShards uint32 } -// -// Keeper -// +// NewRotatorConfig init new [RotatorConfig]. +func NewRotatorConfig(numberOfShards uint16) *RotatorConfig { + return &RotatorConfig{ + numberOfShards: uint32(numberOfShards), + } +} -type Keeper[ - TTask Task, - TShard, TGShard Shard, - THead Head[TTask, TShard, TGShard], -] interface { - Add(head THead) +// NumberOfShards returns current number of shards. +func (c RotatorConfig) NumberOfShards() uint16 { + return uint16(atomic.LoadUint32(&c.numberOfShards)) // #nosec G115 // no overflow +} + +// SetNumberOfShards set new number of shards. +func (c *RotatorConfig) SetNumberOfShards(numberOfShards uint16) { + atomic.StoreUint32(&c.numberOfShards, uint32(numberOfShards)) } // // Rotator // +// Rotator at the end of the specified interval, it creates a new [Head] and makes it active, +// and sends the old [Head] to the [Keeper]. type Rotator[ TTask Task, TShard, TGoShard Shard, THead Head[TTask, TShard, TGoShard], ] struct { - activeHead ActiveHeadContainer[TTask, TShard, TGoShard, THead] - headBuilder HeadBuilder[TTask, TShard, TGoShard, THead] - keeper Keeper[TTask, TShard, TGoShard, THead] - m Mediator + activeHead ActiveHeadContainer[TTask, TShard, TGoShard, THead] + headBuilder HeadBuilder[TTask, TShard, TGoShard, THead] + keeper Keeper[TTask, TShard, TGoShard, THead] + m Mediator + cfg *RotatorConfig + headStatusSetter HeadStatusSetter + rotateCounter prometheus.Counter +} + +// NewRotator init new [Rotator]. +func NewRotator[ + TTask Task, + TShard, TGoShard Shard, + THead Head[TTask, TShard, TGoShard], +]( + activeHead ActiveHeadContainer[TTask, TShard, TGoShard, THead], + headBuilder HeadBuilder[TTask, TShard, TGoShard, THead], + keeper Keeper[TTask, TShard, TGoShard, THead], + m Mediator, + cfg *RotatorConfig, + headStatusSetter HeadStatusSetter, + r prometheus.Registerer, +) *Rotator[TTask, TShard, TGoShard, THead] { + factory := util.NewUnconflictRegisterer(r) + return &Rotator[TTask, TShard, TGoShard, THead]{ + activeHead: activeHead, + headBuilder: headBuilder, + keeper: keeper, + m: m, + cfg: cfg, + headStatusSetter: headStatusSetter, + rotateCounter: factory.NewCounter( + prometheus.CounterOpts{ + Name: "prompp_rotator_rotate_count", + Help: "Total counter of rotate rotatable object.", + }, + ), + } } // Execute starts the [Rotator]. @@ -55,13 +93,12 @@ type Rotator[ func (s *Rotator[TTask, TShard, TGoShard, THead]) Execute(ctx context.Context) error { logger.Infof("The Rotator is running.") - // TODO - var numberOfShards uint16 - for range s.m.C() { - if err := s.rotate(ctx, numberOfShards); err != nil { + if err := s.rotate(ctx, s.cfg.NumberOfShards()); err != nil { logger.Errorf("rotation failed: %v", err) } + + s.rotateCounter.Inc() } logger.Infof("The Rotator stopped.") @@ -69,15 +106,7 @@ func (s *Rotator[TTask, TShard, TGoShard, THead]) Execute(ctx context.Context) e return nil } -// Interrupt interrupts the [Rotator] work. -// -//revive:disable-next-line:confusing-naming // other type of Service. -func (s *Rotator[TTask, TShard, TGoShard, THead]) Interrupt(_ error) { - logger.Infof("Stopping Rotator...") - - s.m.Close() -} - +// rotate it creates a new [Head] and makes it active, and sends the old [Head] to the [Keeper]. func (s *Rotator[TTask, TShard, TGoShard, THead]) rotate( ctx context.Context, numberOfShards uint16, @@ -95,37 +124,26 @@ func (s *Rotator[TTask, TShard, TGoShard, THead]) rotate( s.keeper.Add(oldHead) // TODO if replace error? - err = s.activeHead.Replace(ctx, newHead) - if err != nil { + if err = s.activeHead.Replace(ctx, newHead); err != nil { return fmt.Errorf("failed to replace old to new head: %w", err) } - mergeOutOfOrderChunksWithHead(oldHead) + if err = s.headStatusSetter.SetActiveStatus(newHead.ID()); err != nil { + logger.Warnf("failed set status active for head{%s}: %s", newHead.ID(), err) + } + + if err = MergeOutOfOrderChunksWithHead(oldHead); err != nil { + logger.Warnf("failed merge out of order chunks in data storage: %s", err) + } - if err := CommitAndFlushViaRange(oldHead); err != nil { + if err = CommitAndFlushViaRange(oldHead); err != nil { logger.Warnf("failed commit and flush to wal: %s", err) } + if err = s.headStatusSetter.SetRotatedStatus(oldHead.ID()); err != nil { + logger.Warnf("failed set status rotated for head{%s}: %s", oldHead.ID(), err) + } oldHead.SetReadOnly() return nil } - -// mergeOutOfOrderChunksWithHead merge chunks with out of order data chunks for [Head]. -func mergeOutOfOrderChunksWithHead[ - TTask Task, - TShard, TGShard Shard, - THead Head[TTask, TShard, TGShard], -](h THead) { - t := h.CreateTask( - dsMergeOutOfOrderChunks, - func(shard TGShard) error { - shard.MergeOutOfOrderChunks() - - return nil - }, - ) - h.Enqueue(t) - - _ = t.Wait() -} diff --git a/pp/go/storage/head/shard/shard.go b/pp/go/storage/head/shard/shard.go index 423c5d86c1..9a4e137ed9 100644 --- a/pp/go/storage/head/shard/shard.go +++ b/pp/go/storage/head/shard/shard.go @@ -11,6 +11,9 @@ type Wal interface { // Commit finalize segment from encoder and write to wal. Commit() error + // CurrentSize returns current wal size. + CurrentSize() int64 + // Flush flush all contetnt into wal. Flush() error @@ -48,11 +51,21 @@ func NewShard[TWal Wal]( } } +// AppendInnerSeriesSlice add InnerSeries to [DataStorage]. +func (s *Shard[TWal]) AppendInnerSeriesSlice(innerSeriesSlice []*cppbridge.InnerSeries) { + s.dataStorage.AppendInnerSeriesSlice(innerSeriesSlice) +} + // Close closes the wal segmentWriter. func (s *Shard[TWal]) Close() error { return s.wal.Close() } +// DSAllocatedMemory return size of allocated memory for [DataStorage]. +func (s *Shard[TWal]) DSAllocatedMemory() uint64 { + return s.dataStorage.AllocatedMemory() +} + // DataStorage returns shard [DataStorage]. func (s *Shard[TWal]) DataStorage() *DataStorage { return s.dataStorage @@ -63,6 +76,11 @@ func (s *Shard[TWal]) LSS() *LSS { return s.lss } +// LSSAllocatedMemory return size of allocated memory for labelset storages. +func (s *Shard[TWal]) LSSAllocatedMemory() uint64 { + return s.lss.AllocatedMemory() +} + // MergeOutOfOrderChunks merge chunks with out of order data chunks in [DataStorage]. func (s *Shard[TWal]) MergeOutOfOrderChunks() { s.dataStorage.MergeOutOfOrderChunks() @@ -85,11 +103,21 @@ func (s *Shard[TWal]) WalCommit() error { }) } +// WalCurrentSize returns current [Wal] size. +func (s *Shard[TWal]) WalCurrentSize() int64 { + return s.wal.CurrentSize() +} + // WalFlush flush all contetnt into wal. func (s *Shard[TWal]) WalFlush() error { return s.wal.Flush() } +// WalWrite append the incoming inner series to wal encoder. +func (s *Shard[TWal]) WalWrite(innerSeriesSlice []*cppbridge.InnerSeries) (bool, error) { + return s.wal.Write(innerSeriesSlice) +} + // // PerGoroutineShard // diff --git a/pp/go/storage/loader.go b/pp/go/storage/loader.go index 5dc086cfe4..91e433d9d6 100644 --- a/pp/go/storage/loader.go +++ b/pp/go/storage/loader.go @@ -11,6 +11,7 @@ import ( "github.com/prometheus/prometheus/pp/go/cppbridge" "github.com/prometheus/prometheus/pp/go/storage/catalog" "github.com/prometheus/prometheus/pp/go/storage/head/head" + "github.com/prometheus/prometheus/pp/go/storage/head/services" "github.com/prometheus/prometheus/pp/go/storage/head/shard" "github.com/prometheus/prometheus/pp/go/storage/head/shard/wal" "github.com/prometheus/prometheus/pp/go/storage/head/shard/wal/reader" @@ -90,16 +91,20 @@ func (l *Loader) UploadHead( logger.Errorf("head: %s number of segments mismatched", headRecord.ID()) } - // TODO h.MergeOutOfOrderChunks() - return head.NewHead( - headID, - shards, - shard.NewPerGoroutineShard[*WalOnDisk], - headRecord.Acquire(), - generation, - l.registerer, - ), - corrupted + h := head.NewHead( + headID, + shards, + shard.NewPerGoroutineShard[*WalOnDisk], + headRecord.Acquire(), + generation, + l.registerer, + ) + + if err := services.MergeOutOfOrderChunksWithHead(h); err != nil { + corrupted = true + } + + return h, corrupted } // UploadShard upload [ShardOnDisk] from [WalOnDisk]. diff --git a/pp/go/storage/manager.go b/pp/go/storage/manager.go new file mode 100644 index 0000000000..36e187d368 --- /dev/null +++ b/pp/go/storage/manager.go @@ -0,0 +1,276 @@ +package storage + +import ( + "context" + "errors" + "fmt" + "os" + "sync" + "time" + + "github.com/go-kit/log" + "github.com/go-kit/log/level" + "github.com/jonboulle/clockwork" + "github.com/oklog/run" + "github.com/prometheus/client_golang/prometheus" + + "github.com/prometheus/prometheus/pp/go/storage/catalog" + "github.com/prometheus/prometheus/pp/go/storage/head/container" + "github.com/prometheus/prometheus/pp/go/storage/head/services" + "github.com/prometheus/prometheus/pp/go/storage/logger" + "github.com/prometheus/prometheus/pp/go/storage/querier" + "github.com/prometheus/prometheus/pp/go/util" +) + +type Manager struct { + g run.Group + closer *util.Closer + rotatorConfig *services.RotatorConfig + rotatorMediator *NoopMediator +} + +func NewManager( + l log.Logger, + clock clockwork.Clock, + dataDir string, + hcatalog *catalog.Catalog, + blockDuration time.Duration, + maxSegmentSize uint32, + numberOfShards uint16, + r prometheus.Registerer, +) (*Manager, error) { + dirStat, err := os.Stat(dataDir) + if err != nil { + return nil, fmt.Errorf("failed to stat dir: %w", err) + } + + if !dirStat.IsDir() { + return nil, fmt.Errorf("%s is not directory", dataDir) + } + + initLogHandler(l) + + builder := NewBuilder( + hcatalog, + dataDir, + maxSegmentSize, + r, + ) + + loader := NewLoader( + dataDir, + maxSegmentSize, + r, + ) + + h, err := uploadOrBuildHead( + clock, + hcatalog, + builder, + loader, + blockDuration, + numberOfShards, + ) + if err != nil { + return nil, err + } + + if _, err = hcatalog.SetStatus(h.ID(), catalog.StatusActive); err != nil { + return nil, errors.Join(fmt.Errorf("failed to set active status: %w", err), h.Close()) + } + + // TODO Need close + activeHead := container.NewWeighted(h) + + // TODO implements + headKeeper := &NoopKeeper{} + + m := &Manager{ + g: run.Group{}, + closer: util.NewCloser(), + } + + baseCtx := context.Background() + + // Termination handler. + m.g.Add( + func() error { + <-m.closer.Signal() + + return nil + }, + func(error) { + _ = m.closer.Close() + }, + ) + + statusSetter := &headStatusSetter{catalog: hcatalog} + + // Rotator + m.rotatorConfig = services.NewRotatorConfig(numberOfShards) + m.rotatorMediator = &NoopMediator{c: make(chan struct{})} + rotator := services.NewRotator(activeHead, builder, headKeeper, m.rotatorMediator, m.rotatorConfig, statusSetter, r) + rotatorCtx, rotatorCancel := context.WithCancel(baseCtx) + m.g.Add( + func() error { + return rotator.Execute(rotatorCtx) + }, + func(error) { + m.rotatorMediator.Close() + rotatorCancel() + }, + ) + + // TODO created head + + isNewHead := func(headID string) bool { + rec, err := hcatalog.Get(headID) + if err != nil { + return true + } + + rec.CreatedAt() + } + + // Committer + committerMediator := &NoopMediator{c: make(chan struct{})} + committer := services.NewCommitter(activeHead, committerMediator) + committerCtx, committerCancel := context.WithCancel(baseCtx) + m.g.Add( + func() error { + return committer.Execute(committerCtx) + }, + func(error) { + committerMediator.Close() + committerCancel() + }, + ) + + // Merger + mergerMediator := &NoopMediator{c: make(chan struct{})} + merger := services.NewMerger(activeHead, mergerMediator) + mergerCtx, mergerCancel := context.WithCancel(baseCtx) + m.g.Add( + func() error { + return merger.Execute(mergerCtx) + }, + func(error) { + mergerMediator.Close() + mergerCancel() + }, + ) + + // MetricsUpdater + metricsUpdaterMediator := &NoopMediator{c: make(chan struct{})} + metricsUpdater := services.NewMetricsUpdater( + activeHead, + headKeeper, + metricsUpdaterMediator, + querier.QueryHeadStatus, + r, + ) + metricsUpdaterCtx, metricsUpdaterCancel := context.WithCancel(baseCtx) + m.g.Add( + func() error { + return metricsUpdater.Execute(metricsUpdaterCtx) + }, + func(error) { + metricsUpdaterMediator.Close() + metricsUpdaterCancel() + }, + ) + + return m, nil +} + +// TODO implementation. +func (m *Manager) Run() error { + defer m.closer.Done() + + return m.g.Run() +} + +// TODO implementation. +func (m *Manager) Shutdown(ctx context.Context) { + _ = m.closer.Close() +} + +// initLogHandler init log handler for pp. +func initLogHandler(l log.Logger) { + l = log.With(l, "pp_caller", log.Caller(4)) + + logger.Debugf = func(template string, args ...any) { + _ = level.Debug(l).Log("msg", fmt.Sprintf(template, args...)) + } + + logger.Infof = func(template string, args ...any) { + _ = level.Info(l).Log("msg", fmt.Sprintf(template, args...)) + } + + logger.Warnf = func(template string, args ...any) { + _ = level.Warn(l).Log("msg", fmt.Sprintf(template, args...)) + } + + logger.Errorf = func(template string, args ...any) { + _ = level.Error(l).Log("msg", fmt.Sprintf(template, args...)) + } +} + +// +// NoopKeeper +// + +// NoopKeeper implements Keeper. +type NoopKeeper struct{} + +// Add implements Keeper. +func (*NoopKeeper) Add(*HeadOnDisk) {} + +// RangeQueriableHeads implements Keeper. +func (k *NoopKeeper) RangeQueriableHeads( + mint, maxt int64, +) func(func(*HeadOnDisk) bool) { + return func(func(*HeadOnDisk) bool) {} +} + +// +// NoopMediator +// + +// NoopMediator implements Mediator. +type NoopMediator struct { + c chan struct{} + closeOnce sync.Once +} + +// C implements Mediator. +func (m *NoopMediator) C() <-chan struct{} { + return m.c +} + +// Close close channel and stop [Mediator]. +func (m *NoopMediator) Close() { + m.closeOnce.Do(func() { + close(m.c) + }) +} + +// +// +// + +type headStatusSetter struct { + catalog *catalog.Catalog +} + +// SetActiveStatus sets the [catalog.StatusActive] status by headID. +func (ha *headStatusSetter) SetActiveStatus(headID string) error { + _, err := ha.catalog.SetStatus(headID, catalog.StatusActive) + return err +} + +// SetRotatedStatus sets the [catalog.StatusRotated] status by headID. +func (ha *headStatusSetter) SetRotatedStatus(headID string) error { + _, err := ha.catalog.SetStatus(headID, catalog.StatusRotated) + return err +} diff --git a/pp/go/storage/querier/chunk_querier.go b/pp/go/storage/querier/chunk_querier.go index e75fa114c6..db1ef151b5 100644 --- a/pp/go/storage/querier/chunk_querier.go +++ b/pp/go/storage/querier/chunk_querier.go @@ -2,33 +2,35 @@ package querier import ( "context" + "errors" "github.com/prometheus/prometheus/model/labels" "github.com/prometheus/prometheus/pp/go/cppbridge" "github.com/prometheus/prometheus/pp/go/storage/logger" + "github.com/prometheus/prometheus/pp/go/util/locker" "github.com/prometheus/prometheus/storage" "github.com/prometheus/prometheus/util/annotations" ) const ( - // LSSQueryChunkQuerySelector name of task. - LSSQueryChunkQuerySelector = "lss_query_chunk_query_selector" - // LSSLabelValuesChunkQuerier name of task. - LSSLabelValuesChunkQuerier = "lss_label_values_chunk_querier" - // LSSLabelNamesChunkQuerier name of task. - LSSLabelNamesChunkQuerier = "lss_label_names_chunk_querier" - - // DSQueryChunkQuerier name of task. - DSQueryChunkQuerier = "data_storage_query_chunk_querier" + // lssQueryChunkQuerySelector name of task. + lssQueryChunkQuerySelector = "lss_query_chunk_query_selector" + // lssLabelValuesChunkQuerier name of task. + lssLabelValuesChunkQuerier = "lss_label_values_chunk_querier" + // lssLabelNamesChunkQuerier name of task. + lssLabelNamesChunkQuerier = "lss_label_names_chunk_querier" + + // dsQueryChunkQuerier name of task. + dsQueryChunkQuerier = "data_storage_query_chunk_querier" ) // ChunkQuerier provides querying access over time series data of a fixed time range. type ChunkQuerier[ - TGenericTask GenericTask, + TTask Task, TDataStorage DataStorage, TLSS LSS, TShard Shard[TDataStorage, TLSS], - THead Head[TGenericTask, TDataStorage, TLSS, TShard], + THead Head[TTask, TDataStorage, TLSS, TShard], ] struct { head THead deduplicatorCtor deduplicatorCtor @@ -39,18 +41,18 @@ type ChunkQuerier[ // NewChunkQuerier init new [ChunkQuerier]. func NewChunkQuerier[ - TGenericTask GenericTask, + TTask Task, TDataStorage DataStorage, TLSS LSS, TShard Shard[TDataStorage, TLSS], - THead Head[TGenericTask, TDataStorage, TLSS, TShard], + THead Head[TTask, TDataStorage, TLSS, TShard], ]( head THead, deduplicatorCtor deduplicatorCtor, mint, maxt int64, closer func() error, -) *ChunkQuerier[TGenericTask, TDataStorage, TLSS, TShard, THead] { - return &ChunkQuerier[TGenericTask, TDataStorage, TLSS, TShard, THead]{ +) *ChunkQuerier[TTask, TDataStorage, TLSS, TShard, THead] { + return &ChunkQuerier[TTask, TDataStorage, TLSS, TShard, THead]{ head: head, deduplicatorCtor: deduplicatorCtor, mint: mint, @@ -62,7 +64,7 @@ func NewChunkQuerier[ // Close [ChunkQuerier] if need. // //revive:disable-next-line:confusing-naming // other type of querier. -func (q *ChunkQuerier[TGenericTask, TDataStorage, TLSS, TShard, THead]) Close() error { +func (q *ChunkQuerier[TTask, TDataStorage, TLSS, TShard, THead]) Close() error { if q.closer != nil { err := q.closer() q.closer = nil @@ -75,7 +77,7 @@ func (q *ChunkQuerier[TGenericTask, TDataStorage, TLSS, TShard, THead]) Close() // LabelNames returns label values present in the head for the specific label name. // //revive:disable-next-line:confusing-naming // other type of querier. -func (q *ChunkQuerier[TGenericTask, TDataStorage, TLSS, TShard, THead]) LabelNames( +func (q *ChunkQuerier[TTask, TDataStorage, TLSS, TShard, THead]) LabelNames( ctx context.Context, hints *storage.LabelHints, matchers ...*labels.Matcher, @@ -85,7 +87,7 @@ func (q *ChunkQuerier[TGenericTask, TDataStorage, TLSS, TShard, THead]) LabelNam q.head, q.deduplicatorCtor, nil, - LSSLabelNamesChunkQuerier, + lssLabelNamesChunkQuerier, hints, matchers..., ) @@ -96,7 +98,7 @@ func (q *ChunkQuerier[TGenericTask, TDataStorage, TLSS, TShard, THead]) LabelNam // result set is reduced to label values of metrics matching the matchers. // //revive:disable:confusing-naming // other type of querier. -func (q *ChunkQuerier[TGenericTask, TDataStorage, TLSS, TShard, THead]) LabelValues( +func (q *ChunkQuerier[TTask, TDataStorage, TLSS, TShard, THead]) LabelValues( ctx context.Context, name string, hints *storage.LabelHints, @@ -108,7 +110,7 @@ func (q *ChunkQuerier[TGenericTask, TDataStorage, TLSS, TShard, THead]) LabelVal q.head, q.deduplicatorCtor, nil, - LSSLabelValuesChunkQuerier, + lssLabelValuesChunkQuerier, hints, matchers..., ) @@ -117,7 +119,7 @@ func (q *ChunkQuerier[TGenericTask, TDataStorage, TLSS, TShard, THead]) LabelVal // Select returns a chunk set of series that matches the given label matchers. // //revive:disable-next-line:confusing-naming // other type of querier. -func (q *ChunkQuerier[TGenericTask, TDataStorage, TLSS, TShard, THead]) Select( +func (q *ChunkQuerier[TTask, TDataStorage, TLSS, TShard, THead]) Select( ctx context.Context, _ bool, _ *storage.SelectHints, @@ -125,18 +127,22 @@ func (q *ChunkQuerier[TGenericTask, TDataStorage, TLSS, TShard, THead]) Select( ) storage.ChunkSeriesSet { release, err := q.head.AcquireQuery(ctx) if err != nil { + if errors.Is(err, locker.ErrSemaphoreClosed) { + return &EmptyChunkSeriesSet{} + } + logger.Warnf("[ChunkQuerier]: Select failed: %s", err) return storage.ErrChunkSeriesSet(err) } defer release() - lssQueryResults, snapshots, err := queryLss(LSSQueryChunkQuerySelector, q.head, matchers) + lssQueryResults, snapshots, err := queryLss(lssQueryChunkQuerySelector, q.head, matchers) if err != nil { logger.Warnf("[ChunkQuerier]: failed: %s", err) return storage.ErrChunkSeriesSet(err) } - serializedChunksShards := queryDataStorage(DSQueryChunkQuerier, q.head, lssQueryResults, q.mint, q.maxt) + serializedChunksShards := queryDataStorage(dsQueryChunkQuerier, q.head, lssQueryResults, q.mint, q.maxt) chunkSeriesSets := make([]storage.ChunkSeriesSet, q.head.NumberOfShards()) for shardID, serializedChunks := range serializedChunksShards { if serializedChunks == nil { diff --git a/pp/go/storage/querier/interface.go b/pp/go/storage/querier/interface.go index 766422d6df..6066e89a5f 100644 --- a/pp/go/storage/querier/interface.go +++ b/pp/go/storage/querier/interface.go @@ -26,8 +26,8 @@ type deduplicatorCtor func(numberOfShards uint16) Deduplicator // GenericTask // -// GenericTask the minimum required task [Generic] implementation. -type GenericTask interface { +// Task the minimum required task [Generic] implementation. +type Task interface { // Wait for the task to complete on all shards. Wait() error } @@ -43,6 +43,9 @@ type DataStorage interface { // QueryDataStorage returns serialized chunks from data storage. Query(query cppbridge.HeadDataStorageQuery) *cppbridge.HeadDataStorageSerializedChunks + + // WithRLock calls fn on raw [cppbridge.HeadDataStorage] with read lock. + WithRLock(fn func(ds *cppbridge.HeadDataStorage) error) error } // @@ -69,6 +72,9 @@ type LSS interface { // QuerySelector returns a created selector that matches the given label matchers. QuerySelector(shardID uint16, matchers []model.LabelMatcher) (uintptr, *cppbridge.LabelSetSnapshot, error) + + // WithRLock calls fn on raws [cppbridge.LabelSetStorage] with read lock. + WithRLock(fn func(target, input *cppbridge.LabelSetStorage) error) error } // @@ -93,7 +99,7 @@ type Shard[TDataStorage DataStorage, TLSS LSS] interface { // Head the minimum required [Head] implementation. type Head[ - TGenericTask GenericTask, + TGenericTask Task, TDataStorage DataStorage, TLSS LSS, TShard Shard[TDataStorage, TLSS], diff --git a/pp/go/storage/querier/querier.go b/pp/go/storage/querier/querier.go index 35cbb9bc1c..cf637ebb28 100644 --- a/pp/go/storage/querier/querier.go +++ b/pp/go/storage/querier/querier.go @@ -12,24 +12,25 @@ import ( "github.com/prometheus/prometheus/pp/go/cppbridge" "github.com/prometheus/prometheus/pp/go/model" "github.com/prometheus/prometheus/pp/go/storage/logger" + "github.com/prometheus/prometheus/pp/go/util/locker" "github.com/prometheus/prometheus/storage" "github.com/prometheus/prometheus/util/annotations" ) const ( - // LSSQueryInstantQuerySelector name of task. - LSSQueryInstantQuerySelector = "lss_query_instant_query_selector" - // LSSQueryRangeQuerySelector name of task. - LSSQueryRangeQuerySelector = "lss_query_range_query_selector" - // LSSLabelValuesQuerier name of task. - LSSLabelValuesQuerier = "lss_label_values_querier" - // LSSLabelNamesQuerier name of task. - LSSLabelNamesQuerier = "lss_label_names_querier" - - // DSQueryInstantQuerier name of task. - DSQueryInstantQuerier = "data_storage_query_instant_querier" - // DSQueryRangeQuerier name of task. - DSQueryRangeQuerier = "data_storage_query_range_querier" + // lssQueryInstantQuerySelector name of task. + lssQueryInstantQuerySelector = "lss_query_instant_query_selector" + // lssQueryRangeQuerySelector name of task. + lssQueryRangeQuerySelector = "lss_query_range_query_selector" + // lssLabelValuesQuerier name of task. + lssLabelValuesQuerier = "lss_label_values_querier" + // lssLabelNamesQuerier name of task. + lssLabelNamesQuerier = "lss_label_names_querier" + + // dsQueryInstantQuerier name of task. + dsQueryInstantQuerier = "data_storage_query_instant_querier" + // dsQueryRangeQuerier name of task. + dsQueryRangeQuerier = "data_storage_query_range_querier" // DefaultInstantQueryValueNotFoundTimestampValue default value for not found timestamp value. DefaultInstantQueryValueNotFoundTimestampValue int64 = 0 @@ -41,11 +42,11 @@ const ( // Querier provides querying access over time series data of a fixed time range. type Querier[ - TGenericTask GenericTask, + TTask Task, TDataStorage DataStorage, TLSS LSS, TShard Shard[TDataStorage, TLSS], - THead Head[TGenericTask, TDataStorage, TLSS, TShard], + THead Head[TTask, TDataStorage, TLSS, TShard], ] struct { mint int64 maxt int64 @@ -57,19 +58,19 @@ type Querier[ // NewQuerier init new [Querier]. func NewQuerier[ - TGenericTask GenericTask, + TTask Task, TDataStorage DataStorage, TLSS LSS, TShard Shard[TDataStorage, TLSS], - THead Head[TGenericTask, TDataStorage, TLSS, TShard], + THead Head[TTask, TDataStorage, TLSS, TShard], ]( head THead, deduplicatorCtor deduplicatorCtor, mint, maxt int64, closer func() error, metrics *Metrics, -) *Querier[TGenericTask, TDataStorage, TLSS, TShard, THead] { - return &Querier[TGenericTask, TDataStorage, TLSS, TShard, THead]{ +) *Querier[TTask, TDataStorage, TLSS, TShard, THead] { + return &Querier[TTask, TDataStorage, TLSS, TShard, THead]{ mint: mint, maxt: maxt, head: head, @@ -82,7 +83,7 @@ func NewQuerier[ // Close [Querier] if need. // //revive:disable-next-line:confusing-naming // other type of querier. -func (q *Querier[TGenericTask, TDataStorage, TLSS, TShard, THead]) Close() error { +func (q *Querier[TTask, TDataStorage, TLSS, TShard, THead]) Close() error { if q.closer != nil { return q.closer() } @@ -93,7 +94,7 @@ func (q *Querier[TGenericTask, TDataStorage, TLSS, TShard, THead]) Close() error // LabelNames returns label values present in the head for the specific label name. // //revive:disable-next-line:confusing-naming // other type of querier. -func (q *Querier[TGenericTask, TDataStorage, TLSS, TShard, THead]) LabelNames( +func (q *Querier[TTask, TDataStorage, TLSS, TShard, THead]) LabelNames( ctx context.Context, hints *storage.LabelHints, matchers ...*labels.Matcher, @@ -103,7 +104,7 @@ func (q *Querier[TGenericTask, TDataStorage, TLSS, TShard, THead]) LabelNames( q.head, q.deduplicatorCtor, q.metrics, - LSSLabelNamesQuerier, + lssLabelNamesQuerier, hints, matchers..., ) @@ -114,7 +115,7 @@ func (q *Querier[TGenericTask, TDataStorage, TLSS, TShard, THead]) LabelNames( // result set is reduced to label values of metrics matching the matchers. // //revive:disable-next-line:confusing-naming // other type of querier. -func (q *Querier[TGenericTask, TDataStorage, TLSS, TShard, THead]) LabelValues( +func (q *Querier[TTask, TDataStorage, TLSS, TShard, THead]) LabelValues( ctx context.Context, name string, hints *storage.LabelHints, @@ -126,7 +127,7 @@ func (q *Querier[TGenericTask, TDataStorage, TLSS, TShard, THead]) LabelValues( q.head, q.deduplicatorCtor, q.metrics, - LSSLabelValuesQuerier, + lssLabelValuesQuerier, hints, matchers..., ) @@ -135,7 +136,7 @@ func (q *Querier[TGenericTask, TDataStorage, TLSS, TShard, THead]) LabelValues( // Select returns a set of series that matches the given label matchers. // //revive:disable-next-line:confusing-naming // other type of querier. -func (q *Querier[TGenericTask, TDataStorage, TLSS, TShard, THead]) Select( +func (q *Querier[TTask, TDataStorage, TLSS, TShard, THead]) Select( ctx context.Context, sortSeries bool, hints *storage.SelectHints, @@ -150,7 +151,7 @@ func (q *Querier[TGenericTask, TDataStorage, TLSS, TShard, THead]) Select( // selectInstant returns a instant set of series that matches the given label matchers. // //revive:disable-next-line:function-length long but readable. -func (q *Querier[TGenericTask, TDataStorage, TLSS, TShard, THead]) selectInstant( +func (q *Querier[TTask, TDataStorage, TLSS, TShard, THead]) selectInstant( ctx context.Context, _ bool, _ *storage.SelectHints, @@ -160,6 +161,10 @@ func (q *Querier[TGenericTask, TDataStorage, TLSS, TShard, THead]) selectInstant release, err := q.head.AcquireQuery(ctx) if err != nil { + if errors.Is(err, locker.ErrSemaphoreClosed) { + return &SeriesSet{} + } + logger.Warnf("[QUERIER]: select instant failed on the capture of the read lock query: %s", err) return storage.ErrSeriesSet(err) } @@ -173,7 +178,7 @@ func (q *Querier[TGenericTask, TDataStorage, TLSS, TShard, THead]) selectInstant } }() - lssQueryResults, snapshots, err := queryLss(LSSQueryInstantQuerySelector, q.head, matchers) + lssQueryResults, snapshots, err := queryLss(lssQueryInstantQuerySelector, q.head, matchers) if err != nil { logger.Warnf("[QUERIER]: failed to instant: %s", err) return storage.ErrSeriesSet(err) @@ -187,7 +192,7 @@ func (q *Querier[TGenericTask, TDataStorage, TLSS, TShard, THead]) selectInstant numberOfShards := q.head.NumberOfShards() seriesSets := make([]storage.SeriesSet, numberOfShards) tDataStorageQuery := q.head.CreateTask( - DSQueryInstantQuerier, + dsQueryInstantQuerier, func(shard TShard) error { shardID := shard.ShardID() lssQueryResult := lssQueryResults[shardID] @@ -213,7 +218,7 @@ func (q *Querier[TGenericTask, TDataStorage, TLSS, TShard, THead]) selectInstant } // selectRange returns a range set of series that matches the given label matchers. -func (q *Querier[TGenericTask, TDataStorage, TLSS, TShard, THead]) selectRange( +func (q *Querier[TTask, TDataStorage, TLSS, TShard, THead]) selectRange( ctx context.Context, _ bool, _ *storage.SelectHints, @@ -223,6 +228,10 @@ func (q *Querier[TGenericTask, TDataStorage, TLSS, TShard, THead]) selectRange( release, err := q.head.AcquireQuery(ctx) if err != nil { + if errors.Is(err, locker.ErrSemaphoreClosed) { + return &SeriesSet{} + } + logger.Warnf("[QUERIER]: select range failed on the capture of the read lock query: %s", err) return storage.ErrSeriesSet(err) } @@ -236,13 +245,13 @@ func (q *Querier[TGenericTask, TDataStorage, TLSS, TShard, THead]) selectRange( } }() - lssQueryResults, snapshots, err := queryLss(LSSQueryRangeQuerySelector, q.head, matchers) + lssQueryResults, snapshots, err := queryLss(lssQueryRangeQuerySelector, q.head, matchers) if err != nil { logger.Warnf("[QUERIER]: failed to range: %s", err) return storage.ErrSeriesSet(err) } - serializedChunksShards := queryDataStorage(DSQueryRangeQuerier, q.head, lssQueryResults, q.mint, q.maxt) + serializedChunksShards := queryDataStorage(dsQueryRangeQuerier, q.head, lssQueryResults, q.mint, q.maxt) seriesSets := make([]storage.SeriesSet, q.head.NumberOfShards()) for shardID, serializedChunksShard := range serializedChunksShards { if serializedChunksShard == nil { @@ -280,11 +289,11 @@ func convertPrometheusMatchersToPPMatchers(matchers ...*labels.Matcher) []model. // queryDataStorage returns serialized chunks from data storage for each shard. func queryDataStorage[ - TGenericTask GenericTask, + TTask Task, TDataStorage DataStorage, TLSS LSS, TShard Shard[TDataStorage, TLSS], - THead Head[TGenericTask, TDataStorage, TLSS, TShard], + THead Head[TTask, TDataStorage, TLSS, TShard], ]( taskName string, head THead, @@ -324,11 +333,11 @@ func queryDataStorage[ // queryLabelValues returns label values present in the head for the specific label name. func queryLabelNames[ - TGenericTask GenericTask, + TTask Task, TDataStorage DataStorage, TLSS LSS, TShard Shard[TDataStorage, TLSS], - THead Head[TGenericTask, TDataStorage, TLSS, TShard], + THead Head[TTask, TDataStorage, TLSS, TShard], ]( ctx context.Context, head THead, @@ -343,6 +352,10 @@ func queryLabelNames[ anns := *annotations.New() release, err := head.AcquireQuery(ctx) if err != nil { + if errors.Is(err, locker.ErrSemaphoreClosed) { + return nil, anns, nil + } + logger.Warnf("[QUERIER]: label names failed on the capture of the read lock query: %s", err) return nil, anns, err } @@ -386,11 +399,11 @@ func queryLabelNames[ // queryLabelValues returns label values present in the head for the specific label name. func queryLabelValues[ - TGenericTask GenericTask, + TTask Task, TDataStorage DataStorage, TLSS LSS, TShard Shard[TDataStorage, TLSS], - THead Head[TGenericTask, TDataStorage, TLSS, TShard], + THead Head[TTask, TDataStorage, TLSS, TShard], ]( ctx context.Context, name string, @@ -406,6 +419,10 @@ func queryLabelValues[ anns := *annotations.New() release, err := head.AcquireQuery(ctx) if err != nil { + if errors.Is(err, locker.ErrSemaphoreClosed) { + return nil, anns, nil + } + logger.Warnf("[QUERIER]: label values failed on the capture of the read lock query: %s", err) return nil, anns, err } @@ -449,11 +466,11 @@ func queryLabelValues[ //revive:disable-next-line:cyclomatic but readable. //revive:disable-next-line:function-length long but readable. func queryLss[ - TGenericTask GenericTask, + TTask Task, TDataStorage DataStorage, TLSS LSS, TShard Shard[TDataStorage, TLSS], - THead Head[TGenericTask, TDataStorage, TLSS, TShard], + THead Head[TTask, TDataStorage, TLSS, TShard], ]( taskName string, head THead, diff --git a/pp/go/storage/querier/querier_test.go b/pp/go/storage/querier/querier_test.go index 4baa1425b8..3e182ecf8d 100644 --- a/pp/go/storage/querier/querier_test.go +++ b/pp/go/storage/querier/querier_test.go @@ -52,6 +52,11 @@ func (*testWal) Commit() error { return nil } +// CurrentSize test implementation wal. +func (*testWal) CurrentSize() int64 { + return 0 +} + // Flush test implementation wal. func (*testWal) Flush() error { return nil diff --git a/pp/go/storage/querier/status_querier.go b/pp/go/storage/querier/status_querier.go new file mode 100644 index 0000000000..a46022af64 --- /dev/null +++ b/pp/go/storage/querier/status_querier.go @@ -0,0 +1,188 @@ +package querier + +import ( + "context" + "errors" + "math" + "sort" + + "github.com/prometheus/prometheus/pp/go/cppbridge" + "github.com/prometheus/prometheus/pp/go/storage/head/task" + "github.com/prometheus/prometheus/pp/go/storage/logger" + "github.com/prometheus/prometheus/pp/go/util/locker" +) + +const ( + // dsHeadStatus name of task. + dsHeadStatus = "data_storage_head_status" + + // lssHeadStatus name of task. + lssHeadStatus = "lss_head_status" +) + +// QueryHeadStatus returns [HeadStatus] holds information about all shards from [Head]. +func QueryHeadStatus[ + TTask Task, + TDataStorage DataStorage, + TLSS LSS, + TShard Shard[TDataStorage, TLSS], + THead Head[TTask, TDataStorage, TLSS, TShard], +]( + ctx context.Context, + head THead, + limit int, +) (*HeadStatus, error) { + shardStatuses := make([]*cppbridge.HeadStatus, head.NumberOfShards()) + for i := range shardStatuses { + shardStatuses[i] = cppbridge.NewHeadStatus() + } + + tw := task.NewTaskWaiter[TTask](2) //revive:disable-line:add-constant // 2 task for wait + + release, err := head.AcquireQuery(ctx) + if err != nil { + if !errors.Is(err, locker.ErrSemaphoreClosed) { + logger.Warnf("[HeadStatusQuerier]: query status failed on the capture of the read lock query: %s", err) + } + + return nil, err + } + defer release() + + tLSSHeadStatus := head.CreateTask( + lssHeadStatus, + func(shard TShard) error { + return shard.LSS().WithRLock(func(target, _ *cppbridge.LabelSetStorage) error { + shardStatuses[shard.ShardID()].FromLSS(target, limit) + + return nil + }) + }, + ) + head.Enqueue(tLSSHeadStatus) + + if limit != 0 { + tDataStorageHeadStatus := head.CreateTask( + dsHeadStatus, + func(shard TShard) error { + return shard.DataStorage().WithRLock(func(ds *cppbridge.HeadDataStorage) error { + shardStatuses[shard.ShardID()].FromDataStorage(ds) + + return nil + }) + }, + ) + head.Enqueue(tDataStorageHeadStatus) + tw.Add(tDataStorageHeadStatus) + } + + tw.Add(tLSSHeadStatus) + _ = tw.Wait() + + return sumStatuses(shardStatuses, limit), nil +} + +// sumStatuses summarize the statuses received from the shards. +func sumStatuses(shardStatuses []*cppbridge.HeadStatus, limit int) *HeadStatus { + seriesStats := make(map[string]uint64) + labelsStats := make(map[string]uint64) + memoryStats := make(map[string]uint64) + countStats := make(map[string]uint64) + + headStatus := &HeadStatus{HeadStats: HeadStats{MinTime: math.MaxInt64, MaxTime: math.MinInt64}} + + for _, shardStatus := range shardStatuses { + headStatus.HeadStats.NumSeries += uint64(shardStatus.NumSeries) + if limit == 0 { + continue + } + + headStatus.HeadStats.ChunkCount += int64(shardStatus.ChunkCount) + if headStatus.HeadStats.MaxTime < shardStatus.TimeInterval.Max { + headStatus.HeadStats.MaxTime = shardStatus.TimeInterval.Max + } + if headStatus.HeadStats.MinTime > shardStatus.TimeInterval.Min { + headStatus.HeadStats.MinTime = shardStatus.TimeInterval.Min + } + + headStatus.HeadStats.NumLabelPairs += int(shardStatus.NumLabelPairs) + + for _, stat := range shardStatus.SeriesCountByMetricName { + seriesStats[stat.Name] += uint64(stat.Count) + } + for _, stat := range shardStatus.LabelValueCountByLabelName { + labelsStats[stat.Name] += uint64(stat.Count) + } + for _, stat := range shardStatus.MemoryInBytesByLabelName { + memoryStats[stat.Name] += uint64(stat.Size) + } + for _, stat := range shardStatus.SeriesCountByLabelValuePair { + countStats[stat.Name+"="+stat.Value] += uint64(stat.Count) + } + } + + if limit == 0 { + return headStatus + } + + headStatus.SeriesCountByMetricName = getSortedStats(seriesStats, limit) + headStatus.LabelValueCountByLabelName = getSortedStats(labelsStats, limit) + headStatus.MemoryInBytesByLabelName = getSortedStats(memoryStats, limit) + headStatus.SeriesCountByLabelValuePair = getSortedStats(countStats, limit) + + return headStatus +} + +// getSortedStats returns sorted statistics for the [Head]. +func getSortedStats(stats map[string]uint64, limit int) []HeadStat { + result := make([]HeadStat, 0, len(stats)) + for k, v := range stats { + result = append(result, HeadStat{ + Name: k, + Value: v, + }) + } + + sort.Slice(result, func(i, j int) bool { + return result[i].Value > result[j].Value + }) + + if len(result) > limit { + return result[:limit] + } + + return result +} + +// +// HeadStatus +// + +// HeadStatus holds information about all shards. +type HeadStatus struct { + HeadStats HeadStats `json:"headStats"` + SeriesCountByMetricName []HeadStat `json:"seriesCountByMetricName"` + LabelValueCountByLabelName []HeadStat `json:"labelValueCountByLabelName"` + MemoryInBytesByLabelName []HeadStat `json:"memoryInBytesByLabelName"` + SeriesCountByLabelValuePair []HeadStat `json:"seriesCountByLabelValuePair"` +} + +// NumSeries returns number of series. +func (hs *HeadStatus) NumSeries() uint64 { + return hs.HeadStats.NumSeries +} + +// HeadStat holds the information about individual cardinality. +type HeadStat struct { + Name string `json:"name"` + Value uint64 `json:"value"` +} + +// HeadStats has information about the head. +type HeadStats struct { + NumSeries uint64 `json:"numSeries"` + NumLabelPairs int `json:"numLabelPairs"` + ChunkCount int64 `json:"chunkCount"` + MinTime int64 `json:"minTime"` + MaxTime int64 `json:"maxTime"` +} diff --git a/pp/go/util/closer.go b/pp/go/util/closer.go index f32bbf988e..deedf3f222 100644 --- a/pp/go/util/closer.go +++ b/pp/go/util/closer.go @@ -1,19 +1,27 @@ package util +import "sync" + type Closer struct { - close chan struct{} - closed chan struct{} + close chan struct{} + closeOnce sync.Once + closed chan struct{} + closedOnce sync.Once } func NewCloser() *Closer { return &Closer{ - close: make(chan struct{}), - closed: make(chan struct{}), + close: make(chan struct{}), + closeOnce: sync.Once{}, + closed: make(chan struct{}), + closedOnce: sync.Once{}, } } func (c *Closer) Done() { - close(c.closed) + c.closedOnce.Do(func() { + close(c.closed) + }) } func (c *Closer) Signal() <-chan struct{} { @@ -21,7 +29,9 @@ func (c *Closer) Signal() <-chan struct{} { } func (c *Closer) Close() error { - close(c.close) + c.closeOnce.Do(func() { + close(c.close) + }) <-c.closed return nil } diff --git a/pp/go/util/locker/priweighted.go b/pp/go/util/locker/priweighted.go index aa1a2f7a54..4b0108a289 100644 --- a/pp/go/util/locker/priweighted.go +++ b/pp/go/util/locker/priweighted.go @@ -36,8 +36,8 @@ type Weighted struct { // Close sets the flag that the semaphore is closed under the priority lock // and after unlocking all those waiting will receive the error [ErrSemaphoreClosed]. -func (s *Weighted) Close(ctx context.Context) error { - unlock, err := s.LockWithPriority(ctx) +func (s *Weighted) Close() error { + unlock, err := s.LockWithPriority(context.Background()) if err != nil { return err } From 42192e8142baa96695ff4c60efbadfccde5e8e26 Mon Sep 17 00:00:00 2001 From: Alexandr Yudin Date: Mon, 8 Sep 2025 13:40:13 +0000 Subject: [PATCH 30/96] fix merge --- pp-pkg/receiver/receiver.go | 2 +- pp/go/relabeler/head/headtest/fixtures.go | 6 +-- pp/go/relabeler/head/load_test.go | 20 ++++----- pp/go/relabeler/headcontainer/active.go | 11 +++++ pp/go/relabeler/querier/chunk_querier_test.go | 25 +++-------- pp/go/relabeler/querier/querier_test.go | 40 ++++------------- pp/go/storage/appender/appender_test.go | 7 ++- pp/go/storage/head/head/head_test.go | 5 +++ pp/go/storage/head/services/commiter.go | 15 +++++-- pp/go/storage/head/services/functions.go | 38 +++++++++++++--- pp/go/storage/head/services/interface.go | 3 ++ pp/go/storage/head/services/merger.go | 14 +++++- pp/go/storage/head/services/rotator.go | 2 +- pp/go/storage/head/shard/data_storage.go | 24 +++++++---- pp/go/storage/head/shard/shard.go | 14 ++++-- .../storage/head/shard/wal/reader/segment.go | 5 +++ pp/go/storage/head/shard/wal/wal.go | 17 ++++++++ .../storage/head/shard/wal/writer/buffered.go | 36 +++++++--------- pp/go/storage/manager.go | 43 +++++++++++-------- pp/go/storage/querier/querier_test.go | 5 +++ pp/go/storage/remotewriter/datasource.go | 4 +- pp/go/storage/remotewriter/walreader.go | 10 ++--- web/web_fuzzy_test.go | 3 ++ 23 files changed, 212 insertions(+), 137 deletions(-) diff --git a/pp-pkg/receiver/receiver.go b/pp-pkg/receiver/receiver.go index d1a8fbf8aa..45acb7eb32 100644 --- a/pp-pkg/receiver/receiver.go +++ b/pp-pkg/receiver/receiver.go @@ -193,7 +193,7 @@ func NewReceiver( return nil, fmt.Errorf("failed to create head manager: %w", err) } - activeHead, rotatedHeads, err := headManager.Restore(rotationInfo.BlockDuration, unloadDataStorageInterval) + currentHead, rotatedHeads, err := headManager.Restore(rotationInfo.BlockDuration, unloadDataStorageInterval) if err != nil { return nil, fmt.Errorf("failed to restore heads: %w", err) } diff --git a/pp/go/relabeler/head/headtest/fixtures.go b/pp/go/relabeler/head/headtest/fixtures.go index 33d94c6a02..43ae38b935 100644 --- a/pp/go/relabeler/head/headtest/fixtures.go +++ b/pp/go/relabeler/head/headtest/fixtures.go @@ -23,9 +23,9 @@ func (s *TimeSeries) AppendSamples(samples ...cppbridge.Sample) { func (s *TimeSeries) toModelTimeSeries() []model.TimeSeries { lsBuilder := model.NewLabelSetBuilder() - for i := range s.Labels { - lsBuilder.Add(s.Labels[i].Name, s.Labels[i].Value) - } + s.Labels.Range(func(l labels.Label) { + lsBuilder.Add(l.Name, l.Value) + }) ls := lsBuilder.Build() diff --git a/pp/go/relabeler/head/load_test.go b/pp/go/relabeler/head/load_test.go index 6bff0360da..8bf4d37350 100644 --- a/pp/go/relabeler/head/load_test.go +++ b/pp/go/relabeler/head/load_test.go @@ -173,9 +173,10 @@ func (s *HeadLoadSuite) TestErrorOpenShardFileInAllShards() { func (s *HeadLoadSuite) TestLoadWithDisabledDataUnloading() { // Arrange sourceHead := s.mustCreateHead(0) + series := []headtest.TimeSeries{ { - Labels: labels.Labels{{Name: "__name__", Value: "wal_metric"}}, + Labels: labels.FromStrings("__name__", "wal_metric"), Samples: []cppbridge.Sample{ {Timestamp: 0, Value: 1}, {Timestamp: 1, Value: 2}, @@ -208,7 +209,7 @@ func (s *HeadLoadSuite) TestAppendAfterLoad() { sourceHead := s.mustCreateHead(0) series := []headtest.TimeSeries{ { - Labels: labels.Labels{{Name: "__name__", Value: "wal_metric"}}, + Labels: labels.FromStrings("__name__", "wal_metric"), Samples: []cppbridge.Sample{ {Timestamp: 0, Value: 1}, {Timestamp: 1, Value: 2}, @@ -224,7 +225,7 @@ func (s *HeadLoadSuite) TestAppendAfterLoad() { loadedHead := s.mustLoadHead(0) s.appendTimeSeries(loadedHead, []headtest.TimeSeries{ { - Labels: labels.Labels{{Name: "__name__", Value: "wal_metric"}}, + Labels: labels.FromStrings("__name__", "wal_metric"), Samples: []cppbridge.Sample{ {Timestamp: 3, Value: 4}, }, @@ -239,7 +240,6 @@ func (s *HeadLoadSuite) TestAppendAfterLoad() { series[0].Samples = append(series[0].Samples, cppbridge.Sample{Timestamp: 3, Value: 4}) s.Equal(series, actual) s.Require().NoError(err) - } func (s *HeadLoadSuite) TestLoadWithEnabledDataUnloading() { @@ -247,7 +247,7 @@ func (s *HeadLoadSuite) TestLoadWithEnabledDataUnloading() { sourceHead := s.mustCreateHead(0) series1 := []headtest.TimeSeries{ { - Labels: labels.Labels{{Name: "__name__", Value: "wal_metric"}}, + Labels: labels.FromStrings("__name__", "wal_metric"), Samples: []cppbridge.Sample{ {Timestamp: 0, Value: 1}, {Timestamp: 1, Value: 2}, @@ -258,7 +258,7 @@ func (s *HeadLoadSuite) TestLoadWithEnabledDataUnloading() { s.appendTimeSeries(sourceHead, series1) series2 := []headtest.TimeSeries{ { - Labels: labels.Labels{{Name: "__name__", Value: "wal_metric"}}, + Labels: labels.FromStrings("__name__", "wal_metric"), Samples: []cppbridge.Sample{ {Timestamp: 100, Value: 1}, {Timestamp: 101, Value: 2}, @@ -290,7 +290,7 @@ func (s *HeadLoadSuite) TestLoadWithDataUnloading() { sourceHead := s.mustCreateHead(unloadDataStorageInterval) series1 := []headtest.TimeSeries{ { - Labels: labels.Labels{{Name: "__name__", Value: "wal_metric"}}, + Labels: labels.FromStrings("__name__", "wal_metric"), Samples: []cppbridge.Sample{ {Timestamp: 0, Value: 1}, {Timestamp: 1, Value: 2}, @@ -301,7 +301,7 @@ func (s *HeadLoadSuite) TestLoadWithDataUnloading() { s.appendTimeSeries(sourceHead, series1) series2 := []headtest.TimeSeries{ { - Labels: labels.Labels{{Name: "__name__", Value: "wal_metric"}}, + Labels: labels.FromStrings("__name__", "wal_metric"), Samples: []cppbridge.Sample{ {Timestamp: 100, Value: 1}, {Timestamp: 101, Value: 2}, @@ -334,7 +334,7 @@ func (s *HeadLoadSuite) TestErrorDataUnloading() { sourceHead := s.mustCreateHead(unloadDataStorageInterval) series1 := []headtest.TimeSeries{ { - Labels: labels.Labels{{Name: "__name__", Value: "wal_metric"}}, + Labels: labels.FromStrings("__name__", "wal_metric"), Samples: []cppbridge.Sample{ {Timestamp: 0, Value: 1}, {Timestamp: 1, Value: 2}, @@ -345,7 +345,7 @@ func (s *HeadLoadSuite) TestErrorDataUnloading() { s.appendTimeSeries(sourceHead, series1) series2 := []headtest.TimeSeries{ { - Labels: labels.Labels{{Name: "__name__", Value: "wal_metric"}}, + Labels: labels.FromStrings("__name__", "wal_metric"), Samples: []cppbridge.Sample{ {Timestamp: 100, Value: 1}, {Timestamp: 101, Value: 2}, diff --git a/pp/go/relabeler/headcontainer/active.go b/pp/go/relabeler/headcontainer/active.go index edb113165b..4297c0d771 100644 --- a/pp/go/relabeler/headcontainer/active.go +++ b/pp/go/relabeler/headcontainer/active.go @@ -212,6 +212,17 @@ func (h *Active) Rotate(ctx context.Context) error { return nil } +func (h *Active) UnloadUnusedSeriesData(ctx context.Context) { + runlock, err := h.wlocker.RLock(ctx) + if err != nil { + logger.Warnf("[ActiveHead] UnloadUnusedSeriesData: weighted locker: %s", err) + return + } + defer runlock() + + h.head.UnloadUnusedSeriesData() +} + func (h *Active) WriteMetrics(ctx context.Context) { runlock, err := h.wlocker.RLock(ctx) if err != nil { diff --git a/pp/go/relabeler/querier/chunk_querier_test.go b/pp/go/relabeler/querier/chunk_querier_test.go index 59f6fc992b..337959b56e 100644 --- a/pp/go/relabeler/querier/chunk_querier_test.go +++ b/pp/go/relabeler/querier/chunk_querier_test.go @@ -50,29 +50,20 @@ func (s *ChunkQuerierTestSuite) TestSelect() { // Arrange timeSeries := []headtest.TimeSeries{ { - Labels: labels.Labels{ - {Name: "__name__", Value: "metric"}, - {Name: "job", Value: "test"}, - }, + Labels: labels.FromStrings("__name__", "metric", "job", "test"), Samples: []cppbridge.Sample{ {Timestamp: 0, Value: 1}, {Timestamp: 1, Value: 1}, }, }, { - Labels: labels.Labels{ - {Name: "__name__", Value: "metric"}, - {Name: "job", Value: "test2"}, - }, + Labels: labels.FromStrings("__name__", "metric", "job", "test2"), Samples: []cppbridge.Sample{ {Timestamp: 0, Value: 10}, }, }, { - Labels: labels.Labels{ - {Name: "__name__", Value: "metric"}, - {Name: "job", Value: "test3"}, - }, + Labels: labels.FromStrings("__name__", "metric", "job", "test3"), Samples: []cppbridge.Sample{ {Timestamp: 10, Value: 10}, }, @@ -98,10 +89,7 @@ func (s *ChunkQuerierTestSuite) TestSelectWithDataStorageLoading() { // Arrange timeSeries := []headtest.TimeSeries{ { - Labels: labels.Labels{ - {Name: "__name__", Value: "metric"}, - {Name: "job", Value: "test"}, - }, + Labels: labels.FromStrings("__name__", "metric", "job", "test"), Samples: []cppbridge.Sample{ {Timestamp: 0, Value: 0}, {Timestamp: 1, Value: 1}, @@ -110,10 +98,7 @@ func (s *ChunkQuerierTestSuite) TestSelectWithDataStorageLoading() { }, }, { - Labels: labels.Labels{ - {Name: "__name__", Value: "metric"}, - {Name: "job", Value: "test2"}, - }, + Labels: labels.FromStrings("__name__", "metric", "job", "test2"), Samples: []cppbridge.Sample{ {Timestamp: 0, Value: 10}, {Timestamp: 1, Value: 11}, diff --git a/pp/go/relabeler/querier/querier_test.go b/pp/go/relabeler/querier/querier_test.go index 259fe913b1..282003da7a 100644 --- a/pp/go/relabeler/querier/querier_test.go +++ b/pp/go/relabeler/querier/querier_test.go @@ -66,19 +66,13 @@ func (s *QuerierTestSuite) TestRangeQuery() { // Arrange timeSeries := []headtest.TimeSeries{ { - Labels: labels.Labels{ - {Name: "__name__", Value: "metric"}, - {Name: "job", Value: "test"}, - }, + Labels: labels.FromStrings("__name__", "metric", "job", "test"), Samples: []cppbridge.Sample{ {Timestamp: 0, Value: 1}, }, }, { - Labels: labels.Labels{ - {Name: "__name__", Value: "metric"}, - {Name: "job", Value: "test2"}, - }, + Labels: labels.FromStrings("__name__", "metric", "job", "test2"), Samples: []cppbridge.Sample{ {Timestamp: 0, Value: 10}, }, @@ -101,10 +95,7 @@ func (s *QuerierTestSuite) TestRangeQueryWithDataStorageLoading() { // Arrange timeSeries := []headtest.TimeSeries{ { - Labels: labels.Labels{ - {Name: "__name__", Value: "metric"}, - {Name: "job", Value: "test"}, - }, + Labels: labels.FromStrings("__name__", "metric", "job", "test"), Samples: []cppbridge.Sample{ {Timestamp: 0, Value: 0}, {Timestamp: 1, Value: 1}, @@ -112,10 +103,7 @@ func (s *QuerierTestSuite) TestRangeQueryWithDataStorageLoading() { }, }, { - Labels: labels.Labels{ - {Name: "__name__", Value: "metric"}, - {Name: "job", Value: "test2"}, - }, + Labels: labels.FromStrings("__name__", "metric", "job", "test2"), Samples: []cppbridge.Sample{ {Timestamp: 0, Value: 10}, {Timestamp: 1, Value: 11}, @@ -159,19 +147,13 @@ func (s *QuerierTestSuite) TestInstantQuery() { // Arrange timeSeries := []headtest.TimeSeries{ { - Labels: labels.Labels{ - {Name: "__name__", Value: "metric"}, - {Name: "job", Value: "test"}, - }, + Labels: labels.FromStrings("__name__", "metric", "job", "test"), Samples: []cppbridge.Sample{ {Timestamp: 0, Value: 1}, }, }, { - Labels: labels.Labels{ - {Name: "__name__", Value: "metric"}, - {Name: "job", Value: "test2"}, - }, + Labels: labels.FromStrings("__name__", "metric", "job", "test2"), Samples: []cppbridge.Sample{ {Timestamp: 0, Value: 10}, }, @@ -194,10 +176,7 @@ func (s *QuerierTestSuite) TestInstantQueryWithDataStorageLoading() { // Arrange timeSeries := []headtest.TimeSeries{ { - Labels: labels.Labels{ - {Name: "__name__", Value: "metric"}, - {Name: "job", Value: "test"}, - }, + Labels: labels.FromStrings("__name__", "metric", "job", "test"), Samples: []cppbridge.Sample{ {Timestamp: 0, Value: 0}, {Timestamp: 1, Value: 1}, @@ -205,10 +184,7 @@ func (s *QuerierTestSuite) TestInstantQueryWithDataStorageLoading() { }, }, { - Labels: labels.Labels{ - {Name: "__name__", Value: "metric"}, - {Name: "job", Value: "test2"}, - }, + Labels: labels.FromStrings("__name__", "metric", "job", "test2"), Samples: []cppbridge.Sample{ {Timestamp: 0, Value: 10}, {Timestamp: 1, Value: 11}, diff --git a/pp/go/storage/appender/appender_test.go b/pp/go/storage/appender/appender_test.go index 87af145fc0..cbf0eb1607 100644 --- a/pp/go/storage/appender/appender_test.go +++ b/pp/go/storage/appender/appender_test.go @@ -27,7 +27,7 @@ func TestXxx(t *testing.T) { nil, ) - app := appender.New(h, services.CommitAndFlushViaRange) + app := appender.New(h, services.CFViaRange) _ = app @@ -57,6 +57,11 @@ func (*testWal) Flush() error { return nil } +// Sync test implementation wal. +func (*testWal) Sync() error { + return nil +} + // Write test implementation wal. func (*testWal) Write(_ []*cppbridge.InnerSeries) (bool, error) { return false, nil diff --git a/pp/go/storage/head/head/head_test.go b/pp/go/storage/head/head/head_test.go index 304912fe9c..4b8d9f32ab 100644 --- a/pp/go/storage/head/head/head_test.go +++ b/pp/go/storage/head/head/head_test.go @@ -50,6 +50,11 @@ func (*testWal) Flush() error { return nil } +// Sync test implementation wal. +func (*testWal) Sync() error { + return nil +} + // Write test implementation wal. func (*testWal) Write(_ []*cppbridge.InnerSeries) (bool, error) { return false, nil diff --git a/pp/go/storage/head/services/commiter.go b/pp/go/storage/head/services/commiter.go index 0efa664890..38711a121b 100644 --- a/pp/go/storage/head/services/commiter.go +++ b/pp/go/storage/head/services/commiter.go @@ -19,6 +19,7 @@ type Committer[ ] struct { activeHead ActiveHeadContainer[TTask, TShard, TGoShard, THead] m Mediator + isNewHead func(headID string) bool } // NewCommitter init new [Committer]. @@ -29,10 +30,12 @@ func NewCommitter[ ]( activeHead ActiveHeadContainer[TTask, TShard, TGoShard, THead], m Mediator, + isNewHead func(headID string) bool, ) *Committer[TTask, TShard, TGoShard, THead] { return &Committer[TTask, TShard, TGoShard, THead]{ activeHead: activeHead, m: m, + isNewHead: isNewHead, } } @@ -43,7 +46,7 @@ func (s *Committer[TTask, TShard, TGoShard, THead]) Execute(ctx context.Context) logger.Infof("The Committer is running.") for range s.m.C() { - if err := s.activeHead.With(ctx, s.commitAndFlushViaRange); err != nil { + if err := s.activeHead.With(ctx, s.commitFlushSync); err != nil { logger.Errorf("wal commit failed: %v", err) } } @@ -53,6 +56,12 @@ func (s *Committer[TTask, TShard, TGoShard, THead]) Execute(ctx context.Context) return nil } -func (s *Committer[TTask, TShard, TGoShard, THead]) commitAndFlushViaRange(h THead) error { - return CommitAndFlushViaRange(h) +// commitFlushSync finalize segment from encoder and add to wal +// and flush wal segment writer, write all buffered data to storage and sync, do via range. +func (s *Committer[TTask, TShard, TGoShard, THead]) commitFlushSync(h THead) error { + if s.isNewHead(h.ID()) { + return nil + } + + return CFSViaRange(h) } diff --git a/pp/go/storage/head/services/functions.go b/pp/go/storage/head/services/functions.go index a854616a8e..0295fa2b38 100644 --- a/pp/go/storage/head/services/functions.go +++ b/pp/go/storage/head/services/functions.go @@ -10,15 +10,13 @@ const ( dsMergeOutOfOrderChunks = "data_storage_merge_out_of_order_chunks" ) -// TODO appender Commit - // -// CommitAndFlushViaRange +// Commit, Flush, Sync // -// CommitAndFlushViaRange finalize segment from encoder and add to wal -// and flush wal segment writer, write all buffered data to storage, do via range. -func CommitAndFlushViaRange[ +// CFViaRange finalize segment from encoder and add to wal +// and flush wal segment writer, write all buffered data to storage without sync, do via range. +func CFViaRange[ TTask Task, TShard, TGoShard Shard, THead Head[TTask, TShard, TGoShard], @@ -37,6 +35,34 @@ func CommitAndFlushViaRange[ return errors.Join(errs...) } +// CFSViaRange finalize segment from encoder and add to wal +// and flush wal segment writer, write all buffered data to storage and sync, do via range. +func CFSViaRange[ + TTask Task, + TShard, TGoShard Shard, + THead Head[TTask, TShard, TGoShard], +](h THead) error { + errs := make([]error, 0, h.NumberOfShards()*3) + for shard := range h.RangeShards() { + if err := shard.WalCommit(); err != nil { + errs = append(errs, fmt.Errorf("commit shard id %d: %w", shard.ShardID(), err)) + } + + if err := shard.WalFlush(); err != nil { + errs = append(errs, fmt.Errorf("flush shard id %d: %w", shard.ShardID(), err)) + + // if the flush operation fails, skip the Sinc + continue + } + + if err := shard.WalSync(); err != nil { + errs = append(errs, fmt.Errorf("sync shard id %d: %w", shard.ShardID(), err)) + } + } + + return errors.Join(errs...) +} + // // MergeOutOfOrderChunksWithHead // diff --git a/pp/go/storage/head/services/interface.go b/pp/go/storage/head/services/interface.go index 89df608104..903f3fea6a 100644 --- a/pp/go/storage/head/services/interface.go +++ b/pp/go/storage/head/services/interface.go @@ -138,6 +138,9 @@ type Shard interface { // WalFlush flush all contetnt into wal. WalFlush() error + + // WalSync commits the current contents of the [Wal]. + WalSync() error } // diff --git a/pp/go/storage/head/services/merger.go b/pp/go/storage/head/services/merger.go index 6a6e22d514..3de58cb3c6 100644 --- a/pp/go/storage/head/services/merger.go +++ b/pp/go/storage/head/services/merger.go @@ -18,6 +18,7 @@ type Merger[ ] struct { activeHead ActiveHeadContainer[TTask, TShard, TGoShard, THead] m Mediator + isNewHead func(headID string) bool } // NewMerger init new [Merger]. @@ -28,10 +29,12 @@ func NewMerger[ ]( activeHead ActiveHeadContainer[TTask, TShard, TGoShard, THead], m Mediator, + isNewHead func(headID string) bool, ) *Merger[TTask, TShard, TGoShard, THead] { return &Merger[TTask, TShard, TGoShard, THead]{ activeHead: activeHead, m: m, + isNewHead: isNewHead, } } @@ -42,7 +45,7 @@ func (s *Merger[TTask, TShard, TGoShard, THead]) Execute(ctx context.Context) er logger.Infof("The Merger is running.") for range s.m.C() { - if err := s.activeHead.With(ctx, MergeOutOfOrderChunksWithHead); err != nil { + if err := s.activeHead.With(ctx, s.mergeOutOfOrderChunks); err != nil { logger.Errorf("data storage merge failed: %v", err) } } @@ -51,3 +54,12 @@ func (s *Merger[TTask, TShard, TGoShard, THead]) Execute(ctx context.Context) er return nil } + +// mergeOutOfOrderChunksWithHead merge chunks with out of order data chunks for [Head]. +func (s *Merger[TTask, TShard, TGoShard, THead]) mergeOutOfOrderChunks(h THead) error { + if s.isNewHead(h.ID()) { + return nil + } + + return MergeOutOfOrderChunksWithHead(h) +} diff --git a/pp/go/storage/head/services/rotator.go b/pp/go/storage/head/services/rotator.go index 803afaff9a..03e2e5cdda 100644 --- a/pp/go/storage/head/services/rotator.go +++ b/pp/go/storage/head/services/rotator.go @@ -136,7 +136,7 @@ func (s *Rotator[TTask, TShard, TGoShard, THead]) rotate( logger.Warnf("failed merge out of order chunks in data storage: %s", err) } - if err = CommitAndFlushViaRange(oldHead); err != nil { + if err = CFSViaRange(oldHead); err != nil { logger.Warnf("failed commit and flush to wal: %s", err) } diff --git a/pp/go/storage/head/shard/data_storage.go b/pp/go/storage/head/shard/data_storage.go index 27ea8d83d9..480c83a9a3 100644 --- a/pp/go/storage/head/shard/data_storage.go +++ b/pp/go/storage/head/shard/data_storage.go @@ -40,20 +40,24 @@ func (ds *DataStorage) AppendInnerSeriesSlice(innerSeriesSlice []*cppbridge.Inne } // DecodeSegment decode segment data from decoder [cppbridge.HeadWalDecoder] -// and add to encoder [cppbridge.HeadEncoder]. -func (ds *DataStorage) DecodeSegment(decoder *cppbridge.HeadWalDecoder, data []byte) error { +// and add to encoder [cppbridge.HeadEncoder], returns createTs, encodeTs. +// +//revive:disable-next-line:confusing-results // returns createTs, encodeTs +//nolint:gocritic // returns createTs, encodeTs +func (ds *DataStorage) DecodeSegment(decoder *cppbridge.HeadWalDecoder, data []byte) (int64, int64, error) { return decoder.DecodeToDataStorage(data, ds.encoder) } // InstantQuery make instant query to data storage and returns samples. func (ds *DataStorage) InstantQuery( - targetTimestamp, notFoundValueTimestampValue int64, seriesIDs []uint32, -) []cppbridge.Sample { + targetTimestamp, notFoundValueTimestampValue int64, + seriesIDs []uint32, +) ([]cppbridge.Sample, cppbridge.DataStorageQueryResult) { ds.locker.RLock() - samples := ds.dataStorage.InstantQuery(targetTimestamp, notFoundValueTimestampValue, seriesIDs) + samples, res := ds.dataStorage.InstantQuery(targetTimestamp, notFoundValueTimestampValue, seriesIDs) ds.locker.RUnlock() - return samples + return samples, res } // MergeOutOfOrderChunks merge chunks with out of order data chunks. @@ -64,12 +68,14 @@ func (ds *DataStorage) MergeOutOfOrderChunks() { } // Query make query to data storage and returns serialazed chunks. -func (ds *DataStorage) Query(query cppbridge.HeadDataStorageQuery) *cppbridge.HeadDataStorageSerializedChunks { +func (ds *DataStorage) Query( + query cppbridge.HeadDataStorageQuery, +) (*cppbridge.HeadDataStorageSerializedChunks, cppbridge.DataStorageQueryResult) { ds.locker.RLock() - serializedChunks := ds.dataStorage.Query(query) + serializedChunks, res := ds.dataStorage.Query(query) ds.locker.RUnlock() - return serializedChunks + return serializedChunks, res } // QueryStatus get head status from [DataStorage]. diff --git a/pp/go/storage/head/shard/shard.go b/pp/go/storage/head/shard/shard.go index 9a4e137ed9..d7b0ea400f 100644 --- a/pp/go/storage/head/shard/shard.go +++ b/pp/go/storage/head/shard/shard.go @@ -8,6 +8,9 @@ import ( // Wal the minimum required Wal implementation for a [Shard]. type Wal interface { + // Close closes the wal segmentWriter. + Close() error + // Commit finalize segment from encoder and write to wal. Commit() error @@ -17,11 +20,11 @@ type Wal interface { // Flush flush all contetnt into wal. Flush() error + // Sync commits the current contents of the [Wal]. + Sync() error + // Write append the incoming inner series to wal encoder. Write(innerSeriesSlice []*cppbridge.InnerSeries) (bool, error) - - // Close closes the wal segmentWriter. - Close() error } // @@ -113,6 +116,11 @@ func (s *Shard[TWal]) WalFlush() error { return s.wal.Flush() } +// WalSync commits the current contents of the [Wal]. +func (s *Shard[TWal]) WalSync() error { + return s.wal.Sync() +} + // WalWrite append the incoming inner series to wal encoder. func (s *Shard[TWal]) WalWrite(innerSeriesSlice []*cppbridge.InnerSeries) (bool, error) { return s.wal.Write(innerSeriesSlice) diff --git a/pp/go/storage/head/shard/wal/reader/segment.go b/pp/go/storage/head/shard/wal/reader/segment.go index c46baddb93..3e3c7c4773 100644 --- a/pp/go/storage/head/shard/wal/reader/segment.go +++ b/pp/go/storage/head/shard/wal/reader/segment.go @@ -18,6 +18,11 @@ func (s *Segment) Bytes() []byte { return s.data } +// Length returns the length of slice byte. +func (s *Segment) Length() int { + return len(s.data) +} + // ReadFrom reads [Segment] data from r [io.Reader]. The return value n is the number of bytes read. // Any error encountered during the read is also returned. func (s *Segment) ReadFrom(r io.Reader) (int64, error) { diff --git a/pp/go/storage/head/shard/wal/wal.go b/pp/go/storage/head/shard/wal/wal.go index 7fb9e63089..ff2ebb674d 100644 --- a/pp/go/storage/head/shard/wal/wal.go +++ b/pp/go/storage/head/shard/wal/wal.go @@ -17,10 +17,16 @@ const ( type SegmentWriter[TSegment EncodedSegment] interface { // CurrentSize return current shard wal size. CurrentSize() int64 + // Write encoded segment to writer. Write(segment TSegment) error + // Flush write all buffered segments. Flush() error + + // Sync commits the current contents of the [SegmentWriter]. + Sync() error + // Close closes the storage. Close() error } @@ -83,6 +89,9 @@ func NewCorruptedWal[ // Close closes the wal segmentWriter. func (w *Wal[TSegment, TStats, TWriter]) Close() error { + w.locker.Lock() + defer w.locker.Unlock() + if w.closed { return nil } @@ -132,6 +141,14 @@ func (w *Wal[TSegment, TStats, TWriter]) Flush() error { return w.segmentWriter.Flush() } +// Sync commits the current contents of the [SegmentWriter]. +func (w *Wal[TSegment, TStats, TWriter]) Sync() error { + w.locker.Lock() + defer w.locker.Unlock() + + return w.segmentWriter.Sync() +} + // Write the incoming inner series to wal encoder. func (w *Wal[TSegment, TStats, TWriter]) Write(innerSeriesSlice []*cppbridge.InnerSeries) (bool, error) { if w.corrupted { diff --git a/pp/go/storage/head/shard/wal/writer/buffered.go b/pp/go/storage/head/shard/wal/writer/buffered.go index 2b8ea348fe..e578697db6 100644 --- a/pp/go/storage/head/shard/wal/writer/buffered.go +++ b/pp/go/storage/head/shard/wal/writer/buffered.go @@ -68,10 +68,10 @@ func (w *Buffered[TSegment]) CurrentSize() int64 { return atomic.LoadInt64(&w.currentSize) } -// Flush and sync buffer and collected segments to [WriteSyncCloser]. +// Flush buffer and collected segments to [WriteSyncCloser]. func (w *Buffered[TSegment]) Flush() error { if !w.writeCompleted { - if err := w.flushAndSync(); err != nil { + if err := w.flushBuffer(); err != nil { return fmt.Errorf("flush and sync: %w", err) } } @@ -92,35 +92,31 @@ func (w *Buffered[TSegment]) Flush() error { return nil } +// Sync commits the current contents of the [WriteSyncCloser] and notify [SegmentIsWrittenNotifier]. +func (w *Buffered[TSegment]) Sync() error { + if err := w.writer.Sync(); err != nil { + return fmt.Errorf("writer sync: %w", err) + } + + w.notifier.NotifySegmentIsWritten(w.shardID) + w.writeCompleted = true + return nil +} + // Write to buffer [Buffered] incoming [Segment]. func (w *Buffered[TSegment]) Write(segment TSegment) error { w.segments = append(w.segments, segment) return nil } -// flushAndSync write the contents from buffer to [WriteSyncCloser] and sync. -func (w *Buffered[TSegment]) flushAndSync() error { +// flushBuffer write the contents from buffer to [WriteSyncCloser]. +func (w *Buffered[TSegment]) flushBuffer() error { n, err := w.buffer.WriteTo(w.writer) atomic.AddInt64(&w.currentSize, n) if err != nil { return fmt.Errorf("buffer write: %w", err) } - if err := w.sync(); err != nil { - return fmt.Errorf("writer sync: %w", err) - } - - return nil -} - -// sync commits the current contents of the [WriteSyncCloser] and notify [SegmentIsWrittenNotifier]. -func (w *Buffered[TSegment]) sync() error { - if err := w.writer.Sync(); err != nil { - return fmt.Errorf("writer sync: %w", err) - } - - w.notifier.NotifySegmentIsWritten(w.shardID) - w.writeCompleted = true return nil } @@ -133,7 +129,7 @@ func (w *Buffered[TSegment]) writeToBufferAndFlush(segment TSegment) (encoded bo w.writeCompleted = false - if err := w.flushAndSync(); err != nil { + if err := w.flushBuffer(); err != nil { return true, err } diff --git a/pp/go/storage/manager.go b/pp/go/storage/manager.go index 36e187d368..7ff0a99747 100644 --- a/pp/go/storage/manager.go +++ b/pp/go/storage/manager.go @@ -22,6 +22,11 @@ import ( "github.com/prometheus/prometheus/pp/go/util" ) +const ( + // defaultCommitWaitInterval the minimum interval that the head must exist in order to perform operations on it. + defaultCommitWaitInterval = 5 * time.Minute +) + type Manager struct { g run.Group closer *util.Closer @@ -104,16 +109,21 @@ func NewManager( }, ) - statusSetter := &headStatusSetter{catalog: hcatalog} - // Rotator m.rotatorConfig = services.NewRotatorConfig(numberOfShards) m.rotatorMediator = &NoopMediator{c: make(chan struct{})} - rotator := services.NewRotator(activeHead, builder, headKeeper, m.rotatorMediator, m.rotatorConfig, statusSetter, r) rotatorCtx, rotatorCancel := context.WithCancel(baseCtx) m.g.Add( func() error { - return rotator.Execute(rotatorCtx) + return services.NewRotator( + activeHead, + builder, + headKeeper, + m.rotatorMediator, + m.rotatorConfig, + &headStatusSetter{catalog: hcatalog}, + r, + ).Execute(rotatorCtx) }, func(error) { m.rotatorMediator.Close() @@ -121,24 +131,21 @@ func NewManager( }, ) - // TODO created head - isNewHead := func(headID string) bool { rec, err := hcatalog.Get(headID) if err != nil { return true } - rec.CreatedAt() + return clock.Now().Add(-defaultCommitWaitInterval).UnixMilli() < rec.CreatedAt() } // Committer committerMediator := &NoopMediator{c: make(chan struct{})} - committer := services.NewCommitter(activeHead, committerMediator) committerCtx, committerCancel := context.WithCancel(baseCtx) m.g.Add( func() error { - return committer.Execute(committerCtx) + return services.NewCommitter(activeHead, committerMediator, isNewHead).Execute(committerCtx) }, func(error) { committerMediator.Close() @@ -148,11 +155,10 @@ func NewManager( // Merger mergerMediator := &NoopMediator{c: make(chan struct{})} - merger := services.NewMerger(activeHead, mergerMediator) mergerCtx, mergerCancel := context.WithCancel(baseCtx) m.g.Add( func() error { - return merger.Execute(mergerCtx) + return services.NewMerger(activeHead, mergerMediator, isNewHead).Execute(mergerCtx) }, func(error) { mergerMediator.Close() @@ -162,17 +168,16 @@ func NewManager( // MetricsUpdater metricsUpdaterMediator := &NoopMediator{c: make(chan struct{})} - metricsUpdater := services.NewMetricsUpdater( - activeHead, - headKeeper, - metricsUpdaterMediator, - querier.QueryHeadStatus, - r, - ) metricsUpdaterCtx, metricsUpdaterCancel := context.WithCancel(baseCtx) m.g.Add( func() error { - return metricsUpdater.Execute(metricsUpdaterCtx) + return services.NewMetricsUpdater( + activeHead, + headKeeper, + metricsUpdaterMediator, + querier.QueryHeadStatus, + r, + ).Execute(metricsUpdaterCtx) }, func(error) { metricsUpdaterMediator.Close() diff --git a/pp/go/storage/querier/querier_test.go b/pp/go/storage/querier/querier_test.go index 3e182ecf8d..43483ad280 100644 --- a/pp/go/storage/querier/querier_test.go +++ b/pp/go/storage/querier/querier_test.go @@ -62,6 +62,11 @@ func (*testWal) Flush() error { return nil } +// Sync test implementation wal. +func (*testWal) Sync() error { + return nil +} + // Write test implementation wal. func (*testWal) Write(_ []*cppbridge.InnerSeries) (bool, error) { return false, nil diff --git a/pp/go/storage/remotewriter/datasource.go b/pp/go/storage/remotewriter/datasource.go index ac1e14f6be..f9575f674a 100644 --- a/pp/go/storage/remotewriter/datasource.go +++ b/pp/go/storage/remotewriter/datasource.go @@ -186,9 +186,9 @@ func (s *shard) Read(ctx context.Context, targetSegmentID uint32, minTimestamp i return nil, errors.Join(err, ErrShardIsCorrupted) } - s.segmentSize.Observe(float64(len(segment.Data()))) + s.segmentSize.Observe(float64(segment.Length())) - decodedSegment, err := s.decoder.Decode(segment.Data(), minTimestamp) + decodedSegment, err := s.decoder.Decode(segment.Bytes(), minTimestamp) if err != nil { s.corrupted = true logger.Errorf("remotewritedebug shard %s/%d is corrupted by decode: %v", s.headID, s.shardID, err) diff --git a/pp/go/storage/remotewriter/walreader.go b/pp/go/storage/remotewriter/walreader.go index 19fca78d9f..f60098c841 100644 --- a/pp/go/storage/remotewriter/walreader.go +++ b/pp/go/storage/remotewriter/walreader.go @@ -8,6 +8,7 @@ import ( "os" "github.com/prometheus/prometheus/pp/go/relabeler/head" + "github.com/prometheus/prometheus/pp/go/storage/head/shard/wal/reader" ) type walReader struct { @@ -40,21 +41,18 @@ func (r *walReader) Close() error { // Read [Segment] from wal and return. func (r *walReader) Read() (segment Segment, err error) { - decodedSegment, _, err := head.ReadSegment(r.reader) - if err != nil { + if _, err = segment.ReadFrom(r.reader); err != nil { return segment, fmt.Errorf("failed to read segment: %w", err) } segment.ID = r.nextSegmentID r.nextSegmentID++ - segment.DecodedSegment = decodedSegment return segment, nil } // Segment encoded segment from wal. type Segment struct { - ID uint32 - encoderVersion uint8 - head.DecodedSegment + ID uint32 + reader.Segment } diff --git a/web/web_fuzzy_test.go b/web/web_fuzzy_test.go index 6c3f11d2fb..c102d874b6 100644 --- a/web/web_fuzzy_test.go +++ b/web/web_fuzzy_test.go @@ -140,6 +140,8 @@ func makeReceiver(ctx context.Context, t TestingT, logger log.Logger, dbDir stri transparent := &relabeler_config.InputRelabelerConfig{ Name: "transparent_relabeler", } + + unloadDataStorage := false receiver, err := receiver.NewReceiver( ctx, log.With(logger, "component", "receiver"), @@ -163,6 +165,7 @@ func makeReceiver(ctx context.Context, t TestingT, logger log.Logger, dbDir stri 4*time.Hour, 90*time.Second, 100e3, + unloadDataStorage, ) require.NoError(t, err, "create a receiver") return receiver From df733d257192b99e578eccda1c65b6f4ba4292c7 Mon Sep 17 00:00:00 2001 From: Alexandr Yudin Date: Tue, 9 Sep 2025 16:23:16 +0000 Subject: [PATCH 31/96] manager, mediator and minor edits --- pp-pkg/configadapter/headkeeper.go | 17 +- pp/go/cppbridge/head.go | 6 +- pp/go/storage/appender/appender.go | 32 +- pp/go/storage/constructor.go | 5 +- pp/go/storage/head/proxy/proxy.go | 128 ++++++++ pp/go/storage/head/services/rotator.go | 29 +- pp/go/storage/manager.go | 289 ++++++++++++------ pp/go/storage/mediator/mediator.go | 81 +++++ .../{head/manager => mediator}/timer.go | 16 +- 9 files changed, 450 insertions(+), 153 deletions(-) create mode 100644 pp/go/storage/head/proxy/proxy.go create mode 100644 pp/go/storage/mediator/mediator.go rename pp/go/storage/{head/manager => mediator}/timer.go (94%) diff --git a/pp-pkg/configadapter/headkeeper.go b/pp-pkg/configadapter/headkeeper.go index a0fb389bca..465ee0b3bc 100644 --- a/pp-pkg/configadapter/headkeeper.go +++ b/pp-pkg/configadapter/headkeeper.go @@ -1,31 +1,18 @@ package configadapter import ( - "context" - prom_config "github.com/prometheus/prometheus/config" "github.com/prometheus/prometheus/pp/go/storage" ) -// DefaultNumberOfShards default value for number of shards [pp_storage.Head]. -var DefaultNumberOfShards uint16 = 2 - // HeadKeeperApplyConfig returns func-adapter for apply config on [headkeeper.HeadKeeper]. -func HeadKeeperApplyConfig[THead any]( - ctx context.Context, - hk *storage.HeadManager, -) func(cfg *prom_config.Config) error { +func HeadKeeperApplyConfig(m *storage.Manager) func(cfg *prom_config.Config) error { return func(cfg *prom_config.Config) error { rCfg, err := cfg.GetReceiverConfig() if err != nil { return err } - numberOfShards := rCfg.NumberOfShards - if numberOfShards == 0 { - numberOfShards = DefaultNumberOfShards - } - - return hk.ApplyConfig(ctx, numberOfShards) + return m.ApplyConfig(rCfg.NumberOfShards) } } diff --git a/pp/go/cppbridge/head.go b/pp/go/cppbridge/head.go index b4570d9b12..d0f96a47ad 100644 --- a/pp/go/cppbridge/head.go +++ b/pp/go/cppbridge/head.go @@ -37,13 +37,15 @@ type Sample struct { // HeadDataStorage is Go wrapper around series_data::Data_storage. type HeadDataStorage struct { - dataStorage uintptr + dataStorage uintptr + gcDestroyDetector *uint64 } // NewHeadDataStorage - constructor. func NewHeadDataStorage() *HeadDataStorage { ds := &HeadDataStorage{ - dataStorage: seriesDataDataStorageCtor(), + dataStorage: seriesDataDataStorageCtor(), + gcDestroyDetector: &gcDestroyDetector, } runtime.SetFinalizer(ds, func(ds *HeadDataStorage) { diff --git a/pp/go/storage/appender/appender.go b/pp/go/storage/appender/appender.go index 198581889d..8077f44ddb 100644 --- a/pp/go/storage/appender/appender.go +++ b/pp/go/storage/appender/appender.go @@ -2,6 +2,7 @@ package appender import ( "context" + "errors" "fmt" "sync/atomic" @@ -24,6 +25,9 @@ const ( walWrite = "wal_write" ) +// errNilState error when incoming state is nil. +var errNilState = errors.New("state is nil") + // // Task // @@ -88,6 +92,9 @@ type Head[ // Enqueue the task to be executed on shards [Head]. Enqueue(t TTask) + // Generation returns current generation of [Head]. + Generation() uint64 + // NumberOfShards returns current number of shards in to [Head]. NumberOfShards() uint16 } @@ -123,17 +130,12 @@ func New[ func (a Appender[TTask, TLSS, TShard, THead]) Append( ctx context.Context, incomingData *storage.IncomingData, - incomingState *cppbridge.State, - relabelerID string, + state *cppbridge.State, commitToWal bool, ) ([][]*cppbridge.InnerSeries, cppbridge.RelabelerStats, error) { - // rd, state, err := h.resolveRelabelersData(incomingState, relabelerID) - // if err != nil { - // return nil, cppbridge.RelabelerStats{}, err - // } - - // TODO ? - var state *cppbridge.State + if err := a.resolveState(state); err != nil { + return nil, cppbridge.RelabelerStats{}, err + } shardedInnerSeries := NewShardedInnerSeries(a.head.NumberOfShards()) shardedRelabeledSeries := NewShardedRelabeledSeries(a.head.NumberOfShards()) @@ -371,3 +373,15 @@ func (a Appender[TTask, TLSS, TShard, THead]) appendInnerSeriesAndWriteToWal( return atomicLimitExhausted, tw.Wait() } + +func (a Appender[TTask, TLSS, TShard, THead]) resolveState(state *cppbridge.State) error { + if state == nil { + return errNilState + } + + // TODO delete generationRelabeler 0 + // state.Reconfigure on lock + state.Reconfigure(0, a.head.Generation(), a.head.NumberOfShards()) + + return nil +} diff --git a/pp/go/storage/constructor.go b/pp/go/storage/constructor.go index a49da0057e..d5636fe1a5 100644 --- a/pp/go/storage/constructor.go +++ b/pp/go/storage/constructor.go @@ -15,9 +15,6 @@ import ( "github.com/prometheus/prometheus/pp/go/storage/logger" ) -// DefaultNumberOfShards default number of shards. -var DefaultNumberOfShards uint16 = 2 - func HeadManagerCtor( l log.Logger, clock clockwork.Clock, @@ -37,7 +34,7 @@ func HeadManagerCtor( return nil, fmt.Errorf("%s is not directory", dataDir) } - initLogHandler(l) + InitLogHandler(l) builder := NewBuilder( hcatalog, diff --git a/pp/go/storage/head/proxy/proxy.go b/pp/go/storage/head/proxy/proxy.go new file mode 100644 index 0000000000..a2ed229c70 --- /dev/null +++ b/pp/go/storage/head/proxy/proxy.go @@ -0,0 +1,128 @@ +package proxy + +import ( + "context" + "errors" +) + +// Head the minimum required [Head] implementation for a proxy. +type Head interface { + // Close closes wals, query semaphore for the inability to get query and clear metrics. + Close() error + + // ID returns id [Head]. + ID() string +} + +// +// ActiveHeadContainer +// + +// ActiveHeadContainer container for active [Head], the minimum required [ActiveHeadContainer] implementation. +type ActiveHeadContainer[THead Head] interface { + // Close closes [ActiveHeadContainer] for the inability work with [Head]. + Close() error + + // Get the active [Head]. + Get() THead + + // Replace the active [Head] with a new [Head]. + Replace(ctx context.Context, newHead THead) error + + // With calls fn(h Head). + With(ctx context.Context, fn func(h THead) error) error +} + +// +// Keeper +// + +// TODO Description +type Keeper[THead Head] interface { + Add(head THead) + Close() error + RangeQueriableHeads(mint, maxt int64) func(func(THead) bool) +} + +// +// Proxy +// + +// Proxy it proxies requests to the active [Head] and the keeper of old [Head]s. +type Proxy[THead Head] struct { + activeHeadContainer ActiveHeadContainer[THead] + keeper Keeper[THead] + onClose func(h THead) error +} + +// NewProxy init new [Proxy]. +func NewProxy[THead Head]( + activeHeadContainer ActiveHeadContainer[THead], + keeper Keeper[THead], + onClose func(h THead) error, +) *Proxy[THead] { + return &Proxy[THead]{ + activeHeadContainer: activeHeadContainer, + keeper: keeper, + onClose: onClose, + } +} + +// Add the [Head] to the [Keeper]. +func (p *Proxy[THead]) Add(head THead) { + p.keeper.Add(head) +} + +// Close closes [ActiveHeadContainer] and [Keeper] for the inability work with [Head]. +func (p *Proxy[THead]) Close() error { + ahErr := p.activeHeadContainer.Close() + + h := p.activeHeadContainer.Get() + onCloseErr := p.onClose(h) + headCloseErr := h.Close() + + keeperErr := p.keeper.Close() + + return errors.Join(ahErr, onCloseErr, headCloseErr, keeperErr) +} + +// Get the active [Head]. +func (p *Proxy[THead]) Get() THead { + return p.activeHeadContainer.Get() +} + +// RangeQueriableHeadsWithActive returns the iterator to queriable [Head]s: +// the active [Head] and the [Head]s from the [Keeper]. +func (p *Proxy[THead]) RangeQueriableHeadsWithActive(mint, maxt int64) func(func(THead) bool) { + return func(yield func(h THead) bool) { + ahead := p.activeHeadContainer.Get() + if !yield(ahead) { + return + } + + for head := range p.keeper.RangeQueriableHeads(mint, maxt) { + if ahead.ID() == head.ID() { + continue + } + + if !yield(head) { + return + } + } + } +} + +// RangeQueriableHeads returns the iterator to queriable [Head]s - the [Head]s only from the [Keeper]. +func (p *Proxy[THead]) RangeQueriableHeads(mint, maxt int64) func(func(THead) bool) { + return p.keeper.RangeQueriableHeads(mint, maxt) +} + +// Replace the active [Head] with a new [Head]. +func (p *Proxy[THead]) Replace(ctx context.Context, newHead THead) error { + return p.activeHeadContainer.Replace(ctx, newHead) +} + +// With calls fn(h Head) on active [Head]. +func (p *Proxy[THead]) With(ctx context.Context, fn func(h THead) error) error { + return p.activeHeadContainer.With(ctx, fn) +} diff --git a/pp/go/storage/head/services/rotator.go b/pp/go/storage/head/services/rotator.go index 03e2e5cdda..19bc86dde3 100644 --- a/pp/go/storage/head/services/rotator.go +++ b/pp/go/storage/head/services/rotator.go @@ -3,7 +3,6 @@ package services import ( "context" "fmt" - "sync/atomic" "github.com/prometheus/client_golang/prometheus" "github.com/prometheus/prometheus/pp/go/storage/logger" @@ -15,25 +14,9 @@ import ( // // RotatorConfig config for [Rotator]. -type RotatorConfig struct { - numberOfShards uint32 -} - -// NewRotatorConfig init new [RotatorConfig]. -func NewRotatorConfig(numberOfShards uint16) *RotatorConfig { - return &RotatorConfig{ - numberOfShards: uint32(numberOfShards), - } -} - -// NumberOfShards returns current number of shards. -func (c RotatorConfig) NumberOfShards() uint16 { - return uint16(atomic.LoadUint32(&c.numberOfShards)) // #nosec G115 // no overflow -} - -// SetNumberOfShards set new number of shards. -func (c *RotatorConfig) SetNumberOfShards(numberOfShards uint16) { - atomic.StoreUint32(&c.numberOfShards, uint32(numberOfShards)) +type RotatorConfig interface { + // NumberOfShards returns current number of shards. + NumberOfShards() uint16 } // @@ -51,7 +34,7 @@ type Rotator[ headBuilder HeadBuilder[TTask, TShard, TGoShard, THead] keeper Keeper[TTask, TShard, TGoShard, THead] m Mediator - cfg *RotatorConfig + cfg RotatorConfig headStatusSetter HeadStatusSetter rotateCounter prometheus.Counter } @@ -63,10 +46,10 @@ func NewRotator[ THead Head[TTask, TShard, TGoShard], ]( activeHead ActiveHeadContainer[TTask, TShard, TGoShard, THead], - headBuilder HeadBuilder[TTask, TShard, TGoShard, THead], keeper Keeper[TTask, TShard, TGoShard, THead], + headBuilder HeadBuilder[TTask, TShard, TGoShard, THead], m Mediator, - cfg *RotatorConfig, + cfg RotatorConfig, headStatusSetter HeadStatusSetter, r prometheus.Registerer, ) *Rotator[TTask, TShard, TGoShard, THead] { diff --git a/pp/go/storage/manager.go b/pp/go/storage/manager.go index 7ff0a99747..5d532eb085 100644 --- a/pp/go/storage/manager.go +++ b/pp/go/storage/manager.go @@ -5,7 +5,7 @@ import ( "errors" "fmt" "os" - "sync" + "sync/atomic" "time" "github.com/go-kit/log" @@ -14,34 +14,105 @@ import ( "github.com/oklog/run" "github.com/prometheus/client_golang/prometheus" + "github.com/prometheus/prometheus/pp/go/cppbridge" "github.com/prometheus/prometheus/pp/go/storage/catalog" "github.com/prometheus/prometheus/pp/go/storage/head/container" + "github.com/prometheus/prometheus/pp/go/storage/head/proxy" "github.com/prometheus/prometheus/pp/go/storage/head/services" "github.com/prometheus/prometheus/pp/go/storage/logger" + "github.com/prometheus/prometheus/pp/go/storage/mediator" "github.com/prometheus/prometheus/pp/go/storage/querier" + "github.com/prometheus/prometheus/pp/go/storage/ready" "github.com/prometheus/prometheus/pp/go/util" ) const ( - // defaultCommitWaitInterval the minimum interval that the head must exist in order to perform operations on it. - defaultCommitWaitInterval = 5 * time.Minute + // DefaultRotateDuration default block duration. + DefaultRotateDuration = 2 * time.Hour + + // DefaultMergeDuration the default interval for the merge out of order chunks. + DefaultMergeDuration = 5 * time.Minute + + // DefaultMetricWriteInterval default metric scrape interval. + DefaultMetricWriteInterval = 15 * time.Second ) +// DefaultNumberOfShards default number of shards. +var DefaultNumberOfShards uint16 = 2 + +// +// Options +// + +type Options struct { + Seed uint64 + BlockDuration time.Duration + CommitInterval time.Duration + MaxSegmentSize uint32 + NumberOfShards uint16 +} + +// +// Config +// + +// Config config for [Manager]. +type Config struct { + numberOfShards uint32 +} + +// NewConfig init new [Config]. +func NewConfig(numberOfShards uint16) *Config { + if numberOfShards == 0 { + numberOfShards = DefaultNumberOfShards + } + + return &Config{ + numberOfShards: uint32(numberOfShards), + } +} + +// NumberOfShards returns current number of shards. +func (c *Config) NumberOfShards() uint16 { + return uint16(atomic.LoadUint32(&c.numberOfShards)) // #nosec G115 // no overflow +} + +// SetNumberOfShards set new number of shards. +func (c *Config) SetNumberOfShards(numberOfShards uint16) bool { + if numberOfShards == 0 { + numberOfShards = DefaultNumberOfShards + } + + if c.NumberOfShards() == numberOfShards { + return false + } + + atomic.StoreUint32(&c.numberOfShards, uint32(numberOfShards)) + + return true +} + +// +// Manager +// + type Manager struct { g run.Group closer *util.Closer - rotatorConfig *services.RotatorConfig - rotatorMediator *NoopMediator + proxy *proxy.Proxy[*HeadOnDisk] + cgogc *cppbridge.CGOGC + cfg *Config + rotatorMediator *mediator.Mediator } +// NewManager init new [Manager]. func NewManager( - l log.Logger, clock clockwork.Clock, dataDir string, hcatalog *catalog.Catalog, - blockDuration time.Duration, - maxSegmentSize uint32, - numberOfShards uint16, + options Options, + triggerNotifier *ReloadBlocksTriggerNotifier, + readyNotifier ready.Notifier, r prometheus.Registerer, ) (*Manager, error) { dirStat, err := os.Stat(dataDir) @@ -53,29 +124,13 @@ func NewManager( return nil, fmt.Errorf("%s is not directory", dataDir) } - initLogHandler(l) + builder := NewBuilder(hcatalog, dataDir, options.MaxSegmentSize, r) - builder := NewBuilder( - hcatalog, - dataDir, - maxSegmentSize, - r, - ) + loader := NewLoader(dataDir, options.MaxSegmentSize, r) - loader := NewLoader( - dataDir, - maxSegmentSize, - r, - ) + cfg := NewConfig(options.NumberOfShards) - h, err := uploadOrBuildHead( - clock, - hcatalog, - builder, - loader, - blockDuration, - numberOfShards, - ) + h, err := uploadOrBuildHead(clock, hcatalog, builder, loader, options.BlockDuration, cfg.NumberOfShards()) if err != nil { return nil, err } @@ -84,8 +139,7 @@ func NewManager( return nil, errors.Join(fmt.Errorf("failed to set active status: %w", err), h.Close()) } - // TODO Need close - activeHead := container.NewWeighted(h) + readyNotifier.NotifyReady() // TODO implements headKeeper := &NoopKeeper{} @@ -93,8 +147,60 @@ func NewManager( m := &Manager{ g: run.Group{}, closer: util.NewCloser(), + proxy: proxy.NewProxy(container.NewWeighted(h), headKeeper, services.CFSViaRange), + cgogc: cppbridge.NewCGOGC(r), + cfg: cfg, + rotatorMediator: mediator.NewMediator( + mediator.NewRotateTimerWithSeed(clock, options.BlockDuration, options.Seed), + ), + } + + m.initServices(hcatalog, builder, clock, options.CommitInterval, r) + + logger.Infof("[Manager] created") + + return m, nil +} + +// ApplyConfig update config. +func (m *Manager) ApplyConfig(numberOfShards uint16) error { + logger.Infof("reconfiguration start") + defer logger.Infof("reconfiguration completed") + + h := m.proxy.Get() + if h.NumberOfShards() == numberOfShards { + return nil + } + + if m.cfg.SetNumberOfShards(numberOfShards) { + m.rotatorMediator.Trigger() } + return nil +} + +// Run launches the [Manager]'s services. +func (m *Manager) Run() error { + defer m.closer.Done() + + return m.g.Run() +} + +// Shutdown safe shutdown [Manager]: stop services and close [Head]'s. +func (m *Manager) Shutdown(ctx context.Context) error { + _ = m.closer.Close() + + return errors.Join(m.proxy.Close(), m.cgogc.Shutdown(ctx)) +} + +// initServices initializes services for startup. +func (m *Manager) initServices( + hcatalog *catalog.Catalog, + builder *Builder, + clock clockwork.Clock, + commitInterval time.Duration, + r prometheus.Registerer, +) { baseCtx := context.Background() // Termination handler. @@ -110,18 +216,16 @@ func NewManager( ) // Rotator - m.rotatorConfig = services.NewRotatorConfig(numberOfShards) - m.rotatorMediator = &NoopMediator{c: make(chan struct{})} rotatorCtx, rotatorCancel := context.WithCancel(baseCtx) m.g.Add( func() error { return services.NewRotator( - activeHead, + m.proxy, + m.proxy, builder, - headKeeper, m.rotatorMediator, - m.rotatorConfig, - &headStatusSetter{catalog: hcatalog}, + m.cfg, + &statusSetter{catalog: hcatalog}, r, ).Execute(rotatorCtx) }, @@ -137,15 +241,15 @@ func NewManager( return true } - return clock.Now().Add(-defaultCommitWaitInterval).UnixMilli() < rec.CreatedAt() + return clock.Now().Add(-DefaultMergeDuration).UnixMilli() < rec.CreatedAt() } // Committer - committerMediator := &NoopMediator{c: make(chan struct{})} + committerMediator := mediator.NewMediator(mediator.NewConstantIntervalTimer(clock, commitInterval)) committerCtx, committerCancel := context.WithCancel(baseCtx) m.g.Add( func() error { - return services.NewCommitter(activeHead, committerMediator, isNewHead).Execute(committerCtx) + return services.NewCommitter(m.proxy, committerMediator, isNewHead).Execute(committerCtx) }, func(error) { committerMediator.Close() @@ -154,11 +258,11 @@ func NewManager( ) // Merger - mergerMediator := &NoopMediator{c: make(chan struct{})} + mergerMediator := mediator.NewMediator(mediator.NewConstantIntervalTimer(clock, DefaultMergeDuration)) mergerCtx, mergerCancel := context.WithCancel(baseCtx) m.g.Add( func() error { - return services.NewMerger(activeHead, mergerMediator, isNewHead).Execute(mergerCtx) + return services.NewMerger(m.proxy, mergerMediator, isNewHead).Execute(mergerCtx) }, func(error) { mergerMediator.Close() @@ -167,13 +271,13 @@ func NewManager( ) // MetricsUpdater - metricsUpdaterMediator := &NoopMediator{c: make(chan struct{})} + metricsUpdaterMediator := mediator.NewMediator(mediator.NewConstantIntervalTimer(clock, DefaultMetricWriteInterval)) metricsUpdaterCtx, metricsUpdaterCancel := context.WithCancel(baseCtx) m.g.Add( func() error { return services.NewMetricsUpdater( - activeHead, - headKeeper, + m.proxy, + m.proxy, metricsUpdaterMediator, querier.QueryHeadStatus, r, @@ -184,24 +288,10 @@ func NewManager( metricsUpdaterCancel() }, ) - - return m, nil -} - -// TODO implementation. -func (m *Manager) Run() error { - defer m.closer.Done() - - return m.g.Run() } -// TODO implementation. -func (m *Manager) Shutdown(ctx context.Context) { - _ = m.closer.Close() -} - -// initLogHandler init log handler for pp. -func initLogHandler(l log.Logger) { +// InitLogHandler init log handler for pp. +func InitLogHandler(l log.Logger) { l = log.With(l, "pp_caller", log.Caller(4)) logger.Debugf = func(template string, args ...any) { @@ -222,60 +312,65 @@ func initLogHandler(l log.Logger) { } // -// NoopKeeper +// headStatusSetter // -// NoopKeeper implements Keeper. -type NoopKeeper struct{} +// statusSetter wrapper over [catalog.Catalog] for set statuses. +type statusSetter struct { + catalog *catalog.Catalog +} -// Add implements Keeper. -func (*NoopKeeper) Add(*HeadOnDisk) {} +// SetActiveStatus sets the [catalog.StatusActive] status by headID. +func (ha *statusSetter) SetActiveStatus(headID string) error { + _, err := ha.catalog.SetStatus(headID, catalog.StatusActive) + return err +} -// RangeQueriableHeads implements Keeper. -func (k *NoopKeeper) RangeQueriableHeads( - mint, maxt int64, -) func(func(*HeadOnDisk) bool) { - return func(func(*HeadOnDisk) bool) {} +// SetRotatedStatus sets the [catalog.StatusRotated] status by headID. +func (ha *statusSetter) SetRotatedStatus(headID string) error { + _, err := ha.catalog.SetStatus(headID, catalog.StatusRotated) + return err } // -// NoopMediator +// ReloadBlocksTriggerNotifier // -// NoopMediator implements Mediator. -type NoopMediator struct { - c chan struct{} - closeOnce sync.Once +type ReloadBlocksTriggerNotifier struct { + c chan struct{} } -// C implements Mediator. -func (m *NoopMediator) C() <-chan struct{} { - return m.c +func NewReloadBlocksTriggerNotifier() *ReloadBlocksTriggerNotifier { + return &ReloadBlocksTriggerNotifier{c: make(chan struct{}, 1)} } -// Close close channel and stop [Mediator]. -func (m *NoopMediator) Close() { - m.closeOnce.Do(func() { - close(m.c) - }) +func (tn *ReloadBlocksTriggerNotifier) Chan() <-chan struct{} { + return tn.c +} + +func (tn *ReloadBlocksTriggerNotifier) NotifyWritten() { + select { + case tn.c <- struct{}{}: + default: + } } // -// +// NoopKeeper // -type headStatusSetter struct { - catalog *catalog.Catalog -} +// NoopKeeper implements Keeper. +type NoopKeeper struct{} -// SetActiveStatus sets the [catalog.StatusActive] status by headID. -func (ha *headStatusSetter) SetActiveStatus(headID string) error { - _, err := ha.catalog.SetStatus(headID, catalog.StatusActive) - return err -} +// Add implements Keeper. +func (*NoopKeeper) Add(*HeadOnDisk) {} -// SetRotatedStatus sets the [catalog.StatusRotated] status by headID. -func (ha *headStatusSetter) SetRotatedStatus(headID string) error { - _, err := ha.catalog.SetStatus(headID, catalog.StatusRotated) - return err +// Close implements Keeper. +func (*NoopKeeper) Close() error { return nil } + +// RangeQueriableHeads implements Keeper. +func (k *NoopKeeper) RangeQueriableHeads( + mint, maxt int64, +) func(func(*HeadOnDisk) bool) { + return func(func(*HeadOnDisk) bool) {} } diff --git a/pp/go/storage/mediator/mediator.go b/pp/go/storage/mediator/mediator.go new file mode 100644 index 0000000000..010aa4272e --- /dev/null +++ b/pp/go/storage/mediator/mediator.go @@ -0,0 +1,81 @@ +package mediator + +import ( + "sync" + "time" + + "github.com/prometheus/prometheus/pp/go/util" +) + +// +// Timer +// + +// Timer implementation timer. +type Timer interface { + Chan() <-chan time.Time + Reset() + Stop() +} + +// +// Mediator +// + +// Mediator notifies about events via the channel. +type Mediator struct { + timer Timer + c chan struct{} + closeOnce sync.Once + closer *util.Closer +} + +// NewMediator init new Mediator. +func NewMediator(timer Timer) *Mediator { + m := &Mediator{ + timer: timer, + c: make(chan struct{}), + closeOnce: sync.Once{}, + closer: util.NewCloser(), + } + + go m.loop() + + return m +} + +// C returns channel with events. +func (m *Mediator) C() <-chan struct{} { + return m.c +} + +// Close stops the internal timer and clears the channel. +func (m *Mediator) Close() { + _ = m.closer.Close() + m.timer.Stop() + m.closeOnce.Do(func() { + close(m.c) + }) +} + +// Trigger send notify to channel. +func (m *Mediator) Trigger() { + select { + case m.c <- struct{}{}: + default: + } +} + +// loop by timer. +func (m *Mediator) loop() { + defer m.closer.Done() + + for { + select { + case <-m.timer.Chan(): + m.Trigger() + case <-m.closer.Signal(): + return + } + } +} diff --git a/pp/go/storage/head/manager/timer.go b/pp/go/storage/mediator/timer.go similarity index 94% rename from pp/go/storage/head/manager/timer.go rename to pp/go/storage/mediator/timer.go index d60b86ac38..f6dcd7e19a 100644 --- a/pp/go/storage/head/manager/timer.go +++ b/pp/go/storage/mediator/timer.go @@ -1,4 +1,4 @@ -package manager +package mediator import ( "math" @@ -91,8 +91,12 @@ func (rt *RotateTimer) RotateAtNext() time.Time { // Stop - prevents the Timer from firing. // Stop does not close the channel, to prevent a read from the channel succeeding incorrectly. func (rt *RotateTimer) Stop() { + // drain timer if !rt.timer.Stop() { - <-rt.timer.Chan() + select { + case <-rt.timer.Chan(): + default: + } } } @@ -126,5 +130,11 @@ func (t *ConstantIntervalTimer) Reset() { // Stop timer. func (t *ConstantIntervalTimer) Stop() { - t.timer.Stop() + // drain timer + if !t.timer.Stop() { + select { + case <-t.timer.Chan(): + default: + } + } } From b1ea87f5392d787247cd9490f998ccf130a52ef5 Mon Sep 17 00:00:00 2001 From: Alexandr Yudin Date: Wed, 10 Sep 2025 14:41:31 +0000 Subject: [PATCH 32/96] adapter --- pp-pkg/storage/adapter.go | 255 ++++++++++++------ pp-pkg/storage/appender.go | 43 +-- pp/go/storage/appender/appender.go | 6 +- pp/go/storage/head/head/head.go | 13 +- pp/go/storage/head/services/interface.go | 42 ++- .../storage/head/services/metrics_updater.go | 13 +- pp/go/storage/head/services/rotator.go | 15 +- pp/go/storage/manager.go | 5 +- pp/go/storage/models.go | 21 -- 9 files changed, 252 insertions(+), 161 deletions(-) diff --git a/pp-pkg/storage/adapter.go b/pp-pkg/storage/adapter.go index 7670c357e5..ec38658733 100644 --- a/pp-pkg/storage/adapter.go +++ b/pp-pkg/storage/adapter.go @@ -2,116 +2,210 @@ package storage import ( "context" + "math" - "github.com/prometheus/common/model" "github.com/prometheus/prometheus/pp/go/cppbridge" + "github.com/prometheus/prometheus/pp/go/model" "github.com/prometheus/prometheus/pp/go/relabeler/querier" pp_storage "github.com/prometheus/prometheus/pp/go/storage" "github.com/prometheus/prometheus/storage" ) -var _ storage.Storage = (*Adapter)(nil) +// +// Proxy +// + +// Proxy it proxies requests to the active [Head] and the keeper of old [Head]s. +type Proxy[THead any] interface { + // Get the active [Head]. + Get() THead + + // RangeQueriableHeadsWithActive returns the iterator to queriable [Head]s: + // the active [Head] and the [Head]s from the [Keeper]. + RangeQueriableHeadsWithActive(mint int64, maxt int64) func(func(THead) bool) + + // With calls fn(h Head) on active [Head]. + With(ctx context.Context, fn func(h THead) error) error +} + +// +// HATracker +// + +// HATracker interface for High Availability Tracker. +type HATracker interface { + // IsDrop check whether data needs to be sent or discarded immediately. + IsDrop(cluster, replica string) bool + + // Destroy clear all clusters and stop work. + Destroy() +} + +// +// HeadAppender +// + +// HeadAppender adds incoming data to the [Head]. +type HeadAppender interface { + Append( + ctx context.Context, + incomingData *pp_storage.IncomingData, + state *cppbridge.State, + commitToWal bool, + ) ([][]*cppbridge.InnerSeries, cppbridge.RelabelerStats, error) +} + +// +// ProtobufData +// + +// ProtobufData is an universal interface for blob protobuf data. +type ProtobufData interface { + Bytes() []byte + Destroy() +} + +// +// TimeSeriesData +// + +// TimeSeriesBatch is an universal interface for batch [model.TimeSeries]. +type TimeSeriesBatch interface { + TimeSeries() []model.TimeSeries + Destroy() +} + +// +// Adapter +// + +var _ storage.Storage = (*Adapter[any, HeadAppender])(nil) // Adapter for implementing the [Queryable] interface and append data. -type Adapter struct { - // +type Adapter[THead any, THeadAppender HeadAppender] struct { + proxy Proxy[THead] + haTracker HATracker + appenderCtor func(THead) THeadAppender + hashdexFactory cppbridge.HashdexFactory + hashdexLimits cppbridge.WALHashdexLimits } -// AppendHashdex append incoming [cppbridge.HashdexContent] to [pp_storage.Head]. -func (ar *Adapter) AppendHashdex( +// AppendHashdex append incoming [cppbridge.HashdexContent] to [Head]. +func (ar *Adapter[THead, THeadAppender]) AppendHashdex( ctx context.Context, hashdex cppbridge.ShardedData, - relabelerID string, + state *cppbridge.State, commitToWal bool, ) error { - // if rr.haTracker.IsDrop(hashdex.Cluster(), hashdex.Replica()) { - // return nil - // } - // incomingData := &relabeler.IncomingData{Hashdex: hashdex} - // _, err := rr.activeHead.Append(ctx, incomingData, nil, relabelerID, commitToWal) - return nil + if ar.haTracker.IsDrop(hashdex.Cluster(), hashdex.Replica()) { + return nil + } + + return ar.proxy.With(ctx, func(h THead) error { + _, _, err := ar.appenderCtor(h).Append( + ctx, + &pp_storage.IncomingData{Hashdex: hashdex}, + state, + commitToWal, + ) + + return err + }) } -// AppendScraperHashdex append ScraperHashdex data to [pp_storage.Head]. -func (ar *Adapter) AppendScraperHashdex( +// AppendScraperHashdex append ScraperHashdex data to [Head]. +func (ar *Adapter[THead, THeadAppender]) AppendScraperHashdex( ctx context.Context, hashdex cppbridge.ShardedData, state *cppbridge.State, - relabelerID string, commitToWal bool, -) (cppbridge.RelabelerStats, error) { - // return rr.activeHead.Append( - // ctx, - // &relabeler.IncomingData{Hashdex: hashdex}, - // state, - // relabelerID, - // commitToWal, - // ) - - return cppbridge.RelabelerStats{}, nil +) (stats cppbridge.RelabelerStats, err error) { + _ = ar.proxy.With(ctx, func(h THead) error { + _, stats, err = ar.appenderCtor(h).Append( + ctx, + &pp_storage.IncomingData{Hashdex: hashdex}, + state, + commitToWal, + ) + + return nil + }) + + return stats, err } -// AppendSnappyProtobuf append compressed via snappy Protobuf data to [pp_storage.Head]. -func (ar *Adapter) AppendSnappyProtobuf( +// AppendSnappyProtobuf append compressed via snappy Protobuf data to [Head]. +func (ar *Adapter[THead, THeadAppender]) AppendSnappyProtobuf( ctx context.Context, - compressedData pp_storage.ProtobufData, - relabelerID string, + compressedData ProtobufData, + state *cppbridge.State, commitToWal bool, ) error { - // hx, err := cppbridge.NewWALSnappyProtobufHashdex(compressedData.Bytes(), rr.hashdexLimits) - // compressedData.Destroy() - // if err != nil { - // return err - // } - - // if rr.haTracker.IsDrop(hx.Cluster(), hx.Replica()) { - // return nil - // } - - // incomingData := &relabeler.IncomingData{Hashdex: hx} - // _, err = rr.activeHead.Append(ctx, incomingData, nil, relabelerID, commitToWal) - return nil + hx, err := cppbridge.NewWALSnappyProtobufHashdex(compressedData.Bytes(), ar.hashdexLimits) + compressedData.Destroy() + if err != nil { + return err + } + + if ar.haTracker.IsDrop(hx.Cluster(), hx.Replica()) { + return nil + } + + return ar.proxy.With(ctx, func(h THead) error { + _, _, err := ar.appenderCtor(h).Append( + ctx, + &pp_storage.IncomingData{Hashdex: hx}, + state, + commitToWal, + ) + + return err + }) } // AppendTimeSeries append TimeSeries data to [pp_storage.Head]. -func (ar *Adapter) AppendTimeSeries( +func (ar *Adapter[THead, THeadAppender]) AppendTimeSeries( ctx context.Context, - data pp_storage.TimeSeriesBatch, + data TimeSeriesBatch, state *cppbridge.State, - relabelerID string, commitToWal bool, -) (cppbridge.RelabelerStats, error) { - // hx, err := rr.hashdexFactory.GoModel(data.TimeSeries(), rr.hashdexLimits) - // if err != nil { - // data.Destroy() - // return cppbridge.RelabelerStats{}, err - // } - - // if rr.haTracker.IsDrop(hx.Cluster(), hx.Replica()) { - // data.Destroy() - // return cppbridge.RelabelerStats{}, nil - // } - // incomingData := &relabeler.IncomingData{Hashdex: hx, Data: data} - // return rr.activeHead.Append( - // ctx, - // incomingData, - // state, - // relabelerID, - // commitToWal, - // ) - - return cppbridge.RelabelerStats{}, nil +) (stats cppbridge.RelabelerStats, err error) { + hx, err := ar.hashdexFactory.GoModel(data.TimeSeries(), ar.hashdexLimits) + if err != nil { + data.Destroy() + return stats, err + } + + if ar.haTracker.IsDrop(hx.Cluster(), hx.Replica()) { + data.Destroy() + return stats, nil + } + + _ = ar.proxy.With(ctx, func(h THead) error { + _, stats, err = ar.appenderCtor(h).Append( + ctx, + &pp_storage.IncomingData{Hashdex: hx, Data: data}, + state, + commitToWal, + ) + + return nil + }) + + return stats, err } // Appender create a new [storage.Appender] for [pp_storage.Head]. -func (ar *Adapter) Appender(ctx context.Context) storage.Appender { - // return newPromAppender(ctx, rr, prom_config.TransparentRelabeler) - return nil +func (ar *Adapter[THead, THeadAppender]) Appender(ctx context.Context) storage.Appender { + // TODO state *cppbridge.State + var state *cppbridge.State + + return newTimeSeriesAppender(ctx, ar, state) } // ChunkQuerier provides querying access over time series data of a fixed time range. // Returns new Chunk Querier that merges results of given primary and secondary chunk queriers. -func (ar *Adapter) ChunkQuerier(mint, maxt int64) (storage.ChunkQuerier, error) { +func (ar *Adapter[THead, THeadAppender]) ChunkQuerier(mint, maxt int64) (storage.ChunkQuerier, error) { // TODO return storage.NewMergeChunkQuerier( nil, @@ -122,37 +216,32 @@ func (ar *Adapter) ChunkQuerier(mint, maxt int64) (storage.ChunkQuerier, error) // Close closes the storage and all its underlying resources. // Implements the [storage.Storage] interface. -func (*Adapter) Close() error { +func (*Adapter[THead, THeadAppender]) Close() error { return nil } // HeadQuerier returns [storage.Querier] from active head. -func (ar *Adapter) HeadQuerier(ctx context.Context, mint, maxt int64) (storage.Querier, error) { +func (ar *Adapter[THead, THeadAppender]) HeadQuerier(ctx context.Context, mint, maxt int64) (storage.Querier, error) { // TODO return nil, nil } // HeadStatus returns stats of Head. -func (ar *Adapter) HeadStatus(ctx context.Context, limit int) pp_storage.HeadStatus { +func (ar *Adapter[THead, THeadAppender]) HeadStatus(ctx context.Context, limit int) pp_storage.HeadStatus { // TODO + // ar.proxy. return pp_storage.HeadStatus{} } // Querier calls f() with the given parameters. // Returns a [querier.MultiQuerier] combining of primary and secondary queriers. -func (ar *Adapter) Querier(mint, maxt int64) (storage.Querier, error) { +func (ar *Adapter[THead, THeadAppender]) Querier(mint, maxt int64) (storage.Querier, error) { // TODO return querier.NewMultiQuerier([]storage.Querier{}, nil), nil } -// RelabelerIDIsExist check on exist relabelerID. -func (ar *Adapter) RelabelerIDIsExist(relabelerID string) bool { - // TODO - return true -} - // StartTime returns the oldest timestamp stored in the storage. // Implements the [storage.Storage] interface. -func (*Adapter) StartTime() (int64, error) { - return int64(model.Latest), nil +func (*Adapter[THead, THeadAppender]) StartTime() (int64, error) { + return math.MaxInt64, nil } diff --git a/pp-pkg/storage/appender.go b/pp-pkg/storage/appender.go index 6ab74d2d72..b404aa0f53 100644 --- a/pp-pkg/storage/appender.go +++ b/pp-pkg/storage/appender.go @@ -7,6 +7,7 @@ import ( "github.com/prometheus/prometheus/model/histogram" "github.com/prometheus/prometheus/model/labels" "github.com/prometheus/prometheus/model/metadata" + "github.com/prometheus/prometheus/pp/go/cppbridge" "github.com/prometheus/prometheus/pp/go/model" "github.com/prometheus/prometheus/storage" ) @@ -28,15 +29,28 @@ func (d *timeSeriesBatch) Destroy() { // TimeSeriesAppender appender for rules, aggregates the [model.TimeSeries] batch and append to head, // implementation [storage.Appender]. -type TimeSeriesAppender struct { - ctx context.Context - // receiver *Receiver - relabelerID string - batch *timeSeriesBatch +type TimeSeriesAppender[THead any, THeadAppender HeadAppender] struct { + ctx context.Context + adapter *Adapter[THead, THeadAppender] + state *cppbridge.State + batch *timeSeriesBatch +} + +func newTimeSeriesAppender[THead any, THeadAppender HeadAppender]( + ctx context.Context, + adapter *Adapter[THead, THeadAppender], + state *cppbridge.State, +) *TimeSeriesAppender[THead, THeadAppender] { + return &TimeSeriesAppender[THead, THeadAppender]{ + ctx: ctx, + adapter: adapter, + state: state, + batch: &timeSeriesBatch{}, + } } // Append adds a sample pair for the given series, implementation [storage.Appender]. -func (a *TimeSeriesAppender) Append( +func (a *TimeSeriesAppender[THead, THeadAppender]) Append( _ storage.SeriesRef, l labels.Labels, t int64, @@ -56,7 +70,7 @@ func (a *TimeSeriesAppender) Append( } // AppendCTZeroSample do nothing, implementation [storage.Appender]. -func (*TimeSeriesAppender) AppendCTZeroSample( +func (*TimeSeriesAppender[THead, THeadAppender]) AppendCTZeroSample( _ storage.SeriesRef, _ labels.Labels, _, _ int64, @@ -65,7 +79,7 @@ func (*TimeSeriesAppender) AppendCTZeroSample( } // AppendExemplar do nothing, implementation [storage.Appender]. -func (*TimeSeriesAppender) AppendExemplar( +func (*TimeSeriesAppender[THead, THeadAppender]) AppendExemplar( _ storage.SeriesRef, _ labels.Labels, _ exemplar.Exemplar, @@ -74,7 +88,7 @@ func (*TimeSeriesAppender) AppendExemplar( } // AppendHistogram do nothing, implementation [storage.Appender]. -func (*TimeSeriesAppender) AppendHistogram( +func (*TimeSeriesAppender[THead, THeadAppender]) AppendHistogram( _ storage.SeriesRef, _ labels.Labels, _ int64, @@ -85,23 +99,22 @@ func (*TimeSeriesAppender) AppendHistogram( } // Commit adds aggregated series to the head, implementation [storage.Appender]. -func (a *TimeSeriesAppender) Commit() error { +func (a *TimeSeriesAppender[THead, THeadAppender]) Commit() error { if len(a.batch.timeSeries) == 0 { return nil } - // _, err := a.receiver.AppendTimeSeries(a.ctx, a.batch, nil, a.relabelerID, false) - // return err - return nil + _, err := a.adapter.AppendTimeSeries(a.ctx, a.batch, a.state, false) + return err } // Rollback do nothing, implementation [storage.Appender]. -func (*TimeSeriesAppender) Rollback() error { +func (*TimeSeriesAppender[THead, THeadAppender]) Rollback() error { return nil } // UpdateMetadata do nothing, implementation [storage.Appender]. -func (*TimeSeriesAppender) UpdateMetadata( +func (*TimeSeriesAppender[THead, THeadAppender]) UpdateMetadata( _ storage.SeriesRef, _ labels.Labels, _ metadata.Metadata, diff --git a/pp/go/storage/appender/appender.go b/pp/go/storage/appender/appender.go index 8077f44ddb..af6c596746 100644 --- a/pp/go/storage/appender/appender.go +++ b/pp/go/storage/appender/appender.go @@ -103,6 +103,7 @@ type Head[ // Appender // +// Appender adds incoming data to the [Head]. type Appender[ TTask Task, TLSS LSS, @@ -126,7 +127,7 @@ func New[ } } -// Append incoming data to head. +// Append incoming data to [Head]. func (a Appender[TTask, TLSS, TShard, THead]) Append( ctx context.Context, incomingData *storage.IncomingData, @@ -379,8 +380,7 @@ func (a Appender[TTask, TLSS, TShard, THead]) resolveState(state *cppbridge.Stat return errNilState } - // TODO delete generationRelabeler 0 - // state.Reconfigure on lock + // TODO delete generationRelabeler 0, state.Reconfigure on lock state.Reconfigure(0, a.head.Generation(), a.head.NumberOfShards()) return nil diff --git a/pp/go/storage/head/head/head.go b/pp/go/storage/head/head/head.go index 3105035c0f..df3d90770c 100644 --- a/pp/go/storage/head/head/head.go +++ b/pp/go/storage/head/head/head.go @@ -97,12 +97,13 @@ func NewHead[TShard Shard, TGoroutineShard Shard]( factory := util.NewUnconflictRegisterer(registerer) h := &Head[TShard, TGoroutineShard]{ - id: id, - generation: generation, - gshardCtor: gshardCtor, - releaseHeadFn: releaseHeadFn, - shards: shards, - taskChs: taskChs, + id: id, + generation: generation, + gshardCtor: gshardCtor, + releaseHeadFn: releaseHeadFn, + shards: shards, + taskChs: taskChs, + querySemaphore: locker.NewWeighted(2 * concurrency), // x2 for back pressure // for clearing [Head] metrics memoryInUse: factory.NewGaugeVec(prometheus.GaugeOpts{ diff --git a/pp/go/storage/head/services/interface.go b/pp/go/storage/head/services/interface.go index 903f3fea6a..f9d9dfe9c4 100644 --- a/pp/go/storage/head/services/interface.go +++ b/pp/go/storage/head/services/interface.go @@ -12,15 +12,6 @@ type ActiveHeadContainer[ TShard, TGoShard Shard, THead Head[TTask, TShard, TGoShard], ] interface { - // Close closes [ActiveHeadContainer] for the inability work with [Head]. - Close() error - - // Get the active head [Head]. - Get() THead - - // Replace the active head [Head] with a new head. - Replace(ctx context.Context, newHead THead) error - // With calls fn(h Head). With(ctx context.Context, fn func(h THead) error) error } @@ -34,9 +25,6 @@ type Head[ TTask Task, TShard, TGoShard Shard, ] interface { - // // Close closes wals, query semaphore for the inability to get query and clear metrics. - // Close(ctx context.Context) error - // CreateTask create a task for operations on the [Head] shards. CreateTask(taskName string, shardFn func(shard TGoShard) error) TTask @@ -93,6 +81,7 @@ type HeadStatusSetter interface { // Keeper // +// TODO need? type Keeper[ TTask Task, TShard, TGShard Shard, @@ -112,6 +101,35 @@ type Mediator interface { C() <-chan struct{} } +// +// ProxyHead +// + +// ProxyHead it proxies requests to the active [Head] and the keeper of old [Head]s. +type ProxyHead[ + TTask Task, + TShard, TGoShard Shard, + THead Head[TTask, TShard, TGoShard], +] interface { + Add(head THead) + + // Get the active [Head]. + Get() THead + + // RangeQueriableHeadsWithActive returns the iterator to queriable [Head]s: + // the active [Head] and the [Head]s from the [Keeper]. + RangeQueriableHeadsWithActive(mint int64, maxt int64) func(func(THead) bool) + + // RangeQueriableHeads returns the iterator to queriable [Head]s - the [Head]s only from the [Keeper]. + RangeQueriableHeads(mint, maxt int64) func(func(THead) bool) + + // Replace the active head [Head] with a new head. + Replace(ctx context.Context, newHead THead) error + + // With calls fn(h Head) on active [Head]. + With(ctx context.Context, fn func(h THead) error) error +} + // // Shard // diff --git a/pp/go/storage/head/services/metrics_updater.go b/pp/go/storage/head/services/metrics_updater.go index 8f5a02bb03..a8ae7a6d8c 100644 --- a/pp/go/storage/head/services/metrics_updater.go +++ b/pp/go/storage/head/services/metrics_updater.go @@ -28,8 +28,7 @@ type MetricsUpdater[ THead Head[TTask, TShard, TGoShard], THeadStatus HeadStatus, ] struct { - activeHead ActiveHeadContainer[TTask, TShard, TGoShard, THead] - keeper Keeper[TTask, TShard, TGoShard, THead] + proxyHead ProxyHead[TTask, TShard, TGoShard, THead] m Mediator queryHeadStatus func(ctx context.Context, head THead, limit int) (THeadStatus, error) @@ -47,16 +46,14 @@ func NewMetricsUpdater[ THead Head[TTask, TShard, TGoShard], THeadStatus HeadStatus, ]( - activeHead ActiveHeadContainer[TTask, TShard, TGoShard, THead], - keeper Keeper[TTask, TShard, TGoShard, THead], + proxyHead ProxyHead[TTask, TShard, TGoShard, THead], m Mediator, queryHeadStatus func(ctx context.Context, head THead, limit int) (THeadStatus, error), r prometheus.Registerer, ) *MetricsUpdater[TTask, TShard, TGoShard, THead, THeadStatus] { factory := util.NewUnconflictRegisterer(r) return &MetricsUpdater[TTask, TShard, TGoShard, THead, THeadStatus]{ - activeHead: activeHead, - keeper: keeper, + proxyHead: proxyHead, m: m, queryHeadStatus: queryHeadStatus, @@ -105,7 +102,7 @@ func (s *MetricsUpdater[TTask, TShard, TGoShard, THead, THeadStatus]) Execute(ct // collect metrics from the head. func (s *MetricsUpdater[TTask, TShard, TGoShard, THead, THeadStatus]) collect(ctx context.Context) { - ahead := s.activeHead.Get() + ahead := s.proxyHead.Get() status, err := s.queryHeadStatus(ctx, ahead, 0) if err == nil { @@ -121,7 +118,7 @@ func (s *MetricsUpdater[TTask, TShard, TGoShard, THead, THeadStatus]) collect(ct s.collectFromShards(ahead, true) - for head := range s.keeper.RangeQueriableHeads(0, math.MaxInt64) { + for head := range s.proxyHead.RangeQueriableHeads(0, math.MaxInt64) { if head.ID() == ahead.ID() { continue } diff --git a/pp/go/storage/head/services/rotator.go b/pp/go/storage/head/services/rotator.go index 19bc86dde3..6727c69778 100644 --- a/pp/go/storage/head/services/rotator.go +++ b/pp/go/storage/head/services/rotator.go @@ -30,9 +30,8 @@ type Rotator[ TShard, TGoShard Shard, THead Head[TTask, TShard, TGoShard], ] struct { - activeHead ActiveHeadContainer[TTask, TShard, TGoShard, THead] + proxyHead ProxyHead[TTask, TShard, TGoShard, THead] headBuilder HeadBuilder[TTask, TShard, TGoShard, THead] - keeper Keeper[TTask, TShard, TGoShard, THead] m Mediator cfg RotatorConfig headStatusSetter HeadStatusSetter @@ -45,8 +44,7 @@ func NewRotator[ TShard, TGoShard Shard, THead Head[TTask, TShard, TGoShard], ]( - activeHead ActiveHeadContainer[TTask, TShard, TGoShard, THead], - keeper Keeper[TTask, TShard, TGoShard, THead], + proxyHead ProxyHead[TTask, TShard, TGoShard, THead], headBuilder HeadBuilder[TTask, TShard, TGoShard, THead], m Mediator, cfg RotatorConfig, @@ -55,9 +53,8 @@ func NewRotator[ ) *Rotator[TTask, TShard, TGoShard, THead] { factory := util.NewUnconflictRegisterer(r) return &Rotator[TTask, TShard, TGoShard, THead]{ - activeHead: activeHead, + proxyHead: proxyHead, headBuilder: headBuilder, - keeper: keeper, m: m, cfg: cfg, headStatusSetter: headStatusSetter, @@ -94,7 +91,7 @@ func (s *Rotator[TTask, TShard, TGoShard, THead]) rotate( ctx context.Context, numberOfShards uint16, ) error { - oldHead := s.activeHead.Get() + oldHead := s.proxyHead.Get() newHead, err := s.headBuilder.Build(oldHead.Generation()+1, numberOfShards) if err != nil { @@ -104,10 +101,10 @@ func (s *Rotator[TTask, TShard, TGoShard, THead]) rotate( // TODO CopySeriesFrom only old nunber of shards == new // newHead.CopySeriesFrom(oldHead) - s.keeper.Add(oldHead) + s.proxyHead.Add(oldHead) // TODO if replace error? - if err = s.activeHead.Replace(ctx, newHead); err != nil { + if err = s.proxyHead.Replace(ctx, newHead); err != nil { return fmt.Errorf("failed to replace old to new head: %w", err) } diff --git a/pp/go/storage/manager.go b/pp/go/storage/manager.go index 5d532eb085..36fa54c478 100644 --- a/pp/go/storage/manager.go +++ b/pp/go/storage/manager.go @@ -167,8 +167,7 @@ func (m *Manager) ApplyConfig(numberOfShards uint16) error { logger.Infof("reconfiguration start") defer logger.Infof("reconfiguration completed") - h := m.proxy.Get() - if h.NumberOfShards() == numberOfShards { + if m.proxy.Get().NumberOfShards() == numberOfShards { return nil } @@ -220,7 +219,6 @@ func (m *Manager) initServices( m.g.Add( func() error { return services.NewRotator( - m.proxy, m.proxy, builder, m.rotatorMediator, @@ -276,7 +274,6 @@ func (m *Manager) initServices( m.g.Add( func() error { return services.NewMetricsUpdater( - m.proxy, m.proxy, metricsUpdaterMediator, querier.QueryHeadStatus, diff --git a/pp/go/storage/models.go b/pp/go/storage/models.go index 9df3d60b24..51f35124f1 100644 --- a/pp/go/storage/models.go +++ b/pp/go/storage/models.go @@ -2,7 +2,6 @@ package storage import ( "github.com/prometheus/prometheus/pp/go/cppbridge" - "github.com/prometheus/prometheus/pp/go/model" ) // @@ -15,26 +14,6 @@ type MetricData interface { Destroy() } -// -// ProtobufData -// - -// ProtobufData is an universal interface for blob protobuf data. -type ProtobufData interface { - Bytes() []byte - Destroy() -} - -// -// TimeSeriesData -// - -// TimeSeriesBatch is an universal interface for batch [model.TimeSeries]. -type TimeSeriesBatch interface { - TimeSeries() []model.TimeSeries - Destroy() -} - // // IncomingData // From d010bd6adc10419ca0155b7b1d8c7e639821b829 Mon Sep 17 00:00:00 2001 From: Alexandr Yudin Date: Thu, 11 Sep 2025 06:56:24 +0000 Subject: [PATCH 33/96] adapter --- pp-pkg/storage/adapter.go | 189 ++-- pp-pkg/storage/appender.go | 28 +- pp/go/hatracker/high_availability_tracker.go | 164 ++++ pp/go/storage/appender/appender.go | 7 +- pp/go/storage/appender/models.go | 40 +- pp/go/storage/constructor.go | 195 ++-- pp/go/storage/head/head/head.go | 6 +- pp/go/storage/head/manager/manager.go | 948 +++++++++---------- pp/go/storage/manager.go | 60 ++ pp/go/storage/models.go | 124 ++- pp/go/storage/types.go | 12 +- 11 files changed, 989 insertions(+), 784 deletions(-) create mode 100644 pp/go/hatracker/high_availability_tracker.go diff --git a/pp-pkg/storage/adapter.go b/pp-pkg/storage/adapter.go index ec38658733..8f767d3fe5 100644 --- a/pp-pkg/storage/adapter.go +++ b/pp-pkg/storage/adapter.go @@ -4,57 +4,19 @@ import ( "context" "math" + "github.com/jonboulle/clockwork" + "github.com/prometheus/client_golang/prometheus" + "github.com/prometheus/prometheus/pp/go/cppbridge" + "github.com/prometheus/prometheus/pp/go/hatracker" "github.com/prometheus/prometheus/pp/go/model" - "github.com/prometheus/prometheus/pp/go/relabeler/querier" pp_storage "github.com/prometheus/prometheus/pp/go/storage" + "github.com/prometheus/prometheus/pp/go/storage/appender" + "github.com/prometheus/prometheus/pp/go/storage/head/services" + "github.com/prometheus/prometheus/pp/go/storage/querier" "github.com/prometheus/prometheus/storage" ) -// -// Proxy -// - -// Proxy it proxies requests to the active [Head] and the keeper of old [Head]s. -type Proxy[THead any] interface { - // Get the active [Head]. - Get() THead - - // RangeQueriableHeadsWithActive returns the iterator to queriable [Head]s: - // the active [Head] and the [Head]s from the [Keeper]. - RangeQueriableHeadsWithActive(mint int64, maxt int64) func(func(THead) bool) - - // With calls fn(h Head) on active [Head]. - With(ctx context.Context, fn func(h THead) error) error -} - -// -// HATracker -// - -// HATracker interface for High Availability Tracker. -type HATracker interface { - // IsDrop check whether data needs to be sent or discarded immediately. - IsDrop(cluster, replica string) bool - - // Destroy clear all clusters and stop work. - Destroy() -} - -// -// HeadAppender -// - -// HeadAppender adds incoming data to the [Head]. -type HeadAppender interface { - Append( - ctx context.Context, - incomingData *pp_storage.IncomingData, - state *cppbridge.State, - commitToWal bool, - ) ([][]*cppbridge.InnerSeries, cppbridge.RelabelerStats, error) -} - // // ProtobufData // @@ -66,7 +28,7 @@ type ProtobufData interface { } // -// TimeSeriesData +// TimeSeriesBatch // // TimeSeriesBatch is an universal interface for batch [model.TimeSeries]. @@ -79,19 +41,37 @@ type TimeSeriesBatch interface { // Adapter // -var _ storage.Storage = (*Adapter[any, HeadAppender])(nil) +var _ storage.Storage = (*Adapter)(nil) // Adapter for implementing the [Queryable] interface and append data. -type Adapter[THead any, THeadAppender HeadAppender] struct { - proxy Proxy[THead] - haTracker HATracker - appenderCtor func(THead) THeadAppender +type Adapter struct { + proxy *pp_storage.ProxyHead + haTracker *hatracker.HighAvailabilityTracker hashdexFactory cppbridge.HashdexFactory hashdexLimits cppbridge.WALHashdexLimits + + activeQuerierMetrics *querier.Metrics + storageQuerierMetrics *querier.Metrics +} + +// NewAdapter init new [Adapter]. +func NewAdapter( + clock clockwork.Clock, + proxy *pp_storage.ProxyHead, + registerer prometheus.Registerer, +) *Adapter { + return &Adapter{ + proxy: proxy, + haTracker: hatracker.NewHighAvailabilityTracker(clock, registerer), + hashdexFactory: cppbridge.HashdexFactory{}, + hashdexLimits: cppbridge.DefaultWALHashdexLimits(), + activeQuerierMetrics: querier.NewMetrics(registerer, querier.QueryableAppenderSource), + storageQuerierMetrics: querier.NewMetrics(registerer, querier.QueryableStorageSource), + } } // AppendHashdex append incoming [cppbridge.HashdexContent] to [Head]. -func (ar *Adapter[THead, THeadAppender]) AppendHashdex( +func (ar *Adapter) AppendHashdex( ctx context.Context, hashdex cppbridge.ShardedData, state *cppbridge.State, @@ -101,10 +81,10 @@ func (ar *Adapter[THead, THeadAppender]) AppendHashdex( return nil } - return ar.proxy.With(ctx, func(h THead) error { - _, _, err := ar.appenderCtor(h).Append( + return ar.proxy.With(ctx, func(h *pp_storage.HeadOnDisk) error { + _, _, err := appender.New(h, services.CFViaRange).Append( ctx, - &pp_storage.IncomingData{Hashdex: hashdex}, + &appender.IncomingData{Hashdex: hashdex}, state, commitToWal, ) @@ -114,16 +94,16 @@ func (ar *Adapter[THead, THeadAppender]) AppendHashdex( } // AppendScraperHashdex append ScraperHashdex data to [Head]. -func (ar *Adapter[THead, THeadAppender]) AppendScraperHashdex( +func (ar *Adapter) AppendScraperHashdex( ctx context.Context, hashdex cppbridge.ShardedData, state *cppbridge.State, commitToWal bool, ) (stats cppbridge.RelabelerStats, err error) { - _ = ar.proxy.With(ctx, func(h THead) error { - _, stats, err = ar.appenderCtor(h).Append( + _ = ar.proxy.With(ctx, func(h *pp_storage.HeadOnDisk) error { + _, stats, err = appender.New(h, services.CFViaRange).Append( ctx, - &pp_storage.IncomingData{Hashdex: hashdex}, + &appender.IncomingData{Hashdex: hashdex}, state, commitToWal, ) @@ -135,7 +115,7 @@ func (ar *Adapter[THead, THeadAppender]) AppendScraperHashdex( } // AppendSnappyProtobuf append compressed via snappy Protobuf data to [Head]. -func (ar *Adapter[THead, THeadAppender]) AppendSnappyProtobuf( +func (ar *Adapter) AppendSnappyProtobuf( ctx context.Context, compressedData ProtobufData, state *cppbridge.State, @@ -151,10 +131,10 @@ func (ar *Adapter[THead, THeadAppender]) AppendSnappyProtobuf( return nil } - return ar.proxy.With(ctx, func(h THead) error { - _, _, err := ar.appenderCtor(h).Append( + return ar.proxy.With(ctx, func(h *pp_storage.HeadOnDisk) error { + _, _, err := appender.New(h, services.CFViaRange).Append( ctx, - &pp_storage.IncomingData{Hashdex: hx}, + &appender.IncomingData{Hashdex: hx}, state, commitToWal, ) @@ -163,8 +143,8 @@ func (ar *Adapter[THead, THeadAppender]) AppendSnappyProtobuf( }) } -// AppendTimeSeries append TimeSeries data to [pp_storage.Head]. -func (ar *Adapter[THead, THeadAppender]) AppendTimeSeries( +// AppendTimeSeries append TimeSeries data to [Head]. +func (ar *Adapter) AppendTimeSeries( ctx context.Context, data TimeSeriesBatch, state *cppbridge.State, @@ -181,10 +161,10 @@ func (ar *Adapter[THead, THeadAppender]) AppendTimeSeries( return stats, nil } - _ = ar.proxy.With(ctx, func(h THead) error { - _, stats, err = ar.appenderCtor(h).Append( + _ = ar.proxy.With(ctx, func(h *pp_storage.HeadOnDisk) error { + _, stats, err = appender.New(h, services.CFViaRange).Append( ctx, - &pp_storage.IncomingData{Hashdex: hx, Data: data}, + &appender.IncomingData{Hashdex: hx, Data: data}, state, commitToWal, ) @@ -195,8 +175,8 @@ func (ar *Adapter[THead, THeadAppender]) AppendTimeSeries( return stats, err } -// Appender create a new [storage.Appender] for [pp_storage.Head]. -func (ar *Adapter[THead, THeadAppender]) Appender(ctx context.Context) storage.Appender { +// Appender create a new [storage.Appender] for [Head]. +func (ar *Adapter) Appender(ctx context.Context) storage.Appender { // TODO state *cppbridge.State var state *cppbridge.State @@ -205,43 +185,82 @@ func (ar *Adapter[THead, THeadAppender]) Appender(ctx context.Context) storage.A // ChunkQuerier provides querying access over time series data of a fixed time range. // Returns new Chunk Querier that merges results of given primary and secondary chunk queriers. -func (ar *Adapter[THead, THeadAppender]) ChunkQuerier(mint, maxt int64) (storage.ChunkQuerier, error) { - // TODO +func (ar *Adapter) ChunkQuerier(mint, maxt int64) (storage.ChunkQuerier, error) { + queriers := make([]storage.ChunkQuerier, 0, 2) + ahead := ar.proxy.Get() + queriers = append( + queriers, + querier.NewQuerier(ahead, querier.NewNoOpShardedDeduplicator, mint, maxt, nil, ar.activeQuerierMetrics), + ) + + for head := range ar.proxy.RangeQueriableHeads(mint, maxt) { + if ahead.ID() == head.ID() { + continue + } + + queriers = append( + queriers, + querier.NewQuerier(head, querier.NewNoOpShardedDeduplicator, mint, maxt, nil, ar.storageQuerierMetrics), + ) + } + return storage.NewMergeChunkQuerier( nil, - []storage.ChunkQuerier{}, + queriers, storage.NewConcatenatingChunkSeriesMerger(), ), nil } // Close closes the storage and all its underlying resources. // Implements the [storage.Storage] interface. -func (*Adapter[THead, THeadAppender]) Close() error { +func (ar *Adapter) Close() error { + ar.haTracker.Destroy() return nil } // HeadQuerier returns [storage.Querier] from active head. -func (ar *Adapter[THead, THeadAppender]) HeadQuerier(ctx context.Context, mint, maxt int64) (storage.Querier, error) { - // TODO - return nil, nil +func (ar *Adapter) HeadQuerier(ctx context.Context, mint, maxt int64) (storage.Querier, error) { + return querier.NewQuerier( + ar.proxy.Get(), + querier.NewNoOpShardedDeduplicator, + mint, + maxt, + nil, + ar.activeQuerierMetrics, + ), nil } // HeadStatus returns stats of Head. -func (ar *Adapter[THead, THeadAppender]) HeadStatus(ctx context.Context, limit int) pp_storage.HeadStatus { - // TODO - // ar.proxy. - return pp_storage.HeadStatus{} +func (ar *Adapter) HeadStatus(ctx context.Context, limit int) (querier.HeadStatus, error) { + return querier.QueryHeadStatus(ctx, ar.proxy.Get(), limit) } // Querier calls f() with the given parameters. // Returns a [querier.MultiQuerier] combining of primary and secondary queriers. -func (ar *Adapter[THead, THeadAppender]) Querier(mint, maxt int64) (storage.Querier, error) { - // TODO - return querier.NewMultiQuerier([]storage.Querier{}, nil), nil +func (ar *Adapter) Querier(mint, maxt int64) (storage.Querier, error) { + queriers := make([]storage.Querier, 0, 2) + ahead := ar.proxy.Get() + queriers = append( + queriers, + querier.NewQuerier(ahead, querier.NewNoOpShardedDeduplicator, mint, maxt, nil, ar.activeQuerierMetrics), + ) + + for head := range ar.proxy.RangeQueriableHeads(mint, maxt) { + if ahead.ID() == head.ID() { + continue + } + + queriers = append( + queriers, + querier.NewQuerier(head, querier.NewNoOpShardedDeduplicator, mint, maxt, nil, ar.storageQuerierMetrics), + ) + } + + return querier.NewMultiQuerier(queriers, nil), nil } // StartTime returns the oldest timestamp stored in the storage. // Implements the [storage.Storage] interface. -func (*Adapter[THead, THeadAppender]) StartTime() (int64, error) { +func (*Adapter) StartTime() (int64, error) { return math.MaxInt64, nil } diff --git a/pp-pkg/storage/appender.go b/pp-pkg/storage/appender.go index b404aa0f53..ee29ba66f0 100644 --- a/pp-pkg/storage/appender.go +++ b/pp-pkg/storage/appender.go @@ -12,7 +12,7 @@ import ( "github.com/prometheus/prometheus/storage" ) -// timeSeriesBatch implementation buffer of [ppstorage.TimeSeriesData]. +// timeSeriesBatch implementation buffer of [TimeSeriesBatch]. type timeSeriesBatch struct { timeSeries []model.TimeSeries } @@ -29,19 +29,19 @@ func (d *timeSeriesBatch) Destroy() { // TimeSeriesAppender appender for rules, aggregates the [model.TimeSeries] batch and append to head, // implementation [storage.Appender]. -type TimeSeriesAppender[THead any, THeadAppender HeadAppender] struct { +type TimeSeriesAppender struct { ctx context.Context - adapter *Adapter[THead, THeadAppender] + adapter *Adapter state *cppbridge.State batch *timeSeriesBatch } -func newTimeSeriesAppender[THead any, THeadAppender HeadAppender]( +func newTimeSeriesAppender( ctx context.Context, - adapter *Adapter[THead, THeadAppender], + adapter *Adapter, state *cppbridge.State, -) *TimeSeriesAppender[THead, THeadAppender] { - return &TimeSeriesAppender[THead, THeadAppender]{ +) *TimeSeriesAppender { + return &TimeSeriesAppender{ ctx: ctx, adapter: adapter, state: state, @@ -50,7 +50,7 @@ func newTimeSeriesAppender[THead any, THeadAppender HeadAppender]( } // Append adds a sample pair for the given series, implementation [storage.Appender]. -func (a *TimeSeriesAppender[THead, THeadAppender]) Append( +func (a *TimeSeriesAppender) Append( _ storage.SeriesRef, l labels.Labels, t int64, @@ -70,7 +70,7 @@ func (a *TimeSeriesAppender[THead, THeadAppender]) Append( } // AppendCTZeroSample do nothing, implementation [storage.Appender]. -func (*TimeSeriesAppender[THead, THeadAppender]) AppendCTZeroSample( +func (*TimeSeriesAppender) AppendCTZeroSample( _ storage.SeriesRef, _ labels.Labels, _, _ int64, @@ -79,7 +79,7 @@ func (*TimeSeriesAppender[THead, THeadAppender]) AppendCTZeroSample( } // AppendExemplar do nothing, implementation [storage.Appender]. -func (*TimeSeriesAppender[THead, THeadAppender]) AppendExemplar( +func (*TimeSeriesAppender) AppendExemplar( _ storage.SeriesRef, _ labels.Labels, _ exemplar.Exemplar, @@ -88,7 +88,7 @@ func (*TimeSeriesAppender[THead, THeadAppender]) AppendExemplar( } // AppendHistogram do nothing, implementation [storage.Appender]. -func (*TimeSeriesAppender[THead, THeadAppender]) AppendHistogram( +func (*TimeSeriesAppender) AppendHistogram( _ storage.SeriesRef, _ labels.Labels, _ int64, @@ -99,7 +99,7 @@ func (*TimeSeriesAppender[THead, THeadAppender]) AppendHistogram( } // Commit adds aggregated series to the head, implementation [storage.Appender]. -func (a *TimeSeriesAppender[THead, THeadAppender]) Commit() error { +func (a *TimeSeriesAppender) Commit() error { if len(a.batch.timeSeries) == 0 { return nil } @@ -109,12 +109,12 @@ func (a *TimeSeriesAppender[THead, THeadAppender]) Commit() error { } // Rollback do nothing, implementation [storage.Appender]. -func (*TimeSeriesAppender[THead, THeadAppender]) Rollback() error { +func (*TimeSeriesAppender) Rollback() error { return nil } // UpdateMetadata do nothing, implementation [storage.Appender]. -func (*TimeSeriesAppender[THead, THeadAppender]) UpdateMetadata( +func (*TimeSeriesAppender) UpdateMetadata( _ storage.SeriesRef, _ labels.Labels, _ metadata.Metadata, diff --git a/pp/go/hatracker/high_availability_tracker.go b/pp/go/hatracker/high_availability_tracker.go new file mode 100644 index 0000000000..6bb557de96 --- /dev/null +++ b/pp/go/hatracker/high_availability_tracker.go @@ -0,0 +1,164 @@ +package hatracker + +import ( + "sync" + "time" + + "github.com/jonboulle/clockwork" + "github.com/prometheus/client_golang/prometheus" + + "github.com/prometheus/prometheus/pp/go/util" +) + +var ( + defaultHAOverTime = 30 * time.Second + defaultHACleanup = 10 * time.Minute +) + +// HighAvailabilityTracker track the replica we're accepting samples from +// for each HA cluster we know about. +type HighAvailabilityTracker struct { + stop chan struct{} + storage *sync.Map + overTime int64 + clock clockwork.Clock + // stat + electedReplicaChanges *prometheus.CounterVec + electedReplicaTimestamp *prometheus.GaugeVec + lastElectionTimestamp *prometheus.GaugeVec + deletedReplicas prometheus.Counter + dropedReplicas prometheus.Counter +} + +// NewHighAvailabilityTracker init new [HighAvailabilityTracker]. +func NewHighAvailabilityTracker( + clock clockwork.Clock, + registerer prometheus.Registerer, +) *HighAvailabilityTracker { + factory := util.NewUnconflictRegisterer(registerer) + ha := &HighAvailabilityTracker{ + storage: new(sync.Map), + overTime: int64(defaultHAOverTime.Seconds()), + clock: clock, + stop: make(chan struct{}), + electedReplicaChanges: factory.NewCounterVec( + prometheus.CounterOpts{ + Name: "prompp_high_availability_tracker_elected_replica_changes", + Help: "The total number of times the elected replica has changed for cluster.", + }, + []string{"cluster", "replica"}, + ), + electedReplicaTimestamp: factory.NewGaugeVec( + prometheus.GaugeOpts{ + Name: "prompp_high_availability_tracker_elected_replica_timestamp_seconds", + Help: "The timestamp stored for the currently elected replica.", + }, + []string{"cluster", "replica"}, + ), + lastElectionTimestamp: factory.NewGaugeVec( + prometheus.GaugeOpts{ + Name: "prompp_high_availability_tracker_last_election_timestamp_seconds", + Help: "The timestamp stored for the currently elected replica.", + }, + []string{"cluster", "replica"}, + ), + deletedReplicas: factory.NewCounter( + prometheus.CounterOpts{ + Name: "prompp_high_availability_tracker_deleted_total", + Help: "Number of elected replicas deleted from store.", + }, + ), + dropedReplicas: factory.NewCounter( + prometheus.CounterOpts{ + Name: "prompp_high_availability_tracker_droped_total", + Help: "Number of elected replicas droped.", + }, + ), + } + + go ha.cleanup() + + return ha +} + +// cleanup delete old replicas. +func (ha *HighAvailabilityTracker) cleanup() { + ticker := ha.clock.NewTicker(defaultHACleanup) + defer ticker.Stop() + + for { + select { + case <-ticker.Chan(): + now := ha.clock.Now().Unix() + ha.storage.Range(func(key, value any) bool { + hv := value.(*haValue) + hv.mx.Lock() + if now-hv.electedAt >= int64(defaultHACleanup.Seconds()) { + ha.storage.Delete(key) + ha.electedReplicaChanges.DeleteLabelValues(key.(string), hv.value) + ha.electedReplicaTimestamp.DeleteLabelValues(key.(string), hv.value) + ha.deletedReplicas.Inc() + } + hv.mx.Unlock() + return true + }) + case <-ha.stop: + return + } + } +} + +// IsDrop check whether data needs to be sent or discarded immediately. +func (ha *HighAvailabilityTracker) IsDrop(cluster, replica string) bool { + if replica == "" { + return false + } + now := ha.clock.Now().Unix() + val, ok := ha.storage.LoadOrStore( + cluster, + &haValue{electedAt: now, value: replica, mx: new(sync.Mutex)}, + ) + if !ok { + ha.electedReplicaChanges.With(prometheus.Labels{"cluster": cluster, "replica": replica}).Inc() + ha.electedReplicaTimestamp.With(prometheus.Labels{"cluster": cluster, "replica": replica}).Set(float64(now)) + return false + } + + hv := val.(*haValue) + hv.mx.Lock() + if hv.value == replica { + hv.electedAt = now + ha.electedReplicaTimestamp.With(prometheus.Labels{"cluster": cluster, "replica": replica}).Set(float64(now)) + hv.mx.Unlock() + return false + } + + if (now - hv.electedAt) >= ha.overTime { + ha.lastElectionTimestamp.With(prometheus.Labels{"cluster": cluster, "replica": hv.value}).Set(float64(now)) + hv.value = replica + hv.electedAt = now + ha.electedReplicaChanges.With(prometheus.Labels{"cluster": cluster, "replica": replica}).Inc() + ha.electedReplicaTimestamp.With(prometheus.Labels{"cluster": cluster, "replica": replica}).Set(float64(now)) + hv.mx.Unlock() + return true + } + hv.mx.Unlock() + ha.dropedReplicas.Inc() + return true +} + +// Destroy clear all clusters and stop work. +func (ha *HighAvailabilityTracker) Destroy() { + close(ha.stop) + ha.storage.Range(func(key, _ any) bool { + ha.storage.Delete(key) + return true + }) +} + +// haValue value for HighAvailabilityTracker. +type haValue struct { + mx *sync.Mutex + value string + electedAt int64 +} diff --git a/pp/go/storage/appender/appender.go b/pp/go/storage/appender/appender.go index af6c596746..e2c6ac7127 100644 --- a/pp/go/storage/appender/appender.go +++ b/pp/go/storage/appender/appender.go @@ -7,7 +7,6 @@ import ( "sync/atomic" "github.com/prometheus/prometheus/pp/go/cppbridge" - "github.com/prometheus/prometheus/pp/go/storage" "github.com/prometheus/prometheus/pp/go/storage/head/task" "github.com/prometheus/prometheus/pp/go/storage/logger" ) @@ -128,9 +127,11 @@ func New[ } // Append incoming data to [Head]. +// +//revive:disable-next-line:flag-parameter this is a flag, but it's more convenient this way func (a Appender[TTask, TLSS, TShard, THead]) Append( ctx context.Context, - incomingData *storage.IncomingData, + incomingData *IncomingData, state *cppbridge.State, commitToWal bool, ) ([][]*cppbridge.InnerSeries, cppbridge.RelabelerStats, error) { @@ -187,6 +188,8 @@ func (a Appender[TTask, TLSS, TShard, THead]) Append( } // inputRelabelingStage first stage - relabeling. +// +//revive:disable-next-line:function-length long but this is first stage. func (a Appender[TTask, TLSS, TShard, THead]) inputRelabelingStage( ctx context.Context, state *cppbridge.State, diff --git a/pp/go/storage/appender/models.go b/pp/go/storage/appender/models.go index d70fe1c461..dac36dbbad 100644 --- a/pp/go/storage/appender/models.go +++ b/pp/go/storage/appender/models.go @@ -4,7 +4,6 @@ import ( "sync/atomic" "github.com/prometheus/prometheus/pp/go/cppbridge" - "github.com/prometheus/prometheus/pp/go/storage" ) // @@ -152,18 +151,51 @@ func (sru *ShardedStateUpdates) DataBySourceShard(sourceShardID uint16) ([]*cppb return data, ok } +// +// MetricData +// + +// MetricData is an universal interface for blob protobuf data or batch [model.TimeSeries]. +type MetricData interface { + // Destroy incoming data. + Destroy() +} + +// +// IncomingData +// + +// IncomingData incoming [cppbridge.ShardedData] for shard distribution. +type IncomingData struct { + Hashdex cppbridge.ShardedData + Data MetricData +} + +// Destroy IncomingData. +func (i *IncomingData) Destroy() { + i.Hashdex = nil + if i.Data != nil { + i.Data.Destroy() + } +} + +// ShardedData return hashdex. +func (i *IncomingData) ShardedData() cppbridge.ShardedData { + return i.Hashdex +} + // // DestructibleIncomingData // -// DestructibleIncomingData wrapeer over [storage.IncomingData] with detroy-counter. +// DestructibleIncomingData wrapeer over [IncomingData] with detroy-counter. type DestructibleIncomingData struct { - data *storage.IncomingData + data *IncomingData destructCount atomic.Int64 } // NewDestructibleIncomingData init new [DestructibleIncomingData]. -func NewDestructibleIncomingData(data *storage.IncomingData, destructCount int) *DestructibleIncomingData { +func NewDestructibleIncomingData(data *IncomingData, destructCount int) *DestructibleIncomingData { d := &DestructibleIncomingData{ data: data, } diff --git a/pp/go/storage/constructor.go b/pp/go/storage/constructor.go index d5636fe1a5..46c3bc9ad5 100644 --- a/pp/go/storage/constructor.go +++ b/pp/go/storage/constructor.go @@ -1,135 +1,64 @@ package storage -import ( - "fmt" - "os" - "time" - - "github.com/go-kit/log" - "github.com/jonboulle/clockwork" - "github.com/prometheus/client_golang/prometheus" - - "github.com/prometheus/prometheus/pp/go/storage/catalog" - "github.com/prometheus/prometheus/pp/go/storage/head/container" - "github.com/prometheus/prometheus/pp/go/storage/head/manager" - "github.com/prometheus/prometheus/pp/go/storage/logger" -) - -func HeadManagerCtor( - l log.Logger, - clock clockwork.Clock, - dataDir string, - hcatalog *catalog.Catalog, - blockDuration time.Duration, - maxSegmentSize uint32, - numberOfShards uint16, - registerer prometheus.Registerer, -) (*HeadManager, error) { - dirStat, err := os.Stat(dataDir) - if err != nil { - return nil, fmt.Errorf("failed to stat dir: %w", err) - } - - if !dirStat.IsDir() { - return nil, fmt.Errorf("%s is not directory", dataDir) - } - - InitLogHandler(l) - - builder := NewBuilder( - hcatalog, - dataDir, - maxSegmentSize, - registerer, - ) - - loader := NewLoader( - dataDir, - maxSegmentSize, - registerer, - ) - - h, err := uploadOrBuildHead( - clock, - hcatalog, - builder, - loader, - blockDuration, - numberOfShards, - ) - if err != nil { - return nil, err - } - - if _, err = hcatalog.SetStatus(h.ID(), catalog.StatusActive); err != nil { - return nil, fmt.Errorf("failed to set active status: %w", err) - } - - activeHead := container.NewWeighted(h) - - m := manager.NewManager( - activeHead, - builder, - loader, - numberOfShards, - registerer, - ) - - return m, nil -} - -func uploadOrBuildHead( - clock clockwork.Clock, - hcatalog *catalog.Catalog, - builder *Builder, - loader *Loader, - blockDuration time.Duration, - numberOfShards uint16, -) (*HeadOnDisk, error) { - headRecords := hcatalog.List( - func(record *catalog.Record) bool { - statusIsAppropriate := record.Status() == catalog.StatusNew || - record.Status() == catalog.StatusActive - - isInBlockTimeRange := clock.Now().Sub( - time.UnixMilli(record.CreatedAt()), - ).Milliseconds() < blockDuration.Milliseconds() - - return record.DeletedAt() == 0 && statusIsAppropriate && isInBlockTimeRange - }, - func(lhs, rhs *catalog.Record) bool { - return lhs.CreatedAt() > rhs.CreatedAt() - }, - ) - - if numberOfShards == 0 { - numberOfShards = DefaultNumberOfShards - } - - var generation uint64 - if len(headRecords) == 0 { - // TODO // m.counter.With(prometheus.Labels{"type": "created"}).Inc() - return builder.Build(generation, numberOfShards) - } - - h, corrupted := loader.UploadHead(headRecords[0], generation) - if corrupted { - if !headRecords[0].Corrupted() { - if _, setCorruptedErr := hcatalog.SetCorrupted(headRecords[0].ID()); setCorruptedErr != nil { - logger.Errorf("failed to set corrupted state, head id: %s: %v", headRecords[0].ID(), setCorruptedErr) - } - } - // TODO // m.counter.With(prometheus.Labels{"type": "corrupted"}).Inc() - - if _, err := hcatalog.SetStatus(headRecords[0].ID(), catalog.StatusRotated); err != nil { - logger.Warnf("failed to set rotated status for head {%s}: %s", headRecords[0].ID(), err) - } - - _ = h.Close() - - // TODO // m.counter.With(prometheus.Labels{"type": "created"}).Inc() - return builder.Build(generation, numberOfShards) - } - - return h, nil -} +// func HeadManagerCtor( +// l log.Logger, +// clock clockwork.Clock, +// dataDir string, +// hcatalog *catalog.Catalog, +// blockDuration time.Duration, +// maxSegmentSize uint32, +// numberOfShards uint16, +// registerer prometheus.Registerer, +// ) (*HeadManager, error) { +// dirStat, err := os.Stat(dataDir) +// if err != nil { +// return nil, fmt.Errorf("failed to stat dir: %w", err) +// } + +// if !dirStat.IsDir() { +// return nil, fmt.Errorf("%s is not directory", dataDir) +// } + +// InitLogHandler(l) + +// builder := NewBuilder( +// hcatalog, +// dataDir, +// maxSegmentSize, +// registerer, +// ) + +// loader := NewLoader( +// dataDir, +// maxSegmentSize, +// registerer, +// ) + +// h, err := uploadOrBuildHead( +// clock, +// hcatalog, +// builder, +// loader, +// blockDuration, +// numberOfShards, +// ) +// if err != nil { +// return nil, err +// } + +// if _, err = hcatalog.SetStatus(h.ID(), catalog.StatusActive); err != nil { +// return nil, fmt.Errorf("failed to set active status: %w", err) +// } + +// activeHead := container.NewWeighted(h) + +// m := manager.NewManager( +// activeHead, +// builder, +// loader, +// numberOfShards, +// registerer, +// ) + +// return m, nil +// } diff --git a/pp/go/storage/head/head/head.go b/pp/go/storage/head/head/head.go index df3d90770c..2c56e65a79 100644 --- a/pp/go/storage/head/head/head.go +++ b/pp/go/storage/head/head/head.go @@ -80,6 +80,8 @@ type Head[TShard Shard, TGorutineShard Shard] struct { } // NewHead init new [Head]. +// +//revive:disable-next-line:function-length long but readable. func NewHead[TShard Shard, TGoroutineShard Shard]( id string, shards []TShard, @@ -104,13 +106,14 @@ func NewHead[TShard Shard, TGoroutineShard Shard]( shards: shards, taskChs: taskChs, querySemaphore: locker.NewWeighted(2 * concurrency), // x2 for back pressure + stopc: make(chan struct{}), + wg: sync.WaitGroup{}, // for clearing [Head] metrics memoryInUse: factory.NewGaugeVec(prometheus.GaugeOpts{ Name: "prompp_head_cgo_memory_bytes", Help: "Current value memory in use in bytes.", }, []string{"head_id", "allocator", "shard_id"}), - // for tasks metrics tasksCreated: factory.NewCounterVec(prometheus.CounterOpts{ Name: "prompp_head_task_created_count", @@ -134,7 +137,6 @@ func NewHead[TShard Shard, TGoroutineShard Shard]( runtime.SetFinalizer(h, func(h *Head[TShard, TGoroutineShard]) { h.memoryInUse.DeletePartialMatch(prometheus.Labels{"head_id": h.id}) - logger.Debugf("head %s destroyed", h.String()) }) diff --git a/pp/go/storage/head/manager/manager.go b/pp/go/storage/head/manager/manager.go index 90e3dd0e33..a1eae1f46b 100644 --- a/pp/go/storage/head/manager/manager.go +++ b/pp/go/storage/head/manager/manager.go @@ -1,476 +1,476 @@ package manager -import ( - "context" - "errors" - "fmt" - "time" - - "github.com/prometheus/client_golang/prometheus" - - "github.com/prometheus/prometheus/pp/go/cppbridge" - "github.com/prometheus/prometheus/pp/go/storage/catalog" - "github.com/prometheus/prometheus/pp/go/storage/logger" - "github.com/prometheus/prometheus/pp/go/util" -) - -const ( - // DSMergeOutOfOrderChunks name of task. - DSMergeOutOfOrderChunks = "data_storage_merge_out_of_order_chunks" - - // WalCommit name of task. - WalCommit = "wal_commit" -) - -// -// Timer -// - -// Timer implementation timer. -type Timer interface { - Chan() <-chan time.Time - Reset() - Stop() -} - -// -// GenericTask -// - -// GenericTask the minimum required task [Generic] implementation. -type GenericTask interface { - // Wait for the task to complete on all shards. - Wait() error -} - -// -// DataStorage -// - -// DataStorage the minimum required [DataStorage] implementation. -type DataStorage interface { - // MergeOutOfOrderChunks merge chunks with out of order data chunks. - MergeOutOfOrderChunks() -} - -// -// LSS -// - -// LSS the minimum required [LSS] implementation. -type LSS interface { - // WithRLock calls fn on raws [cppbridge.LabelSetStorage] with read lock. - WithRLock(fn func(target, input *cppbridge.LabelSetStorage) error) error -} - -// -// Wal -// - -// Wal the minimum required Wal implementation for a [Shard]. -type Wal interface { - // Commit finalize segment from encoder and add to wal. - // It is necessary to lock the LSS for reading for the commit. - Commit() error - - // Flush wal segment writer, write all buffered data to storage. - Flush() error -} - -// -// Shard -// - -// Shard the minimum required head [Shard] implementation. -type Shard[TDataStorage DataStorage, TLSS LSS, TWal Wal] interface { - // DataStorage returns shard [DataStorage]. - DataStorage() TDataStorage - - // LSS returns shard labelset storage [LSS]. - LSS() TLSS - - // ShardID returns the shard ID. - ShardID() uint16 - - // Wal returns write-ahead log. - Wal() TWal -} - -// -// Head -// - -// Head the minimum required [Head] implementation. -type Head[ - TGenericTask GenericTask, - TDataStorage DataStorage, - TLSS LSS, - TWal Wal, - TShard, TGShard Shard[TDataStorage, TLSS, TWal], -] interface { - // Close closes wals, query semaphore for the inability to get query and clear metrics. - Close() error - - // CreateTask create a task for operations on the [Head] shards. - CreateTask(taskName string, shardFn func(shard TGShard) error) TGenericTask - - // Enqueue the task to be executed on shards [Head]. - Enqueue(t TGenericTask) - - // Generation returns current generation of [Head]. - Generation() uint64 - - // NumberOfShards returns current number of shards in to [Head]. - NumberOfShards() uint16 - - // RangeShards returns an iterator over the [Head] [Shard]s, through which the shard can be directly accessed. - RangeShards() func(func(TShard) bool) - - // SetReadOnly sets the read-only flag for the [Head]. - SetReadOnly() -} - -// -// ActiveHeadContainer -// - -// ActiveHeadContainer container for active [Head], the minimum required [ActiveHeadContainer] implementation. -type ActiveHeadContainer[ - TGenericTask GenericTask, - TDataStorage DataStorage, - TLSS LSS, - TWal Wal, - TShard, TGShard Shard[TDataStorage, TLSS, TWal], - THead Head[TGenericTask, TDataStorage, TLSS, TWal, TShard, TGShard], -] interface { - // Close closes [ActiveHeadContainer] for the inability work with [Head]. - Close() error - - // Get the active head [Head]. - Get() THead - - // Replace the active head [Head] with a new head. - Replace(ctx context.Context, newHead THead) error - - // With calls fn(h Head). - With(ctx context.Context, fn func(h THead) error) error -} - -// -// Keeper -// - -type Keeper[ - TGenericTask GenericTask, - TDataStorage DataStorage, - TLSS LSS, - TWal Wal, - TShard, TGShard Shard[TDataStorage, TLSS, TWal], - THead Head[TGenericTask, TDataStorage, TLSS, TWal, TShard, TGShard], -] interface { - Add(head THead) - RangeQueriableHeads(mint, maxt int64) func(func(THead) bool) -} - -// -// Loader -// - -// Loader loads [Head] from wal, the minimum required [Loader] implementation. -type Loader[ - TGenericTask GenericTask, - TDataStorage DataStorage, - TLSS LSS, - TWal Wal, - TShard, TGShard Shard[TDataStorage, TLSS, TWal], - THead Head[TGenericTask, TDataStorage, TLSS, TWal, TShard, TGShard], -] interface { - // UploadHead upload [THead] from wal by head ID. - UploadHead(headRecord *catalog.Record, generation uint64) (head THead, corrupted bool) -} - -// HeadBuilder building new [Head] with parameters, the minimum required [HeadBuilder] implementation. -type HeadBuilder[ - TGenericTask GenericTask, - TDataStorage DataStorage, - TLSS LSS, - TWal Wal, - TShard, TGShard Shard[TDataStorage, TLSS, TWal], - THead Head[TGenericTask, TDataStorage, TLSS, TWal, TShard, TGShard], -] interface { - // Build new [Head]. - Build(generation uint64, numberOfShards uint16) (THead, error) -} - -type Manager[ - TGenericTask GenericTask, - TDataStorage DataStorage, - TLSS LSS, - TWal Wal, - TShard, TGShard Shard[TDataStorage, TLSS, TWal], - THead Head[TGenericTask, TDataStorage, TLSS, TWal, TShard, TGShard], -] struct { - activeHead ActiveHeadContainer[TGenericTask, TDataStorage, TLSS, TWal, TShard, TGShard, THead] - keeper Keeper[TGenericTask, TDataStorage, TLSS, TWal, TShard, TGShard, THead] - headBuilder HeadBuilder[TGenericTask, TDataStorage, TLSS, TWal, TShard, TGShard, THead] - headLoader Loader[TGenericTask, TDataStorage, TLSS, TWal, TShard, TGShard, THead] - rotateTimer Timer - commitTimer Timer - mergeTimer Timer - - numberOfShards uint16 - - // TODO closer vs shutdowner - closer *util.Closer - shutdowner *util.GracefulShutdowner - - rotateCounter prometheus.Counter - counter *prometheus.CounterVec -} - -// NewManager init new [Manager] of [Head]s. -func NewManager[ - TGenericTask GenericTask, - TDataStorage DataStorage, - TLSS LSS, - TWal Wal, - TShard, TGShard Shard[TDataStorage, TLSS, TWal], - THead Head[TGenericTask, TDataStorage, TLSS, TWal, TShard, TGShard], -]( - activeHead ActiveHeadContainer[TGenericTask, TDataStorage, TLSS, TWal, TShard, TGShard, THead], - headBuilder HeadBuilder[TGenericTask, TDataStorage, TLSS, TWal, TShard, TGShard, THead], - headLoader Loader[TGenericTask, TDataStorage, TLSS, TWal, TShard, TGShard, THead], - numberOfShards uint16, - registerer prometheus.Registerer, -) *Manager[TGenericTask, TDataStorage, TLSS, TWal, TShard, TGShard, THead] { - factory := util.NewUnconflictRegisterer(registerer) - return &Manager[TGenericTask, TDataStorage, TLSS, TWal, TShard, TGShard, THead]{ - activeHead: activeHead, - headBuilder: headBuilder, - headLoader: headLoader, - - numberOfShards: numberOfShards, - - counter: factory.NewCounterVec( - prometheus.CounterOpts{ - Name: "prompp_head_event_count", - Help: "Number of head events", - }, - []string{"type"}, - ), - } -} - -// ApplyConfig update config. -func (m *Manager[TGenericTask, TDataStorage, TLSS, TWal, TShard, TGShard, THead]) ApplyConfig( - ctx context.Context, - numberOfShards uint16, -) error { - logger.Infof("reconfiguration start") - defer logger.Infof("reconfiguration completed") - - m.numberOfShards = numberOfShards - - h := m.activeHead.Get() - if h.NumberOfShards() == numberOfShards { - return nil - } - - return m.rotate(ctx) -} - -// MergeOutOfOrderChunks merge chunks with out of order data chunks. -func (m *Manager[TGenericTask, TDataStorage, TLSS, TWal, TShard, TGShard, THead]) MergeOutOfOrderChunks( - ctx context.Context, -) error { - return m.activeHead.With(ctx, func(h THead) error { - mergeOutOfOrderChunksWithHead(h) - - return nil - }) -} - -// Run starts processing of the [Manager]. -// TODO implementation. -func (m *Manager[TGenericTask, TDataStorage, TLSS, TWal, TShard, TGShard, THead]) Run(ctx context.Context) error { - go m.loop(ctx) - return nil -} - -// Shutdown safe shutdown [Manager]. -func (m *Manager[TGenericTask, TDataStorage, TLSS, TWal, TShard, TGShard, THead]) Shutdown(ctx context.Context) error { - // TODO - // cgogcErr := rr.cgogc.Shutdown(ctx) - // err := rr.shutdowner.Shutdown(ctx) - activeHeadErr := m.activeHead.Close() - - h := m.activeHead.Get() - commitErr := commitAndFlushViaRange(h) - - headCloseErr := h.Close() - - return errors.Join(activeHeadErr, commitErr, headCloseErr) -} - -// commitToWal commit and flush the accumulated data into the wal. -func (m *Manager[TGenericTask, TDataStorage, TLSS, TWal, TShard, TGShard, THead]) commitToWal( - ctx context.Context, -) error { - return m.activeHead.With(ctx, func(h THead) error { - t := h.CreateTask( - WalCommit, - func(shard TGShard) error { - swal := shard.Wal() - - // wal contains LSS and it is necessary to lock the LSS for reading for the commit. - if err := shard.LSS().WithRLock(func(_, _ *cppbridge.LabelSetStorage) error { - return swal.Commit() - }); err != nil { - return err - } - - return swal.Flush() - }, - ) - h.Enqueue(t) - - return t.Wait() - }) -} - -// TODO implementation. -func (m *Manager[TGenericTask, TDataStorage, TLSS, TWal, TShard, TGShard, THead]) loop(ctx context.Context) { - defer m.closer.Done() - - for { - select { - case <-m.closer.Signal(): - return - - case <-m.commitTimer.Chan(): - if err := m.commitToWal(ctx); err != nil { - logger.Errorf("wal commit failed: %v", err) - } - m.commitTimer.Reset() - - case <-m.mergeTimer.Chan(): - if err := m.MergeOutOfOrderChunks(ctx); err != nil { - logger.Errorf("merge out of order chunks failed: %v", err) - } - m.mergeTimer.Reset() - - case <-m.rotateTimer.Chan(): - logger.Debugf("start rotation") - - if err := m.rotate(ctx); err != nil { - logger.Errorf("rotation failed: %v", err) - } - m.rotateCounter.Inc() - - m.rotateTimer.Reset() - m.commitTimer.Reset() - m.mergeTimer.Reset() - } - } -} - -func (m *Manager[TGenericTask, TDataStorage, TLSS, TWal, TShard, TGShard, THead]) rotate(ctx context.Context) error { - oldHead := m.activeHead.Get() - - newHead, err := m.headBuilder.Build(oldHead.Generation()+1, m.numberOfShards) - if err != nil { - return fmt.Errorf("failed to build a new head: %w", err) - } - - // TODO CopySeriesFrom only old nunber of shards == new - // newHead.CopySeriesFrom(oldHead) - - m.keeper.Add(oldHead) - - // TODO if replace error? - err = m.activeHead.Replace(ctx, newHead) - if err != nil { - return fmt.Errorf("failed to replace old to new head: %w", err) - } - - mergeOutOfOrderChunksWithHead(oldHead) - - if err := commitAndFlushViaRange(oldHead); err != nil { - logger.Warnf("failed commit and flush to wal: %s", err) - } - - oldHead.SetReadOnly() - - return nil -} - -// WithAppendableHead -// TODO implementation. -func (m *Manager[TGenericTask, TDataStorage, TLSS, TWal, TShard, TGShard, THead]) WithAppendableHead( - ctx context.Context, - fn func(h THead) error, -) error { - return m.activeHead.With(ctx, fn) -} - -// RangeQueriableHeads -// TODO implementation. -func (m *Manager[TGenericTask, TDataStorage, TLSS, TWal, TShard, TGShard, THead]) RangeQueriableHeads( - mint, maxt int64, -) func(func(THead) bool) { - // ahead := m.activeHead.Get() - // for h := range m.keeper.RangeQueriableHeads(mint, maxt) { - // TODO - // if h == ahead { - // continue - // } - // } - - return nil -} - -// mergeOutOfOrderChunksWithHead merge chunks with out of order data chunks for [Head]. -func mergeOutOfOrderChunksWithHead[ - TGenericTask GenericTask, - TDataStorage DataStorage, - TLSS LSS, - TWal Wal, - TShard, TGShard Shard[TDataStorage, TLSS, TWal], - THead Head[TGenericTask, TDataStorage, TLSS, TWal, TShard, TGShard], -](h THead) { - t := h.CreateTask( - DSMergeOutOfOrderChunks, - func(shard TGShard) error { - shard.DataStorage().MergeOutOfOrderChunks() - - return nil - }, - ) - h.Enqueue(t) - - _ = t.Wait() -} - -// commitAndFlushViaRange finalize segment from encoder and add to wal -// and flush wal segment writer, write all buffered data to storage. -func commitAndFlushViaRange[ - TGenericTask GenericTask, - TDataStorage DataStorage, - TLSS LSS, - TWal Wal, - TShard, TGShard Shard[TDataStorage, TLSS, TWal], - THead Head[TGenericTask, TDataStorage, TLSS, TWal, TShard, TGShard], -](h THead) error { - errs := make([]error, 0, h.NumberOfShards()*2) - for shard := range h.RangeShards() { - if err := shard.Wal().Commit(); err != nil { - errs = append(errs, fmt.Errorf("commit shard id %d: %w", shard.ShardID(), err)) - } - - if err := shard.Wal().Flush(); err != nil { - errs = append(errs, fmt.Errorf("flush shard id %d: %w", shard.ShardID(), err)) - } - } - - return errors.Join(errs...) -} +// import ( +// "context" +// "errors" +// "fmt" +// "time" + +// "github.com/prometheus/client_golang/prometheus" + +// "github.com/prometheus/prometheus/pp/go/cppbridge" +// "github.com/prometheus/prometheus/pp/go/storage/catalog" +// "github.com/prometheus/prometheus/pp/go/storage/logger" +// "github.com/prometheus/prometheus/pp/go/util" +// ) + +// const ( +// // DSMergeOutOfOrderChunks name of task. +// DSMergeOutOfOrderChunks = "data_storage_merge_out_of_order_chunks" + +// // WalCommit name of task. +// WalCommit = "wal_commit" +// ) + +// // +// // Timer +// // + +// // Timer implementation timer. +// type Timer interface { +// Chan() <-chan time.Time +// Reset() +// Stop() +// } + +// // +// // GenericTask +// // + +// // GenericTask the minimum required task [Generic] implementation. +// type GenericTask interface { +// // Wait for the task to complete on all shards. +// Wait() error +// } + +// // +// // DataStorage +// // + +// // DataStorage the minimum required [DataStorage] implementation. +// type DataStorage interface { +// // MergeOutOfOrderChunks merge chunks with out of order data chunks. +// MergeOutOfOrderChunks() +// } + +// // +// // LSS +// // + +// // LSS the minimum required [LSS] implementation. +// type LSS interface { +// // WithRLock calls fn on raws [cppbridge.LabelSetStorage] with read lock. +// WithRLock(fn func(target, input *cppbridge.LabelSetStorage) error) error +// } + +// // +// // Wal +// // + +// // Wal the minimum required Wal implementation for a [Shard]. +// type Wal interface { +// // Commit finalize segment from encoder and add to wal. +// // It is necessary to lock the LSS for reading for the commit. +// Commit() error + +// // Flush wal segment writer, write all buffered data to storage. +// Flush() error +// } + +// // +// // Shard +// // + +// // Shard the minimum required head [Shard] implementation. +// type Shard[TDataStorage DataStorage, TLSS LSS, TWal Wal] interface { +// // DataStorage returns shard [DataStorage]. +// DataStorage() TDataStorage + +// // LSS returns shard labelset storage [LSS]. +// LSS() TLSS + +// // ShardID returns the shard ID. +// ShardID() uint16 + +// // Wal returns write-ahead log. +// Wal() TWal +// } + +// // +// // Head +// // + +// // Head the minimum required [Head] implementation. +// type Head[ +// TGenericTask GenericTask, +// TDataStorage DataStorage, +// TLSS LSS, +// TWal Wal, +// TShard, TGShard Shard[TDataStorage, TLSS, TWal], +// ] interface { +// // Close closes wals, query semaphore for the inability to get query and clear metrics. +// Close() error + +// // CreateTask create a task for operations on the [Head] shards. +// CreateTask(taskName string, shardFn func(shard TGShard) error) TGenericTask + +// // Enqueue the task to be executed on shards [Head]. +// Enqueue(t TGenericTask) + +// // Generation returns current generation of [Head]. +// Generation() uint64 + +// // NumberOfShards returns current number of shards in to [Head]. +// NumberOfShards() uint16 + +// // RangeShards returns an iterator over the [Head] [Shard]s, through which the shard can be directly accessed. +// RangeShards() func(func(TShard) bool) + +// // SetReadOnly sets the read-only flag for the [Head]. +// SetReadOnly() +// } + +// // +// // ActiveHeadContainer +// // + +// // ActiveHeadContainer container for active [Head], the minimum required [ActiveHeadContainer] implementation. +// type ActiveHeadContainer[ +// TGenericTask GenericTask, +// TDataStorage DataStorage, +// TLSS LSS, +// TWal Wal, +// TShard, TGShard Shard[TDataStorage, TLSS, TWal], +// THead Head[TGenericTask, TDataStorage, TLSS, TWal, TShard, TGShard], +// ] interface { +// // Close closes [ActiveHeadContainer] for the inability work with [Head]. +// Close() error + +// // Get the active head [Head]. +// Get() THead + +// // Replace the active head [Head] with a new head. +// Replace(ctx context.Context, newHead THead) error + +// // With calls fn(h Head). +// With(ctx context.Context, fn func(h THead) error) error +// } + +// // +// // Keeper +// // + +// type Keeper[ +// TGenericTask GenericTask, +// TDataStorage DataStorage, +// TLSS LSS, +// TWal Wal, +// TShard, TGShard Shard[TDataStorage, TLSS, TWal], +// THead Head[TGenericTask, TDataStorage, TLSS, TWal, TShard, TGShard], +// ] interface { +// Add(head THead) +// RangeQueriableHeads(mint, maxt int64) func(func(THead) bool) +// } + +// // +// // Loader +// // + +// // Loader loads [Head] from wal, the minimum required [Loader] implementation. +// type Loader[ +// TGenericTask GenericTask, +// TDataStorage DataStorage, +// TLSS LSS, +// TWal Wal, +// TShard, TGShard Shard[TDataStorage, TLSS, TWal], +// THead Head[TGenericTask, TDataStorage, TLSS, TWal, TShard, TGShard], +// ] interface { +// // UploadHead upload [THead] from wal by head ID. +// UploadHead(headRecord *catalog.Record, generation uint64) (head THead, corrupted bool) +// } + +// // HeadBuilder building new [Head] with parameters, the minimum required [HeadBuilder] implementation. +// type HeadBuilder[ +// TGenericTask GenericTask, +// TDataStorage DataStorage, +// TLSS LSS, +// TWal Wal, +// TShard, TGShard Shard[TDataStorage, TLSS, TWal], +// THead Head[TGenericTask, TDataStorage, TLSS, TWal, TShard, TGShard], +// ] interface { +// // Build new [Head]. +// Build(generation uint64, numberOfShards uint16) (THead, error) +// } + +// type Manager[ +// TGenericTask GenericTask, +// TDataStorage DataStorage, +// TLSS LSS, +// TWal Wal, +// TShard, TGShard Shard[TDataStorage, TLSS, TWal], +// THead Head[TGenericTask, TDataStorage, TLSS, TWal, TShard, TGShard], +// ] struct { +// activeHead ActiveHeadContainer[TGenericTask, TDataStorage, TLSS, TWal, TShard, TGShard, THead] +// keeper Keeper[TGenericTask, TDataStorage, TLSS, TWal, TShard, TGShard, THead] +// headBuilder HeadBuilder[TGenericTask, TDataStorage, TLSS, TWal, TShard, TGShard, THead] +// headLoader Loader[TGenericTask, TDataStorage, TLSS, TWal, TShard, TGShard, THead] +// rotateTimer Timer +// commitTimer Timer +// mergeTimer Timer + +// numberOfShards uint16 + +// // TODO closer vs shutdowner +// closer *util.Closer +// shutdowner *util.GracefulShutdowner + +// rotateCounter prometheus.Counter +// counter *prometheus.CounterVec +// } + +// // NewManager init new [Manager] of [Head]s. +// func NewManager[ +// TGenericTask GenericTask, +// TDataStorage DataStorage, +// TLSS LSS, +// TWal Wal, +// TShard, TGShard Shard[TDataStorage, TLSS, TWal], +// THead Head[TGenericTask, TDataStorage, TLSS, TWal, TShard, TGShard], +// ]( +// activeHead ActiveHeadContainer[TGenericTask, TDataStorage, TLSS, TWal, TShard, TGShard, THead], +// headBuilder HeadBuilder[TGenericTask, TDataStorage, TLSS, TWal, TShard, TGShard, THead], +// headLoader Loader[TGenericTask, TDataStorage, TLSS, TWal, TShard, TGShard, THead], +// numberOfShards uint16, +// registerer prometheus.Registerer, +// ) *Manager[TGenericTask, TDataStorage, TLSS, TWal, TShard, TGShard, THead] { +// factory := util.NewUnconflictRegisterer(registerer) +// return &Manager[TGenericTask, TDataStorage, TLSS, TWal, TShard, TGShard, THead]{ +// activeHead: activeHead, +// headBuilder: headBuilder, +// headLoader: headLoader, + +// numberOfShards: numberOfShards, + +// counter: factory.NewCounterVec( +// prometheus.CounterOpts{ +// Name: "prompp_head_event_count", +// Help: "Number of head events", +// }, +// []string{"type"}, +// ), +// } +// } + +// // ApplyConfig update config. +// func (m *Manager[TGenericTask, TDataStorage, TLSS, TWal, TShard, TGShard, THead]) ApplyConfig( +// ctx context.Context, +// numberOfShards uint16, +// ) error { +// logger.Infof("reconfiguration start") +// defer logger.Infof("reconfiguration completed") + +// m.numberOfShards = numberOfShards + +// h := m.activeHead.Get() +// if h.NumberOfShards() == numberOfShards { +// return nil +// } + +// return m.rotate(ctx) +// } + +// // MergeOutOfOrderChunks merge chunks with out of order data chunks. +// func (m *Manager[TGenericTask, TDataStorage, TLSS, TWal, TShard, TGShard, THead]) MergeOutOfOrderChunks( +// ctx context.Context, +// ) error { +// return m.activeHead.With(ctx, func(h THead) error { +// mergeOutOfOrderChunksWithHead(h) + +// return nil +// }) +// } + +// // Run starts processing of the [Manager]. +// // TODO implementation. +// func (m *Manager[TGenericTask, TDataStorage, TLSS, TWal, TShard, TGShard, THead]) Run(ctx context.Context) error { +// go m.loop(ctx) +// return nil +// } + +// // Shutdown safe shutdown [Manager]. +// func (m *Manager[TGenericTask, TDataStorage, TLSS, TWal, TShard, TGShard, THead]) Shutdown(ctx context.Context) error { +// // TODO +// // cgogcErr := rr.cgogc.Shutdown(ctx) +// // err := rr.shutdowner.Shutdown(ctx) +// activeHeadErr := m.activeHead.Close() + +// h := m.activeHead.Get() +// commitErr := commitAndFlushViaRange(h) + +// headCloseErr := h.Close() + +// return errors.Join(activeHeadErr, commitErr, headCloseErr) +// } + +// // commitToWal commit and flush the accumulated data into the wal. +// func (m *Manager[TGenericTask, TDataStorage, TLSS, TWal, TShard, TGShard, THead]) commitToWal( +// ctx context.Context, +// ) error { +// return m.activeHead.With(ctx, func(h THead) error { +// t := h.CreateTask( +// WalCommit, +// func(shard TGShard) error { +// swal := shard.Wal() + +// // wal contains LSS and it is necessary to lock the LSS for reading for the commit. +// if err := shard.LSS().WithRLock(func(_, _ *cppbridge.LabelSetStorage) error { +// return swal.Commit() +// }); err != nil { +// return err +// } + +// return swal.Flush() +// }, +// ) +// h.Enqueue(t) + +// return t.Wait() +// }) +// } + +// // TODO implementation. +// func (m *Manager[TGenericTask, TDataStorage, TLSS, TWal, TShard, TGShard, THead]) loop(ctx context.Context) { +// defer m.closer.Done() + +// for { +// select { +// case <-m.closer.Signal(): +// return + +// case <-m.commitTimer.Chan(): +// if err := m.commitToWal(ctx); err != nil { +// logger.Errorf("wal commit failed: %v", err) +// } +// m.commitTimer.Reset() + +// case <-m.mergeTimer.Chan(): +// if err := m.MergeOutOfOrderChunks(ctx); err != nil { +// logger.Errorf("merge out of order chunks failed: %v", err) +// } +// m.mergeTimer.Reset() + +// case <-m.rotateTimer.Chan(): +// logger.Debugf("start rotation") + +// if err := m.rotate(ctx); err != nil { +// logger.Errorf("rotation failed: %v", err) +// } +// m.rotateCounter.Inc() + +// m.rotateTimer.Reset() +// m.commitTimer.Reset() +// m.mergeTimer.Reset() +// } +// } +// } + +// func (m *Manager[TGenericTask, TDataStorage, TLSS, TWal, TShard, TGShard, THead]) rotate(ctx context.Context) error { +// oldHead := m.activeHead.Get() + +// newHead, err := m.headBuilder.Build(oldHead.Generation()+1, m.numberOfShards) +// if err != nil { +// return fmt.Errorf("failed to build a new head: %w", err) +// } + +// // TODO CopySeriesFrom only old nunber of shards == new +// // newHead.CopySeriesFrom(oldHead) + +// m.keeper.Add(oldHead) + +// // TODO if replace error? +// err = m.activeHead.Replace(ctx, newHead) +// if err != nil { +// return fmt.Errorf("failed to replace old to new head: %w", err) +// } + +// mergeOutOfOrderChunksWithHead(oldHead) + +// if err := commitAndFlushViaRange(oldHead); err != nil { +// logger.Warnf("failed commit and flush to wal: %s", err) +// } + +// oldHead.SetReadOnly() + +// return nil +// } + +// // WithAppendableHead +// // TODO implementation. +// func (m *Manager[TGenericTask, TDataStorage, TLSS, TWal, TShard, TGShard, THead]) WithAppendableHead( +// ctx context.Context, +// fn func(h THead) error, +// ) error { +// return m.activeHead.With(ctx, fn) +// } + +// // RangeQueriableHeads +// // TODO implementation. +// func (m *Manager[TGenericTask, TDataStorage, TLSS, TWal, TShard, TGShard, THead]) RangeQueriableHeads( +// mint, maxt int64, +// ) func(func(THead) bool) { +// // ahead := m.activeHead.Get() +// // for h := range m.keeper.RangeQueriableHeads(mint, maxt) { +// // TODO +// // if h == ahead { +// // continue +// // } +// // } + +// return nil +// } + +// // mergeOutOfOrderChunksWithHead merge chunks with out of order data chunks for [Head]. +// func mergeOutOfOrderChunksWithHead[ +// TGenericTask GenericTask, +// TDataStorage DataStorage, +// TLSS LSS, +// TWal Wal, +// TShard, TGShard Shard[TDataStorage, TLSS, TWal], +// THead Head[TGenericTask, TDataStorage, TLSS, TWal, TShard, TGShard], +// ](h THead) { +// t := h.CreateTask( +// DSMergeOutOfOrderChunks, +// func(shard TGShard) error { +// shard.DataStorage().MergeOutOfOrderChunks() + +// return nil +// }, +// ) +// h.Enqueue(t) + +// _ = t.Wait() +// } + +// // commitAndFlushViaRange finalize segment from encoder and add to wal +// // and flush wal segment writer, write all buffered data to storage. +// func commitAndFlushViaRange[ +// TGenericTask GenericTask, +// TDataStorage DataStorage, +// TLSS LSS, +// TWal Wal, +// TShard, TGShard Shard[TDataStorage, TLSS, TWal], +// THead Head[TGenericTask, TDataStorage, TLSS, TWal, TShard, TGShard], +// ](h THead) error { +// errs := make([]error, 0, h.NumberOfShards()*2) +// for shard := range h.RangeShards() { +// if err := shard.Wal().Commit(); err != nil { +// errs = append(errs, fmt.Errorf("commit shard id %d: %w", shard.ShardID(), err)) +// } + +// if err := shard.Wal().Flush(); err != nil { +// errs = append(errs, fmt.Errorf("flush shard id %d: %w", shard.ShardID(), err)) +// } +// } + +// return errors.Join(errs...) +// } diff --git a/pp/go/storage/manager.go b/pp/go/storage/manager.go index 36fa54c478..03cee9d302 100644 --- a/pp/go/storage/manager.go +++ b/pp/go/storage/manager.go @@ -352,6 +352,66 @@ func (tn *ReloadBlocksTriggerNotifier) NotifyWritten() { } } +// +// uploadOrBuildHead +// + +func uploadOrBuildHead( + clock clockwork.Clock, + hcatalog *catalog.Catalog, + builder *Builder, + loader *Loader, + blockDuration time.Duration, + numberOfShards uint16, +) (*HeadOnDisk, error) { + headRecords := hcatalog.List( + func(record *catalog.Record) bool { + statusIsAppropriate := record.Status() == catalog.StatusNew || + record.Status() == catalog.StatusActive + + isInBlockTimeRange := clock.Now().Sub( + time.UnixMilli(record.CreatedAt()), + ).Milliseconds() < blockDuration.Milliseconds() + + return record.DeletedAt() == 0 && statusIsAppropriate && isInBlockTimeRange + }, + func(lhs, rhs *catalog.Record) bool { + return lhs.CreatedAt() > rhs.CreatedAt() + }, + ) + + if numberOfShards == 0 { + numberOfShards = DefaultNumberOfShards + } + + var generation uint64 + if len(headRecords) == 0 { + // TODO // m.counter.With(prometheus.Labels{"type": "created"}).Inc() + return builder.Build(generation, numberOfShards) + } + + h, corrupted := loader.UploadHead(headRecords[0], generation) + if corrupted { + if !headRecords[0].Corrupted() { + if _, setCorruptedErr := hcatalog.SetCorrupted(headRecords[0].ID()); setCorruptedErr != nil { + logger.Errorf("failed to set corrupted state, head id: %s: %v", headRecords[0].ID(), setCorruptedErr) + } + } + // TODO // m.counter.With(prometheus.Labels{"type": "corrupted"}).Inc() + + if _, err := hcatalog.SetStatus(headRecords[0].ID(), catalog.StatusRotated); err != nil { + logger.Warnf("failed to set rotated status for head {%s}: %s", headRecords[0].ID(), err) + } + + _ = h.Close() + + // TODO // m.counter.With(prometheus.Labels{"type": "created"}).Inc() + return builder.Build(generation, numberOfShards) + } + + return h, nil +} + // // NoopKeeper // diff --git a/pp/go/storage/models.go b/pp/go/storage/models.go index 51f35124f1..b2261fdc4d 100644 --- a/pp/go/storage/models.go +++ b/pp/go/storage/models.go @@ -1,66 +1,62 @@ package storage -import ( - "github.com/prometheus/prometheus/pp/go/cppbridge" -) - -// -// MetricData -// - -// MetricData is an universal interface for blob protobuf data or batch [model.TimeSeries]. -type MetricData interface { - // Destroy incoming data. - Destroy() -} - -// -// IncomingData -// - -// IncomingData implements. -type IncomingData struct { - Hashdex cppbridge.ShardedData - Data MetricData -} - -// ShardedData return hashdex. -func (i *IncomingData) ShardedData() cppbridge.ShardedData { - return i.Hashdex -} - -// Destroy IncomingData. -func (i *IncomingData) Destroy() { - i.Hashdex = nil - if i.Data != nil { - i.Data.Destroy() - } -} - -// -// HeadStatus -// - -// HeadStatus holds information about all shards. -type HeadStatus struct { - HeadStats HeadStats `json:"headStats"` - SeriesCountByMetricName []HeadStat `json:"seriesCountByMetricName"` - LabelValueCountByLabelName []HeadStat `json:"labelValueCountByLabelName"` - MemoryInBytesByLabelName []HeadStat `json:"memoryInBytesByLabelName"` - SeriesCountByLabelValuePair []HeadStat `json:"seriesCountByLabelValuePair"` -} - -// HeadStat holds the information about individual cardinality. -type HeadStat struct { - Name string `json:"name"` - Value uint64 `json:"value"` -} - -// HeadStats has information about the head. -type HeadStats struct { - NumSeries uint64 `json:"numSeries"` - NumLabelPairs int `json:"numLabelPairs"` - ChunkCount int64 `json:"chunkCount"` - MinTime int64 `json:"minTime"` - MaxTime int64 `json:"maxTime"` -} +// // +// // MetricData +// // + +// // MetricData is an universal interface for blob protobuf data or batch [model.TimeSeries]. +// type MetricData interface { +// // Destroy incoming data. +// Destroy() +// } + +// // +// // IncomingData +// // + +// // IncomingData implements. +// type IncomingData struct { +// Hashdex cppbridge.ShardedData +// Data MetricData +// } + +// // Destroy IncomingData. +// func (i *IncomingData) Destroy() { +// i.Hashdex = nil +// if i.Data != nil { +// i.Data.Destroy() +// } +// } + +// // ShardedData return hashdex. +// func (i *IncomingData) ShardedData() cppbridge.ShardedData { +// return i.Hashdex +// } + +// // +// // HeadStatus +// // + +// // HeadStatus holds information about all shards. +// type HeadStatus struct { +// HeadStats HeadStats `json:"headStats"` +// SeriesCountByMetricName []HeadStat `json:"seriesCountByMetricName"` +// LabelValueCountByLabelName []HeadStat `json:"labelValueCountByLabelName"` +// MemoryInBytesByLabelName []HeadStat `json:"memoryInBytesByLabelName"` +// SeriesCountByLabelValuePair []HeadStat `json:"seriesCountByLabelValuePair"` +// } + +// // HeadStat holds the information about individual cardinality. +// type HeadStat struct { +// Name string `json:"name"` +// Value uint64 `json:"value"` +// } + +// // HeadStats has information about the head. +// type HeadStats struct { +// NumSeries uint64 `json:"numSeries"` +// NumLabelPairs int `json:"numLabelPairs"` +// ChunkCount int64 `json:"chunkCount"` +// MinTime int64 `json:"minTime"` +// MaxTime int64 `json:"maxTime"` +// } diff --git a/pp/go/storage/types.go b/pp/go/storage/types.go index 9dcb11a354..65967639b1 100644 --- a/pp/go/storage/types.go +++ b/pp/go/storage/types.go @@ -2,8 +2,9 @@ package storage import ( "github.com/prometheus/prometheus/pp/go/cppbridge" + "github.com/prometheus/prometheus/pp/go/storage/appender" "github.com/prometheus/prometheus/pp/go/storage/head/head" - "github.com/prometheus/prometheus/pp/go/storage/head/manager" + "github.com/prometheus/prometheus/pp/go/storage/head/proxy" "github.com/prometheus/prometheus/pp/go/storage/head/shard" "github.com/prometheus/prometheus/pp/go/storage/head/shard/wal" "github.com/prometheus/prometheus/pp/go/storage/head/shard/wal/writer" @@ -26,13 +27,12 @@ type PerGoroutineShard = shard.PerGoroutineShard[*WalOnDisk] // HeadOnDisk [head.Head] with [ShardOnDisk]. type HeadOnDisk = head.Head[*ShardOnDisk, *PerGoroutineShard] -// HeadManager [manager.Manager] for [HeadOnDisk]s. -type HeadManager = manager.Manager[ +// ProxyHead [proxy.Proxy] for [HeadOnDisk]s. +type ProxyHead = proxy.Proxy[*HeadOnDisk] + +type AppenderHead = appender.Appender[ *task.Generic[*PerGoroutineShard], - *shard.DataStorage, *shard.LSS, - *WalOnDisk, - *ShardOnDisk, *PerGoroutineShard, *HeadOnDisk, ] From d9e0fac5f1811303fa2d92f8dd5b4c0fa934c9af Mon Sep 17 00:00:00 2001 From: Alexandr Yudin Date: Thu, 11 Sep 2025 07:42:26 +0000 Subject: [PATCH 34/96] fix walreader --- pp/go/storage/head/head/head.go | 1 + pp/go/storage/head/shard/wal/reader/segment.go | 5 +++++ pp/go/storage/head/shard/wal/wal_reader.go | 13 +++++++++---- pp/go/storage/loader.go | 2 +- 4 files changed, 16 insertions(+), 5 deletions(-) diff --git a/pp/go/storage/head/head/head.go b/pp/go/storage/head/head/head.go index 2c56e65a79..ef3d871fb9 100644 --- a/pp/go/storage/head/head/head.go +++ b/pp/go/storage/head/head/head.go @@ -54,6 +54,7 @@ type Shard interface { // Head // +// Head stores and manages shards, handles reads and writes of time series data within a time window. type Head[TShard Shard, TGorutineShard Shard] struct { id string generation uint64 diff --git a/pp/go/storage/head/shard/wal/reader/segment.go b/pp/go/storage/head/shard/wal/reader/segment.go index 3e3c7c4773..7c2f35d000 100644 --- a/pp/go/storage/head/shard/wal/reader/segment.go +++ b/pp/go/storage/head/shard/wal/reader/segment.go @@ -13,6 +13,11 @@ type Segment struct { sampleCount uint32 } +// NewSegment init new [Segment]. +func NewSegment() *Segment { + return &Segment{} +} + // Bytes returns the data read. func (s *Segment) Bytes() []byte { return s.data diff --git a/pp/go/storage/head/shard/wal/wal_reader.go b/pp/go/storage/head/shard/wal/wal_reader.go index 658072a946..aaba648156 100644 --- a/pp/go/storage/head/shard/wal/wal_reader.go +++ b/pp/go/storage/head/shard/wal/wal_reader.go @@ -22,20 +22,25 @@ type ReadSegment[T any] interface { // SegmentWalReader buffered reader [ReadSegment]s from wal. type SegmentWalReader[T any, TReadSegment ReadSegment[T]] struct { - reader *bufio.Reader + reader *bufio.Reader + segmentCtor func() TReadSegment } // NewSegmentWalReader init new [SegmentWalReader]. -func NewSegmentWalReader[T any, TReadSegment ReadSegment[T]](r io.Reader) *SegmentWalReader[T, TReadSegment] { +func NewSegmentWalReader[T any, TReadSegment ReadSegment[T]]( + r io.Reader, + segmentCtor func() TReadSegment, +) *SegmentWalReader[T, TReadSegment] { return &SegmentWalReader[T, TReadSegment]{ - reader: bufio.NewReaderSize(r, 1024*1024*4), + reader: bufio.NewReaderSize(r, 1024*1024*4), + segmentCtor: segmentCtor, } } // ForEachSegment reads [ReadSegment]s from the reader and for each [ReadSegment] a [do] is called for each, // if an error occurs during reading it will return and reading will stop. func (r *SegmentWalReader[T, TReadSegment]) ForEachSegment(do func(TReadSegment) error) error { - var segment TReadSegment + segment := r.segmentCtor() for { segment.Reset() diff --git a/pp/go/storage/loader.go b/pp/go/storage/loader.go index 91e433d9d6..51caf5329b 100644 --- a/pp/go/storage/loader.go +++ b/pp/go/storage/loader.go @@ -137,7 +137,7 @@ func (l *Loader) UploadShard( decoder := cppbridge.NewHeadWalDecoder(lss.Target(), encoderVersion) dataStorage := shard.NewDataStorage() - if err = wal.NewSegmentWalReader[reader.Segment](shardFile).ForEachSegment(func(s *reader.Segment) error { + if err = wal.NewSegmentWalReader(shardFile, reader.NewSegment).ForEachSegment(func(s *reader.Segment) error { if decodeErr := dataStorage.DecodeSegment(decoder, s.Bytes()); decodeErr != nil { return fmt.Errorf("failed to decode segment: %w", decodeErr) } From 64d12f0367e8cf24a49c2a3ab4a5e527aba11adc Mon Sep 17 00:00:00 2001 From: Alexandr Yudin Date: Thu, 11 Sep 2025 07:43:50 +0000 Subject: [PATCH 35/96] rebuild sreader --- pp/go/storage/head/shard/wal/wal_reader.go | 15 ++++++--------- 1 file changed, 6 insertions(+), 9 deletions(-) diff --git a/pp/go/storage/head/shard/wal/wal_reader.go b/pp/go/storage/head/shard/wal/wal_reader.go index aaba648156..2aa637fe06 100644 --- a/pp/go/storage/head/shard/wal/wal_reader.go +++ b/pp/go/storage/head/shard/wal/wal_reader.go @@ -8,30 +8,27 @@ import ( ) // ReadSegment the minimum required [Segment] implementation for a [Wal]. -type ReadSegment[T any] interface { +type ReadSegment interface { // ReadFrom reads [ReadSegment] data from r [io.Reader]. The return value n is the number of bytes read. // Any error encountered during the read is also returned. ReadFrom(r io.Reader) (int64, error) // Reset [ReadSegment] data. Reset() - - // for use as a pointer - *T } // SegmentWalReader buffered reader [ReadSegment]s from wal. -type SegmentWalReader[T any, TReadSegment ReadSegment[T]] struct { +type SegmentWalReader[TReadSegment ReadSegment] struct { reader *bufio.Reader segmentCtor func() TReadSegment } // NewSegmentWalReader init new [SegmentWalReader]. -func NewSegmentWalReader[T any, TReadSegment ReadSegment[T]]( +func NewSegmentWalReader[TReadSegment ReadSegment]( r io.Reader, segmentCtor func() TReadSegment, -) *SegmentWalReader[T, TReadSegment] { - return &SegmentWalReader[T, TReadSegment]{ +) *SegmentWalReader[TReadSegment] { + return &SegmentWalReader[TReadSegment]{ reader: bufio.NewReaderSize(r, 1024*1024*4), segmentCtor: segmentCtor, } @@ -39,7 +36,7 @@ func NewSegmentWalReader[T any, TReadSegment ReadSegment[T]]( // ForEachSegment reads [ReadSegment]s from the reader and for each [ReadSegment] a [do] is called for each, // if an error occurs during reading it will return and reading will stop. -func (r *SegmentWalReader[T, TReadSegment]) ForEachSegment(do func(TReadSegment) error) error { +func (r *SegmentWalReader[TReadSegment]) ForEachSegment(do func(TReadSegment) error) error { segment := r.segmentCtor() for { segment.Reset() From a57a26eb6665d072186f0bd3c69b09163d46ea6f Mon Sep 17 00:00:00 2001 From: Alexandr Yudin Date: Thu, 11 Sep 2025 12:30:19 +0000 Subject: [PATCH 36/96] for save --- pp-pkg/storage/adapter.go | 1 + 1 file changed, 1 insertion(+) diff --git a/pp-pkg/storage/adapter.go b/pp-pkg/storage/adapter.go index 8f767d3fe5..129b05f4e9 100644 --- a/pp-pkg/storage/adapter.go +++ b/pp-pkg/storage/adapter.go @@ -46,6 +46,7 @@ var _ storage.Storage = (*Adapter)(nil) // Adapter for implementing the [Queryable] interface and append data. type Adapter struct { proxy *pp_storage.ProxyHead + cfg *pp_storage.Config haTracker *hatracker.HighAvailabilityTracker hashdexFactory cppbridge.HashdexFactory hashdexLimits cppbridge.WALHashdexLimits From 7a7e2596444e97cecc1f0f63811c1995166b294c Mon Sep 17 00:00:00 2001 From: Alexandr Yudin Date: Tue, 16 Sep 2025 08:10:40 +0000 Subject: [PATCH 37/96] for save --- config/pp_scrape_config.go | 17 + pp-pkg/scrape/scrape.go | 4 + pp-pkg/storage/adapter.go | 33 +- pp-pkg/storage/appender.go | 4 +- pp/go/cppbridge/prometheus_relabeler.go | 335 ++++++++++++++++--- pp/go/cppbridge/prometheus_relabeler_test.go | 35 +- pp/go/storage/appender/appender.go | 11 +- 7 files changed, 354 insertions(+), 85 deletions(-) create mode 100644 config/pp_scrape_config.go diff --git a/config/pp_scrape_config.go b/config/pp_scrape_config.go new file mode 100644 index 0000000000..c3de0253d4 --- /dev/null +++ b/config/pp_scrape_config.go @@ -0,0 +1,17 @@ +package config + +import "github.com/prometheus/prometheus/pp/go/cppbridge" + +// +// ScrapeConfig +// + +// PPMetricRelabelConfigs returns slice the converted [relabel.Config] to the [cppbridge.RelabelConfig]'s. +func (c *ScrapeConfig) PPMetricRelabelConfigs() ([]*cppbridge.RelabelConfig, error) { + cfgs, err := convertingRelabelConfigs(c.MetricRelabelConfigs) + if err != nil { + return nil, err + } + + return cfgs, nil +} diff --git a/pp-pkg/scrape/scrape.go b/pp-pkg/scrape/scrape.go index da23a2a5d8..608f3fd388 100644 --- a/pp-pkg/scrape/scrape.go +++ b/pp-pkg/scrape/scrape.go @@ -192,6 +192,10 @@ func newScrapePool( httpOpts: options.HTTPClientOptions, noDefaultPort: options.NoDefaultPort, } + + // cfg.RelabelConfigs + // scrapeState, err := cppbridge.NewEmptyState() + sp.newLoop = func(opts scrapeLoopOptions) loop { // Update the targets retrieval function for metadata to a new scrape cache. cache := opts.cache diff --git a/pp-pkg/storage/adapter.go b/pp-pkg/storage/adapter.go index 129b05f4e9..2a140b300e 100644 --- a/pp-pkg/storage/adapter.go +++ b/pp-pkg/storage/adapter.go @@ -45,11 +45,11 @@ var _ storage.Storage = (*Adapter)(nil) // Adapter for implementing the [Queryable] interface and append data. type Adapter struct { - proxy *pp_storage.ProxyHead - cfg *pp_storage.Config - haTracker *hatracker.HighAvailabilityTracker - hashdexFactory cppbridge.HashdexFactory - hashdexLimits cppbridge.WALHashdexLimits + proxy *pp_storage.ProxyHead + haTracker *hatracker.HighAvailabilityTracker + hashdexFactory cppbridge.HashdexFactory + hashdexLimits cppbridge.WALHashdexLimits + transparentState *cppbridge.StateV2 activeQuerierMetrics *querier.Metrics storageQuerierMetrics *querier.Metrics @@ -60,22 +60,28 @@ func NewAdapter( clock clockwork.Clock, proxy *pp_storage.ProxyHead, registerer prometheus.Registerer, -) *Adapter { +) (*Adapter, error) { + transparentState, err := cppbridge.NewEmptyStateV2WithConfig([]*cppbridge.RelabelConfig{}) + if err != nil { + return nil, err + } + return &Adapter{ proxy: proxy, haTracker: hatracker.NewHighAvailabilityTracker(clock, registerer), hashdexFactory: cppbridge.HashdexFactory{}, hashdexLimits: cppbridge.DefaultWALHashdexLimits(), + transparentState: transparentState, activeQuerierMetrics: querier.NewMetrics(registerer, querier.QueryableAppenderSource), storageQuerierMetrics: querier.NewMetrics(registerer, querier.QueryableStorageSource), - } + }, nil } // AppendHashdex append incoming [cppbridge.HashdexContent] to [Head]. func (ar *Adapter) AppendHashdex( ctx context.Context, hashdex cppbridge.ShardedData, - state *cppbridge.State, + state *cppbridge.StateV2, commitToWal bool, ) error { if ar.haTracker.IsDrop(hashdex.Cluster(), hashdex.Replica()) { @@ -98,7 +104,7 @@ func (ar *Adapter) AppendHashdex( func (ar *Adapter) AppendScraperHashdex( ctx context.Context, hashdex cppbridge.ShardedData, - state *cppbridge.State, + state *cppbridge.StateV2, commitToWal bool, ) (stats cppbridge.RelabelerStats, err error) { _ = ar.proxy.With(ctx, func(h *pp_storage.HeadOnDisk) error { @@ -119,7 +125,7 @@ func (ar *Adapter) AppendScraperHashdex( func (ar *Adapter) AppendSnappyProtobuf( ctx context.Context, compressedData ProtobufData, - state *cppbridge.State, + state *cppbridge.StateV2, commitToWal bool, ) error { hx, err := cppbridge.NewWALSnappyProtobufHashdex(compressedData.Bytes(), ar.hashdexLimits) @@ -148,7 +154,7 @@ func (ar *Adapter) AppendSnappyProtobuf( func (ar *Adapter) AppendTimeSeries( ctx context.Context, data TimeSeriesBatch, - state *cppbridge.State, + state *cppbridge.StateV2, commitToWal bool, ) (stats cppbridge.RelabelerStats, err error) { hx, err := ar.hashdexFactory.GoModel(data.TimeSeries(), ar.hashdexLimits) @@ -178,10 +184,7 @@ func (ar *Adapter) AppendTimeSeries( // Appender create a new [storage.Appender] for [Head]. func (ar *Adapter) Appender(ctx context.Context) storage.Appender { - // TODO state *cppbridge.State - var state *cppbridge.State - - return newTimeSeriesAppender(ctx, ar, state) + return newTimeSeriesAppender(ctx, ar, ar.transparentState) } // ChunkQuerier provides querying access over time series data of a fixed time range. diff --git a/pp-pkg/storage/appender.go b/pp-pkg/storage/appender.go index ee29ba66f0..2db4f3b731 100644 --- a/pp-pkg/storage/appender.go +++ b/pp-pkg/storage/appender.go @@ -32,14 +32,14 @@ func (d *timeSeriesBatch) Destroy() { type TimeSeriesAppender struct { ctx context.Context adapter *Adapter - state *cppbridge.State + state *cppbridge.StateV2 batch *timeSeriesBatch } func newTimeSeriesAppender( ctx context.Context, adapter *Adapter, - state *cppbridge.State, + state *cppbridge.StateV2, ) *TimeSeriesAppender { return &TimeSeriesAppender{ ctx: ctx, diff --git a/pp/go/cppbridge/prometheus_relabeler.go b/pp/go/cppbridge/prometheus_relabeler.go index 309fce2422..c297571bc4 100644 --- a/pp/go/cppbridge/prometheus_relabeler.go +++ b/pp/go/cppbridge/prometheus_relabeler.go @@ -986,7 +986,6 @@ func (c *Cache) Update(ctx context.Context, shardsRelabelerStateUpdate []*Relabe type State struct { caches []*Cache staleNansStates []*StaleNansState - statelessRelabeler *StatelessRelabeler defTimestamp int64 generationRelabeler uint64 generationHead uint64 @@ -1017,10 +1016,6 @@ func (s *State) CacheByShard(shardID uint16) *Cache { )) } - if s.caches[shardID] == nil { - s.caches[shardID] = NewCache() - } - return s.caches[shardID] } @@ -1072,11 +1067,6 @@ func (s *State) SetRelabelerOptions(options *RelabelerOptions) { s.options = *options } -// SetStatelessRelabeler set [StatelessRelabeler] for [PerGoroutineRelabeler]. -func (s *State) SetStatelessRelabeler(relabeler *StatelessRelabeler) { - s.statelessRelabeler = relabeler -} - // StaleNansStateByShard return SourceStaleNansState for shard. func (s *State) StaleNansStateByShard(shardID uint16) *StaleNansState { if int(shardID) >= len(s.staleNansStates) { @@ -1087,19 +1077,9 @@ func (s *State) StaleNansStateByShard(shardID uint16) *StaleNansState { )) } - if s.staleNansStates[shardID] == nil { - s.staleNansStates[shardID] = NewStaleNansState() - } - return s.staleNansStates[shardID] } -// StatelessRelabeler returns [StatelessRelabeler] for [PerGoroutineRelabeler]. -func (s *State) StatelessRelabeler() *StatelessRelabeler { - // TODO validate nil and reconfigure - return s.statelessRelabeler -} - // TrackStaleness return state track stalenans. func (s *State) TrackStaleness() bool { return s.trackStaleness @@ -1113,21 +1093,21 @@ func (s *State) resetCaches(numberOfShards uint16, equaledGeneration bool) { return } - for shardID := range s.caches { - s.caches[shardID] = nil - } + switch { + case len(s.caches) > int(numberOfShards): + for shardID := range s.caches[numberOfShards:] { + s.caches[shardID] = nil + } - if len(s.caches) > int(numberOfShards) { // cut s.caches = s.caches[:numberOfShards] + case len(s.caches) < int(numberOfShards): + // grow + s.caches = make([]*Cache, numberOfShards) } - if len(s.caches) < int(numberOfShards) { - // grow - s.caches = append( - s.caches, - make([]*Cache, int(numberOfShards)-len(s.caches))..., - ) + for shardID := range s.caches { + s.caches[shardID] = NewCache() } } @@ -1143,24 +1123,21 @@ func (s *State) resetStaleNansStates(numberOfShards uint16, equaledGeneration bo return } - for shardID := range s.staleNansStates { - state := s.staleNansStates[shardID] - if state != nil { - state.Reset() + switch { + case len(s.staleNansStates) > int(numberOfShards): + for shardID := range s.staleNansStates[numberOfShards:] { + s.staleNansStates[shardID] = nil } - } - if len(s.staleNansStates) > int(numberOfShards) { // cut s.staleNansStates = s.staleNansStates[:numberOfShards] + case len(s.staleNansStates) < int(numberOfShards): + // grow + s.staleNansStates = make([]*StaleNansState, numberOfShards) } - if len(s.staleNansStates) < int(numberOfShards) { - // grow - s.staleNansStates = append( - s.staleNansStates, - make([]*StaleNansState, int(numberOfShards)-len(s.staleNansStates))..., - ) + for shardID := range s.staleNansStates { + s.staleNansStates[shardID] = NewStaleNansState() } } @@ -1219,7 +1196,7 @@ func (pgr *PerGoroutineRelabeler) InputRelabeling( ctx context.Context, inputLss *LabelSetStorage, targetLss *LabelSetStorage, - state *State, + state *StateV2, shardedData ShardedData, shardsInnerSeries []*InnerSeries, shardsRelabeledSeries []*RelabeledSeries, @@ -1262,7 +1239,7 @@ func (pgr *PerGoroutineRelabeler) InputRelabelingFromCache( ctx context.Context, inputLss *LabelSetStorage, targetLss *LabelSetStorage, - state *State, + state *StateV2, shardedData ShardedData, shardsInnerSeries []*InnerSeries, ) (RelabelerStats, bool, error) { @@ -1302,7 +1279,7 @@ func (pgr *PerGoroutineRelabeler) InputRelabelingWithStalenans( ctx context.Context, inputLss *LabelSetStorage, targetLss *LabelSetStorage, - state *State, + state *StateV2, shardedData ShardedData, shardsInnerSeries []*InnerSeries, shardsRelabeledSeries []*RelabeledSeries, @@ -1347,7 +1324,7 @@ func (pgr *PerGoroutineRelabeler) InputRelabelingWithStalenansFromCache( ctx context.Context, inputLss *LabelSetStorage, targetLss *LabelSetStorage, - state *State, + state *StateV2, shardedData ShardedData, shardsInnerSeries []*InnerSeries, ) (RelabelerStats, bool, error) { @@ -1389,7 +1366,7 @@ func (pgr *PerGoroutineRelabeler) Relabeling( ctx context.Context, inputLss *LabelSetStorage, targetLss *LabelSetStorage, - state *State, + state *StateV2, shardedData ShardedData, shardsInnerSeries []*InnerSeries, shardsRelabeledSeries []*RelabeledSeries, @@ -1422,7 +1399,7 @@ func (pgr *PerGoroutineRelabeler) RelabelingFromCache( ctx context.Context, inputLss *LabelSetStorage, targetLss *LabelSetStorage, - state *State, + state *StateV2, shardedData ShardedData, shardsInnerSeries []*InnerSeries, ) (RelabelerStats, bool, error) { @@ -1468,3 +1445,265 @@ func (pgr *PerGoroutineRelabeler) UpdateRelabelerState( return handleException(exception) } + +// +// TransitionLocker +// + +// TransitionLocker is an implementing [sync.Mutex] that, depending on the situation, does not block. +type TransitionLocker struct { + mx sync.Mutex + lock bool +} + +// NewTransitionLocker init new [TransitionLocker]. +func NewTransitionLocker() TransitionLocker { + return TransitionLocker{ + mx: sync.Mutex{}, + lock: true, + } +} + +// NewTransitionLockerWithoutLock init new [TransitionLocker], without locks. +func NewTransitionLockerWithoutLock() TransitionLocker { + return TransitionLocker{ + mx: sync.Mutex{}, + lock: false, + } +} + +// Lock locks rw for writing, if need. +func (l *TransitionLocker) Lock() { + if l.lock { + l.mx.Lock() + } +} + +// Unlock unlocks rw for writing, if need. +func (l *TransitionLocker) Unlock() { + if l.lock { + l.mx.Unlock() + } +} + +// +// StateV2 +// + +const ( + initStatus uint8 = 0 + inited uint8 = 15 + transitionStatus uint8 = 240 +) + +// StateV2 of relabelers per shard. +type StateV2 struct { + caches []*Cache + staleNansStates []*StaleNansState + statelessRelabeler *StatelessRelabeler + locker TransitionLocker + defTimestamp int64 + generationHead uint64 + options RelabelerOptions + status uint8 + trackStaleness bool +} + +// NewTransitionStateV2WithLock init empty [StateV2], with locks. +func NewTransitionStateV2WithLock() *StateV2 { + return &StateV2{ + locker: NewTransitionLocker(), + generationHead: math.MaxUint64, + status: transitionStatus, + trackStaleness: false, + } +} + +// NewTransitionStateV2WithoutLock init empty [StateV2], without locks. +func NewTransitionStateV2WithoutLock() *StateV2 { + return &StateV2{ + locker: NewTransitionLockerWithoutLock(), + generationHead: math.MaxUint64, + status: transitionStatus, + trackStaleness: false, + } +} + +// NewStateV2WithLock init empty [StateV2], with locks. +func NewStateV2WithLock() *StateV2 { + return &StateV2{ + locker: NewTransitionLocker(), + generationHead: math.MaxUint64, + status: initStatus, + trackStaleness: false, + } +} + +// NewStateV2WithoutLock init empty [StateV2], without locks. +func NewStateV2WithoutLock() *StateV2 { + return &StateV2{ + locker: NewTransitionLockerWithoutLock(), + generationHead: math.MaxUint64, + status: initStatus, + trackStaleness: false, + } +} + +// CacheByShard return *Cache for shard. +func (s *StateV2) CacheByShard(shardID uint16) *Cache { + if int(shardID) >= len(s.caches) { + panic(fmt.Sprintf( + "shardID(%d) out of range in caches(%d)", + shardID, + len(s.caches), + )) + } + + return s.caches[shardID] +} + +// DefTimestamp return timestamp for scrape time and stalenan. +func (s *StateV2) DefTimestamp() int64 { + if s.defTimestamp == 0 { + return time.Now().UnixMilli() + } + + return s.defTimestamp +} + +// DisableTrackStaleness disable track stalenans. +func (s *StateV2) DisableTrackStaleness() { + s.trackStaleness = false +} + +// EnableTrackStaleness enable track stalenans. +func (s *StateV2) EnableTrackStaleness() { + s.trackStaleness = true +} + +// Reconfigure recreate caches and stalenans states if need and set new generations. +func (s *StateV2) Reconfigure( + generationHead uint64, + numberOfShards uint16, +) { + if s.status&inited == inited && generationHead == s.generationHead { + return + } + + // long way + s.locker.Lock() + + // we check it a second time, but under lock + if s.status&inited == inited && generationHead == s.generationHead { + s.locker.Unlock() + return + } + + // the transition state does not require caches and staleNaNs + if s.status&transitionStatus == transitionStatus { + s.status |= inited + s.generationHead = generationHead + s.locker.Unlock() + return + } + + s.resetCaches(numberOfShards) + s.resetStaleNansStates(numberOfShards) + + s.locker.Unlock() +} + +// RelabelerOptions return Options for relabeler. +func (s *StateV2) RelabelerOptions() RelabelerOptions { + return s.options +} + +// SetDefTimestamp set timestamp for scrape time and stalenan. +func (s *StateV2) SetDefTimestamp(ts int64) { + s.defTimestamp = ts +} + +// SetRelabelerOptions set Options for relabeler. +func (s *StateV2) SetRelabelerOptions(options *RelabelerOptions) { + s.options = *options +} + +// SetStatelessRelabeler sets [StatelessRelabeler] for [PerGoroutineRelabeler]. +func (s *StateV2) SetStatelessRelabeler(statelessRelabeler *StatelessRelabeler) { + if s.status&transitionStatus == transitionStatus { + panic("state is transition") + } + + s.statelessRelabeler = statelessRelabeler +} + +// StaleNansStateByShard return SourceStaleNansState for shard. +func (s *StateV2) StaleNansStateByShard(shardID uint16) *StaleNansState { + if int(shardID) >= len(s.staleNansStates) { + panic(fmt.Sprintf( + "shardID(%d) out of range in staleNansStates(%d)", + shardID, + len(s.caches), + )) + } + + return s.staleNansStates[shardID] +} + +// StatelessRelabeler returns [StatelessRelabeler] for [PerGoroutineRelabeler]. +func (s *StateV2) StatelessRelabeler() *StatelessRelabeler { + if s.status&transitionStatus == transitionStatus { + panic("state is transition") + } + + return s.statelessRelabeler +} + +// TrackStaleness return state track stalenans. +func (s *StateV2) TrackStaleness() bool { + return s.trackStaleness +} + +// resetCaches recreate Caches. +func (s *StateV2) resetCaches(numberOfShards uint16) { + switch { + case len(s.caches) > int(numberOfShards): + for shardID := range s.caches[numberOfShards:] { + s.caches[shardID] = nil + } + + // cut + s.caches = s.caches[:numberOfShards] + case len(s.caches) < int(numberOfShards): + // grow + s.caches = make([]*Cache, numberOfShards) + } + + for shardID := range s.caches { + s.caches[shardID] = NewCache() + } +} + +// resetStaleNansStates recreate StaleNansStates. +func (s *StateV2) resetStaleNansStates(numberOfShards uint16) { + if !s.trackStaleness { + return + } + + switch { + case len(s.staleNansStates) > int(numberOfShards): + for shardID := range s.staleNansStates[numberOfShards:] { + s.staleNansStates[shardID] = nil + } + + // cut + s.staleNansStates = s.staleNansStates[:numberOfShards] + case len(s.staleNansStates) < int(numberOfShards): + // grow + s.staleNansStates = make([]*StaleNansState, numberOfShards) + } + + for shardID := range s.staleNansStates { + s.staleNansStates[shardID] = NewStaleNansState() + } +} diff --git a/pp/go/cppbridge/prometheus_relabeler_test.go b/pp/go/cppbridge/prometheus_relabeler_test.go index ee56df001b..ed7ed3532f 100644 --- a/pp/go/cppbridge/prometheus_relabeler_test.go +++ b/pp/go/cppbridge/prometheus_relabeler_test.go @@ -639,9 +639,6 @@ func (s *RelabelerSuite) TestInputPerGoroutineRelabeler() { inputLss := cppbridge.NewLssStorage() targetLss := cppbridge.NewQueryableLssStorage() - statelessRelabeler, err := cppbridge.NewStatelessRelabeler(rCfgs) - s.Require().NoError(err) - var numberOfShards uint16 = 1 hlimits := cppbridge.DefaultWALHashdexLimits() @@ -650,9 +647,14 @@ func (s *RelabelerSuite) TestInputPerGoroutineRelabeler() { shardsInnerSeries := cppbridge.NewShardsInnerSeries(numberOfShards) shardsRelabeledSeries := cppbridge.NewShardsRelabeledSeries(numberOfShards) - state := cppbridge.NewState(numberOfShards) + + statelessRelabeler, err := cppbridge.NewStatelessRelabeler(rCfgs) + s.Require().NoError(err) + + state := cppbridge.NewStateV2WithoutLock() state.SetRelabelerOptions(&s.options) state.SetStatelessRelabeler(statelessRelabeler) + state.Reconfigure(0, numberOfShards) pgr := cppbridge.NewPerGoroutineRelabeler(numberOfShards, 0) stats, hasReallocations, err := pgr.InputRelabeling( @@ -698,9 +700,6 @@ func (s *RelabelerSuite) TestInputPerGoroutineRelabelerFromCacheTrue() { inputLss := cppbridge.NewLssStorage() targetLss := cppbridge.NewQueryableLssStorage() - statelessRelabeler, err := cppbridge.NewStatelessRelabeler(rCfgs) - s.Require().NoError(err) - var numberOfShards uint16 = 1 hlimits := cppbridge.DefaultWALHashdexLimits() @@ -709,9 +708,14 @@ func (s *RelabelerSuite) TestInputPerGoroutineRelabelerFromCacheTrue() { shardsInnerSeries := cppbridge.NewShardsInnerSeries(numberOfShards) shardsRelabeledSeries := cppbridge.NewShardsRelabeledSeries(numberOfShards) - state := cppbridge.NewState(numberOfShards) - state.SetRelabelerOptions(&s.options) + + statelessRelabeler, err := cppbridge.NewStatelessRelabeler(rCfgs) + s.Require().NoError(err) + + state := cppbridge.NewStateV2WithoutLock() state.SetStatelessRelabeler(statelessRelabeler) + state.SetRelabelerOptions(&s.options) + state.Reconfigure(0, numberOfShards) pgr := cppbridge.NewPerGoroutineRelabeler(numberOfShards, 0) stats, hasReallocations, err := pgr.InputRelabeling( @@ -777,8 +781,9 @@ func (s *RelabelerSuite) TestInputPerGoroutineRelabelerFromCacheFalse() { s.Require().NoError(err) shardsInnerSeries := cppbridge.NewShardsInnerSeries(numberOfShards) - state := cppbridge.NewState(numberOfShards) + state := cppbridge.NewStateV2WithoutLock() state.SetRelabelerOptions(&s.options) + state.Reconfigure(0, numberOfShards) pgr := cppbridge.NewPerGoroutineRelabeler(numberOfShards, 0) stats, ok, err := pgr.InputRelabelingFromCache( @@ -953,16 +958,18 @@ func (s *RelabelerSuite) TestInputPerGoroutineRelabelerFromCachePartially() { inputLss := cppbridge.NewLssStorage() targetLss := cppbridge.NewQueryableLssStorage() - statelessRelabeler, err := cppbridge.NewStatelessRelabeler(rCfgs) - s.Require().NoError(err) - var numberOfShards uint16 = 1 shardsInnerSeries := cppbridge.NewShardsInnerSeries(numberOfShards) shardsRelabeledSeries := cppbridge.NewShardsRelabeledSeries(numberOfShards) - state := cppbridge.NewState(numberOfShards) + + statelessRelabeler, err := cppbridge.NewStatelessRelabeler(rCfgs) + s.Require().NoError(err) + + state := cppbridge.NewStateV2WithoutLock() state.SetRelabelerOptions(&s.options) state.SetStatelessRelabeler(statelessRelabeler) + state.Reconfigure(0, numberOfShards) pgr := cppbridge.NewPerGoroutineRelabeler(numberOfShards, 0) stats, hasReallocations, err := pgr.InputRelabeling( diff --git a/pp/go/storage/appender/appender.go b/pp/go/storage/appender/appender.go index e2c6ac7127..0e3ea6c69d 100644 --- a/pp/go/storage/appender/appender.go +++ b/pp/go/storage/appender/appender.go @@ -132,7 +132,7 @@ func New[ func (a Appender[TTask, TLSS, TShard, THead]) Append( ctx context.Context, incomingData *IncomingData, - state *cppbridge.State, + state *cppbridge.StateV2, commitToWal bool, ) ([][]*cppbridge.InnerSeries, cppbridge.RelabelerStats, error) { if err := a.resolveState(state); err != nil { @@ -192,7 +192,7 @@ func (a Appender[TTask, TLSS, TShard, THead]) Append( //revive:disable-next-line:function-length long but this is first stage. func (a Appender[TTask, TLSS, TShard, THead]) inputRelabelingStage( ctx context.Context, - state *cppbridge.State, + state *cppbridge.StateV2, incomingData *DestructibleIncomingData, shardedInnerSeries *ShardedInnerSeries, shardedRelabeledSeries *ShardedRelabeledSeries, @@ -320,7 +320,7 @@ func (a Appender[TTask, TLSS, TShard, THead]) appendRelabelerSeriesStage( // updateRelabelerStateStage third stage - update state cache. func (a Appender[TTask, TLSS, TShard, THead]) updateRelabelerStateStage( ctx context.Context, - state *cppbridge.State, + state *cppbridge.StateV2, shardedStateUpdates *ShardedStateUpdates, ) error { numberOfShards := a.head.NumberOfShards() @@ -378,13 +378,12 @@ func (a Appender[TTask, TLSS, TShard, THead]) appendInnerSeriesAndWriteToWal( return atomicLimitExhausted, tw.Wait() } -func (a Appender[TTask, TLSS, TShard, THead]) resolveState(state *cppbridge.State) error { +func (a Appender[TTask, TLSS, TShard, THead]) resolveState(state *cppbridge.StateV2) error { if state == nil { return errNilState } - // TODO delete generationRelabeler 0, state.Reconfigure on lock - state.Reconfigure(0, a.head.Generation(), a.head.NumberOfShards()) + state.Reconfigure(a.head.Generation(), a.head.NumberOfShards()) return nil } From 6f48484f58476f4a157169d4546c732b96c7df08 Mon Sep 17 00:00:00 2001 From: Alexandr Yudin Date: Thu, 18 Sep 2025 15:40:17 +0000 Subject: [PATCH 38/96] rebuild state --- cmd/prometheus/main.go | 5 +- config/pp_remote_write_config.go | 5 + pp-pkg/handler/interface.go | 44 +- pp-pkg/handler/otlp_handler.go | 30 +- pp-pkg/handler/pp_handler.go | 28 +- pp-pkg/handler/processor/interface.go | 46 +- .../handler/processor/processor_moq_test.go | 852 ++++++++++++++++++ pp-pkg/handler/processor/refill_processor.go | 27 +- .../processor/refill_processor_test.go | 116 +-- .../processor/remote_write_processor.go | 30 +- .../processor/remote_write_processor_test.go | 181 ++-- pp-pkg/handler/processor/stream_processor.go | 23 +- .../processor/stream_processor_test.go | 119 +-- pp-pkg/handler/states.go | 84 ++ pp-pkg/model/interface.go | 23 + pp-pkg/scrape/manager.go | 59 +- pp-pkg/scrape/scrape.go | 132 +-- pp-pkg/storage/adapter.go | 37 +- pp/entrypoint/prometheus_relabeler.cpp | 75 +- pp/entrypoint/prometheus_relabeler.h | 49 +- pp/go/cppbridge/entrypoint.go | 129 ++- pp/go/cppbridge/entrypoint.h | 49 +- pp/go/cppbridge/prometheus_relabeler.go | 272 +++--- pp/go/cppbridge/prometheus_relabeler_test.go | 462 +++++++++- pp/go/storage/querier/querier.go | 2 +- pp/prometheus/relabeler.h | 122 ++- web/api/v1/api.go | 10 +- web/api/v1/pp_api.go | 17 +- web/web.go | 14 +- 29 files changed, 2504 insertions(+), 538 deletions(-) create mode 100644 pp-pkg/handler/processor/processor_moq_test.go create mode 100644 pp-pkg/handler/states.go create mode 100644 pp-pkg/model/interface.go diff --git a/cmd/prometheus/main.go b/cmd/prometheus/main.go index d3e8dc223b..fdec4f77a8 100644 --- a/cmd/prometheus/main.go +++ b/cmd/prometheus/main.go @@ -875,7 +875,7 @@ func main() { scrapeManager, err := scrape.NewManager( &cfg.scrape, log.With(logger, "component", "scrape manager"), - receiver, + nil, // TODO receiver adapter prometheus.DefaultRegisterer, ) if err != nil { @@ -992,7 +992,8 @@ func main() { } // Depends on cfg.web.ScrapeManager so needs to be after cfg.web.ScrapeManager = scrapeManager. - webHandler := web.New(log.With(logger, "component", "web"), &cfg.web, receiver) // PP_CHANGES.md: rebuild on cpp + // TODO receiver adapter + webHandler := web.New(log.With(logger, "component", "web"), &cfg.web, nil) // PP_CHANGES.md: rebuild on cpp // Monitor outgoing connections on default transport with conntrack. http.DefaultTransport.(*http.Transport).DialContext = conntrack.NewDialContextFunc( diff --git a/config/pp_remote_write_config.go b/config/pp_remote_write_config.go index 08ed9d597e..247d71cd04 100644 --- a/config/pp_remote_write_config.go +++ b/config/pp_remote_write_config.go @@ -61,6 +61,11 @@ func (c *Config) GetReceiverConfig() (*pp_pkg_config.RemoteWriteReceiverConfig, return rcCfg, nil } +// RemoteWriteReceiverConfig returns configs for RemoteWriteReceiver. +func (c *Config) RemoteWriteReceiverConfig() *pp_pkg_config.RemoteWriteReceiverConfig { + return c.ReceiverConfig.Copy() +} + func convertingRelabelConfigs(rCfgs []*relabel.Config) ([]*cppbridge.RelabelConfig, error) { var oprCfgs []*cppbridge.RelabelConfig raw, err := yaml.Marshal(rCfgs) diff --git a/pp-pkg/handler/interface.go b/pp-pkg/handler/interface.go index f2fd23edd2..e3a89b985e 100644 --- a/pp-pkg/handler/interface.go +++ b/pp-pkg/handler/interface.go @@ -4,26 +4,52 @@ import ( "context" "github.com/prometheus/prometheus/pp-pkg/handler/processor" + "github.com/prometheus/prometheus/pp-pkg/model" "github.com/prometheus/prometheus/pp/go/cppbridge" "github.com/prometheus/prometheus/pp/go/relabeler" "github.com/prometheus/prometheus/storage" ) -// Receiver interface. -type Receiver interface { - Appender(ctx context.Context) storage.Appender - AppendSnappyProtobuf(ctx context.Context, compressedData relabeler.ProtobufData, relabelerID string, commitToWal bool) error - AppendHashdex(ctx context.Context, hashdex cppbridge.ShardedData, relabelerID string, commitToWal bool) error +// Adapter for implementing the [Queryable] interface and append data. +type Adapter interface { + // AppendHashdex append incoming [cppbridge.HashdexContent] to [Head]. + AppendHashdex( + ctx context.Context, + hashdex cppbridge.ShardedData, + state *cppbridge.StateV2, + commitToWal bool, + ) error + + // AppendTimeSeries append TimeSeries data to [Head]. AppendTimeSeries( ctx context.Context, - data relabeler.TimeSeriesData, - state *cppbridge.State, - relabelerID string, + data model.TimeSeriesBatch, + state *cppbridge.StateV2, + commitToWal bool, + ) (cppbridge.RelabelerStats, error) + + // AppendScraperHashdex append ScraperHashdex data to [Head]. + AppendScraperHashdex( + ctx context.Context, + hashdex cppbridge.ShardedData, + state *cppbridge.StateV2, commitToWal bool, ) (cppbridge.RelabelerStats, error) - RelabelerIDIsExist(relabelerID string) bool + + // AppendSnappyProtobuf append compressed via snappy Protobuf data to [Head]. + AppendSnappyProtobuf( + ctx context.Context, + compressedData model.ProtobufData, + state *cppbridge.StateV2, + commitToWal bool, + ) error + + // HeadQuerier returns [storage.Querier] from active head. HeadQuerier(ctx context.Context, mint, maxt int64) (storage.Querier, error) + + // HeadStatus returns stats of Head. HeadStatus(ctx context.Context, limit int) relabeler.HeadStatus + // MergeOutOfOrderChunks merge chunks with out of order data chunks. MergeOutOfOrderChunks(ctx context.Context) } diff --git a/pp-pkg/handler/otlp_handler.go b/pp-pkg/handler/otlp_handler.go index e5b4d24711..80c146c1a3 100644 --- a/pp-pkg/handler/otlp_handler.go +++ b/pp-pkg/handler/otlp_handler.go @@ -22,6 +22,7 @@ import ( conventions "go.opentelemetry.io/collector/semconv/v1.6.1" "go.uber.org/multierr" + "github.com/prometheus/prometheus/config" prom_config "github.com/prometheus/prometheus/config" "github.com/prometheus/prometheus/model/value" ppmodel "github.com/prometheus/prometheus/pp/go/model" @@ -49,19 +50,33 @@ var OTLPAlwaysCommit = true // OTLPWriteHandler handler for otlp data via remote write. type OTLPWriteHandler struct { - logger log.Logger - receiver Receiver + logger log.Logger + adapter Adapter + states *StatesStorage } -func NewOTLPWriteHandler(logger log.Logger, receiver Receiver) *OTLPWriteHandler { +func NewOTLPWriteHandler(logger log.Logger, adapter Adapter) *OTLPWriteHandler { return &OTLPWriteHandler{ - logger: logger, - receiver: receiver, + logger: logger, + adapter: adapter, + states: NewStatesStorage(), } } +// ApplyConfig updates the configs for [StatesStorage]. +func (h *OTLPWriteHandler) ApplyConfig(conf *config.Config) error { + return h.states.ApplyConfig(conf) +} + // ServeHTTP implementation http.Handler. func (h *OTLPWriteHandler) ServeHTTP(w http.ResponseWriter, r *http.Request) { + state, ok := h.states.GetStateByID(prom_config.TransparentRelabeler) + if !ok { + level.Error(h.logger).Log("msg", "failed get state", "err", "unknown relabler id") + w.WriteHeader(http.StatusServiceUnavailable) + return + } + req, err := DecodeOTLPWriteRequest(r) if err != nil { level.Error(h.logger).Log("msg", "Error decoding remote write request", "err", err.Error()) @@ -74,11 +89,10 @@ func (h *OTLPWriteHandler) ServeHTTP(w http.ResponseWriter, r *http.Request) { level.Warn(h.logger).Log("msg", "Error translating OTLP metrics to Prometheus write request", "err", err) } - stats, err := h.receiver.AppendTimeSeries( + stats, err := h.adapter.AppendTimeSeries( r.Context(), converter.TimeSeries(), - nil, - prom_config.TransparentRelabeler, + state, OTLPAlwaysCommit, ) diff --git a/pp-pkg/handler/pp_handler.go b/pp-pkg/handler/pp_handler.go index 1641e2c7d9..25787c9c38 100644 --- a/pp-pkg/handler/pp_handler.go +++ b/pp-pkg/handler/pp_handler.go @@ -11,6 +11,7 @@ import ( "go.uber.org/atomic" "golang.org/x/net/websocket" + "github.com/prometheus/prometheus/config" "github.com/prometheus/prometheus/pp-pkg/handler/adapter" "github.com/prometheus/prometheus/pp-pkg/handler/decoder/ppcore" "github.com/prometheus/prometheus/pp-pkg/handler/middleware" @@ -25,7 +26,8 @@ const ppLocalStoragePath = "ppdata/" // PPHandler service for remote write via pp-protocol. type PPHandler struct { - receiver Receiver + adapter Adapter + states *StatesStorage logger log.Logger stream StreamProcessor refill RefillProcessor @@ -39,24 +41,27 @@ type PPHandler struct { // NewPPHandler init new PPHandler. func NewPPHandler( workDir string, - receiver Receiver, + ar Adapter, logger log.Logger, registerer prometheus.Registerer, ) *PPHandler { - buffers := pool.New(8, 1e6, 2, func(sz int) interface{} { return make([]byte, 0, sz) }) + buffers := pool.New(8, 1e6, 2, func(sz int) any { return make([]byte, 0, sz) }) ppBlockStorage := block.NewStorage(filepath.Join(workDir, ppLocalStoragePath), buffers) + states := NewStatesStorage() factory := util.NewUnconflictRegisterer(registerer) h := &PPHandler{ - receiver: receiver, - logger: log.With(logger, "component", "pp_handler"), - stream: processor.NewStreamProcessor(ppcore.NewBuilder(ppBlockStorage), receiver, registerer), + adapter: ar, + states: states, + logger: log.With(logger, "component", "pp_handler"), + stream: processor.NewStreamProcessor(ppcore.NewBuilder(ppBlockStorage), ar, states, registerer), refill: processor.NewRefillProcessor( ppcore.NewReplayDecoderBuilder(ppBlockStorage), - receiver, + ar, + states, logger, registerer, ), - remoteWrite: processor.NewRemoteWriteProcessor(receiver, registerer), + remoteWrite: processor.NewRemoteWriteProcessor(ar, states, registerer), buffers: buffers, stop: new(atomic.Bool), // stats @@ -74,6 +79,11 @@ func NewPPHandler( return h } +// ApplyConfig updates the configs for [StatesStorage]. +func (h *PPHandler) ApplyConfig(conf *config.Config) error { + return h.states.ApplyConfig(conf) +} + // Websocket handler for websocket stream. func (h *PPHandler) Websocket(middlewares ...middleware.Middleware) http.HandlerFunc { hf := h.metadataValidator(websocket.Handler(h.websocketHandler).ServeHTTP) @@ -120,7 +130,7 @@ func (h *PPHandler) measure(next http.Handler, typeHandler string) http.HandlerF func (h *PPHandler) metadataValidator(next http.HandlerFunc) http.HandlerFunc { return func(rw http.ResponseWriter, r *http.Request) { metadata := middleware.MetadataFromContext(r.Context()) - if ok := h.receiver.RelabelerIDIsExist(metadata.RelabelerID); !ok { + if _, ok := h.states.GetStateByID(metadata.RelabelerID); !ok { level.Error(h.logger).Log("msg", "relabeler id not found", "relabeler_id", metadata.RelabelerID) rw.WriteHeader(http.StatusPreconditionFailed) return diff --git a/pp-pkg/handler/processor/interface.go b/pp-pkg/handler/processor/interface.go index e15c944fe5..c719a93f22 100644 --- a/pp-pkg/handler/processor/interface.go +++ b/pp-pkg/handler/processor/interface.go @@ -5,8 +5,8 @@ import ( "github.com/prometheus/prometheus/pp-pkg/handler/decoder" "github.com/prometheus/prometheus/pp-pkg/handler/model" + pp_pkg_model "github.com/prometheus/prometheus/pp-pkg/model" "github.com/prometheus/prometheus/pp/go/cppbridge" - "github.com/prometheus/prometheus/pp/go/relabeler" ) type MetricStream interface { @@ -31,10 +31,46 @@ type DecoderBuilder interface { Build(metadata model.Metadata) decoder.Decoder } -// Receiver interface. -type Receiver interface { - AppendSnappyProtobuf(ctx context.Context, compressedData relabeler.ProtobufData, relabelerID string, commitToWal bool) error - AppendHashdex(ctx context.Context, hashdex cppbridge.ShardedData, relabelerID string, commitToWal bool) error +// Adapter for implementing the [Queryable] interface and append data. +type Adapter interface { + // AppendHashdex append incoming [cppbridge.HashdexContent] to [Head]. + AppendHashdex( + ctx context.Context, + hashdex cppbridge.ShardedData, + state *cppbridge.StateV2, + commitToWal bool, + ) error + + // AppendTimeSeries append TimeSeries data to [Head]. + AppendTimeSeries( + ctx context.Context, + data pp_pkg_model.TimeSeriesBatch, + state *cppbridge.StateV2, + commitToWal bool, + ) (cppbridge.RelabelerStats, error) + + // AppendScraperHashdex append ScraperHashdex data to [Head]. + AppendScraperHashdex( + ctx context.Context, + hashdex cppbridge.ShardedData, + state *cppbridge.StateV2, + commitToWal bool, + ) (cppbridge.RelabelerStats, error) + + // AppendSnappyProtobuf append compressed via snappy Protobuf data to [Head]. + AppendSnappyProtobuf( + ctx context.Context, + compressedData pp_pkg_model.ProtobufData, + state *cppbridge.StateV2, + commitToWal bool, + ) error + // MergeOutOfOrderChunks merge chunks with out of order data chunks. MergeOutOfOrderChunks(ctx context.Context) } + +// StatesStorage stores the [cppbridge.State]'s. +type StatesStorage interface { + // GetStateByID returns [cppbridge.State] by state ID if exist. + GetStateByID(stateID string) (*cppbridge.StateV2, bool) +} diff --git a/pp-pkg/handler/processor/processor_moq_test.go b/pp-pkg/handler/processor/processor_moq_test.go new file mode 100644 index 0000000000..d538fa9148 --- /dev/null +++ b/pp-pkg/handler/processor/processor_moq_test.go @@ -0,0 +1,852 @@ +// Code generated by moq; DO NOT EDIT. +// github.com/matryer/moq + +package processor_test + +import ( + "context" + "github.com/prometheus/prometheus/pp-pkg/handler/model" + "github.com/prometheus/prometheus/pp-pkg/handler/processor" + pp_pkg_model "github.com/prometheus/prometheus/pp-pkg/model" + "github.com/prometheus/prometheus/pp/go/cppbridge" + "sync" +) + +// Ensure, that AdapterMock does implement processor.Adapter. +// If this is not the case, regenerate this file with moq. +var _ processor.Adapter = &AdapterMock{} + +// AdapterMock is a mock implementation of processor.Adapter. +// +// func TestSomethingThatUsesAdapter(t *testing.T) { +// +// // make and configure a mocked processor.Adapter +// mockedAdapter := &AdapterMock{ +// AppendHashdexFunc: func(ctx context.Context, hashdex cppbridge.ShardedData, state *cppbridge.StateV2, commitToWal bool) error { +// panic("mock out the AppendHashdex method") +// }, +// AppendScraperHashdexFunc: func(ctx context.Context, hashdex cppbridge.ShardedData, state *cppbridge.StateV2, commitToWal bool) (cppbridge.RelabelerStats, error) { +// panic("mock out the AppendScraperHashdex method") +// }, +// AppendSnappyProtobufFunc: func(ctx context.Context, compressedData pp_pkg_model.ProtobufData, state *cppbridge.StateV2, commitToWal bool) error { +// panic("mock out the AppendSnappyProtobuf method") +// }, +// AppendTimeSeriesFunc: func(ctx context.Context, data pp_pkg_model.TimeSeriesBatch, state *cppbridge.StateV2, commitToWal bool) (cppbridge.RelabelerStats, error) { +// panic("mock out the AppendTimeSeries method") +// }, +// MergeOutOfOrderChunksFunc: func(ctx context.Context) { +// panic("mock out the MergeOutOfOrderChunks method") +// }, +// } +// +// // use mockedAdapter in code that requires processor.Adapter +// // and then make assertions. +// +// } +type AdapterMock struct { + // AppendHashdexFunc mocks the AppendHashdex method. + AppendHashdexFunc func(ctx context.Context, hashdex cppbridge.ShardedData, state *cppbridge.StateV2, commitToWal bool) error + + // AppendScraperHashdexFunc mocks the AppendScraperHashdex method. + AppendScraperHashdexFunc func(ctx context.Context, hashdex cppbridge.ShardedData, state *cppbridge.StateV2, commitToWal bool) (cppbridge.RelabelerStats, error) + + // AppendSnappyProtobufFunc mocks the AppendSnappyProtobuf method. + AppendSnappyProtobufFunc func(ctx context.Context, compressedData pp_pkg_model.ProtobufData, state *cppbridge.StateV2, commitToWal bool) error + + // AppendTimeSeriesFunc mocks the AppendTimeSeries method. + AppendTimeSeriesFunc func(ctx context.Context, data pp_pkg_model.TimeSeriesBatch, state *cppbridge.StateV2, commitToWal bool) (cppbridge.RelabelerStats, error) + + // MergeOutOfOrderChunksFunc mocks the MergeOutOfOrderChunks method. + MergeOutOfOrderChunksFunc func(ctx context.Context) + + // calls tracks calls to the methods. + calls struct { + // AppendHashdex holds details about calls to the AppendHashdex method. + AppendHashdex []struct { + // Ctx is the ctx argument value. + Ctx context.Context + // Hashdex is the hashdex argument value. + Hashdex cppbridge.ShardedData + // State is the state argument value. + State *cppbridge.StateV2 + // CommitToWal is the commitToWal argument value. + CommitToWal bool + } + // AppendScraperHashdex holds details about calls to the AppendScraperHashdex method. + AppendScraperHashdex []struct { + // Ctx is the ctx argument value. + Ctx context.Context + // Hashdex is the hashdex argument value. + Hashdex cppbridge.ShardedData + // State is the state argument value. + State *cppbridge.StateV2 + // CommitToWal is the commitToWal argument value. + CommitToWal bool + } + // AppendSnappyProtobuf holds details about calls to the AppendSnappyProtobuf method. + AppendSnappyProtobuf []struct { + // Ctx is the ctx argument value. + Ctx context.Context + // CompressedData is the compressedData argument value. + CompressedData pp_pkg_model.ProtobufData + // State is the state argument value. + State *cppbridge.StateV2 + // CommitToWal is the commitToWal argument value. + CommitToWal bool + } + // AppendTimeSeries holds details about calls to the AppendTimeSeries method. + AppendTimeSeries []struct { + // Ctx is the ctx argument value. + Ctx context.Context + // Data is the data argument value. + Data pp_pkg_model.TimeSeriesBatch + // State is the state argument value. + State *cppbridge.StateV2 + // CommitToWal is the commitToWal argument value. + CommitToWal bool + } + // MergeOutOfOrderChunks holds details about calls to the MergeOutOfOrderChunks method. + MergeOutOfOrderChunks []struct { + // Ctx is the ctx argument value. + Ctx context.Context + } + } + lockAppendHashdex sync.RWMutex + lockAppendScraperHashdex sync.RWMutex + lockAppendSnappyProtobuf sync.RWMutex + lockAppendTimeSeries sync.RWMutex + lockMergeOutOfOrderChunks sync.RWMutex +} + +// AppendHashdex calls AppendHashdexFunc. +func (mock *AdapterMock) AppendHashdex(ctx context.Context, hashdex cppbridge.ShardedData, state *cppbridge.StateV2, commitToWal bool) error { + if mock.AppendHashdexFunc == nil { + panic("AdapterMock.AppendHashdexFunc: method is nil but Adapter.AppendHashdex was just called") + } + callInfo := struct { + Ctx context.Context + Hashdex cppbridge.ShardedData + State *cppbridge.StateV2 + CommitToWal bool + }{ + Ctx: ctx, + Hashdex: hashdex, + State: state, + CommitToWal: commitToWal, + } + mock.lockAppendHashdex.Lock() + mock.calls.AppendHashdex = append(mock.calls.AppendHashdex, callInfo) + mock.lockAppendHashdex.Unlock() + return mock.AppendHashdexFunc(ctx, hashdex, state, commitToWal) +} + +// AppendHashdexCalls gets all the calls that were made to AppendHashdex. +// Check the length with: +// +// len(mockedAdapter.AppendHashdexCalls()) +func (mock *AdapterMock) AppendHashdexCalls() []struct { + Ctx context.Context + Hashdex cppbridge.ShardedData + State *cppbridge.StateV2 + CommitToWal bool +} { + var calls []struct { + Ctx context.Context + Hashdex cppbridge.ShardedData + State *cppbridge.StateV2 + CommitToWal bool + } + mock.lockAppendHashdex.RLock() + calls = mock.calls.AppendHashdex + mock.lockAppendHashdex.RUnlock() + return calls +} + +// AppendScraperHashdex calls AppendScraperHashdexFunc. +func (mock *AdapterMock) AppendScraperHashdex(ctx context.Context, hashdex cppbridge.ShardedData, state *cppbridge.StateV2, commitToWal bool) (cppbridge.RelabelerStats, error) { + if mock.AppendScraperHashdexFunc == nil { + panic("AdapterMock.AppendScraperHashdexFunc: method is nil but Adapter.AppendScraperHashdex was just called") + } + callInfo := struct { + Ctx context.Context + Hashdex cppbridge.ShardedData + State *cppbridge.StateV2 + CommitToWal bool + }{ + Ctx: ctx, + Hashdex: hashdex, + State: state, + CommitToWal: commitToWal, + } + mock.lockAppendScraperHashdex.Lock() + mock.calls.AppendScraperHashdex = append(mock.calls.AppendScraperHashdex, callInfo) + mock.lockAppendScraperHashdex.Unlock() + return mock.AppendScraperHashdexFunc(ctx, hashdex, state, commitToWal) +} + +// AppendScraperHashdexCalls gets all the calls that were made to AppendScraperHashdex. +// Check the length with: +// +// len(mockedAdapter.AppendScraperHashdexCalls()) +func (mock *AdapterMock) AppendScraperHashdexCalls() []struct { + Ctx context.Context + Hashdex cppbridge.ShardedData + State *cppbridge.StateV2 + CommitToWal bool +} { + var calls []struct { + Ctx context.Context + Hashdex cppbridge.ShardedData + State *cppbridge.StateV2 + CommitToWal bool + } + mock.lockAppendScraperHashdex.RLock() + calls = mock.calls.AppendScraperHashdex + mock.lockAppendScraperHashdex.RUnlock() + return calls +} + +// AppendSnappyProtobuf calls AppendSnappyProtobufFunc. +func (mock *AdapterMock) AppendSnappyProtobuf(ctx context.Context, compressedData pp_pkg_model.ProtobufData, state *cppbridge.StateV2, commitToWal bool) error { + if mock.AppendSnappyProtobufFunc == nil { + panic("AdapterMock.AppendSnappyProtobufFunc: method is nil but Adapter.AppendSnappyProtobuf was just called") + } + callInfo := struct { + Ctx context.Context + CompressedData pp_pkg_model.ProtobufData + State *cppbridge.StateV2 + CommitToWal bool + }{ + Ctx: ctx, + CompressedData: compressedData, + State: state, + CommitToWal: commitToWal, + } + mock.lockAppendSnappyProtobuf.Lock() + mock.calls.AppendSnappyProtobuf = append(mock.calls.AppendSnappyProtobuf, callInfo) + mock.lockAppendSnappyProtobuf.Unlock() + return mock.AppendSnappyProtobufFunc(ctx, compressedData, state, commitToWal) +} + +// AppendSnappyProtobufCalls gets all the calls that were made to AppendSnappyProtobuf. +// Check the length with: +// +// len(mockedAdapter.AppendSnappyProtobufCalls()) +func (mock *AdapterMock) AppendSnappyProtobufCalls() []struct { + Ctx context.Context + CompressedData pp_pkg_model.ProtobufData + State *cppbridge.StateV2 + CommitToWal bool +} { + var calls []struct { + Ctx context.Context + CompressedData pp_pkg_model.ProtobufData + State *cppbridge.StateV2 + CommitToWal bool + } + mock.lockAppendSnappyProtobuf.RLock() + calls = mock.calls.AppendSnappyProtobuf + mock.lockAppendSnappyProtobuf.RUnlock() + return calls +} + +// AppendTimeSeries calls AppendTimeSeriesFunc. +func (mock *AdapterMock) AppendTimeSeries(ctx context.Context, data pp_pkg_model.TimeSeriesBatch, state *cppbridge.StateV2, commitToWal bool) (cppbridge.RelabelerStats, error) { + if mock.AppendTimeSeriesFunc == nil { + panic("AdapterMock.AppendTimeSeriesFunc: method is nil but Adapter.AppendTimeSeries was just called") + } + callInfo := struct { + Ctx context.Context + Data pp_pkg_model.TimeSeriesBatch + State *cppbridge.StateV2 + CommitToWal bool + }{ + Ctx: ctx, + Data: data, + State: state, + CommitToWal: commitToWal, + } + mock.lockAppendTimeSeries.Lock() + mock.calls.AppendTimeSeries = append(mock.calls.AppendTimeSeries, callInfo) + mock.lockAppendTimeSeries.Unlock() + return mock.AppendTimeSeriesFunc(ctx, data, state, commitToWal) +} + +// AppendTimeSeriesCalls gets all the calls that were made to AppendTimeSeries. +// Check the length with: +// +// len(mockedAdapter.AppendTimeSeriesCalls()) +func (mock *AdapterMock) AppendTimeSeriesCalls() []struct { + Ctx context.Context + Data pp_pkg_model.TimeSeriesBatch + State *cppbridge.StateV2 + CommitToWal bool +} { + var calls []struct { + Ctx context.Context + Data pp_pkg_model.TimeSeriesBatch + State *cppbridge.StateV2 + CommitToWal bool + } + mock.lockAppendTimeSeries.RLock() + calls = mock.calls.AppendTimeSeries + mock.lockAppendTimeSeries.RUnlock() + return calls +} + +// MergeOutOfOrderChunks calls MergeOutOfOrderChunksFunc. +func (mock *AdapterMock) MergeOutOfOrderChunks(ctx context.Context) { + if mock.MergeOutOfOrderChunksFunc == nil { + panic("AdapterMock.MergeOutOfOrderChunksFunc: method is nil but Adapter.MergeOutOfOrderChunks was just called") + } + callInfo := struct { + Ctx context.Context + }{ + Ctx: ctx, + } + mock.lockMergeOutOfOrderChunks.Lock() + mock.calls.MergeOutOfOrderChunks = append(mock.calls.MergeOutOfOrderChunks, callInfo) + mock.lockMergeOutOfOrderChunks.Unlock() + mock.MergeOutOfOrderChunksFunc(ctx) +} + +// MergeOutOfOrderChunksCalls gets all the calls that were made to MergeOutOfOrderChunks. +// Check the length with: +// +// len(mockedAdapter.MergeOutOfOrderChunksCalls()) +func (mock *AdapterMock) MergeOutOfOrderChunksCalls() []struct { + Ctx context.Context +} { + var calls []struct { + Ctx context.Context + } + mock.lockMergeOutOfOrderChunks.RLock() + calls = mock.calls.MergeOutOfOrderChunks + mock.lockMergeOutOfOrderChunks.RUnlock() + return calls +} + +// Ensure, that StatesStorageMock does implement processor.StatesStorage. +// If this is not the case, regenerate this file with moq. +var _ processor.StatesStorage = &StatesStorageMock{} + +// StatesStorageMock is a mock implementation of processor.StatesStorage. +// +// func TestSomethingThatUsesStatesStorage(t *testing.T) { +// +// // make and configure a mocked processor.StatesStorage +// mockedStatesStorage := &StatesStorageMock{ +// GetStateByIDFunc: func(stateID string) (*cppbridge.StateV2, bool) { +// panic("mock out the GetStateByID method") +// }, +// } +// +// // use mockedStatesStorage in code that requires processor.StatesStorage +// // and then make assertions. +// +// } +type StatesStorageMock struct { + // GetStateByIDFunc mocks the GetStateByID method. + GetStateByIDFunc func(stateID string) (*cppbridge.StateV2, bool) + + // calls tracks calls to the methods. + calls struct { + // GetStateByID holds details about calls to the GetStateByID method. + GetStateByID []struct { + // StateID is the stateID argument value. + StateID string + } + } + lockGetStateByID sync.RWMutex +} + +// GetStateByID calls GetStateByIDFunc. +func (mock *StatesStorageMock) GetStateByID(stateID string) (*cppbridge.StateV2, bool) { + if mock.GetStateByIDFunc == nil { + panic("StatesStorageMock.GetStateByIDFunc: method is nil but StatesStorage.GetStateByID was just called") + } + callInfo := struct { + StateID string + }{ + StateID: stateID, + } + mock.lockGetStateByID.Lock() + mock.calls.GetStateByID = append(mock.calls.GetStateByID, callInfo) + mock.lockGetStateByID.Unlock() + return mock.GetStateByIDFunc(stateID) +} + +// GetStateByIDCalls gets all the calls that were made to GetStateByID. +// Check the length with: +// +// len(mockedStatesStorage.GetStateByIDCalls()) +func (mock *StatesStorageMock) GetStateByIDCalls() []struct { + StateID string +} { + var calls []struct { + StateID string + } + mock.lockGetStateByID.RLock() + calls = mock.calls.GetStateByID + mock.lockGetStateByID.RUnlock() + return calls +} + +// Ensure, that RemoteWriteMock does implement processor.RemoteWrite. +// If this is not the case, regenerate this file with moq. +var _ processor.RemoteWrite = &RemoteWriteMock{} + +// RemoteWriteMock is a mock implementation of processor.RemoteWrite. +// +// func TestSomethingThatUsesRemoteWrite(t *testing.T) { +// +// // make and configure a mocked processor.RemoteWrite +// mockedRemoteWrite := &RemoteWriteMock{ +// MetadataFunc: func() model.Metadata { +// panic("mock out the Metadata method") +// }, +// ReadFunc: func(ctx context.Context) (*model.RemoteWriteBuffer, error) { +// panic("mock out the Read method") +// }, +// WriteFunc: func(ctx context.Context, status model.RemoteWriteProcessingStatus) error { +// panic("mock out the Write method") +// }, +// } +// +// // use mockedRemoteWrite in code that requires processor.RemoteWrite +// // and then make assertions. +// +// } +type RemoteWriteMock struct { + // MetadataFunc mocks the Metadata method. + MetadataFunc func() model.Metadata + + // ReadFunc mocks the Read method. + ReadFunc func(ctx context.Context) (*model.RemoteWriteBuffer, error) + + // WriteFunc mocks the Write method. + WriteFunc func(ctx context.Context, status model.RemoteWriteProcessingStatus) error + + // calls tracks calls to the methods. + calls struct { + // Metadata holds details about calls to the Metadata method. + Metadata []struct { + } + // Read holds details about calls to the Read method. + Read []struct { + // Ctx is the ctx argument value. + Ctx context.Context + } + // Write holds details about calls to the Write method. + Write []struct { + // Ctx is the ctx argument value. + Ctx context.Context + // Status is the status argument value. + Status model.RemoteWriteProcessingStatus + } + } + lockMetadata sync.RWMutex + lockRead sync.RWMutex + lockWrite sync.RWMutex +} + +// Metadata calls MetadataFunc. +func (mock *RemoteWriteMock) Metadata() model.Metadata { + if mock.MetadataFunc == nil { + panic("RemoteWriteMock.MetadataFunc: method is nil but RemoteWrite.Metadata was just called") + } + callInfo := struct { + }{} + mock.lockMetadata.Lock() + mock.calls.Metadata = append(mock.calls.Metadata, callInfo) + mock.lockMetadata.Unlock() + return mock.MetadataFunc() +} + +// MetadataCalls gets all the calls that were made to Metadata. +// Check the length with: +// +// len(mockedRemoteWrite.MetadataCalls()) +func (mock *RemoteWriteMock) MetadataCalls() []struct { +} { + var calls []struct { + } + mock.lockMetadata.RLock() + calls = mock.calls.Metadata + mock.lockMetadata.RUnlock() + return calls +} + +// Read calls ReadFunc. +func (mock *RemoteWriteMock) Read(ctx context.Context) (*model.RemoteWriteBuffer, error) { + if mock.ReadFunc == nil { + panic("RemoteWriteMock.ReadFunc: method is nil but RemoteWrite.Read was just called") + } + callInfo := struct { + Ctx context.Context + }{ + Ctx: ctx, + } + mock.lockRead.Lock() + mock.calls.Read = append(mock.calls.Read, callInfo) + mock.lockRead.Unlock() + return mock.ReadFunc(ctx) +} + +// ReadCalls gets all the calls that were made to Read. +// Check the length with: +// +// len(mockedRemoteWrite.ReadCalls()) +func (mock *RemoteWriteMock) ReadCalls() []struct { + Ctx context.Context +} { + var calls []struct { + Ctx context.Context + } + mock.lockRead.RLock() + calls = mock.calls.Read + mock.lockRead.RUnlock() + return calls +} + +// Write calls WriteFunc. +func (mock *RemoteWriteMock) Write(ctx context.Context, status model.RemoteWriteProcessingStatus) error { + if mock.WriteFunc == nil { + panic("RemoteWriteMock.WriteFunc: method is nil but RemoteWrite.Write was just called") + } + callInfo := struct { + Ctx context.Context + Status model.RemoteWriteProcessingStatus + }{ + Ctx: ctx, + Status: status, + } + mock.lockWrite.Lock() + mock.calls.Write = append(mock.calls.Write, callInfo) + mock.lockWrite.Unlock() + return mock.WriteFunc(ctx, status) +} + +// WriteCalls gets all the calls that were made to Write. +// Check the length with: +// +// len(mockedRemoteWrite.WriteCalls()) +func (mock *RemoteWriteMock) WriteCalls() []struct { + Ctx context.Context + Status model.RemoteWriteProcessingStatus +} { + var calls []struct { + Ctx context.Context + Status model.RemoteWriteProcessingStatus + } + mock.lockWrite.RLock() + calls = mock.calls.Write + mock.lockWrite.RUnlock() + return calls +} + +// Ensure, that MetricStreamMock does implement processor.MetricStream. +// If this is not the case, regenerate this file with moq. +var _ processor.MetricStream = &MetricStreamMock{} + +// MetricStreamMock is a mock implementation of processor.MetricStream. +// +// func TestSomethingThatUsesMetricStream(t *testing.T) { +// +// // make and configure a mocked processor.MetricStream +// mockedMetricStream := &MetricStreamMock{ +// MetadataFunc: func() model.Metadata { +// panic("mock out the Metadata method") +// }, +// ReadFunc: func(ctx context.Context) (*model.Segment, error) { +// panic("mock out the Read method") +// }, +// WriteFunc: func(ctx context.Context, status model.StreamSegmentProcessingStatus) error { +// panic("mock out the Write method") +// }, +// } +// +// // use mockedMetricStream in code that requires processor.MetricStream +// // and then make assertions. +// +// } +type MetricStreamMock struct { + // MetadataFunc mocks the Metadata method. + MetadataFunc func() model.Metadata + + // ReadFunc mocks the Read method. + ReadFunc func(ctx context.Context) (*model.Segment, error) + + // WriteFunc mocks the Write method. + WriteFunc func(ctx context.Context, status model.StreamSegmentProcessingStatus) error + + // calls tracks calls to the methods. + calls struct { + // Metadata holds details about calls to the Metadata method. + Metadata []struct { + } + // Read holds details about calls to the Read method. + Read []struct { + // Ctx is the ctx argument value. + Ctx context.Context + } + // Write holds details about calls to the Write method. + Write []struct { + // Ctx is the ctx argument value. + Ctx context.Context + // Status is the status argument value. + Status model.StreamSegmentProcessingStatus + } + } + lockMetadata sync.RWMutex + lockRead sync.RWMutex + lockWrite sync.RWMutex +} + +// Metadata calls MetadataFunc. +func (mock *MetricStreamMock) Metadata() model.Metadata { + if mock.MetadataFunc == nil { + panic("MetricStreamMock.MetadataFunc: method is nil but MetricStream.Metadata was just called") + } + callInfo := struct { + }{} + mock.lockMetadata.Lock() + mock.calls.Metadata = append(mock.calls.Metadata, callInfo) + mock.lockMetadata.Unlock() + return mock.MetadataFunc() +} + +// MetadataCalls gets all the calls that were made to Metadata. +// Check the length with: +// +// len(mockedMetricStream.MetadataCalls()) +func (mock *MetricStreamMock) MetadataCalls() []struct { +} { + var calls []struct { + } + mock.lockMetadata.RLock() + calls = mock.calls.Metadata + mock.lockMetadata.RUnlock() + return calls +} + +// Read calls ReadFunc. +func (mock *MetricStreamMock) Read(ctx context.Context) (*model.Segment, error) { + if mock.ReadFunc == nil { + panic("MetricStreamMock.ReadFunc: method is nil but MetricStream.Read was just called") + } + callInfo := struct { + Ctx context.Context + }{ + Ctx: ctx, + } + mock.lockRead.Lock() + mock.calls.Read = append(mock.calls.Read, callInfo) + mock.lockRead.Unlock() + return mock.ReadFunc(ctx) +} + +// ReadCalls gets all the calls that were made to Read. +// Check the length with: +// +// len(mockedMetricStream.ReadCalls()) +func (mock *MetricStreamMock) ReadCalls() []struct { + Ctx context.Context +} { + var calls []struct { + Ctx context.Context + } + mock.lockRead.RLock() + calls = mock.calls.Read + mock.lockRead.RUnlock() + return calls +} + +// Write calls WriteFunc. +func (mock *MetricStreamMock) Write(ctx context.Context, status model.StreamSegmentProcessingStatus) error { + if mock.WriteFunc == nil { + panic("MetricStreamMock.WriteFunc: method is nil but MetricStream.Write was just called") + } + callInfo := struct { + Ctx context.Context + Status model.StreamSegmentProcessingStatus + }{ + Ctx: ctx, + Status: status, + } + mock.lockWrite.Lock() + mock.calls.Write = append(mock.calls.Write, callInfo) + mock.lockWrite.Unlock() + return mock.WriteFunc(ctx, status) +} + +// WriteCalls gets all the calls that were made to Write. +// Check the length with: +// +// len(mockedMetricStream.WriteCalls()) +func (mock *MetricStreamMock) WriteCalls() []struct { + Ctx context.Context + Status model.StreamSegmentProcessingStatus +} { + var calls []struct { + Ctx context.Context + Status model.StreamSegmentProcessingStatus + } + mock.lockWrite.RLock() + calls = mock.calls.Write + mock.lockWrite.RUnlock() + return calls +} + +// Ensure, that RefillMock does implement processor.Refill. +// If this is not the case, regenerate this file with moq. +var _ processor.Refill = &RefillMock{} + +// RefillMock is a mock implementation of processor.Refill. +// +// func TestSomethingThatUsesRefill(t *testing.T) { +// +// // make and configure a mocked processor.Refill +// mockedRefill := &RefillMock{ +// MetadataFunc: func() model.Metadata { +// panic("mock out the Metadata method") +// }, +// ReadFunc: func(ctx context.Context) (*model.Segment, error) { +// panic("mock out the Read method") +// }, +// WriteFunc: func(ctx context.Context, status model.RefillProcessingStatus) error { +// panic("mock out the Write method") +// }, +// } +// +// // use mockedRefill in code that requires processor.Refill +// // and then make assertions. +// +// } +type RefillMock struct { + // MetadataFunc mocks the Metadata method. + MetadataFunc func() model.Metadata + + // ReadFunc mocks the Read method. + ReadFunc func(ctx context.Context) (*model.Segment, error) + + // WriteFunc mocks the Write method. + WriteFunc func(ctx context.Context, status model.RefillProcessingStatus) error + + // calls tracks calls to the methods. + calls struct { + // Metadata holds details about calls to the Metadata method. + Metadata []struct { + } + // Read holds details about calls to the Read method. + Read []struct { + // Ctx is the ctx argument value. + Ctx context.Context + } + // Write holds details about calls to the Write method. + Write []struct { + // Ctx is the ctx argument value. + Ctx context.Context + // Status is the status argument value. + Status model.RefillProcessingStatus + } + } + lockMetadata sync.RWMutex + lockRead sync.RWMutex + lockWrite sync.RWMutex +} + +// Metadata calls MetadataFunc. +func (mock *RefillMock) Metadata() model.Metadata { + if mock.MetadataFunc == nil { + panic("RefillMock.MetadataFunc: method is nil but Refill.Metadata was just called") + } + callInfo := struct { + }{} + mock.lockMetadata.Lock() + mock.calls.Metadata = append(mock.calls.Metadata, callInfo) + mock.lockMetadata.Unlock() + return mock.MetadataFunc() +} + +// MetadataCalls gets all the calls that were made to Metadata. +// Check the length with: +// +// len(mockedRefill.MetadataCalls()) +func (mock *RefillMock) MetadataCalls() []struct { +} { + var calls []struct { + } + mock.lockMetadata.RLock() + calls = mock.calls.Metadata + mock.lockMetadata.RUnlock() + return calls +} + +// Read calls ReadFunc. +func (mock *RefillMock) Read(ctx context.Context) (*model.Segment, error) { + if mock.ReadFunc == nil { + panic("RefillMock.ReadFunc: method is nil but Refill.Read was just called") + } + callInfo := struct { + Ctx context.Context + }{ + Ctx: ctx, + } + mock.lockRead.Lock() + mock.calls.Read = append(mock.calls.Read, callInfo) + mock.lockRead.Unlock() + return mock.ReadFunc(ctx) +} + +// ReadCalls gets all the calls that were made to Read. +// Check the length with: +// +// len(mockedRefill.ReadCalls()) +func (mock *RefillMock) ReadCalls() []struct { + Ctx context.Context +} { + var calls []struct { + Ctx context.Context + } + mock.lockRead.RLock() + calls = mock.calls.Read + mock.lockRead.RUnlock() + return calls +} + +// Write calls WriteFunc. +func (mock *RefillMock) Write(ctx context.Context, status model.RefillProcessingStatus) error { + if mock.WriteFunc == nil { + panic("RefillMock.WriteFunc: method is nil but Refill.Write was just called") + } + callInfo := struct { + Ctx context.Context + Status model.RefillProcessingStatus + }{ + Ctx: ctx, + Status: status, + } + mock.lockWrite.Lock() + mock.calls.Write = append(mock.calls.Write, callInfo) + mock.lockWrite.Unlock() + return mock.WriteFunc(ctx, status) +} + +// WriteCalls gets all the calls that were made to Write. +// Check the length with: +// +// len(mockedRefill.WriteCalls()) +func (mock *RefillMock) WriteCalls() []struct { + Ctx context.Context + Status model.RefillProcessingStatus +} { + var calls []struct { + Ctx context.Context + Status model.RefillProcessingStatus + } + mock.lockWrite.RLock() + calls = mock.calls.Write + mock.lockWrite.RUnlock() + return calls +} diff --git a/pp-pkg/handler/processor/refill_processor.go b/pp-pkg/handler/processor/refill_processor.go index c9991622e3..556c6b4d1f 100644 --- a/pp-pkg/handler/processor/refill_processor.go +++ b/pp-pkg/handler/processor/refill_processor.go @@ -16,7 +16,8 @@ import ( type RefillProcessor struct { decoderBuilder DecoderBuilder - receiver Receiver + adapter Adapter + states StatesStorage logger log.Logger criticalErrorCount *prometheus.CounterVec @@ -29,14 +30,16 @@ type RefillProcessor struct { func NewRefillProcessor( decoderBuilder DecoderBuilder, - receiver Receiver, + adapter Adapter, + states StatesStorage, logger log.Logger, registerer prometheus.Registerer, ) *RefillProcessor { factory := util.NewUnconflictRegisterer(registerer) return &RefillProcessor{ decoderBuilder: decoderBuilder, - receiver: receiver, + adapter: adapter, + states: states, logger: log.With(logger, "component", "refill_processor"), criticalErrorCount: factory.NewCounterVec(prometheus.CounterOpts{ Name: "remote_write_opprotocol_processor_critical_error_count", @@ -67,6 +70,16 @@ func NewRefillProcessor( func (p *RefillProcessor) Process(ctx context.Context, refill Refill) error { meta := refill.Metadata() + + state, ok := p.states.GetStateByID(meta.RelabelerID) + if !ok { + p.criticalErrorCount.With(prometheus.Labels{ + "error": ErrUnknownRelablerID.Error(), + "processor_type": "stream", + }).Inc() + return ErrUnknownRelablerID + } + decoder := p.decoderBuilder.Build(meta) defer func() { _ = decoder.Close() }() @@ -80,7 +93,9 @@ func (p *RefillProcessor) Process(ctx context.Context, refill Refill) error { if err != nil { if errors.Is(err, io.EOF) { if disErr := decoder.Discard(); disErr != nil { - p.criticalErrorCount.With(prometheus.Labels{"error": disErr.Error(), "processor_type": "refill"}).Inc() + p.criticalErrorCount.With( + prometheus.Labels{"error": disErr.Error(), "processor_type": "refill"}, + ).Inc() } p.writtenSeriesCount.With(prometheus.Labels{"processor_type": "refill"}).Add(float64(decodedSeries)) @@ -90,7 +105,7 @@ func (p *RefillProcessor) Process(ctx context.Context, refill Refill) error { prometheus.Labels{"processor_type": "refill", "status_code": "200"}, ).Inc() - p.receiver.MergeOutOfOrderChunks(ctx) + p.adapter.MergeOutOfOrderChunks(ctx) return refill.Write(ctx, model.RefillProcessingStatus{Code: http.StatusOK}) } @@ -111,7 +126,7 @@ func (p *RefillProcessor) Process(ctx context.Context, refill Refill) error { decodedSamples += hashdexContent.Samples() p.decodedSampleCount.With(prometheus.Labels{"processor_type": "refill"}).Add(float64(hashdexContent.Samples())) - if err = p.receiver.AppendHashdex(ctx, hashdexContent.ShardedData(), meta.RelabelerID, true); err != nil { + if err = p.adapter.AppendHashdex(ctx, hashdexContent.ShardedData(), state, true); err != nil { p.criticalErrorCount.With(prometheus.Labels{"error": err.Error(), "processor_type": "refill"}).Inc() return fmt.Errorf("failed to append decoded segment: %w", err) } diff --git a/pp-pkg/handler/processor/refill_processor_test.go b/pp-pkg/handler/processor/refill_processor_test.go index 3f7d9f2360..2a9d39557f 100644 --- a/pp-pkg/handler/processor/refill_processor_test.go +++ b/pp-pkg/handler/processor/refill_processor_test.go @@ -37,28 +37,41 @@ func (s *RefillProcessorSuite) TestProcess() { buffers := pool.New(8, 100e3, 2, func(sz int) any { return make([]byte, 0, sz) }) blockStorage := block.NewStorage(tmpDir, buffers) - mr := &metricReceiver{appendFn: func(ctx context.Context, hashdex cppbridge.ShardedData, relabelerID string) error { - return nil - }} + ar := &AdapterMock{ + AppendHashdexFunc: func(context.Context, cppbridge.ShardedData, *cppbridge.StateV2, bool) error { + return nil + }, + MergeOutOfOrderChunksFunc: func(context.Context) {}, + } + + metadata := model.Metadata{ + TenantID: "", + BlockID: uuid.New(), + ShardID: 0, + ShardsLog: 0, + SegmentEncodingVersion: 3, + RelabelerID: uuid.New().String(), + } + + states := &StatesStorageMock{ + GetStateByIDFunc: func(stateID string) (*cppbridge.StateV2, bool) { + if metadata.RelabelerID != stateID { + return nil, false + } - blockID := uuid.New() - shardID := uint16(0) - shardLog := uint8(0) - segmentEncodingVersion := cppbridge.EncodersVersion() + return nil, true + }, + } var expectedStatus model.RefillProcessingStatus gen := &segmentGenerator{segmentSize: 10} - refill := &testRefill{ - metadata: model.Metadata{ - TenantID: "", - BlockID: blockID, - ShardID: shardID, - ShardsLog: shardLog, - SegmentEncodingVersion: segmentEncodingVersion, + refill := &RefillMock{ + MetadataFunc: func() model.Metadata { + return metadata }, - readFn: func(ctx context.Context) (*model.Segment, error) { + ReadFunc: func(context.Context) (*model.Segment, error) { if len(gen.segments) == 5 { return nil, io.EOF } @@ -75,14 +88,14 @@ func (s *RefillProcessorSuite) TestProcess() { segment, readErr := gen.generate() return &segment.encoded, readErr }, - writeFn: func(ctx context.Context, status model.RefillProcessingStatus) error { + WriteFunc: func(_ context.Context, status model.RefillProcessingStatus) error { expectedStatus = status return nil }, } decoderBuilder := ppcore.NewBuilder(blockStorage) - refillProcessor := processor.NewRefillProcessor(decoderBuilder, mr, log.NewNopLogger(), nil) + refillProcessor := processor.NewRefillProcessor(decoderBuilder, ar, states, log.NewNopLogger(), nil) ctx, cancel := context.WithTimeout(context.Background(), time.Minute*5) defer cancel() @@ -101,28 +114,41 @@ func (s *RefillProcessorSuite) TestProcessWithError() { buffers := pool.New(8, 100e3, 2, func(sz int) any { return make([]byte, 0, sz) }) blockStorage := block.NewStorage(tmpDir, buffers) - mr := &metricReceiver{appendFn: func(ctx context.Context, hashdex cppbridge.ShardedData, relabelerID string) error { - return nil - }} + ar := &AdapterMock{ + AppendHashdexFunc: func(context.Context, cppbridge.ShardedData, *cppbridge.StateV2, bool) error { + return nil + }, + MergeOutOfOrderChunksFunc: func(context.Context) {}, + } - blockID := uuid.New() - shardID := uint16(0) - shardLog := uint8(0) - segmentEncodingVersion := cppbridge.EncodersVersion() + metadata := model.Metadata{ + TenantID: "", + BlockID: uuid.New(), + ShardID: 0, + ShardsLog: 0, + SegmentEncodingVersion: 3, + RelabelerID: uuid.New().String(), + } + + states := &StatesStorageMock{ + GetStateByIDFunc: func(stateID string) (*cppbridge.StateV2, bool) { + if metadata.RelabelerID != stateID { + return nil, false + } + + return nil, true + }, + } gen := &segmentGenerator{segmentSize: 10} fakeErr := errors.New("read error") - refill := &testRefill{ - metadata: model.Metadata{ - TenantID: "", - BlockID: blockID, - ShardID: shardID, - ShardsLog: shardLog, - SegmentEncodingVersion: segmentEncodingVersion, + refill := &RefillMock{ + MetadataFunc: func() model.Metadata { + return metadata }, - readFn: func(ctx context.Context) (*model.Segment, error) { + ReadFunc: func(context.Context) (*model.Segment, error) { if len(gen.segments) == 3 { return &model.Segment{}, fakeErr } @@ -130,13 +156,13 @@ func (s *RefillProcessorSuite) TestProcessWithError() { segment, readErr := gen.generate() return &segment.encoded, readErr }, - writeFn: func(ctx context.Context, status model.RefillProcessingStatus) error { + WriteFunc: func(context.Context, model.RefillProcessingStatus) error { return nil }, } decoderBuilder := ppcore.NewBuilder(blockStorage) - refillProcessor := processor.NewRefillProcessor(decoderBuilder, mr, log.NewNopLogger(), nil) + refillProcessor := processor.NewRefillProcessor(decoderBuilder, ar, states, log.NewNopLogger(), nil) ctx, cancel := context.WithTimeout(context.Background(), time.Minute*5) defer cancel() @@ -144,25 +170,3 @@ func (s *RefillProcessorSuite) TestProcessWithError() { err = refillProcessor.Process(ctx, refill) s.Require().ErrorIs(err, fakeErr) } - -// -// testRefill -// - -type testRefill struct { - metadata model.Metadata - readFn func(ctx context.Context) (*model.Segment, error) - writeFn func(ctx context.Context, status model.RefillProcessingStatus) error -} - -func (s *testRefill) Metadata() model.Metadata { - return s.metadata -} - -func (s *testRefill) Read(ctx context.Context) (*model.Segment, error) { - return s.readFn(ctx) -} - -func (s *testRefill) Write(ctx context.Context, status model.RefillProcessingStatus) error { - return s.writeFn(ctx, status) -} diff --git a/pp-pkg/handler/processor/remote_write_processor.go b/pp-pkg/handler/processor/remote_write_processor.go index 94970a85dd..2c8abf8529 100644 --- a/pp-pkg/handler/processor/remote_write_processor.go +++ b/pp-pkg/handler/processor/remote_write_processor.go @@ -2,6 +2,7 @@ package processor import ( "context" + "errors" "net/http" "strconv" @@ -11,22 +12,33 @@ import ( "github.com/prometheus/prometheus/pp/go/util" ) -var AlwaysCommit = true +var ( + // AlwaysCommit commit flags. + AlwaysCommit = true + // ErrUnknownRelablerID error when relabler ID not found. + ErrUnknownRelablerID = errors.New("unknown relabler id") +) + +// RemoteWriteProcessor RemoteWrite processor. type RemoteWriteProcessor struct { - receiver Receiver + adapter Adapter + states StatesStorage responseStatusCodeCount *prometheus.CounterVec } +// NewRemoteWriteProcessor init new [RemoteWriteProcessor]. func NewRemoteWriteProcessor( - receiver Receiver, + adapter Adapter, + states StatesStorage, registerer prometheus.Registerer, ) *RemoteWriteProcessor { factory := util.NewUnconflictRegisterer(registerer) return &RemoteWriteProcessor{ - receiver: receiver, + adapter: adapter, + states: states, responseStatusCodeCount: factory.NewCounterVec(prometheus.CounterOpts{ Name: "remote_write_opprotocol_processor_response_status_code", Help: "Number of 200/400 status codes responded with.", @@ -34,6 +46,7 @@ func NewRemoteWriteProcessor( } } +// Process read remote write data and append to adapter. func (p *RemoteWriteProcessor) Process(ctx context.Context, remoteWrite RemoteWrite) error { status := model.RemoteWriteProcessingStatus{Code: http.StatusOK} defer func() { @@ -43,6 +56,13 @@ func (p *RemoteWriteProcessor) Process(ctx context.Context, remoteWrite RemoteWr _ = remoteWrite.Write(ctx, status) }() + state, ok := p.states.GetStateByID(remoteWrite.Metadata().RelabelerID) + if !ok { + status.Code = http.StatusPreconditionFailed + status.Message = ErrUnknownRelablerID.Error() + return ErrUnknownRelablerID + } + rwb, err := remoteWrite.Read(ctx) if err != nil { status.Code = http.StatusBadRequest @@ -50,7 +70,7 @@ func (p *RemoteWriteProcessor) Process(ctx context.Context, remoteWrite RemoteWr return err } - if err := p.receiver.AppendSnappyProtobuf(ctx, rwb, remoteWrite.Metadata().RelabelerID, AlwaysCommit); err != nil { + if err := p.adapter.AppendSnappyProtobuf(ctx, rwb, state, AlwaysCommit); err != nil { status.Code = http.StatusBadRequest status.Message = err.Error() return err diff --git a/pp-pkg/handler/processor/remote_write_processor_test.go b/pp-pkg/handler/processor/remote_write_processor_test.go index a8270454c1..e4b1e97a5d 100644 --- a/pp-pkg/handler/processor/remote_write_processor_test.go +++ b/pp-pkg/handler/processor/remote_write_processor_test.go @@ -12,6 +12,7 @@ import ( "github.com/prometheus/prometheus/pp-pkg/handler/model" "github.com/prometheus/prometheus/pp-pkg/handler/processor" + pp_pkg_model "github.com/prometheus/prometheus/pp-pkg/model" "github.com/prometheus/prometheus/pp/go/cppbridge" ) @@ -24,35 +25,49 @@ func TestRemoteWriteProcessorSuite(t *testing.T) { } func (s *RemoteWriteProcessorSuite) TestProcess() { - mr := &metricReceiver{appendFn: func(ctx context.Context, hashdex cppbridge.ShardedData, relabelerID string) error { - return nil - }} - - blockID := uuid.New() - shardID := uint16(0) - shardLog := uint8(0) - segmentEncodingVersion := cppbridge.EncodersVersion() - - var expectedStatus model.RemoteWriteProcessingStatus - - rw := &testRemoteWrite{ - metadata: model.Metadata{ - TenantID: "", - BlockID: blockID, - ShardID: shardID, - ShardsLog: shardLog, - SegmentEncodingVersion: segmentEncodingVersion, + ar := &AdapterMock{ + AppendSnappyProtobufFunc: func(context.Context, pp_pkg_model.ProtobufData, *cppbridge.StateV2, bool) error { + return nil + }, + } + + metadata := model.Metadata{ + TenantID: "", + BlockID: uuid.New(), + ShardID: 0, + ShardsLog: 0, + SegmentEncodingVersion: 3, + RelabelerID: uuid.New().String(), + } + + states := &StatesStorageMock{ + GetStateByIDFunc: func(stateID string) (*cppbridge.StateV2, bool) { + if metadata.RelabelerID != stateID { + return nil, false + } + + return nil, true + }, + } + + var actualStatus model.RemoteWriteProcessingStatus + + rw := &RemoteWriteMock{ + MetadataFunc: func() model.Metadata { + return metadata }, - readFn: func(ctx context.Context) (*model.RemoteWriteBuffer, error) { + + ReadFunc: func(context.Context) (*model.RemoteWriteBuffer, error) { return &model.RemoteWriteBuffer{}, nil }, - writeFn: func(ctx context.Context, status model.RemoteWriteProcessingStatus) error { - expectedStatus = status + + WriteFunc: func(_ context.Context, status model.RemoteWriteProcessingStatus) error { + actualStatus = status return nil }, } - rwProcessor := processor.NewRemoteWriteProcessor(mr, nil) + rwProcessor := processor.NewRemoteWriteProcessor(ar, states, nil) ctx, cancel := context.WithTimeout(context.Background(), time.Minute*5) defer cancel() @@ -60,41 +75,54 @@ func (s *RemoteWriteProcessorSuite) TestProcess() { err := rwProcessor.Process(ctx, rw) s.Require().NoError(err) - s.Equal(http.StatusOK, expectedStatus.Code) + s.Equal(http.StatusOK, actualStatus.Code) } -func (s *RemoteWriteProcessorSuite) TestProcessWithError() { - mr := &metricReceiver{appendFn: func(ctx context.Context, hashdex cppbridge.ShardedData, relabelerID string) error { - return nil - }} +func (s *RemoteWriteProcessorSuite) TestProcessWithErrorRead() { + ar := &AdapterMock{ + AppendSnappyProtobufFunc: func(context.Context, pp_pkg_model.ProtobufData, *cppbridge.StateV2, bool) error { + return nil + }, + } - blockID := uuid.New() - shardID := uint16(0) - shardLog := uint8(0) - segmentEncodingVersion := cppbridge.EncodersVersion() + metadata := model.Metadata{ + TenantID: "", + BlockID: uuid.New(), + ShardID: 0, + ShardsLog: 0, + SegmentEncodingVersion: 3, + RelabelerID: uuid.New().String(), + } - fakeErr := errors.New("read error") + states := &StatesStorageMock{ + GetStateByIDFunc: func(stateID string) (*cppbridge.StateV2, bool) { + if metadata.RelabelerID != stateID { + return nil, false + } - var expectedStatus model.RemoteWriteProcessingStatus + return nil, true + }, + } + + fakeErr := errors.New("read error") + var actualStatus model.RemoteWriteProcessingStatus - rw := &testRemoteWrite{ - metadata: model.Metadata{ - TenantID: "", - BlockID: blockID, - ShardID: shardID, - ShardsLog: shardLog, - SegmentEncodingVersion: segmentEncodingVersion, + rw := &RemoteWriteMock{ + MetadataFunc: func() model.Metadata { + return metadata }, - readFn: func(ctx context.Context) (*model.RemoteWriteBuffer, error) { + + ReadFunc: func(context.Context) (*model.RemoteWriteBuffer, error) { return nil, fakeErr }, - writeFn: func(ctx context.Context, status model.RemoteWriteProcessingStatus) error { - expectedStatus = status + + WriteFunc: func(_ context.Context, status model.RemoteWriteProcessingStatus) error { + actualStatus = status return nil }, } - rwProcessor := processor.NewRemoteWriteProcessor(mr, nil) + rwProcessor := processor.NewRemoteWriteProcessor(ar, states, nil) ctx, cancel := context.WithTimeout(context.Background(), time.Minute*5) defer cancel() @@ -102,28 +130,57 @@ func (s *RemoteWriteProcessorSuite) TestProcessWithError() { err := rwProcessor.Process(ctx, rw) s.Require().ErrorIs(err, fakeErr) - s.Equal(expectedStatus.Code, http.StatusBadRequest) - s.Equal(expectedStatus.Message, fakeErr.Error()) + s.Equal(http.StatusBadRequest, actualStatus.Code) + s.Equal(fakeErr.Error(), actualStatus.Message) } -// -// testRemoteWrite -// +func (s *RemoteWriteProcessorSuite) TestProcessWithErrorGetStateByID() { + ar := &AdapterMock{ + AppendSnappyProtobufFunc: func(context.Context, pp_pkg_model.ProtobufData, *cppbridge.StateV2, bool) error { + return nil + }, + } -type testRemoteWrite struct { - metadata model.Metadata - readFn func(ctx context.Context) (*model.RemoteWriteBuffer, error) - writeFn func(ctx context.Context, status model.RemoteWriteProcessingStatus) error -} + metadata := model.Metadata{ + TenantID: "", + BlockID: uuid.New(), + ShardID: 0, + ShardsLog: 0, + SegmentEncodingVersion: 3, + RelabelerID: uuid.New().String(), + } -func (s *testRemoteWrite) Metadata() model.Metadata { - return s.metadata -} + states := &StatesStorageMock{ + GetStateByIDFunc: func(string) (*cppbridge.StateV2, bool) { + return nil, false + }, + } -func (s *testRemoteWrite) Read(ctx context.Context) (*model.RemoteWriteBuffer, error) { - return s.readFn(ctx) -} + var actualStatus model.RemoteWriteProcessingStatus + + rw := &RemoteWriteMock{ + MetadataFunc: func() model.Metadata { + return metadata + }, + + ReadFunc: func(context.Context) (*model.RemoteWriteBuffer, error) { + return &model.RemoteWriteBuffer{}, nil + }, + + WriteFunc: func(_ context.Context, status model.RemoteWriteProcessingStatus) error { + actualStatus = status + return nil + }, + } + + rwProcessor := processor.NewRemoteWriteProcessor(ar, states, nil) + + ctx, cancel := context.WithTimeout(context.Background(), time.Minute*5) + defer cancel() + + err := rwProcessor.Process(ctx, rw) + s.Require().ErrorIs(err, processor.ErrUnknownRelablerID) -func (s *testRemoteWrite) Write(ctx context.Context, status model.RemoteWriteProcessingStatus) error { - return s.writeFn(ctx, status) + s.Equal(http.StatusPreconditionFailed, actualStatus.Code) + s.Equal(processor.ErrUnknownRelablerID.Error(), actualStatus.Message) } diff --git a/pp-pkg/handler/processor/stream_processor.go b/pp-pkg/handler/processor/stream_processor.go index 1b80ac4677..2bfa79a929 100644 --- a/pp-pkg/handler/processor/stream_processor.go +++ b/pp-pkg/handler/processor/stream_processor.go @@ -13,7 +13,8 @@ import ( type StreamProcessor struct { decoderBuilder DecoderBuilder - receiver Receiver + adapter Adapter + states StatesStorage criticalErrorCount *prometheus.CounterVec rejectedSegmentCount *prometheus.CounterVec @@ -26,14 +27,16 @@ type StreamProcessor struct { func NewStreamProcessor( decoderBuilder DecoderBuilder, - receiver Receiver, + adapter Adapter, + states StatesStorage, registerer prometheus.Registerer, ) *StreamProcessor { factory := util.NewUnconflictRegisterer(registerer) return &StreamProcessor{ decoderBuilder: decoderBuilder, - receiver: receiver, + adapter: adapter, + states: states, criticalErrorCount: factory.NewCounterVec(prometheus.CounterOpts{ Name: "remote_write_opprotocol_processor_critical_error_count", Help: "Total number of critical errors occurred during serving metric stream.", @@ -67,6 +70,16 @@ func NewStreamProcessor( func (p *StreamProcessor) Process(ctx context.Context, stream MetricStream) error { meta := stream.Metadata() + + state, ok := p.states.GetStateByID(meta.RelabelerID) + if !ok { + p.criticalErrorCount.With(prometheus.Labels{ + "error": ErrUnknownRelablerID.Error(), + "processor_type": "stream", + }).Inc() + return ErrUnknownRelablerID + } + decoder := p.decoderBuilder.Build(meta) defer func() { _ = decoder.Close() }() @@ -98,10 +111,10 @@ func (p *StreamProcessor) Process(ctx context.Context, stream MetricStream) erro Timestamp: hashdexContent.CreatedAt(), } - if err = p.receiver.AppendHashdex( + if err = p.adapter.AppendHashdex( ctx, hashdexContent.ShardedData(), - meta.RelabelerID, + state, AlwaysCommit, ); err != nil { processingStatus.Code = model.ProcessingStatusRejected diff --git a/pp-pkg/handler/processor/stream_processor_test.go b/pp-pkg/handler/processor/stream_processor_test.go index 16129c6246..7e3a9bb8d0 100644 --- a/pp-pkg/handler/processor/stream_processor_test.go +++ b/pp-pkg/handler/processor/stream_processor_test.go @@ -19,52 +19,9 @@ import ( "github.com/prometheus/prometheus/pp-pkg/handler/storage/block" "github.com/prometheus/prometheus/pp/go/cppbridge" coremodel "github.com/prometheus/prometheus/pp/go/model" - "github.com/prometheus/prometheus/pp/go/relabeler" "github.com/prometheus/prometheus/util/pool" ) -type testStream struct { - metadata model.Metadata - readFn func(ctx context.Context) (*model.Segment, error) - writeFn func(ctx context.Context, status model.StreamSegmentProcessingStatus) error -} - -func (s *testStream) Metadata() model.Metadata { - return s.metadata -} - -func (s *testStream) Read(ctx context.Context) (*model.Segment, error) { - return s.readFn(ctx) -} - -func (s *testStream) Write(ctx context.Context, status model.StreamSegmentProcessingStatus) error { - return s.writeFn(ctx, status) -} - -type metricReceiver struct { - appendFn func(ctx context.Context, hashdex cppbridge.ShardedData, relabelerID string) error -} - -func (mr *metricReceiver) AppendHashdex( - ctx context.Context, - hashdex cppbridge.ShardedData, - relabelerID string, - _ bool, -) error { - return mr.appendFn(ctx, hashdex, relabelerID) -} - -func (mr *metricReceiver) AppendSnappyProtobuf( - ctx context.Context, - data relabeler.ProtobufData, - relabelerID string, - _ bool, -) error { - return nil -} - -func (*metricReceiver) MergeOutOfOrderChunks(_ context.Context) {} - type segmentContainer struct { timeSeries []coremodel.TimeSeries encoded model.Segment @@ -129,33 +86,47 @@ func TestStreamProcessor_Process(t *testing.T) { buffers := pool.New(8, 100e3, 2, func(sz int) interface{} { return make([]byte, 0, sz) }) blockStorage := block.NewStorage(tmpDir, buffers) - mr := &metricReceiver{appendFn: func(ctx context.Context, hashdex cppbridge.ShardedData, relabelerID string) error { - return nil - }} + + ar := &AdapterMock{ + AppendHashdexFunc: func(context.Context, cppbridge.ShardedData, *cppbridge.StateV2, bool) error { + return nil + }, + } + + metadata := model.Metadata{ + TenantID: "", + BlockID: uuid.New(), + ShardID: 0, + ShardsLog: 0, + SegmentEncodingVersion: 3, + RelabelerID: uuid.New().String(), + } + + states := &StatesStorageMock{ + GetStateByIDFunc: func(stateID string) (*cppbridge.StateV2, bool) { + if metadata.RelabelerID != stateID { + return nil, false + } + + return nil, true + }, + } decoderBuilder := ppcore.NewBuilder(blockStorage) - streamProcessor := processor.NewStreamProcessor(decoderBuilder, mr, nil) + streamProcessor := processor.NewStreamProcessor(decoderBuilder, ar, states, nil) resolvec := make(chan struct{}, 1) - blockID := uuid.New() - shardID := uint16(0) - shardLog := uint8(0) - segmentEncodingVersion := cppbridge.EncodersVersion() gen := &segmentGenerator{segmentSize: 10} iteration := 0 fakeErr := errors.New("read error") - stream := &testStream{ - metadata: model.Metadata{ - TenantID: "", - BlockID: blockID, - ShardID: shardID, - ShardsLog: shardLog, - SegmentEncodingVersion: segmentEncodingVersion, + stream := &MetricStreamMock{ + MetadataFunc: func() model.Metadata { + return metadata }, - readFn: func(ctx context.Context) (*model.Segment, error) { + ReadFunc: func(context.Context) (*model.Segment, error) { resolvec <- struct{}{} if len(gen.segments) == 3 && iteration == 0 { @@ -167,7 +138,7 @@ func TestStreamProcessor_Process(t *testing.T) { segment, readErr := gen.generate() return &segment.encoded, readErr }, - writeFn: func(ctx context.Context, status model.StreamSegmentProcessingStatus) error { + WriteFunc: func(context.Context, model.StreamSegmentProcessingStatus) error { <-resolvec return nil }, @@ -179,15 +150,11 @@ func TestStreamProcessor_Process(t *testing.T) { err = streamProcessor.Process(ctx, stream) require.ErrorIs(t, err, fakeErr) - stream = &testStream{ - metadata: model.Metadata{ - TenantID: "", - BlockID: blockID, - ShardID: shardID, - ShardsLog: shardLog, - SegmentEncodingVersion: segmentEncodingVersion, + stream = &MetricStreamMock{ + MetadataFunc: func() model.Metadata { + return metadata }, - readFn: func(ctx context.Context) (*model.Segment, error) { + ReadFunc: func(context.Context) (*model.Segment, error) { resolvec <- struct{}{} if len(gen.segments) == 5 && iteration == 1 { @@ -199,7 +166,7 @@ func TestStreamProcessor_Process(t *testing.T) { segment, readErr := gen.generate() return &segment.encoded, readErr }, - writeFn: func(ctx context.Context, status model.StreamSegmentProcessingStatus) error { + WriteFunc: func(context.Context, model.StreamSegmentProcessingStatus) error { <-resolvec return nil }, @@ -208,15 +175,11 @@ func TestStreamProcessor_Process(t *testing.T) { err = streamProcessor.Process(ctx, stream) require.ErrorIs(t, err, fakeErr) - stream = &testStream{ - metadata: model.Metadata{ - TenantID: "", - BlockID: blockID, - ShardID: shardID, - ShardsLog: shardLog, - SegmentEncodingVersion: segmentEncodingVersion, + stream = &MetricStreamMock{ + MetadataFunc: func() model.Metadata { + return metadata }, - readFn: func(ctx context.Context) (*model.Segment, error) { + ReadFunc: func(context.Context) (*model.Segment, error) { resolvec <- struct{}{} if len(gen.segments) == 5 && iteration == 2 { @@ -236,7 +199,7 @@ func TestStreamProcessor_Process(t *testing.T) { segment, readErr := gen.generate() return &segment.encoded, readErr }, - writeFn: func(ctx context.Context, status model.StreamSegmentProcessingStatus) error { + WriteFunc: func(context.Context, model.StreamSegmentProcessingStatus) error { <-resolvec return nil }, diff --git a/pp-pkg/handler/states.go b/pp-pkg/handler/states.go new file mode 100644 index 0000000000..d47bc99d46 --- /dev/null +++ b/pp-pkg/handler/states.go @@ -0,0 +1,84 @@ +package handler + +import ( + "fmt" + "sync" + + "github.com/prometheus/prometheus/config" + "github.com/prometheus/prometheus/pp/go/cppbridge" +) + +// +// States +// + +// void empty struct. +var void = struct{}{} + +// StatesStorage stores the [cppbridge.State]'s. +type StatesStorage struct { + m map[string]*cppbridge.StateV2 + mx sync.RWMutex +} + +// NewStatesStorage init new [StatesStorage]. +func NewStatesStorage() *StatesStorage { + return &StatesStorage{ + m: map[string]*cppbridge.StateV2{config.TransparentRelabeler: cppbridge.NewTransitionStateV2()}, + mx: sync.RWMutex{}, + } +} + +// ApplyConfig updates the [StatesStorage]'s configs. +func (s *StatesStorage) ApplyConfig(conf *config.Config) error { + rwcfgs := conf.RemoteWriteReceiverConfig() + if len(rwcfgs.Configs) == 0 { + return nil + } + + updated := make(map[string]struct{}, len(rwcfgs.Configs)+1) + updated[config.TransparentRelabeler] = void + + s.mx.Lock() + defer s.mx.Unlock() + for _, cfg := range rwcfgs.Configs { + stateID := cfg.GetName() + rcfg := cfg.GetConfigs() + + if st, ok := s.m[stateID]; ok { + if st.StatelessRelabeler().EqualConfigs(rcfg) { + updated[stateID] = void + continue + } + } + + statelessRelabeler, err := cppbridge.NewStatelessRelabeler(rcfg) + if err != nil { + return fmt.Errorf("failed creating stateless relabeler for %s: %w", stateID, err) + } + + state := cppbridge.NewStateV2() + state.SetStatelessRelabeler(statelessRelabeler) + + s.m[stateID] = state + updated[stateID] = void + } + + for stateID := range s.m { + if _, ok := updated[stateID]; !ok { + // clear unnecessary + delete(s.m, stateID) + } + } + + return nil +} + +// GetStateByID returns [cppbridge.State] by state ID if exist. +func (s *StatesStorage) GetStateByID(stateID string) (*cppbridge.StateV2, bool) { + s.mx.RLock() + state, ok := s.m[stateID] + s.mx.RUnlock() + + return state, ok +} diff --git a/pp-pkg/model/interface.go b/pp-pkg/model/interface.go new file mode 100644 index 0000000000..6f8f1c845c --- /dev/null +++ b/pp-pkg/model/interface.go @@ -0,0 +1,23 @@ +package model + +import "github.com/prometheus/prometheus/pp/go/model" + +// +// ProtobufData +// + +// ProtobufData is an universal interface for blob protobuf data. +type ProtobufData interface { + Bytes() []byte + Destroy() +} + +// +// TimeSeriesBatch +// + +// TimeSeriesBatch is an universal interface for batch [model.TimeSeries]. +type TimeSeriesBatch interface { + TimeSeries() []model.TimeSeries + Destroy() +} diff --git a/pp-pkg/scrape/manager.go b/pp-pkg/scrape/manager.go index 64e994376c..376fdea297 100644 --- a/pp-pkg/scrape/manager.go +++ b/pp-pkg/scrape/manager.go @@ -18,31 +18,29 @@ import ( "github.com/prometheus/prometheus/config" "github.com/prometheus/prometheus/discovery/targetgroup" "github.com/prometheus/prometheus/model/labels" + pp_pkg_model "github.com/prometheus/prometheus/pp-pkg/model" "github.com/prometheus/prometheus/pp/go/cppbridge" - "github.com/prometheus/prometheus/pp/go/relabeler" "github.com/prometheus/prometheus/util/osutil" "github.com/prometheus/prometheus/util/pool" ) -type Receiver interface { - // AppendTimeSeries append TimeSeries data to relabeling hashdex data. +// Adapter for implementing the [Queryable] interface and append data. +type Adapter interface { + // AppendTimeSeries append TimeSeries data to [Head]. AppendTimeSeries( ctx context.Context, - data relabeler.TimeSeriesData, - state *cppbridge.State, - relabelerID string, + data pp_pkg_model.TimeSeriesBatch, + state *cppbridge.StateV2, commitToWal bool, ) (cppbridge.RelabelerStats, error) - // AppendTimeSeries append TimeSeries data to relabeling hashdex data. - AppendTimeSeriesHashdex( + + // AppendScraperHashdex append ScraperHashdex data to [Head]. + AppendScraperHashdex( ctx context.Context, hashdex cppbridge.ShardedData, - state *cppbridge.State, - relabelerID string, + state *cppbridge.StateV2, commitToWal bool, ) (cppbridge.RelabelerStats, error) - RelabelerIDIsExist(relabelerID string) bool - GetState() *cppbridge.State } // Options are the configuration parameters to the scrape manager. @@ -79,7 +77,7 @@ const DefaultNameEscapingScheme = model.ValueEncodingEscaping type Manager struct { opts *Options logger log.Logger - receiver Receiver + adapter Adapter graceShut chan struct{} offsetSeed uint64 // Global offsetSeed seed is used to spread scrape workload across HA setup. @@ -93,6 +91,8 @@ type Manager struct { triggerReload chan struct{} + reportStatelessRelabeler *cppbridge.StatelessRelabeler + metrics *scrapeMetrics } @@ -100,7 +100,7 @@ type Manager struct { func NewManager( o *Options, logger log.Logger, - receiver Receiver, + adapter Adapter, registerer prometheus.Registerer, ) (*Manager, error) { if o == nil { @@ -115,18 +115,24 @@ func NewManager( return nil, fmt.Errorf("failed to create scrape manager due to error: %w", err) } + reportStatelessRelabeler, err := cppbridge.NewStatelessRelabeler([]*cppbridge.RelabelConfig{}) + if err != nil { + return nil, fmt.Errorf("failed creating report stateless relabeler: %w", err) + } + m := &Manager{ - receiver: receiver, - opts: o, - logger: logger, - scrapeConfigs: make(map[string]*config.ScrapeConfig), - scrapePools: make(map[string]*scrapePool), - graceShut: make(chan struct{}), - triggerReload: make(chan struct{}, 1), - metrics: sm, - buffers: pool.New(1e3, 100e6, 2, func(sz int) interface{} { return make([]byte, 0, sz) }), - bufferBuilders: newBuildersPool(), - bufferBatches: newbatchesPool(), + adapter: adapter, + opts: o, + logger: logger, + scrapeConfigs: make(map[string]*config.ScrapeConfig), + scrapePools: make(map[string]*scrapePool), + graceShut: make(chan struct{}), + triggerReload: make(chan struct{}, 1), + metrics: sm, + buffers: pool.New(1e3, 100e6, 2, func(sz int) interface{} { return make([]byte, 0, sz) }), + bufferBuilders: newBuildersPool(), + bufferBatches: newbatchesPool(), + reportStatelessRelabeler: reportStatelessRelabeler, } m.metrics.setTargetMetadataCacheGatherer(m) @@ -197,7 +203,8 @@ func (m *Manager) reload() { m.metrics.targetScrapePools.Inc() sp, err := newScrapePool( scrapeConfig, - m.receiver, + m.adapter, + m.reportStatelessRelabeler, m.offsetSeed, log.With(m.logger, "scrape_pool", setName), m.buffers, diff --git a/pp-pkg/scrape/scrape.go b/pp-pkg/scrape/scrape.go index 608f3fd388..72c5a615ac 100644 --- a/pp-pkg/scrape/scrape.go +++ b/pp-pkg/scrape/scrape.go @@ -100,7 +100,7 @@ type ScraperHashdex interface { Parse(buffer []byte, defaultTimestamp int64) (uint32, error) // RangeMetadata calls f sequentially for each metadata present in the hashdex. // If f returns false, range stops the iteration. - RangeMetadata(f func(metadata cppbridge.WALScraperHashdexMetadata) bool) + RangeMetadata(f func(md cppbridge.WALScraperHashdexMetadata) bool) // Cluster get Cluster name. Cluster() string // Replica get Replica name. @@ -111,24 +111,26 @@ type scrapeLoopOptions struct { target *Target scraper scraper metricLimits *cppbridge.MetricLimits + statelessRelabeler *cppbridge.StatelessRelabeler + cache *scrapeCache + mrc []*relabel.Config + interval time.Duration + timeout time.Duration + validationScheme model.ValidationScheme honorLabels bool honorTimestamps bool trackTimestampsStaleness bool - interval time.Duration - timeout time.Duration scrapeClassicHistograms bool - validationScheme model.ValidationScheme - mrc []*relabel.Config - cache *scrapeCache enableCompression bool } // scrapePool manages scrapes for sets of targets. type scrapePool struct { - receiver Receiver - logger log.Logger - cancel context.CancelFunc - httpOpts []config_util.HTTPClientOption + statelessRelabeler *cppbridge.StatelessRelabeler + reportStatelessRelabeler *cppbridge.StatelessRelabeler + logger log.Logger + cancel context.CancelFunc + httpOpts []config_util.HTTPClientOption // mtx must not be taken after targetMtx. mtx sync.Mutex @@ -154,7 +156,8 @@ type scrapePool struct { func newScrapePool( cfg *config.ScrapeConfig, - receiver Receiver, + adapter Adapter, + reportStatelessRelabeler *cppbridge.StatelessRelabeler, offsetSeed uint64, logger log.Logger, buffers *pool.Pool, @@ -163,11 +166,6 @@ func newScrapePool( options *Options, metrics *scrapeMetrics, ) (*scrapePool, error) { - scrapeName := config.ScrapePrefix + cfg.JobName - if !receiver.RelabelerIDIsExist(scrapeName) { - return nil, fmt.Errorf("relabeler id not found for scrape name: %s", scrapeName) - } - if logger == nil { logger = log.NewNopLogger() } @@ -177,24 +175,39 @@ func newScrapePool( return nil, fmt.Errorf("error creating HTTP client: %w", err) } + if reportStatelessRelabeler == nil { + reportStatelessRelabeler, err = cppbridge.NewStatelessRelabeler([]*cppbridge.RelabelConfig{}) + if err != nil { + return nil, fmt.Errorf("failed creating report stateless relabeler: %w", err) + } + } + + rcfgs, err := cfg.PPMetricRelabelConfigs() + if err != nil { + return nil, fmt.Errorf("failed get pp relabel configs: %w", err) + } + + statelessRelabeler, err := cppbridge.NewStatelessRelabeler(rcfgs) + if err != nil { + return nil, fmt.Errorf("failed creating stateless relabeler: %w", err) + } + ctx, cancel := context.WithCancel(context.Background()) sp := &scrapePool{ - cancel: cancel, - receiver: receiver, - config: cfg, - client: client, - activeTargets: map[uint64]*Target{}, - targetsCache: map[uint64][]*Target{}, - loops: map[uint64]loop{}, - symbolTable: labels.NewSymbolTable(), - logger: logger, - metrics: metrics, - httpOpts: options.HTTPClientOptions, - noDefaultPort: options.NoDefaultPort, - } - - // cfg.RelabelConfigs - // scrapeState, err := cppbridge.NewEmptyState() + statelessRelabeler: statelessRelabeler, + reportStatelessRelabeler: reportStatelessRelabeler, + logger: logger, + cancel: cancel, + config: cfg, + client: client, + activeTargets: map[uint64]*Target{}, + targetsCache: map[uint64][]*Target{}, + loops: map[uint64]loop{}, + symbolTable: labels.NewSymbolTable(), + metrics: metrics, + httpOpts: options.HTTPClientOptions, + noDefaultPort: options.NoDefaultPort, + } sp.newLoop = func(opts scrapeLoopOptions) loop { // Update the targets retrieval function for metadata to a new scrape cache. @@ -231,8 +244,10 @@ func newScrapePool( return newScrapeLoop( ctx, opts.scraper, - receiver, + adapter, log.With(logger, "target", opts.target), + opts.statelessRelabeler, + sp.reportStatelessRelabeler, targetOptions, buffers, bufferBuilders, @@ -338,6 +353,17 @@ func (sp *scrapePool) reload(cfg *config.ScrapeConfig) error { oldClient := sp.client sp.client = client + rcfgs, err := cfg.PPMetricRelabelConfigs() + if err != nil { + return fmt.Errorf("failed get pp relabel configs: %w", err) + } + + statelessRelabeler, err := cppbridge.NewStatelessRelabeler(rcfgs) + if err != nil { + return fmt.Errorf("failed creating stateless relabeler: %w", err) + } + sp.statelessRelabeler = statelessRelabeler + sp.metrics.targetScrapePoolTargetLimit.WithLabelValues(sp.config.JobName).Set(float64(sp.config.TargetLimit)) sp.restartLoops(reuseCache) @@ -367,6 +393,7 @@ func (sp *scrapePool) restartLoops(reuseCache bool) { enableCompression = sp.config.EnableCompression trackTimestampsStaleness = sp.config.TrackTimestampsStaleness mrc = sp.config.MetricRelabelConfigs + statelessRelabeler = sp.statelessRelabeler ) validationScheme := model.LegacyValidation @@ -401,6 +428,7 @@ func (sp *scrapePool) restartLoops(reuseCache bool) { target: t, scraper: s, metricLimits: metricLimits, + statelessRelabeler: statelessRelabeler, honorLabels: honorLabels, honorTimestamps: honorTimestamps, enableCompression: enableCompression, @@ -511,6 +539,7 @@ func (sp *scrapePool) sync(targets []*Target) { trackTimestampsStaleness = sp.config.TrackTimestampsStaleness mrc = sp.config.MetricRelabelConfigs scrapeClassicHistograms = sp.config.ScrapeClassicHistograms + statelessRelabeler = sp.statelessRelabeler ) validationScheme := model.LegacyValidation @@ -541,6 +570,7 @@ func (sp *scrapePool) sync(targets []*Target) { target: t, scraper: s, metricLimits: metricLimits, + statelessRelabeler: statelessRelabeler, honorLabels: honorLabels, honorTimestamps: honorTimestamps, enableCompression: enableCompression, @@ -759,10 +789,10 @@ type loop interface { type scrapeLoop struct { scraper scraper - receiver Receiver + adapter Adapter logger log.Logger - state *cppbridge.State - reportState *cppbridge.State + state *cppbridge.StateV2 + reportState *cppbridge.StateV2 cache *scrapeCache buffers *pool.Pool bufferBuilders *buildersPool @@ -800,8 +830,10 @@ type scrapeLoop struct { func newScrapeLoop( ctx context.Context, sc scraper, - receiver Receiver, + adapter Adapter, logger log.Logger, + statelessRelabeler *cppbridge.StatelessRelabeler, + reportStatelessRelabeler *cppbridge.StatelessRelabeler, options *cppbridge.RelabelerOptions, buffers *pool.Pool, bufferBuilders *buildersPool, @@ -844,11 +876,13 @@ func newScrapeLoop( appenderCtx = ContextWithTarget(appenderCtx, target) } - state := receiver.GetState() + state := cppbridge.NewStateV2WithoutLock() + state.SetStatelessRelabeler(statelessRelabeler) state.EnableTrackStaleness() state.SetRelabelerOptions(options) - reportState := receiver.GetState() + reportState := cppbridge.NewStateV2WithoutLock() + reportState.SetStatelessRelabeler(reportStatelessRelabeler) reportState.SetRelabelerOptions(&cppbridge.RelabelerOptions{ TargetLabels: options.TargetLabels, HonorTimestamps: true, @@ -856,7 +890,7 @@ func newScrapeLoop( sl := &scrapeLoop{ scraper: sc, - receiver: receiver, + adapter: adapter, logger: logger, state: state, reportState: reportState, @@ -1101,11 +1135,10 @@ func (sl *scrapeLoop) endOfRunStaleness(last time.Time, ticker *time.Ticker, int // sl.context would have been cancelled, hence using sl.appenderCtx. emptyBatch := sl.bufferBatches.get() sl.state.SetDefTimestamp(timestamp.FromTime(staleTime)) - if _, err := sl.receiver.AppendTimeSeries( + if _, err := sl.adapter.AppendTimeSeries( sl.appenderCtx, emptyBatch, sl.state, - sl.scrapeName, CommitToWalOnAppend, ); err != nil { level.Warn(sl.logger).Log("msg", "Stale append failed", "err", err) @@ -1140,11 +1173,10 @@ func (sl *scrapeLoop) append( // if input is empty for stalenan if len(b) == 0 { sl.state.SetDefTimestamp(timestamp.FromTime(ts)) - _, err = sl.receiver.AppendTimeSeries( + _, err = sl.adapter.AppendTimeSeries( sl.appenderCtx, sl.bufferBatches.get(), sl.state, - sl.scrapeName, CommitToWalOnAppend, ) return 0, stats, err @@ -1239,11 +1271,10 @@ loop: } sl.state.SetDefTimestamp(defTime) - stats, err = sl.receiver.AppendTimeSeries( + stats, err = sl.adapter.AppendTimeSeries( sl.appenderCtx, batch, sl.state, - sl.scrapeName, CommitToWalOnAppend, ) if err != nil { @@ -1293,11 +1324,10 @@ func (sl *scrapeLoop) appendCpp( } sl.state.SetDefTimestamp(timestamp.FromTime(ts)) - stats, err = sl.receiver.AppendTimeSeriesHashdex( + stats, err = sl.adapter.AppendScraperHashdex( sl.appenderCtx, hashdex, sl.state, - sl.scrapeName, CommitToWalOnAppend, ) if err != nil { @@ -1446,11 +1476,10 @@ func (sl *scrapeLoop) report( } } - if _, err = sl.receiver.AppendTimeSeries( + if _, err = sl.adapter.AppendTimeSeries( sl.appenderCtx, batch, sl.reportState, - config.TransparentRelabeler, CommitToWalOnAppend, ); err != nil { return @@ -1495,11 +1524,10 @@ func (sl *scrapeLoop) reportStale(start time.Time) (err error) { } } - if _, err = sl.receiver.AppendTimeSeries( + if _, err = sl.adapter.AppendTimeSeries( sl.appenderCtx, batch, sl.reportState, - config.TransparentRelabeler, CommitToWalOnAppend, ); err != nil { return diff --git a/pp-pkg/storage/adapter.go b/pp-pkg/storage/adapter.go index 2a140b300e..16c2636407 100644 --- a/pp-pkg/storage/adapter.go +++ b/pp-pkg/storage/adapter.go @@ -7,9 +7,9 @@ import ( "github.com/jonboulle/clockwork" "github.com/prometheus/client_golang/prometheus" + "github.com/prometheus/prometheus/pp-pkg/model" "github.com/prometheus/prometheus/pp/go/cppbridge" "github.com/prometheus/prometheus/pp/go/hatracker" - "github.com/prometheus/prometheus/pp/go/model" pp_storage "github.com/prometheus/prometheus/pp/go/storage" "github.com/prometheus/prometheus/pp/go/storage/appender" "github.com/prometheus/prometheus/pp/go/storage/head/services" @@ -17,26 +17,6 @@ import ( "github.com/prometheus/prometheus/storage" ) -// -// ProtobufData -// - -// ProtobufData is an universal interface for blob protobuf data. -type ProtobufData interface { - Bytes() []byte - Destroy() -} - -// -// TimeSeriesBatch -// - -// TimeSeriesBatch is an universal interface for batch [model.TimeSeries]. -type TimeSeriesBatch interface { - TimeSeries() []model.TimeSeries - Destroy() -} - // // Adapter // @@ -60,21 +40,16 @@ func NewAdapter( clock clockwork.Clock, proxy *pp_storage.ProxyHead, registerer prometheus.Registerer, -) (*Adapter, error) { - transparentState, err := cppbridge.NewEmptyStateV2WithConfig([]*cppbridge.RelabelConfig{}) - if err != nil { - return nil, err - } - +) *Adapter { return &Adapter{ proxy: proxy, haTracker: hatracker.NewHighAvailabilityTracker(clock, registerer), hashdexFactory: cppbridge.HashdexFactory{}, hashdexLimits: cppbridge.DefaultWALHashdexLimits(), - transparentState: transparentState, + transparentState: cppbridge.NewTransitionStateV2(), activeQuerierMetrics: querier.NewMetrics(registerer, querier.QueryableAppenderSource), storageQuerierMetrics: querier.NewMetrics(registerer, querier.QueryableStorageSource), - }, nil + } } // AppendHashdex append incoming [cppbridge.HashdexContent] to [Head]. @@ -124,7 +99,7 @@ func (ar *Adapter) AppendScraperHashdex( // AppendSnappyProtobuf append compressed via snappy Protobuf data to [Head]. func (ar *Adapter) AppendSnappyProtobuf( ctx context.Context, - compressedData ProtobufData, + compressedData model.ProtobufData, state *cppbridge.StateV2, commitToWal bool, ) error { @@ -153,7 +128,7 @@ func (ar *Adapter) AppendSnappyProtobuf( // AppendTimeSeries append TimeSeries data to [Head]. func (ar *Adapter) AppendTimeSeries( ctx context.Context, - data TimeSeriesBatch, + data model.TimeSeriesBatch, state *cppbridge.StateV2, commitToWal bool, ) (stats cppbridge.RelabelerStats, err error) { diff --git a/pp/entrypoint/prometheus_relabeler.cpp b/pp/entrypoint/prometheus_relabeler.cpp index aaf88a408a..e671c843e1 100644 --- a/pp/entrypoint/prometheus_relabeler.cpp +++ b/pp/entrypoint/prometheus_relabeler.cpp @@ -231,14 +231,6 @@ extern "C" void prompp_prometheus_relabel_stalenans_state_dtor(void* args) { static_cast(args)->~Arguments(); } -extern "C" void prompp_prometheus_relabel_stalenans_state_reset(void* args) { - struct Arguments { - StaleNaNsStatePtr state; - }; - - static_cast(args)->state->reset(); -} - extern "C" void prompp_prometheus_per_shard_relabeler_input_relabeling_with_stalenans(void* args, void* res) { struct Arguments { PromPP::Primitives::Go::SliceView shards_inner_series; @@ -756,6 +748,73 @@ extern "C" void prompp_prometheus_per_goroutine_relabeler_input_relabeling_with_ } } +extern "C" void prompp_prometheus_per_goroutine_relabeler_input_transition_relabeling(void* args, void* res) { + struct Arguments { + PromPP::Primitives::Go::SliceView shards_inner_series; + PerGoroutineRelabelerPtr per_goroutine_relabeler; + HashdexVariant* hashdex; + LssVariantPtr target_lss; + }; + struct Result { + uint32_t samples_added{0}; + uint32_t series_added{0}; + uint32_t series_drop{0}; + PromPP::Primitives::Go::Slice error; + bool target_lss_has_reallocations{}; + }; + + auto in = static_cast(args); + auto out = new (res) Result(); + + try { + std::visit( + [in, out](auto& hashdex) { + auto& target_lss = std::get(*in->target_lss); + + const entrypoint::head::ReallocationsDetector reallocation_detector(target_lss); + in->per_goroutine_relabeler->input_transition_relabeling(target_lss, hashdex, *out, in->shards_inner_series); + target_lss.build_deferred_indexes(); + out->target_lss_has_reallocations = reallocation_detector.has_reallocations(); + }, + *in->hashdex); + } catch (...) { + auto err_stream = PromPP::Primitives::Go::BytesStream(&out->error); + entrypoint::handle_current_exception(err_stream); + } +} + +extern "C" void prompp_prometheus_per_goroutine_relabeler_input_transition_relabeling_only_read(void* args, void* res) { + struct Arguments { + PromPP::Primitives::Go::SliceView shards_inner_series; + PerGoroutineRelabelerPtr per_goroutine_relabeler; + HashdexVariant* hashdex; + LssVariantPtr target_lss; + }; + struct Result { + uint32_t samples_added{0}; + uint32_t series_added{0}; + uint32_t series_drop{0}; + bool ok{}; + PromPP::Primitives::Go::Slice error; + }; + + auto in = static_cast(args); + auto out = new (res) Result(); + + try { + std::visit( + [in, out](auto& hashdex) { + auto& target_lss = std::get(*in->target_lss); + + out->ok = in->per_goroutine_relabeler->input_transition_relabeling_only_read(target_lss, hashdex, *out, in->shards_inner_series); + }, + *in->hashdex); + } catch (...) { + auto err_stream = PromPP::Primitives::Go::BytesStream(&out->error); + entrypoint::handle_current_exception(err_stream); + } +} + extern "C" void prompp_prometheus_per_goroutine_relabeler_append_relabeler_series(void* args, void* res) { struct Arguments { PromPP::Primitives::Go::SliceView shards_inner_series; diff --git a/pp/entrypoint/prometheus_relabeler.h b/pp/entrypoint/prometheus_relabeler.h index a075ca4a0f..9cf824cad2 100644 --- a/pp/entrypoint/prometheus_relabeler.h +++ b/pp/entrypoint/prometheus_relabeler.h @@ -181,15 +181,6 @@ void prompp_prometheus_relabel_stalenans_state_ctor(void* res); */ void prompp_prometheus_relabel_stalenans_state_dtor(void* args); -/** - * @brief Reset StaleNaNsState. - * - * @param args { - * state uintptr // pointer to StaleNaNsState; - * } - */ -void prompp_prometheus_relabel_stalenans_state_reset(void* args); - /** * @brief relabeling incomig hashdex(first stage) with state stalenans. * @@ -540,6 +531,46 @@ void prompp_prometheus_per_goroutine_relabeler_input_relabeling_with_stalenans(v */ void prompp_prometheus_per_goroutine_relabeler_input_relabeling_with_stalenans_from_cache(void* args, void* res); +/** + * @brief transparent relabeling incomig hashdex(first stage). + * + * @param args { + * shards_inner_series []*InnerSeries // go slice with InnerSeries; + * per_goroutine_relabeler uintptr // pointer to constructed per goroutine relabeler; + * hashdex uintptr // pointer to filled hashdex; + * target_lss uintptr // pointer to constructed target label sets; + * } + * + * @param res { + * samples_added uint32 // number of added samples; + * series_added uint32 // number of added series; + * series_drop uint32 // number of dropped series; + * error []byte // error string if thrown; + * target_lss_has_reallocations bool // true if target lss has reallocations + * } + */ +void prompp_prometheus_per_goroutine_relabeler_input_transition_relabeling(void* args, void* res); + +/** + * @brief transparent relabeling incomig hashdex(first stage) from cache. + * + * @param args { + * shards_inner_series []*InnerSeries // go slice with InnerSeries; + * per_goroutine_relabeler uintptr // pointer to constructed per goroutine relabeler; + * hashdex uintptr // pointer to filled hashdex; + * target_lss uintptr // pointer to constructed target label sets; + * } + * + * @param res { + * samples_added uint32 // number of added samples; + * series_added uint32 // number of added series; + * series_drop uint32 // number of dropped series; + * ok bool // true if all label set find in cache; + * error []byte // error string if thrown; + * } + */ +void prompp_prometheus_per_goroutine_relabeler_input_transition_relabeling_only_read(void* args, void* res); + /** * @brief add relabeled ls to lss, add to result and add to cache update(second stage). * diff --git a/pp/go/cppbridge/entrypoint.go b/pp/go/cppbridge/entrypoint.go index bb400b1bde..b6a9826b0f 100644 --- a/pp/go/cppbridge/entrypoint.go +++ b/pp/go/cppbridge/entrypoint.go @@ -98,6 +98,58 @@ var ( }, ) + // per_goroutine_relabeler input_transition_relabeling + perGoroutineRelabelerInputTransitionRelabelingSum = util.NewUnconflictRegisterer( + prometheus.DefaultRegisterer, + ).NewCounter( + prometheus.CounterOpts{ + Name: "prompp_cppbridge_unsafecall_nanoseconds_sum", + Help: "The time duration cpp call.", + ConstLabels: prometheus.Labels{ + "object": "per_goroutine_relabeler", + "method": "input_transition_relabeling", + }, + }, + ) + perGoroutineRelabelerInputTransitionRelabelingCount = util.NewUnconflictRegisterer( + prometheus.DefaultRegisterer, + ).NewCounter( + prometheus.CounterOpts{ + Name: "prompp_cppbridge_unsafecall_nanoseconds_count", + Help: "The time duration cpp call.", + ConstLabels: prometheus.Labels{ + "object": "per_goroutine_relabeler", + "method": "input_transition_relabeling", + }, + }, + ) + + // per_goroutine_relabeler input_transition_relabeling_only_read + perGoroutineRelabelerInputTransitionRelabelingOnlyReadSum = util.NewUnconflictRegisterer( + prometheus.DefaultRegisterer, + ).NewCounter( + prometheus.CounterOpts{ + Name: "prompp_cppbridge_unsafecall_nanoseconds_sum", + Help: "The time duration cpp call.", + ConstLabels: prometheus.Labels{ + "object": "per_goroutine_relabeler", + "method": "input_transition_relabeling_only_read", + }, + }, + ) + perGoroutineRelabelerInputTransitionRelabelingOnlyReadCount = util.NewUnconflictRegisterer( + prometheus.DefaultRegisterer, + ).NewCounter( + prometheus.CounterOpts{ + Name: "prompp_cppbridge_unsafecall_nanoseconds_count", + Help: "The time duration cpp call.", + ConstLabels: prometheus.Labels{ + "object": "per_goroutine_relabeler", + "method": "input_transition_relabeling_only_read", + }, + }, + ) + // input_relabeler append_relabeler_series inputRelabelerAppendRelabelerSeriesSum = util.NewUnconflictRegisterer(prometheus.DefaultRegisterer).NewCounter( prometheus.CounterOpts{ @@ -1530,18 +1582,6 @@ func prometheusRelabelStaleNansStateCtor() uintptr { return res.state } -func prometheusRelabelStaleNansStateReset(state uintptr) { - args := struct { - state uintptr - }{state} - - testGC() - fastcgo.UnsafeCall1( - C.prompp_prometheus_relabel_stalenans_state_reset, - uintptr(unsafe.Pointer(&args)), - ) -} - func prometheusRelabelStaleNansStateDtor(state uintptr) { args := struct { state uintptr @@ -3426,6 +3466,71 @@ func prometheusPerGoroutineRelabelerInputRelabelingWithStalenansFromCache( return res.RelabelerStats, res.exception, res.ok } +// prometheusPerGoroutineRelabelerInputTransitionRelabeling wrapper for +// transparent relabeling incoming hashdex(first stage). +func prometheusPerGoroutineRelabelerInputTransitionRelabeling( + perGoroutineRelabeler, targetLss, hashdex uintptr, + shardsInnerSeries []*InnerSeries, +) (stats RelabelerStats, exception []byte, targetLssHasReallocations bool) { + args := struct { + shardsInnerSeries []*InnerSeries + perGoroutineRelabeler uintptr + hashdex uintptr + targetLss uintptr + }{ + shardsInnerSeries, + perGoroutineRelabeler, + hashdex, + targetLss, + } + var res struct { + RelabelerStats + exception []byte + targetLssHasReallocations bool + } + start := time.Now().UnixNano() + testGC() + fastcgo.UnsafeCall2( + C.prompp_prometheus_per_goroutine_relabeler_input_transition_relabeling, + uintptr(unsafe.Pointer(&args)), + uintptr(unsafe.Pointer(&res)), + ) + perGoroutineRelabelerInputTransitionRelabelingSum.Add(float64(time.Now().UnixNano() - start)) + perGoroutineRelabelerInputTransitionRelabelingCount.Inc() + + return res.RelabelerStats, res.exception, res.targetLssHasReallocations +} + +// prometheusPerGoroutineRelabelerInputRelabelingOnlyRead wrapper for transparent relabeling +// incoming hashdex(first stage) from cache. +func prometheusPerGoroutineRelabelerInputRelabelingOnlyRead( + perGoroutineRelabeler, targetLss, hashdex uintptr, + shardsInnerSeries []*InnerSeries, +) (stats RelabelerStats, exception []byte, ok bool) { + args := struct { + shardsInnerSeries []*InnerSeries + perGoroutineRelabeler uintptr + hashdex uintptr + targetLss uintptr + }{shardsInnerSeries, perGoroutineRelabeler, hashdex, targetLss} + var res struct { + RelabelerStats + ok bool + exception []byte + } + start := time.Now().UnixNano() + testGC() + fastcgo.UnsafeCall2( + C.prompp_prometheus_per_goroutine_relabeler_input_transition_relabeling_only_read, + uintptr(unsafe.Pointer(&args)), + uintptr(unsafe.Pointer(&res)), + ) + perGoroutineRelabelerInputTransitionRelabelingOnlyReadSum.Add(float64(time.Now().UnixNano() - start)) + perGoroutineRelabelerInputTransitionRelabelingOnlyReadCount.Inc() + + return res.RelabelerStats, res.exception, res.ok +} + // prometheusPerGoroutineRelabelerAppendRelabelerSeries wrapper for add relabeled ls to lss, // add to result and add to cache update(second stage). func prometheusPerGoroutineRelabelerAppendRelabelerSeries( diff --git a/pp/go/cppbridge/entrypoint.h b/pp/go/cppbridge/entrypoint.h index ddf8c4a11d..5e21d0a0b7 100755 --- a/pp/go/cppbridge/entrypoint.h +++ b/pp/go/cppbridge/entrypoint.h @@ -861,15 +861,6 @@ void prompp_prometheus_relabel_stalenans_state_ctor(void* res); */ void prompp_prometheus_relabel_stalenans_state_dtor(void* args); -/** - * @brief Reset StaleNaNsState. - * - * @param args { - * state uintptr // pointer to StaleNaNsState; - * } - */ -void prompp_prometheus_relabel_stalenans_state_reset(void* args); - /** * @brief relabeling incomig hashdex(first stage) with state stalenans. * @@ -1220,6 +1211,46 @@ void prompp_prometheus_per_goroutine_relabeler_input_relabeling_with_stalenans(v */ void prompp_prometheus_per_goroutine_relabeler_input_relabeling_with_stalenans_from_cache(void* args, void* res); +/** + * @brief transparent relabeling incomig hashdex(first stage). + * + * @param args { + * shards_inner_series []*InnerSeries // go slice with InnerSeries; + * per_goroutine_relabeler uintptr // pointer to constructed per goroutine relabeler; + * hashdex uintptr // pointer to filled hashdex; + * target_lss uintptr // pointer to constructed target label sets; + * } + * + * @param res { + * samples_added uint32 // number of added samples; + * series_added uint32 // number of added series; + * series_drop uint32 // number of dropped series; + * error []byte // error string if thrown; + * target_lss_has_reallocations bool // true if target lss has reallocations + * } + */ +void prompp_prometheus_per_goroutine_relabeler_input_transition_relabeling(void* args, void* res); + +/** + * @brief transparent relabeling incomig hashdex(first stage) from cache. + * + * @param args { + * shards_inner_series []*InnerSeries // go slice with InnerSeries; + * per_goroutine_relabeler uintptr // pointer to constructed per goroutine relabeler; + * hashdex uintptr // pointer to filled hashdex; + * target_lss uintptr // pointer to constructed target label sets; + * } + * + * @param res { + * samples_added uint32 // number of added samples; + * series_added uint32 // number of added series; + * series_drop uint32 // number of dropped series; + * ok bool // true if all label set find in cache; + * error []byte // error string if thrown; + * } + */ +void prompp_prometheus_per_goroutine_relabeler_input_transition_relabeling_only_read(void* args, void* res); + /** * @brief add relabeled ls to lss, add to result and add to cache update(second stage). * diff --git a/pp/go/cppbridge/prometheus_relabeler.go b/pp/go/cppbridge/prometheus_relabeler.go index c297571bc4..b0ed6dd00d 100644 --- a/pp/go/cppbridge/prometheus_relabeler.go +++ b/pp/go/cppbridge/prometheus_relabeler.go @@ -16,6 +16,7 @@ import ( "github.com/prometheus/common/model" ) +// NullTimestamp the timestamp that is used as the nil value. const NullTimestamp = math.MinInt64 // ErrLSSNullPointer - error when lss is null pointer @@ -526,10 +527,6 @@ func NewStaleNansState() *StaleNansState { return s } -func (s *StaleNansState) Reset() { - prometheusRelabelStaleNansStateReset(s.state) -} - // RelabelerStats statistics return from relabeler. type RelabelerStats struct { SamplesAdded uint32 @@ -1191,8 +1188,8 @@ func (pgr *PerGoroutineRelabeler) AppendRelabelerSeries( return hasReallocations, handleException(exception) } -// InputRelabeling relabeling incoming hashdex(first stage). -func (pgr *PerGoroutineRelabeler) InputRelabeling( +// Relabeling relabeling incoming hashdex(first stage). +func (pgr *PerGoroutineRelabeler) Relabeling( ctx context.Context, inputLss *LabelSetStorage, targetLss *LabelSetStorage, @@ -1210,6 +1207,113 @@ func (pgr *PerGoroutineRelabeler) InputRelabeling( return RelabelerStats{}, false, ErrMustImplementCptrable } + if state.TrackStaleness() { + return pgr.inputRelabelingWithStalenans( + inputLss, + targetLss, + state, + cptrContainer, + shardsInnerSeries, + shardsRelabeledSeries, + ) + } + + if state.IsTransition() { + return pgr.inputTransitionRelabeling( + targetLss, + state, + cptrContainer, + shardsInnerSeries, + ) + } + + return pgr.inputRelabeling( + inputLss, + targetLss, + state, + cptrContainer, + shardsInnerSeries, + shardsRelabeledSeries, + ) +} + +// RelabelingFromCache relabeling incoming hashdex(first stage) from cache. +func (pgr *PerGoroutineRelabeler) RelabelingFromCache( + ctx context.Context, + inputLss *LabelSetStorage, + targetLss *LabelSetStorage, + state *StateV2, + shardedData ShardedData, + shardsInnerSeries []*InnerSeries, +) (RelabelerStats, bool, error) { + if ctx.Err() != nil { + return RelabelerStats{}, false, ctx.Err() + } + + cptrContainer, ok := shardedData.(cptrable) + if !ok { + return RelabelerStats{}, false, ErrMustImplementCptrable + } + + if state.TrackStaleness() { + return pgr.inputRelabelingWithStalenansFromCache( + inputLss, + targetLss, + state, + cptrContainer, + shardsInnerSeries, + ) + } + + if state.IsTransition() { + return pgr.inputTransitionRelabelingOnlyRead( + targetLss, + state, + cptrContainer, + shardsInnerSeries, + ) + } + + return pgr.inputRelabelingFromCache( + inputLss, + targetLss, + state, + cptrContainer, + shardsInnerSeries, + ) +} + +// UpdateRelabelerState add to cache relabled data(third stage). +func (pgr *PerGoroutineRelabeler) UpdateRelabelerState( + ctx context.Context, + cache *Cache, + shardsRelabelerStateUpdate []*RelabelerStateUpdate, +) error { + if ctx.Err() != nil { + return ctx.Err() + } + + exception := prometheusPerGoroutineRelabelerUpdateRelabelerState( + shardsRelabelerStateUpdate, + pgr.cptr, + cache.cPointer, + ) + + runtime.KeepAlive(pgr) + runtime.KeepAlive(cache) + + return handleException(exception) +} + +// inputRelabeling relabeling incoming hashdex(first stage). +func (pgr *PerGoroutineRelabeler) inputRelabeling( + inputLss *LabelSetStorage, + targetLss *LabelSetStorage, + state *StateV2, + cptrContainer cptrable, + shardsInnerSeries []*InnerSeries, + shardsRelabeledSeries []*RelabeledSeries, +) (RelabelerStats, bool, error) { cache := state.CacheByShard(pgr.shardID) cache.lock.Lock() stats, exception, hasReallocations := prometheusPerGoroutineRelabelerInputRelabeling( @@ -1235,23 +1339,13 @@ func (pgr *PerGoroutineRelabeler) InputRelabeling( } // InputRelabelingFromCache relabeling incoming hashdex(first stage) from cache. -func (pgr *PerGoroutineRelabeler) InputRelabelingFromCache( - ctx context.Context, +func (pgr *PerGoroutineRelabeler) inputRelabelingFromCache( inputLss *LabelSetStorage, targetLss *LabelSetStorage, state *StateV2, - shardedData ShardedData, + cptrContainer cptrable, shardsInnerSeries []*InnerSeries, ) (RelabelerStats, bool, error) { - if ctx.Err() != nil { - return RelabelerStats{}, false, ctx.Err() - } - - cptrContainer, ok := shardedData.(cptrable) - if !ok { - return RelabelerStats{}, false, ErrMustImplementCptrable - } - cache := state.CacheByShard(pgr.shardID) cache.lock.RLock() stats, exception, ok := prometheusPerGoroutineRelabelerInputRelabelingFromCache( @@ -1274,25 +1368,15 @@ func (pgr *PerGoroutineRelabeler) InputRelabelingFromCache( return stats, ok, handleException(exception) } -// InputRelabelingWithStalenans relabeling incoming hashdex(first stage) with state stalenans. -func (pgr *PerGoroutineRelabeler) InputRelabelingWithStalenans( - ctx context.Context, +// inputRelabelingWithStalenans relabeling incoming hashdex(first stage) with state stalenans. +func (pgr *PerGoroutineRelabeler) inputRelabelingWithStalenans( inputLss *LabelSetStorage, targetLss *LabelSetStorage, state *StateV2, - shardedData ShardedData, + cptrContainer cptrable, shardsInnerSeries []*InnerSeries, shardsRelabeledSeries []*RelabeledSeries, ) (RelabelerStats, bool, error) { - if ctx.Err() != nil { - return RelabelerStats{}, false, ctx.Err() - } - - cptrContainer, ok := shardedData.(cptrable) - if !ok { - return RelabelerStats{}, false, ErrMustImplementCptrable - } - cache := state.CacheByShard(pgr.shardID) cache.lock.Lock() stats, exception, hasReallocations := prometheusPerGoroutineRelabelerInputRelabelingWithStalenans( @@ -1319,24 +1403,14 @@ func (pgr *PerGoroutineRelabeler) InputRelabelingWithStalenans( return stats, hasReallocations, handleException(exception) } -// InputRelabelingWithStalenansFromCache relabeling incoming hashdex(first stage) from cache with state stalenans. -func (pgr *PerGoroutineRelabeler) InputRelabelingWithStalenansFromCache( - ctx context.Context, +// inputRelabelingWithStalenansFromCache relabeling incoming hashdex(first stage) from cache with state stalenans. +func (pgr *PerGoroutineRelabeler) inputRelabelingWithStalenansFromCache( inputLss *LabelSetStorage, targetLss *LabelSetStorage, state *StateV2, - shardedData ShardedData, + cptrContainer cptrable, shardsInnerSeries []*InnerSeries, ) (RelabelerStats, bool, error) { - if ctx.Err() != nil { - return RelabelerStats{}, false, ctx.Err() - } - - cptrContainer, ok := shardedData.(cptrable) - if !ok { - return RelabelerStats{}, false, ErrMustImplementCptrable - } - cache := state.CacheByShard(pgr.shardID) cache.lock.RLock() stats, exception, ok := prometheusPerGoroutineRelabelerInputRelabelingWithStalenansFromCache( @@ -1361,89 +1435,48 @@ func (pgr *PerGoroutineRelabeler) InputRelabelingWithStalenansFromCache( return stats, ok, handleException(exception) } -// Relabeling relabeling incoming hashdex(first stage). -func (pgr *PerGoroutineRelabeler) Relabeling( - ctx context.Context, - inputLss *LabelSetStorage, +// inputTransitionRelabeling transparent relabeling incoming hashdex(first stage). +func (pgr *PerGoroutineRelabeler) inputTransitionRelabeling( targetLss *LabelSetStorage, state *StateV2, - shardedData ShardedData, + cptrContainer cptrable, shardsInnerSeries []*InnerSeries, - shardsRelabeledSeries []*RelabeledSeries, ) (RelabelerStats, bool, error) { - if state.TrackStaleness() { - return pgr.InputRelabelingWithStalenans( - ctx, - inputLss, - targetLss, - state, - shardedData, - shardsInnerSeries, - shardsRelabeledSeries, - ) - } - - return pgr.InputRelabeling( - ctx, - inputLss, - targetLss, - state, - shardedData, + stats, exception, hasReallocations := prometheusPerGoroutineRelabelerInputTransitionRelabeling( + pgr.cptr, + targetLss.Pointer(), + cptrContainer.cptr(), shardsInnerSeries, - shardsRelabeledSeries, ) + + runtime.KeepAlive(pgr) + runtime.KeepAlive(targetLss) + runtime.KeepAlive(state) + runtime.KeepAlive(cptrContainer) + + return stats, hasReallocations, handleException(exception) } -// RelabelingFromCache relabeling incoming hashdex(first stage) from cache. -func (pgr *PerGoroutineRelabeler) RelabelingFromCache( - ctx context.Context, - inputLss *LabelSetStorage, +// inputTransitionRelabelingOnlyRead transparent relabeling incoming hashdex(first stage) from cache. +func (pgr *PerGoroutineRelabeler) inputTransitionRelabelingOnlyRead( targetLss *LabelSetStorage, state *StateV2, - shardedData ShardedData, + cptrContainer cptrable, shardsInnerSeries []*InnerSeries, ) (RelabelerStats, bool, error) { - if state.TrackStaleness() { - return pgr.InputRelabelingWithStalenansFromCache( - ctx, - inputLss, - targetLss, - state, - shardedData, - shardsInnerSeries, - ) - } - - return pgr.InputRelabelingFromCache( - ctx, - inputLss, - targetLss, - state, - shardedData, - shardsInnerSeries, - ) -} - -// UpdateRelabelerState add to cache relabled data(third stage). -func (pgr *PerGoroutineRelabeler) UpdateRelabelerState( - ctx context.Context, - cache *Cache, - shardsRelabelerStateUpdate []*RelabelerStateUpdate, -) error { - if ctx.Err() != nil { - return ctx.Err() - } - - exception := prometheusPerGoroutineRelabelerUpdateRelabelerState( - shardsRelabelerStateUpdate, + stats, exception, ok := prometheusPerGoroutineRelabelerInputRelabelingOnlyRead( pgr.cptr, - cache.cPointer, + targetLss.Pointer(), + cptrContainer.cptr(), + shardsInnerSeries, ) runtime.KeepAlive(pgr) - runtime.KeepAlive(cache) + runtime.KeepAlive(targetLss) + runtime.KeepAlive(state) + runtime.KeepAlive(cptrContainer) - return handleException(exception) + return stats, ok, handleException(exception) } // @@ -1509,8 +1542,8 @@ type StateV2 struct { trackStaleness bool } -// NewTransitionStateV2WithLock init empty [StateV2], with locks. -func NewTransitionStateV2WithLock() *StateV2 { +// NewTransitionStateV2 init empty [StateV2], with locks. +func NewTransitionStateV2() *StateV2 { return &StateV2{ locker: NewTransitionLocker(), generationHead: math.MaxUint64, @@ -1529,8 +1562,8 @@ func NewTransitionStateV2WithoutLock() *StateV2 { } } -// NewStateV2WithLock init empty [StateV2], with locks. -func NewStateV2WithLock() *StateV2 { +// NewStateV2 init empty [StateV2], with locks. +func NewStateV2() *StateV2 { return &StateV2{ locker: NewTransitionLocker(), generationHead: math.MaxUint64, @@ -1600,7 +1633,7 @@ func (s *StateV2) Reconfigure( } // the transition state does not require caches and staleNaNs - if s.status&transitionStatus == transitionStatus { + if s.IsTransition() { s.status |= inited s.generationHead = generationHead s.locker.Unlock() @@ -1613,6 +1646,11 @@ func (s *StateV2) Reconfigure( s.locker.Unlock() } +// IsTransition indicates whether the state is transition. +func (s *StateV2) IsTransition() bool { + return s.status&transitionStatus == transitionStatus +} + // RelabelerOptions return Options for relabeler. func (s *StateV2) RelabelerOptions() RelabelerOptions { return s.options @@ -1630,7 +1668,7 @@ func (s *StateV2) SetRelabelerOptions(options *RelabelerOptions) { // SetStatelessRelabeler sets [StatelessRelabeler] for [PerGoroutineRelabeler]. func (s *StateV2) SetStatelessRelabeler(statelessRelabeler *StatelessRelabeler) { - if s.status&transitionStatus == transitionStatus { + if s.IsTransition() { panic("state is transition") } @@ -1652,7 +1690,7 @@ func (s *StateV2) StaleNansStateByShard(shardID uint16) *StaleNansState { // StatelessRelabeler returns [StatelessRelabeler] for [PerGoroutineRelabeler]. func (s *StateV2) StatelessRelabeler() *StatelessRelabeler { - if s.status&transitionStatus == transitionStatus { + if s.IsTransition() { panic("state is transition") } diff --git a/pp/go/cppbridge/prometheus_relabeler_test.go b/pp/go/cppbridge/prometheus_relabeler_test.go index ed7ed3532f..98eb537b1c 100644 --- a/pp/go/cppbridge/prometheus_relabeler_test.go +++ b/pp/go/cppbridge/prometheus_relabeler_test.go @@ -601,7 +601,7 @@ func (s *RelabelerSuite) TestToHash_EmptyConfig() { s.Require().Equal(xxhash.Sum64String("0"+a.String()), cppbridge.ToHash(rCfgs)) } -func (s *RelabelerSuite) TestInputPerGoroutineRelabeler() { +func (s *RelabelerSuite) TestPerGoroutineRelabeler() { wr := prompb.WriteRequest{ Timeseries: []prompb.TimeSeries{ { @@ -657,7 +657,7 @@ func (s *RelabelerSuite) TestInputPerGoroutineRelabeler() { state.Reconfigure(0, numberOfShards) pgr := cppbridge.NewPerGoroutineRelabeler(numberOfShards, 0) - stats, hasReallocations, err := pgr.InputRelabeling( + stats, hasReallocations, err := pgr.Relabeling( s.baseCtx, inputLss, targetLss, @@ -671,7 +671,75 @@ func (s *RelabelerSuite) TestInputPerGoroutineRelabeler() { s.True(hasReallocations) } -func (s *RelabelerSuite) TestInputPerGoroutineRelabelerFromCacheTrue() { +func (s *RelabelerSuite) TestPerGoroutineRelabelerDrop() { + wr := prompb.WriteRequest{ + Timeseries: []prompb.TimeSeries{ + { + Labels: []prompb.Label{ + {Name: "job", Value: "abc"}, + {Name: "instance", Value: "value1"}, + }, + Samples: []prompb.Sample{ + {Value: 0.1, Timestamp: time.Now().UnixMilli()}, + }, + }, + { + Labels: []prompb.Label{ + {Name: "instance", Value: "value1"}, + }, + Samples: []prompb.Sample{ + {Value: 0.1, Timestamp: time.Now().UnixMilli()}, + }, + }, + }, + } + data, err := wr.Marshal() + s.Require().NoError(err) + + rCfgs := []*cppbridge.RelabelConfig{ + { + SourceLabels: []string{"job"}, + Regex: "abc", + Action: cppbridge.Keep, + }, + } + + inputLss := cppbridge.NewLssStorage() + targetLss := cppbridge.NewQueryableLssStorage() + + var numberOfShards uint16 = 1 + + hlimits := cppbridge.DefaultWALHashdexLimits() + h, err := cppbridge.NewWALSnappyProtobufHashdex(snappy.Encode(nil, data), hlimits) + s.Require().NoError(err) + + shardsInnerSeries := cppbridge.NewShardsInnerSeries(numberOfShards) + shardsRelabeledSeries := cppbridge.NewShardsRelabeledSeries(numberOfShards) + + statelessRelabeler, err := cppbridge.NewStatelessRelabeler(rCfgs) + s.Require().NoError(err) + + state := cppbridge.NewStateV2WithoutLock() + state.SetRelabelerOptions(&s.options) + state.SetStatelessRelabeler(statelessRelabeler) + state.Reconfigure(0, numberOfShards) + + pgr := cppbridge.NewPerGoroutineRelabeler(numberOfShards, 0) + stats, hasReallocations, err := pgr.Relabeling( + s.baseCtx, + inputLss, + targetLss, + state, + h, + shardsInnerSeries, + shardsRelabeledSeries, + ) + s.Require().NoError(err) + s.Equal(cppbridge.RelabelerStats{0, 0, 2}, stats) + s.True(hasReallocations) +} + +func (s *RelabelerSuite) TestPerGoroutineRelabelerFromCacheTrue() { wr := prompb.WriteRequest{ Timeseries: []prompb.TimeSeries{ { @@ -718,7 +786,7 @@ func (s *RelabelerSuite) TestInputPerGoroutineRelabelerFromCacheTrue() { state.Reconfigure(0, numberOfShards) pgr := cppbridge.NewPerGoroutineRelabeler(numberOfShards, 0) - stats, hasReallocations, err := pgr.InputRelabeling( + stats, hasReallocations, err := pgr.Relabeling( s.baseCtx, inputLss, targetLss, @@ -731,7 +799,7 @@ func (s *RelabelerSuite) TestInputPerGoroutineRelabelerFromCacheTrue() { s.Equal(cppbridge.RelabelerStats{1, 1, 0}, stats) s.True(hasReallocations) - stats, ok, err := pgr.InputRelabelingFromCache( + stats, ok, err := pgr.RelabelingFromCache( s.baseCtx, inputLss, targetLss, @@ -744,7 +812,7 @@ func (s *RelabelerSuite) TestInputPerGoroutineRelabelerFromCacheTrue() { s.True(ok) } -func (s *RelabelerSuite) TestInputPerGoroutineRelabelerFromCacheFalse() { +func (s *RelabelerSuite) TestPerGoroutineRelabelerFromCacheFalse() { wr := prompb.WriteRequest{ Timeseries: []prompb.TimeSeries{ { @@ -786,7 +854,7 @@ func (s *RelabelerSuite) TestInputPerGoroutineRelabelerFromCacheFalse() { state.Reconfigure(0, numberOfShards) pgr := cppbridge.NewPerGoroutineRelabeler(numberOfShards, 0) - stats, ok, err := pgr.InputRelabelingFromCache( + stats, ok, err := pgr.RelabelingFromCache( s.baseCtx, inputLss, targetLss, @@ -799,7 +867,7 @@ func (s *RelabelerSuite) TestInputPerGoroutineRelabelerFromCacheFalse() { s.False(ok) } -func (s *RelabelerSuite) TestInputPerGoroutineRelabelerFromCachePartially() { +func (s *RelabelerSuite) TestPerGoroutineRelabelerFromCachePartially() { ts := time.Now().UnixMilli() wr1 := prompb.WriteRequest{ Timeseries: []prompb.TimeSeries{ @@ -972,7 +1040,7 @@ func (s *RelabelerSuite) TestInputPerGoroutineRelabelerFromCachePartially() { state.Reconfigure(0, numberOfShards) pgr := cppbridge.NewPerGoroutineRelabeler(numberOfShards, 0) - stats, hasReallocations, err := pgr.InputRelabeling( + stats, hasReallocations, err := pgr.Relabeling( s.baseCtx, inputLss, targetLss, @@ -986,7 +1054,379 @@ func (s *RelabelerSuite) TestInputPerGoroutineRelabelerFromCachePartially() { s.True(hasReallocations) shardsInnerSeries = cppbridge.NewShardsInnerSeries(numberOfShards) - stats, ok, err := pgr.InputRelabelingFromCache( + stats, ok, err := pgr.RelabelingFromCache( + s.baseCtx, + inputLss, + targetLss, + state, + h2, + shardsInnerSeries, + ) + s.Require().NoError(err) + s.Equal(cppbridge.RelabelerStats{4, 0, 0}, stats) + s.False(ok) + s.Equal(uint64(4), shardsInnerSeries[0].Size()) + + stats, _, err = pgr.Relabeling( + s.baseCtx, + inputLss, + targetLss, + state, + h2, + shardsInnerSeries, + shardsRelabeledSeries, + ) + s.Require().NoError(err) + s.Equal(cppbridge.RelabelerStats{1, 1, 0}, stats) + s.Equal(uint64(5), shardsInnerSeries[0].Size()) +} + +func (s *RelabelerSuite) TestPerGoroutineRelabelerTransition() { + wr := prompb.WriteRequest{ + Timeseries: []prompb.TimeSeries{ + { + Labels: []prompb.Label{ + {Name: "__name__", Value: "value"}, + {Name: "job", Value: "abc"}, + {Name: "instance", Value: "value1"}, + }, + Samples: []prompb.Sample{ + {Value: 0.1, Timestamp: time.Now().UnixMilli()}, + }, + }, + { + Labels: []prompb.Label{ + {Name: "__name__", Value: "value"}, + {Name: "instance", Value: "value1"}, + }, + Samples: []prompb.Sample{ + {Value: 0.1, Timestamp: time.Now().UnixMilli()}, + }, + }, + }, + } + data, err := wr.Marshal() + s.Require().NoError(err) + + inputLss := cppbridge.NewLssStorage() + targetLss := cppbridge.NewQueryableLssStorage() + + var numberOfShards uint16 = 1 + + hlimits := cppbridge.DefaultWALHashdexLimits() + h, err := cppbridge.NewWALSnappyProtobufHashdex(snappy.Encode(nil, data), hlimits) + s.Require().NoError(err) + + shardsInnerSeries := cppbridge.NewShardsInnerSeries(numberOfShards) + shardsRelabeledSeries := cppbridge.NewShardsRelabeledSeries(numberOfShards) + + state := cppbridge.NewTransitionStateV2() + state.Reconfigure(0, numberOfShards) + + pgr := cppbridge.NewPerGoroutineRelabeler(numberOfShards, 0) + stats, hasReallocations, err := pgr.Relabeling( + s.baseCtx, + inputLss, + targetLss, + state, + h, + shardsInnerSeries, + shardsRelabeledSeries, + ) + s.Require().NoError(err) + s.Equal(cppbridge.RelabelerStats{2, 2, 0}, stats) + s.True(hasReallocations) +} + +func (s *RelabelerSuite) TestPerGoroutineRelabelerFromCacheTrueTransition() { + wr := prompb.WriteRequest{ + Timeseries: []prompb.TimeSeries{ + { + Labels: []prompb.Label{ + {Name: "__name__", Value: "value"}, + {Name: "job", Value: "abc"}, + {Name: "instance", Value: "value1"}, + }, + Samples: []prompb.Sample{ + {Value: 0.1, Timestamp: time.Now().UnixMilli()}, + }, + }, + }, + } + data, err := wr.Marshal() + s.Require().NoError(err) + + inputLss := cppbridge.NewLssStorage() + targetLss := cppbridge.NewQueryableLssStorage() + + var numberOfShards uint16 = 1 + + hlimits := cppbridge.DefaultWALHashdexLimits() + h, err := cppbridge.NewWALSnappyProtobufHashdex(snappy.Encode(nil, data), hlimits) + s.Require().NoError(err) + + shardsInnerSeries := cppbridge.NewShardsInnerSeries(numberOfShards) + shardsRelabeledSeries := cppbridge.NewShardsRelabeledSeries(numberOfShards) + + state := cppbridge.NewTransitionStateV2() + state.Reconfigure(0, numberOfShards) + + pgr := cppbridge.NewPerGoroutineRelabeler(numberOfShards, 0) + stats, hasReallocations, err := pgr.Relabeling( + s.baseCtx, + inputLss, + targetLss, + state, + h, + shardsInnerSeries, + shardsRelabeledSeries, + ) + s.Require().NoError(err) + s.Equal(cppbridge.RelabelerStats{1, 1, 0}, stats) + s.True(hasReallocations) + + stats, ok, err := pgr.RelabelingFromCache( + s.baseCtx, + inputLss, + targetLss, + state, + h, + shardsInnerSeries, + ) + s.Require().NoError(err) + s.Equal(cppbridge.RelabelerStats{1, 0, 0}, stats) + s.True(ok) +} + +func (s *RelabelerSuite) TestPerGoroutineRelabelerFromCacheFalseTransition() { + wr := prompb.WriteRequest{ + Timeseries: []prompb.TimeSeries{ + { + Labels: []prompb.Label{ + {Name: "__name__", Value: "value"}, + {Name: "job", Value: "abc"}, + {Name: "instance", Value: "value1"}, + }, + Samples: []prompb.Sample{ + {Value: 0.1, Timestamp: time.Now().UnixMilli()}, + }, + }, + { + Labels: []prompb.Label{ + {Name: "__name__", Value: "value"}, + {Name: "instance", Value: "value1"}, + }, + Samples: []prompb.Sample{ + {Value: 0.1, Timestamp: time.Now().UnixMilli()}, + }, + }, + }, + } + data, err := wr.Marshal() + s.Require().NoError(err) + + inputLss := cppbridge.NewLssStorage() + targetLss := cppbridge.NewQueryableLssStorage() + + var numberOfShards uint16 = 1 + + hlimits := cppbridge.DefaultWALHashdexLimits() + h, err := cppbridge.NewWALSnappyProtobufHashdex(snappy.Encode(nil, data), hlimits) + s.Require().NoError(err) + + shardsInnerSeries := cppbridge.NewShardsInnerSeries(numberOfShards) + state := cppbridge.NewTransitionStateV2() + state.Reconfigure(0, numberOfShards) + + pgr := cppbridge.NewPerGoroutineRelabeler(numberOfShards, 0) + stats, ok, err := pgr.RelabelingFromCache( + s.baseCtx, + inputLss, + targetLss, + state, + h, + shardsInnerSeries, + ) + s.Require().NoError(err) + s.Equal(cppbridge.RelabelerStats{0, 0, 0}, stats) + s.False(ok) +} + +func (s *RelabelerSuite) TestPerGoroutineRelabelerFromCachePartiallyTransition() { + ts := time.Now().UnixMilli() + wr1 := prompb.WriteRequest{ + Timeseries: []prompb.TimeSeries{ + { + Labels: []prompb.Label{ + {Name: "__name__", Value: "value0"}, + {Name: "job", Value: "abc"}, + {Name: "instance", Value: "value0"}, + }, + Samples: []prompb.Sample{ + {Value: 0.1, Timestamp: ts}, + }, + }, + { + Labels: []prompb.Label{ + {Name: "__name__", Value: "value1"}, + {Name: "job", Value: "abc"}, + {Name: "instance", Value: "value1"}, + }, + Samples: []prompb.Sample{ + {Value: 0.1, Timestamp: ts}, + }, + }, + { + Labels: []prompb.Label{ + {Name: "__name__", Value: "value2"}, + {Name: "job", Value: "abc"}, + {Name: "instance", Value: "value2"}, + }, + Samples: []prompb.Sample{ + {Value: 0.1, Timestamp: ts}, + }, + }, + { + Labels: []prompb.Label{ + {Name: "__name__", Value: "value3"}, + {Name: "job", Value: "abc"}, + {Name: "instance", Value: "value3"}, + }, + Samples: []prompb.Sample{ + {Value: 0.1, Timestamp: ts}, + }, + }, + { + Labels: []prompb.Label{ + {Name: "__name__", Value: "value3"}, + {Name: "instance", Value: "value3"}, + }, + Samples: []prompb.Sample{ + {Value: 0.1, Timestamp: ts}, + }, + }, + { + Labels: []prompb.Label{ + {Name: "__name__", Value: "value4"}, + {Name: "instance", Value: "value4"}, + }, + Samples: []prompb.Sample{ + {Value: 0.1, Timestamp: ts}, + }, + }, + { + Labels: []prompb.Label{ + {Name: "__name__", Value: "value5"}, + {Name: "instance", Value: "value5"}, + }, + Samples: []prompb.Sample{ + {Value: 0.1, Timestamp: ts}, + }, + }, + { + Labels: []prompb.Label{ + {Name: "__name__", Value: "value6"}, + {Name: "instance", Value: "value6"}, + }, + Samples: []prompb.Sample{ + {Value: 0.1, Timestamp: ts}, + }, + }, + }, + } + data1, err := wr1.Marshal() + s.Require().NoError(err) + hlimits := cppbridge.DefaultWALHashdexLimits() + h1, err := cppbridge.NewWALSnappyProtobufHashdex(snappy.Encode(nil, data1), hlimits) + s.Require().NoError(err) + + ts += 6000 + wr2 := prompb.WriteRequest{ + Timeseries: []prompb.TimeSeries{ + { + Labels: []prompb.Label{ + {Name: "__name__", Value: "value0"}, + {Name: "job", Value: "abc"}, + {Name: "instance", Value: "value0"}, + }, + Samples: []prompb.Sample{ + {Value: 0.1, Timestamp: ts}, + }, + }, + { + Labels: []prompb.Label{ + {Name: "__name__", Value: "value1"}, + {Name: "job", Value: "abc"}, + {Name: "instance", Value: "value1"}, + }, + Samples: []prompb.Sample{ + {Value: 0.1, Timestamp: ts}, + }, + }, + { + Labels: []prompb.Label{ + {Name: "__name__", Value: "value2"}, + {Name: "job", Value: "abc"}, + {Name: "instance", Value: "value2"}, + }, + Samples: []prompb.Sample{ + {Value: 0.1, Timestamp: ts}, + }, + }, + { + Labels: []prompb.Label{ + {Name: "__name__", Value: "value3"}, + {Name: "job", Value: "abc"}, + {Name: "instance", Value: "value3"}, + }, + Samples: []prompb.Sample{ + {Value: 0.1, Timestamp: ts}, + }, + }, + { + Labels: []prompb.Label{ + {Name: "__name__", Value: "value4"}, + {Name: "job", Value: "abc"}, + {Name: "instance", Value: "value4"}, + }, + Samples: []prompb.Sample{ + {Value: 0.1, Timestamp: ts}, + }, + }, + }, + } + data2, err := wr2.Marshal() + s.Require().NoError(err) + h2, err := cppbridge.NewWALSnappyProtobufHashdex(snappy.Encode(nil, data2), hlimits) + s.Require().NoError(err) + + inputLss := cppbridge.NewLssStorage() + targetLss := cppbridge.NewQueryableLssStorage() + + var numberOfShards uint16 = 1 + + shardsInnerSeries := cppbridge.NewShardsInnerSeries(numberOfShards) + shardsRelabeledSeries := cppbridge.NewShardsRelabeledSeries(numberOfShards) + + state := cppbridge.NewTransitionStateV2() + state.Reconfigure(0, numberOfShards) + + pgr := cppbridge.NewPerGoroutineRelabeler(numberOfShards, 0) + stats, hasReallocations, err := pgr.Relabeling( + s.baseCtx, + inputLss, + targetLss, + state, + h1, + shardsInnerSeries, + shardsRelabeledSeries, + ) + s.Require().NoError(err) + s.Equal(cppbridge.RelabelerStats{8, 8, 0}, stats) + s.True(hasReallocations) + + shardsInnerSeries = cppbridge.NewShardsInnerSeries(numberOfShards) + stats, ok, err := pgr.RelabelingFromCache( s.baseCtx, inputLss, targetLss, @@ -999,7 +1439,7 @@ func (s *RelabelerSuite) TestInputPerGoroutineRelabelerFromCachePartially() { s.False(ok) s.Equal(uint64(4), shardsInnerSeries[0].Size()) - stats, _, err = pgr.InputRelabeling( + stats, _, err = pgr.Relabeling( s.baseCtx, inputLss, targetLss, diff --git a/pp/go/storage/querier/querier.go b/pp/go/storage/querier/querier.go index cf637ebb28..ea3f3073a3 100644 --- a/pp/go/storage/querier/querier.go +++ b/pp/go/storage/querier/querier.go @@ -171,7 +171,7 @@ func (q *Querier[TTask, TDataStorage, TLSS, TShard, THead]) selectInstant( defer release() defer func() { - if q.metrics == nil { + if q.metrics != nil { q.metrics.SelectDuration.With( prometheus.Labels{"query_type": "instant"}, ).Observe(float64(time.Since(start).Microseconds())) diff --git a/pp/prometheus/relabeler.h b/pp/prometheus/relabeler.h index aa4ef12b00..5792dec99d 100644 --- a/pp/prometheus/relabeler.h +++ b/pp/prometheus/relabeler.h @@ -230,13 +230,6 @@ class StaleNaNsState { // drop old, store new.. prev_target_bitset_ = std::move(target_bitset_); } - - PROMPP_ALWAYS_INLINE void reset() { - input_bitset_ = roaring::Roaring{}; - target_bitset_ = roaring::Roaring{}; - prev_input_bitset_ = roaring::Roaring{}; - prev_target_bitset_ = roaring::Roaring{}; - } }; // Cache stateless cache for relabeler. @@ -934,6 +927,16 @@ class PerGoroutineRelabeler { return samples_count; } + // check_target_lss check label_set in target lss. + template + PROMPP_ALWAYS_INLINE Cache::CheckResult check_target_lss(const TargetLSS& target_lss, LabelSet& label_set, size_t hash) { + if (std::optional ls_id = target_lss.find(label_set, hash); ls_id.has_value()) { + return {.status = Cache::CheckResult::Status::kKeep, .ls_id = ls_id.value()}; + } + + return {}; + } + // inject_target_labels add labels from target to builder. template PROMPP_ALWAYS_INLINE bool inject_target_labels(LabelsBuilder& target_builder, const RelabelerOptions& options) { @@ -1338,6 +1341,111 @@ class PerGoroutineRelabeler { return input_relabeling_from_cache_internal(input_lss, target_lss, cache, hashdex, options, stats, shards_inner_series, state, def_timestamp); } + // input_transition_relabeling transparent relabeling. + template + PROMPP_ALWAYS_INLINE void input_transition_relabeling(TargetLSS& target_lss, + const Hashdex& hashdex, + Stats& stats, + PromPP::Primitives::Go::SliceView& shards_inner_series) { + assert(number_of_shards_ > 0); + + size_t n = std::min(static_cast(hashdex.size()), static_cast((hashdex.size() * 1.1) / number_of_shards_)); + for (auto i = 0; i < number_of_shards_; ++i) { + if (shards_inner_series[i]->size() >= n) { + continue; + } + + shards_inner_series[i]->reserve(n); + } + + PromPP::Primitives::LabelsBuilder builder{builder_state_}; + + for (auto it = skip_shard_inner_series(hashdex, shards_inner_series[shard_id_]->size()); it != hashdex.end(); ++it) { + if ((it->hash() % number_of_shards_) != shard_id_) { + continue; + } + + timeseries_buf_.clear(); + it->read(timeseries_buf_); + + Cache::CheckResult check_result = check_target_lss(target_lss, timeseries_buf_.label_set(), it->hash()); + switch (check_result.status) { + case Cache::CheckResult::kNotFound: { + builder.reset(timeseries_buf_.label_set()); + auto ls_id = target_lss.find_or_emplace(timeseries_buf_.label_set(), it->hash()); + auto& samples = timeseries_buf_.samples(); + for (const PromPP::Primitives::Sample& sample : samples) { + shards_inner_series[shard_id_]->emplace_back(sample, ls_id); + } + + ++stats.series_added; + + break; + } + + case Cache::CheckResult::kKeep: { + auto& samples = timeseries_buf_.samples(); + for (const PromPP::Primitives::Sample& sample : samples) { + shards_inner_series[shard_id_]->emplace_back(sample, check_result.ls_id); + } + + break; + } + + default: + continue; + } + + stats.samples_added += static_cast(timeseries_buf_.samples().size()); + } + } + + // input_transition_relabeling_from_cache transparent relabeling with only reading from the lss. + template + PROMPP_ALWAYS_INLINE bool input_transition_relabeling_only_read(TargetLSS& target_lss, + const Hashdex& hashdex, + Stats& stats, + PromPP::Primitives::Go::SliceView& shards_inner_series) { + assert(number_of_shards_ > 0); + + size_t n = std::min(static_cast(hashdex.size()), static_cast((hashdex.size() * 1.1) / number_of_shards_)); + for (auto i = 0; i < number_of_shards_; ++i) { + shards_inner_series[i]->reserve(n); + } + + for (const auto& item : hashdex) { + if ((item.hash() % number_of_shards_) != shard_id_) { + continue; + } + + timeseries_buf_.clear(); + item.read(timeseries_buf_); + + Cache::CheckResult check_result = check_target_lss(target_lss, timeseries_buf_.label_set(), item.hash()); + switch (check_result.status) { + case Cache::CheckResult::kNotFound: { + return false; + }; + + case Cache::CheckResult::kKeep: { + auto& samples = timeseries_buf_.samples(); + for (const PromPP::Primitives::Sample& sample : samples) { + shards_inner_series[shard_id_]->emplace_back(sample, check_result.ls_id); + } + + break; + } + + default: + continue; + } + + stats.samples_added += static_cast(timeseries_buf_.samples().size()); + } + + return true; + } + // second stage // append_relabeler_series add relabeled ls to lss, add to result and add to cache update. template diff --git a/web/api/v1/api.go b/web/api/v1/api.go index 9b521cdb5c..837d0e9249 100644 --- a/web/api/v1/api.go +++ b/web/api/v1/api.go @@ -194,7 +194,7 @@ type API struct { QueryEngine promql.QueryEngine ExemplarQueryable storage.ExemplarQueryable - receiver handler.Receiver // PP_CHANGES.md: rebuild on cpp + adapter handler.Adapter // PP_CHANGES.md: rebuild on cpp opHandler *handler.PPHandler // PP_CHANGES.md: rebuild on cpp scrapePoolsRetriever func(context.Context) ScrapePoolsRetriever @@ -232,7 +232,7 @@ func NewAPI( ap storage.Appendable, eq storage.ExemplarQueryable, - receiver handler.Receiver, // PP_CHANGES.md: rebuild on cpp + adapter handler.Adapter, // PP_CHANGES.md: rebuild on cpp spsr func(context.Context) ScrapePoolsRetriever, tr func(context.Context) TargetRetriever, @@ -265,7 +265,7 @@ func NewAPI( Queryable: q, ExemplarQueryable: eq, - receiver: receiver, // PP_CHANGES.md: rebuild on cpp + adapter: adapter, // PP_CHANGES.md: rebuild on cpp scrapePoolsRetriever: spsr, targetRetriever: tr, @@ -302,10 +302,10 @@ func NewAPI( } if rwEnabled { - a.opHandler = handler.NewPPHandler(dbDir, receiver, logger, registerer) // PP_CHANGES.md: rebuild on cpp + a.opHandler = handler.NewPPHandler(dbDir, adapter, logger, registerer) // PP_CHANGES.md: rebuild on cpp } if otlpEnabled { - a.otlpWriteHandler = handler.NewOTLPWriteHandler(logger, receiver) // PP_CHANGES.md: rebuild on cpp + a.otlpWriteHandler = handler.NewOTLPWriteHandler(logger, adapter) // PP_CHANGES.md: rebuild on cpp } return a diff --git a/web/api/v1/pp_api.go b/web/api/v1/pp_api.go index ac03b638b6..b6623d9c86 100644 --- a/web/api/v1/pp_api.go +++ b/web/api/v1/pp_api.go @@ -3,11 +3,13 @@ package v1 import ( "context" "errors" + "fmt" "net/http" "strconv" "github.com/prometheus/common/route" + "github.com/prometheus/prometheus/config" "github.com/prometheus/prometheus/model/labels" "github.com/prometheus/prometheus/pp-pkg/handler/middleware" "github.com/prometheus/prometheus/promql" @@ -15,6 +17,17 @@ import ( "github.com/prometheus/prometheus/tsdb/chunkenc" ) +// ApplyConfig updates the configs for opHandler and otlpWriteHandler. +func (api *API) ApplyConfig(conf *config.Config) (err error) { + if api.opHandler != nil { + if err := api.opHandler.ApplyConfig(conf); err != nil { + return fmt.Errorf("failed apply opHandler config: %w", err) + } + } + + return err +} + // Register the API's endpoints in the given router from op. func (api *API) opRegister(r *route.Router, wrapAgent func(f apiFunc) http.HandlerFunc) { r.Get("/query_head", wrapAgent(api.queryHead)) @@ -70,7 +83,7 @@ func (api *API) queryHead(r *http.Request) apiFuncResult { matchers = append(matchers, selector...) } - q, err := api.receiver.HeadQuerier(ctx, start.UnixMilli(), end.UnixMilli()) + q, err := api.adapter.HeadQuerier(ctx, start.UnixMilli(), end.UnixMilli()) if err != nil { return apiFuncResult{nil, &apiError{errorBadData, err}, nil, nil} } @@ -109,7 +122,7 @@ func (api *API) serveHeadStatus(r *http.Request) apiFuncResult { } } - return apiFuncResult{api.receiver.HeadStatus(r.Context(), limit), nil, nil, nil} + return apiFuncResult{api.adapter.HeadStatus(r.Context(), limit), nil, nil, nil} } func (api *API) opRemoteWrite(middlewares ...middleware.Middleware) http.HandlerFunc { diff --git a/web/web.go b/web/web.go index 59e62e6f65..90c80b84cf 100644 --- a/web/web.go +++ b/web/web.go @@ -225,7 +225,7 @@ func (h *Handler) ApplyConfig(conf *config.Config) error { h.config = conf - return nil + return h.apiV1.ApplyConfig(conf) // PP_CHANGES.md: rebuild on cpp } // Options for the web Handler. @@ -273,7 +273,7 @@ type Options struct { } // New initializes a new web Handler. -func New(logger log.Logger, o *Options, receiver handler.Receiver) *Handler { // PP_CHANGES.md: rebuild on cpp +func New(logger log.Logger, o *Options, adapter handler.Adapter) *Handler { // PP_CHANGES.md: rebuild on cpp if logger == nil { logger = log.NewNopLogger() } @@ -328,7 +328,15 @@ func New(logger log.Logger, o *Options, receiver handler.Receiver) *Handler { // } // PP_CHANGES.md: rebuild on cpp start - h.apiV1 = api_v1.NewAPI(h.queryEngine, h.storage, app, h.exemplarStorage, receiver, factorySPr, factoryTr, factoryAr, + h.apiV1 = api_v1.NewAPI( + h.queryEngine, + h.storage, + app, + h.exemplarStorage, + adapter, + factorySPr, + factoryTr, + factoryAr, func() config.Config { h.mtx.RLock() defer h.mtx.RUnlock() From e82d6a09d07ffa7d5dcae46351953c0b8dd7adbc Mon Sep 17 00:00:00 2001 From: Alexandr Yudin Date: Fri, 19 Sep 2025 07:35:55 +0000 Subject: [PATCH 39/96] some fix --- pp-pkg/handler/processor/interface.go | 2 + pp-pkg/handler/states_test.go | 159 ++++++++++++++++++++++++++ 2 files changed, 161 insertions(+) create mode 100644 pp-pkg/handler/states_test.go diff --git a/pp-pkg/handler/processor/interface.go b/pp-pkg/handler/processor/interface.go index c719a93f22..9ad00ff660 100644 --- a/pp-pkg/handler/processor/interface.go +++ b/pp-pkg/handler/processor/interface.go @@ -1,5 +1,7 @@ package processor +//go:generate -command moq go run github.com/matryer/moq -out processor_moq_test.go -pkg processor_test -rm . Adapter StatesStorage RemoteWrite MetricStream Refill + import ( "context" diff --git a/pp-pkg/handler/states_test.go b/pp-pkg/handler/states_test.go new file mode 100644 index 0000000000..977355d202 --- /dev/null +++ b/pp-pkg/handler/states_test.go @@ -0,0 +1,159 @@ +package handler_test + +import ( + "testing" + + "github.com/prometheus/prometheus/config" + pp_pkg_config "github.com/prometheus/prometheus/pp-pkg/config" + "github.com/prometheus/prometheus/pp-pkg/handler" + "github.com/prometheus/prometheus/pp/go/cppbridge" + rconfig "github.com/prometheus/prometheus/pp/go/relabeler/config" + "github.com/stretchr/testify/suite" +) + +type StatesStorageSuite struct { + suite.Suite +} + +func TestStatesStorageSuite(t *testing.T) { + suite.Run(t, new(StatesStorageSuite)) +} + +func (s *StatesStorageSuite) TestHappyPath() { + states := handler.NewStatesStorage() + + state, ok := states.GetStateByID(config.TransparentRelabeler) + + s.Require().True(ok) + s.Require().NotNil(state) + s.Require().True(state.IsTransition()) +} + +func (s *StatesStorageSuite) TestNotExist() { + states := handler.NewStatesStorage() + + state, ok := states.GetStateByID("test") + s.Require().False(ok) + s.Require().Nil(state) +} + +func (s *StatesStorageSuite) TestApplyConfigEmpty() { + states := handler.NewStatesStorage() + cfg := &config.Config{ + RemoteWriteConfigs: []*config.PPRemoteWriteConfig{}, + } + + states.ApplyConfig(cfg) + + state, ok := states.GetStateByID(config.TransparentRelabeler) + s.Require().True(ok) + s.Require().NotNil(state) + s.Require().True(state.IsTransition()) +} + +func (s *StatesStorageSuite) TestApplyConfig() { + states := handler.NewStatesStorage() + cfg := &config.Config{ + ReceiverConfig: pp_pkg_config.RemoteWriteReceiverConfig{ + Configs: []*rconfig.InputRelabelerConfig{ + { + Name: "test", + RelabelConfigs: []*cppbridge.RelabelConfig{}, + }, + }, + }, + } + + states.ApplyConfig(cfg) + + state, ok := states.GetStateByID(config.TransparentRelabeler) + s.Require().True(ok) + s.Require().NotNil(state) + s.Require().True(state.IsTransition()) + + state, ok = states.GetStateByID("test") + s.Require().True(ok) + s.Require().NotNil(state) + s.Require().False(state.IsTransition()) +} + +func (s *StatesStorageSuite) TestApplyConfigDouble() { + states := handler.NewStatesStorage() + cfg := &config.Config{ + ReceiverConfig: pp_pkg_config.RemoteWriteReceiverConfig{ + Configs: []*rconfig.InputRelabelerConfig{ + { + Name: "test", + RelabelConfigs: []*cppbridge.RelabelConfig{}, + }, + }, + }, + } + + states.ApplyConfig(cfg) + + state, ok := states.GetStateByID(config.TransparentRelabeler) + s.Require().True(ok) + s.Require().NotNil(state) + s.Require().True(state.IsTransition()) + + state, ok = states.GetStateByID("test") + s.Require().True(ok) + s.Require().NotNil(state) + s.Require().False(state.IsTransition()) + + states.ApplyConfig(cfg) + + state, ok = states.GetStateByID("test") + s.Require().True(ok) + s.Require().NotNil(state) + s.Require().False(state.IsTransition()) +} + +func (s *StatesStorageSuite) TestApplyConfigDoubleChange() { + states := handler.NewStatesStorage() + cfg := &config.Config{ + ReceiverConfig: pp_pkg_config.RemoteWriteReceiverConfig{ + Configs: []*rconfig.InputRelabelerConfig{ + { + Name: "test", + RelabelConfigs: []*cppbridge.RelabelConfig{}, + }, + }, + }, + } + + states.ApplyConfig(cfg) + + state, ok := states.GetStateByID(config.TransparentRelabeler) + s.Require().True(ok) + s.Require().NotNil(state) + s.Require().True(state.IsTransition()) + + state, ok = states.GetStateByID("test") + s.Require().True(ok) + s.Require().NotNil(state) + s.Require().False(state.IsTransition()) + + cfg = &config.Config{ + ReceiverConfig: pp_pkg_config.RemoteWriteReceiverConfig{ + Configs: []*rconfig.InputRelabelerConfig{ + { + Name: "test2", + RelabelConfigs: []*cppbridge.RelabelConfig{}, + }, + }, + }, + } + + states.ApplyConfig(cfg) + + state, ok = states.GetStateByID("test") + s.Require().False(ok) + s.Require().Nil(state) + + state, ok = states.GetStateByID("test2") + s.Require().True(ok) + s.Require().NotNil(state) + s.Require().False(state.IsTransition()) +} From 407c001471d0d8339f0e81a727f7cbb8a953016a Mon Sep 17 00:00:00 2001 From: Alexandr Yudin Date: Mon, 22 Sep 2025 07:54:21 +0000 Subject: [PATCH 40/96] add test --- pp/entrypoint/prometheus_relabeler.cpp | 28 -- pp/entrypoint/prometheus_relabeler.h | 16 - pp/go/cppbridge/entrypoint.go | 26 - pp/go/cppbridge/entrypoint.h | 16 - pp/go/cppbridge/prometheus_relabeler.go | 22 - pp/go/storage/head/container/weighted_test.go | 26 + pp/go/storage/head/head/head.go | 14 - pp/go/storage/head/manager/manager.go | 476 ------------------ .../head/shard/wal/reader/byte_reader.go | 12 +- .../head/shard/wal/reader/byte_reader_test.go | 37 +- pp/go/storage/head/shard/wal/reader/header.go | 2 +- .../head/shard/wal/reader/header_test.go | 24 + .../storage/head/shard/wal/reader/segment.go | 2 +- .../head/shard/wal/writer/header_test.go | 23 + .../head/shard/wal/writer/notifier_test.go | 26 + .../storage/head/shard/wal/writer/segment.go | 2 +- .../head/shard/wal/writer/segment_test.go | 61 +++ pp/go/storage/mediator/mediator.go | 1 - pp/go/storage/mediator/mediator_moq_test.go | 106 ++++ pp/go/storage/mediator/mediator_test.go | 188 +++++++ pp/go/storage/remotewriter/datasource.go | 6 +- pp/go/storage/remotewriter/decoder.go | 39 +- pp/go/storage/remotewriter/remotewriter.go | 53 +- pp/go/storage/remotewriter/walreader.go | 3 +- pp/go/storage/remotewriter/writeloop.go | 8 +- pp/go/storage/remotewriter/writer.go | 7 +- pp/prometheus/relabeler.h | 8 - 27 files changed, 568 insertions(+), 664 deletions(-) delete mode 100644 pp/go/storage/head/manager/manager.go create mode 100644 pp/go/storage/head/shard/wal/reader/header_test.go create mode 100644 pp/go/storage/head/shard/wal/writer/header_test.go create mode 100644 pp/go/storage/head/shard/wal/writer/notifier_test.go create mode 100644 pp/go/storage/head/shard/wal/writer/segment_test.go create mode 100644 pp/go/storage/mediator/mediator_moq_test.go create mode 100644 pp/go/storage/mediator/mediator_test.go diff --git a/pp/entrypoint/prometheus_relabeler.cpp b/pp/entrypoint/prometheus_relabeler.cpp index e671c843e1..0184fa35e5 100644 --- a/pp/entrypoint/prometheus_relabeler.cpp +++ b/pp/entrypoint/prometheus_relabeler.cpp @@ -851,31 +851,3 @@ extern "C" void prompp_prometheus_per_goroutine_relabeler_append_relabeler_serie entrypoint::handle_current_exception(err_stream); } } - -extern "C" void prompp_prometheus_per_goroutine_relabeler_update_relabeler_state(void* args, void* res) { - struct Arguments { - PromPP::Primitives::Go::SliceView shards_relabeler_state_update; - PerGoroutineRelabelerPtr per_goroutine_relabeler; - CachePtr cache; - uint16_t relabeled_shard_id; - }; - struct Result { - PromPP::Primitives::Go::Slice error; - }; - - const auto* in = static_cast(args); - - try { - for (size_t id = 0; id != in->shards_relabeler_state_update.size(); ++id) { - if (in->shards_relabeler_state_update[id] == nullptr || in->shards_relabeler_state_update[id]->size() == 0) { - continue; - } - - in->per_goroutine_relabeler->update_relabeler_state(*in->cache, in->shards_relabeler_state_update[id], id); - } - } catch (...) { - auto* out = new (res) Result(); - auto err_stream = PromPP::Primitives::Go::BytesStream(&out->error); - entrypoint::handle_current_exception(err_stream); - } -} diff --git a/pp/entrypoint/prometheus_relabeler.h b/pp/entrypoint/prometheus_relabeler.h index 9cf824cad2..9c0a26b7aa 100644 --- a/pp/entrypoint/prometheus_relabeler.h +++ b/pp/entrypoint/prometheus_relabeler.h @@ -589,22 +589,6 @@ void prompp_prometheus_per_goroutine_relabeler_input_transition_relabeling_only_ */ void prompp_prometheus_per_goroutine_relabeler_append_relabeler_series(void* args, void* res); -/** - * @brief add to cache relabled data(third stage). - * - * @param args { - * shards_relabeler_state_update []*RelabelerStateUpdate // pointer to RelabelerStateUpdate per source shard; - * per_shard_relabeler uintptr // pointer to constructed per goroutine relabeler; - * cache uintptr // pointer to constructed Cache; - * relabeled_shard_id uint16 // relabeled shard id; - * } - * - * @param res { - * error []byte // error string if thrown; - * } - */ -void prompp_prometheus_per_goroutine_relabeler_update_relabeler_state(void* args, void* res); - #ifdef __cplusplus } // extern "C" #endif diff --git a/pp/go/cppbridge/entrypoint.go b/pp/go/cppbridge/entrypoint.go index b6a9826b0f..e3f5888f28 100644 --- a/pp/go/cppbridge/entrypoint.go +++ b/pp/go/cppbridge/entrypoint.go @@ -3562,29 +3562,3 @@ func prometheusPerGoroutineRelabelerAppendRelabelerSeries( return res.exception, res.targetLssHasReallocations } - -// prometheusPerGoroutineRelabelerUpdateRelabelerState wrapper for add to cache relabled data(third stage). -func prometheusPerGoroutineRelabelerUpdateRelabelerState( - shardsRelabelerStateUpdate []*RelabelerStateUpdate, - perGoroutineRelabeler, cache uintptr, -) []byte { - args := struct { - relabelerStateUpdates []*RelabelerStateUpdate - perGoroutineRelabeler uintptr - cache uintptr - }{shardsRelabelerStateUpdate, perGoroutineRelabeler, cache} - var res struct { - exception []byte - } - start := time.Now().UnixNano() - testGC() - fastcgo.UnsafeCall2( - C.prompp_prometheus_per_goroutine_relabeler_update_relabeler_state, - uintptr(unsafe.Pointer(&args)), - uintptr(unsafe.Pointer(&res)), - ) - inputRelabelerUpdateRelabelerStateSum.Add(float64(time.Now().UnixNano() - start)) - inputRelabelerUpdateRelabelerStateCount.Inc() - - return res.exception -} diff --git a/pp/go/cppbridge/entrypoint.h b/pp/go/cppbridge/entrypoint.h index 5e21d0a0b7..188c5a8b10 100755 --- a/pp/go/cppbridge/entrypoint.h +++ b/pp/go/cppbridge/entrypoint.h @@ -1269,22 +1269,6 @@ void prompp_prometheus_per_goroutine_relabeler_input_transition_relabeling_only_ */ void prompp_prometheus_per_goroutine_relabeler_append_relabeler_series(void* args, void* res); -/** - * @brief add to cache relabled data(third stage). - * - * @param args { - * shards_relabeler_state_update []*RelabelerStateUpdate // pointer to RelabelerStateUpdate per source shard; - * per_shard_relabeler uintptr // pointer to constructed per goroutine relabeler; - * cache uintptr // pointer to constructed Cache; - * relabeled_shard_id uint16 // relabeled shard id; - * } - * - * @param res { - * error []byte // error string if thrown; - * } - */ -void prompp_prometheus_per_goroutine_relabeler_update_relabeler_state(void* args, void* res); - #ifdef __cplusplus } // extern "C" #endif diff --git a/pp/go/cppbridge/prometheus_relabeler.go b/pp/go/cppbridge/prometheus_relabeler.go index b0ed6dd00d..4b5e5fb551 100644 --- a/pp/go/cppbridge/prometheus_relabeler.go +++ b/pp/go/cppbridge/prometheus_relabeler.go @@ -1283,28 +1283,6 @@ func (pgr *PerGoroutineRelabeler) RelabelingFromCache( ) } -// UpdateRelabelerState add to cache relabled data(third stage). -func (pgr *PerGoroutineRelabeler) UpdateRelabelerState( - ctx context.Context, - cache *Cache, - shardsRelabelerStateUpdate []*RelabelerStateUpdate, -) error { - if ctx.Err() != nil { - return ctx.Err() - } - - exception := prometheusPerGoroutineRelabelerUpdateRelabelerState( - shardsRelabelerStateUpdate, - pgr.cptr, - cache.cPointer, - ) - - runtime.KeepAlive(pgr) - runtime.KeepAlive(cache) - - return handleException(exception) -} - // inputRelabeling relabeling incoming hashdex(first stage). func (pgr *PerGoroutineRelabeler) inputRelabeling( inputLss *LabelSetStorage, diff --git a/pp/go/storage/head/container/weighted_test.go b/pp/go/storage/head/container/weighted_test.go index 87b879fa84..7ced3aa0fe 100644 --- a/pp/go/storage/head/container/weighted_test.go +++ b/pp/go/storage/head/container/weighted_test.go @@ -9,6 +9,7 @@ import ( "github.com/stretchr/testify/suite" "github.com/prometheus/prometheus/pp/go/storage/head/container" + "github.com/prometheus/prometheus/pp/go/util/locker" ) type WeightedSuite struct { @@ -96,6 +97,31 @@ func (s *WeightedSuite) TestWithError() { s.Error(err) } +func (s *WeightedSuite) TestClose() { + baseCtx := context.Background() + expectedHead := &testHead{c: 2} + c := container.NewWeighted(expectedHead) + + err := c.Close() + s.Require().NoError(err) + + actualHead := c.Get() + s.Require().NotNil(actualHead) + s.Equal(expectedHead.c, actualHead.c) + + err = c.Replace(baseCtx, &testHead{c: 3}) + s.Require().ErrorIs(err, locker.ErrSemaphoreClosed) + + err = c.With(baseCtx, func(h *testHead) error { + if expectedHead.c != h.c { + return fmt.Errorf("expectedHead(%d) not equal actual(%d)", expectedHead.c, h.c) + } + + return nil + }) + s.Require().ErrorIs(err, locker.ErrSemaphoreClosed) +} + // // testHead // diff --git a/pp/go/storage/head/head/head.go b/pp/go/storage/head/head/head.go index ef3d871fb9..b4a9acff14 100644 --- a/pp/go/storage/head/head/head.go +++ b/pp/go/storage/head/head/head.go @@ -21,20 +21,6 @@ var ExtraWorkers = 0 // defaultNumberOfWorkers default number of workers. const defaultNumberOfWorkers = 2 -// const ( -// // DSAllocatedMemory name of task. -// DSAllocatedMemory = "data_storage_allocated_memory" - -// // DSHeadStatus name of task. -// DSHeadStatus = "data_storage_head_status" - -// // LSSAllocatedMemory name of task. -// LSSAllocatedMemory = "lss_allocated_memory" - -// // LSSHeadStatus name of task. -// LSSHeadStatus = "lss_head_status" -// ) - // // Shard // diff --git a/pp/go/storage/head/manager/manager.go b/pp/go/storage/head/manager/manager.go deleted file mode 100644 index a1eae1f46b..0000000000 --- a/pp/go/storage/head/manager/manager.go +++ /dev/null @@ -1,476 +0,0 @@ -package manager - -// import ( -// "context" -// "errors" -// "fmt" -// "time" - -// "github.com/prometheus/client_golang/prometheus" - -// "github.com/prometheus/prometheus/pp/go/cppbridge" -// "github.com/prometheus/prometheus/pp/go/storage/catalog" -// "github.com/prometheus/prometheus/pp/go/storage/logger" -// "github.com/prometheus/prometheus/pp/go/util" -// ) - -// const ( -// // DSMergeOutOfOrderChunks name of task. -// DSMergeOutOfOrderChunks = "data_storage_merge_out_of_order_chunks" - -// // WalCommit name of task. -// WalCommit = "wal_commit" -// ) - -// // -// // Timer -// // - -// // Timer implementation timer. -// type Timer interface { -// Chan() <-chan time.Time -// Reset() -// Stop() -// } - -// // -// // GenericTask -// // - -// // GenericTask the minimum required task [Generic] implementation. -// type GenericTask interface { -// // Wait for the task to complete on all shards. -// Wait() error -// } - -// // -// // DataStorage -// // - -// // DataStorage the minimum required [DataStorage] implementation. -// type DataStorage interface { -// // MergeOutOfOrderChunks merge chunks with out of order data chunks. -// MergeOutOfOrderChunks() -// } - -// // -// // LSS -// // - -// // LSS the minimum required [LSS] implementation. -// type LSS interface { -// // WithRLock calls fn on raws [cppbridge.LabelSetStorage] with read lock. -// WithRLock(fn func(target, input *cppbridge.LabelSetStorage) error) error -// } - -// // -// // Wal -// // - -// // Wal the minimum required Wal implementation for a [Shard]. -// type Wal interface { -// // Commit finalize segment from encoder and add to wal. -// // It is necessary to lock the LSS for reading for the commit. -// Commit() error - -// // Flush wal segment writer, write all buffered data to storage. -// Flush() error -// } - -// // -// // Shard -// // - -// // Shard the minimum required head [Shard] implementation. -// type Shard[TDataStorage DataStorage, TLSS LSS, TWal Wal] interface { -// // DataStorage returns shard [DataStorage]. -// DataStorage() TDataStorage - -// // LSS returns shard labelset storage [LSS]. -// LSS() TLSS - -// // ShardID returns the shard ID. -// ShardID() uint16 - -// // Wal returns write-ahead log. -// Wal() TWal -// } - -// // -// // Head -// // - -// // Head the minimum required [Head] implementation. -// type Head[ -// TGenericTask GenericTask, -// TDataStorage DataStorage, -// TLSS LSS, -// TWal Wal, -// TShard, TGShard Shard[TDataStorage, TLSS, TWal], -// ] interface { -// // Close closes wals, query semaphore for the inability to get query and clear metrics. -// Close() error - -// // CreateTask create a task for operations on the [Head] shards. -// CreateTask(taskName string, shardFn func(shard TGShard) error) TGenericTask - -// // Enqueue the task to be executed on shards [Head]. -// Enqueue(t TGenericTask) - -// // Generation returns current generation of [Head]. -// Generation() uint64 - -// // NumberOfShards returns current number of shards in to [Head]. -// NumberOfShards() uint16 - -// // RangeShards returns an iterator over the [Head] [Shard]s, through which the shard can be directly accessed. -// RangeShards() func(func(TShard) bool) - -// // SetReadOnly sets the read-only flag for the [Head]. -// SetReadOnly() -// } - -// // -// // ActiveHeadContainer -// // - -// // ActiveHeadContainer container for active [Head], the minimum required [ActiveHeadContainer] implementation. -// type ActiveHeadContainer[ -// TGenericTask GenericTask, -// TDataStorage DataStorage, -// TLSS LSS, -// TWal Wal, -// TShard, TGShard Shard[TDataStorage, TLSS, TWal], -// THead Head[TGenericTask, TDataStorage, TLSS, TWal, TShard, TGShard], -// ] interface { -// // Close closes [ActiveHeadContainer] for the inability work with [Head]. -// Close() error - -// // Get the active head [Head]. -// Get() THead - -// // Replace the active head [Head] with a new head. -// Replace(ctx context.Context, newHead THead) error - -// // With calls fn(h Head). -// With(ctx context.Context, fn func(h THead) error) error -// } - -// // -// // Keeper -// // - -// type Keeper[ -// TGenericTask GenericTask, -// TDataStorage DataStorage, -// TLSS LSS, -// TWal Wal, -// TShard, TGShard Shard[TDataStorage, TLSS, TWal], -// THead Head[TGenericTask, TDataStorage, TLSS, TWal, TShard, TGShard], -// ] interface { -// Add(head THead) -// RangeQueriableHeads(mint, maxt int64) func(func(THead) bool) -// } - -// // -// // Loader -// // - -// // Loader loads [Head] from wal, the minimum required [Loader] implementation. -// type Loader[ -// TGenericTask GenericTask, -// TDataStorage DataStorage, -// TLSS LSS, -// TWal Wal, -// TShard, TGShard Shard[TDataStorage, TLSS, TWal], -// THead Head[TGenericTask, TDataStorage, TLSS, TWal, TShard, TGShard], -// ] interface { -// // UploadHead upload [THead] from wal by head ID. -// UploadHead(headRecord *catalog.Record, generation uint64) (head THead, corrupted bool) -// } - -// // HeadBuilder building new [Head] with parameters, the minimum required [HeadBuilder] implementation. -// type HeadBuilder[ -// TGenericTask GenericTask, -// TDataStorage DataStorage, -// TLSS LSS, -// TWal Wal, -// TShard, TGShard Shard[TDataStorage, TLSS, TWal], -// THead Head[TGenericTask, TDataStorage, TLSS, TWal, TShard, TGShard], -// ] interface { -// // Build new [Head]. -// Build(generation uint64, numberOfShards uint16) (THead, error) -// } - -// type Manager[ -// TGenericTask GenericTask, -// TDataStorage DataStorage, -// TLSS LSS, -// TWal Wal, -// TShard, TGShard Shard[TDataStorage, TLSS, TWal], -// THead Head[TGenericTask, TDataStorage, TLSS, TWal, TShard, TGShard], -// ] struct { -// activeHead ActiveHeadContainer[TGenericTask, TDataStorage, TLSS, TWal, TShard, TGShard, THead] -// keeper Keeper[TGenericTask, TDataStorage, TLSS, TWal, TShard, TGShard, THead] -// headBuilder HeadBuilder[TGenericTask, TDataStorage, TLSS, TWal, TShard, TGShard, THead] -// headLoader Loader[TGenericTask, TDataStorage, TLSS, TWal, TShard, TGShard, THead] -// rotateTimer Timer -// commitTimer Timer -// mergeTimer Timer - -// numberOfShards uint16 - -// // TODO closer vs shutdowner -// closer *util.Closer -// shutdowner *util.GracefulShutdowner - -// rotateCounter prometheus.Counter -// counter *prometheus.CounterVec -// } - -// // NewManager init new [Manager] of [Head]s. -// func NewManager[ -// TGenericTask GenericTask, -// TDataStorage DataStorage, -// TLSS LSS, -// TWal Wal, -// TShard, TGShard Shard[TDataStorage, TLSS, TWal], -// THead Head[TGenericTask, TDataStorage, TLSS, TWal, TShard, TGShard], -// ]( -// activeHead ActiveHeadContainer[TGenericTask, TDataStorage, TLSS, TWal, TShard, TGShard, THead], -// headBuilder HeadBuilder[TGenericTask, TDataStorage, TLSS, TWal, TShard, TGShard, THead], -// headLoader Loader[TGenericTask, TDataStorage, TLSS, TWal, TShard, TGShard, THead], -// numberOfShards uint16, -// registerer prometheus.Registerer, -// ) *Manager[TGenericTask, TDataStorage, TLSS, TWal, TShard, TGShard, THead] { -// factory := util.NewUnconflictRegisterer(registerer) -// return &Manager[TGenericTask, TDataStorage, TLSS, TWal, TShard, TGShard, THead]{ -// activeHead: activeHead, -// headBuilder: headBuilder, -// headLoader: headLoader, - -// numberOfShards: numberOfShards, - -// counter: factory.NewCounterVec( -// prometheus.CounterOpts{ -// Name: "prompp_head_event_count", -// Help: "Number of head events", -// }, -// []string{"type"}, -// ), -// } -// } - -// // ApplyConfig update config. -// func (m *Manager[TGenericTask, TDataStorage, TLSS, TWal, TShard, TGShard, THead]) ApplyConfig( -// ctx context.Context, -// numberOfShards uint16, -// ) error { -// logger.Infof("reconfiguration start") -// defer logger.Infof("reconfiguration completed") - -// m.numberOfShards = numberOfShards - -// h := m.activeHead.Get() -// if h.NumberOfShards() == numberOfShards { -// return nil -// } - -// return m.rotate(ctx) -// } - -// // MergeOutOfOrderChunks merge chunks with out of order data chunks. -// func (m *Manager[TGenericTask, TDataStorage, TLSS, TWal, TShard, TGShard, THead]) MergeOutOfOrderChunks( -// ctx context.Context, -// ) error { -// return m.activeHead.With(ctx, func(h THead) error { -// mergeOutOfOrderChunksWithHead(h) - -// return nil -// }) -// } - -// // Run starts processing of the [Manager]. -// // TODO implementation. -// func (m *Manager[TGenericTask, TDataStorage, TLSS, TWal, TShard, TGShard, THead]) Run(ctx context.Context) error { -// go m.loop(ctx) -// return nil -// } - -// // Shutdown safe shutdown [Manager]. -// func (m *Manager[TGenericTask, TDataStorage, TLSS, TWal, TShard, TGShard, THead]) Shutdown(ctx context.Context) error { -// // TODO -// // cgogcErr := rr.cgogc.Shutdown(ctx) -// // err := rr.shutdowner.Shutdown(ctx) -// activeHeadErr := m.activeHead.Close() - -// h := m.activeHead.Get() -// commitErr := commitAndFlushViaRange(h) - -// headCloseErr := h.Close() - -// return errors.Join(activeHeadErr, commitErr, headCloseErr) -// } - -// // commitToWal commit and flush the accumulated data into the wal. -// func (m *Manager[TGenericTask, TDataStorage, TLSS, TWal, TShard, TGShard, THead]) commitToWal( -// ctx context.Context, -// ) error { -// return m.activeHead.With(ctx, func(h THead) error { -// t := h.CreateTask( -// WalCommit, -// func(shard TGShard) error { -// swal := shard.Wal() - -// // wal contains LSS and it is necessary to lock the LSS for reading for the commit. -// if err := shard.LSS().WithRLock(func(_, _ *cppbridge.LabelSetStorage) error { -// return swal.Commit() -// }); err != nil { -// return err -// } - -// return swal.Flush() -// }, -// ) -// h.Enqueue(t) - -// return t.Wait() -// }) -// } - -// // TODO implementation. -// func (m *Manager[TGenericTask, TDataStorage, TLSS, TWal, TShard, TGShard, THead]) loop(ctx context.Context) { -// defer m.closer.Done() - -// for { -// select { -// case <-m.closer.Signal(): -// return - -// case <-m.commitTimer.Chan(): -// if err := m.commitToWal(ctx); err != nil { -// logger.Errorf("wal commit failed: %v", err) -// } -// m.commitTimer.Reset() - -// case <-m.mergeTimer.Chan(): -// if err := m.MergeOutOfOrderChunks(ctx); err != nil { -// logger.Errorf("merge out of order chunks failed: %v", err) -// } -// m.mergeTimer.Reset() - -// case <-m.rotateTimer.Chan(): -// logger.Debugf("start rotation") - -// if err := m.rotate(ctx); err != nil { -// logger.Errorf("rotation failed: %v", err) -// } -// m.rotateCounter.Inc() - -// m.rotateTimer.Reset() -// m.commitTimer.Reset() -// m.mergeTimer.Reset() -// } -// } -// } - -// func (m *Manager[TGenericTask, TDataStorage, TLSS, TWal, TShard, TGShard, THead]) rotate(ctx context.Context) error { -// oldHead := m.activeHead.Get() - -// newHead, err := m.headBuilder.Build(oldHead.Generation()+1, m.numberOfShards) -// if err != nil { -// return fmt.Errorf("failed to build a new head: %w", err) -// } - -// // TODO CopySeriesFrom only old nunber of shards == new -// // newHead.CopySeriesFrom(oldHead) - -// m.keeper.Add(oldHead) - -// // TODO if replace error? -// err = m.activeHead.Replace(ctx, newHead) -// if err != nil { -// return fmt.Errorf("failed to replace old to new head: %w", err) -// } - -// mergeOutOfOrderChunksWithHead(oldHead) - -// if err := commitAndFlushViaRange(oldHead); err != nil { -// logger.Warnf("failed commit and flush to wal: %s", err) -// } - -// oldHead.SetReadOnly() - -// return nil -// } - -// // WithAppendableHead -// // TODO implementation. -// func (m *Manager[TGenericTask, TDataStorage, TLSS, TWal, TShard, TGShard, THead]) WithAppendableHead( -// ctx context.Context, -// fn func(h THead) error, -// ) error { -// return m.activeHead.With(ctx, fn) -// } - -// // RangeQueriableHeads -// // TODO implementation. -// func (m *Manager[TGenericTask, TDataStorage, TLSS, TWal, TShard, TGShard, THead]) RangeQueriableHeads( -// mint, maxt int64, -// ) func(func(THead) bool) { -// // ahead := m.activeHead.Get() -// // for h := range m.keeper.RangeQueriableHeads(mint, maxt) { -// // TODO -// // if h == ahead { -// // continue -// // } -// // } - -// return nil -// } - -// // mergeOutOfOrderChunksWithHead merge chunks with out of order data chunks for [Head]. -// func mergeOutOfOrderChunksWithHead[ -// TGenericTask GenericTask, -// TDataStorage DataStorage, -// TLSS LSS, -// TWal Wal, -// TShard, TGShard Shard[TDataStorage, TLSS, TWal], -// THead Head[TGenericTask, TDataStorage, TLSS, TWal, TShard, TGShard], -// ](h THead) { -// t := h.CreateTask( -// DSMergeOutOfOrderChunks, -// func(shard TGShard) error { -// shard.DataStorage().MergeOutOfOrderChunks() - -// return nil -// }, -// ) -// h.Enqueue(t) - -// _ = t.Wait() -// } - -// // commitAndFlushViaRange finalize segment from encoder and add to wal -// // and flush wal segment writer, write all buffered data to storage. -// func commitAndFlushViaRange[ -// TGenericTask GenericTask, -// TDataStorage DataStorage, -// TLSS LSS, -// TWal Wal, -// TShard, TGShard Shard[TDataStorage, TLSS, TWal], -// THead Head[TGenericTask, TDataStorage, TLSS, TWal, TShard, TGShard], -// ](h THead) error { -// errs := make([]error, 0, h.NumberOfShards()*2) -// for shard := range h.RangeShards() { -// if err := shard.Wal().Commit(); err != nil { -// errs = append(errs, fmt.Errorf("commit shard id %d: %w", shard.ShardID(), err)) -// } - -// if err := shard.Wal().Flush(); err != nil { -// errs = append(errs, fmt.Errorf("flush shard id %d: %w", shard.ShardID(), err)) -// } -// } - -// return errors.Join(errs...) -// } diff --git a/pp/go/storage/head/shard/wal/reader/byte_reader.go b/pp/go/storage/head/shard/wal/reader/byte_reader.go index 54fe2354e5..e7fc00ef31 100644 --- a/pp/go/storage/head/shard/wal/reader/byte_reader.go +++ b/pp/go/storage/head/shard/wal/reader/byte_reader.go @@ -2,23 +2,23 @@ package reader import "io" -// byteReader reads from the reader 1 byte at a time. -type byteReader struct { +// ByteReader reads from the reader 1 byte at a time. +type ByteReader struct { r io.Reader buf []byte n int } -// newByteReader init new [byteReader] -func newByteReader(r io.Reader) *byteReader { - return &byteReader{ +// NewByteReader init new [byteReader] +func NewByteReader(r io.Reader) *ByteReader { + return &ByteReader{ r: r, buf: make([]byte, 1), } } // ReadByte reads from the reader 1 byte. -func (r *byteReader) ReadByte() (byte, error) { +func (r *ByteReader) ReadByte() (byte, error) { n, err := io.ReadFull(r.r, r.buf) if err != nil { return 0, err diff --git a/pp/go/storage/head/shard/wal/reader/byte_reader_test.go b/pp/go/storage/head/shard/wal/reader/byte_reader_test.go index ab133840bd..da7c4de11e 100644 --- a/pp/go/storage/head/shard/wal/reader/byte_reader_test.go +++ b/pp/go/storage/head/shard/wal/reader/byte_reader_test.go @@ -1,32 +1,43 @@ -package reader +package reader_test import ( "bytes" + "io" "testing" + + "github.com/stretchr/testify/require" + + "github.com/prometheus/prometheus/pp/go/storage/head/shard/wal/reader" ) -func TestXxx(t *testing.T) { +func TestRead(t *testing.T) { bb := &bytes.Buffer{} - br := newByteReader(bb) + br := reader.NewByteReader(bb) + data := []byte{1, 42, 3, 0} - bb.Write([]byte{1, 2, 3, 0}) + _, err := bb.Write(data) + require.NoError(t, err) + + for _, expectedV := range data { + actualV, errRead := br.ReadByte() + require.NoError(t, errRead) + require.Equal(t, expectedV, actualV) + } - t.Log(br.ReadByte()) - t.Log(br.ReadByte()) - t.Log(br.ReadByte()) - t.Log(br.ReadByte()) + _, err = br.ReadByte() + require.ErrorIs(t, err, io.EOF) } func BenchmarkBR1(b *testing.B) { bb := &bytes.Buffer{} - br := newByteReader(bb) + br := reader.NewByteReader(bb) buf := []byte{1, 2, 3} for i := 0; i < b.N; i++ { - bb.Write(buf) - br.ReadByte() - br.ReadByte() - br.ReadByte() + _, _ = bb.Write(buf) + _, _ = br.ReadByte() + _, _ = br.ReadByte() + _, _ = br.ReadByte() } } diff --git a/pp/go/storage/head/shard/wal/reader/header.go b/pp/go/storage/head/shard/wal/reader/header.go index 2097f1b4e4..f5b2be57c2 100644 --- a/pp/go/storage/head/shard/wal/reader/header.go +++ b/pp/go/storage/head/shard/wal/reader/header.go @@ -10,7 +10,7 @@ import ( // //revive:disable-next-line:function-result-limit there is no point in packing it into a structure. func ReadHeader(reader io.Reader) (fileFormatVersion, encoderVersion uint8, n int, err error) { - br := newByteReader(reader) + br := NewByteReader(reader) fileFormatVersionU64, err := binary.ReadUvarint(br) if err != nil { return 0, 0, n, fmt.Errorf("failed to read file format version: %w", err) diff --git a/pp/go/storage/head/shard/wal/reader/header_test.go b/pp/go/storage/head/shard/wal/reader/header_test.go new file mode 100644 index 0000000000..0cc3066d0c --- /dev/null +++ b/pp/go/storage/head/shard/wal/reader/header_test.go @@ -0,0 +1,24 @@ +package reader_test + +import ( + "bytes" + "testing" + + "github.com/stretchr/testify/require" + + "github.com/prometheus/prometheus/pp/go/storage/head/shard/wal/reader" +) + +func TestWriteHeader(t *testing.T) { + buf := &bytes.Buffer{} + expected := []byte{21, 42} + + _, err := buf.Write(expected) + require.NoError(t, err) + + fileFormatVersion, encoderVersion, _, err := reader.ReadHeader(buf) + require.NoError(t, err) + + require.Equal(t, expected[0], fileFormatVersion) + require.Equal(t, expected[1], encoderVersion) +} diff --git a/pp/go/storage/head/shard/wal/reader/segment.go b/pp/go/storage/head/shard/wal/reader/segment.go index 7c2f35d000..b5eb5ddb07 100644 --- a/pp/go/storage/head/shard/wal/reader/segment.go +++ b/pp/go/storage/head/shard/wal/reader/segment.go @@ -56,7 +56,7 @@ func (s *Segment) resize(size int) { // ReadSegment read and decode [Segment] from [io.Reader] and returns. func ReadSegment(reader io.Reader, segment *Segment) (int64, error) { - br := newByteReader(reader) + br := NewByteReader(reader) size, err := binary.ReadUvarint(br) if err != nil { return int64(br.n), fmt.Errorf("failed to read segment size: %w", err) diff --git a/pp/go/storage/head/shard/wal/writer/header_test.go b/pp/go/storage/head/shard/wal/writer/header_test.go new file mode 100644 index 0000000000..e829c3af72 --- /dev/null +++ b/pp/go/storage/head/shard/wal/writer/header_test.go @@ -0,0 +1,23 @@ +package writer_test + +import ( + "bytes" + "testing" + + "github.com/stretchr/testify/require" + + "github.com/prometheus/prometheus/pp/go/storage/head/shard/wal/writer" +) + +func TestWriteHeader(t *testing.T) { + buf := &bytes.Buffer{} + fileFormatVersion := uint8(21) + encoderVersion := uint8(42) + expected := []byte{fileFormatVersion, encoderVersion} + + n, err := writer.WriteHeader(buf, fileFormatVersion, encoderVersion) + require.NoError(t, err) + + require.Equal(t, len(expected), n) + require.Equal(t, expected, buf.Bytes()) +} diff --git a/pp/go/storage/head/shard/wal/writer/notifier_test.go b/pp/go/storage/head/shard/wal/writer/notifier_test.go new file mode 100644 index 0000000000..9b288df51d --- /dev/null +++ b/pp/go/storage/head/shard/wal/writer/notifier_test.go @@ -0,0 +1,26 @@ +package writer_test + +import ( + "testing" + + "github.com/prometheus/prometheus/pp/go/storage/head/shard/wal/writer" + "github.com/stretchr/testify/suite" +) + +type SegmentWriteNotifierSuite struct { + suite.Suite +} + +func TestSegmentWriteNotifierSuite(t *testing.T) { + suite.Run(t, new(SegmentWriteNotifierSuite)) +} + +func (s *SegmentWriteNotifierSuite) TestHappyPath() { + // + + numberOfShards := uint16(2) + swn := writer.NewSegmentWriteNotifier(numberOfShards, func(segmentID uint32) { s.T().Log(segmentID) }) + + swn.NotifySegmentIsWritten(1) + swn.NotifySegmentIsWritten(0) +} diff --git a/pp/go/storage/head/shard/wal/writer/segment.go b/pp/go/storage/head/shard/wal/writer/segment.go index ce3b3b6626..a396f8537d 100644 --- a/pp/go/storage/head/shard/wal/writer/segment.go +++ b/pp/go/storage/head/shard/wal/writer/segment.go @@ -8,9 +8,9 @@ import ( // EncodedSegment the minimum required Segment implementation for a [WriteSegment]. type EncodedSegment interface { - Size() int64 CRC32() uint32 Samples() uint32 + Size() int64 io.WriterTo } diff --git a/pp/go/storage/head/shard/wal/writer/segment_test.go b/pp/go/storage/head/shard/wal/writer/segment_test.go new file mode 100644 index 0000000000..a679761fa4 --- /dev/null +++ b/pp/go/storage/head/shard/wal/writer/segment_test.go @@ -0,0 +1,61 @@ +package writer_test + +import ( + "bytes" + "io" + "testing" + + "github.com/stretchr/testify/require" + + "github.com/prometheus/prometheus/pp/go/storage/head/shard/wal/writer" +) + +func TestWriteSegment(t *testing.T) { + data := []byte{1, 2, 3, 2, 1, 0, 42} + segment := &testSegment{ + size: int64(len(data)), + samples: 42, + data: data, + } + buf := &bytes.Buffer{} + expected := []byte{byte(len(data)), byte(segment.crc32), byte(segment.samples)} + expected = append(expected, data...) + + _, err := writer.WriteSegment(buf, segment) + require.NoError(t, err) + + require.Equal(t, expected, buf.Bytes()) +} + +// +// testSegment +// + +// testSegment implementation [writer.EncodedSegment]. +type testSegment struct { + size int64 + samples uint32 + crc32 uint32 + data []byte +} + +// CRC32 implementation [writer.EncodedSegment]. +func (s *testSegment) CRC32() uint32 { + return s.crc32 +} + +// Samples implementation [writer.EncodedSegment]. +func (s *testSegment) Samples() uint32 { + return s.samples +} + +// Size implementation [writer.EncodedSegment]. +func (s *testSegment) Size() int64 { + return s.size +} + +// WriteTo implementation [writer.EncodedSegment]. +func (s *testSegment) WriteTo(w io.Writer) (int64, error) { + n, err := w.Write(s.data) + return int64(n), err +} diff --git a/pp/go/storage/mediator/mediator.go b/pp/go/storage/mediator/mediator.go index 010aa4272e..7d12912946 100644 --- a/pp/go/storage/mediator/mediator.go +++ b/pp/go/storage/mediator/mediator.go @@ -14,7 +14,6 @@ import ( // Timer implementation timer. type Timer interface { Chan() <-chan time.Time - Reset() Stop() } diff --git a/pp/go/storage/mediator/mediator_moq_test.go b/pp/go/storage/mediator/mediator_moq_test.go new file mode 100644 index 0000000000..bc3b50e51b --- /dev/null +++ b/pp/go/storage/mediator/mediator_moq_test.go @@ -0,0 +1,106 @@ +// Code generated by moq; DO NOT EDIT. +// github.com/matryer/moq + +package mediator_test + +import ( + "github.com/prometheus/prometheus/pp/go/storage/mediator" + "sync" + "time" +) + +// Ensure, that TimerMock does implement mediator.Timer. +// If this is not the case, regenerate this file with moq. +var _ mediator.Timer = &TimerMock{} + +// TimerMock is a mock implementation of mediator.Timer. +// +// func TestSomethingThatUsesTimer(t *testing.T) { +// +// // make and configure a mocked mediator.Timer +// mockedTimer := &TimerMock{ +// ChanFunc: func() <-chan time.Time { +// panic("mock out the Chan method") +// }, +// StopFunc: func() { +// panic("mock out the Stop method") +// }, +// } +// +// // use mockedTimer in code that requires mediator.Timer +// // and then make assertions. +// +// } +type TimerMock struct { + // ChanFunc mocks the Chan method. + ChanFunc func() <-chan time.Time + + // StopFunc mocks the Stop method. + StopFunc func() + + // calls tracks calls to the methods. + calls struct { + // Chan holds details about calls to the Chan method. + Chan []struct { + } + // Stop holds details about calls to the Stop method. + Stop []struct { + } + } + lockChan sync.RWMutex + lockStop sync.RWMutex +} + +// Chan calls ChanFunc. +func (mock *TimerMock) Chan() <-chan time.Time { + if mock.ChanFunc == nil { + panic("TimerMock.ChanFunc: method is nil but Timer.Chan was just called") + } + callInfo := struct { + }{} + mock.lockChan.Lock() + mock.calls.Chan = append(mock.calls.Chan, callInfo) + mock.lockChan.Unlock() + return mock.ChanFunc() +} + +// ChanCalls gets all the calls that were made to Chan. +// Check the length with: +// +// len(mockedTimer.ChanCalls()) +func (mock *TimerMock) ChanCalls() []struct { +} { + var calls []struct { + } + mock.lockChan.RLock() + calls = mock.calls.Chan + mock.lockChan.RUnlock() + return calls +} + +// Stop calls StopFunc. +func (mock *TimerMock) Stop() { + if mock.StopFunc == nil { + panic("TimerMock.StopFunc: method is nil but Timer.Stop was just called") + } + callInfo := struct { + }{} + mock.lockStop.Lock() + mock.calls.Stop = append(mock.calls.Stop, callInfo) + mock.lockStop.Unlock() + mock.StopFunc() +} + +// StopCalls gets all the calls that were made to Stop. +// Check the length with: +// +// len(mockedTimer.StopCalls()) +func (mock *TimerMock) StopCalls() []struct { +} { + var calls []struct { + } + mock.lockStop.RLock() + calls = mock.calls.Stop + mock.lockStop.RUnlock() + return calls +} diff --git a/pp/go/storage/mediator/mediator_test.go b/pp/go/storage/mediator/mediator_test.go new file mode 100644 index 0000000000..ef7eb86755 --- /dev/null +++ b/pp/go/storage/mediator/mediator_test.go @@ -0,0 +1,188 @@ +package mediator_test + +import ( + "context" + "sync" + "testing" + "time" + + "github.com/prometheus/prometheus/pp/go/storage/mediator" + "github.com/stretchr/testify/suite" +) + +type MediatorSuite struct { + suite.Suite +} + +func TestMediatorSuite(t *testing.T) { + suite.Run(t, new(MediatorSuite)) +} + +func (s *MediatorSuite) TestC() { + chTimer := make(chan time.Time, 1) + + timer := &TimerMock{ + ChanFunc: func() <-chan time.Time { + return chTimer + }, + StopFunc: func() {}, + } + + m := mediator.NewMediator(timer) + defer m.Close() + + ctx, cancel := context.WithTimeout(context.Background(), 5*time.Second) + counter := 0 + done := make(chan struct{}) + start := sync.WaitGroup{} + start.Add(1) + go func() { + start.Done() + select { + case <-m.C(): + counter++ + close(done) + case <-ctx.Done(): + } + }() + + start.Wait() + s.T().Log("timer tick") + chTimer <- time.Time{} + + select { + case <-done: + case <-ctx.Done(): + } + cancel() + + s.Equal(1, counter) +} + +func (s *MediatorSuite) TestClose() { + chTimer := make(chan time.Time, 1) + stopCounter := 0 + + timer := &TimerMock{ + ChanFunc: func() <-chan time.Time { + return chTimer + }, + StopFunc: func() { stopCounter++ }, + } + + m := mediator.NewMediator(timer) + + ctx, cancel := context.WithTimeout(context.Background(), 5*time.Second) + counter := 0 + done := make(chan struct{}) + start := sync.WaitGroup{} + start.Add(1) + go func() { + start.Done() + + for range m.C() { + counter++ + break + } + + close(done) + }() + + start.Wait() + s.T().Log("mediator close") + m.Close() + + select { + case <-done: + case <-ctx.Done(): + m.Trigger() + } + cancel() + + <-done + + s.Equal(0, counter) + s.Equal(1, stopCounter) +} + +func (s *MediatorSuite) TestTrigger() { + chTimer := make(chan time.Time, 1) + + timer := &TimerMock{ + ChanFunc: func() <-chan time.Time { + return chTimer + }, + StopFunc: func() {}, + } + + m := mediator.NewMediator(timer) + defer m.Close() + + ctx, cancel := context.WithTimeout(context.Background(), 5*time.Second) + counter := 0 + done := make(chan struct{}) + start := sync.WaitGroup{} + start.Add(1) + go func() { + start.Done() + select { + case <-m.C(): + counter++ + close(done) + case <-ctx.Done(): + } + }() + + start.Wait() + s.T().Log("trigger") + m.Trigger() + + select { + case <-done: + case <-ctx.Done(): + } + cancel() + + s.Equal(1, counter) +} + +func (s *MediatorSuite) TestTriggerSkip() { + chTimer := make(chan time.Time, 1) + + timer := &TimerMock{ + ChanFunc: func() <-chan time.Time { + return chTimer + }, + StopFunc: func() {}, + } + + m := mediator.NewMediator(timer) + defer m.Close() + + ctx, cancel := context.WithTimeout(context.Background(), 1*time.Second) + counter := 0 + done := make(chan struct{}) + start := sync.WaitGroup{} + start.Add(1) + go func() { + start.Wait() + select { + case <-m.C(): + counter++ + close(done) + case <-ctx.Done(): + } + }() + + s.T().Log("trigger") + m.Trigger() + start.Done() + + select { + case <-done: + case <-ctx.Done(): + } + cancel() + + s.Equal(0, counter) +} diff --git a/pp/go/storage/remotewriter/datasource.go b/pp/go/storage/remotewriter/datasource.go index f9575f674a..a2e00b9f95 100644 --- a/pp/go/storage/remotewriter/datasource.go +++ b/pp/go/storage/remotewriter/datasource.go @@ -228,9 +228,11 @@ func newSegmentReadyChecker(headRecord *catalog.Record) *segmentReadyChecker { return &segmentReadyChecker{headRecord: headRecord} } -func (src *segmentReadyChecker) SegmentIsReady(segmentID uint32) (ready bool, outOfRange bool) { +func (src *segmentReadyChecker) SegmentIsReady(segmentID uint32) (ready, outOfRange bool) { ready = src.headRecord.LastAppendedSegmentID() != nil && *src.headRecord.LastAppendedSegmentID() >= segmentID - outOfRange = (src.headRecord.Status() != catalog.StatusNew && src.headRecord.Status() != catalog.StatusActive) && !ready + outOfRange = (src.headRecord.Status() != catalog.StatusNew && + src.headRecord.Status() != catalog.StatusActive) && + !ready return ready, outOfRange } diff --git a/pp/go/storage/remotewriter/decoder.go b/pp/go/storage/remotewriter/decoder.go index a889f678b7..fed79236ee 100644 --- a/pp/go/storage/remotewriter/decoder.go +++ b/pp/go/storage/remotewriter/decoder.go @@ -8,12 +8,30 @@ import ( "github.com/prometheus/prometheus/pp/go/cppbridge" ) +// +// DecodedSegment +// + +// DecodedSegment the segment decoded from the file [Wal]. +type DecodedSegment struct { + ID uint32 + Samples *cppbridge.DecodedRefSamples + MaxTimestamp int64 + OutdatedSamplesCount uint32 + DroppedSamplesCount uint32 + AddSeriesCount uint32 + DroppedSeriesCount uint32 +} + +// Decoder decodes and relabeling series in segments from a file [Wal]. +// Saves its state in the file for recovery upon restart. type Decoder struct { relabeler *cppbridge.StatelessRelabeler lss *cppbridge.LabelSetStorage outputDecoder *cppbridge.WALOutputDecoder } +// NewDecoder init new [Decoder]. func NewDecoder( externalLabels labels.Labels, relabelConfigs []*cppbridge.RelabelConfig, @@ -26,7 +44,13 @@ func NewDecoder( } lss := cppbridge.NewLssStorage() - outputDecoder := cppbridge.NewWALOutputDecoder(LabelsToCppBridgeLabels(externalLabels), relabeler, lss, shardID, encoderVersion) + outputDecoder := cppbridge.NewWALOutputDecoder( + LabelsToCppBridgeLabels(externalLabels), + relabeler, + lss, + shardID, + encoderVersion, + ) return &Decoder{ relabeler: relabeler, @@ -35,16 +59,7 @@ func NewDecoder( }, nil } -type DecodedSegment struct { - ID uint32 - Samples *cppbridge.DecodedRefSamples - MaxTimestamp int64 - OutdatedSamplesCount uint32 - DroppedSamplesCount uint32 - AddSeriesCount uint32 - DroppedSeriesCount uint32 -} - +// Decode and relabeling series in segments from a file [Wal]. func (d *Decoder) Decode(segment []byte, minTimestamp int64) (*DecodedSegment, error) { samples, stats, err := d.outputDecoder.Decode(segment, minTimestamp) if err != nil { @@ -60,6 +75,7 @@ func (d *Decoder) Decode(segment []byte, minTimestamp int64) (*DecodedSegment, e }, nil } +// LoadFrom loads the state from a file. func (d *Decoder) LoadFrom(reader io.Reader) error { state, err := io.ReadAll(reader) if err != nil { @@ -74,6 +90,7 @@ func (d *Decoder) WriteTo(writer io.Writer) (int64, error) { return d.outputDecoder.WriteTo(writer) } +// LabelsToCppBridgeLabels converts [labels.Labels] to slice [cppbridge.Label]. func LabelsToCppBridgeLabels(lbls labels.Labels) []cppbridge.Label { result := make([]cppbridge.Label, 0, lbls.Len()) lbls.Range(func(l labels.Label) { diff --git a/pp/go/storage/remotewriter/remotewriter.go b/pp/go/storage/remotewriter/remotewriter.go index a1e375ad53..e99489b4e4 100644 --- a/pp/go/storage/remotewriter/remotewriter.go +++ b/pp/go/storage/remotewriter/remotewriter.go @@ -12,11 +12,23 @@ import ( "github.com/prometheus/prometheus/pp/go/storage/ready" ) +// +// Catalog +// + +// Catalog of current head records. type Catalog interface { - List(filterFn func(record *catalog.Record) bool, sortLess func(lhs, rhs *catalog.Record) bool) (records []*catalog.Record, err error) + // List returns slice of records with filter and sort. + List( + filterFn func(record *catalog.Record) bool, + sortLess func(lhs, rhs *catalog.Record) bool, + ) ([]*catalog.Record, error) + + // SetCorrupted set corrupted flag for ID and returns [catalog.Record] if exist. SetCorrupted(id string) (*catalog.Record, error) } +// RemoteWriter sent samples to the remote write storage. type RemoteWriter struct { dataDir string configQueue chan []DestinationConfig @@ -26,10 +38,17 @@ type RemoteWriter struct { registerer prometheus.Registerer } -func New(dataDir string, catalog Catalog, clock clockwork.Clock, readyNotifier ready.Notifier, registerer prometheus.Registerer) *RemoteWriter { +// New init new [RemoteWriter]. +func New( + dataDir string, + hcatalog Catalog, + clock clockwork.Clock, + readyNotifier ready.Notifier, + registerer prometheus.Registerer, +) *RemoteWriter { return &RemoteWriter{ dataDir: dataDir, - catalog: catalog, + catalog: hcatalog, clock: clock, configQueue: make(chan []DestinationConfig), readyNotifier: readyNotifier, @@ -37,6 +56,21 @@ func New(dataDir string, catalog Catalog, clock clockwork.Clock, readyNotifier r } } +// ApplyConfig updates the state as the new config requires. +func (rw *RemoteWriter) ApplyConfig(configs ...DestinationConfig) (err error) { + select { + case rw.configQueue <- configs: + return nil + case <-time.After(time.Minute): + return fmt.Errorf("failed to apply remote write configs, timeout") + } +} + +// Run sending data via RemoteWriter. +// +//revive:disable-next-line:cyclomatic but readable +//revive:disable-next-line:function-length long but readable +//revive:disable-next-line:cognitive-complexity function is not complicated. func (rw *RemoteWriter) Run(ctx context.Context) error { writeLoopRunners := make(map[string]*writeLoopRunner) defer func() { @@ -53,8 +87,8 @@ func (rw *RemoteWriter) Run(ctx context.Context) error { return nil case configs := <-rw.configQueue: destinationConfigs := make(map[string]DestinationConfig) - for _, destinationConfig := range configs { - destinationConfigs[destinationConfig.Name] = destinationConfig + for i := range configs { + destinationConfigs[configs[i].Name] = configs[i] } for _, destination := range destinations { @@ -98,15 +132,6 @@ func (rw *RemoteWriter) Run(ctx context.Context) error { } } -func (rw *RemoteWriter) ApplyConfig(configs ...DestinationConfig) (err error) { - select { - case rw.configQueue <- configs: - return nil - case <-time.After(time.Minute): - return fmt.Errorf("failed to apply remote write configs, timeout") - } -} - type writeLoopRunner struct { wl *writeLoop stopc chan struct{} diff --git a/pp/go/storage/remotewriter/walreader.go b/pp/go/storage/remotewriter/walreader.go index f60098c841..86acf5f748 100644 --- a/pp/go/storage/remotewriter/walreader.go +++ b/pp/go/storage/remotewriter/walreader.go @@ -7,7 +7,6 @@ import ( "io" "os" - "github.com/prometheus/prometheus/pp/go/relabeler/head" "github.com/prometheus/prometheus/pp/go/storage/head/shard/wal/reader" ) @@ -23,7 +22,7 @@ func newWalReader(fileName string) (*walReader, uint8, error) { return nil, 0, fmt.Errorf("failed to read wal file: %w", err) } - _, encoderVersion, _, err := head.ReadHeader(file) + _, encoderVersion, _, err := reader.ReadHeader(file) if err != nil { return nil, 0, errors.Join(fmt.Errorf("failed to read header: %w", err), file.Close()) } diff --git a/pp/go/storage/remotewriter/writeloop.go b/pp/go/storage/remotewriter/writeloop.go index 6d80af45f2..a97fabfe04 100644 --- a/pp/go/storage/remotewriter/writeloop.go +++ b/pp/go/storage/remotewriter/writeloop.go @@ -27,11 +27,11 @@ type writeLoop struct { client remote.WriteClient } -func newWriteLoop(dataDir string, destination *Destination, catalog Catalog, clock clockwork.Clock) *writeLoop { +func newWriteLoop(dataDir string, destination *Destination, hcatalog Catalog, clock clockwork.Clock) *writeLoop { return &writeLoop{ dataDir: dataDir, destination: destination, - catalog: catalog, + catalog: hcatalog, clock: clock, } } @@ -389,6 +389,6 @@ func (rpw *readyProtobufWriter) SetProtobufWriter(protobufWriter ProtobufWriter) rpw.protobufWriter = protobufWriter } -func (rw *readyProtobufWriter) Write(ctx context.Context, protobuf *cppbridge.SnappyProtobufEncodedData) error { - return rw.protobufWriter.Write(ctx, protobuf) +func (rpw *readyProtobufWriter) Write(ctx context.Context, protobuf *cppbridge.SnappyProtobufEncodedData) error { + return rpw.protobufWriter.Write(ctx, protobuf) } diff --git a/pp/go/storage/remotewriter/writer.go b/pp/go/storage/remotewriter/writer.go index e47e8fa764..b5483d9e11 100644 --- a/pp/go/storage/remotewriter/writer.go +++ b/pp/go/storage/remotewriter/writer.go @@ -7,17 +7,19 @@ import ( "github.com/prometheus/prometheus/storage/remote" ) -// protobufWriter +// protobufWriter the wrapper over the [remote.WriteClient]. type protobufWriter struct { client remote.WriteClient } +// newProtobufWriter init new [protobufWriter]. func newProtobufWriter(client remote.WriteClient) *protobufWriter { return &protobufWriter{ client: client, } } +// Write [cppbridge.SnappyProtobufEncodedData] to [remote.WriteClient] func (w *protobufWriter) Write(ctx context.Context, protobuf *cppbridge.SnappyProtobufEncodedData) error { return protobuf.Do(func(buf []byte) error { if len(buf) == 0 { @@ -30,6 +32,7 @@ func (w *protobufWriter) Write(ctx context.Context, protobuf *cppbridge.SnappyPr }) } -func (w *protobufWriter) Close() error { +// Close implementation [io.Closer]. +func (*protobufWriter) Close() error { return nil } diff --git a/pp/prometheus/relabeler.h b/pp/prometheus/relabeler.h index 5792dec99d..09feecf9a1 100644 --- a/pp/prometheus/relabeler.h +++ b/pp/prometheus/relabeler.h @@ -1470,14 +1470,6 @@ class PerGoroutineRelabeler { } } - // third stage - // update_relabeler_state add to cache relabled data. - PROMPP_ALWAYS_INLINE void update_relabeler_state(Cache& cache, const RelabelerStateUpdate* relabeler_state_update, const uint16_t relabeled_shard_id) { - for (const auto& update : *relabeler_state_update) { - cache.add_relabel(update.incoming_ls_id, update.relabeled_ls_id, relabeled_shard_id); - } - } - // destructor. PROMPP_ALWAYS_INLINE ~PerGoroutineRelabeler() = default; }; From 552e885bfe849e75e59c2d3367e5e32f72ff9ed4 Mon Sep 17 00:00:00 2001 From: Alexandr Yudin Date: Tue, 23 Sep 2025 07:34:22 +0000 Subject: [PATCH 41/96] add test --- .../storage/head/shard/wal/writer/buffered.go | 13 +- .../head/shard/wal/writer/buffered_test.go | 285 +++++++- .../head/shard/wal/writer/notifier_test.go | 45 +- .../head/shard/wal/writer/segment_test.go | 57 +- .../head/shard/wal/writer/writer_moq_test.go | 676 ++++++++++++++++++ 5 files changed, 1009 insertions(+), 67 deletions(-) create mode 100644 pp/go/storage/head/shard/wal/writer/writer_moq_test.go diff --git a/pp/go/storage/head/shard/wal/writer/buffered.go b/pp/go/storage/head/shard/wal/writer/buffered.go index e578697db6..a11dbc8459 100644 --- a/pp/go/storage/head/shard/wal/writer/buffered.go +++ b/pp/go/storage/head/shard/wal/writer/buffered.go @@ -8,6 +8,9 @@ import ( "sync/atomic" ) +// FileInfo alias for [os.FileInfo]. +type FileInfo = os.FileInfo + // SegmentIsWrittenNotifier notify when new segment write. type SegmentIsWrittenNotifier interface { NotifySegmentIsWritten(shardID uint16) @@ -17,7 +20,7 @@ type SegmentIsWrittenNotifier interface { type WriteSyncCloser interface { io.WriteCloser Sync() error - Stat() (os.FileInfo, error) + Stat() (FileInfo, error) } // SegmentWriterFN encode to slice byte and write to [io.Writer]. @@ -88,7 +91,13 @@ func (w *Buffered[TSegment]) Flush() error { } } - w.segments = nil + if len(w.segments) != 0 && cap(w.segments) >= len(w.segments)*2 { //revive:disable-line:add-constant // x2 + w.segments = make([]TSegment, 0, len(w.segments)) + } else { + clear(w.segments) + w.segments = w.segments[:0] + } + return nil } diff --git a/pp/go/storage/head/shard/wal/writer/buffered_test.go b/pp/go/storage/head/shard/wal/writer/buffered_test.go index 8f373cc07a..ef18136208 100644 --- a/pp/go/storage/head/shard/wal/writer/buffered_test.go +++ b/pp/go/storage/head/shard/wal/writer/buffered_test.go @@ -1,42 +1,279 @@ package writer_test import ( - "fmt" + "bytes" + "errors" + "io" "os" - "path/filepath" "testing" + "github.com/go-faker/faker/v4" + "github.com/stretchr/testify/suite" + "github.com/prometheus/prometheus/pp/go/storage/head/shard/wal/writer" - "github.com/stretchr/testify/require" ) -func TestXxx(t *testing.T) { - // +type BufferedSuite struct { + suite.Suite +} + +func TestBufferedSuite(t *testing.T) { + suite.Run(t, new(BufferedSuite)) +} + +func (s *BufferedSuite) TestHappyPath() { + actual := &bytes.Buffer{} + sfile := s.openfile(actual) + + swn := &SegmentIsWrittenNotifierMock{NotifySegmentIsWrittenFunc: func(uint16) {}} + + shardID := uint16(0) + wrBuf, err := writer.NewBuffered(shardID, sfile, writer.WriteSegment[*EncodedSegmentMock], swn) + s.Require().NoError(err) + s.Equal(int64(0), wrBuf.CurrentSize()) + + segment, expectedSegment := s.genSegment() + + err = wrBuf.Write(segment) + s.Require().NoError(err) + s.Empty(sfile.WriteCalls()) + + err = wrBuf.Flush() + s.Require().NoError(err) + s.Equal(int64(len(expectedSegment)), wrBuf.CurrentSize()) + s.Equal(expectedSegment, actual.Bytes()) + + err = wrBuf.Sync() + s.Require().NoError(err) + s.Len(sfile.SyncCalls(), 1) + s.Len(swn.NotifySegmentIsWrittenCalls(), 1) + + err = wrBuf.Close() + s.Require().NoError(err) + s.Len(sfile.CloseCalls(), 1) +} + +func (s *BufferedSuite) TestBuffered() { + actual := &bytes.Buffer{} + sfile := s.openfile(actual) + + swn := &SegmentIsWrittenNotifierMock{NotifySegmentIsWrittenFunc: func(uint16) {}} + + shardID := uint16(0) + wrBuf, err := writer.NewBuffered(shardID, sfile, writer.WriteSegment[*EncodedSegmentMock], swn) + s.Require().NoError(err) + s.Equal(int64(0), wrBuf.CurrentSize()) + + expectedSegments := []byte{} + expectedSize := 0 + for range 10 { + segment, expectedSegment := s.genSegment() + err = wrBuf.Write(segment) + s.Require().NoError(err) + s.Empty(sfile.WriteCalls()) + expectedSegments = append(expectedSegments, expectedSegment...) + expectedSize += len(expectedSegment) + } + + err = wrBuf.Flush() + s.Require().NoError(err) + s.Equal(int64(expectedSize), wrBuf.CurrentSize()) + s.Equal(expectedSegments, actual.Bytes()) + s.Empty(swn.NotifySegmentIsWrittenCalls()) + + err = wrBuf.Sync() + s.Require().NoError(err) + s.Len(sfile.SyncCalls(), 1) + s.Len(swn.NotifySegmentIsWrittenCalls(), 1) + + err = wrBuf.Close() + s.Require().NoError(err) + s.Len(sfile.CloseCalls(), 1) +} + +func (s *BufferedSuite) TestStatError() { + actual := &bytes.Buffer{} + sfile := s.openfile(actual) + sfile.StatFunc = func() (os.FileInfo, error) { return nil, errors.New("some error") } + + swn := &SegmentIsWrittenNotifierMock{NotifySegmentIsWrittenFunc: func(uint16) {}} + + shardID := uint16(0) + wrBuf, err := writer.NewBuffered(shardID, sfile, writer.WriteSegment[*EncodedSegmentMock], swn) + s.Require().Error(err) + s.Require().Nil(wrBuf) +} + +func (s *BufferedSuite) TestSyncError() { + actual := &bytes.Buffer{} + sfile := s.openfile(actual) + sfile.SyncFunc = func() error { return errors.New("some error") } + + swn := &SegmentIsWrittenNotifierMock{NotifySegmentIsWrittenFunc: func(uint16) {}} + shardID := uint16(0) - tmpDir, err := os.MkdirTemp("", "shard") - require.NoError(t, err) - defer func() { - _ = os.RemoveAll(tmpDir) - }() + wrBuf, err := writer.NewBuffered(shardID, sfile, writer.WriteSegment[*EncodedSegmentMock], swn) + s.Require().NoError(err) + s.Equal(int64(0), wrBuf.CurrentSize()) + + segment, expectedSegment := s.genSegment() - shardFile, err := os.Create(filepath.Join(filepath.Clean(tmpDir), fmt.Sprintf("shard_%d.wal", shardID))) - require.NoError(t, err) + err = wrBuf.Write(segment) + s.Require().NoError(err) + s.Empty(sfile.WriteCalls()) - swn := &segmentWriteNotifier{} + err = wrBuf.Flush() + s.Require().NoError(err) + s.Equal(int64(len(expectedSegment)), wrBuf.CurrentSize()) + s.Equal(expectedSegment, actual.Bytes()) - defer func() { - if err == nil { - return + err = wrBuf.Sync() + s.Require().Error(err) + s.Len(sfile.SyncCalls(), 1) + s.Empty(swn.NotifySegmentIsWrittenCalls()) +} + +func (s *BufferedSuite) TestWriteToBufferWithError() { + actual := &bytes.Buffer{} + sfile := s.openfile(actual) + + swn := &SegmentIsWrittenNotifierMock{NotifySegmentIsWrittenFunc: func(uint16) {}} + + scount := 0 + writeSegment := func(w io.Writer, segment *EncodedSegmentMock) (n int, err error) { + if scount == 5 { + scount++ + return 0, errors.New("some error") } - _ = shardFile.Close() - }() - writer.NewBuffered(shardID, shardFile, writer.WriteSegment[writer.EncodedSegment], swn) + + scount++ + return writer.WriteSegment(w, segment) + } + + shardID := uint16(0) + wrBuf, err := writer.NewBuffered(shardID, sfile, writeSegment, swn) + s.Require().NoError(err) + s.Equal(int64(0), wrBuf.CurrentSize()) + + expectedSegments := []byte{} + expectedSize := 0 + for range 10 { + segment, expectedSegment := s.genSegment() + err = wrBuf.Write(segment) + s.Require().NoError(err) + s.Empty(sfile.WriteCalls()) + expectedSegments = append(expectedSegments, expectedSegment...) + expectedSize += len(expectedSegment) + } + + err = wrBuf.Flush() + s.Require().Error(err) + + err = wrBuf.Flush() + s.Require().NoError(err) + s.Equal(int64(expectedSize), wrBuf.CurrentSize()) + s.Equal(expectedSegments, actual.Bytes()) + s.Empty(swn.NotifySegmentIsWrittenCalls()) + + err = wrBuf.Sync() + s.Require().NoError(err) + s.Len(sfile.SyncCalls(), 1) + s.Len(swn.NotifySegmentIsWrittenCalls(), 1) + + err = wrBuf.Close() + s.Require().NoError(err) + s.Len(sfile.CloseCalls(), 1) } -// segmentWriteNotifier test implementation [writer.SegmentIsWrittenNotifier]. -type segmentWriteNotifier struct{} +func (s *BufferedSuite) TestFlushWithError() { + actual := &bytes.Buffer{} + sfile := s.openfile(actual) + + swn := &SegmentIsWrittenNotifierMock{NotifySegmentIsWrittenFunc: func(uint16) {}} + + scount := 0 + sfile.WriteFunc = func(p []byte) (int, error) { + if scount == 4 || scount == 5 { + scount++ + return 0, errors.New("some error") + } + + scount++ + + return actual.Write(p) + } + + shardID := uint16(0) + wrBuf, err := writer.NewBuffered(shardID, sfile, writer.WriteSegment[*EncodedSegmentMock], swn) + s.Require().NoError(err) + s.Equal(int64(0), wrBuf.CurrentSize()) + + expectedSegments := []byte{} + expectedSize := 0 + for range 10 { + segment, expectedSegment := s.genSegment() + err = wrBuf.Write(segment) + s.Require().NoError(err) + s.Empty(sfile.WriteCalls()) + expectedSegments = append(expectedSegments, expectedSegment...) + expectedSize += len(expectedSegment) + } + + err = wrBuf.Flush() + s.Require().Error(err) + + err = wrBuf.Flush() + s.Require().Error(err) + + err = wrBuf.Flush() + s.Require().NoError(err) + s.Equal(int64(expectedSize), wrBuf.CurrentSize()) + s.Equal(expectedSegments, actual.Bytes()) + s.Empty(swn.NotifySegmentIsWrittenCalls()) + + err = wrBuf.Sync() + s.Require().NoError(err) + s.Len(sfile.SyncCalls(), 1) + s.Len(swn.NotifySegmentIsWrittenCalls(), 1) + + err = wrBuf.Close() + s.Require().NoError(err) + s.Len(sfile.CloseCalls(), 1) +} + +func (*BufferedSuite) openfile(buf *bytes.Buffer) *WriteSyncCloserMock { + return &WriteSyncCloserMock{ + CloseFunc: func() error { return nil }, + StatFunc: func() (os.FileInfo, error) { return &FileInfoMock{SizeFunc: func() int64 { return 0 }}, nil }, + SyncFunc: func() error { return nil }, + WriteFunc: buf.Write, + } +} + +func (s *BufferedSuite) genSegment() (segment *EncodedSegmentMock, expected []byte) { + segmentCrc32 := uint32(0) + segmentSamples := uint32(42) + data := []byte(faker.Paragraph()) + + segment = &EncodedSegmentMock{ + CRC32Func: func() uint32 { + return segmentCrc32 + }, + SamplesFunc: func() uint32 { + return segmentSamples + }, + SizeFunc: func() int64 { + return int64(len(data)) + }, + WriteToFunc: func(w io.Writer) (int64, error) { + n, errWr := w.Write(data) + return int64(n), errWr + }, + } + + buf := &bytes.Buffer{} + _, err := writer.WriteSegment(buf, segment) + s.Require().NoError(err) -// NotifySegmentIsWritten test implementation [writer.SegmentIsWrittenNotifier]. -func (*segmentWriteNotifier) NotifySegmentIsWritten(shardID uint16) { - _ = shardID + return segment, buf.Bytes() } diff --git a/pp/go/storage/head/shard/wal/writer/notifier_test.go b/pp/go/storage/head/shard/wal/writer/notifier_test.go index 9b288df51d..b9f6911b85 100644 --- a/pp/go/storage/head/shard/wal/writer/notifier_test.go +++ b/pp/go/storage/head/shard/wal/writer/notifier_test.go @@ -1,6 +1,7 @@ package writer_test import ( + "math" "testing" "github.com/prometheus/prometheus/pp/go/storage/head/shard/wal/writer" @@ -16,11 +17,49 @@ func TestSegmentWriteNotifierSuite(t *testing.T) { } func (s *SegmentWriteNotifierSuite) TestHappyPath() { - // + actualSegmentID := uint32(math.MaxUint32) numberOfShards := uint16(2) - swn := writer.NewSegmentWriteNotifier(numberOfShards, func(segmentID uint32) { s.T().Log(segmentID) }) + swn := writer.NewSegmentWriteNotifier(numberOfShards, func(segmentID uint32) { actualSegmentID = segmentID }) + + for id := range numberOfShards { + swn.NotifySegmentIsWritten(id) + } + + s.Equal(uint32(0), actualSegmentID) +} + +func (s *SegmentWriteNotifierSuite) TestNotifyOnlyOneShard() { + actualSegmentID := uint32(math.MaxUint32) + + numberOfShards := uint16(2) + swn := writer.NewSegmentWriteNotifier(numberOfShards, func(segmentID uint32) { actualSegmentID = segmentID }) - swn.NotifySegmentIsWritten(1) swn.NotifySegmentIsWritten(0) + + s.Equal(uint32(math.MaxUint32), actualSegmentID) +} + +func (s *SegmentWriteNotifierSuite) TestSetAndNotifyOnlyOneShard() { + actualSegmentID := uint32(math.MaxUint32) + + numberOfShards := uint16(2) + swn := writer.NewSegmentWriteNotifier(numberOfShards, func(segmentID uint32) { actualSegmentID = segmentID }) + swn.Set(0, 42) + + swn.NotifySegmentIsWritten(0) + + s.Equal(uint32(math.MaxUint32), actualSegmentID) +} + +func (s *SegmentWriteNotifierSuite) TestSetAndNotifyOnlyOneShard_2() { + actualSegmentID := uint32(math.MaxUint32) + + numberOfShards := uint16(2) + swn := writer.NewSegmentWriteNotifier(numberOfShards, func(segmentID uint32) { actualSegmentID = segmentID }) + swn.Set(1, 42) + + swn.NotifySegmentIsWritten(0) + + s.Equal(uint32(0), actualSegmentID) } diff --git a/pp/go/storage/head/shard/wal/writer/segment_test.go b/pp/go/storage/head/shard/wal/writer/segment_test.go index a679761fa4..4cec02c318 100644 --- a/pp/go/storage/head/shard/wal/writer/segment_test.go +++ b/pp/go/storage/head/shard/wal/writer/segment_test.go @@ -12,13 +12,27 @@ import ( func TestWriteSegment(t *testing.T) { data := []byte{1, 2, 3, 2, 1, 0, 42} - segment := &testSegment{ - size: int64(len(data)), - samples: 42, - data: data, + segmentCrc32 := uint32(0) + segmentSamples := uint32(42) + + segment := &EncodedSegmentMock{ + CRC32Func: func() uint32 { + return segmentCrc32 + }, + SamplesFunc: func() uint32 { + return segmentSamples + }, + SizeFunc: func() int64 { + return int64(len(data)) + }, + WriteToFunc: func(w io.Writer) (int64, error) { + n, err := w.Write(data) + return int64(n), err + }, } + buf := &bytes.Buffer{} - expected := []byte{byte(len(data)), byte(segment.crc32), byte(segment.samples)} + expected := []byte{byte(len(data)), byte(segmentCrc32), byte(segmentSamples)} expected = append(expected, data...) _, err := writer.WriteSegment(buf, segment) @@ -26,36 +40,3 @@ func TestWriteSegment(t *testing.T) { require.Equal(t, expected, buf.Bytes()) } - -// -// testSegment -// - -// testSegment implementation [writer.EncodedSegment]. -type testSegment struct { - size int64 - samples uint32 - crc32 uint32 - data []byte -} - -// CRC32 implementation [writer.EncodedSegment]. -func (s *testSegment) CRC32() uint32 { - return s.crc32 -} - -// Samples implementation [writer.EncodedSegment]. -func (s *testSegment) Samples() uint32 { - return s.samples -} - -// Size implementation [writer.EncodedSegment]. -func (s *testSegment) Size() int64 { - return s.size -} - -// WriteTo implementation [writer.EncodedSegment]. -func (s *testSegment) WriteTo(w io.Writer) (int64, error) { - n, err := w.Write(s.data) - return int64(n), err -} diff --git a/pp/go/storage/head/shard/wal/writer/writer_moq_test.go b/pp/go/storage/head/shard/wal/writer/writer_moq_test.go new file mode 100644 index 0000000000..7c835b7f33 --- /dev/null +++ b/pp/go/storage/head/shard/wal/writer/writer_moq_test.go @@ -0,0 +1,676 @@ +// Code generated by moq; DO NOT EDIT. +// github.com/matryer/moq + +package writer_test + +import ( + "github.com/prometheus/prometheus/pp/go/storage/head/shard/wal/writer" + "io" + "io/fs" + "sync" + "time" +) + +// Ensure, that EncodedSegmentMock does implement writer.EncodedSegment. +// If this is not the case, regenerate this file with moq. +var _ writer.EncodedSegment = &EncodedSegmentMock{} + +// EncodedSegmentMock is a mock implementation of writer.EncodedSegment. +// +// func TestSomethingThatUsesEncodedSegment(t *testing.T) { +// +// // make and configure a mocked writer.EncodedSegment +// mockedEncodedSegment := &EncodedSegmentMock{ +// CRC32Func: func() uint32 { +// panic("mock out the CRC32 method") +// }, +// SamplesFunc: func() uint32 { +// panic("mock out the Samples method") +// }, +// SizeFunc: func() int64 { +// panic("mock out the Size method") +// }, +// WriteToFunc: func(w io.Writer) (int64, error) { +// panic("mock out the WriteTo method") +// }, +// } +// +// // use mockedEncodedSegment in code that requires writer.EncodedSegment +// // and then make assertions. +// +// } +type EncodedSegmentMock struct { + // CRC32Func mocks the CRC32 method. + CRC32Func func() uint32 + + // SamplesFunc mocks the Samples method. + SamplesFunc func() uint32 + + // SizeFunc mocks the Size method. + SizeFunc func() int64 + + // WriteToFunc mocks the WriteTo method. + WriteToFunc func(w io.Writer) (int64, error) + + // calls tracks calls to the methods. + calls struct { + // CRC32 holds details about calls to the CRC32 method. + CRC32 []struct { + } + // Samples holds details about calls to the Samples method. + Samples []struct { + } + // Size holds details about calls to the Size method. + Size []struct { + } + // WriteTo holds details about calls to the WriteTo method. + WriteTo []struct { + // W is the w argument value. + W io.Writer + } + } + lockCRC32 sync.RWMutex + lockSamples sync.RWMutex + lockSize sync.RWMutex + lockWriteTo sync.RWMutex +} + +// CRC32 calls CRC32Func. +func (mock *EncodedSegmentMock) CRC32() uint32 { + if mock.CRC32Func == nil { + panic("EncodedSegmentMock.CRC32Func: method is nil but EncodedSegment.CRC32 was just called") + } + callInfo := struct { + }{} + mock.lockCRC32.Lock() + mock.calls.CRC32 = append(mock.calls.CRC32, callInfo) + mock.lockCRC32.Unlock() + return mock.CRC32Func() +} + +// CRC32Calls gets all the calls that were made to CRC32. +// Check the length with: +// +// len(mockedEncodedSegment.CRC32Calls()) +func (mock *EncodedSegmentMock) CRC32Calls() []struct { +} { + var calls []struct { + } + mock.lockCRC32.RLock() + calls = mock.calls.CRC32 + mock.lockCRC32.RUnlock() + return calls +} + +// Samples calls SamplesFunc. +func (mock *EncodedSegmentMock) Samples() uint32 { + if mock.SamplesFunc == nil { + panic("EncodedSegmentMock.SamplesFunc: method is nil but EncodedSegment.Samples was just called") + } + callInfo := struct { + }{} + mock.lockSamples.Lock() + mock.calls.Samples = append(mock.calls.Samples, callInfo) + mock.lockSamples.Unlock() + return mock.SamplesFunc() +} + +// SamplesCalls gets all the calls that were made to Samples. +// Check the length with: +// +// len(mockedEncodedSegment.SamplesCalls()) +func (mock *EncodedSegmentMock) SamplesCalls() []struct { +} { + var calls []struct { + } + mock.lockSamples.RLock() + calls = mock.calls.Samples + mock.lockSamples.RUnlock() + return calls +} + +// Size calls SizeFunc. +func (mock *EncodedSegmentMock) Size() int64 { + if mock.SizeFunc == nil { + panic("EncodedSegmentMock.SizeFunc: method is nil but EncodedSegment.Size was just called") + } + callInfo := struct { + }{} + mock.lockSize.Lock() + mock.calls.Size = append(mock.calls.Size, callInfo) + mock.lockSize.Unlock() + return mock.SizeFunc() +} + +// SizeCalls gets all the calls that were made to Size. +// Check the length with: +// +// len(mockedEncodedSegment.SizeCalls()) +func (mock *EncodedSegmentMock) SizeCalls() []struct { +} { + var calls []struct { + } + mock.lockSize.RLock() + calls = mock.calls.Size + mock.lockSize.RUnlock() + return calls +} + +// WriteTo calls WriteToFunc. +func (mock *EncodedSegmentMock) WriteTo(w io.Writer) (int64, error) { + if mock.WriteToFunc == nil { + panic("EncodedSegmentMock.WriteToFunc: method is nil but EncodedSegment.WriteTo was just called") + } + callInfo := struct { + W io.Writer + }{ + W: w, + } + mock.lockWriteTo.Lock() + mock.calls.WriteTo = append(mock.calls.WriteTo, callInfo) + mock.lockWriteTo.Unlock() + return mock.WriteToFunc(w) +} + +// WriteToCalls gets all the calls that were made to WriteTo. +// Check the length with: +// +// len(mockedEncodedSegment.WriteToCalls()) +func (mock *EncodedSegmentMock) WriteToCalls() []struct { + W io.Writer +} { + var calls []struct { + W io.Writer + } + mock.lockWriteTo.RLock() + calls = mock.calls.WriteTo + mock.lockWriteTo.RUnlock() + return calls +} + +// Ensure, that WriteSyncCloserMock does implement writer.WriteSyncCloser. +// If this is not the case, regenerate this file with moq. +var _ writer.WriteSyncCloser = &WriteSyncCloserMock{} + +// WriteSyncCloserMock is a mock implementation of writer.WriteSyncCloser. +// +// func TestSomethingThatUsesWriteSyncCloser(t *testing.T) { +// +// // make and configure a mocked writer.WriteSyncCloser +// mockedWriteSyncCloser := &WriteSyncCloserMock{ +// CloseFunc: func() error { +// panic("mock out the Close method") +// }, +// StatFunc: func() (writer.FileInfo, error) { +// panic("mock out the Stat method") +// }, +// SyncFunc: func() error { +// panic("mock out the Sync method") +// }, +// WriteFunc: func(p []byte) (int, error) { +// panic("mock out the Write method") +// }, +// } +// +// // use mockedWriteSyncCloser in code that requires writer.WriteSyncCloser +// // and then make assertions. +// +// } +type WriteSyncCloserMock struct { + // CloseFunc mocks the Close method. + CloseFunc func() error + + // StatFunc mocks the Stat method. + StatFunc func() (writer.FileInfo, error) + + // SyncFunc mocks the Sync method. + SyncFunc func() error + + // WriteFunc mocks the Write method. + WriteFunc func(p []byte) (int, error) + + // calls tracks calls to the methods. + calls struct { + // Close holds details about calls to the Close method. + Close []struct { + } + // Stat holds details about calls to the Stat method. + Stat []struct { + } + // Sync holds details about calls to the Sync method. + Sync []struct { + } + // Write holds details about calls to the Write method. + Write []struct { + // P is the p argument value. + P []byte + } + } + lockClose sync.RWMutex + lockStat sync.RWMutex + lockSync sync.RWMutex + lockWrite sync.RWMutex +} + +// Close calls CloseFunc. +func (mock *WriteSyncCloserMock) Close() error { + if mock.CloseFunc == nil { + panic("WriteSyncCloserMock.CloseFunc: method is nil but WriteSyncCloser.Close was just called") + } + callInfo := struct { + }{} + mock.lockClose.Lock() + mock.calls.Close = append(mock.calls.Close, callInfo) + mock.lockClose.Unlock() + return mock.CloseFunc() +} + +// CloseCalls gets all the calls that were made to Close. +// Check the length with: +// +// len(mockedWriteSyncCloser.CloseCalls()) +func (mock *WriteSyncCloserMock) CloseCalls() []struct { +} { + var calls []struct { + } + mock.lockClose.RLock() + calls = mock.calls.Close + mock.lockClose.RUnlock() + return calls +} + +// Stat calls StatFunc. +func (mock *WriteSyncCloserMock) Stat() (writer.FileInfo, error) { + if mock.StatFunc == nil { + panic("WriteSyncCloserMock.StatFunc: method is nil but WriteSyncCloser.Stat was just called") + } + callInfo := struct { + }{} + mock.lockStat.Lock() + mock.calls.Stat = append(mock.calls.Stat, callInfo) + mock.lockStat.Unlock() + return mock.StatFunc() +} + +// StatCalls gets all the calls that were made to Stat. +// Check the length with: +// +// len(mockedWriteSyncCloser.StatCalls()) +func (mock *WriteSyncCloserMock) StatCalls() []struct { +} { + var calls []struct { + } + mock.lockStat.RLock() + calls = mock.calls.Stat + mock.lockStat.RUnlock() + return calls +} + +// Sync calls SyncFunc. +func (mock *WriteSyncCloserMock) Sync() error { + if mock.SyncFunc == nil { + panic("WriteSyncCloserMock.SyncFunc: method is nil but WriteSyncCloser.Sync was just called") + } + callInfo := struct { + }{} + mock.lockSync.Lock() + mock.calls.Sync = append(mock.calls.Sync, callInfo) + mock.lockSync.Unlock() + return mock.SyncFunc() +} + +// SyncCalls gets all the calls that were made to Sync. +// Check the length with: +// +// len(mockedWriteSyncCloser.SyncCalls()) +func (mock *WriteSyncCloserMock) SyncCalls() []struct { +} { + var calls []struct { + } + mock.lockSync.RLock() + calls = mock.calls.Sync + mock.lockSync.RUnlock() + return calls +} + +// Write calls WriteFunc. +func (mock *WriteSyncCloserMock) Write(p []byte) (int, error) { + if mock.WriteFunc == nil { + panic("WriteSyncCloserMock.WriteFunc: method is nil but WriteSyncCloser.Write was just called") + } + callInfo := struct { + P []byte + }{ + P: p, + } + mock.lockWrite.Lock() + mock.calls.Write = append(mock.calls.Write, callInfo) + mock.lockWrite.Unlock() + return mock.WriteFunc(p) +} + +// WriteCalls gets all the calls that were made to Write. +// Check the length with: +// +// len(mockedWriteSyncCloser.WriteCalls()) +func (mock *WriteSyncCloserMock) WriteCalls() []struct { + P []byte +} { + var calls []struct { + P []byte + } + mock.lockWrite.RLock() + calls = mock.calls.Write + mock.lockWrite.RUnlock() + return calls +} + +// Ensure, that SegmentIsWrittenNotifierMock does implement writer.SegmentIsWrittenNotifier. +// If this is not the case, regenerate this file with moq. +var _ writer.SegmentIsWrittenNotifier = &SegmentIsWrittenNotifierMock{} + +// SegmentIsWrittenNotifierMock is a mock implementation of writer.SegmentIsWrittenNotifier. +// +// func TestSomethingThatUsesSegmentIsWrittenNotifier(t *testing.T) { +// +// // make and configure a mocked writer.SegmentIsWrittenNotifier +// mockedSegmentIsWrittenNotifier := &SegmentIsWrittenNotifierMock{ +// NotifySegmentIsWrittenFunc: func(shardID uint16) { +// panic("mock out the NotifySegmentIsWritten method") +// }, +// } +// +// // use mockedSegmentIsWrittenNotifier in code that requires writer.SegmentIsWrittenNotifier +// // and then make assertions. +// +// } +type SegmentIsWrittenNotifierMock struct { + // NotifySegmentIsWrittenFunc mocks the NotifySegmentIsWritten method. + NotifySegmentIsWrittenFunc func(shardID uint16) + + // calls tracks calls to the methods. + calls struct { + // NotifySegmentIsWritten holds details about calls to the NotifySegmentIsWritten method. + NotifySegmentIsWritten []struct { + // ShardID is the shardID argument value. + ShardID uint16 + } + } + lockNotifySegmentIsWritten sync.RWMutex +} + +// NotifySegmentIsWritten calls NotifySegmentIsWrittenFunc. +func (mock *SegmentIsWrittenNotifierMock) NotifySegmentIsWritten(shardID uint16) { + if mock.NotifySegmentIsWrittenFunc == nil { + panic("SegmentIsWrittenNotifierMock.NotifySegmentIsWrittenFunc: method is nil but SegmentIsWrittenNotifier.NotifySegmentIsWritten was just called") + } + callInfo := struct { + ShardID uint16 + }{ + ShardID: shardID, + } + mock.lockNotifySegmentIsWritten.Lock() + mock.calls.NotifySegmentIsWritten = append(mock.calls.NotifySegmentIsWritten, callInfo) + mock.lockNotifySegmentIsWritten.Unlock() + mock.NotifySegmentIsWrittenFunc(shardID) +} + +// NotifySegmentIsWrittenCalls gets all the calls that were made to NotifySegmentIsWritten. +// Check the length with: +// +// len(mockedSegmentIsWrittenNotifier.NotifySegmentIsWrittenCalls()) +func (mock *SegmentIsWrittenNotifierMock) NotifySegmentIsWrittenCalls() []struct { + ShardID uint16 +} { + var calls []struct { + ShardID uint16 + } + mock.lockNotifySegmentIsWritten.RLock() + calls = mock.calls.NotifySegmentIsWritten + mock.lockNotifySegmentIsWritten.RUnlock() + return calls +} + +// Ensure, that FileInfoMock does implement writer.FileInfo. +// If this is not the case, regenerate this file with moq. +var _ writer.FileInfo = &FileInfoMock{} + +// FileInfoMock is a mock implementation of writer.FileInfo. +// +// func TestSomethingThatUsesFileInfo(t *testing.T) { +// +// // make and configure a mocked writer.FileInfo +// mockedFileInfo := &FileInfoMock{ +// IsDirFunc: func() bool { +// panic("mock out the IsDir method") +// }, +// ModTimeFunc: func() time.Time { +// panic("mock out the ModTime method") +// }, +// ModeFunc: func() fs.FileMode { +// panic("mock out the Mode method") +// }, +// NameFunc: func() string { +// panic("mock out the Name method") +// }, +// SizeFunc: func() int64 { +// panic("mock out the Size method") +// }, +// SysFunc: func() any { +// panic("mock out the Sys method") +// }, +// } +// +// // use mockedFileInfo in code that requires writer.FileInfo +// // and then make assertions. +// +// } +type FileInfoMock struct { + // IsDirFunc mocks the IsDir method. + IsDirFunc func() bool + + // ModTimeFunc mocks the ModTime method. + ModTimeFunc func() time.Time + + // ModeFunc mocks the Mode method. + ModeFunc func() fs.FileMode + + // NameFunc mocks the Name method. + NameFunc func() string + + // SizeFunc mocks the Size method. + SizeFunc func() int64 + + // SysFunc mocks the Sys method. + SysFunc func() any + + // calls tracks calls to the methods. + calls struct { + // IsDir holds details about calls to the IsDir method. + IsDir []struct { + } + // ModTime holds details about calls to the ModTime method. + ModTime []struct { + } + // Mode holds details about calls to the Mode method. + Mode []struct { + } + // Name holds details about calls to the Name method. + Name []struct { + } + // Size holds details about calls to the Size method. + Size []struct { + } + // Sys holds details about calls to the Sys method. + Sys []struct { + } + } + lockIsDir sync.RWMutex + lockModTime sync.RWMutex + lockMode sync.RWMutex + lockName sync.RWMutex + lockSize sync.RWMutex + lockSys sync.RWMutex +} + +// IsDir calls IsDirFunc. +func (mock *FileInfoMock) IsDir() bool { + if mock.IsDirFunc == nil { + panic("FileInfoMock.IsDirFunc: method is nil but FileInfo.IsDir was just called") + } + callInfo := struct { + }{} + mock.lockIsDir.Lock() + mock.calls.IsDir = append(mock.calls.IsDir, callInfo) + mock.lockIsDir.Unlock() + return mock.IsDirFunc() +} + +// IsDirCalls gets all the calls that were made to IsDir. +// Check the length with: +// +// len(mockedFileInfo.IsDirCalls()) +func (mock *FileInfoMock) IsDirCalls() []struct { +} { + var calls []struct { + } + mock.lockIsDir.RLock() + calls = mock.calls.IsDir + mock.lockIsDir.RUnlock() + return calls +} + +// ModTime calls ModTimeFunc. +func (mock *FileInfoMock) ModTime() time.Time { + if mock.ModTimeFunc == nil { + panic("FileInfoMock.ModTimeFunc: method is nil but FileInfo.ModTime was just called") + } + callInfo := struct { + }{} + mock.lockModTime.Lock() + mock.calls.ModTime = append(mock.calls.ModTime, callInfo) + mock.lockModTime.Unlock() + return mock.ModTimeFunc() +} + +// ModTimeCalls gets all the calls that were made to ModTime. +// Check the length with: +// +// len(mockedFileInfo.ModTimeCalls()) +func (mock *FileInfoMock) ModTimeCalls() []struct { +} { + var calls []struct { + } + mock.lockModTime.RLock() + calls = mock.calls.ModTime + mock.lockModTime.RUnlock() + return calls +} + +// Mode calls ModeFunc. +func (mock *FileInfoMock) Mode() fs.FileMode { + if mock.ModeFunc == nil { + panic("FileInfoMock.ModeFunc: method is nil but FileInfo.Mode was just called") + } + callInfo := struct { + }{} + mock.lockMode.Lock() + mock.calls.Mode = append(mock.calls.Mode, callInfo) + mock.lockMode.Unlock() + return mock.ModeFunc() +} + +// ModeCalls gets all the calls that were made to Mode. +// Check the length with: +// +// len(mockedFileInfo.ModeCalls()) +func (mock *FileInfoMock) ModeCalls() []struct { +} { + var calls []struct { + } + mock.lockMode.RLock() + calls = mock.calls.Mode + mock.lockMode.RUnlock() + return calls +} + +// Name calls NameFunc. +func (mock *FileInfoMock) Name() string { + if mock.NameFunc == nil { + panic("FileInfoMock.NameFunc: method is nil but FileInfo.Name was just called") + } + callInfo := struct { + }{} + mock.lockName.Lock() + mock.calls.Name = append(mock.calls.Name, callInfo) + mock.lockName.Unlock() + return mock.NameFunc() +} + +// NameCalls gets all the calls that were made to Name. +// Check the length with: +// +// len(mockedFileInfo.NameCalls()) +func (mock *FileInfoMock) NameCalls() []struct { +} { + var calls []struct { + } + mock.lockName.RLock() + calls = mock.calls.Name + mock.lockName.RUnlock() + return calls +} + +// Size calls SizeFunc. +func (mock *FileInfoMock) Size() int64 { + if mock.SizeFunc == nil { + panic("FileInfoMock.SizeFunc: method is nil but FileInfo.Size was just called") + } + callInfo := struct { + }{} + mock.lockSize.Lock() + mock.calls.Size = append(mock.calls.Size, callInfo) + mock.lockSize.Unlock() + return mock.SizeFunc() +} + +// SizeCalls gets all the calls that were made to Size. +// Check the length with: +// +// len(mockedFileInfo.SizeCalls()) +func (mock *FileInfoMock) SizeCalls() []struct { +} { + var calls []struct { + } + mock.lockSize.RLock() + calls = mock.calls.Size + mock.lockSize.RUnlock() + return calls +} + +// Sys calls SysFunc. +func (mock *FileInfoMock) Sys() any { + if mock.SysFunc == nil { + panic("FileInfoMock.SysFunc: method is nil but FileInfo.Sys was just called") + } + callInfo := struct { + }{} + mock.lockSys.Lock() + mock.calls.Sys = append(mock.calls.Sys, callInfo) + mock.lockSys.Unlock() + return mock.SysFunc() +} + +// SysCalls gets all the calls that were made to Sys. +// Check the length with: +// +// len(mockedFileInfo.SysCalls()) +func (mock *FileInfoMock) SysCalls() []struct { +} { + var calls []struct { + } + mock.lockSys.RLock() + calls = mock.calls.Sys + mock.lockSys.RUnlock() + return calls +} From 363bbf1b68f93d2ad13f84bf07a30501c3122a07 Mon Sep 17 00:00:00 2001 From: Alexandr Yudin Date: Tue, 23 Sep 2025 08:29:51 +0000 Subject: [PATCH 42/96] add test --- .../storage/head/shard/wal/reader/segment.go | 6 +- .../head/shard/wal/reader/segment_test.go | 122 ++++++++++++++++++ .../head/shard/wal/writer/segment_test.go | 8 +- 3 files changed, 131 insertions(+), 5 deletions(-) create mode 100644 pp/go/storage/head/shard/wal/reader/segment_test.go diff --git a/pp/go/storage/head/shard/wal/reader/segment.go b/pp/go/storage/head/shard/wal/reader/segment.go index b5eb5ddb07..42bcafaff8 100644 --- a/pp/go/storage/head/shard/wal/reader/segment.go +++ b/pp/go/storage/head/shard/wal/reader/segment.go @@ -31,7 +31,7 @@ func (s *Segment) Length() int { // ReadFrom reads [Segment] data from r [io.Reader]. The return value n is the number of bytes read. // Any error encountered during the read is also returned. func (s *Segment) ReadFrom(r io.Reader) (int64, error) { - return ReadSegment(r, s) + return readSegment(r, s) } // Reset [Segment] data. @@ -54,8 +54,8 @@ func (s *Segment) resize(size int) { } } -// ReadSegment read and decode [Segment] from [io.Reader] and returns. -func ReadSegment(reader io.Reader, segment *Segment) (int64, error) { +// readSegment read and decode [Segment] from [io.Reader] and returns. +func readSegment(reader io.Reader, segment *Segment) (int64, error) { br := NewByteReader(reader) size, err := binary.ReadUvarint(br) if err != nil { diff --git a/pp/go/storage/head/shard/wal/reader/segment_test.go b/pp/go/storage/head/shard/wal/reader/segment_test.go new file mode 100644 index 0000000000..83c661d4bd --- /dev/null +++ b/pp/go/storage/head/shard/wal/reader/segment_test.go @@ -0,0 +1,122 @@ +package reader_test + +import ( + "bytes" + "encoding/binary" + "hash/crc32" + "io" + "testing" + + "github.com/go-faker/faker/v4" + "github.com/stretchr/testify/suite" + + "github.com/prometheus/prometheus/pp/go/storage/head/shard/wal/reader" +) + +type SegmentSuite struct { + suite.Suite +} + +func TestSegmentSuite(t *testing.T) { + suite.Run(t, new(SegmentSuite)) +} + +func (s *SegmentSuite) TestHappyPath() { + data := []byte(faker.Paragraph()) + segmentSamples := uint32(42) + toWrite := []byte{} + toWrite = append(toWrite, binary.AppendUvarint(nil, uint64(len(data)))...) + toWrite = append(toWrite, binary.AppendUvarint(nil, uint64(crc32.ChecksumIEEE(data)))...) + toWrite = append(toWrite, byte(segmentSamples)) + toWrite = append(toWrite, data...) + + buf := &bytes.Buffer{} + _, err := buf.Write(toWrite) + s.Require().NoError(err) + + segment := reader.NewSegment() + _, err = segment.ReadFrom(buf) + s.Require().NoError(err) + + s.Require().Equal(len(data), segment.Length()) + s.Require().Equal(segmentSamples, segment.Samples()) + s.Require().Equal(data, segment.Bytes()) +} + +func (s *SegmentSuite) TestReuseSegment() { + data := []byte(faker.Paragraph()) + segmentSamples := uint32(42) + toWrite := []byte{} + toWrite = append(toWrite, binary.AppendUvarint(nil, uint64(len(data)))...) + toWrite = append(toWrite, binary.AppendUvarint(nil, uint64(crc32.ChecksumIEEE(data)))...) + toWrite = append(toWrite, byte(segmentSamples)) + toWrite = append(toWrite, data...) + + buf := &bytes.Buffer{} + _, err := buf.Write(toWrite) + s.Require().NoError(err) + + segment := reader.NewSegment() + _, err = segment.ReadFrom(buf) + s.Require().NoError(err) + + s.Require().Equal(len(data), segment.Length()) + s.Require().Equal(segmentSamples, segment.Samples()) + s.Require().Equal(data, segment.Bytes()) + + buf.Reset() + toWrite = toWrite[:0] + data = []byte(faker.Paragraph()) + toWrite = append(toWrite, binary.AppendUvarint(nil, uint64(len(data)))...) + toWrite = append(toWrite, binary.AppendUvarint(nil, uint64(crc32.ChecksumIEEE(data)))...) + toWrite = append(toWrite, byte(segmentSamples)) + toWrite = append(toWrite, data...) + + _, err = buf.Write(toWrite) + s.Require().NoError(err) + + segment.Reset() + _, err = segment.ReadFrom(buf) + s.Require().NoError(err) + + s.Require().Equal(len(data), segment.Length()) + s.Require().Equal(segmentSamples, segment.Samples()) + s.Require().Equal(data, segment.Bytes()) +} + +func (s *SegmentSuite) TestCrc32Error() { + data := []byte(faker.Paragraph()) + segmentSamples := uint32(42) + toWrite := []byte{} + toWrite = append(toWrite, binary.AppendUvarint(nil, uint64(len(data)))...) + toWrite = append(toWrite, binary.AppendUvarint(nil, uint64(0))...) + toWrite = append(toWrite, byte(segmentSamples)) + toWrite = append(toWrite, data...) + + buf := &bytes.Buffer{} + _, err := buf.Write(toWrite) + s.Require().NoError(err) + + segment := reader.NewSegment() + _, err = segment.ReadFrom(buf) + s.Require().Error(err) +} + +func (s *SegmentSuite) TestCutSegment() { + data := []byte(faker.Paragraph()) + segmentCrc32 := crc32.ChecksumIEEE(data) + segmentSamples := uint32(42) + toWrite := []byte{} + toWrite = append(toWrite, binary.AppendUvarint(nil, uint64(len(data)))...) + toWrite = append(toWrite, binary.AppendUvarint(nil, uint64(segmentCrc32))...) + toWrite = append(toWrite, byte(segmentSamples)) + toWrite = append(toWrite, data[:len(data)-2]...) + + buf := &bytes.Buffer{} + _, err := buf.Write(toWrite) + s.Require().NoError(err) + + segment := reader.NewSegment() + _, err = segment.ReadFrom(buf) + s.Require().ErrorIs(err, io.ErrUnexpectedEOF) +} diff --git a/pp/go/storage/head/shard/wal/writer/segment_test.go b/pp/go/storage/head/shard/wal/writer/segment_test.go index 4cec02c318..54c889b7b4 100644 --- a/pp/go/storage/head/shard/wal/writer/segment_test.go +++ b/pp/go/storage/head/shard/wal/writer/segment_test.go @@ -2,16 +2,18 @@ package writer_test import ( "bytes" + "encoding/binary" "io" "testing" + "github.com/go-faker/faker/v4" "github.com/stretchr/testify/require" "github.com/prometheus/prometheus/pp/go/storage/head/shard/wal/writer" ) func TestWriteSegment(t *testing.T) { - data := []byte{1, 2, 3, 2, 1, 0, 42} + data := []byte(faker.Paragraph()) segmentCrc32 := uint32(0) segmentSamples := uint32(42) @@ -32,7 +34,9 @@ func TestWriteSegment(t *testing.T) { } buf := &bytes.Buffer{} - expected := []byte{byte(len(data)), byte(segmentCrc32), byte(segmentSamples)} + expected := []byte{} + expected = append(expected, binary.AppendUvarint(nil, uint64(len(data)))...) + expected = append(expected, byte(segmentCrc32), byte(segmentSamples)) expected = append(expected, data...) _, err := writer.WriteSegment(buf, segment) From 9ced693efb4d3a20d128d1bfbcb157c7c8991277 Mon Sep 17 00:00:00 2001 From: Alexandr Yudin Date: Wed, 24 Sep 2025 06:11:47 +0000 Subject: [PATCH 43/96] add wal reader test --- pp/go/storage/head/shard/wal/wal_moq_test.go | 113 ++++++++++++++++++ .../storage/head/shard/wal/wal_reader_test.go | 80 +++++++++++++ 2 files changed, 193 insertions(+) create mode 100644 pp/go/storage/head/shard/wal/wal_moq_test.go create mode 100644 pp/go/storage/head/shard/wal/wal_reader_test.go diff --git a/pp/go/storage/head/shard/wal/wal_moq_test.go b/pp/go/storage/head/shard/wal/wal_moq_test.go new file mode 100644 index 0000000000..3484eb7cec --- /dev/null +++ b/pp/go/storage/head/shard/wal/wal_moq_test.go @@ -0,0 +1,113 @@ +// Code generated by moq; DO NOT EDIT. +// github.com/matryer/moq + +package wal_test + +import ( + "github.com/prometheus/prometheus/pp/go/storage/head/shard/wal" + "io" + "sync" +) + +// Ensure, that ReadSegmentMock does implement wal.ReadSegment. +// If this is not the case, regenerate this file with moq. +var _ wal.ReadSegment = &ReadSegmentMock{} + +// ReadSegmentMock is a mock implementation of wal.ReadSegment. +// +// func TestSomethingThatUsesReadSegment(t *testing.T) { +// +// // make and configure a mocked wal.ReadSegment +// mockedReadSegment := &ReadSegmentMock{ +// ReadFromFunc: func(r io.Reader) (int64, error) { +// panic("mock out the ReadFrom method") +// }, +// ResetFunc: func() { +// panic("mock out the Reset method") +// }, +// } +// +// // use mockedReadSegment in code that requires wal.ReadSegment +// // and then make assertions. +// +// } +type ReadSegmentMock struct { + // ReadFromFunc mocks the ReadFrom method. + ReadFromFunc func(r io.Reader) (int64, error) + + // ResetFunc mocks the Reset method. + ResetFunc func() + + // calls tracks calls to the methods. + calls struct { + // ReadFrom holds details about calls to the ReadFrom method. + ReadFrom []struct { + // R is the r argument value. + R io.Reader + } + // Reset holds details about calls to the Reset method. + Reset []struct { + } + } + lockReadFrom sync.RWMutex + lockReset sync.RWMutex +} + +// ReadFrom calls ReadFromFunc. +func (mock *ReadSegmentMock) ReadFrom(r io.Reader) (int64, error) { + if mock.ReadFromFunc == nil { + panic("ReadSegmentMock.ReadFromFunc: method is nil but ReadSegment.ReadFrom was just called") + } + callInfo := struct { + R io.Reader + }{ + R: r, + } + mock.lockReadFrom.Lock() + mock.calls.ReadFrom = append(mock.calls.ReadFrom, callInfo) + mock.lockReadFrom.Unlock() + return mock.ReadFromFunc(r) +} + +// ReadFromCalls gets all the calls that were made to ReadFrom. +// Check the length with: +// +// len(mockedReadSegment.ReadFromCalls()) +func (mock *ReadSegmentMock) ReadFromCalls() []struct { + R io.Reader +} { + var calls []struct { + R io.Reader + } + mock.lockReadFrom.RLock() + calls = mock.calls.ReadFrom + mock.lockReadFrom.RUnlock() + return calls +} + +// Reset calls ResetFunc. +func (mock *ReadSegmentMock) Reset() { + if mock.ResetFunc == nil { + panic("ReadSegmentMock.ResetFunc: method is nil but ReadSegment.Reset was just called") + } + callInfo := struct { + }{} + mock.lockReset.Lock() + mock.calls.Reset = append(mock.calls.Reset, callInfo) + mock.lockReset.Unlock() + mock.ResetFunc() +} + +// ResetCalls gets all the calls that were made to Reset. +// Check the length with: +// +// len(mockedReadSegment.ResetCalls()) +func (mock *ReadSegmentMock) ResetCalls() []struct { +} { + var calls []struct { + } + mock.lockReset.RLock() + calls = mock.calls.Reset + mock.lockReset.RUnlock() + return calls +} diff --git a/pp/go/storage/head/shard/wal/wal_reader_test.go b/pp/go/storage/head/shard/wal/wal_reader_test.go new file mode 100644 index 0000000000..c246eaac70 --- /dev/null +++ b/pp/go/storage/head/shard/wal/wal_reader_test.go @@ -0,0 +1,80 @@ +package wal_test + +import ( + "bytes" + "errors" + "io" + "testing" + + "github.com/go-faker/faker/v4" + "github.com/prometheus/prometheus/pp/go/storage/head/shard/wal" + "github.com/stretchr/testify/suite" +) + +type SegmentWalReaderSuite struct { + suite.Suite +} + +func TestSegmentWalReaderSuite(t *testing.T) { + suite.Run(t, new(SegmentWalReaderSuite)) +} + +func (s *SegmentWalReaderSuite) TestHappyPath() { + buf := &bytes.Buffer{} + data := []byte(faker.Paragraph()) + data = data[:(len(data)/10)*10] + _, err := buf.Write(data) + s.Require().NoError(err) + + swr := wal.NewSegmentWalReader(buf, newTestSegment) + limiter := 0 + actual := make([]byte, 0, len(data)) + err = swr.ForEachSegment(func(rsm *testSegment) error { + actual = append(actual, rsm.Bytes()...) + + limiter++ + if limiter == 1000 { + return errors.New("limiter") + } + return nil + }) + s.Require().NoError(err) + + s.Equal(data, actual) +} + +// +// testSegment +// + +// testSegment implements [ReadSegment]. +type testSegment struct { + buf []byte + *ReadSegmentMock +} + +// newTestSegment init new [testSegment]. +func newTestSegment() *testSegment { + s := &testSegment{ + buf: make([]byte, 10), + } + + s.ReadSegmentMock = &ReadSegmentMock{ + ReadFromFunc: func(r io.Reader) (int64, error) { + n, err := io.ReadFull(r, s.buf) + return int64(n), err + }, + ResetFunc: func() { + for i := range s.buf { + s.buf[i] = 0 + } + }, + } + + return s +} + +// Bytes returns data. +func (s *testSegment) Bytes() []byte { + return s.buf +} From a2247d9c9b83e63dcde81bc596d154bbb90be144 Mon Sep 17 00:00:00 2001 From: Alexandr Yudin Date: Wed, 24 Sep 2025 07:57:35 +0000 Subject: [PATCH 44/96] for save --- .../processor/stream_processor_test.go | 2 + pp/go/storage/head/shard/wal/wal_moq_test.go | 554 ++++++++++++++++++ pp/go/storage/head/shard/wal/wal_test.go | 37 ++ .../head/shard/wal/writer/buffered_test.go | 2 + pp/go/storage/mediator/mediator_test.go | 2 + 5 files changed, 597 insertions(+) diff --git a/pp-pkg/handler/processor/stream_processor_test.go b/pp-pkg/handler/processor/stream_processor_test.go index 7e3a9bb8d0..9333951a81 100644 --- a/pp-pkg/handler/processor/stream_processor_test.go +++ b/pp-pkg/handler/processor/stream_processor_test.go @@ -22,6 +22,8 @@ import ( "github.com/prometheus/prometheus/util/pool" ) +// TODO //go:generate -command moq go run github.com/matryer/moq -out processor_moq_test.go -pkg processor_test -rm . Adapter StatesStorage RemoteWrite MetricStream Refill + type segmentContainer struct { timeSeries []coremodel.TimeSeries encoded model.Segment diff --git a/pp/go/storage/head/shard/wal/wal_moq_test.go b/pp/go/storage/head/shard/wal/wal_moq_test.go index 3484eb7cec..924a724449 100644 --- a/pp/go/storage/head/shard/wal/wal_moq_test.go +++ b/pp/go/storage/head/shard/wal/wal_moq_test.go @@ -4,6 +4,7 @@ package wal_test import ( + "github.com/prometheus/prometheus/pp/go/cppbridge" "github.com/prometheus/prometheus/pp/go/storage/head/shard/wal" "io" "sync" @@ -111,3 +112,556 @@ func (mock *ReadSegmentMock) ResetCalls() []struct { mock.lockReset.RUnlock() return calls } + +// Ensure, that EncodedSegmentMock does implement wal.EncodedSegment. +// If this is not the case, regenerate this file with moq. +var _ wal.EncodedSegment = &EncodedSegmentMock{} + +// EncodedSegmentMock is a mock implementation of wal.EncodedSegment. +// +// func TestSomethingThatUsesEncodedSegment(t *testing.T) { +// +// // make and configure a mocked wal.EncodedSegment +// mockedEncodedSegment := &EncodedSegmentMock{ +// CRC32Func: func() uint32 { +// panic("mock out the CRC32 method") +// }, +// SamplesFunc: func() uint32 { +// panic("mock out the Samples method") +// }, +// SizeFunc: func() int64 { +// panic("mock out the Size method") +// }, +// WriteToFunc: func(w io.Writer) (int64, error) { +// panic("mock out the WriteTo method") +// }, +// } +// +// // use mockedEncodedSegment in code that requires wal.EncodedSegment +// // and then make assertions. +// +// } +type EncodedSegmentMock struct { + // CRC32Func mocks the CRC32 method. + CRC32Func func() uint32 + + // SamplesFunc mocks the Samples method. + SamplesFunc func() uint32 + + // SizeFunc mocks the Size method. + SizeFunc func() int64 + + // WriteToFunc mocks the WriteTo method. + WriteToFunc func(w io.Writer) (int64, error) + + // calls tracks calls to the methods. + calls struct { + // CRC32 holds details about calls to the CRC32 method. + CRC32 []struct { + } + // Samples holds details about calls to the Samples method. + Samples []struct { + } + // Size holds details about calls to the Size method. + Size []struct { + } + // WriteTo holds details about calls to the WriteTo method. + WriteTo []struct { + // W is the w argument value. + W io.Writer + } + } + lockCRC32 sync.RWMutex + lockSamples sync.RWMutex + lockSize sync.RWMutex + lockWriteTo sync.RWMutex +} + +// CRC32 calls CRC32Func. +func (mock *EncodedSegmentMock) CRC32() uint32 { + if mock.CRC32Func == nil { + panic("EncodedSegmentMock.CRC32Func: method is nil but EncodedSegment.CRC32 was just called") + } + callInfo := struct { + }{} + mock.lockCRC32.Lock() + mock.calls.CRC32 = append(mock.calls.CRC32, callInfo) + mock.lockCRC32.Unlock() + return mock.CRC32Func() +} + +// CRC32Calls gets all the calls that were made to CRC32. +// Check the length with: +// +// len(mockedEncodedSegment.CRC32Calls()) +func (mock *EncodedSegmentMock) CRC32Calls() []struct { +} { + var calls []struct { + } + mock.lockCRC32.RLock() + calls = mock.calls.CRC32 + mock.lockCRC32.RUnlock() + return calls +} + +// Samples calls SamplesFunc. +func (mock *EncodedSegmentMock) Samples() uint32 { + if mock.SamplesFunc == nil { + panic("EncodedSegmentMock.SamplesFunc: method is nil but EncodedSegment.Samples was just called") + } + callInfo := struct { + }{} + mock.lockSamples.Lock() + mock.calls.Samples = append(mock.calls.Samples, callInfo) + mock.lockSamples.Unlock() + return mock.SamplesFunc() +} + +// SamplesCalls gets all the calls that were made to Samples. +// Check the length with: +// +// len(mockedEncodedSegment.SamplesCalls()) +func (mock *EncodedSegmentMock) SamplesCalls() []struct { +} { + var calls []struct { + } + mock.lockSamples.RLock() + calls = mock.calls.Samples + mock.lockSamples.RUnlock() + return calls +} + +// Size calls SizeFunc. +func (mock *EncodedSegmentMock) Size() int64 { + if mock.SizeFunc == nil { + panic("EncodedSegmentMock.SizeFunc: method is nil but EncodedSegment.Size was just called") + } + callInfo := struct { + }{} + mock.lockSize.Lock() + mock.calls.Size = append(mock.calls.Size, callInfo) + mock.lockSize.Unlock() + return mock.SizeFunc() +} + +// SizeCalls gets all the calls that were made to Size. +// Check the length with: +// +// len(mockedEncodedSegment.SizeCalls()) +func (mock *EncodedSegmentMock) SizeCalls() []struct { +} { + var calls []struct { + } + mock.lockSize.RLock() + calls = mock.calls.Size + mock.lockSize.RUnlock() + return calls +} + +// WriteTo calls WriteToFunc. +func (mock *EncodedSegmentMock) WriteTo(w io.Writer) (int64, error) { + if mock.WriteToFunc == nil { + panic("EncodedSegmentMock.WriteToFunc: method is nil but EncodedSegment.WriteTo was just called") + } + callInfo := struct { + W io.Writer + }{ + W: w, + } + mock.lockWriteTo.Lock() + mock.calls.WriteTo = append(mock.calls.WriteTo, callInfo) + mock.lockWriteTo.Unlock() + return mock.WriteToFunc(w) +} + +// WriteToCalls gets all the calls that were made to WriteTo. +// Check the length with: +// +// len(mockedEncodedSegment.WriteToCalls()) +func (mock *EncodedSegmentMock) WriteToCalls() []struct { + W io.Writer +} { + var calls []struct { + W io.Writer + } + mock.lockWriteTo.RLock() + calls = mock.calls.WriteTo + mock.lockWriteTo.RUnlock() + return calls +} + +// Ensure, that SegmentWriterMock does implement wal.SegmentWriter. +// If this is not the case, regenerate this file with moq. +var _ wal.SegmentWriter[wal.EncodedSegment] = &SegmentWriterMock[wal.EncodedSegment]{} + +// SegmentWriterMock is a mock implementation of wal.SegmentWriter. +// +// func TestSomethingThatUsesSegmentWriter(t *testing.T) { +// +// // make and configure a mocked wal.SegmentWriter +// mockedSegmentWriter := &SegmentWriterMock{ +// CloseFunc: func() error { +// panic("mock out the Close method") +// }, +// CurrentSizeFunc: func() int64 { +// panic("mock out the CurrentSize method") +// }, +// FlushFunc: func() error { +// panic("mock out the Flush method") +// }, +// SyncFunc: func() error { +// panic("mock out the Sync method") +// }, +// WriteFunc: func(segment TSegment) error { +// panic("mock out the Write method") +// }, +// } +// +// // use mockedSegmentWriter in code that requires wal.SegmentWriter +// // and then make assertions. +// +// } +type SegmentWriterMock[TSegment wal.EncodedSegment] struct { + // CloseFunc mocks the Close method. + CloseFunc func() error + + // CurrentSizeFunc mocks the CurrentSize method. + CurrentSizeFunc func() int64 + + // FlushFunc mocks the Flush method. + FlushFunc func() error + + // SyncFunc mocks the Sync method. + SyncFunc func() error + + // WriteFunc mocks the Write method. + WriteFunc func(segment TSegment) error + + // calls tracks calls to the methods. + calls struct { + // Close holds details about calls to the Close method. + Close []struct { + } + // CurrentSize holds details about calls to the CurrentSize method. + CurrentSize []struct { + } + // Flush holds details about calls to the Flush method. + Flush []struct { + } + // Sync holds details about calls to the Sync method. + Sync []struct { + } + // Write holds details about calls to the Write method. + Write []struct { + // Segment is the segment argument value. + Segment TSegment + } + } + lockClose sync.RWMutex + lockCurrentSize sync.RWMutex + lockFlush sync.RWMutex + lockSync sync.RWMutex + lockWrite sync.RWMutex +} + +// Close calls CloseFunc. +func (mock *SegmentWriterMock[TSegment]) Close() error { + if mock.CloseFunc == nil { + panic("SegmentWriterMock.CloseFunc: method is nil but SegmentWriter.Close was just called") + } + callInfo := struct { + }{} + mock.lockClose.Lock() + mock.calls.Close = append(mock.calls.Close, callInfo) + mock.lockClose.Unlock() + return mock.CloseFunc() +} + +// CloseCalls gets all the calls that were made to Close. +// Check the length with: +// +// len(mockedSegmentWriter.CloseCalls()) +func (mock *SegmentWriterMock[TSegment]) CloseCalls() []struct { +} { + var calls []struct { + } + mock.lockClose.RLock() + calls = mock.calls.Close + mock.lockClose.RUnlock() + return calls +} + +// CurrentSize calls CurrentSizeFunc. +func (mock *SegmentWriterMock[TSegment]) CurrentSize() int64 { + if mock.CurrentSizeFunc == nil { + panic("SegmentWriterMock.CurrentSizeFunc: method is nil but SegmentWriter.CurrentSize was just called") + } + callInfo := struct { + }{} + mock.lockCurrentSize.Lock() + mock.calls.CurrentSize = append(mock.calls.CurrentSize, callInfo) + mock.lockCurrentSize.Unlock() + return mock.CurrentSizeFunc() +} + +// CurrentSizeCalls gets all the calls that were made to CurrentSize. +// Check the length with: +// +// len(mockedSegmentWriter.CurrentSizeCalls()) +func (mock *SegmentWriterMock[TSegment]) CurrentSizeCalls() []struct { +} { + var calls []struct { + } + mock.lockCurrentSize.RLock() + calls = mock.calls.CurrentSize + mock.lockCurrentSize.RUnlock() + return calls +} + +// Flush calls FlushFunc. +func (mock *SegmentWriterMock[TSegment]) Flush() error { + if mock.FlushFunc == nil { + panic("SegmentWriterMock.FlushFunc: method is nil but SegmentWriter.Flush was just called") + } + callInfo := struct { + }{} + mock.lockFlush.Lock() + mock.calls.Flush = append(mock.calls.Flush, callInfo) + mock.lockFlush.Unlock() + return mock.FlushFunc() +} + +// FlushCalls gets all the calls that were made to Flush. +// Check the length with: +// +// len(mockedSegmentWriter.FlushCalls()) +func (mock *SegmentWriterMock[TSegment]) FlushCalls() []struct { +} { + var calls []struct { + } + mock.lockFlush.RLock() + calls = mock.calls.Flush + mock.lockFlush.RUnlock() + return calls +} + +// Sync calls SyncFunc. +func (mock *SegmentWriterMock[TSegment]) Sync() error { + if mock.SyncFunc == nil { + panic("SegmentWriterMock.SyncFunc: method is nil but SegmentWriter.Sync was just called") + } + callInfo := struct { + }{} + mock.lockSync.Lock() + mock.calls.Sync = append(mock.calls.Sync, callInfo) + mock.lockSync.Unlock() + return mock.SyncFunc() +} + +// SyncCalls gets all the calls that were made to Sync. +// Check the length with: +// +// len(mockedSegmentWriter.SyncCalls()) +func (mock *SegmentWriterMock[TSegment]) SyncCalls() []struct { +} { + var calls []struct { + } + mock.lockSync.RLock() + calls = mock.calls.Sync + mock.lockSync.RUnlock() + return calls +} + +// Write calls WriteFunc. +func (mock *SegmentWriterMock[TSegment]) Write(segment TSegment) error { + if mock.WriteFunc == nil { + panic("SegmentWriterMock.WriteFunc: method is nil but SegmentWriter.Write was just called") + } + callInfo := struct { + Segment TSegment + }{ + Segment: segment, + } + mock.lockWrite.Lock() + mock.calls.Write = append(mock.calls.Write, callInfo) + mock.lockWrite.Unlock() + return mock.WriteFunc(segment) +} + +// WriteCalls gets all the calls that were made to Write. +// Check the length with: +// +// len(mockedSegmentWriter.WriteCalls()) +func (mock *SegmentWriterMock[TSegment]) WriteCalls() []struct { + Segment TSegment +} { + var calls []struct { + Segment TSegment + } + mock.lockWrite.RLock() + calls = mock.calls.Write + mock.lockWrite.RUnlock() + return calls +} + +// Ensure, that EncoderMock does implement wal.Encoder. +// If this is not the case, regenerate this file with moq. +var _ wal.Encoder[wal.EncodedSegment, wal.StatsSegment] = &EncoderMock[wal.EncodedSegment, wal.StatsSegment]{} + +// EncoderMock is a mock implementation of wal.Encoder. +// +// func TestSomethingThatUsesEncoder(t *testing.T) { +// +// // make and configure a mocked wal.Encoder +// mockedEncoder := &EncoderMock{ +// EncodeFunc: func(innerSeriesSlice []*cppbridge.InnerSeries) (TStats, error) { +// panic("mock out the Encode method") +// }, +// FinalizeFunc: func() (TSegment, error) { +// panic("mock out the Finalize method") +// }, +// } +// +// // use mockedEncoder in code that requires wal.Encoder +// // and then make assertions. +// +// } +type EncoderMock[TSegment wal.EncodedSegment, TStats wal.StatsSegment] struct { + // EncodeFunc mocks the Encode method. + EncodeFunc func(innerSeriesSlice []*cppbridge.InnerSeries) (TStats, error) + + // FinalizeFunc mocks the Finalize method. + FinalizeFunc func() (TSegment, error) + + // calls tracks calls to the methods. + calls struct { + // Encode holds details about calls to the Encode method. + Encode []struct { + // InnerSeriesSlice is the innerSeriesSlice argument value. + InnerSeriesSlice []*cppbridge.InnerSeries + } + // Finalize holds details about calls to the Finalize method. + Finalize []struct { + } + } + lockEncode sync.RWMutex + lockFinalize sync.RWMutex +} + +// Encode calls EncodeFunc. +func (mock *EncoderMock[TSegment, TStats]) Encode(innerSeriesSlice []*cppbridge.InnerSeries) (TStats, error) { + if mock.EncodeFunc == nil { + panic("EncoderMock.EncodeFunc: method is nil but Encoder.Encode was just called") + } + callInfo := struct { + InnerSeriesSlice []*cppbridge.InnerSeries + }{ + InnerSeriesSlice: innerSeriesSlice, + } + mock.lockEncode.Lock() + mock.calls.Encode = append(mock.calls.Encode, callInfo) + mock.lockEncode.Unlock() + return mock.EncodeFunc(innerSeriesSlice) +} + +// EncodeCalls gets all the calls that were made to Encode. +// Check the length with: +// +// len(mockedEncoder.EncodeCalls()) +func (mock *EncoderMock[TSegment, TStats]) EncodeCalls() []struct { + InnerSeriesSlice []*cppbridge.InnerSeries +} { + var calls []struct { + InnerSeriesSlice []*cppbridge.InnerSeries + } + mock.lockEncode.RLock() + calls = mock.calls.Encode + mock.lockEncode.RUnlock() + return calls +} + +// Finalize calls FinalizeFunc. +func (mock *EncoderMock[TSegment, TStats]) Finalize() (TSegment, error) { + if mock.FinalizeFunc == nil { + panic("EncoderMock.FinalizeFunc: method is nil but Encoder.Finalize was just called") + } + callInfo := struct { + }{} + mock.lockFinalize.Lock() + mock.calls.Finalize = append(mock.calls.Finalize, callInfo) + mock.lockFinalize.Unlock() + return mock.FinalizeFunc() +} + +// FinalizeCalls gets all the calls that were made to Finalize. +// Check the length with: +// +// len(mockedEncoder.FinalizeCalls()) +func (mock *EncoderMock[TSegment, TStats]) FinalizeCalls() []struct { +} { + var calls []struct { + } + mock.lockFinalize.RLock() + calls = mock.calls.Finalize + mock.lockFinalize.RUnlock() + return calls +} + +// Ensure, that StatsSegmentMock does implement wal.StatsSegment. +// If this is not the case, regenerate this file with moq. +var _ wal.StatsSegment = &StatsSegmentMock{} + +// StatsSegmentMock is a mock implementation of wal.StatsSegment. +// +// func TestSomethingThatUsesStatsSegment(t *testing.T) { +// +// // make and configure a mocked wal.StatsSegment +// mockedStatsSegment := &StatsSegmentMock{ +// SamplesFunc: func() uint32 { +// panic("mock out the Samples method") +// }, +// } +// +// // use mockedStatsSegment in code that requires wal.StatsSegment +// // and then make assertions. +// +// } +type StatsSegmentMock struct { + // SamplesFunc mocks the Samples method. + SamplesFunc func() uint32 + + // calls tracks calls to the methods. + calls struct { + // Samples holds details about calls to the Samples method. + Samples []struct { + } + } + lockSamples sync.RWMutex +} + +// Samples calls SamplesFunc. +func (mock *StatsSegmentMock) Samples() uint32 { + if mock.SamplesFunc == nil { + panic("StatsSegmentMock.SamplesFunc: method is nil but StatsSegment.Samples was just called") + } + callInfo := struct { + }{} + mock.lockSamples.Lock() + mock.calls.Samples = append(mock.calls.Samples, callInfo) + mock.lockSamples.Unlock() + return mock.SamplesFunc() +} + +// SamplesCalls gets all the calls that were made to Samples. +// Check the length with: +// +// len(mockedStatsSegment.SamplesCalls()) +func (mock *StatsSegmentMock) SamplesCalls() []struct { +} { + var calls []struct { + } + mock.lockSamples.RLock() + calls = mock.calls.Samples + mock.lockSamples.RUnlock() + return calls +} diff --git a/pp/go/storage/head/shard/wal/wal_test.go b/pp/go/storage/head/shard/wal/wal_test.go index d5f3083a0b..e6e4fc00c5 100644 --- a/pp/go/storage/head/shard/wal/wal_test.go +++ b/pp/go/storage/head/shard/wal/wal_test.go @@ -7,12 +7,15 @@ import ( "testing" "github.com/stretchr/testify/require" + "github.com/stretchr/testify/suite" "github.com/prometheus/prometheus/pp/go/cppbridge" "github.com/prometheus/prometheus/pp/go/storage/head/shard/wal" "github.com/prometheus/prometheus/pp/go/storage/head/shard/wal/writer" ) +// TODO moq -out wal_moq_test.go -pkg wal_test -rm . ReadSegment EncodedSegment SegmentWriter Encoder StatsSegment + func TestXxx(t *testing.T) { shardID := uint16(0) tmpDir, err := os.MkdirTemp("", "shard") @@ -49,3 +52,37 @@ type segmentWriteNotifier struct{} func (*segmentWriteNotifier) NotifySegmentIsWritten(shardID uint16) { _ = shardID } + +type WalSuite struct { + suite.Suite +} + +func TestWalSuite(t *testing.T) { + suite.Run(t, new(WalSuite)) +} + +func (s *WalSuite) TestCurrentSize() { + expectedWalSize := int64(42) + enc := &EncoderMock[*EncodedSegmentMock, *StatsSegmentMock]{} + segmentWriter := &SegmentWriterMock[*EncodedSegmentMock]{ + CurrentSizeFunc: func() int64 { + return expectedWalSize + }, + } + maxSegmentSize := uint32(100) + + wl := wal.NewWal(enc, segmentWriter, maxSegmentSize) + + s.Equal(expectedWalSize, wl.CurrentSize()) +} + +func (s *WalSuite) TestCurrentSize2() { + maxSegmentSize := uint32(100) + // enSegment := &EncodedSegmentMock{} + // stats := &StatsSegmentMock{} + enc := &EncoderMock[*EncodedSegmentMock, *StatsSegmentMock]{} + segmentWriter := &SegmentWriterMock[*EncodedSegmentMock]{} + + wl := wal.NewWal(enc, segmentWriter, maxSegmentSize) + _ = wl +} diff --git a/pp/go/storage/head/shard/wal/writer/buffered_test.go b/pp/go/storage/head/shard/wal/writer/buffered_test.go index ef18136208..1e061d5a21 100644 --- a/pp/go/storage/head/shard/wal/writer/buffered_test.go +++ b/pp/go/storage/head/shard/wal/writer/buffered_test.go @@ -13,6 +13,8 @@ import ( "github.com/prometheus/prometheus/pp/go/storage/head/shard/wal/writer" ) +// TODO moq -out writer_moq_test.go -pkg writer_test -rm . EncodedSegment WriteSyncCloser SegmentIsWrittenNotifier FileInfo + type BufferedSuite struct { suite.Suite } diff --git a/pp/go/storage/mediator/mediator_test.go b/pp/go/storage/mediator/mediator_test.go index ef7eb86755..002c6d4356 100644 --- a/pp/go/storage/mediator/mediator_test.go +++ b/pp/go/storage/mediator/mediator_test.go @@ -10,6 +10,8 @@ import ( "github.com/stretchr/testify/suite" ) +// TODO moq -out mediator_moq_test.go -pkg mediator_test -rm . Timer + type MediatorSuite struct { suite.Suite } From 820ec16901436651ae6fcb99fe4e8873540b71f8 Mon Sep 17 00:00:00 2001 From: cherep58 Date: Wed, 24 Sep 2025 12:17:50 +0300 Subject: [PATCH 45/96] Head keeper (#154) * created HeadConvertingQueue * added timeInterval cache for HeadDataStorage * rewrite Keeper * changes in cppbridge/relabeler * added data loading/unloading * created unit tests for block/writer * created persistener * fixed data race in LoadAndQuerySeriesDataTask * fixed compilation error * added mock * fixed compilation error * fix adapter --- pp-pkg/storage/adapter.go | 8 +- pp/go/cppbridge/head.go | 32 +- pp/go/cppbridge/head_test.go | 7 +- pp/go/cppbridge/head_wal_test.go | 2 +- pp/go/relabeler/block/writer.go | 2 +- pp/go/relabeler/head/shard.go | 4 +- pp/go/relabeler/interface.go | 2 +- pp/go/storage/appender/appender_test.go | 2 +- pp/go/storage/block/block.go | 142 +++++ pp/go/storage/block/block_writer.go | 321 +++++++++++ pp/go/storage/block/writer.go | 337 +++-------- pp/go/storage/block/writer_test.go | 278 +++++++++ pp/go/storage/builder.go | 35 +- pp/go/storage/catalog/catalog.go | 19 +- pp/go/storage/catalog/record.go | 5 +- pp/go/storage/head/head/head_test.go | 2 +- pp/go/storage/head/keeper/keeper.go | 284 +++------- pp/go/storage/head/keeper/keeper_test.go | 115 ++++ pp/go/storage/head/services/functions.go | 24 + pp/go/storage/head/services/interface.go | 34 +- pp/go/storage/head/services/merger.go | 18 +- .../storage/head/services/mock/persistener.go | 72 +++ pp/go/storage/head/services/persistener.go | 237 ++++++++ .../storage/head/services/persistener_test.go | 387 +++++++++++++ pp/go/storage/head/shard/data_storage.go | 41 ++ pp/go/storage/head/shard/file_storage.go | 62 ++ .../shard/load_and_query_series_data_task.go | 35 ++ pp/go/storage/head/shard/shard.go | 111 +++- .../head/shard/unloaded_data_storage.go | 316 +++++++++++ .../head/shard/unloaded_data_storage_test.go | 529 ++++++++++++++++++ pp/go/storage/head/shard/wal/wal.go | 25 +- pp/go/storage/head/task/task.go | 7 +- pp/go/storage/head/task/waiter.go | 10 +- pp/go/storage/loader.go | 316 ++++++++--- pp/go/storage/loader_test.go | 393 +++++++++++++ pp/go/storage/manager.go | 7 +- pp/go/storage/querier/interface.go | 15 +- .../storage/querier/load_and_query_waiter.go | 48 ++ pp/go/storage/querier/querier.go | 39 +- pp/go/storage/querier/querier_test.go | 330 +++++++++-- pp/go/storage/storagetest/fixtures.go | 119 ++++ pp/go/util/closer.go | 14 +- 42 files changed, 4150 insertions(+), 636 deletions(-) create mode 100644 pp/go/storage/block/block.go create mode 100644 pp/go/storage/block/block_writer.go create mode 100644 pp/go/storage/block/writer_test.go create mode 100644 pp/go/storage/head/keeper/keeper_test.go create mode 100644 pp/go/storage/head/services/mock/persistener.go create mode 100644 pp/go/storage/head/services/persistener.go create mode 100644 pp/go/storage/head/services/persistener_test.go create mode 100644 pp/go/storage/head/shard/file_storage.go create mode 100644 pp/go/storage/head/shard/load_and_query_series_data_task.go create mode 100644 pp/go/storage/head/shard/unloaded_data_storage.go create mode 100644 pp/go/storage/head/shard/unloaded_data_storage_test.go create mode 100644 pp/go/storage/loader_test.go create mode 100644 pp/go/storage/querier/load_and_query_waiter.go create mode 100644 pp/go/storage/storagetest/fixtures.go diff --git a/pp-pkg/storage/adapter.go b/pp-pkg/storage/adapter.go index 16c2636407..f6e7dc01a9 100644 --- a/pp-pkg/storage/adapter.go +++ b/pp-pkg/storage/adapter.go @@ -169,7 +169,7 @@ func (ar *Adapter) ChunkQuerier(mint, maxt int64) (storage.ChunkQuerier, error) ahead := ar.proxy.Get() queriers = append( queriers, - querier.NewQuerier(ahead, querier.NewNoOpShardedDeduplicator, mint, maxt, nil, ar.activeQuerierMetrics), + querier.NewChunkQuerier(ahead, querier.NewNoOpShardedDeduplicator, mint, maxt, nil), ) for head := range ar.proxy.RangeQueriableHeads(mint, maxt) { @@ -179,7 +179,7 @@ func (ar *Adapter) ChunkQuerier(mint, maxt int64) (storage.ChunkQuerier, error) queriers = append( queriers, - querier.NewQuerier(head, querier.NewNoOpShardedDeduplicator, mint, maxt, nil, ar.storageQuerierMetrics), + querier.NewChunkQuerier(head, querier.NewNoOpShardedDeduplicator, mint, maxt, nil), ) } @@ -198,7 +198,7 @@ func (ar *Adapter) Close() error { } // HeadQuerier returns [storage.Querier] from active head. -func (ar *Adapter) HeadQuerier(ctx context.Context, mint, maxt int64) (storage.Querier, error) { +func (ar *Adapter) HeadQuerier(mint, maxt int64) (storage.Querier, error) { return querier.NewQuerier( ar.proxy.Get(), querier.NewNoOpShardedDeduplicator, @@ -210,7 +210,7 @@ func (ar *Adapter) HeadQuerier(ctx context.Context, mint, maxt int64) (storage.Q } // HeadStatus returns stats of Head. -func (ar *Adapter) HeadStatus(ctx context.Context, limit int) (querier.HeadStatus, error) { +func (ar *Adapter) HeadStatus(ctx context.Context, limit int) (*querier.HeadStatus, error) { return querier.QueryHeadStatus(ctx, ar.proxy.Get(), limit) } diff --git a/pp/go/cppbridge/head.go b/pp/go/cppbridge/head.go index d0f96a47ad..79e8d33c70 100644 --- a/pp/go/cppbridge/head.go +++ b/pp/go/cppbridge/head.go @@ -30,6 +30,17 @@ type TimeInterval struct { MaxT int64 } +func NewInvalidTimeInterval() TimeInterval { + return TimeInterval{ + MinT: math.MaxInt64, + MaxT: math.MinInt64, + } +} + +func (t *TimeInterval) IsInvalid() bool { + return t.MinT == math.MaxInt64 && t.MaxT == math.MinInt64 +} + type Sample struct { Timestamp int64 Value float64 @@ -39,6 +50,7 @@ type Sample struct { type HeadDataStorage struct { dataStorage uintptr gcDestroyDetector *uint64 + timeInterval TimeInterval } // NewHeadDataStorage - constructor. @@ -46,6 +58,7 @@ func NewHeadDataStorage() *HeadDataStorage { ds := &HeadDataStorage{ dataStorage: seriesDataDataStorageCtor(), gcDestroyDetector: &gcDestroyDetector, + timeInterval: NewInvalidTimeInterval(), } runtime.SetFinalizer(ds, func(ds *HeadDataStorage) { @@ -58,12 +71,16 @@ func NewHeadDataStorage() *HeadDataStorage { // Reset - resets data storage. func (ds *HeadDataStorage) Reset() { seriesDataDataStorageReset(ds.dataStorage) + ds.timeInterval = NewInvalidTimeInterval() } -func (ds *HeadDataStorage) TimeInterval() TimeInterval { - res := seriesDataDataStorageTimeInterval(ds.dataStorage) - runtime.KeepAlive(ds) - return res +func (ds *HeadDataStorage) TimeInterval(invalidateCache bool) TimeInterval { + if invalidateCache || ds.timeInterval.IsInvalid() { + ds.timeInterval = seriesDataDataStorageTimeInterval(ds.dataStorage) + runtime.KeepAlive(ds) + } + + return ds.timeInterval } func (ds *HeadDataStorage) GetQueriedSeriesBitset() []byte { @@ -237,7 +254,7 @@ type HeadDataStorageSerializedChunks struct { type HeadDataStorageSerializedChunkMetadata [SerializedChunkMetadataSize]byte -func (cm *HeadDataStorageSerializedChunkMetadata) SeriesID() uint32 { +func (cm HeadDataStorageSerializedChunkMetadata) SeriesID() uint32 { return *(*uint32)(unsafe.Pointer(&cm[0])) } @@ -257,6 +274,11 @@ func (r *HeadDataStorageSerializedChunks) Data() []byte { return r.data } +func (r *HeadDataStorageSerializedChunks) Metadata(chunkIndex int) HeadDataStorageSerializedChunkMetadata { + offset := Uint32Size + chunkIndex*SerializedChunkMetadataSize + return HeadDataStorageSerializedChunkMetadata(r.data[offset : offset+SerializedChunkMetadataSize]) +} + type HeadDataStorageSerializedChunkIndex struct { m map[uint32][]int } diff --git a/pp/go/cppbridge/head_test.go b/pp/go/cppbridge/head_test.go index 468f3f43a1..17962d957d 100644 --- a/pp/go/cppbridge/head_test.go +++ b/pp/go/cppbridge/head_test.go @@ -164,10 +164,15 @@ func (s *HeadSuite) TestTimeInterval() { encoder.Encode(1, 3, 1.0) // Act - timeInterval := dataStorage.TimeInterval() + timeInterval := dataStorage.TimeInterval(false) + encoder.Encode(1, 4, 1.0) + cachedTimeInterval := dataStorage.TimeInterval(false) + actualTimeInterval := dataStorage.TimeInterval(true) // Assert s.Equal(cppbridge.TimeInterval{MinT: 1, MaxT: 3}, timeInterval) + s.Equal(cppbridge.TimeInterval{MinT: 1, MaxT: 3}, cachedTimeInterval) + s.Equal(cppbridge.TimeInterval{MinT: 1, MaxT: 4}, actualTimeInterval) } func (s *HeadSuite) TestInstantQuery() { diff --git a/pp/go/cppbridge/head_wal_test.go b/pp/go/cppbridge/head_wal_test.go index d400d01bd6..e914fe4814 100644 --- a/pp/go/cppbridge/head_wal_test.go +++ b/pp/go/cppbridge/head_wal_test.go @@ -30,5 +30,5 @@ func TestHeadWalDecoder_DecodeToDataStorage(t *testing.T) { // Assert require.NoError(t, err) - require.Equal(t, cppbridge.TimeInterval{MinT: 1660828401000, MaxT: 1660828410000}, dataStorage.TimeInterval()) + require.Equal(t, cppbridge.TimeInterval{MinT: 1660828401000, MaxT: 1660828410000}, dataStorage.TimeInterval(false)) } diff --git a/pp/go/relabeler/block/writer.go b/pp/go/relabeler/block/writer.go index 5d84e2adeb..ce0a685b52 100644 --- a/pp/go/relabeler/block/writer.go +++ b/pp/go/relabeler/block/writer.go @@ -301,7 +301,7 @@ func (w *Writer) createWriters(shard relabeler.Shard) (blockWriters, error) { var writers blockWriters shard.DataStorageRLock() - timeInterval := shard.DataStorage().TimeInterval() + timeInterval := shard.DataStorage().TimeInterval(false) shard.DataStorageRUnlock() quantStart := (timeInterval.MinT / w.blockDurationMs) * w.blockDurationMs diff --git a/pp/go/relabeler/head/shard.go b/pp/go/relabeler/head/shard.go index f04302323a..c195b636c2 100644 --- a/pp/go/relabeler/head/shard.go +++ b/pp/go/relabeler/head/shard.go @@ -133,8 +133,8 @@ func (ds *DataStorage) CreateRevertableLoader(lss *cppbridge.LabelSetStorage, ls return ds.dataStorage.CreateRevertableLoader(lss, lsIdBatchSize) } -func (ds *DataStorage) TimeInterval() cppbridge.TimeInterval { - return ds.dataStorage.TimeInterval() +func (ds *DataStorage) TimeInterval(invalidateCache bool) cppbridge.TimeInterval { + return ds.dataStorage.TimeInterval(invalidateCache) } func (ds *DataStorage) GetQueriedSeriesBitset() []byte { diff --git a/pp/go/relabeler/interface.go b/pp/go/relabeler/interface.go index 945997b798..fc0b30b87f 100644 --- a/pp/go/relabeler/interface.go +++ b/pp/go/relabeler/interface.go @@ -23,7 +23,7 @@ type DataStorage interface { CreateUnusedSeriesDataUnloader() *cppbridge.UnusedSeriesDataUnloader CreateLoader(queriers []uintptr) *cppbridge.UnloadedDataLoader CreateRevertableLoader(lss *cppbridge.LabelSetStorage, lsIdBatchSize uint32) *cppbridge.UnloadedDataRevertableLoader - TimeInterval() cppbridge.TimeInterval + TimeInterval(invalidateCache bool) cppbridge.TimeInterval GetQueriedSeriesBitset() []byte } diff --git a/pp/go/storage/appender/appender_test.go b/pp/go/storage/appender/appender_test.go index cbf0eb1607..c96fef8e2c 100644 --- a/pp/go/storage/appender/appender_test.go +++ b/pp/go/storage/appender/appender_test.go @@ -14,7 +14,7 @@ func TestXxx(t *testing.T) { lss := &shard.LSS{} ds := shard.NewDataStorage() wl := &testWal{} - sd := shard.NewShard(lss, ds, wl, 0) + sd := shard.NewShard(lss, ds, nil, nil, wl, 0) id := "test-head-id" generation := uint64(0) diff --git a/pp/go/storage/block/block.go b/pp/go/storage/block/block.go new file mode 100644 index 0000000000..6ba726d949 --- /dev/null +++ b/pp/go/storage/block/block.go @@ -0,0 +1,142 @@ +package block + +import ( + "fmt" + "io" + "math" + "unsafe" + + "github.com/prometheus/prometheus/pp/go/cppbridge" + "github.com/prometheus/prometheus/tsdb/chunkenc" +) + +type Chunk struct { + rc *cppbridge.RecodedChunk +} + +func (c *Chunk) MinT() int64 { + return c.rc.MinT +} + +func (c *Chunk) MaxT() int64 { + return c.rc.MaxT +} + +func (c *Chunk) SeriesID() uint32 { + return c.rc.SeriesId +} + +func (c *Chunk) Encoding() chunkenc.Encoding { + return chunkenc.EncXOR +} + +func (c *Chunk) SampleCount() uint8 { + return c.rc.SamplesCount +} + +func (c *Chunk) Bytes() []byte { + return c.rc.ChunkData +} + +type ChunkIterator struct { + r *cppbridge.ChunkRecoder + rc *cppbridge.RecodedChunk +} + +func NewChunkIterator(lss *cppbridge.LabelSetStorage, lsIdBatchSize uint32, ds *cppbridge.HeadDataStorage, minT, maxT int64) ChunkIterator { + return ChunkIterator{r: cppbridge.NewChunkRecoder(lss, lsIdBatchSize, ds, cppbridge.TimeInterval{MinT: minT, MaxT: maxT})} +} + +func (i *ChunkIterator) Next() bool { + if i.rc != nil && !i.rc.HasMoreData { + return false + } + + rc := i.r.RecodeNextChunk() + i.rc = &rc + return rc.SeriesId != math.MaxUint32 +} + +func (i *ChunkIterator) NextBatch() bool { + i.rc.HasMoreData = i.r.NextBatch() + return i.rc.HasMoreData +} + +func (i *ChunkIterator) At() Chunk { + return Chunk{rc: i.rc} +} + +type IndexWriter struct { + cppIndexWriter *cppbridge.IndexWriter + isPrefixWritten bool +} + +func (iw *IndexWriter) WriteSeriesTo(id uint32, chunks []ChunkMetadata, w io.Writer) (n int64, err error) { + if !iw.isPrefixWritten { + var bytesWritten int + bytesWritten, err = w.Write(iw.cppIndexWriter.WriteHeader()) + n += int64(bytesWritten) + if err != nil { + return n, fmt.Errorf("failed to write header: %w", err) + } + + bytesWritten, err = w.Write(iw.cppIndexWriter.WriteSymbols()) + n += int64(bytesWritten) + if err != nil { + return n, fmt.Errorf("failed to write symbols: %w", err) + } + iw.isPrefixWritten = true + } + + bytesWritten, err := w.Write(iw.cppIndexWriter.WriteSeries(id, *(*[]cppbridge.ChunkMetadata)(unsafe.Pointer(&chunks)))) + n += int64(bytesWritten) + if err != nil { + return n, fmt.Errorf("failed to write series: %w", err) + } + + return n, nil +} + +func (iw *IndexWriter) WriteRestTo(w io.Writer) (n int64, err error) { + bytesWritten, err := w.Write(iw.cppIndexWriter.WriteLabelIndices()) + n += int64(bytesWritten) + if err != nil { + return n, fmt.Errorf("failed to write label indicies: %w", err) + } + + for { + data, hasMoreData := iw.cppIndexWriter.WriteNextPostingsBatch(1 << 20) + bytesWritten, err = w.Write(data) + if err != nil { + return n, fmt.Errorf("failed to write postings: %w", err) + } + n += int64(bytesWritten) + if !hasMoreData { + break + } + } + + bytesWritten, err = w.Write(iw.cppIndexWriter.WriteLabelIndicesTable()) + if err != nil { + return n, fmt.Errorf("failed to write label indicies table: %w", err) + } + n += int64(bytesWritten) + + bytesWritten, err = w.Write(iw.cppIndexWriter.WritePostingsTableOffsets()) + if err != nil { + return n, fmt.Errorf("failed to write posting table offsets: %w", err) + } + n += int64(bytesWritten) + + bytesWritten, err = w.Write(iw.cppIndexWriter.WriteTableOfContents()) + if err != nil { + return n, fmt.Errorf("failed to write table of content: %w", err) + } + n += int64(bytesWritten) + + return n, nil +} + +func NewIndexWriter(lss *cppbridge.LabelSetStorage) IndexWriter { + return IndexWriter{cppIndexWriter: cppbridge.NewIndexWriter(lss)} +} diff --git a/pp/go/storage/block/block_writer.go b/pp/go/storage/block/block_writer.go new file mode 100644 index 0000000000..0e8d695295 --- /dev/null +++ b/pp/go/storage/block/block_writer.go @@ -0,0 +1,321 @@ +package block + +import ( + "crypto/rand" + "encoding/json" + "fmt" + "math" + "os" + "path/filepath" + + "github.com/oklog/ulid" + "github.com/prometheus/prometheus/pp/go/storage/logger" + "github.com/prometheus/prometheus/pp/go/util" + "github.com/prometheus/prometheus/tsdb" + "github.com/prometheus/prometheus/tsdb/fileutil" +) + +const ( + tmpForCreationBlockDirSuffix = ".tmp-for-creation" + + indexFilename = "index" + metaFilename = "meta.json" + + metaVersion1 = 1 +) + +type WrittenBlock struct { + Dir string + Meta tsdb.BlockMeta +} + +func (block *WrittenBlock) ChunkDir() string { + return filepath.Join(block.Dir, "chunks") +} + +func (block *WrittenBlock) IndexFilename() string { + return filepath.Join(block.Dir, indexFilename) +} + +func (block *WrittenBlock) MetaFilename() string { + return filepath.Join(block.Dir, metaFilename) +} + +type blockWriter struct { + WrittenBlock + + chunkWriter *ChunkWriter + indexFileWriter *FileWriter + indexWriter IndexWriter + + chunkRecoder chunkRecoder +} + +func newBlockWriter(dir string, maxBlockChunkSegmentSize int64, indexWriter IndexWriter, chunkIterator ChunkIterator) (writer blockWriter, err error) { + uid := ulid.MustNew(ulid.Now(), rand.Reader) + writer.Dir = filepath.Join(dir, uid.String()) + tmpForCreationBlockDirSuffix + + if err = createTmpDir(writer.Dir); err != nil { + return + } + + if err = writer.createWriters(maxBlockChunkSegmentSize); err != nil { + return + } + + writer.Meta = tsdb.BlockMeta{ + ULID: uid, + MinTime: math.MaxInt64, + MaxTime: math.MinInt64, + Version: metaVersion1, + Compaction: tsdb.BlockMetaCompaction{ + Level: 1, + Sources: []ulid.ULID{uid}, + }, + } + + writer.indexWriter = indexWriter + writer.chunkRecoder = newChunkRecoder(chunkIterator) + return +} + +func (writer *blockWriter) createWriters(maxBlockChunkSegmentSize int64) error { + chunkWriter, err := NewChunkWriter(writer.ChunkDir(), maxBlockChunkSegmentSize) + if err != nil { + return fmt.Errorf("failed to create chunk writer: %w", err) + } + + indexFileWriter, err := NewFileWriter(writer.IndexFilename()) + if err != nil { + _ = chunkWriter.Close() + return fmt.Errorf("failed to create index file writer: %w", err) + } + + writer.chunkWriter = chunkWriter + writer.indexFileWriter = indexFileWriter + return nil +} + +func (writer *blockWriter) close() error { + return util.CloseAll(writer.chunkWriter, writer.indexFileWriter) +} + +func (writer *blockWriter) recodeAndWriteChunksBatch() error { + return writer.chunkRecoder.recode(writer.chunkWriter, &writer.Meta, writer.writeSeries) +} + +func (writer *blockWriter) writeRestOfRecodedChunks() error { + return writer.writeSeries(writer.chunkRecoder.previousSeriesID, writer.chunkRecoder.chunksMetadata) +} + +func (writer *blockWriter) writeSeries(seriesID uint32, chunksMetadata []ChunkMetadata) error { + if len(chunksMetadata) > 0 { + if _, err := writer.indexWriter.WriteSeriesTo(seriesID, chunksMetadata, writer.indexFileWriter); err != nil { + return fmt.Errorf("failed to write series %d: %w", seriesID, err) + } + } + + return nil +} + +func (writer *blockWriter) writeIndex() error { + if _, err := writer.indexWriter.WriteRestTo(writer.indexFileWriter); err != nil { + return fmt.Errorf("failed to write index: %w", err) + } + + writer.Meta.MaxTime += 1 + if _, err := writeBlockMetaFile(writer.MetaFilename(), &writer.Meta); err != nil { + return fmt.Errorf("failed to write block meta file: %w", err) + } + + return nil +} + +func (writer *blockWriter) moveTmpDirToDir() error { + if err := syncDir(writer.Dir); err != nil { + return fmt.Errorf("failed to sync temporary block dir: %w", err) + } + + dir := writer.Dir[:len(writer.Dir)-len(tmpForCreationBlockDirSuffix)] + + if err := fileutil.Replace(writer.Dir, dir); err != nil { + return fmt.Errorf("failed to move temporary block dir {%s} to {%s}: %w", writer.Dir, dir, err) + } + + writer.Dir = dir + return nil +} + +type blockWriters []blockWriter + +func (bw *blockWriters) append(writer blockWriter) { + *bw = append(*bw, writer) +} + +func (bw *blockWriters) close() { + for i := range *bw { + _ = (*bw)[i].close() + } +} + +func (bw *blockWriters) recodeAndWriteChunksBatch() error { + for i := range *bw { + if err := (*bw)[i].recodeAndWriteChunksBatch(); err != nil { + return err + } + } + + return nil +} + +func (bw *blockWriters) writeRestOfRecodedChunks() error { + for i := range *bw { + if err := (*bw)[i].writeRestOfRecodedChunks(); err != nil { + return err + } + } + + return nil +} + +func (bw *blockWriters) writeIndexAndMoveTmpDirToDir() ([]WrittenBlock, error) { + writtenBlocks := make([]WrittenBlock, 0, len(*bw)) + for i := range *bw { + if err := (*bw)[i].writeIndex(); err != nil { + return nil, err + } + + if err := (*bw)[i].moveTmpDirToDir(); err != nil { + return nil, err + } + + writtenBlocks = append(writtenBlocks, (*bw)[i].WrittenBlock) + } + + return writtenBlocks, nil +} + +type chunkRecoder struct { + chunkIterator ChunkIterator + chunksMetadata []ChunkMetadata + previousSeriesID uint32 +} + +func (recoder *chunkRecoder) recode( + chunkWriter *ChunkWriter, + blockMeta *tsdb.BlockMeta, + writeSeries func(seriesID uint32, chunksMetadata []ChunkMetadata) error, +) (err error) { + for recoder.chunkIterator.Next() { + chunk := recoder.chunkIterator.At() + + var chunkMetadata ChunkMetadata + if chunkMetadata, err = chunkWriter.Write(chunk); err != nil { + return fmt.Errorf("failed to write chunk: %w", err) + } + + adjustBlockMetaTimeRange(blockMeta, chunk.MinT(), chunk.MaxT()) + blockMeta.Stats.NumChunks++ + blockMeta.Stats.NumSamples += uint64(chunk.SampleCount()) + seriesID := chunk.SeriesID() + + if recoder.previousSeriesID == seriesID { + recoder.chunksMetadata = append(recoder.chunksMetadata, chunkMetadata) + } else { + if err = writeSeries(recoder.previousSeriesID, recoder.chunksMetadata); err != nil { + return err + } + blockMeta.Stats.NumSeries++ + recoder.chunksMetadata = append(recoder.chunksMetadata[:0], chunkMetadata) + recoder.previousSeriesID = seriesID + } + } + + recoder.chunkIterator.NextBatch() + return nil +} + +func newChunkRecoder(chunkIterator ChunkIterator) chunkRecoder { + return chunkRecoder{ + chunkIterator: chunkIterator, + previousSeriesID: math.MaxUint32, + } +} + +func adjustBlockMetaTimeRange(blockMeta *tsdb.BlockMeta, mint, maxt int64) { + if mint < blockMeta.MinTime { + blockMeta.MinTime = mint + } + + if maxt > blockMeta.MaxTime { + blockMeta.MaxTime = maxt + } +} + +func writeBlockMetaFile(fileName string, blockMeta *tsdb.BlockMeta) (int64, error) { + tmp := fileName + ".tmp" + defer func() { + if err := os.RemoveAll(tmp); err != nil { + logger.Errorf("failed to remove directory: %v", err) + } + }() + + metaFile, err := os.Create(tmp) + if err != nil { + return 0, fmt.Errorf("failed to create block meta file: %w", err) + } + defer func() { + if metaFile != nil { + if err = metaFile.Close(); err != nil { + logger.Errorf("failed to close metadata file: %v", err) + } + } + }() + + jsonBlockMeta, err := json.MarshalIndent(blockMeta, "", "\t") + if err != nil { + return 0, fmt.Errorf("failed to marshal meta json: %w", err) + } + + n, err := metaFile.Write(jsonBlockMeta) + if err != nil { + return 0, fmt.Errorf("failed to write meta json: %w", err) + } + + if err = metaFile.Sync(); err != nil { + return 0, fmt.Errorf("failed to sync meta file: %w", err) + } + + if err = metaFile.Close(); err != nil { + return 0, fmt.Errorf("faield to close meta file: %w", err) + } + metaFile = nil + + return int64(n), fileutil.Replace(tmp, fileName) +} + +func createTmpDir(dir string) error { + if err := os.RemoveAll(dir); err != nil { + return err + } + + return os.MkdirAll(dir, 0o777) +} + +func syncDir(dir string) error { + df, err := fileutil.OpenDir(dir) + if err != nil { + return err + } + defer func() { + if df != nil { + _ = df.Close() + } + }() + + if err = df.Sync(); err != nil { + return err + } + + return df.Close() +} diff --git a/pp/go/storage/block/writer.go b/pp/go/storage/block/writer.go index 1e9fc95f4e..3bceb88478 100644 --- a/pp/go/storage/block/writer.go +++ b/pp/go/storage/block/writer.go @@ -1,56 +1,47 @@ package block import ( - "crypto/rand" - "encoding/json" - "errors" - "fmt" - "io" - "math" - "os" - "path/filepath" "time" - "github.com/oklog/ulid" "github.com/prometheus/client_golang/prometheus" + "github.com/prometheus/prometheus/pp/go/cppbridge" + "github.com/prometheus/prometheus/pp/go/storage/head/shard" "github.com/prometheus/prometheus/pp/go/util" - "github.com/prometheus/prometheus/tsdb" - "github.com/prometheus/prometheus/tsdb/chunkenc" - "github.com/prometheus/prometheus/tsdb/fileutil" ) const ( // DefaultChunkSegmentSize is the default chunks segment size. DefaultChunkSegmentSize = 512 * 1024 * 1024 // DefaultBlockDuration is the default block duration. - DefaultBlockDuration = 2 * time.Hour - tmpForCreationBlockDirSuffix = ".tmp-for-creation" - indexFilename = "index" - metaFilename = "meta.json" - metaVersion1 = 1 + DefaultBlockDuration = 2 * time.Hour ) -type HBlockWriter[T any] interface { - Write(block T) error -} +var LsIdBatchSize uint32 = 100000 + +// Shard the minimum required head [Shard] implementation. +type Shard interface { + LSS() *shard.LSS + + DataStorage() *shard.DataStorage -var _ HBlockWriter[Block] = (*BlockWriter)(nil) + UnloadedDataStorage() *shard.UnloadedDataStorage +} -type BlockWriter struct { +type Writer[TShard Shard] struct { dataDir string maxBlockChunkSegmentSize int64 blockDurationMs int64 blockWriteDuration *prometheus.GaugeVec } -func NewBlockWriter( +func NewWriter[TShard Shard]( dataDir string, maxBlockChunkSegmentSize int64, blockDuration time.Duration, registerer prometheus.Registerer, -) *BlockWriter { +) *Writer[TShard] { factory := util.NewUnconflictRegisterer(registerer) - return &BlockWriter{ + return &Writer[TShard]{ dataDir: dataDir, maxBlockChunkSegmentSize: maxBlockChunkSegmentSize, blockDurationMs: blockDuration.Milliseconds(), @@ -61,263 +52,107 @@ func NewBlockWriter( } } -type Chunk interface { - MinT() int64 - MaxT() int64 - SeriesID() uint32 - Encoding() chunkenc.Encoding - SampleCount() uint8 - Bytes() []byte -} +func (w *Writer[TShard]) Write(shard TShard) (writtenBlocks []WrittenBlock, err error) { + _ = shard.LSS().WithRLock(func(_, _ *cppbridge.LabelSetStorage) error { + var writers blockWriters + writers, err = w.createWriters(shard) + if err != nil { + return err + } -type ChunkIterator interface { - Next() bool - At() Chunk -} + defer func() { + writers.close() + }() -type IndexWriter interface { - WriteSeriesTo(uint32, []ChunkMetadata, io.Writer) (int64, error) - WriteRestTo(io.Writer) (int64, error) -} + if err = w.recodeAndWriteChunks(shard, writers); err != nil { + return err + } -type Block interface { - TimeBounds() (minT, maxT int64) - ChunkIterator(minT, maxT int64) ChunkIterator - IndexWriter() IndexWriter + writtenBlocks, err = writers.writeIndexAndMoveTmpDirToDir() + return nil + }) + return } -func (w *BlockWriter) Write(block Block) error { - blockMinT, blockMaxT := block.TimeBounds() +func (w *Writer[TShard]) createWriters(shard TShard) (blockWriters, error) { + var writers blockWriters - quantStart := (blockMinT / w.blockDurationMs) * w.blockDurationMs - for ; quantStart <= blockMaxT; quantStart += w.blockDurationMs { + timeInterval := shard.DataStorage().TimeInterval(false) + + quantStart := (timeInterval.MinT / w.blockDurationMs) * w.blockDurationMs + for ; quantStart <= timeInterval.MaxT; quantStart += w.blockDurationMs { minT, maxT := quantStart, quantStart+w.blockDurationMs-1 - if minT < blockMinT { - minT = blockMinT - } - if maxT > blockMaxT { - maxT = blockMaxT + if minT < timeInterval.MinT { + minT = timeInterval.MinT } - if err := w.write(block, minT, maxT); err != nil { - return err + if maxT > timeInterval.MaxT { + maxT = timeInterval.MaxT } - } - return nil -} + var chunkIterator ChunkIterator + _ = shard.DataStorage().WithRLock(func(ds *cppbridge.HeadDataStorage) error { + chunkIterator = NewChunkIterator(shard.LSS().Target(), LsIdBatchSize, shard.DataStorage().Raw(), minT, maxT) + return nil + }) -func (w *BlockWriter) write(block Block, minT, maxT int64) (err error) { - start := time.Now() - uid := ulid.MustNew(ulid.Now(), rand.Reader) - dir := filepath.Join(w.dataDir, uid.String()) - tmp := dir + tmpForCreationBlockDirSuffix - var closers []io.Closer - defer func() { - err = errors.Join(err, closeAll(closers...)) - if cleanUpErr := os.RemoveAll(tmp); err != nil { - // todo: log error - _ = cleanUpErr + if writer, err := newBlockWriter(w.dataDir, w.maxBlockChunkSegmentSize, NewIndexWriter(shard.LSS().Target()), chunkIterator); err == nil { + writers.append(writer) + } else { + writers.close() + return blockWriters{}, err } - }() - - if err = os.RemoveAll(tmp); err != nil { - return fmt.Errorf("failed to cleanup tmp directory {%s}: %w", tmp, err) - } - - if err = os.MkdirAll(tmp, 0o777); err != nil { - return fmt.Errorf("failed to create tmp directory {%s}: %w", tmp, err) } - chunkw, err := NewChunkWriter(chunkDir(tmp), w.maxBlockChunkSegmentSize) - if err != nil { - return fmt.Errorf("failed to create chunk writer: %w", err) - } - closers = append(closers, chunkw) - - indexFileWriter, err := NewFileWriter(filepath.Join(tmp, indexFilename)) - if err != nil { - return fmt.Errorf("failed to create index file writer: %w", err) - } - - closers = append(closers, indexFileWriter) - indexWriter := block.IndexWriter() - - chunkIterator := block.ChunkIterator(minT, maxT) - var chunksMetadata []ChunkMetadata - var chunkMetadata ChunkMetadata - var previousSeriesID uint32 = math.MaxUint32 - var chunk Chunk + return writers, nil +} - writeSeries := func() error { - if len(chunksMetadata) == 0 { - return nil - } - _, err = indexWriter.WriteSeriesTo(previousSeriesID, chunksMetadata, indexFileWriter) - if err != nil { - return fmt.Errorf("failed to write series %d: %w", previousSeriesID, err) - } +func (w *Writer[TShard]) recodeAndWriteChunks(shard TShard, writers blockWriters) error { + var loader *cppbridge.UnloadedDataRevertableLoader + _ = shard.DataStorage().WithRLock(func(ds *cppbridge.HeadDataStorage) error { + loader = shard.DataStorage().CreateRevertableLoader(shard.LSS().Target(), LsIdBatchSize) return nil - } - - blockMeta := &tsdb.BlockMeta{ - ULID: uid, - MinTime: math.MaxInt64, - MaxTime: math.MinInt64, - Version: metaVersion1, - Compaction: tsdb.BlockMetaCompaction{ - Level: 1, - Sources: []ulid.ULID{uid}, - }, - } + }) - var hasChunks bool - for chunkIterator.Next() { - hasChunks = true - chunk = chunkIterator.At() - chunkMetadata, err = chunkw.Write(chunk) - if err != nil { - return fmt.Errorf("failed to write chunk: %w", err) - } - - adjustBlockMetaTimeRange(blockMeta, chunk.MinT(), chunk.MaxT()) - blockMeta.Stats.NumChunks++ - blockMeta.Stats.NumSamples += uint64(chunk.SampleCount()) - seriesID := chunk.SeriesID() + isFirstBatch := true - if previousSeriesID == seriesID { - chunksMetadata = append(chunksMetadata, chunkMetadata) + loadData := func() (bool, error) { + if isFirstBatch { + isFirstBatch = false } else { - if err = writeSeries(); err != nil { - return err + if !loader.NextBatch() { + return false, nil } - blockMeta.Stats.NumSeries++ - chunksMetadata = append(chunksMetadata[:0], chunkMetadata) - previousSeriesID = seriesID } - } - if !hasChunks { - return nil - } - - if err = writeSeries(); err != nil { - return err - } - indexFileSize, err := indexWriter.WriteRestTo(indexFileWriter) - if err != nil { - return fmt.Errorf("failed to write index: %w", err) - } - // todo: logs & metrics - _ = indexFileSize - - // write meta - blockMeta.MaxTime += 1 - metaFileSize, err := writeBlockMetaFile(filepath.Join(tmp, metaFilename), blockMeta) - if err != nil { - return fmt.Errorf("failed to write block meta file: %w", err) - } - // todo: log & metrics - _ = metaFileSize - - closeErr := err - for _, closer := range closers { - closeErr = errors.Join(closeErr, closer.Close()) - } - closers = closers[:0] - - if closeErr != nil { - return closeErr - } - - var df *os.File - df, err = fileutil.OpenDir(tmp) - if err != nil { - return fmt.Errorf("failed to open temporary block dir: %w", err) - } - defer func() { - if df != nil { - _ = df.Close() + if shard.UnloadedDataStorage() == nil { + return true, nil } - }() - - if err = df.Sync(); err != nil { - return fmt.Errorf("failed to sync temporary block dir: %w", err) - } - - if err = df.Close(); err != nil { - return fmt.Errorf("failed to close temporary block dir: %w", err) - } - df = nil - if err = fileutil.Replace(tmp, dir); err != nil { - return fmt.Errorf("failed to move temporary block dir {%s} to {%s}: %w", tmp, dir, err) + return true, shard.UnloadedDataStorage().ForEachSnapshot(loader.Load) } - w.blockWriteDuration.With(prometheus.Labels{ - "block_id": blockMeta.ULID.String(), - }).Set(float64(time.Since(start).Milliseconds())) - - return -} - -func chunkDir(dir string) string { return filepath.Join(dir, "chunks") } - -func closeAll(closers ...io.Closer) error { - errs := make([]error, len(closers)) - for i, closer := range closers { - errs[i] = closer.Close() - } - return errors.Join(errs...) -} - -func adjustBlockMetaTimeRange(blockMeta *tsdb.BlockMeta, mint, maxt int64) { - if mint < blockMeta.MinTime { - blockMeta.MinTime = mint - } - - if maxt > blockMeta.MaxTime { - blockMeta.MaxTime = maxt - } -} + for { + var hasMoreData bool + var err error + _ = shard.DataStorage().WithLock(func(ds *cppbridge.HeadDataStorage) error { + hasMoreData, err = loadData() + return nil + }) -func writeBlockMetaFile(fileName string, blockMeta *tsdb.BlockMeta) (int64, error) { - tmp := fileName + ".tmp" - defer func() { - if err := os.RemoveAll(tmp); err != nil { - // todo: log error + if !hasMoreData { + break } - }() - metaFile, err := os.Create(tmp) - if err != nil { - return 0, fmt.Errorf("failed to create block meta file: %w", err) - } - defer func() { - if metaFile != nil { - if err = metaFile.Close(); err != nil { - // todo: log error - } + if err != nil { + return err } - }() - - jsonBlockMeta, err := json.MarshalIndent(blockMeta, "", "\t") - if err != nil { - return 0, fmt.Errorf("failed to marshal meta json: %w", err) - } - - n, err := metaFile.Write(jsonBlockMeta) - if err != nil { - return 0, fmt.Errorf("failed to write meta json: %w", err) - } - - if err = metaFile.Sync(); err != nil { - return 0, fmt.Errorf("failed to sync meta file: %w", err) - } - if err = metaFile.Close(); err != nil { - return 0, fmt.Errorf("faield to close meta file: %w", err) + if err = shard.DataStorage().WithRLock(func(ds *cppbridge.HeadDataStorage) error { + return writers.recodeAndWriteChunksBatch() + }); err != nil { + return err + } } - metaFile = nil - return int64(n), fileutil.Replace(tmp, fileName) + return writers.writeRestOfRecodedChunks() } diff --git a/pp/go/storage/block/writer_test.go b/pp/go/storage/block/writer_test.go new file mode 100644 index 0000000000..572d195b4d --- /dev/null +++ b/pp/go/storage/block/writer_test.go @@ -0,0 +1,278 @@ +package block_test + +import ( + "encoding/json" + "os" + "path/filepath" + "testing" + "time" + + "github.com/jonboulle/clockwork" + "github.com/oklog/ulid" + "github.com/prometheus/client_golang/prometheus" + "github.com/prometheus/prometheus/model/labels" + "github.com/prometheus/prometheus/pp/go/cppbridge" + "github.com/prometheus/prometheus/pp/go/storage" + "github.com/prometheus/prometheus/pp/go/storage/block" + "github.com/prometheus/prometheus/pp/go/storage/catalog" + "github.com/prometheus/prometheus/pp/go/storage/head/services" + "github.com/prometheus/prometheus/pp/go/storage/storagetest" + "github.com/prometheus/prometheus/tsdb" + "github.com/stretchr/testify/suite" +) + +const ( + numberOfShards uint16 = 1 + + maxSegmentSize uint32 = 1024 + + unloadDataStorageInterval time.Duration = time.Second +) + +type WriterSuite struct { + suite.Suite + dataDir string + head *storage.HeadOnDisk + blockWriter *block.Writer[*storage.ShardOnDisk] +} + +func TestWriterSuite(t *testing.T) { + suite.Run(t, new(WriterSuite)) +} + +func (s *WriterSuite) SetupTest() { + s.dataDir = s.createDataDirectory() + s.head = s.mustCreateHead() + s.blockWriter = block.NewWriter[*storage.ShardOnDisk]( + s.dataDir, + block.DefaultChunkSegmentSize, + 2*time.Hour, + prometheus.DefaultRegisterer, + ) +} + +func (s *WriterSuite) createDataDirectory() string { + dataDir := filepath.Join(s.T().TempDir(), "data") + s.Require().NoError(os.MkdirAll(dataDir, os.ModeDir)) + return dataDir +} + +func (s *WriterSuite) mustCreateCatalog() *catalog.Catalog { + l, err := catalog.NewFileLogV2(filepath.Join(s.dataDir, "catalog.log")) + s.Require().NoError(err) + + c, err := catalog.New( + clockwork.NewFakeClock(), + l, + &catalog.DefaultIDGenerator{}, + catalog.DefaultMaxLogFileSize, + nil, + ) + s.Require().NoError(err) + + return c +} + +func (s *WriterSuite) mustCreateHead() *storage.HeadOnDisk { + h, err := storage.NewBuilder( + s.mustCreateCatalog(), + s.dataDir, + maxSegmentSize, + prometheus.DefaultRegisterer, + unloadDataStorageInterval, + ).Build(0, numberOfShards) + s.Require().NoError(err) + return h +} + +func (s *WriterSuite) mustReadFile(filename string) []byte { + b, err := os.ReadFile(filename) + s.Require().NoError(err) + return b +} + +func (s *WriterSuite) mustReadBlockMeta(filename string) tsdb.BlockMeta { + var meta tsdb.BlockMeta + s.NoError(json.Unmarshal(s.mustReadFile(filename), &meta)) + return meta +} + +func (s *WriterSuite) shard() *storage.ShardOnDisk { + for shard := range s.head.RangeShards() { + return shard + } + + return nil +} + +func (s *WriterSuite) fillHead() { + ts := time.UnixMilli(1753805651969) + storagetest.MustAppendTimeSeries(&s.Suite, s.head, []storagetest.TimeSeries{ + { + Labels: labels.FromStrings("__name__", "value1"), + Samples: []cppbridge.Sample{ + {Timestamp: ts.UnixMilli(), Value: 0}, + {Timestamp: ts.Add(time.Minute).UnixMilli(), Value: 1}, + {Timestamp: ts.Add(time.Hour * 2).UnixMilli(), Value: 2}, + }, + }, + { + Labels: labels.FromStrings("__name__", "value2"), + Samples: []cppbridge.Sample{ + {Timestamp: ts.UnixMilli(), Value: 0}, + {Timestamp: ts.Add(time.Minute).UnixMilli(), Value: 1}, + {Timestamp: ts.Add(time.Hour * 2).UnixMilli(), Value: 2}, + }, + }, + }) +} + +func (s *WriterSuite) assertWrittenBlocks(blocks []block.WrittenBlock, err error) { + s.Require().NoError(err) + + s.Require().Equal(2, len(blocks)) + + meta1 := s.mustReadBlockMeta(blocks[0].MetaFilename()) + s.Equal(tsdb.BlockMeta{ + MinTime: 1753805651969, + MaxTime: 1753805711970, + Stats: tsdb.BlockStats{ + NumSamples: 4, + NumSeries: 2, + NumChunks: 2, + }, + ULID: meta1.ULID, + Compaction: tsdb.BlockMetaCompaction{ + Level: 1, + Sources: []ulid.ULID{meta1.ULID}, + }, + Version: 1, + }, meta1) + + meta2 := s.mustReadBlockMeta(blocks[1].MetaFilename()) + s.Equal(tsdb.BlockMeta{ + MinTime: 1753812851969, + MaxTime: 1753812851970, + Stats: tsdb.BlockStats{ + NumSamples: 2, + NumSeries: 2, + NumChunks: 2, + }, + ULID: meta2.ULID, + Compaction: tsdb.BlockMetaCompaction{ + Level: 1, + Sources: []ulid.ULID{meta2.ULID}, + }, + Version: 1, + }, meta2) + + s.Equal([]byte{ + 0xba, 0xaa, 0xd7, 0x00, 0x02, 0x00, 0x00, 0x00, 0x1c, 0x00, 0x00, 0x00, 0x04, 0x00, 0x08, 0x5f, + 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x5f, 0x5f, 0x06, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x31, 0x06, 0x76, + 0x61, 0x6c, 0x75, 0x65, 0x32, 0x4b, 0x1c, 0x0b, 0x1f, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, + 0x0e, 0x01, 0x01, 0x02, 0x01, 0x82, 0x80, 0xb4, 0xef, 0x8a, 0x66, 0xe0, 0xd4, 0x03, 0x08, 0xfc, + 0x04, 0xb8, 0xde, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, + 0x0e, 0x01, 0x01, 0x03, 0x01, 0x82, 0x80, 0xb4, 0xef, 0x8a, 0x66, 0xe0, 0xd4, 0x03, 0x24, 0xa0, + 0x8b, 0x2d, 0xe0, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, + 0x00, 0x00, 0x00, 0x10, 0x00, 0x00, 0x00, 0x01, 0x00, 0x00, 0x00, 0x02, 0x00, 0x00, 0x00, 0x02, + 0x00, 0x00, 0x00, 0x03, 0xfc, 0x41, 0x54, 0x42, 0x00, 0x00, 0x00, 0x0c, 0x00, 0x00, 0x00, 0x02, + 0x00, 0x00, 0x00, 0x03, 0x00, 0x00, 0x00, 0x05, 0xbb, 0x33, 0xcb, 0xd4, 0x00, 0x00, 0x00, 0x08, + 0x00, 0x00, 0x00, 0x01, 0x00, 0x00, 0x00, 0x03, 0xa7, 0x69, 0x2e, 0xd2, 0x00, 0x00, 0x00, 0x08, + 0x00, 0x00, 0x00, 0x01, 0x00, 0x00, 0x00, 0x05, 0x81, 0xc8, 0xc9, 0x3a, 0x00, 0x00, 0x00, 0x0f, + 0x00, 0x00, 0x00, 0x01, 0x01, 0x08, 0x5f, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x5f, 0x5f, 0x70, 0x10, + 0x86, 0x91, 0x1b, 0x00, 0x00, 0x00, 0x2f, 0x00, 0x00, 0x00, 0x03, 0x02, 0x00, 0x00, 0x88, 0x01, + 0x02, 0x08, 0x5f, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x5f, 0x5f, 0x06, 0x76, 0x61, 0x6c, 0x75, 0x65, + 0x31, 0x9c, 0x01, 0x02, 0x08, 0x5f, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x5f, 0x5f, 0x06, 0x76, 0x61, + 0x6c, 0x75, 0x65, 0x32, 0xac, 0x01, 0xb4, 0x22, 0x64, 0x7c, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, + 0x00, 0x05, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x29, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, + 0x00, 0x70, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0xbc, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, + 0x00, 0x88, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0xd3, 0x13, 0xb6, 0x16, 0x2c, + }, s.mustReadFile(blocks[0].IndexFilename())) + + s.Equal([]byte{ + 0xba, 0xaa, 0xd7, 0x00, 0x02, 0x00, 0x00, 0x00, 0x1c, 0x00, 0x00, 0x00, 0x04, 0x00, 0x08, 0x5f, + 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x5f, 0x5f, 0x06, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x31, 0x06, 0x76, + 0x61, 0x6c, 0x75, 0x65, 0x32, 0x4b, 0x1c, 0x0b, 0x1f, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, + 0x0c, 0x01, 0x01, 0x02, 0x01, 0x82, 0xf4, 0xa2, 0xf6, 0x8a, 0x66, 0x00, 0x08, 0xbe, 0xfa, 0x4f, + 0x7a, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, + 0x0c, 0x01, 0x01, 0x03, 0x01, 0x82, 0xf4, 0xa2, 0xf6, 0x8a, 0x66, 0x00, 0x1e, 0x43, 0x53, 0x14, + 0x58, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, + 0x00, 0x00, 0x00, 0x10, 0x00, 0x00, 0x00, 0x01, 0x00, 0x00, 0x00, 0x02, 0x00, 0x00, 0x00, 0x02, + 0x00, 0x00, 0x00, 0x03, 0xfc, 0x41, 0x54, 0x42, 0x00, 0x00, 0x00, 0x0c, 0x00, 0x00, 0x00, 0x02, + 0x00, 0x00, 0x00, 0x03, 0x00, 0x00, 0x00, 0x05, 0xbb, 0x33, 0xcb, 0xd4, 0x00, 0x00, 0x00, 0x08, + 0x00, 0x00, 0x00, 0x01, 0x00, 0x00, 0x00, 0x03, 0xa7, 0x69, 0x2e, 0xd2, 0x00, 0x00, 0x00, 0x08, + 0x00, 0x00, 0x00, 0x01, 0x00, 0x00, 0x00, 0x05, 0x81, 0xc8, 0xc9, 0x3a, 0x00, 0x00, 0x00, 0x0f, + 0x00, 0x00, 0x00, 0x01, 0x01, 0x08, 0x5f, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x5f, 0x5f, 0x70, 0x10, + 0x86, 0x91, 0x1b, 0x00, 0x00, 0x00, 0x2f, 0x00, 0x00, 0x00, 0x03, 0x02, 0x00, 0x00, 0x88, 0x01, + 0x02, 0x08, 0x5f, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x5f, 0x5f, 0x06, 0x76, 0x61, 0x6c, 0x75, 0x65, + 0x31, 0x9c, 0x01, 0x02, 0x08, 0x5f, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x5f, 0x5f, 0x06, 0x76, 0x61, + 0x6c, 0x75, 0x65, 0x32, 0xac, 0x01, 0xb4, 0x22, 0x64, 0x7c, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, + 0x00, 0x05, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x29, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, + 0x00, 0x70, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0xbc, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, + 0x00, 0x88, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0xd3, 0x13, 0xb6, 0x16, 0x2c, + }, s.mustReadFile(blocks[1].IndexFilename())) + + s.Equal([]byte{ + 0x85, 0xBD, 0x40, 0xDD, 0x01, 0x00, 0x00, 0x00, 0x16, 0x01, 0x00, 0x02, 0x82, 0x80, 0xB4, 0xEF, + 0x8A, 0x66, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0xE0, 0xD4, 0x03, 0xC4, 0x57, 0xFE, + 0xC0, 0xEE, 0xF0, 0x5E, 0x16, 0x01, 0x00, 0x02, 0x82, 0x80, 0xB4, 0xEF, 0x8A, 0x66, 0x00, 0x00, + 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0xE0, 0xD4, 0x03, 0xC4, 0x57, 0xFE, 0xC0, 0xEE, 0xF0, 0x5E, + }, s.mustReadFile(filepath.Join(blocks[0].ChunkDir(), "000000"))) + + s.Equal([]byte{ + 0x85, 0xBD, 0x40, 0xDD, 0x01, 0x00, 0x00, 0x00, 0x10, 0x01, 0x00, 0x01, 0x82, 0xF4, 0xA2, 0xF6, + 0x8A, 0x66, 0x40, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x29, 0x21, 0xAB, 0xF9, 0x10, 0x01, + 0x00, 0x01, 0x82, 0xF4, 0xA2, 0xF6, 0x8A, 0x66, 0x40, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, + 0x29, 0x21, 0xAB, 0xF9, + }, s.mustReadFile(filepath.Join(blocks[1].ChunkDir(), "000000"))) +} + +func (s *WriterSuite) TestWrite() { + // Arrange + s.fillHead() + + // Act + block.LsIdBatchSize = 2 + blocks, err := s.blockWriter.Write(s.shard()) + + // Assert + s.assertWrittenBlocks(blocks, err) +} + +func (s *WriterSuite) TestWriteInBatches() { + // Arrange + s.fillHead() + + // Act + block.LsIdBatchSize = 1 + blocks, err := s.blockWriter.Write(s.shard()) + + // Assert + s.assertWrittenBlocks(blocks, err) +} + +func (s *WriterSuite) TestWriteWithDataUnloading() { + // Arrange + s.fillHead() + s.Require().NoError(services.UnloadUnusedSeriesDataWithHead(s.head)) + + // Act + block.LsIdBatchSize = 2 + blocks, err := s.blockWriter.Write(s.shard()) + + // Assert + s.assertWrittenBlocks(blocks, err) +} + +func (s *WriterSuite) TestWriteWithDataUnloadingInBatches() { + // Arrange + s.fillHead() + s.Require().NoError(services.UnloadUnusedSeriesDataWithHead(s.head)) + + // Act + block.LsIdBatchSize = 1 + blocks, err := s.blockWriter.Write(s.shard()) + + // Assert + s.assertWrittenBlocks(blocks, err) +} diff --git a/pp/go/storage/builder.go b/pp/go/storage/builder.go index 8625df676f..488fcecc2e 100644 --- a/pp/go/storage/builder.go +++ b/pp/go/storage/builder.go @@ -5,6 +5,7 @@ import ( "fmt" "os" "path/filepath" + "time" "github.com/prometheus/client_golang/prometheus" "github.com/prometheus/prometheus/pp/go/cppbridge" @@ -21,10 +22,11 @@ import ( // Builder building new [HeadOnDisk] with parameters. type Builder struct { - catalog *catalog.Catalog - dataDir string - maxSegmentSize uint32 - registerer prometheus.Registerer + catalog *catalog.Catalog + dataDir string + maxSegmentSize uint32 + registerer prometheus.Registerer + unloadDataStorageInterval time.Duration } // NewBuilder init new [Builder]. @@ -33,12 +35,14 @@ func NewBuilder( dataDir string, maxSegmentSize uint32, registerer prometheus.Registerer, + unloadDataStorageInterval time.Duration, ) *Builder { return &Builder{ - catalog: hcatalog, - dataDir: dataDir, - maxSegmentSize: maxSegmentSize, - registerer: registerer, + catalog: hcatalog, + dataDir: dataDir, + maxSegmentSize: maxSegmentSize, + registerer: registerer, + unloadDataStorageInterval: unloadDataStorageInterval, } } @@ -87,7 +91,8 @@ func (b *Builder) createShardOnDisk( swn *writer.SegmentWriteNotifier, shardID uint16, ) (*ShardOnDisk, error) { - shardFile, err := os.Create(filepath.Join(filepath.Clean(headDir), fmt.Sprintf("shard_%d.wal", shardID))) + headDir = filepath.Clean(headDir) + shardFile, err := os.Create(GetShardWalFilename(headDir, shardID)) if err != nil { return nil, fmt.Errorf("failed to create shard wal file id %d: %w", shardID, err) } @@ -113,9 +118,21 @@ func (b *Builder) createShardOnDisk( return nil, fmt.Errorf("failed to create buffered writer shard id %d: %w", shardID, err) } + var unloadedDataStorage *shard.UnloadedDataStorage + var queriedSeriesStorage *shard.QueriedSeriesStorage + if b.unloadDataStorageInterval != 0 { + unloadedDataStorage = shard.NewUnloadedDataStorage(shard.NewFileStorage(GetUnloadedDataStorageFilename(headDir, shardID))) + queriedSeriesStorage = shard.NewQueriedSeriesStorage( + shard.NewFileStorage(GetQueriedSeriesStorageFilename(headDir, shardID, 0)), + shard.NewFileStorage(GetQueriedSeriesStorageFilename(headDir, shardID, 1)), + ) + } + return shard.NewShard( lss, shard.NewDataStorage(), + unloadedDataStorage, + queriedSeriesStorage, wal.NewWal(shardWalEncoder, sw, b.maxSegmentSize), shardID, ), nil diff --git a/pp/go/storage/catalog/catalog.go b/pp/go/storage/catalog/catalog.go index 8ded7b67b9..6eee902384 100644 --- a/pp/go/storage/catalog/catalog.go +++ b/pp/go/storage/catalog/catalog.go @@ -195,6 +195,19 @@ func (c *Catalog) Get(id string) (*Record, error) { // List returns slice of records with filter and sort. func (c *Catalog) List(filterFn func(record *Record) bool, sortLess func(lhs, rhs *Record) bool) []*Record { + records := c.list(filterFn) + + if sortLess != nil { + sort.Slice(records, func(i, j int) bool { + return sortLess(records[i], records[j]) + }) + } + + return records +} + +// list returns slice of filtered records +func (c *Catalog) list(filterFn func(record *Record) bool) []*Record { c.mtx.Lock() defer c.mtx.Unlock() @@ -206,12 +219,6 @@ func (c *Catalog) List(filterFn func(record *Record) bool, sortLess func(lhs, rh records = append(records, record) } - if sortLess != nil { - sort.Slice(records, func(i, j int) bool { - return sortLess(records[i], records[j]) - }) - } - return records } diff --git a/pp/go/storage/catalog/record.go b/pp/go/storage/catalog/record.go index e2ac07a984..12b89ed828 100644 --- a/pp/go/storage/catalog/record.go +++ b/pp/go/storage/catalog/record.go @@ -5,7 +5,6 @@ import ( "sync/atomic" "github.com/google/uuid" - "github.com/prometheus/prometheus/pp/go/util/optional" ) @@ -209,3 +208,7 @@ func applyRecordChanges(r, changed *Record) { r.mint = changed.mint r.maxt = changed.maxt } + +func LessByUpdateAt(lhs, rhs *Record) bool { + return lhs.UpdatedAt() < rhs.UpdatedAt() +} diff --git a/pp/go/storage/head/head/head_test.go b/pp/go/storage/head/head/head_test.go index 4b8d9f32ab..994808d147 100644 --- a/pp/go/storage/head/head/head_test.go +++ b/pp/go/storage/head/head/head_test.go @@ -12,7 +12,7 @@ func TestXxx(t *testing.T) { lss := &shard.LSS{} ds := shard.NewDataStorage() wl := &testWal{} - sd := shard.NewShard(lss, ds, wl, 0) + sd := shard.NewShard(lss, ds, nil, nil, wl, 0) id := "test-head-id" generation := uint64(0) diff --git a/pp/go/storage/head/keeper/keeper.go b/pp/go/storage/head/keeper/keeper.go index 0f01510350..f649bb88c6 100644 --- a/pp/go/storage/head/keeper/keeper.go +++ b/pp/go/storage/head/keeper/keeper.go @@ -1,243 +1,145 @@ package keeper import ( + "container/heap" + "errors" "sync" - "sync/atomic" "time" - - "github.com/jonboulle/clockwork" - "github.com/prometheus/client_golang/prometheus" - "github.com/prometheus/prometheus/pp/go/relabeler/querier" - "github.com/prometheus/prometheus/pp/go/storage/logger" - "github.com/prometheus/prometheus/pp/go/util" ) -// type Block interface { -// TimeBounds() (minT, maxT int64) -// // ChunkIterator(minT, maxT int64) ChunkIterator -// // IndexWriter() IndexWriter -// } - const ( - writeRetryTimeout = 5 * time.Minute - maxAddIter uint32 = 5 -) + MinHeadConvertingQueueSize = 2 -const ( - // BlockWrite name of task. - BlockWrite = "block_write" + Add = 0 + AddWithReplace = 1 ) -// HeadBlockWriter writes block on disk from [Head]. -type HeadBlockWriter[TBlock any] interface { - Write(block TBlock) error -} +var ( + ErrorNoSlots error = errors.New("keeper has no slots") +) -type WriteNotifier interface { - NotifyWritten() -} +type AddPolicy = uint8 -// GenericTask the minimum required task [Generic] implementation. -type GenericTask interface { - // Wait for the task to complete on all shards. - Wait() error +type sortableHead[THead any] struct { + head THead + createdAt time.Duration } -// -// DataStorage -// +type headSortedSlice[THead any] []sortableHead[THead] -// DataStorage the minimum required [DataStorage] implementation. -type DataStorage interface { - // TODO +func (q *headSortedSlice[THead]) Len() int { + return len(*q) } -// -// LSS -// - -// LSS the minimum required [LSS] implementation. -type LSS interface { - // TODO +func (q *headSortedSlice[THead]) Less(i, j int) bool { + return (*q)[i].createdAt < (*q)[j].createdAt } -// -// Shard -// - -// Shard the minimum required head [Shard] implementation. -type Shard[TDataStorage DataStorage, TLSS LSS] interface { - // DataStorage returns shard [DataStorage]. - DataStorage() TDataStorage - - // LSS returns shard labelset storage [LSS]. - LSS() TLSS +func (q *headSortedSlice[THead]) Swap(i, j int) { + (*q)[i], (*q)[j] = (*q)[j], (*q)[i] } -// -// Head -// - -// Head the minimum required [Head] implementation. -type Head[ - TGenericTask GenericTask, - TDataStorage DataStorage, - TLSS LSS, - TShard Shard[TDataStorage, TLSS], -] interface { - // CreateTask create a task for operations on the [Head] shards. - CreateTask(taskName string, shardFn func(shard TShard) error) TGenericTask +func (q *headSortedSlice[THead]) Push(head any) { + *q = append(*q, head.(sortableHead[THead])) +} - // Enqueue the task to be executed on shards [Head]. - Enqueue(t TGenericTask) +func (q *headSortedSlice[THead]) Pop() any { + n := len(*q) + item := (*q)[n-1] + *q = (*q)[0 : n-1] + return item +} +type Head interface { // ID returns id [Head]. ID() string - // String serialize as string. - String() string + // Close closes wals, query semaphore for the inability to get query and clear metrics. + Close() error } -type HeadBlockBuilder[TBlock any] func() TBlock - -type Keeper[ - TGenericTask GenericTask, - TDataStorage DataStorage, - TLSS LSS, - TShard Shard[TDataStorage, TLSS], - THead Head[TGenericTask, TDataStorage, TLSS, TShard], - TBlock any, -] struct { - hbWriter HeadBlockWriter[TBlock] - headBlockBuilder HeadBlockBuilder[TBlock] - - writeNotifier WriteNotifier - mtx sync.Mutex - heads []THead - headRetentionTimeout time.Duration - - writeTimer clockwork.Timer - writeTimeout time.Duration - addCount uint32 - closer *util.Closer - - clock clockwork.Clock - maxRetentionDuration time.Duration - headPersistenceDuration prometheus.Histogram - querierMetrics *querier.Metrics +type Keeper[THead Head] struct { + heads headSortedSlice[THead] + lock sync.Mutex } -func (k *Keeper[TGenericTask, TDataStorage, TLSS, TShard, THead, TBlock]) Add(head THead) { - k.mtx.Lock() - k.heads = append(k.heads, head) - logger.Infof("QUERYABLE STORAGE: head %s added", head.String()) - k.mtx.Unlock() - - if atomic.AddUint32(&k.addCount, 1) < maxAddIter { - k.writeTimer.Reset(k.writeTimeout) +func NewKeeper[THead Head](queueSize int) *Keeper[THead] { + return &Keeper[THead]{ + heads: make(headSortedSlice[THead], 0, max(queueSize, MinHeadConvertingQueueSize)), } } -func (k *Keeper[TGenericTask, TDataStorage, TLSS, TShard, THead, TBlock]) write() bool { - k.mtx.Lock() - lenHeads := len(k.heads) - if lenHeads == 0 { - // quick exit - k.mtx.Unlock() - return true - } - heads := make([]THead, lenHeads) - copy(heads, k.heads) - k.mtx.Unlock() - - successful := true - shouldNotify := false - persisted := make([]string, 0, lenHeads) - for _, head := range heads { - start := k.clock.Now() - if k.headIsOutdated(head) { - persisted = append(persisted, head.ID()) - shouldNotify = true - continue - } - // TODO - // if err := head.Flush(); err != nil { - // logger.Errorf("QUERYABLE STORAGE: failed to flush head %s: %s", head.String(), err.Error()) - // successful = false - // continue - // } - // if err := head.Rotate(); err != nil { - // logger.Errorf("QUERYABLE STORAGE: failed to rotate head %s: %s", head.String(), err.Error()) - // successful = false - // continue - // } - - tBlockWrite := head.CreateTask( - BlockWrite, - func(shard TShard) error { - // shard.LSSLock() - // defer shard.LSSUnlock() - - bl := k.headBlockBuilder() // relabeler.NewBlock(shard.LSS().Raw(), shard.DataStorage().Raw()) - return k.hbWriter.Write(bl) - }, - ) - head.Enqueue(tBlockWrite) - if err := tBlockWrite.Wait(); err != nil { - logger.Errorf("QUERYABLE STORAGE: failed to write head %s: %s", head.String(), err.Error()) - successful = false - continue - } +func (k *Keeper[THead]) Add(head THead, createdAt time.Duration, policy AddPolicy) error { + k.lock.Lock() + result := k.addHead(head, createdAt, policy) + k.lock.Unlock() + return result +} - k.headPersistenceDuration.Observe(float64(k.clock.Since(start).Milliseconds())) - persisted = append(persisted, head.ID()) - shouldNotify = true - logger.Infof("QUERYABLE STORAGE: head %s persisted, duration: %v", head.String(), k.clock.Since(start)) +func (k *Keeper[THead]) addHead(head THead, createdAt time.Duration, policy AddPolicy) error { + if len(k.heads) < cap(k.heads) { + heap.Push(&k.heads, sortableHead[THead]{head: head, createdAt: createdAt}) + return nil } - if shouldNotify { - k.writeNotifier.NotifyWritten() + if policy == AddWithReplace && k.heads[0].createdAt < createdAt { + k.heads[0].head = head + k.heads[0].createdAt = createdAt + heap.Fix(&k.heads, 0) + return nil } - time.AfterFunc(k.headRetentionTimeout, func() { - select { - case <-k.closer.Signal(): - return - default: - k.shrink(persisted...) - } - }) + return ErrorNoSlots +} - return successful +func (k *Keeper[THead]) setHeads(heads headSortedSlice[THead]) { + k.heads = heads + heap.Init(&k.heads) } -func (k *Keeper[TGenericTask, TDataStorage, TLSS, TShard, THead, TBlock]) headIsOutdated(head THead) bool { - // TODO - // headMaxTimestampMs := head.Status(1).HeadStats.MaxTime - // return k.clock.Now().Sub(time.Unix(headMaxTimestampMs/1000, 0)) > k.maxRetentionDuration +func (k *Keeper[THead]) Heads() []THead { + k.lock.Lock() + headsCopy := make([]THead, 0, len(k.heads)) + for _, head := range k.heads { + headsCopy = append(headsCopy, head.head) + } + k.lock.Unlock() - return false + return headsCopy } -func (k *Keeper[TGenericTask, TDataStorage, TLSS, TShard, THead, TBlock]) shrink(persisted ...string) { - k.mtx.Lock() - defer k.mtx.Unlock() +func (k *Keeper[THead]) Remove(headsForRemove []THead) { + if len(headsForRemove) == 0 { + return + } - persistedMap := make(map[string]struct{}) - for _, headID := range persisted { - persistedMap[headID] = struct{}{} + headsMap := make(map[string]*THead, len(headsForRemove)) + for _, head := range headsForRemove { + headsMap[head.ID()] = nil } - heads := make([]THead, len(k.heads)) + k.lock.Lock() + newHeads := make([]sortableHead[THead], 0, cap(k.heads)) for _, head := range k.heads { - if _, ok := persistedMap[head.ID()]; ok { - // TODO - // _ = head.Close() - // _ = head.Discard() - logger.Infof("QUERYABLE STORAGE: head %s persisted, closed and discarded", head.String()) - continue + if _, ok := headsMap[head.head.ID()]; ok { + headsMap[head.head.ID()] = &head.head + } else { + newHeads = append(newHeads, head) } - heads = append(heads, head) } - k.heads = heads + k.setHeads(newHeads) + k.lock.Unlock() + + for _, head := range headsMap { + if head != nil { + _ = (*head).Close() + } + } +} + +func (k *Keeper[THead]) HasSlot() bool { + k.lock.Lock() + result := cap(k.heads) > len(k.heads) + k.lock.Unlock() + return result } diff --git a/pp/go/storage/head/keeper/keeper_test.go b/pp/go/storage/head/keeper/keeper_test.go new file mode 100644 index 0000000000..717f2a0c2d --- /dev/null +++ b/pp/go/storage/head/keeper/keeper_test.go @@ -0,0 +1,115 @@ +package keeper + +import ( + "testing" + + "github.com/stretchr/testify/suite" +) + +type headForTest struct { + id string +} + +func newHeadForTest(id string) *headForTest { + return &headForTest{id: id} +} + +func (h *headForTest) ID() string { + return h.id +} + +func (h *headForTest) Close() error { + return nil +} + +type sortedSlice = headSortedSlice[*headForTest] + +type KeeperSuite struct { + suite.Suite + keeper *Keeper[*headForTest] +} + +func TestKeeperSuite(t *testing.T) { + suite.Run(t, new(KeeperSuite)) +} + +func (s *KeeperSuite) getHeads() []*headForTest { + heads := make([]*headForTest, 0, len(s.keeper.heads)) + for _, head := range s.keeper.heads { + heads = append(heads, head.head) + } + return heads +} + +func (s *KeeperSuite) TestAdd() { + // Arrange + s.keeper = NewKeeper[*headForTest](2) + + // Act + _ = s.keeper.Add(newHeadForTest("d"), 4, Add) + _ = s.keeper.Add(newHeadForTest("c"), 3, Add) + err := s.keeper.Add(newHeadForTest("b"), 2, Add) + + // Assert + s.Equal(sortedSlice{ + {head: newHeadForTest("c"), createdAt: 3}, + {head: newHeadForTest("d"), createdAt: 4}, + }, s.keeper.heads) + s.Equal(err, ErrorNoSlots) +} + +func (s *KeeperSuite) TestAddWithReplaceNoReplace() { + // Arrange + s.keeper = NewKeeper[*headForTest](2) + + // Act + _ = s.keeper.Add(newHeadForTest("d"), 4, Add) + _ = s.keeper.Add(newHeadForTest("c"), 3, Add) + err := s.keeper.Add(newHeadForTest("b"), 3, AddWithReplace) + + // Assert + s.Equal(sortedSlice{ + {head: newHeadForTest("c"), createdAt: 3}, + {head: newHeadForTest("d"), createdAt: 4}, + }, s.keeper.heads) + s.Equal(err, ErrorNoSlots) +} + +func (s *KeeperSuite) TestAddWithReplace() { + // Arrange + s.keeper = NewKeeper[*headForTest](2) + + // Act + _ = s.keeper.Add(newHeadForTest("d"), 4, Add) + _ = s.keeper.Add(newHeadForTest("c"), 3, Add) + err := s.keeper.Add(newHeadForTest("b"), 4, AddWithReplace) + + // Assert + s.Equal(sortedSlice{ + {head: newHeadForTest("b"), createdAt: 4}, + {head: newHeadForTest("d"), createdAt: 4}, + }, s.keeper.heads) + s.NoError(err) +} + +func (s *KeeperSuite) TestRemove() { + // Arrange + const Slots = 5 + + s.keeper = NewKeeper[*headForTest](Slots) + _ = s.keeper.Add(newHeadForTest("a"), 1, Add) + _ = s.keeper.Add(newHeadForTest("b"), 2, Add) + _ = s.keeper.Add(newHeadForTest("c"), 3, Add) + _ = s.keeper.Add(newHeadForTest("d"), 4, Add) + _ = s.keeper.Add(newHeadForTest("e"), 5, Add) + + // Act + s.keeper.Remove([]*headForTest{newHeadForTest("a"), newHeadForTest("c"), newHeadForTest("e")}) + + // Assert + s.Equal(sortedSlice{ + {head: newHeadForTest("b"), createdAt: 2}, + {head: newHeadForTest("d"), createdAt: 4}, + }, s.keeper.heads) + s.Equal(Slots, cap(s.keeper.heads)) +} diff --git a/pp/go/storage/head/services/functions.go b/pp/go/storage/head/services/functions.go index 0295fa2b38..a6ec3cea12 100644 --- a/pp/go/storage/head/services/functions.go +++ b/pp/go/storage/head/services/functions.go @@ -8,6 +8,9 @@ import ( const ( // dsMergeOutOfOrderChunks name of task. dsMergeOutOfOrderChunks = "data_storage_merge_out_of_order_chunks" + + // dsUnloadUnusedSeriesData name of task + dsUnloadUnusedSeriesData = "data_storage_unload_unused_series_data" ) // @@ -63,6 +66,27 @@ func CFSViaRange[ return errors.Join(errs...) } +// +// UnloadUnusedSeriesDataWithHead +// + +// UnloadUnusedSeriesDataWithHead unload unused series data for [Head]. +func UnloadUnusedSeriesDataWithHead[ + TTask Task, + TShard, TGShard Shard, + THead Head[TTask, TShard, TGShard], +](h THead) error { + t := h.CreateTask( + dsUnloadUnusedSeriesData, + func(shard TGShard) error { + return shard.UnloadUnusedSeriesData() + }, + ) + h.Enqueue(t) + + return t.Wait() +} + // // MergeOutOfOrderChunksWithHead // diff --git a/pp/go/storage/head/services/interface.go b/pp/go/storage/head/services/interface.go index f9d9dfe9c4..1193293b4b 100644 --- a/pp/go/storage/head/services/interface.go +++ b/pp/go/storage/head/services/interface.go @@ -1,6 +1,12 @@ package services -import "context" +import ( + "context" + "time" + + "github.com/prometheus/prometheus/pp/go/cppbridge" + "github.com/prometheus/prometheus/pp/go/storage/head/keeper" +) // // ActiveHeadContainer @@ -46,8 +52,14 @@ type Head[ // RangeShards returns an iterator over the [Head] [Shard]s, through which the shard can be directly accessed. RangeShards() func(func(TShard) bool) + // IsReadOnly returns true if the [Head] has switched to read-only. + IsReadOnly() bool + // SetReadOnly sets the read-only flag for the [Head]. SetReadOnly() + + // Close closes wals, query semaphore for the inability to get query and clear metrics. + Close() error } // @@ -81,14 +93,18 @@ type HeadStatusSetter interface { // Keeper // -// TODO need? type Keeper[ TTask Task, - TShard, TGShard Shard, - THead Head[TTask, TShard, TGShard], + TShard, TGoShard Shard, + THead Head[TTask, TShard, TGoShard], ] interface { - Add(head THead) - RangeQueriableHeads(mint, maxt int64) func(func(THead) bool) + Add(head THead, createdAt time.Duration, policy keeper.AddPolicy) error + + Heads() []THead + + Remove(headsForRemove []THead) + + HasSlot() bool } // @@ -159,6 +175,12 @@ type Shard interface { // WalSync commits the current contents of the [Wal]. WalSync() error + + // TimeInterval get time interval of data storage + TimeInterval(bool) cppbridge.TimeInterval + + // UnloadUnusedSeriesData unload unused series data + UnloadUnusedSeriesData() error } // diff --git a/pp/go/storage/head/services/merger.go b/pp/go/storage/head/services/merger.go index 3de58cb3c6..6a2b6d95df 100644 --- a/pp/go/storage/head/services/merger.go +++ b/pp/go/storage/head/services/merger.go @@ -45,9 +45,7 @@ func (s *Merger[TTask, TShard, TGoShard, THead]) Execute(ctx context.Context) er logger.Infof("The Merger is running.") for range s.m.C() { - if err := s.activeHead.With(ctx, s.mergeOutOfOrderChunks); err != nil { - logger.Errorf("data storage merge failed: %v", err) - } + _ = s.activeHead.With(ctx, s.UnloadAndMerge) } logger.Infof("The Merger stopped.") @@ -55,11 +53,19 @@ func (s *Merger[TTask, TShard, TGoShard, THead]) Execute(ctx context.Context) er return nil } -// mergeOutOfOrderChunksWithHead merge chunks with out of order data chunks for [Head]. -func (s *Merger[TTask, TShard, TGoShard, THead]) mergeOutOfOrderChunks(h THead) error { +// UnloadAndMerge unload unused series data and merge chunks with out of order data chunks for [Head]. +func (s *Merger[TTask, TShard, TGoShard, THead]) UnloadAndMerge(h THead) error { if s.isNewHead(h.ID()) { return nil } - return MergeOutOfOrderChunksWithHead(h) + if err := UnloadUnusedSeriesDataWithHead(h); err != nil { + logger.Errorf("unload unused series data failed: %v", err) + } + + if err := MergeOutOfOrderChunksWithHead(h); err != nil { + logger.Errorf("data storage merge failed: %v", err) + } + + return nil } diff --git a/pp/go/storage/head/services/mock/persistener.go b/pp/go/storage/head/services/mock/persistener.go new file mode 100644 index 0000000000..2fd61c3bf9 --- /dev/null +++ b/pp/go/storage/head/services/mock/persistener.go @@ -0,0 +1,72 @@ +// Code generated by moq; DO NOT EDIT. +// github.com/matryer/moq + +package mock + +import ( + "github.com/prometheus/prometheus/pp/go/storage/block" + "github.com/prometheus/prometheus/pp/go/storage/head/services" + "sync" +) + +// HeadBlockWriterMock is a mock implementation of services.HeadBlockWriter. +// +// func TestSomethingThatUsesHeadBlockWriter(t *testing.T) { +// +// // make and configure a mocked services.HeadBlockWriter +// mockedHeadBlockWriter := &HeadBlockWriterMock{ +// WriteFunc: func(shard TShard) ([]block.WrittenBlock, error) { +// panic("mock out the Write method") +// }, +// } +// +// // use mockedHeadBlockWriter in code that requires services.HeadBlockWriter +// // and then make assertions. +// +// } +type HeadBlockWriterMock[TShard services.Shard] struct { + // WriteFunc mocks the Write method. + WriteFunc func(shard TShard) ([]block.WrittenBlock, error) + + // calls tracks calls to the methods. + calls struct { + // Write holds details about calls to the Write method. + Write []struct { + // Shard is the shard argument value. + Shard TShard + } + } + lockWrite sync.RWMutex +} + +// Write calls WriteFunc. +func (mock *HeadBlockWriterMock[TShard]) Write(shard TShard) ([]block.WrittenBlock, error) { + if mock.WriteFunc == nil { + panic("HeadBlockWriterMock.WriteFunc: method is nil but HeadBlockWriter.Write was just called") + } + callInfo := struct { + Shard TShard + }{ + Shard: shard, + } + mock.lockWrite.Lock() + mock.calls.Write = append(mock.calls.Write, callInfo) + mock.lockWrite.Unlock() + return mock.WriteFunc(shard) +} + +// WriteCalls gets all the calls that were made to Write. +// Check the length with: +// +// len(mockedHeadBlockWriter.WriteCalls()) +func (mock *HeadBlockWriterMock[TShard]) WriteCalls() []struct { + Shard TShard +} { + var calls []struct { + Shard TShard + } + mock.lockWrite.RLock() + calls = mock.calls.Write + mock.lockWrite.RUnlock() + return calls +} diff --git a/pp/go/storage/head/services/persistener.go b/pp/go/storage/head/services/persistener.go new file mode 100644 index 0000000000..187b05db1e --- /dev/null +++ b/pp/go/storage/head/services/persistener.go @@ -0,0 +1,237 @@ +package services + +import ( + "slices" + "time" + + "github.com/jonboulle/clockwork" + "github.com/prometheus/prometheus/pp/go/cppbridge" + "github.com/prometheus/prometheus/pp/go/storage/block" + "github.com/prometheus/prometheus/pp/go/storage/catalog" + "github.com/prometheus/prometheus/pp/go/storage/head/keeper" + "github.com/prometheus/prometheus/pp/go/storage/logger" +) + +//go:generate -command moq go run github.com/matryer/moq --rm --skip-ensure --pkg mock --out + +// +// HeadBlockWriter +// + +// HeadBlockWriter writes block on disk from [Head]. +// +//go:generate moq mock/persistener.go . HeadBlockWriter +type HeadBlockWriter[TShard Shard] interface { + Write(shard TShard) ([]block.WrittenBlock, error) +} + +type Persistener[ + TTask Task, + TShard, TGoShard Shard, + THeadBlockWriter HeadBlockWriter[TShard], + THead Head[TTask, TShard, TGoShard], +] struct { + catalog *catalog.Catalog + blockWriter THeadBlockWriter + + clock clockwork.Clock + tsdbRetentionPeriod time.Duration + retentionPeriod time.Duration +} + +func NewPersistener[ + TTask Task, + TShard, TGoShard Shard, + THeadBlockWriter HeadBlockWriter[TShard], + THead Head[TTask, TShard, TGoShard], +]( + catalog *catalog.Catalog, + blockWriter THeadBlockWriter, + clock clockwork.Clock, + tsdbRetentionPeriod time.Duration, + retentionPeriod time.Duration, +) *Persistener[TTask, TShard, TGoShard, THeadBlockWriter, THead] { + return &Persistener[TTask, TShard, TGoShard, THeadBlockWriter, THead]{ + catalog: catalog, + blockWriter: blockWriter, + clock: clock, + tsdbRetentionPeriod: tsdbRetentionPeriod, + retentionPeriod: retentionPeriod, + } +} + +func (p *Persistener[TTask, TShard, TGoShard, THeadBlockWriter, THead]) Persist(heads []THead) (outdatedHeads []THead) { + for _, head := range heads { + if !head.IsReadOnly() { + continue + } + + if record, err := p.catalog.Get(head.ID()); err == nil { + if record.Status() == catalog.StatusPersisted { + if p.persistedHeadIsOutdated(record.UpdatedAt()) { + outdatedHeads = append(outdatedHeads, head) + } + + continue + } + } + + if p.HeadIsOutdated(head) { + outdatedHeads = append(outdatedHeads, head) + continue + } + + if err := p.flushHead(head); err == nil { + if err = p.persistHead(head); err == nil { + if _, err = p.catalog.SetStatus(head.ID(), catalog.StatusPersisted); err != nil { + logger.Errorf("keeper: set head status in catalog %s: %v", head.ID(), err) + } + } + } + } + + return +} + +func (p *Persistener[TTask, TShard, TGoShard, THeadBlockWriter, THead]) headTimeInterval(head THead) cppbridge.TimeInterval { + timeInterval := cppbridge.NewInvalidTimeInterval() + for shard := range head.RangeShards() { + interval := shard.TimeInterval(false) + timeInterval.MinT = min(interval.MinT, timeInterval.MinT) + timeInterval.MaxT = max(interval.MaxT, timeInterval.MaxT) + } + return timeInterval +} + +func (p *Persistener[TTask, TShard, TGoShard, THeadBlockWriter, THead]) HeadIsOutdated(head THead) bool { + return p.clock.Since(time.UnixMilli(p.headTimeInterval(head).MaxT)) >= p.tsdbRetentionPeriod +} + +func (p *Persistener[TTask, TShard, TGoShard, THeadBlockWriter, THead]) persistedHeadIsOutdated(persistTimeMs int64) bool { + return p.clock.Since(time.UnixMilli(persistTimeMs)) >= p.retentionPeriod +} + +func (p *Persistener[TTask, TShard, TGoShard, THeadBlockWriter, THead]) flushHead(head THead) error { + for shard := range head.RangeShards() { + if err := shard.WalFlush(); err != nil { + return err + } + } + + return nil +} + +func (p *Persistener[TTask, TShard, TGoShard, THeadBlockWriter, THead]) persistHead(head THead) error { + for shard := range head.RangeShards() { + if _, err := p.blockWriter.Write(shard); err != nil { + return err + } + } + + return nil +} + +type Loader[ + TTask Task, + TShard, TGoShard Shard, + THead Head[TTask, TShard, TGoShard], +] interface { + Load(headRecord *catalog.Record, generation uint64) (THead, bool) +} + +type PersistenerService[ + TTask Task, + TShard, TGoShard Shard, + THeadBlockWriter HeadBlockWriter[TShard], + THead Head[TTask, TShard, TGoShard], + TKeeper Keeper[TTask, TShard, TGoShard, THead], + TLoader Loader[TTask, TShard, TGoShard, THead], +] struct { + persistener *Persistener[TTask, TShard, TGoShard, THeadBlockWriter, THead] + keeper TKeeper + loader TLoader + catalog *catalog.Catalog + mediator Mediator +} + +func NewPersistenerService[ + TTask Task, + TShard, TGoShard Shard, + THeadBlockWriter HeadBlockWriter[TShard], + THead Head[TTask, TShard, TGoShard], + TKeeper Keeper[TTask, TShard, TGoShard, THead], + TLoader Loader[TTask, TShard, TGoShard, THead], +]( + keeper TKeeper, + loader TLoader, + catalog *catalog.Catalog, + blockWriter THeadBlockWriter, + clock clockwork.Clock, + mediator Mediator, + tsdbRetentionPeriod time.Duration, + retentionPeriod time.Duration, +) PersistenerService[TTask, TShard, TGoShard, THeadBlockWriter, THead, TKeeper, TLoader] { + return PersistenerService[TTask, TShard, TGoShard, THeadBlockWriter, THead, TKeeper, TLoader]{ + persistener: NewPersistener[TTask, TShard, TGoShard, THeadBlockWriter, THead]( + catalog, + blockWriter, + clock, + tsdbRetentionPeriod, + retentionPeriod, + ), + keeper: keeper, + loader: loader, + catalog: catalog, + mediator: mediator, + } +} + +func (pg *PersistenerService[TTask, TShard, TGoShard, THeadBlockWriter, THead, TKeeper, TLoader]) Run() { + go func() { + for range pg.mediator.C() { + pg.ProcessHeads() + } + }() +} + +func (pg *PersistenerService[TTask, TShard, TGoShard, THeadBlockWriter, THead, TKeeper, TLoader]) ProcessHeads() { + heads := pg.keeper.Heads() + pg.persistHeads(heads) + pg.loadRotatedHeadsInKeeper(heads) +} + +func (pg *PersistenerService[TTask, TShard, TGoShard, THeadBlockWriter, THead, TKeeper, TLoader]) persistHeads(heads []THead) { + pg.keeper.Remove(pg.persistener.Persist(heads)) +} + +func (pg *PersistenerService[TTask, TShard, TGoShard, THeadBlockWriter, THead, TKeeper, TLoader]) loadRotatedHeadsInKeeper(keeperHeads []THead) { + if !pg.keeper.HasSlot() { + return + } + + headExists := func(id string) bool { + return slices.ContainsFunc(keeperHeads, func(head THead) bool { + return head.ID() == id + }) + } + + records := pg.catalog.List(func(record *catalog.Record) bool { + return record.Status() == catalog.StatusRotated && !headExists(record.ID()) + }, catalog.LessByUpdateAt) + + for _, record := range records { + if !pg.loadAndAddHeadToKeeper(record) { + break + } + } +} + +func (pg *PersistenerService[TTask, TShard, TGoShard, THeadBlockWriter, THead, TKeeper, TLoader]) loadAndAddHeadToKeeper(record *catalog.Record) bool { + head, _ := pg.loader.Load(record, 0) + if err := pg.keeper.Add(head, time.Duration(record.CreatedAt())*time.Millisecond, keeper.Add); err != nil { + _ = head.Close() + return false + } + + return true +} diff --git a/pp/go/storage/head/services/persistener_test.go b/pp/go/storage/head/services/persistener_test.go new file mode 100644 index 0000000000..f29271f43e --- /dev/null +++ b/pp/go/storage/head/services/persistener_test.go @@ -0,0 +1,387 @@ +package services_test + +import ( + "errors" + "os" + "path/filepath" + "testing" + "time" + + "github.com/jonboulle/clockwork" + "github.com/prometheus/client_golang/prometheus" + "github.com/prometheus/prometheus/model/labels" + "github.com/prometheus/prometheus/pp/go/cppbridge" + "github.com/prometheus/prometheus/pp/go/storage" + "github.com/prometheus/prometheus/pp/go/storage/block" + "github.com/prometheus/prometheus/pp/go/storage/catalog" + "github.com/prometheus/prometheus/pp/go/storage/head/keeper" + "github.com/prometheus/prometheus/pp/go/storage/head/services" + "github.com/prometheus/prometheus/pp/go/storage/head/services/mock" + "github.com/prometheus/prometheus/pp/go/storage/head/task" + "github.com/prometheus/prometheus/pp/go/storage/storagetest" + "github.com/stretchr/testify/suite" +) + +const ( + shardsCount = 2 + maxSegmentSize = 1024 + unloadDataStorageInterval = time.Duration(1) + + tsdbRetentionPeriod = time.Millisecond * 100 + + retentionPeriod = time.Minute * 5 +) + +type GenericPersistenceSuite struct { + suite.Suite + dataDir string + clock *clockwork.FakeClock + catalog *catalog.Catalog + head *storage.HeadOnDisk + keeper *keeper.Keeper[*storage.HeadOnDisk] + blockWriter *mock.HeadBlockWriterMock[*storage.ShardOnDisk] +} + +func (s *GenericPersistenceSuite) SetupTest() { + s.dataDir = s.createDataDirectory() + + s.clock = clockwork.NewFakeClockAt(time.UnixMilli(0)) + s.createCatalog() + s.keeper = keeper.NewKeeper[*storage.HeadOnDisk](1) + s.blockWriter = &mock.HeadBlockWriterMock[*storage.ShardOnDisk]{} +} + +func (s *GenericPersistenceSuite) createDataDirectory() string { + dataDir := filepath.Join(s.T().TempDir(), "data") + s.Require().NoError(os.MkdirAll(dataDir, os.ModeDir)) + return dataDir +} + +func (s *GenericPersistenceSuite) createHead() (*storage.HeadOnDisk, error) { + return storage.NewBuilder( + s.catalog, + s.dataDir, + maxSegmentSize, + prometheus.DefaultRegisterer, + unloadDataStorageInterval, + ).Build(0, shardsCount) +} + +func (s *GenericPersistenceSuite) mustCreateHead() *storage.HeadOnDisk { + h, err := s.createHead() + s.Require().NoError(err) + return h +} + +func (s *GenericPersistenceSuite) createCatalog() { + l, err := catalog.NewFileLogV2(filepath.Join(s.dataDir, "catalog.log")) + s.Require().NoError(err) + + s.catalog, err = catalog.New( + s.clock, + l, + &catalog.DefaultIDGenerator{}, + catalog.DefaultMaxLogFileSize, + nil, + ) + s.Require().NoError(err) +} + +type PersistenerSuite struct { + GenericPersistenceSuite + persistener *services.Persistener[ + *task.Generic[*storage.PerGoroutineShard], + *storage.ShardOnDisk, + *storage.PerGoroutineShard, + *mock.HeadBlockWriterMock[*storage.ShardOnDisk], + *storage.HeadOnDisk, + ] +} + +func (s *PersistenerSuite) SetupTest() { + s.GenericPersistenceSuite.SetupTest() + + s.persistener = services.NewPersistener[ + *task.Generic[*storage.PerGoroutineShard], + *storage.ShardOnDisk, + *storage.PerGoroutineShard, + *mock.HeadBlockWriterMock[*storage.ShardOnDisk], + *storage.HeadOnDisk, + ](s.catalog, s.blockWriter, s.clock, tsdbRetentionPeriod, retentionPeriod) +} + +func TestPersistenerSuite(t *testing.T) { + suite.Run(t, new(PersistenerSuite)) +} + +func (s *PersistenerSuite) TestNoHeads() { + // Arrange + + // Act + outdated := s.persistener.Persist(nil) + + // Assert + s.Equal([]*storage.HeadOnDisk(nil), outdated) + s.Equal(0, len(s.blockWriter.WriteCalls())) +} + +func (s *PersistenerSuite) TestNoPersistWritableHead() { + // Arrange + heads := []*storage.HeadOnDisk{s.mustCreateHead()} + + // Act + outdated := s.persistener.Persist(heads) + + // Assert + s.Equal([]*storage.HeadOnDisk(nil), outdated) + s.Equal(0, len(s.blockWriter.WriteCalls())) +} + +func (s *PersistenerSuite) TestNoPersistPersistedHead() { + // Arrange + head := s.mustCreateHead() + storagetest.MustAppendTimeSeries(&s.Suite, head, []storagetest.TimeSeries{ + { + Labels: labels.FromStrings("__name__", "value1"), + Samples: []cppbridge.Sample{ + {Timestamp: 0, Value: 0}, + }, + }, + }) + head.SetReadOnly() + _, err := s.catalog.SetStatus(head.ID(), catalog.StatusPersisted) + s.Require().NoError(err) + + s.clock.Advance(retentionPeriod - 1) + + // Act + outdated := s.persistener.Persist([]*storage.HeadOnDisk{head}) + + // Assert + s.Equal([]*storage.HeadOnDisk(nil), outdated) + s.Equal(0, len(s.blockWriter.WriteCalls())) +} + +func (s *PersistenerSuite) TestOutdatedPersistedHead() { + // Arrange + head := s.mustCreateHead() + storagetest.MustAppendTimeSeries(&s.Suite, head, []storagetest.TimeSeries{ + { + Labels: labels.FromStrings("__name__", "value1"), + Samples: []cppbridge.Sample{ + {Timestamp: 0, Value: 0}, + }, + }, + }) + head.SetReadOnly() + _, err := s.catalog.SetStatus(head.ID(), catalog.StatusPersisted) + s.Require().NoError(err) + + s.clock.Advance(retentionPeriod) + + // Act + outdated := s.persistener.Persist([]*storage.HeadOnDisk{head}) + + // Assert + s.Equal([]*storage.HeadOnDisk{head}, outdated) + s.Equal(0, len(s.blockWriter.WriteCalls())) +} + +func (s *PersistenerSuite) TestOutdatedHead() { + // Arrange + s.clock.Advance(tsdbRetentionPeriod) + + head := s.mustCreateHead() + storagetest.MustAppendTimeSeries(&s.Suite, head, []storagetest.TimeSeries{ + { + Labels: labels.FromStrings("__name__", "value1"), + Samples: []cppbridge.Sample{ + {Timestamp: 0, Value: 0}, + }, + }, + }) + head.SetReadOnly() + + // Act + outdated := s.persistener.Persist([]*storage.HeadOnDisk{head}) + + // Assert + s.Equal([]*storage.HeadOnDisk{head}, outdated) + s.Equal(0, len(s.blockWriter.WriteCalls())) +} + +func (s *PersistenerSuite) TestPersistHeadSuccess() { + // Arrange + s.clock.Advance(tsdbRetentionPeriod) + blockWriter := block.NewWriter[*storage.ShardOnDisk]( + s.dataDir, + block.DefaultChunkSegmentSize, + 2*time.Hour, + prometheus.DefaultRegisterer, + ) + s.blockWriter.WriteFunc = func(shard *storage.ShardOnDisk) ([]block.WrittenBlock, error) { + return blockWriter.Write(shard) + } + + head := s.mustCreateHead() + storagetest.MustAppendTimeSeries(&s.Suite, head, []storagetest.TimeSeries{ + { + Labels: labels.FromStrings("__name__", "value1"), + Samples: []cppbridge.Sample{ + {Timestamp: 1, Value: 0}, + }, + }, + }) + head.SetReadOnly() + + // Act + outdated := s.persistener.Persist([]*storage.HeadOnDisk{head}) + record, err := s.catalog.Get(head.ID()) + + // Assert + s.Equal([]*storage.HeadOnDisk(nil), outdated) + s.Equal(2, len(s.blockWriter.WriteCalls())) + s.Require().NoError(err) + s.Equal(catalog.StatusPersisted, record.Status()) +} + +func (s *PersistenerSuite) TestPersistHeadErrorOnBlockWriterForSecondShard() { + // Arrange + s.clock.Advance(tsdbRetentionPeriod) + blockWriter := block.NewWriter[*storage.ShardOnDisk]( + s.dataDir, + block.DefaultChunkSegmentSize, + 2*time.Hour, + prometheus.DefaultRegisterer, + ) + s.blockWriter.WriteFunc = func(shard *storage.ShardOnDisk) ([]block.WrittenBlock, error) { + if len(s.blockWriter.WriteCalls()) == 2 { + return nil, errors.New("some error") + } + + return blockWriter.Write(shard) + } + + head := s.mustCreateHead() + storagetest.MustAppendTimeSeries(&s.Suite, head, []storagetest.TimeSeries{ + { + Labels: labels.FromStrings("__name__", "value1"), + Samples: []cppbridge.Sample{ + {Timestamp: 1, Value: 0}, + }, + }, + }) + head.SetReadOnly() + + // Act + outdated := s.persistener.Persist([]*storage.HeadOnDisk{head}) + record, err := s.catalog.Get(head.ID()) + + // Assert + s.Equal([]*storage.HeadOnDisk(nil), outdated) + s.Equal(2, len(s.blockWriter.WriteCalls())) + s.Require().NoError(err) + s.Equal(catalog.StatusNew, record.Status()) +} + +type PersistenerServiceSuite struct { + GenericPersistenceSuite + loader *storage.Loader + service services.PersistenerService[ + *task.Generic[*storage.PerGoroutineShard], + *storage.ShardOnDisk, + *storage.PerGoroutineShard, + *mock.HeadBlockWriterMock[*storage.ShardOnDisk], + *storage.HeadOnDisk, + *keeper.Keeper[*storage.HeadOnDisk], + *storage.Loader, + ] +} + +func (s *PersistenerServiceSuite) SetupTest() { + s.GenericPersistenceSuite.SetupTest() + + s.loader = storage.NewLoader(s.dataDir, maxSegmentSize, prometheus.DefaultRegisterer, unloadDataStorageInterval) + s.service = services.NewPersistenerService[ + *task.Generic[*storage.PerGoroutineShard], + *storage.ShardOnDisk, + *storage.PerGoroutineShard, + *mock.HeadBlockWriterMock[*storage.ShardOnDisk], + *storage.HeadOnDisk, + *keeper.Keeper[*storage.HeadOnDisk], + *storage.Loader, + ](s.keeper, s.loader, s.catalog, s.blockWriter, s.clock, nil, tsdbRetentionPeriod, retentionPeriod) +} + +func TestPersistenerServiceSuite(t *testing.T) { + suite.Run(t, new(PersistenerServiceSuite)) +} + +func (s *PersistenerServiceSuite) TestRemoveOutdatedHeadFromKeeper() { + // Arrange + s.clock.Advance(tsdbRetentionPeriod) + head := s.mustCreateHead() + storagetest.MustAppendTimeSeries(&s.Suite, head, []storagetest.TimeSeries{ + { + Labels: labels.FromStrings("__name__", "value1"), + Samples: []cppbridge.Sample{ + {Timestamp: 0, Value: 0}, + }, + }, + }) + head.SetReadOnly() + record, _ := s.catalog.SetStatus(head.ID(), catalog.StatusRotated) + _ = s.keeper.Add(head, time.Duration(s.clock.Now().Nanosecond()), keeper.Add) + + // Act + s.service.ProcessHeads() + + // Assert + s.Equal(0, len(s.keeper.Heads())) + s.Equal(catalog.StatusRotated, record.Status()) +} + +func (s *PersistenerServiceSuite) TestLoadHeadsInKeeper() { + // Arrange + head := s.mustCreateHead() + storagetest.MustAppendTimeSeries(&s.Suite, head, []storagetest.TimeSeries{ + { + Labels: labels.FromStrings("__name__", "value1"), + Samples: []cppbridge.Sample{ + {Timestamp: 1, Value: 0}, + }, + }, + }) + record, _ := s.catalog.SetStatus(head.ID(), catalog.StatusRotated) + s.Require().NoError(head.Close()) + + // Act + s.service.ProcessHeads() + + // Assert + s.Require().Equal(1, len(s.keeper.Heads())) + s.Equal(head.ID(), s.keeper.Heads()[0].ID()) + s.Equal(int64(0), record.CreatedAt()) +} + +func (s *PersistenerServiceSuite) TestHeadAlreadyExistsInKeeper() { + // Arrange + head := s.mustCreateHead() + storagetest.MustAppendTimeSeries(&s.Suite, head, []storagetest.TimeSeries{ + { + Labels: labels.FromStrings("__name__", "value1"), + Samples: []cppbridge.Sample{ + {Timestamp: 1, Value: 0}, + }, + }, + }) + _, _ = s.catalog.SetStatus(head.ID(), catalog.StatusRotated) + _ = s.keeper.Add(head, 0, keeper.Add) + + // Act + s.service.ProcessHeads() + + // Assert + s.Require().Equal(1, len(s.keeper.Heads())) + s.Equal(head.ID(), s.keeper.Heads()[0].ID()) +} diff --git a/pp/go/storage/head/shard/data_storage.go b/pp/go/storage/head/shard/data_storage.go index 480c83a9a3..8d499b6d31 100644 --- a/pp/go/storage/head/shard/data_storage.go +++ b/pp/go/storage/head/shard/data_storage.go @@ -78,6 +78,12 @@ func (ds *DataStorage) Query( return serializedChunks, res } +func (ds *DataStorage) QueryFinal(queriers []uintptr) { + ds.locker.RLock() + ds.dataStorage.QueryFinal(queriers) + ds.locker.RUnlock() +} + // QueryStatus get head status from [DataStorage]. func (ds *DataStorage) QueryStatus(status *cppbridge.HeadStatus) { ds.locker.RLock() @@ -107,3 +113,38 @@ func (ds *DataStorage) WithRLock(fn func(ds *cppbridge.HeadDataStorage) error) e return err } + +// TimeInterval get time interval from [DataStorage]. +func (ds *DataStorage) TimeInterval(invalidateCache bool) cppbridge.TimeInterval { + ds.locker.RLock() + result := ds.dataStorage.TimeInterval(invalidateCache) + ds.locker.RUnlock() + + return result +} + +// CreateUnusedSeriesDataUnloader create unused series data unloader +func (ds *DataStorage) CreateUnusedSeriesDataUnloader() *cppbridge.UnusedSeriesDataUnloader { + return ds.dataStorage.CreateUnusedSeriesDataUnloader() +} + +// CreateLoader create series data unloader +func (ds *DataStorage) CreateLoader(queriers []uintptr) *cppbridge.UnloadedDataLoader { + return ds.dataStorage.CreateLoader(queriers) +} + +// CreateRevertableLoader create series data revertable unloader +func (ds *DataStorage) CreateRevertableLoader( + lss *cppbridge.LabelSetStorage, + lsIdBatchSize uint32, +) *cppbridge.UnloadedDataRevertableLoader { + return ds.dataStorage.CreateRevertableLoader(lss, lsIdBatchSize) +} + +func (ds *DataStorage) GetQueriedSeriesBitset() []byte { + return ds.dataStorage.GetQueriedSeriesBitset() +} + +func (ds *DataStorage) SetQueriedSeriesBitset(bitset []byte) bool { + return ds.dataStorage.SetQueriedSeriesBitset(bitset) +} diff --git a/pp/go/storage/head/shard/file_storage.go b/pp/go/storage/head/shard/file_storage.go new file mode 100644 index 0000000000..aaaeeec5e2 --- /dev/null +++ b/pp/go/storage/head/shard/file_storage.go @@ -0,0 +1,62 @@ +package shard + +import "os" + +type FileStorage struct { + fileName string + file *os.File +} + +func NewFileStorage(fileName string) *FileStorage { + return &FileStorage{fileName: fileName} +} + +func (q *FileStorage) ReadAt(p []byte, off int64) (n int, err error) { + return q.file.ReadAt(p, off) +} + +func (q *FileStorage) Open(flags int) (err error) { + if q.file == nil { + q.file, err = os.OpenFile(q.fileName, flags, 0666) + } + + return +} + +func (q *FileStorage) Write(p []byte) (n int, err error) { + return q.file.Write(p) +} + +func (q *FileStorage) Close() error { + if q.file != nil { + return q.file.Close() + } + + return nil +} + +func (q *FileStorage) Read(p []byte) (n int, err error) { + return q.file.Read(p) +} + +func (q *FileStorage) Seek(offset int64, whence int) (int64, error) { + return q.file.Seek(offset, whence) +} + +func (q *FileStorage) Sync() error { + return q.file.Sync() +} + +func (q *FileStorage) Truncate(size int64) error { + return q.file.Truncate(size) +} + +func (q *FileStorage) IsEmpty() bool { + if q.file != nil { + if info, err := q.file.Stat(); err == nil { + return info.Size() == 0 + } + } + + return true +} diff --git a/pp/go/storage/head/shard/load_and_query_series_data_task.go b/pp/go/storage/head/shard/load_and_query_series_data_task.go new file mode 100644 index 0000000000..2ab38b9871 --- /dev/null +++ b/pp/go/storage/head/shard/load_and_query_series_data_task.go @@ -0,0 +1,35 @@ +package shard + +import ( + "sync" +) + +// Task the minimum required Task implementation. +type Task interface { + Wait() error +} + +type LoadAndQuerySeriesDataTask struct { + queriers []uintptr + task Task + lock sync.Mutex +} + +func (t *LoadAndQuerySeriesDataTask) Add(querier uintptr, createAndEnqueueTask func() Task) Task { + t.lock.Lock() + defer t.lock.Unlock() + t.queriers = append(t.queriers, querier) + if len(t.queriers) == 1 { + t.task = createAndEnqueueTask() + } + + return t.task +} + +func (t *LoadAndQuerySeriesDataTask) Release(callback func([]uintptr)) { + t.lock.Lock() + callback(t.queriers) + t.queriers = nil + t.task = nil + t.lock.Unlock() +} diff --git a/pp/go/storage/head/shard/shard.go b/pp/go/storage/head/shard/shard.go index d7b0ea400f..1426846d96 100644 --- a/pp/go/storage/head/shard/shard.go +++ b/pp/go/storage/head/shard/shard.go @@ -3,6 +3,10 @@ package shard import ( + "errors" + "fmt" + "time" + "github.com/prometheus/prometheus/pp/go/cppbridge" ) @@ -33,24 +37,32 @@ type Wal interface { // Shard bridge to labelset storage, data storage and wal. type Shard[TWal Wal] struct { - lss *LSS - dataStorage *DataStorage - wal TWal - id uint16 + lss *LSS + dataStorage *DataStorage + unloadedDataStorage *UnloadedDataStorage + queriedSeriesStorage *QueriedSeriesStorage + loadAndQueryTask *LoadAndQuerySeriesDataTask + wal TWal + id uint16 } // NewShard init new [Shard]. func NewShard[TWal Wal]( lss *LSS, dataStorage *DataStorage, + unloadedDataStorage *UnloadedDataStorage, + queriedSeriesStorage *QueriedSeriesStorage, wal TWal, shardID uint16, ) *Shard[TWal] { return &Shard[TWal]{ - id: shardID, - lss: lss, - dataStorage: dataStorage, - wal: wal, + id: shardID, + lss: lss, + dataStorage: dataStorage, + unloadedDataStorage: unloadedDataStorage, + queriedSeriesStorage: queriedSeriesStorage, + loadAndQueryTask: &LoadAndQuerySeriesDataTask{}, + wal: wal, } } @@ -61,7 +73,17 @@ func (s *Shard[TWal]) AppendInnerSeriesSlice(innerSeriesSlice []*cppbridge.Inner // Close closes the wal segmentWriter. func (s *Shard[TWal]) Close() error { - return s.wal.Close() + err := s.wal.Close() + + if s.unloadedDataStorage != nil { + err = errors.Join(err, s.unloadedDataStorage.Close()) + } + + if s.queriedSeriesStorage != nil { + err = errors.Join(err, s.queriedSeriesStorage.Close()) + } + + return err } // DSAllocatedMemory return size of allocated memory for [DataStorage]. @@ -126,6 +148,77 @@ func (s *Shard[TWal]) WalWrite(innerSeriesSlice []*cppbridge.InnerSeries) (bool, return s.wal.Write(innerSeriesSlice) } +// TimeInterval get time interval from [DataStorage]. +func (s *Shard[TWal]) TimeInterval(invalidateCache bool) cppbridge.TimeInterval { + return s.dataStorage.TimeInterval(invalidateCache) +} + +// UnloadedDataStorage get unloaded data storage +func (s *Shard[TWal]) UnloadedDataStorage() *UnloadedDataStorage { + return s.unloadedDataStorage +} + +// QueriedSeriesStorage get queried series storage +func (s *Shard[TWal]) QueriedSeriesStorage() *QueriedSeriesStorage { + return s.queriedSeriesStorage +} + +// LoadAndQuerySeriesDataTask get load and query series data task +func (s *Shard[TWal]) LoadAndQuerySeriesDataTask() *LoadAndQuerySeriesDataTask { + return s.loadAndQueryTask +} + +// UnloadUnusedSeriesData unload unused series data +func (s *Shard[TWal]) UnloadUnusedSeriesData() error { + if s.UnloadedDataStorage() == nil { + return nil + } + + unloader := s.DataStorage().CreateUnusedSeriesDataUnloader() + + var snapshot, queriedSeries []byte + _ = s.DataStorage().WithRLock(func(ds *cppbridge.HeadDataStorage) error { + snapshot = unloader.CreateSnapshot() + queriedSeries = s.DataStorage().GetQueriedSeriesBitset() + return nil + }) + + header, err := s.UnloadedDataStorage().WriteSnapshot(snapshot) + if err != nil { + return fmt.Errorf("unable to write unloaded series data snapshot: %v", err) + } + + _ = s.DataStorage().WithLock(func(ds *cppbridge.HeadDataStorage) error { + s.UnloadedDataStorage().WriteIndex(header) + unloader.Unload() + return nil + }) + + if err = s.QueriedSeriesStorage().Write(queriedSeries, time.Now().UnixMilli()); err != nil { + return fmt.Errorf("unable to write queried series data: %v", err) + } + + return nil +} + +func (s *Shard[TWal]) LoadAndQuerySeriesData() (err error) { + var queriers []uintptr + s.loadAndQueryTask.Release(func(q []uintptr) { + queriers = q + err = s.DataStorage().WithLock(func(ds *cppbridge.HeadDataStorage) error { + loader := s.DataStorage().CreateLoader(queriers) + return s.UnloadedDataStorage().ForEachSnapshot(loader.Load) + }) + }) + + if err != nil { + return + } + + s.DataStorage().QueryFinal(queriers) + return +} + // // PerGoroutineShard // diff --git a/pp/go/storage/head/shard/unloaded_data_storage.go b/pp/go/storage/head/shard/unloaded_data_storage.go new file mode 100644 index 0000000000..07f76e6cfe --- /dev/null +++ b/pp/go/storage/head/shard/unloaded_data_storage.go @@ -0,0 +1,316 @@ +package shard + +import ( + "errors" + "fmt" + "hash/crc32" + "io" + "os" + "unsafe" + + "github.com/prometheus/prometheus/pp/go/storage/logger" +) + +const ( + UnloadedDataStorageVersion = 1 + QueriedSeriesStorageVersion = 1 +) + +type StorageFile interface { + Open(flags int) error + io.WriteCloser + io.ReadSeeker + io.ReaderAt + Sync() error + Truncate(size int64) error + IsEmpty() bool +} + +type UnloadedDataSnapshotHeader struct { + Crc32 uint32 + SnapshotSize uint32 +} + +func NewUnloadedDataSnapshotHeader(snapshot []byte) UnloadedDataSnapshotHeader { + return UnloadedDataSnapshotHeader{Crc32: crc32.ChecksumIEEE(snapshot), SnapshotSize: uint32(len(snapshot))} +} + +func (h UnloadedDataSnapshotHeader) IsValid(snapshot []byte) bool { + return h.Crc32 == crc32.ChecksumIEEE(snapshot) +} + +type UnloadedDataStorage struct { + storage StorageFile + snapshots []UnloadedDataSnapshotHeader + maxSnapshotSize uint32 +} + +func NewUnloadedDataStorage(storage StorageFile) *UnloadedDataStorage { + return &UnloadedDataStorage{ + storage: storage, + } +} + +func (s *UnloadedDataStorage) WriteSnapshot(snapshot []byte) (UnloadedDataSnapshotHeader, error) { + if len(snapshot) == 0 { + return UnloadedDataSnapshotHeader{}, nil + } + + if err := s.storage.Open(os.O_RDWR | os.O_CREATE | os.O_TRUNC); err != nil { + return UnloadedDataSnapshotHeader{}, err + } + + if len(s.snapshots) == 0 { + if err := s.WriteFormatVersion(); err != nil { + return UnloadedDataSnapshotHeader{}, err + } + } + + _, err := s.storage.Write(snapshot) + if err == nil { + err = s.storage.Sync() + } + return NewUnloadedDataSnapshotHeader(snapshot), err +} + +func (s *UnloadedDataStorage) WriteIndex(header UnloadedDataSnapshotHeader) { + s.snapshots = append(s.snapshots, header) + s.maxSnapshotSize = max(header.SnapshotSize, s.maxSnapshotSize) +} + +func (s *UnloadedDataStorage) WriteFormatVersion() error { + _, err := s.storage.Write([]byte{UnloadedDataStorageVersion}) + return err +} + +func (s *UnloadedDataStorage) ForEachSnapshot(f func(snapshot []byte, isLast bool)) error { + if len(s.snapshots) == 0 { + return nil + } + + offset, err := s.validateFormatVersion() + if err != nil { + return err + } + + snapshot := make([]byte, 0, s.maxSnapshotSize) + for index, header := range s.snapshots { + snapshot = snapshot[:header.SnapshotSize] + size, err := s.storage.ReadAt(snapshot, offset) + if uint32(size) != header.SnapshotSize { + return err + } + offset += int64(size) + + if !header.IsValid(snapshot) { + return fmt.Errorf("invalid snapshot at index %d", index) + } + + f(snapshot, index == len(s.snapshots)-1) + } + + return nil +} + +func (s *UnloadedDataStorage) validateFormatVersion() (offset int64, err error) { + version := []byte{0} + if _, err = s.storage.ReadAt(version, 0); err != nil { + return 0, err + } + + if version[0] != UnloadedDataStorageVersion { + return 0, fmt.Errorf("UnloadedDataStorage invalid version %d", version[0]) + } + + return int64(len(version)), nil +} + +func (s *UnloadedDataStorage) Close() (err error) { + if s.storage != nil { + err = s.storage.Close() + s.storage = nil + } + + return +} + +func (s *UnloadedDataStorage) IsEmpty() bool { + return len(s.snapshots) == 0 +} + +type QueriedSeriesStorage struct { + storages [2]StorageFile + validStorage StorageFile +} + +func NewQueriedSeriesStorage(storage1, storage2 StorageFile) *QueriedSeriesStorage { + return &QueriedSeriesStorage{ + storages: [2]StorageFile{storage1, storage2}, + } +} + +type queriedSeriesStorageHeader struct { + timestamp int64 + crc32 uint32 + size uint32 +} + +func (h *queriedSeriesStorageHeader) toSlice() []byte { + return (*(*[unsafe.Sizeof(queriedSeriesStorageHeader{})]byte)(unsafe.Pointer(h)))[:] +} + +func (h *queriedSeriesStorageHeader) CalculateCrc32(queriedSeriesBitset []byte) uint32 { + h.crc32 = 0 + + writer := crc32.NewIEEE() + _, _ = writer.Write(h.toSlice()) + _, _ = writer.Write(queriedSeriesBitset) + h.crc32 = writer.Sum32() + + return h.crc32 +} + +func (s *QueriedSeriesStorage) Write(queriedSeriesBitset []byte, timestamp int64) error { + storage := s.storages[0] + if err := storage.Open(os.O_RDWR | os.O_CREATE | os.O_TRUNC); err != nil { + s.changeActiveStorageIfNoValidStorage() + return err + } + + var headerBuffer [1 + unsafe.Sizeof(queriedSeriesStorageHeader{})]byte + headerBuffer[0] = UnloadedDataStorageVersion + + header := (*queriedSeriesStorageHeader)(unsafe.Pointer(&headerBuffer[1])) + header.timestamp = timestamp + header.size = uint32(len(queriedSeriesBitset)) + header.CalculateCrc32(queriedSeriesBitset) + + if err := s.writeToStorage(storage, headerBuffer[:], queriedSeriesBitset); err != nil { + s.changeActiveStorageIfNoValidStorage() + return err + } + + s.validStorage = s.storages[0] + s.changeActiveStorage() + return nil +} + +func (s *QueriedSeriesStorage) writeToStorage(storage StorageFile, headerBuffer, queriedSeriesBitset []byte) error { + if _, err := storage.Seek(0, io.SeekStart); err != nil { + return err + } + + if _, err := storage.Write(headerBuffer); err != nil { + return err + } + + if _, err := storage.Write(queriedSeriesBitset); err != nil { + return err + } + + if err := storage.Sync(); err != nil { + return err + } + + return storage.Truncate(int64(len(headerBuffer) + len(queriedSeriesBitset))) +} + +func (s *QueriedSeriesStorage) changeActiveStorage() { + s.storages[0], s.storages[1] = s.storages[1], s.storages[0] +} + +func (s *QueriedSeriesStorage) changeActiveStorageIfNoValidStorage() { + if s.validStorage == nil { + s.changeActiveStorage() + } +} + +func (s *QueriedSeriesStorage) Read() (data []byte, err error) { + readers, maxSize := s.readStorageHeaders() + data = make([]byte, 0, maxSize) + + for i := range readers { + data = data[:readers[i].size] + + if len(data) > 0 { + if _, err = io.ReadFull(readers[i].storage, data); err != nil { + logger.Warnf("failed to read data from queried series storage: %v", err) + continue + } + } + + if storageCrc32 := readers[i].crc32; storageCrc32 != readers[i].CalculateCrc32(data) { + logger.Warnf("invalid queried series storage crc32: %d != %d", storageCrc32, readers[i].crc32) + continue + } + + s.validStorage = readers[i].storage + if readers[i].storage == s.storages[0] { + s.changeActiveStorage() + } + + return data, nil + } + + return nil, errors.New("no valid queried series storage") +} + +func (s *QueriedSeriesStorage) readStorageHeaders() (result []storageHeaderReader, maxSize uint32) { + for _, storage := range s.storages { + reader := storageHeaderReader{storage: storage} + + if err := reader.read(); err == nil { + result = append(result, reader) + maxSize = max(maxSize, reader.size) + } else { + if !os.IsNotExist(err) && !errors.Is(err, io.EOF) { + logger.Warnf("failed to read header: %v", err) + } + } + } + + if len(result) == 2 && result[0].timestamp < result[1].timestamp { + result[0], result[1] = result[1], result[0] + } + + return +} + +func (s *QueriedSeriesStorage) Close() error { + return errors.Join(s.storages[0].Close(), s.storages[1].Close()) +} + +type storageHeaderReader struct { + queriedSeriesStorageHeader + storage StorageFile +} + +func (s *storageHeaderReader) read() error { + if err := s.storage.Open(os.O_RDWR); err != nil { + return err + } + + if err := s.readAndValidateFormatVersion(); err != nil { + return err + } + + _, err := io.ReadFull(s.storage, s.toSlice()) + return err +} + +func (s *storageHeaderReader) readAndValidateFormatVersion() error { + if _, err := s.storage.Seek(0, io.SeekStart); err != nil { + return err + } + + version := []byte{0} + if _, err := s.storage.Read(version); err != nil { + return err + } + + if version[0] != QueriedSeriesStorageVersion { + return fmt.Errorf("QueriedSeriesStorage invalid version %d", version[0]) + } + + return nil +} diff --git a/pp/go/storage/head/shard/unloaded_data_storage_test.go b/pp/go/storage/head/shard/unloaded_data_storage_test.go new file mode 100644 index 0000000000..6bbd919ad6 --- /dev/null +++ b/pp/go/storage/head/shard/unloaded_data_storage_test.go @@ -0,0 +1,529 @@ +package shard + +import ( + "bytes" + "errors" + "fmt" + "io" + "os" + "path/filepath" + "testing" + + "github.com/stretchr/testify/suite" +) + +type BufferReaderAtWriterCloser struct { + buffer []byte +} + +func (s *BufferReaderAtWriterCloser) IsEmpty() bool { + return true +} + +func (s *BufferReaderAtWriterCloser) Open(flags int) error { + return nil +} + +func (s *BufferReaderAtWriterCloser) Read(p []byte) (n int, err error) { + return 0, nil +} + +func (s *BufferReaderAtWriterCloser) Seek(offset int64, whence int) (int64, error) { + return 0, nil +} + +func (s *BufferReaderAtWriterCloser) Sync() error { + return nil +} + +func (s *BufferReaderAtWriterCloser) Truncate(size int64) error { + return nil +} + +func (s *BufferReaderAtWriterCloser) ReadAt(p []byte, off int64) (n int, err error) { + return bytes.NewReader(s.buffer).ReadAt(p, off) +} + +func (s *BufferReaderAtWriterCloser) Write(p []byte) (n int, err error) { + s.buffer = append(s.buffer, p...) + return len(p), nil +} + +func (s *BufferReaderAtWriterCloser) Close() error { + return nil +} + +type UnloadedDataStorageSuite struct { + suite.Suite + storageBuffer *BufferReaderAtWriterCloser + storage *UnloadedDataStorage +} + +func TestUnloadedDataStorageSuite(t *testing.T) { + suite.Run(t, new(UnloadedDataStorageSuite)) +} + +func (s *UnloadedDataStorageSuite) SetupTest() { + s.storageBuffer = &BufferReaderAtWriterCloser{} + s.storage = NewUnloadedDataStorage(s.storageBuffer) +} + +func (s *UnloadedDataStorageSuite) Write(snapshot []byte) { + header, _ := s.storage.WriteSnapshot(snapshot) + s.storage.WriteIndex(header) +} + +func (s *UnloadedDataStorageSuite) readSnapshots() ([]string, error) { + var snapshots []string + return snapshots, s.storage.ForEachSnapshot(func(snapshot []byte, isLast bool) { + snapshots = append(snapshots, string(snapshot)) + }) +} + +func (s *UnloadedDataStorageSuite) TestWriteEmptySnapshot() { + // Arrange + + // Act + header, err := s.storage.WriteSnapshot(nil) + + // Assert + s.Require().NoError(err) + s.Equal(UnloadedDataSnapshotHeader{}, header) +} + +func (s *UnloadedDataStorageSuite) TestReadEmptySnapshots() { + // Arrange + s.storageBuffer.buffer = []byte{UnloadedDataStorageVersion} + + // Act + snapshots, err := s.readSnapshots() + + // Assert + s.Equal([]string(nil), snapshots) + s.Equal(nil, err) +} + +func (s *UnloadedDataStorageSuite) TestReadOneSnapshot() { + // Arrange + s.Write([]byte("12345")) + + // Act + snapshots, err := s.readSnapshots() + + // Assert + s.Equal([]string{"12345"}, snapshots) + s.Equal(nil, err) +} + +func (s *UnloadedDataStorageSuite) TestReadMultipleSnapshots() { + // Arrange + s.Write([]byte("123")) + s.Write([]byte("45678")) + s.Write([]byte("90")) + + // Act + snapshots, err := s.readSnapshots() + + // Assert + s.Equal([]string{"123", "45678", "90"}, snapshots) + s.Equal(nil, err) +} + +func (s *UnloadedDataStorageSuite) TestReadEof() { + // Arrange + s.Write([]byte("123")) + s.storageBuffer.buffer = s.storageBuffer.buffer[:len(s.storageBuffer.buffer)-1] + + // Act + snapshots, err := s.readSnapshots() + + // Assert + s.Equal([]string(nil), snapshots) + s.Equal(fmt.Errorf("EOF"), err) +} + +func (s *UnloadedDataStorageSuite) TestReadVersionError() { + // Arrange + s.Write([]byte("123")) + s.storageBuffer.buffer = nil + + // Act + snapshots, err := s.readSnapshots() + + // Assert + s.Equal([]string(nil), snapshots) + s.Equal(io.EOF, err) +} + +func (s *UnloadedDataStorageSuite) TestInvalidVersion() { + // Arrange + s.Write([]byte("123")) + var invalidVersion byte = UnloadedDataStorageVersion + 1 + s.storageBuffer.buffer = []byte{invalidVersion} + + // Act + snapshots, err := s.readSnapshots() + + // Assert + s.Equal([]string(nil), snapshots) + s.Equal(fmt.Errorf("UnloadedDataStorage invalid version %d", invalidVersion), err) +} + +func (s *UnloadedDataStorageSuite) TestReadInvalidSnapshot() { + // Arrange + s.Write([]byte("123")) + s.Write([]byte("45678")) + s.storageBuffer.buffer[4] = 0x00 + + // Act + snapshots, err := s.readSnapshots() + + // Assert + s.Equal([]string{"123"}, snapshots) + s.Equal(fmt.Errorf("invalid snapshot at index 1"), err) +} + +type QueriedSeriesStorageSuite struct { + suite.Suite + file1 *FileStorage + file2 *FileStorage + storage *QueriedSeriesStorage +} + +func TestQueriedSeriesStorageWriterSuite(t *testing.T) { + suite.Run(t, new(QueriedSeriesStorageSuite)) +} + +func (s *QueriedSeriesStorageSuite) SetupTest() { + tempDir := s.T().TempDir() + s.file1 = &FileStorage{fileName: filepath.Join(tempDir, "file1")} + s.file2 = &FileStorage{fileName: filepath.Join(tempDir, "file2")} + s.storage = NewQueriedSeriesStorage(s.file1, s.file2) +} + +func (s *QueriedSeriesStorageSuite) TearDownTest() { + s.Require().NoError(s.storage.Close()) +} + +func (s *QueriedSeriesStorageSuite) writeFile(file *FileStorage, data []byte) { + s.Require().NoError(file.Open(os.O_RDWR | os.O_CREATE | os.O_TRUNC)) + _, err := file.Write(data) + s.Require().NoError(err) +} + +func (s *QueriedSeriesStorageSuite) readFile(file *FileStorage) []byte { + _, err := file.Seek(0, io.SeekStart) + s.Require().NoError(err) + + data, err := io.ReadAll(file) + s.Require().NoError(err) + + return data +} + +func (s *QueriedSeriesStorageSuite) TestOpenErrorOnWrite() { + // Arrange + s.file1.fileName = "" + + // Act + err := s.storage.Write([]byte("12345"), 1234567890) + + // Assert + s.Error(err) + s.Nil(s.storage.validStorage) + s.Equal(s.file2, s.storage.storages[0]) +} + +func (s *QueriedSeriesStorageSuite) TestWriteInFirstStorage() { + // Arrange + + // Act + err := s.storage.Write([]byte("12345"), 1234567890) + + // Assert + s.NoError(err) + s.Equal([]byte{ + QueriedSeriesStorageVersion, // version + 0xd2, 0x02, 0x96, 0x49, 0x00, 0x00, 0x00, 0x00, // timestamp + 0x4e, 0x78, 0xf9, 0xf3, //crc32 + 0x05, 0x00, 0x00, 0x00, // size + '1', '2', '3', '4', '5', // content + }, s.readFile(s.file1)) + s.Nil(s.file2.file) +} + +func (s *QueriedSeriesStorageSuite) TestWriteInAllStorages() { + // Arrange + + // Act + err1 := s.storage.Write([]byte("12345"), 1234567890) + err2 := s.storage.Write([]byte("67890"), 987654321) + + // Assert + s.NoError(err1) + s.NoError(err2) + s.Equal([]byte{ + QueriedSeriesStorageVersion, // version + 0xd2, 0x02, 0x96, 0x49, 0x00, 0x00, 0x00, 0x00, // timestamp + 0x4e, 0x78, 0xf9, 0xf3, //crc32 + 0x05, 0x00, 0x00, 0x00, // size + '1', '2', '3', '4', '5', // content + }, s.readFile(s.file1)) + s.Equal([]byte{ + QueriedSeriesStorageVersion, // version + 0xb1, 0x68, 0xde, 0x3a, 0x00, 0x00, 0x00, 0x00, // timestamp + 0x21, 0x33, 0xf7, 0xb8, //crc32 + 0x05, 0x00, 0x00, 0x00, // size + '6', '7', '8', '9', '0', // content + }, s.readFile(s.file2)) +} + +func (s *QueriedSeriesStorageSuite) TestMultipleWriteInFirstStorage() { + // Arrange + + // Act + _ = s.storage.Write([]byte("12345"), 1234567890) + _ = s.storage.Write([]byte("67890"), 987654321) + _ = s.storage.Write([]byte("67890"), 987654321) + _ = s.storage.Write([]byte("12345"), 1234567890) + + // Assert + s.Equal([]byte{ + QueriedSeriesStorageVersion, // version + 0xb1, 0x68, 0xde, 0x3a, 0x00, 0x00, 0x00, 0x00, // timestamp + 0x21, 0x33, 0xf7, 0xb8, //crc32 + 0x05, 0x00, 0x00, 0x00, // size + '6', '7', '8', '9', '0', // content + }, s.readFile(s.file1)) + s.Equal([]byte{ + QueriedSeriesStorageVersion, // version + 0xd2, 0x02, 0x96, 0x49, 0x00, 0x00, 0x00, 0x00, // timestamp + 0x4e, 0x78, 0xf9, 0xf3, //crc32 + 0x05, 0x00, 0x00, 0x00, // size + '1', '2', '3', '4', '5', // content + }, s.readFile(s.file2)) +} + +func (s *QueriedSeriesStorageSuite) TestOpenErrorInRead() { + // Arrange + s.file1.fileName = "" + + // Act + data, err := s.storage.Read() + + // Assert + s.Equal([]byte(nil), data) + s.Error(err) +} + +func (s *QueriedSeriesStorageSuite) TestChangeActiveFileOnOpenErrorWithoutValidFile() { + // Arrange + s.file1.fileName = "" + + // Act + writeErr1 := s.storage.Write([]byte("12345"), 1234567890) + writeErr2 := s.storage.Write([]byte("12345"), 1234567890) + + // Assert + s.Error(writeErr1) + s.NoError(writeErr2) + s.Equal([]byte{ + QueriedSeriesStorageVersion, // version + 0xd2, 0x02, 0x96, 0x49, 0x00, 0x00, 0x00, 0x00, // timestamp + 0x4e, 0x78, 0xf9, 0xf3, //crc32 + 0x05, 0x00, 0x00, 0x00, // size + '1', '2', '3', '4', '5', // content + }, s.readFile(s.file2)) +} + +func (s *QueriedSeriesStorageSuite) TestNoChangeActiveFileOnOpenErrorWithValidFile() { + // Arrange + s.file2.fileName = "" + s.writeFile(s.file1, []byte{ + QueriedSeriesStorageVersion, // version + 0xd2, 0x02, 0x96, 0x49, 0x00, 0x00, 0x00, 0x00, // timestamp + 0x4e, 0x78, 0xf9, 0xf3, //crc32 + 0x05, 0x00, 0x00, 0x00, + '1', '2', '3', '4', '5', + }) + + // Act + data, readErr := s.storage.Read() + writeErr1 := s.storage.Write([]byte("67890"), 987654321) + writeErr2 := s.storage.Write([]byte("67890"), 987654321) + + // Assert + s.Require().NoError(readErr) + s.Equal([]byte("12345"), data) + s.Error(writeErr1) + s.Error(writeErr2) +} + +func (s *QueriedSeriesStorageSuite) TestReadEmptyFiles() { + // Arrange + + // Act + data, err := s.storage.Read() + + // Assert + s.Equal([]byte(nil), data) + s.Equal(errors.New("no valid queried series storage"), err) +} + +func (s *QueriedSeriesStorageSuite) TestInvalidVersionInAllStorages() { + // Arrange + s.writeFile(s.file1, []byte{QueriedSeriesStorageVersion + 1}) + s.writeFile(s.file2, []byte{QueriedSeriesStorageVersion + 1}) + + // Act + data, err := s.storage.Read() + + // Assert + s.Equal([]byte(nil), data) + s.Equal(errors.New("no valid queried series storage"), err) +} + +func (s *QueriedSeriesStorageSuite) TestInvalidHeaderInAllStorages() { + // Arrange + invalidHeader := []byte{ + QueriedSeriesStorageVersion, // version + 0xd2, 0x02, 0x96, 0x49, 0x00, 0x00, 0x00, 0x00, // timestamp + 0x4e, 0x78, 0xf9, 0xf3, //crc32 + 0x05, 0x00, 0x00, + } + s.writeFile(s.file1, invalidHeader) + s.writeFile(s.file2, invalidHeader) + + // Act + data, err := s.storage.Read() + + // Assert + s.Equal([]byte(nil), data) + s.Equal(errors.New("no valid queried series storage"), err) +} + +func (s *QueriedSeriesStorageSuite) TestInvalidDataInAllStorages() { + // Arrange + invalidData := []byte{ + QueriedSeriesStorageVersion, // version + 0xd2, 0x02, 0x96, 0x49, 0x00, 0x00, 0x00, 0x00, // timestamp + 0x4e, 0x78, 0xf9, 0xf3, //crc32 + 0x05, 0x00, 0x00, 0x00, + '1', '2', '3', '4', + } + s.writeFile(s.file1, invalidData) + s.writeFile(s.file2, invalidData) + + // Act + data, err := s.storage.Read() + + // Assert + s.Equal([]byte(nil), data) + s.Equal(errors.New("no valid queried series storage"), err) +} + +func (s *QueriedSeriesStorageSuite) TestInvalidCrc32InAllStorages() { + // Arrange + invalidCrc32 := []byte{ + QueriedSeriesStorageVersion, // version + 0xd2, 0x02, 0x96, 0x49, 0x00, 0x00, 0x00, 0x00, // timestamp + 0x4e, 0x78, 0xf9, 0xf2, //crc32 + 0x05, 0x00, 0x00, 0x00, + '1', '2', '3', '4', '5', + } + s.writeFile(s.file1, invalidCrc32) + s.writeFile(s.file2, invalidCrc32) + + // Act + data, err := s.storage.Read() + + // Assert + s.Equal([]byte(nil), data) + s.Equal(errors.New("no valid queried series storage"), err) +} + +func (s *QueriedSeriesStorageSuite) TestReadFromFirstStorageAndChangeActiveStorage() { + // Arrange + s.writeFile(s.file1, []byte{ + QueriedSeriesStorageVersion, // version + 0xd2, 0x02, 0x96, 0x49, 0x00, 0x00, 0x00, 0x00, // timestamp + 0x4e, 0x78, 0xf9, 0xf3, //crc32 + 0x05, 0x00, 0x00, 0x00, + '1', '2', '3', '4', '5', + }) + + // Act + data, readErr := s.storage.Read() + _ = s.storage.Write([]byte("67890"), 987654321) + + // Assert + s.Require().NoError(readErr) + s.Equal([]byte("12345"), data) + s.Equal([]byte{ + QueriedSeriesStorageVersion, // version + 0xb1, 0x68, 0xde, 0x3a, 0x00, 0x00, 0x00, 0x00, // timestamp + 0x21, 0x33, 0xf7, 0xb8, //crc32 + 0x05, 0x00, 0x00, 0x00, // size + '6', '7', '8', '9', '0', // content + }, s.readFile(s.file2)) +} + +func (s *QueriedSeriesStorageSuite) TestReadFromSecondStorage() { + // Arrange + s.writeFile(s.file2, []byte{ + QueriedSeriesStorageVersion, // version + 0xd2, 0x02, 0x96, 0x49, 0x00, 0x00, 0x00, 0x00, // timestamp + 0x4e, 0x78, 0xf9, 0xf3, //crc32 + 0x05, 0x00, 0x00, 0x00, + '1', '2', '3', '4', '5', + }) + + // Act + data, err := s.storage.Read() + + // Assert + s.Require().NoError(err) + s.Equal([]byte("12345"), data) +} + +func (s *QueriedSeriesStorageSuite) TestReadFromStorageWithMaxTimestamp() { + // Arrange + s.writeFile(s.file1, []byte{ + QueriedSeriesStorageVersion, // version + 0xb1, 0x68, 0xde, 0x3a, 0x00, 0x00, 0x00, 0x00, // timestamp + 0x21, 0x33, 0xf7, 0xb8, //crc32 + 0x05, 0x00, 0x00, 0x00, // size + '6', '7', '8', '9', '0', // content + }) + s.writeFile(s.file2, []byte{ + QueriedSeriesStorageVersion, // version + 0xd3, 0x02, 0x96, 0x49, 0x00, 0x00, 0x00, 0x00, // timestamp + 0xfd, 0x12, 0xf7, 0xe0, //crc32 + 0x04, 0x00, 0x00, 0x00, + '6', '7', '8', '9', + }) + + // Act + data, err := s.storage.Read() + + // Assert + s.Require().NoError(err) + s.Equal([]byte("6789"), data) +} + +func (s *QueriedSeriesStorageSuite) TestReadEmptyContent() { + // Arrange + s.writeFile(s.file1, []byte{ + QueriedSeriesStorageVersion, // version + 0xb1, 0x68, 0xde, 0x3a, 0x00, 0x00, 0x00, 0x00, // timestamp + 0x41, 0x01, 0x44, 0x30, //crc32 + 0x00, 0x00, 0x00, 0x00, // size + }) + + // Act + data, err := s.storage.Read() + + // Assert + s.Require().NoError(err) + s.Equal([]byte{}, data) +} diff --git a/pp/go/storage/head/shard/wal/wal.go b/pp/go/storage/head/shard/wal/wal.go index ff2ebb674d..37c2651937 100644 --- a/pp/go/storage/head/shard/wal/wal.go +++ b/pp/go/storage/head/shard/wal/wal.go @@ -1,6 +1,7 @@ package wal import ( + "errors" "fmt" "io" "sync" @@ -13,6 +14,10 @@ const ( FileFormatVersion = 1 ) +var ( + ErrWalIsCorrupted = errors.New("wal is corrupted") +) + // SegmentWriter writer for wal segments. type SegmentWriter[TSegment EncodedSegment] interface { // CurrentSize return current shard wal size. @@ -89,6 +94,10 @@ func NewCorruptedWal[ // Close closes the wal segmentWriter. func (w *Wal[TSegment, TStats, TWriter]) Close() error { + if w.corrupted { + return nil + } + w.locker.Lock() defer w.locker.Unlock() @@ -109,7 +118,7 @@ func (w *Wal[TSegment, TStats, TWriter]) Close() error { // It is necessary to lock the LSS for reading for the commit. func (w *Wal[TSegment, TStats, TWriter]) Commit() error { if w.corrupted { - return fmt.Errorf("committing corrupted wal") + return ErrWalIsCorrupted } w.locker.Lock() @@ -130,11 +139,19 @@ func (w *Wal[TSegment, TStats, TWriter]) Commit() error { // CurrentSize returns current wal size. func (w *Wal[TSegment, TStats, TWriter]) CurrentSize() int64 { + if w.corrupted { + return 0 + } + return w.segmentWriter.CurrentSize() } // Flush wal [SegmentWriter], write all buffered data to storage. func (w *Wal[TSegment, TStats, TWriter]) Flush() error { + if w.corrupted { + return ErrWalIsCorrupted + } + w.locker.Lock() defer w.locker.Unlock() @@ -143,6 +160,10 @@ func (w *Wal[TSegment, TStats, TWriter]) Flush() error { // Sync commits the current contents of the [SegmentWriter]. func (w *Wal[TSegment, TStats, TWriter]) Sync() error { + if w.corrupted { + return ErrWalIsCorrupted + } + w.locker.Lock() defer w.locker.Unlock() @@ -152,7 +173,7 @@ func (w *Wal[TSegment, TStats, TWriter]) Sync() error { // Write the incoming inner series to wal encoder. func (w *Wal[TSegment, TStats, TWriter]) Write(innerSeriesSlice []*cppbridge.InnerSeries) (bool, error) { if w.corrupted { - return false, fmt.Errorf("writing in corrupted wal") + return false, ErrWalIsCorrupted } w.locker.Lock() diff --git a/pp/go/storage/head/task/task.go b/pp/go/storage/head/task/task.go index cc7d8645df..b69c44a8b1 100644 --- a/pp/go/storage/head/task/task.go +++ b/pp/go/storage/head/task/task.go @@ -73,7 +73,12 @@ func (t *Generic[TShard]) SetShardsNumber(number uint16) { // ExecuteOnShard execute task on shard. func (t *Generic[TShard]) ExecuteOnShard(shard TShard) { atomic.CompareAndSwapInt64(&t.executeTS, 0, time.Now().UnixMicro()) - t.errs[shard.ShardID()] = t.shardFn(shard) + if len(t.errs) == 1 { + t.errs[0] = t.shardFn(shard) + } else { + t.errs[shard.ShardID()] = t.shardFn(shard) + } + t.wg.Done() } diff --git a/pp/go/storage/head/task/waiter.go b/pp/go/storage/head/task/waiter.go index 245707e0ac..bf13ee0904 100644 --- a/pp/go/storage/head/task/waiter.go +++ b/pp/go/storage/head/task/waiter.go @@ -21,8 +21,8 @@ type Waiter[TTask Task] struct { } // NewTaskWaiter init new TaskWaiter for n task. -func NewTaskWaiter[TTask Task](n int) *Waiter[TTask] { - return &Waiter[TTask]{ +func NewTaskWaiter[TTask Task](n int) Waiter[TTask] { + return Waiter[TTask]{ tasks: make([]TTask, 0, n), } } @@ -34,10 +34,10 @@ func (tw *Waiter[TTask]) Add(t TTask) { // Wait for tasks to be completed. func (tw *Waiter[TTask]) Wait() error { - errs := make([]error, len(tw.tasks)) + var err error for _, t := range tw.tasks { - errs = append(errs, t.Wait()) + err = errors.Join(err, t.Wait()) } - return errors.Join(errs...) + return err } diff --git a/pp/go/storage/loader.go b/pp/go/storage/loader.go index 51caf5329b..03b60ff7c9 100644 --- a/pp/go/storage/loader.go +++ b/pp/go/storage/loader.go @@ -1,10 +1,14 @@ package storage import ( + "bufio" "fmt" + "io" + "math" "os" "path/filepath" "sync" + "time" "github.com/prometheus/client_golang/prometheus" @@ -22,22 +26,29 @@ import ( // Loader loads [HeadOnDisk] or [ShardOnDisk] from [WalOnDisk]. type Loader struct { - dataDir string - maxSegmentSize uint32 - registerer prometheus.Registerer + dataDir string + maxSegmentSize uint32 + registerer prometheus.Registerer + unloadDataStorageInterval time.Duration } // NewLoader init new [Loader]. -func NewLoader(dataDir string, maxSegmentSize uint32, registerer prometheus.Registerer) *Loader { +func NewLoader( + dataDir string, + maxSegmentSize uint32, + registerer prometheus.Registerer, + unloadDataStorageInterval time.Duration, +) *Loader { return &Loader{ - dataDir: dataDir, - maxSegmentSize: maxSegmentSize, - registerer: registerer, + dataDir: dataDir, + maxSegmentSize: maxSegmentSize, + registerer: registerer, + unloadDataStorageInterval: unloadDataStorageInterval, } } -// UploadHead upload [HeadOnDisk] from [WalOnDisk] by head ID. -func (l *Loader) UploadHead( +// Load upload [HeadOnDisk] from [WalOnDisk] by head ID. +func (l *Loader) Load( headRecord *catalog.Record, generation uint64, ) (_ *HeadOnDisk, corrupted bool) { @@ -50,28 +61,34 @@ func (l *Loader) UploadHead( swn := writer.NewSegmentWriteNotifier(numberOfShards, headRecord.SetLastAppendedSegmentID) for shardID := range numberOfShards { wg.Add(1) - go func(shardID uint16, shardWalFilePath string) { + go func(shardID uint16) { defer wg.Done() - shardLoadResults[shardID] = l.UploadShard(shardWalFilePath, swn, shardID) - }(shardID, filepath.Join(headDir, fmt.Sprintf("shard_%d.wal", shardID))) + shardLoadResults[shardID] = l.loadShard( + shardID, + headDir, + l.maxSegmentSize, + swn, + l.unloadDataStorageInterval, + ) + }(shardID) } wg.Wait() shards := make([]*ShardOnDisk, numberOfShards) numberOfSegmentsRead := optional.Optional[uint32]{} for shardID, res := range shardLoadResults { - shards[shardID] = res.Shard() - if res.Corrupted() { + shards[shardID] = res.shard + if res.corrupted { corrupted = true } if numberOfSegmentsRead.IsNil() { - numberOfSegmentsRead.Set(res.NumberOfSegments()) - } else if numberOfSegmentsRead.Value() != res.NumberOfSegments() { + numberOfSegmentsRead.Set(res.numberOfSegments) + } else if numberOfSegmentsRead.Value() != res.numberOfSegments { corrupted = true // calculating maximum number of segments (critical for remote write). - if numberOfSegmentsRead.Value() < res.NumberOfSegments() { - numberOfSegmentsRead.Set(res.NumberOfSegments()) + if numberOfSegmentsRead.Value() < res.numberOfSegments { + numberOfSegmentsRead.Set(res.numberOfSegments) } } } @@ -107,92 +124,251 @@ func (l *Loader) UploadHead( return h, corrupted } -// UploadShard upload [ShardOnDisk] from [WalOnDisk]. -func (l *Loader) UploadShard( - shardFilePath string, - swn *writer.SegmentWriteNotifier, +func (l *Loader) loadShard( shardID uint16, + dir string, + maxSegmentSize uint32, + notifier *writer.SegmentWriteNotifier, + unloadDataStorageInterval time.Duration, ) ShardLoadResult { - res := ShardLoadResult{corrupted: true} + shardDataLoader := NewShardDataLoader(shardID, dir, maxSegmentSize, notifier, unloadDataStorageInterval) + err := shardDataLoader.Load() + return ShardLoadResult{ + corrupted: err != nil, + numberOfSegments: shardDataLoader.shardData.numberOfSegments, + shard: shard.NewShard( + shardDataLoader.shardData.lss, + shardDataLoader.shardData.dataStorage, + shardDataLoader.shardData.unloadedDataStorage, + shardDataLoader.shardData.queriedSeriesStorage, + shardDataLoader.shardData.wal, + shardID, + ), + } +} + +type ShardLoadResult struct { + shard *ShardOnDisk + numberOfSegments uint32 + corrupted bool +} - //revive:disable-next-line:add-constant file permissions simple readable as octa-number - shardFile, err := os.OpenFile(shardFilePath, os.O_RDWR, 0o600) // #nosec G304 // it's meant to be that way +type ShardData struct { + notifier *writer.SegmentWriteNotifier + lss *shard.LSS + dataStorage *shard.DataStorage + wal *WalOnDisk + unloadedDataStorage *shard.UnloadedDataStorage + queriedSeriesStorage *shard.QueriedSeriesStorage + numberOfSegments uint32 +} + +type ShardDataLoader struct { + shardID uint16 + dir string + maxSegmentSize uint32 + shardData ShardData + notifier *writer.SegmentWriteNotifier + unloadDataStorageInterval time.Duration +} + +func NewShardDataLoader( + shardID uint16, + dir string, + maxSegmentSize uint32, + notifier *writer.SegmentWriteNotifier, + unloadDataStorageInterval time.Duration, +) ShardDataLoader { + return ShardDataLoader{ + shardID: shardID, + dir: dir, + maxSegmentSize: maxSegmentSize, + notifier: notifier, + unloadDataStorageInterval: unloadDataStorageInterval, + } +} + +func (l *ShardDataLoader) Load() (err error) { + l.shardData = ShardData{ + lss: shard.NewLSS(), + dataStorage: shard.NewDataStorage(), + wal: wal.NewCorruptedWal[ + *cppbridge.EncodedSegment, + cppbridge.WALEncoderStats, + *writer.Buffered[*cppbridge.EncodedSegment], + ](), + } + + shardWalFile, err := os.OpenFile(GetShardWalFilename(l.dir, l.shardID), os.O_RDWR, 0666) if err != nil { - logger.Debugf("failed to open file shard id %d: %w", shardID, err) - return res + return err } + defer func() { - if res.corrupted { - _ = shardFile.Close() + if err != nil { + _ = shardWalFile.Close() } }() - _, encoderVersion, _, err := reader.ReadHeader(shardFile) + queriedSeriesStorageIsEmpty := true + if l.unloadDataStorageInterval > 0 { + l.shardData.unloadedDataStorage = shard.NewUnloadedDataStorage(shard.NewFileStorage(GetUnloadedDataStorageFilename(l.dir, l.shardID))) + queriedSeriesStorageIsEmpty, _ = l.loadQueriedSeries() + } + + decoder, err := l.loadWalFile(bufio.NewReaderSize(shardWalFile, 1024*1024*4), queriedSeriesStorageIsEmpty) if err != nil { - logger.Debugf("failed to read wal header: %w", err) - return res + return err } - lss := shard.NewLSS() - decoder := cppbridge.NewHeadWalDecoder(lss.Target(), encoderVersion) - dataStorage := shard.NewDataStorage() + if err = l.createShardWal(shardWalFile, decoder); err != nil { + return err + } + + return nil +} + +func (l *ShardDataLoader) loadWalFile( + rd io.Reader, + queriedSeriesStorageIsEmpty bool, +) (*cppbridge.HeadWalDecoder, error) { + _, encoderVersion, _, err := reader.ReadHeader(rd) + if err != nil { + return nil, fmt.Errorf("failed to read wal header: %w", err) + } + + var unloader *dataUnloader + if !queriedSeriesStorageIsEmpty { + unloader = &dataUnloader{ + unloadedDataStorage: l.shardData.unloadedDataStorage, + unloadedIntervalIndex: math.MinInt64, + unloadInterval: l.unloadDataStorageInterval, + unloader: l.shardData.dataStorage.CreateUnusedSeriesDataUnloader(), + } + } + + decoder := cppbridge.NewHeadWalDecoder(l.shardData.lss.Target(), encoderVersion) + l.shardData.numberOfSegments, err = l.loadSegments( + rd, + decoder, + l.shardData.dataStorage, + unloader, + ) + return decoder, err +} - if err = wal.NewSegmentWalReader(shardFile, reader.NewSegment).ForEachSegment(func(s *reader.Segment) error { - if decodeErr := dataStorage.DecodeSegment(decoder, s.Bytes()); decodeErr != nil { +func (l *ShardDataLoader) createShardWal(shardWalFile *os.File, walDecoder *cppbridge.HeadWalDecoder) error { + if sw, err := writer.NewBuffered(l.shardID, shardWalFile, writer.WriteSegment[*cppbridge.EncodedSegment], l.notifier); err != nil { + return err + } else { + l.notifier.Set(l.shardID, l.shardData.numberOfSegments) + l.shardData.wal = wal.NewWal(walDecoder.CreateEncoder(), sw, l.maxSegmentSize) + return nil + } +} + +type dataUnloader struct { + unloader *cppbridge.UnusedSeriesDataUnloader + unloadedDataStorage *shard.UnloadedDataStorage + unloadedIntervalIndex int64 + unloadInterval time.Duration +} + +func (d *dataUnloader) Unload(createTs, encodeTs time.Duration) error { + intervalIndex := int64(createTs / d.unloadInterval) + + if d.unloadedIntervalIndex == math.MinInt64 { + d.unloadedIntervalIndex = intervalIndex + + createTs = encodeTs + intervalIndex = int64(createTs / d.unloadInterval) + } + + if intervalIndex > d.unloadedIntervalIndex { + if header, err := d.unloadedDataStorage.WriteSnapshot(d.unloader.CreateSnapshot()); err != nil { + return fmt.Errorf("failed to write unloaded data: %w", err) + } else { + d.unloadedDataStorage.WriteIndex(header) + } + d.unloader.Unload() + + d.unloadedIntervalIndex = intervalIndex + } + + return nil +} + +func (l *ShardDataLoader) loadSegments( + rd io.Reader, + walDecoder *cppbridge.HeadWalDecoder, + dataStorage *shard.DataStorage, + unloader *dataUnloader, +) (uint32, error) { + numberOfSegments := uint32(0) + + if err := wal.NewSegmentWalReader(rd, reader.NewSegment).ForEachSegment(func(segment *reader.Segment) error { + createTs, encodeTs, decodeErr := dataStorage.DecodeSegment(walDecoder, segment.Bytes()) + if decodeErr != nil { return fmt.Errorf("failed to decode segment: %w", decodeErr) } - res.numberOfSegments++ + numberOfSegments++ + + if createTs != 0 && unloader != nil { + if err := unloader.Unload(time.Duration(createTs), time.Duration(encodeTs)); err != nil { + return fmt.Errorf("failed to unload data: %w", err) + } + } return nil }); err != nil { logger.Debugf(err.Error()) - return res + return 0, err } - sw, err := writer.NewBuffered(shardID, shardFile, writer.WriteSegment[*cppbridge.EncodedSegment], swn) - if err != nil { - logger.Debugf("failed to create buffered writer shard id %d: %w", shardID, err) - return res - } + return numberOfSegments, nil +} - swn.Set(shardID, res.numberOfSegments) - res.corrupted = false - res.shard = shard.NewShard(lss, dataStorage, wal.NewWal(decoder.CreateEncoder(), sw, l.maxSegmentSize), shardID) +func (l *ShardDataLoader) loadQueriedSeries() (bool, error) { + file1 := shard.NewFileStorage(GetQueriedSeriesStorageFilename(l.dir, l.shardID, 0)) + file2 := shard.NewFileStorage(GetQueriedSeriesStorageFilename(l.dir, l.shardID, 1)) - return res -} + l.shardData.queriedSeriesStorage = shard.NewQueriedSeriesStorage(file1, file2) -// -// ShardLoadResult -// + if queriedSeries, err := l.shardData.queriedSeriesStorage.Read(); err != nil { + if file1.IsEmpty() && file2.IsEmpty() { + return true, nil + } -// ShardLoadResult the result of uploading the [ShardOnDisk] from the [WalOnDisk]. -type ShardLoadResult struct { - shard *ShardOnDisk - numberOfSegments uint32 - corrupted bool + logger.Warnf("error loading queried series: %v", err) + } else { + if !l.shardData.dataStorage.SetQueriedSeriesBitset(queriedSeries) { + logger.Warnf("error set queried series in storage: %v", err) + } + } + + return false, nil } -// Corrupted returns true if [ShardOnDisk] is corrupted. -func (sr *ShardLoadResult) Corrupted() bool { - return sr.corrupted +func GetShardWalFilename(dir string, shardID uint16) string { + return filepath.Join(dir, fmt.Sprintf("shard_%d.wal", shardID)) } -// NumberOfSegments returns number of segments in [ShardOnDisk]s. -func (sr *ShardLoadResult) NumberOfSegments() uint32 { - return sr.numberOfSegments +func GetUnloadedDataStorageFilename(dir string, shardID uint16) string { + return filepath.Join(dir, fmt.Sprintf("unloaded_%d.ds", shardID)) } -// Shard returns [*ShardOnDisk] or nil. -func (sr *ShardLoadResult) Shard() *ShardOnDisk { - return sr.shard +func GetQueriedSeriesStorageFilename(dir string, shardID uint16, index uint8) string { + return filepath.Join(dir, fmt.Sprintf("queried_series_%d_%d.ds", shardID, index)) } // isNumberOfSegmentsMismatched check number of segments loaded and last appended to record. func isNumberOfSegmentsMismatched(record *catalog.Record, loadedSegments uint32) bool { - if record.LastAppendedSegmentID() == nil { - return loadedSegments != 0 - } - return *record.LastAppendedSegmentID()+1 != loadedSegments + return false + + // TODO: uncomment this code block + //if record.LastAppendedSegmentID() == nil { + // return loadedSegments != 0 + //} + //return *record.LastAppendedSegmentID()+1 != loadedSegments } diff --git a/pp/go/storage/loader_test.go b/pp/go/storage/loader_test.go new file mode 100644 index 0000000000..c339f02857 --- /dev/null +++ b/pp/go/storage/loader_test.go @@ -0,0 +1,393 @@ +package storage_test + +import ( + "context" + "os" + "path/filepath" + "testing" + "time" + + "github.com/google/uuid" + "github.com/jonboulle/clockwork" + "github.com/prometheus/client_golang/prometheus" + "github.com/prometheus/prometheus/model/labels" + "github.com/prometheus/prometheus/pp/go/cppbridge" + "github.com/prometheus/prometheus/pp/go/storage" + "github.com/prometheus/prometheus/pp/go/storage/catalog" + "github.com/prometheus/prometheus/pp/go/storage/head/services" + "github.com/prometheus/prometheus/pp/go/storage/storagetest" + "github.com/stretchr/testify/suite" +) + +const ( + numberOfShards uint16 = 2 + + maxSegmentSize uint32 = 1024 + + unloadDataStorageInterval time.Duration = 100 +) + +type idGeneratorStub struct { + uuid uuid.UUID +} + +func newIdGeneratorStub() *idGeneratorStub { + return &idGeneratorStub{ + uuid: uuid.New(), + } +} + +func (g *idGeneratorStub) Generate() uuid.UUID { + return g.uuid +} + +func (g *idGeneratorStub) last() string { + return g.uuid.String() +} + +type HeadLoadSuite struct { + suite.Suite + dataDir string + ctx context.Context + clock clockwork.Clock + headIdGenerator *idGeneratorStub + catalog *catalog.Catalog +} + +func TestHeadLoadSuite(t *testing.T) { + suite.Run(t, new(HeadLoadSuite)) +} + +func (s *HeadLoadSuite) SetupTest() { + s.dataDir = s.createDataDirectory() + + s.clock = clockwork.NewFakeClockAt(time.Now()) + s.headIdGenerator = newIdGeneratorStub() + s.createCatalog() +} + +func (s *HeadLoadSuite) createDataDirectory() string { + dataDir := filepath.Join(s.T().TempDir(), "data") + s.Require().NoError(os.MkdirAll(dataDir, os.ModeDir)) + return dataDir +} + +func (s *HeadLoadSuite) createCatalog() { + l, err := catalog.NewFileLogV2(filepath.Join(s.dataDir, "catalog.log")) + s.Require().NoError(err) + + s.catalog, err = catalog.New( + s.clock, + l, + s.headIdGenerator, + catalog.DefaultMaxLogFileSize, + nil, + ) + s.Require().NoError(err) +} + +func (s *HeadLoadSuite) headDir() string { + return filepath.Join(s.dataDir, s.headIdGenerator.last()) +} + +func (s *HeadLoadSuite) createHead(unloadDataStorageInterval time.Duration) (*storage.HeadOnDisk, error) { + return storage.NewBuilder( + s.catalog, + s.dataDir, + maxSegmentSize, + prometheus.DefaultRegisterer, + unloadDataStorageInterval, + ).Build(0, numberOfShards) +} + +func (s *HeadLoadSuite) mustCreateHead(unloadDataStorageInterval time.Duration) *storage.HeadOnDisk { + h, err := s.createHead(unloadDataStorageInterval) + s.Require().NoError(err) + return h +} + +func (s *HeadLoadSuite) loadHead(unloadDataStorageInterval time.Duration) (*storage.HeadOnDisk, bool) { + record, err := s.catalog.Get(s.headIdGenerator.last()) + s.Require().NoError(err) + + return storage.NewLoader(s.dataDir, maxSegmentSize, prometheus.DefaultRegisterer, unloadDataStorageInterval).Load(record, 0) +} + +func (s *HeadLoadSuite) mustLoadHead(unloadDataStorageInterval time.Duration) *storage.HeadOnDisk { + loadedHead, corrupted := s.loadHead(unloadDataStorageInterval) + s.False(corrupted) + + return loadedHead +} + +func (s *HeadLoadSuite) lockFileForCreation(fileName string) { + s.Require().NoError(os.RemoveAll(fileName)) + s.Require().NoError(os.Mkdir(fileName, os.ModeDir)) +} + +func (s *HeadLoadSuite) appendTimeSeries(head *storage.HeadOnDisk, timeSeries []storagetest.TimeSeries) { + storagetest.MustAppendTimeSeries(&s.Suite, head, timeSeries) +} + +func (s *HeadLoadSuite) shards(head *storage.HeadOnDisk) (result []*storage.ShardOnDisk) { + for shard := range head.RangeShards() { + result = append(result, shard) + } + + return +} + +func (s *HeadLoadSuite) TestErrorCreateShardFileInOneShard() { + // Arrange + s.Require().NoError(os.Mkdir(s.headDir(), 0), os.ModeDir) + s.lockFileForCreation(storage.GetShardWalFilename(s.headDir(), 0)) + + // Act + head, err := s.createHead(0) + + // Assert + s.Require().Error(err) + s.Nil(head) +} + +func (s *HeadLoadSuite) TestErrorOpenShardFileInOneShard() { + // Arrange + sourceHead := s.mustCreateHead(0) + s.NoError(sourceHead.Close()) + + s.Require().NoError(os.Remove(storage.GetShardWalFilename(s.headDir(), 0))) + + // Act + head, corrupted := s.loadHead(0) + + // Assert + s.True(corrupted) + s.Nil(s.shards(head)[0].UnloadedDataStorage()) + s.Require().NoError(head.Close()) +} + +func (s *HeadLoadSuite) TestErrorOpenShardFileInAllShards() { + // Arrange + sourceHead := s.mustCreateHead(0) + s.NoError(sourceHead.Close()) + + s.Require().NoError(os.Remove(storage.GetShardWalFilename(s.headDir(), 0))) + s.Require().NoError(os.Remove(storage.GetShardWalFilename(s.headDir(), 1))) + + // Act + head, corrupted := s.loadHead(0) + + // Assert + s.True(corrupted) + s.Nil(s.shards(head)[0].UnloadedDataStorage()) + s.Nil(s.shards(head)[1].UnloadedDataStorage()) + s.Require().NoError(head.Close()) +} + +func (s *HeadLoadSuite) TestLoadWithDisabledDataUnloading() { + // Arrange + sourceHead := s.mustCreateHead(0) + s.appendTimeSeries(sourceHead, []storagetest.TimeSeries{ + { + Labels: labels.FromStrings("__name__", "wal_metric"), + Samples: []cppbridge.Sample{ + {Timestamp: 0, Value: 1}, + {Timestamp: 1, Value: 2}, + {Timestamp: 2, Value: 3}, + }, + }, + }) + s.Require().NoError(sourceHead.Close()) + + // Act + loadedHead := s.mustLoadHead(0) + + chunks, queryResult := s.shards(loadedHead)[0].DataStorage().Query(cppbridge.HeadDataStorageQuery{ + StartTimestampMs: 0, + EndTimestampMs: 2, + LabelSetIDs: []uint32{0}, + }) + err := loadedHead.Close() + + // Assert + s.Require().NoError(err) + s.Nil(s.shards(loadedHead)[0].UnloadedDataStorage()) + s.Nil(s.shards(loadedHead)[0].QueriedSeriesStorage()) + s.Nil(s.shards(loadedHead)[1].UnloadedDataStorage()) + s.Nil(s.shards(loadedHead)[1].QueriedSeriesStorage()) + s.Equal(cppbridge.DataStorageQueryStatusSuccess, queryResult.Status) + s.Equal(storagetest.SamplesMap{ + 0: []cppbridge.Sample{ + {Timestamp: 0, Value: 1}, + {Timestamp: 1, Value: 2}, + {Timestamp: 2, Value: 3}, + }, + }, storagetest.GetSamplesFromSerializedChunks(chunks)) + s.Equal([]cppbridge.Labels{ + {{Name: "__name__", Value: "wal_metric"}}, + }, s.shards(loadedHead)[0].LSS().Target().GetLabelSets([]uint32{0}).LabelsSets()) +} + +func (s *HeadLoadSuite) TestAppendAfterLoad() { + // Arrange + sourceHead := s.mustCreateHead(0) + s.appendTimeSeries(sourceHead, []storagetest.TimeSeries{ + { + Labels: labels.FromStrings("__name__", "wal_metric"), + Samples: []cppbridge.Sample{ + {Timestamp: 0, Value: 1}, + {Timestamp: 1, Value: 2}, + {Timestamp: 2, Value: 3}, + }, + }, + }) + s.Require().NoError(sourceHead.Close()) + + // Act + loadedHead := s.mustLoadHead(0) + s.appendTimeSeries(loadedHead, []storagetest.TimeSeries{ + { + Labels: labels.FromStrings("__name__", "wal_metric"), + Samples: []cppbridge.Sample{ + {Timestamp: 3, Value: 4}, + }, + }, + }) + + chunks, queryResult := s.shards(loadedHead)[0].DataStorage().Query(cppbridge.HeadDataStorageQuery{ + StartTimestampMs: 0, + EndTimestampMs: 4, + LabelSetIDs: []uint32{0}, + }) + + err := loadedHead.Close() + + // Assert + s.Require().NoError(err) + s.Equal(cppbridge.DataStorageQueryStatusSuccess, queryResult.Status) + s.Equal(storagetest.SamplesMap{ + 0: []cppbridge.Sample{ + {Timestamp: 0, Value: 1}, + {Timestamp: 1, Value: 2}, + {Timestamp: 2, Value: 3}, + {Timestamp: 3, Value: 4}, + }, + }, storagetest.GetSamplesFromSerializedChunks(chunks)) + s.Equal([]cppbridge.Labels{ + {{Name: "__name__", Value: "wal_metric"}}, + }, s.shards(loadedHead)[0].LSS().Target().GetLabelSets([]uint32{0}).LabelsSets()) +} + +func (s *HeadLoadSuite) TestLoadWithEnabledDataUnloading() { + // Arrange + sourceHead := s.mustCreateHead(0) + s.appendTimeSeries(sourceHead, []storagetest.TimeSeries{ + { + Labels: labels.FromStrings("__name__", "wal_metric"), + Samples: []cppbridge.Sample{ + {Timestamp: 0, Value: 1}, + {Timestamp: 1, Value: 2}, + {Timestamp: 2, Value: 3}, + }, + }, + }) + s.appendTimeSeries(sourceHead, []storagetest.TimeSeries{ + { + Labels: labels.FromStrings("__name__", "wal_metric"), + Samples: []cppbridge.Sample{ + {Timestamp: 100, Value: 1}, + {Timestamp: 101, Value: 2}, + {Timestamp: 102, Value: 3}, + }, + }, + }) + + s.Require().NoError(sourceHead.Close()) + + // Act + loadedHead := s.mustLoadHead(unloadDataStorageInterval) + + // Assert + s.Require().NotNil(s.shards(loadedHead)[0].UnloadedDataStorage()) + s.Require().NotNil(s.shards(loadedHead)[1].UnloadedDataStorage()) + s.True(s.shards(loadedHead)[0].UnloadedDataStorage().IsEmpty()) + s.True(s.shards(loadedHead)[1].UnloadedDataStorage().IsEmpty()) + s.Require().NoError(loadedHead.Close()) +} + +func (s *HeadLoadSuite) TestLoadWithDataUnloading() { + // Arrange + sourceHead := s.mustCreateHead(unloadDataStorageInterval) + s.appendTimeSeries(sourceHead, []storagetest.TimeSeries{ + { + Labels: labels.FromStrings("__name__", "wal_metric"), + Samples: []cppbridge.Sample{ + {Timestamp: 0, Value: 1}, + {Timestamp: 1, Value: 2}, + {Timestamp: 2, Value: 3}, + }, + }, + }) + s.appendTimeSeries(sourceHead, []storagetest.TimeSeries{ + { + Labels: labels.FromStrings("__name__", "wal_metric"), + Samples: []cppbridge.Sample{ + {Timestamp: 100, Value: 1}, + {Timestamp: 101, Value: 2}, + {Timestamp: 102, Value: 3}, + }, + }, + }) + + s.Require().NoError(services.UnloadUnusedSeriesDataWithHead(sourceHead)) + s.Require().NoError(sourceHead.Close()) + + // Act + loadedHead := s.mustLoadHead(unloadDataStorageInterval) + + // Assert + s.NotNil(s.shards(loadedHead)[0].UnloadedDataStorage()) + s.NotNil(s.shards(loadedHead)[1].UnloadedDataStorage()) + s.False(s.shards(loadedHead)[0].UnloadedDataStorage().IsEmpty()) + s.True(s.shards(loadedHead)[1].UnloadedDataStorage().IsEmpty()) + s.Require().NoError(loadedHead.Close()) +} + +func (s *HeadLoadSuite) TestErrorDataUnloading() { + // Arrange + sourceHead := s.mustCreateHead(unloadDataStorageInterval) + s.appendTimeSeries(sourceHead, []storagetest.TimeSeries{ + { + Labels: labels.FromStrings("__name__", "wal_metric"), + Samples: []cppbridge.Sample{ + {Timestamp: 0, Value: 1}, + {Timestamp: 1, Value: 2}, + {Timestamp: 2, Value: 3}, + }, + }, + }) + s.appendTimeSeries(sourceHead, []storagetest.TimeSeries{ + { + Labels: labels.FromStrings("__name__", "wal_metric"), + Samples: []cppbridge.Sample{ + {Timestamp: 100, Value: 1}, + {Timestamp: 101, Value: 2}, + {Timestamp: 102, Value: 3}, + }, + }, + }) + + s.Require().NoError(services.UnloadUnusedSeriesDataWithHead(sourceHead)) + s.Require().NoError(sourceHead.Close()) + + // Act + s.lockFileForCreation(storage.GetUnloadedDataStorageFilename(s.headDir(), 0)) + s.lockFileForCreation(storage.GetUnloadedDataStorageFilename(s.headDir(), 1)) + loadedHead, corrupted := s.loadHead(unloadDataStorageInterval) + + // Assert + s.True(corrupted) + s.NotNil(s.shards(loadedHead)[0].UnloadedDataStorage()) + s.NotNil(s.shards(loadedHead)[1].UnloadedDataStorage()) + s.Require().NoError(loadedHead.Close()) +} diff --git a/pp/go/storage/manager.go b/pp/go/storage/manager.go index 03cee9d302..ee93604a2b 100644 --- a/pp/go/storage/manager.go +++ b/pp/go/storage/manager.go @@ -114,6 +114,7 @@ func NewManager( triggerNotifier *ReloadBlocksTriggerNotifier, readyNotifier ready.Notifier, r prometheus.Registerer, + unloadDataStorageInterval time.Duration, ) (*Manager, error) { dirStat, err := os.Stat(dataDir) if err != nil { @@ -124,9 +125,9 @@ func NewManager( return nil, fmt.Errorf("%s is not directory", dataDir) } - builder := NewBuilder(hcatalog, dataDir, options.MaxSegmentSize, r) + builder := NewBuilder(hcatalog, dataDir, options.MaxSegmentSize, r, unloadDataStorageInterval) - loader := NewLoader(dataDir, options.MaxSegmentSize, r) + loader := NewLoader(dataDir, options.MaxSegmentSize, r, unloadDataStorageInterval) cfg := NewConfig(options.NumberOfShards) @@ -390,7 +391,7 @@ func uploadOrBuildHead( return builder.Build(generation, numberOfShards) } - h, corrupted := loader.UploadHead(headRecords[0], generation) + h, corrupted := loader.Load(headRecords[0], generation) if corrupted { if !headRecords[0].Corrupted() { if _, setCorruptedErr := hcatalog.SetCorrupted(headRecords[0].ID()); setCorruptedErr != nil { diff --git a/pp/go/storage/querier/interface.go b/pp/go/storage/querier/interface.go index 6066e89a5f..ca1ed3a0ef 100644 --- a/pp/go/storage/querier/interface.go +++ b/pp/go/storage/querier/interface.go @@ -5,6 +5,7 @@ import ( "github.com/prometheus/prometheus/pp/go/cppbridge" "github.com/prometheus/prometheus/pp/go/model" + "github.com/prometheus/prometheus/pp/go/storage/head/shard" ) // @@ -39,10 +40,10 @@ type Task interface { // DataStorage the minimum required [DataStorage] implementation. type DataStorage interface { // InstantQuery returns samples for instant query from data storage. - InstantQuery(maxt, valueNotFoundTimestampValue int64, ids []uint32) []cppbridge.Sample + InstantQuery(maxt, valueNotFoundTimestampValue int64, ids []uint32) ([]cppbridge.Sample, cppbridge.DataStorageQueryResult) // QueryDataStorage returns serialized chunks from data storage. - Query(query cppbridge.HeadDataStorageQuery) *cppbridge.HeadDataStorageSerializedChunks + Query(query cppbridge.HeadDataStorageQuery) (*cppbridge.HeadDataStorageSerializedChunks, cppbridge.DataStorageQueryResult) // WithRLock calls fn on raw [cppbridge.HeadDataStorage] with read lock. WithRLock(fn func(ds *cppbridge.HeadDataStorage) error) error @@ -91,6 +92,10 @@ type Shard[TDataStorage DataStorage, TLSS LSS] interface { // ShardID returns the shard ID. ShardID() uint16 + + LoadAndQuerySeriesData() error + + LoadAndQuerySeriesDataTask() *shard.LoadAndQuerySeriesDataTask } // @@ -115,6 +120,12 @@ type Head[ // Enqueue the task to be executed on shards [Head]. Enqueue(t TGenericTask) + // EnqueueOnShard the task to be executed on head on specific shard. + EnqueueOnShard(t TGenericTask, shardID uint16) + // NumberOfShards returns current number of shards in to [Head]. NumberOfShards() uint16 + + // IsReadOnly returns true if the [Head] has switched to read-only. + IsReadOnly() bool } diff --git a/pp/go/storage/querier/load_and_query_waiter.go b/pp/go/storage/querier/load_and_query_waiter.go new file mode 100644 index 0000000000..5c5bb8a289 --- /dev/null +++ b/pp/go/storage/querier/load_and_query_waiter.go @@ -0,0 +1,48 @@ +package querier + +import ( + "github.com/prometheus/prometheus/pp/go/storage/head/shard" + "github.com/prometheus/prometheus/pp/go/storage/head/task" +) + +const ( + // dsLoadAndQuerySeriesData + dsLoadAndQuerySeriesData = "data_storage_load_and_query_series_data" +) + +type LoadAndQueryWaiter[ + TTask Task, + TDataStorage DataStorage, + TLSS LSS, + TShard Shard[TDataStorage, TLSS], + THead Head[TTask, TDataStorage, TLSS, TShard], +] struct { + waiter task.Waiter[shard.Task] + head THead +} + +func NewLoadAndQueryWaiter[ + TTask Task, + TDataStorage DataStorage, + TLSS LSS, + TShard Shard[TDataStorage, TLSS], + THead Head[TTask, TDataStorage, TLSS, TShard], +](head THead) LoadAndQueryWaiter[TTask, TDataStorage, TLSS, TShard, THead] { + return LoadAndQueryWaiter[TTask, TDataStorage, TLSS, TShard, THead]{ + head: head, + } +} + +func (l *LoadAndQueryWaiter[TTask, TDataStorage, TLSS, TShard, THead]) Add(s TShard, querier uintptr) { + l.waiter.Add(s.LoadAndQuerySeriesDataTask().Add(querier, func() shard.Task { + t := l.head.CreateTask(dsLoadAndQuerySeriesData, func(s TShard) error { + return s.LoadAndQuerySeriesData() + }) + l.head.EnqueueOnShard(t, s.ShardID()) + return t + })) +} + +func (l *LoadAndQueryWaiter[TTask, TDataStorage, TLSS, TShard, THead]) Wait() error { + return l.waiter.Wait() +} diff --git a/pp/go/storage/querier/querier.go b/pp/go/storage/querier/querier.go index ea3f3073a3..b456fec7f0 100644 --- a/pp/go/storage/querier/querier.go +++ b/pp/go/storage/querier/querier.go @@ -191,21 +191,27 @@ func (q *Querier[TTask, TDataStorage, TLSS, TShard, THead]) selectInstant( numberOfShards := q.head.NumberOfShards() seriesSets := make([]storage.SeriesSet, numberOfShards) + loadAndQueryWaiter := NewLoadAndQueryWaiter[TTask, TDataStorage, TLSS, TShard, THead](q.head) tDataStorageQuery := q.head.CreateTask( dsQueryInstantQuerier, - func(shard TShard) error { - shardID := shard.ShardID() + func(s TShard) error { + shardID := s.ShardID() lssQueryResult := lssQueryResults[shardID] if lssQueryResult == nil { seriesSets[shardID] = &SeriesSet{} return nil } + samples, result := s.DataStorage().InstantQuery(q.maxt, valueNotFoundTimestampValue, lssQueryResult.IDs()) + if result.Status == cppbridge.DataStorageQueryStatusNeedDataLoad { + loadAndQueryWaiter.Add(s, result.Querier) + } + seriesSets[shardID] = NewInstantSeriesSet( lssQueryResult, snapshots[shardID], valueNotFoundTimestampValue, - shard.DataStorage().InstantQuery(q.maxt, valueNotFoundTimestampValue, lssQueryResult.IDs()), + samples, ) return nil @@ -214,6 +220,12 @@ func (q *Querier[TTask, TDataStorage, TLSS, TShard, THead]) selectInstant( q.head.Enqueue(tDataStorageQuery) _ = tDataStorageQuery.Wait() + if err := loadAndQueryWaiter.Wait(); err != nil { + // TODO: Unrecoverable error + // q.head.UnrecoverableError(err) + return storage.ErrSeriesSet(err) + } + return storage.NewMergeSeriesSet(seriesSets, storage.ChainedSeriesMerge) } @@ -301,33 +313,38 @@ func queryDataStorage[ mint, maxt int64, ) []*cppbridge.HeadDataStorageSerializedChunks { serializedChunksShards := make([]*cppbridge.HeadDataStorageSerializedChunks, head.NumberOfShards()) + loadAndQueryWaiter := NewLoadAndQueryWaiter[TTask, TDataStorage, TLSS, TShard, THead](head) tDataStorageQuery := head.CreateTask( taskName, - func(shard TShard) error { - shardID := shard.ShardID() + func(s TShard) error { + shardID := s.ShardID() lssQueryResult := lssQueryResults[shardID] if lssQueryResult == nil { return nil } - serializedChunks := shard.DataStorage().Query(cppbridge.HeadDataStorageQuery{ + var result cppbridge.DataStorageQueryResult + serializedChunksShards[shardID], result = s.DataStorage().Query(cppbridge.HeadDataStorageQuery{ StartTimestampMs: mint, EndTimestampMs: maxt, LabelSetIDs: lssQueryResult.IDs(), }) - - if serializedChunks.NumberOfChunks() == 0 { - return nil + if result.Status == cppbridge.DataStorageQueryStatusNeedDataLoad { + loadAndQueryWaiter.Add(s, result.Querier) } - serializedChunksShards[shardID] = serializedChunks - return nil }, ) head.Enqueue(tDataStorageQuery) _ = tDataStorageQuery.Wait() + if err := loadAndQueryWaiter.Wait(); err != nil { + // TODO: Unrecoverable error + // q.head.UnrecoverableError(err) + return nil + } + return serializedChunksShards } diff --git a/pp/go/storage/querier/querier_test.go b/pp/go/storage/querier/querier_test.go index 43483ad280..4264873e4d 100644 --- a/pp/go/storage/querier/querier_test.go +++ b/pp/go/storage/querier/querier_test.go @@ -1,73 +1,313 @@ package querier_test import ( + "context" + "os" + "path/filepath" "testing" + "time" + "github.com/jonboulle/clockwork" + "github.com/prometheus/client_golang/prometheus" + "github.com/prometheus/prometheus/model/labels" "github.com/prometheus/prometheus/pp/go/cppbridge" - "github.com/prometheus/prometheus/pp/go/storage/head/head" + "github.com/prometheus/prometheus/pp/go/storage" + "github.com/prometheus/prometheus/pp/go/storage/catalog" + "github.com/prometheus/prometheus/pp/go/storage/head/services" "github.com/prometheus/prometheus/pp/go/storage/head/shard" + "github.com/prometheus/prometheus/pp/go/storage/head/task" "github.com/prometheus/prometheus/pp/go/storage/querier" + "github.com/prometheus/prometheus/pp/go/storage/storagetest" + "github.com/stretchr/testify/suite" ) -func TestXxx(t *testing.T) { - lss := &shard.LSS{} - ds := shard.NewDataStorage() - wl := &testWal{} - sd := shard.NewShard(lss, ds, wl, 0) - id := "test-head-id" - generation := uint64(0) - - h := head.NewHead( - id, - []*shard.Shard[*testWal]{sd}, - shard.NewPerGoroutineShard[*testWal], - nil, - generation, - nil, - ) +const ( + numberOfShards uint16 = 2 + + maxSegmentSize uint32 = 1024 +) - q := querier.NewQuerier( - h, - querier.NewNoOpShardedDeduplicator, - 0, - 0, +type Querier = querier.Querier[ + *task.Generic[*storage.PerGoroutineShard], + *shard.DataStorage, + *shard.LSS, + *storage.PerGoroutineShard, + *storage.HeadOnDisk, +] + +type QuerierSuite struct { + suite.Suite + dataDir string + context context.Context + head *storage.HeadOnDisk +} + +func TestQuerierSuite(t *testing.T) { + suite.Run(t, new(QuerierSuite)) +} + +func (s *QuerierSuite) SetupTest() { + s.dataDir = s.createDataDirectory() + s.context = context.Background() + + s.head = s.mustCreateHead(1) +} + +func (s *QuerierSuite) createDataDirectory() string { + dataDir := filepath.Join(s.T().TempDir(), "data") + s.Require().NoError(os.MkdirAll(dataDir, os.ModeDir)) + return dataDir +} + +func (s *QuerierSuite) mustCreateCatalog() *catalog.Catalog { + l, err := catalog.NewFileLogV2(filepath.Join(s.dataDir, "catalog.log")) + s.Require().NoError(err) + + c, err := catalog.New( + clockwork.NewFakeClock(), + l, + &catalog.DefaultIDGenerator{}, + catalog.DefaultMaxLogFileSize, nil, - querier.NewMetrics(nil, "test"), ) - _ = q + s.Require().NoError(err) - t.Log("end") + return c } -// testWal test implementation wal. -type testWal struct{} +func (s *QuerierSuite) mustCreateHead(unloadDataStorageInterval time.Duration) *storage.HeadOnDisk { + h, err := storage.NewBuilder( + s.mustCreateCatalog(), + s.dataDir, + maxSegmentSize, + prometheus.DefaultRegisterer, + unloadDataStorageInterval, + ).Build(0, numberOfShards) + s.Require().NoError(err) + return h +} -// Close test implementation wal. -func (*testWal) Close() error { - return nil +func (s *QuerierSuite) appendTimeSeries(timeSeries []storagetest.TimeSeries) { + storagetest.MustAppendTimeSeries(&s.Suite, s.head, timeSeries) } -// Commit test implementation wal. -func (*testWal) Commit() error { - return nil +func (s *QuerierSuite) TestRangeQuery() { + // Arrange + timeSeries := []storagetest.TimeSeries{ + { + Labels: labels.FromStrings("__name__", "metric", "job", "test"), + Samples: []cppbridge.Sample{ + {Timestamp: 0, Value: 1}, + }, + }, + { + Labels: labels.FromStrings("__name__", "metric", "job", "test2"), + Samples: []cppbridge.Sample{ + {Timestamp: 0, Value: 10}, + }, + }, + } + s.appendTimeSeries(timeSeries) + + q := querier.NewQuerier[*task.Generic[*storage.PerGoroutineShard], + *shard.DataStorage, + *shard.LSS, + *storage.PerGoroutineShard, + *storage.HeadOnDisk, + ](s.head, querier.NewNoOpShardedDeduplicator, 0, 2, nil, nil) + defer func() { _ = q.Close() }() + matcher, _ := labels.NewMatcher(labels.MatchEqual, "__name__", "metric") + + // Act + seriesSet := q.Select(s.context, false, nil, matcher) + + // Assert + s.Equal(timeSeries, storagetest.TimeSeriesFromSeriesSet(seriesSet)) } -// CurrentSize test implementation wal. -func (*testWal) CurrentSize() int64 { - return 0 +func (s *QuerierSuite) TestRangeQueryWithoutMatching() { + // Arrange + timeSeries := []storagetest.TimeSeries{ + { + Labels: labels.FromStrings("__name__", "metric", "job", "test"), + Samples: []cppbridge.Sample{ + {Timestamp: 0, Value: 1}, + }, + }, + } + s.appendTimeSeries(timeSeries) + + q := querier.NewQuerier[*task.Generic[*storage.PerGoroutineShard], + *shard.DataStorage, + *shard.LSS, + *storage.PerGoroutineShard, + *storage.HeadOnDisk, + ](s.head, querier.NewNoOpShardedDeduplicator, 0, 2, nil, nil) + defer func() { _ = q.Close() }() + matcher, _ := labels.NewMatcher(labels.MatchEqual, "__name__", "unknown_metric") + + // Act + seriesSet := q.Select(s.context, false, nil, matcher) + + // Assert + s.Equal([]storagetest.TimeSeries(nil), storagetest.TimeSeriesFromSeriesSet(seriesSet)) } -// Flush test implementation wal. -func (*testWal) Flush() error { - return nil +func (s *QuerierSuite) TestRangeQueryWithDataStorageLoading() { + // Arrange + timeSeries := []storagetest.TimeSeries{ + { + Labels: labels.FromStrings("__name__", "metric", "job", "test"), + Samples: []cppbridge.Sample{ + {Timestamp: 0, Value: 0}, + {Timestamp: 1, Value: 1}, + {Timestamp: 2, Value: 2}, + }, + }, + { + Labels: labels.FromStrings("__name__", "metric", "job", "test2"), + Samples: []cppbridge.Sample{ + {Timestamp: 0, Value: 10}, + {Timestamp: 1, Value: 11}, + {Timestamp: 2, Value: 12}, + }, + }, + } + s.appendTimeSeries(timeSeries) + + timeSeriesAfterUnload := []storagetest.TimeSeries{ + { + Labels: timeSeries[0].Labels, + Samples: []cppbridge.Sample{ + {Timestamp: 3, Value: 3}, + }, + }, + { + Labels: timeSeries[1].Labels, + Samples: []cppbridge.Sample{ + {Timestamp: 3, Value: 13}, + }, + }, + } + + q := querier.NewQuerier[*task.Generic[*storage.PerGoroutineShard], + *shard.DataStorage, + *shard.LSS, + *storage.PerGoroutineShard, + *storage.HeadOnDisk, + ](s.head, querier.NewNoOpShardedDeduplicator, 0, 3, nil, nil) + defer func() { _ = q.Close() }() + matcher, _ := labels.NewMatcher(labels.MatchEqual, "__name__", "metric") + + // Act + s.Require().NoError(services.UnloadUnusedSeriesDataWithHead(s.head)) + s.appendTimeSeries(timeSeriesAfterUnload) + seriesSet := q.Select(s.context, false, nil, matcher) + + // Assert + timeSeries[0].AppendSamples(timeSeriesAfterUnload[0].Samples...) + timeSeries[1].AppendSamples(timeSeriesAfterUnload[1].Samples...) + s.Equal(timeSeries, storagetest.TimeSeriesFromSeriesSet(seriesSet)) } -// Sync test implementation wal. -func (*testWal) Sync() error { - return nil +func (s *QuerierSuite) TestInstantQuery() { + // Arrange + timeSeries := []storagetest.TimeSeries{ + { + Labels: labels.FromStrings("__name__", "metric", "job", "test"), + Samples: []cppbridge.Sample{ + {Timestamp: 0, Value: 1}, + }, + }, + { + Labels: labels.FromStrings("__name__", "metric", "job", "test2"), + Samples: []cppbridge.Sample{ + {Timestamp: 0, Value: 10}, + }, + }, + } + s.appendTimeSeries(timeSeries) + + q := querier.NewQuerier[*task.Generic[*storage.PerGoroutineShard], + *shard.DataStorage, + *shard.LSS, + *storage.PerGoroutineShard, + *storage.HeadOnDisk, + ](s.head, querier.NewNoOpShardedDeduplicator, 0, 0, nil, nil) + defer func() { _ = q.Close() }() + matcher, _ := labels.NewMatcher(labels.MatchEqual, "__name__", "metric") + + // Act + seriesSet := q.Select(s.context, false, nil, matcher) + + // Assert + s.Equal(timeSeries, storagetest.TimeSeriesFromSeriesSet(seriesSet)) } -// Write test implementation wal. -func (*testWal) Write(_ []*cppbridge.InnerSeries) (bool, error) { - return false, nil +func (s *QuerierSuite) TestInstantQueryWithDataStorageLoading() { + // Arrange + timeSeries := []storagetest.TimeSeries{ + { + Labels: labels.FromStrings("__name__", "metric", "job", "test"), + Samples: []cppbridge.Sample{ + {Timestamp: 0, Value: 0}, + {Timestamp: 1, Value: 1}, + {Timestamp: 2, Value: 2}, + }, + }, + { + Labels: labels.FromStrings("__name__", "metric", "job", "test2"), + Samples: []cppbridge.Sample{ + {Timestamp: 0, Value: 10}, + {Timestamp: 1, Value: 11}, + {Timestamp: 2, Value: 12}, + }, + }, + } + s.appendTimeSeries(timeSeries) + + timeSeriesAfterUnload := []storagetest.TimeSeries{ + { + Labels: timeSeries[0].Labels, + Samples: []cppbridge.Sample{ + {Timestamp: 3, Value: 3}, + }, + }, + { + Labels: timeSeries[1].Labels, + Samples: []cppbridge.Sample{ + {Timestamp: 3, Value: 13}, + }, + }, + } + + q := querier.NewQuerier[*task.Generic[*storage.PerGoroutineShard], + *shard.DataStorage, + *shard.LSS, + *storage.PerGoroutineShard, + *storage.HeadOnDisk, + ](s.head, querier.NewNoOpShardedDeduplicator, 0, 0, nil, nil) + defer func() { _ = q.Close() }() + matcher, _ := labels.NewMatcher(labels.MatchEqual, "__name__", "metric") + + // Act + s.Require().NoError(services.UnloadUnusedSeriesDataWithHead(s.head)) + s.appendTimeSeries(timeSeriesAfterUnload) + seriesSet := q.Select(s.context, false, nil, matcher) + + // Assert + s.Equal([]storagetest.TimeSeries{ + { + Labels: timeSeries[0].Labels, + Samples: []cppbridge.Sample{ + {Timestamp: 0, Value: 0}, + }, + }, + { + Labels: timeSeries[1].Labels, + Samples: []cppbridge.Sample{ + {Timestamp: 0, Value: 10}, + }, + }, + }, storagetest.TimeSeriesFromSeriesSet(seriesSet)) } diff --git a/pp/go/storage/storagetest/fixtures.go b/pp/go/storage/storagetest/fixtures.go new file mode 100644 index 0000000000..d56d20ead9 --- /dev/null +++ b/pp/go/storage/storagetest/fixtures.go @@ -0,0 +1,119 @@ +package storagetest + +import ( + "context" + + "github.com/prometheus/prometheus/model/labels" + "github.com/prometheus/prometheus/pp/go/cppbridge" + "github.com/prometheus/prometheus/pp/go/model" + "github.com/prometheus/prometheus/pp/go/storage" + "github.com/prometheus/prometheus/pp/go/storage/appender" + "github.com/prometheus/prometheus/pp/go/storage/head/services" + promstorage "github.com/prometheus/prometheus/storage" + "github.com/prometheus/prometheus/tsdb/chunkenc" + "github.com/stretchr/testify/suite" +) + +type TimeSeries struct { + Labels labels.Labels + Samples []cppbridge.Sample +} + +func (s *TimeSeries) AppendSamples(samples ...cppbridge.Sample) { + s.Samples = append(s.Samples, samples...) +} + +func (s *TimeSeries) toModelTimeSeries() []model.TimeSeries { + lsBuilder := model.NewLabelSetBuilder() + for i := range s.Labels { + lsBuilder.Add(s.Labels[i].Name, s.Labels[i].Value) + } + + ls := lsBuilder.Build() + + timeSeries := make([]model.TimeSeries, 0, len(s.Samples)) + for i := range s.Samples { + timeSeries = append(timeSeries, model.TimeSeries{ + LabelSet: ls, + Timestamp: uint64(s.Samples[i].Timestamp), + Value: s.Samples[i].Value, + }) + } + + return timeSeries +} + +type timeSeriesDataSlice struct { + timeSeries []model.TimeSeries +} + +func (tsd *timeSeriesDataSlice) TimeSeries() []model.TimeSeries { + return tsd.timeSeries +} + +func (tsd *timeSeriesDataSlice) Destroy() { + tsd.timeSeries = nil +} + +func MustAppendTimeSeries(s *suite.Suite, head *storage.HeadOnDisk, timeSeries []TimeSeries) { + headAppender := appender.New(head, services.CFViaRange) + + statelessRelabeler, err := cppbridge.NewStatelessRelabeler([]*cppbridge.RelabelConfig{}) + s.Require().NoError(err) + + state := cppbridge.NewStateV2WithoutLock() + state.SetStatelessRelabeler(statelessRelabeler) + + for i := range timeSeries { + tsd := timeSeriesDataSlice{timeSeries: timeSeries[i].toModelTimeSeries()} + hx, err := (cppbridge.HashdexFactory{}).GoModel(tsd.TimeSeries(), cppbridge.DefaultWALHashdexLimits()) + s.NoError(err) + + _, _, err = headAppender.Append( + context.Background(), + &appender.IncomingData{Hashdex: hx, Data: &tsd}, + state, + true) + s.NoError(err) + } +} + +type SamplesMap map[uint32][]cppbridge.Sample + +func GetSamplesFromSerializedChunks(chunks *cppbridge.HeadDataStorageSerializedChunks) SamplesMap { + result := make(SamplesMap) + + deserializer := cppbridge.NewHeadDataStorageDeserializer(chunks) + + n := chunks.NumberOfChunks() + for i := 0; i < n; i++ { + metadata := chunks.Metadata(i) + seriesId := metadata.SeriesID() + iterator := deserializer.CreateDecodeIterator(metadata) + for iterator.Next() { + ts, value := iterator.Sample() + result[seriesId] = append(result[seriesId], cppbridge.Sample{Timestamp: ts, Value: value}) + + } + } + + return result +} + +func TimeSeriesFromSeriesSet(seriesSet promstorage.SeriesSet) []TimeSeries { + var timeSeries []TimeSeries + for seriesSet.Next() { + series := seriesSet.At() + + timeSeries = append(timeSeries, TimeSeries{Labels: series.Labels()}) + currentSeries := &timeSeries[len(timeSeries)-1] + + chunkIterator := series.Iterator(nil) + for chunkIterator.Next() != chunkenc.ValNone { + ts, v := chunkIterator.At() + currentSeries.Samples = append(currentSeries.Samples, cppbridge.Sample{Timestamp: ts, Value: v}) + } + } + + return timeSeries +} diff --git a/pp/go/util/closer.go b/pp/go/util/closer.go index deedf3f222..15b48afbea 100644 --- a/pp/go/util/closer.go +++ b/pp/go/util/closer.go @@ -1,6 +1,10 @@ package util -import "sync" +import ( + "errors" + "io" + "sync" +) type Closer struct { close chan struct{} @@ -35,3 +39,11 @@ func (c *Closer) Close() error { <-c.closed return nil } + +func CloseAll(closers ...io.Closer) error { + var errs error + for _, closer := range closers { + errs = errors.Join(errs, closer.Close()) + } + return errs +} From fb9a2e9b09775a8d0dccf8b99b321add870e003b Mon Sep 17 00:00:00 2001 From: Alexandr Yudin Date: Wed, 24 Sep 2025 16:27:44 +0000 Subject: [PATCH 46/96] prototype, for save --- cmd/prometheus/main.go | 173 ++++++++++------- config/pp_remote_write_config.go | 5 + pp-pkg/configadapter/headkeeper.go | 18 -- pp-pkg/handler/interface.go | 10 +- pp-pkg/handler/processor/interface.go | 4 +- .../handler/processor/processor_moq_test.go | 17 +- pp-pkg/handler/processor/refill_processor.go | 2 +- .../processor/refill_processor_test.go | 4 +- pp-pkg/remote/config.go | 2 +- pp-pkg/storage/adapter.go | 31 ++- pp-pkg/tsdb/db.go | 9 +- pp/go/storage/head/keeper/keeper.go | 99 +++++++--- pp/go/storage/head/keeper/keeper_test.go | 30 +-- pp/go/storage/head/proxy/proxy.go | 65 ++++--- pp/go/storage/head/services/interface.go | 42 +++-- .../storage/head/services/metrics_updater.go | 9 +- pp/go/storage/head/services/persistener.go | 67 +++++-- .../storage/head/services/persistener_test.go | 6 +- pp/go/storage/head/services/rotator.go | 32 ++-- pp/go/storage/manager.go | 178 +++++++++++++----- pp/go/storage/querier/querier.go | 34 +++- pp/go/storage/remotewriter/remotewriter.go | 2 +- pp/go/storage/remotewriter/writeloop.go | 15 +- pp/go/storage/storagetest/fixtures.go | 6 +- web/api/v1/pp_api.go | 9 +- 25 files changed, 563 insertions(+), 306 deletions(-) delete mode 100644 pp-pkg/configadapter/headkeeper.go diff --git a/cmd/prometheus/main.go b/cmd/prometheus/main.go index fdec4f77a8..541f893f04 100644 --- a/cmd/prometheus/main.go +++ b/cmd/prometheus/main.go @@ -35,10 +35,6 @@ import ( "syscall" "time" - pphandler "github.com/prometheus/prometheus/pp-pkg/handler" - rwprocessor "github.com/prometheus/prometheus/pp-pkg/handler/processor" - pptsdb "github.com/prometheus/prometheus/pp-pkg/tsdb" - "github.com/KimMachineGun/automemlimit/memlimit" "github.com/alecthomas/kingpin/v2" "github.com/alecthomas/units" @@ -61,15 +57,20 @@ import ( "k8s.io/klog" klogv2 "k8s.io/klog/v2" + pp_pkg_handler "github.com/prometheus/prometheus/pp-pkg/handler" + rwprocessor "github.com/prometheus/prometheus/pp-pkg/handler/processor" "github.com/prometheus/prometheus/pp-pkg/receiver" // PP_CHANGES.md: rebuild on cpp "github.com/prometheus/prometheus/pp-pkg/remote" // PP_CHANGES.md: rebuild on cpp "github.com/prometheus/prometheus/pp-pkg/scrape" // PP_CHANGES.md: rebuild on cpp pp_pkg_storage "github.com/prometheus/prometheus/pp-pkg/storage" // PP_CHANGES.md: rebuild on cpp - "github.com/prometheus/prometheus/pp/go/relabeler/appender" // PP_CHANGES.md: rebuild on cpp - "github.com/prometheus/prometheus/pp/go/relabeler/head" // PP_CHANGES.md: rebuild on cpp - "github.com/prometheus/prometheus/pp/go/relabeler/head/catalog" // PP_CHANGES.md: rebuild on cpp - "github.com/prometheus/prometheus/pp/go/relabeler/head/ready" // PP_CHANGES.md: rebuild on cpp - "github.com/prometheus/prometheus/pp/go/relabeler/remotewriter" // PP_CHANGES.md: rebuild on cpp + pp_pkg_tsdb "github.com/prometheus/prometheus/pp-pkg/tsdb" + + pp_storage "github.com/prometheus/prometheus/pp/go/storage" // PP_CHANGES.md: rebuild on cpp + "github.com/prometheus/prometheus/pp/go/storage/catalog" // PP_CHANGES.md: rebuild on cpp + "github.com/prometheus/prometheus/pp/go/storage/head/head" // PP_CHANGES.md: rebuild on cpp + "github.com/prometheus/prometheus/pp/go/storage/querier" + "github.com/prometheus/prometheus/pp/go/storage/ready" // PP_CHANGES.md: rebuild on cpp + "github.com/prometheus/prometheus/pp/go/storage/remotewriter" // PP_CHANGES.md: rebuild on cpp "github.com/prometheus/prometheus/config" "github.com/prometheus/prometheus/discovery" @@ -720,9 +721,8 @@ func main() { os.Exit(1) } - reloadBlocksTriggerNotifier := receiver.NewReloadBlocksTriggerNotifier() + reloadBlocksTriggerNotifier := pp_storage.NewReloadBlocksTriggerNotifier() cfg.tsdb.ReloadBlocksExternalTrigger = reloadBlocksTriggerNotifier - ctxReceiver, cancelReceiver := context.WithCancel(context.Background()) dataDir, err := filepath.Abs(localStoragePath) if err != nil { @@ -742,44 +742,89 @@ func main() { } clock := clockwork.NewRealClock() - headCatalog, err := catalog.New(clock, fileLog, catalog.DefaultIDGenerator{}, int(catalogMaxLogFileSize), prometheus.DefaultRegisterer) + headCatalog, err := catalog.New( + clock, + fileLog, + catalog.DefaultIDGenerator{}, + int(catalogMaxLogFileSize), + prometheus.DefaultRegisterer, + ) if err != nil { level.Error(logger).Log("msg", "failed to create head catalog", "err", err) os.Exit(1) } - receiverReadyNotifier := ready.NewNotifiableNotifier() - // create receiver - receiver, err := receiver.NewReceiver( - ctxReceiver, - log.With(logger, "component", "receiver"), - prometheus.DefaultRegisterer, - receiverConfig, - localStoragePath, - cfgFile.RemoteWriteConfigs, - localStoragePath, - receiver.RotationInfo{ - BlockDuration: time.Duration(cfg.tsdb.MinBlockDuration), - Seed: cfgFile.GlobalConfig.ExternalLabels.Hash(), + pp_storage.InitLogHandler(log.With(logger, "component", "pp_storage")) + + hManagerReadyNotifier := ready.NewNotifiableNotifier() + hManager, err := pp_storage.NewManager( + &pp_storage.Options{ + Seed: cfgFile.GlobalConfig.ExternalLabels.Hash(), + BlockDuration: time.Duration(cfg.tsdb.MinBlockDuration), + CommitInterval: time.Duration(cfg.WalCommitInterval), + MaxRetentionPeriod: time.Duration(cfg.tsdb.RetentionDuration), + HeadRetentionPeriod: time.Duration(cfg.HeadRetentionTimeout), + QueueSize: 2, + DataDir: localStoragePath, + MaxSegmentSize: cfg.WalMaxSamplesPerSegment, + NumberOfShards: receiverConfig.NumberOfShards, }, + clock, headCatalog, reloadBlocksTriggerNotifier, - receiverReadyNotifier, - time.Duration(cfg.WalCommitInterval), - time.Duration(cfg.tsdb.RetentionDuration), - time.Duration(cfg.HeadRetentionTimeout), - // x3 ScrapeInterval timeout for write block - time.Duration(cfgFile.GlobalConfig.ScrapeInterval*3), - cfg.WalMaxSamplesPerSegment, - appender.UnloadDataStorage, + hManagerReadyNotifier, + prometheus.DefaultRegisterer, ) if err != nil { - level.Error(logger).Log("msg", "failed to create a receiver", "err", err) + level.Error(logger).Log("msg", "failed to create a head manager", "err", err) os.Exit(1) } + // receiverReadyNotifier := ready.NewNotifiableNotifier() + // // create receiver + // receiver, err := receiver.NewReceiver( + // ctxReceiver, + // log.With(logger, "component", "receiver"), + // prometheus.DefaultRegisterer, + // receiverConfig, + // localStoragePath, + // cfgFile.RemoteWriteConfigs, + // localStoragePath, + // receiver.RotationInfo{ + // BlockDuration: time.Duration(cfg.tsdb.MinBlockDuration), + // Seed: cfgFile.GlobalConfig.ExternalLabels.Hash(), + // }, + // headCatalog, + // reloadBlocksTriggerNotifier, + // receiverReadyNotifier, + // time.Duration(cfg.WalCommitInterval), + // time.Duration(cfg.tsdb.RetentionDuration), + // time.Duration(cfg.HeadRetentionTimeout), + // // x3 ScrapeInterval timeout for write block + // time.Duration(cfgFile.GlobalConfig.ScrapeInterval*3), + // cfg.WalMaxSamplesPerSegment, + // appender.UnloadDataStorage, + // ) + // if err != nil { + // level.Error(logger).Log("msg", "failed to create a receiver", "err", err) + // os.Exit(1) + // } + remoteWriterReadyNotifier := ready.NewNotifiableNotifier() - remoteWriter := remotewriter.New(dataDir, headCatalog, clock, remoteWriterReadyNotifier, prometheus.DefaultRegisterer) + remoteWriter := remotewriter.New( + dataDir, + headCatalog, + clock, + remoteWriterReadyNotifier, + prometheus.DefaultRegisterer, + ) + + adapter := pp_pkg_storage.NewAdapter( + clock, + hManager.Proxy(), + hManager.MergeOutOfOrderChunks, + prometheus.DefaultRegisterer, + ) // PP_CHANGES.md: rebuild on cpp end @@ -797,7 +842,7 @@ func main() { ) fanoutStorage = storage.NewFanout( logger, - pp_pkg_storage.NewQueryableStorage(receiver), + adapter, localStorage, remoteRead, ) @@ -875,7 +920,7 @@ func main() { scrapeManager, err := scrape.NewManager( &cfg.scrape, log.With(logger, "component", "scrape manager"), - nil, // TODO receiver adapter + adapter, prometheus.DefaultRegisterer, ) if err != nil { @@ -934,8 +979,8 @@ func main() { queryEngine = promql.NewEngine(opts) ruleManager = rules.NewManager(&rules.ManagerOptions{ - Appendable: receiver, // PP_CHANGES.md: rebuild on cpp - Queryable: receiver, // PP_CHANGES.md: rebuild on cpp + Appendable: adapter, // PP_CHANGES.md: rebuild on cpp + Queryable: adapter, // PP_CHANGES.md: rebuild on cpp QueryFunc: rules.EngineQueryFunc(queryEngine, fanoutStorage), NotifyFunc: rules.SendAlerts(notifierManager, cfg.web.ExternalURL.String()), Context: ctxRule, @@ -993,7 +1038,7 @@ func main() { // Depends on cfg.web.ScrapeManager so needs to be after cfg.web.ScrapeManager = scrapeManager. // TODO receiver adapter - webHandler := web.New(log.With(logger, "component", "web"), &cfg.web, nil) // PP_CHANGES.md: rebuild on cpp + webHandler := web.New(log.With(logger, "component", "web"), &cfg.web, adapter) // PP_CHANGES.md: rebuild on cpp // Monitor outgoing connections on default transport with conntrack. http.DefaultTransport.(*http.Transport).DialContext = conntrack.NewDialContextFunc( @@ -1005,8 +1050,8 @@ func main() { reloaders := []reloader{ { // PP_CHANGES.md: rebuild on cpp start - name: "receiver", - reloader: receiver.ApplyConfig, + name: "head_manager", + reloader: hManager.ApplyConfig, }, { // PP_CHANGES.md: rebuild on cpp end name: "db_storage", reloader: localStorage.ApplyConfig, @@ -1142,7 +1187,11 @@ func main() { os.Exit(1) } - multiNotifiable := ready.New().With(receiverReadyNotifier).With(remoteWriterReadyNotifier).Build() + multiNotifiable := ready.NewMultiNotifiableBuilder().Add( + hManagerReadyNotifier, + ).Add( + remoteWriterReadyNotifier, + ).Build() opGC := catalog.NewGC(dataDir, headCatalog, multiNotifiable) var g run.Group @@ -1343,7 +1392,7 @@ func main() { return fmt.Errorf("opening storage failed: %w", err) } - tsdb.DBSetBlocksToDelete(db, pptsdb.PPBlocksToDelete(db, dataDir, headCatalog)) + tsdb.DBSetBlocksToDelete(db, pp_pkg_tsdb.PPBlocksToDelete(db, dataDir, headCatalog)) switch fsType := prom_runtime.Statfs(localStoragePath); fsType { case "NFS_SUPER_MAGIC": level.Warn(logger).Log("fs_type", fsType, "msg", "This filesystem is not supported and may lead to data corruption and data loss. Please carefully read https://prometheus.io/docs/prometheus/latest/storage/ to learn more about supported filesystems.") @@ -1392,7 +1441,7 @@ func main() { db, err := agent.Open( logger, prometheus.DefaultRegisterer, - receiver, // PP_CHANGES.md: rebuild on cpp + adapter, // TODO RW // PP_CHANGES.md: rebuild on cpp localStoragePath, &opts, ) @@ -1451,31 +1500,27 @@ func main() { g.Add( func() error { <-dbOpen - return receiver.Run(ctxReceiver) + return hManager.Run() }, func(err error) { - receiverCancelCtx, receiverCancelCtxCancel := context.WithCancel(ctxReceiver) - defer receiverCancelCtxCancel() - - level.Info(logger).Log("msg", "Stopping Receiver...") - if err := receiver.Shutdown(receiverCancelCtx); err != nil { - level.Error(logger).Log("msg", "Receiver shutdown failed", "err", err) + level.Info(logger).Log("msg", "Stopping head manager...") + if err := hManager.Shutdown(context.Background()); err != nil { + level.Error(logger).Log("msg", "Head manager shutdown failed", "err", err) } - cancelReceiver() + level.Info(logger).Log("msg", "Head manager stopped.") }, ) } // PP_CHANGES.md: rebuild on cpp end { // PP_CHANGES.md: rebuild on cpp start g.Add( - func() error { return <-head.UnrecoverableErrorChan }, + func() error { + return <-querier.UnrecoverableErrorChan + }, func(err error) { - select { - case head.UnrecoverableErrorChan <- nil: - // stop execute func if need - default: - } + // stop execute func if need + querier.SendUnrecoverableError(nil) - if errors.Is(err, head.UnrecoverableError{}) { + if errors.Is(err, querier.UnrecoverableError{}) { level.Error(logger).Log("msg", "Received unrecoverable error", "err", err) } }, @@ -2052,7 +2097,7 @@ func readPromPPFeatures(logger log.Logger) { fname, fvalue, _ := strings.Cut(feature, "=") switch strings.TrimSpace(fname) { case "head_copy_series_on_rotate": - appender.CopySeriesOnRotate = true + pp_storage.CopySeriesOnRotate = true level.Info(logger).Log( "msg", "[FEATURE] Copying active series from current head to new head during rotation is enabled.", @@ -2072,7 +2117,7 @@ func readPromPPFeatures(logger log.Logger) { } } - head.ExtraReadConcurrency = v + head.ExtraWorkers = v level.Info(logger).Log( "msg", "[FEATURE] Concurrency reading is enabled.", @@ -2122,10 +2167,10 @@ func readPromPPFeatures(logger log.Logger) { case "disable_commits_on_remote_write": rwprocessor.AlwaysCommit = false - pphandler.OTLPAlwaysCommit = false + pp_pkg_handler.OTLPAlwaysCommit = false case "unload_data_storage": - appender.UnloadDataStorage = true + pp_storage.UnloadDataStorage = true _ = level.Info(logger).Log("msg", "[FEATURE] Data storage unloading is enabled.") } } diff --git a/config/pp_remote_write_config.go b/config/pp_remote_write_config.go index 247d71cd04..e37ba8b7b8 100644 --- a/config/pp_remote_write_config.go +++ b/config/pp_remote_write_config.go @@ -61,6 +61,11 @@ func (c *Config) GetReceiverConfig() (*pp_pkg_config.RemoteWriteReceiverConfig, return rcCfg, nil } +// PPNumberOfShards returns number of shards for manager. +func (c *Config) PPNumberOfShards() uint16 { + return c.ReceiverConfig.NumberOfShards +} + // RemoteWriteReceiverConfig returns configs for RemoteWriteReceiver. func (c *Config) RemoteWriteReceiverConfig() *pp_pkg_config.RemoteWriteReceiverConfig { return c.ReceiverConfig.Copy() diff --git a/pp-pkg/configadapter/headkeeper.go b/pp-pkg/configadapter/headkeeper.go deleted file mode 100644 index 465ee0b3bc..0000000000 --- a/pp-pkg/configadapter/headkeeper.go +++ /dev/null @@ -1,18 +0,0 @@ -package configadapter - -import ( - prom_config "github.com/prometheus/prometheus/config" - "github.com/prometheus/prometheus/pp/go/storage" -) - -// HeadKeeperApplyConfig returns func-adapter for apply config on [headkeeper.HeadKeeper]. -func HeadKeeperApplyConfig(m *storage.Manager) func(cfg *prom_config.Config) error { - return func(cfg *prom_config.Config) error { - rCfg, err := cfg.GetReceiverConfig() - if err != nil { - return err - } - - return m.ApplyConfig(rCfg.NumberOfShards) - } -} diff --git a/pp-pkg/handler/interface.go b/pp-pkg/handler/interface.go index e3a89b985e..98e806bd3c 100644 --- a/pp-pkg/handler/interface.go +++ b/pp-pkg/handler/interface.go @@ -6,7 +6,7 @@ import ( "github.com/prometheus/prometheus/pp-pkg/handler/processor" "github.com/prometheus/prometheus/pp-pkg/model" "github.com/prometheus/prometheus/pp/go/cppbridge" - "github.com/prometheus/prometheus/pp/go/relabeler" + "github.com/prometheus/prometheus/pp/go/storage/querier" "github.com/prometheus/prometheus/storage" ) @@ -45,13 +45,13 @@ type Adapter interface { ) error // HeadQuerier returns [storage.Querier] from active head. - HeadQuerier(ctx context.Context, mint, maxt int64) (storage.Querier, error) + HeadQuerier(mint, maxt int64) (storage.Querier, error) // HeadStatus returns stats of Head. - HeadStatus(ctx context.Context, limit int) relabeler.HeadStatus + HeadStatus(ctx context.Context, limit int) (*querier.HeadStatus, error) - // MergeOutOfOrderChunks merge chunks with out of order data chunks. - MergeOutOfOrderChunks(ctx context.Context) + // MergeOutOfOrderChunks send signal to merge chunks with out of order data chunks. + MergeOutOfOrderChunks() } // StreamProcessor interface. diff --git a/pp-pkg/handler/processor/interface.go b/pp-pkg/handler/processor/interface.go index 9ad00ff660..814bfb3f51 100644 --- a/pp-pkg/handler/processor/interface.go +++ b/pp-pkg/handler/processor/interface.go @@ -67,8 +67,8 @@ type Adapter interface { commitToWal bool, ) error - // MergeOutOfOrderChunks merge chunks with out of order data chunks. - MergeOutOfOrderChunks(ctx context.Context) + // MergeOutOfOrderChunks send signal to merge chunks with out of order data chunks. + MergeOutOfOrderChunks() } // StatesStorage stores the [cppbridge.State]'s. diff --git a/pp-pkg/handler/processor/processor_moq_test.go b/pp-pkg/handler/processor/processor_moq_test.go index d538fa9148..7a8974e9dc 100644 --- a/pp-pkg/handler/processor/processor_moq_test.go +++ b/pp-pkg/handler/processor/processor_moq_test.go @@ -34,7 +34,7 @@ var _ processor.Adapter = &AdapterMock{} // AppendTimeSeriesFunc: func(ctx context.Context, data pp_pkg_model.TimeSeriesBatch, state *cppbridge.StateV2, commitToWal bool) (cppbridge.RelabelerStats, error) { // panic("mock out the AppendTimeSeries method") // }, -// MergeOutOfOrderChunksFunc: func(ctx context.Context) { +// MergeOutOfOrderChunksFunc: func() { // panic("mock out the MergeOutOfOrderChunks method") // }, // } @@ -57,7 +57,7 @@ type AdapterMock struct { AppendTimeSeriesFunc func(ctx context.Context, data pp_pkg_model.TimeSeriesBatch, state *cppbridge.StateV2, commitToWal bool) (cppbridge.RelabelerStats, error) // MergeOutOfOrderChunksFunc mocks the MergeOutOfOrderChunks method. - MergeOutOfOrderChunksFunc func(ctx context.Context) + MergeOutOfOrderChunksFunc func() // calls tracks calls to the methods. calls struct { @@ -107,8 +107,6 @@ type AdapterMock struct { } // MergeOutOfOrderChunks holds details about calls to the MergeOutOfOrderChunks method. MergeOutOfOrderChunks []struct { - // Ctx is the ctx argument value. - Ctx context.Context } } lockAppendHashdex sync.RWMutex @@ -295,19 +293,16 @@ func (mock *AdapterMock) AppendTimeSeriesCalls() []struct { } // MergeOutOfOrderChunks calls MergeOutOfOrderChunksFunc. -func (mock *AdapterMock) MergeOutOfOrderChunks(ctx context.Context) { +func (mock *AdapterMock) MergeOutOfOrderChunks() { if mock.MergeOutOfOrderChunksFunc == nil { panic("AdapterMock.MergeOutOfOrderChunksFunc: method is nil but Adapter.MergeOutOfOrderChunks was just called") } callInfo := struct { - Ctx context.Context - }{ - Ctx: ctx, - } + }{} mock.lockMergeOutOfOrderChunks.Lock() mock.calls.MergeOutOfOrderChunks = append(mock.calls.MergeOutOfOrderChunks, callInfo) mock.lockMergeOutOfOrderChunks.Unlock() - mock.MergeOutOfOrderChunksFunc(ctx) + mock.MergeOutOfOrderChunksFunc() } // MergeOutOfOrderChunksCalls gets all the calls that were made to MergeOutOfOrderChunks. @@ -315,10 +310,8 @@ func (mock *AdapterMock) MergeOutOfOrderChunks(ctx context.Context) { // // len(mockedAdapter.MergeOutOfOrderChunksCalls()) func (mock *AdapterMock) MergeOutOfOrderChunksCalls() []struct { - Ctx context.Context } { var calls []struct { - Ctx context.Context } mock.lockMergeOutOfOrderChunks.RLock() calls = mock.calls.MergeOutOfOrderChunks diff --git a/pp-pkg/handler/processor/refill_processor.go b/pp-pkg/handler/processor/refill_processor.go index 556c6b4d1f..8232aac644 100644 --- a/pp-pkg/handler/processor/refill_processor.go +++ b/pp-pkg/handler/processor/refill_processor.go @@ -105,7 +105,7 @@ func (p *RefillProcessor) Process(ctx context.Context, refill Refill) error { prometheus.Labels{"processor_type": "refill", "status_code": "200"}, ).Inc() - p.adapter.MergeOutOfOrderChunks(ctx) + p.adapter.MergeOutOfOrderChunks() return refill.Write(ctx, model.RefillProcessingStatus{Code: http.StatusOK}) } diff --git a/pp-pkg/handler/processor/refill_processor_test.go b/pp-pkg/handler/processor/refill_processor_test.go index 2a9d39557f..922846c0e4 100644 --- a/pp-pkg/handler/processor/refill_processor_test.go +++ b/pp-pkg/handler/processor/refill_processor_test.go @@ -41,7 +41,7 @@ func (s *RefillProcessorSuite) TestProcess() { AppendHashdexFunc: func(context.Context, cppbridge.ShardedData, *cppbridge.StateV2, bool) error { return nil }, - MergeOutOfOrderChunksFunc: func(context.Context) {}, + MergeOutOfOrderChunksFunc: func() {}, } metadata := model.Metadata{ @@ -118,7 +118,7 @@ func (s *RefillProcessorSuite) TestProcessWithError() { AppendHashdexFunc: func(context.Context, cppbridge.ShardedData, *cppbridge.StateV2, bool) error { return nil }, - MergeOutOfOrderChunksFunc: func(context.Context) {}, + MergeOutOfOrderChunksFunc: func() {}, } metadata := model.Metadata{ diff --git a/pp-pkg/remote/config.go b/pp-pkg/remote/config.go index 764d74c2b8..2ee2223362 100644 --- a/pp-pkg/remote/config.go +++ b/pp-pkg/remote/config.go @@ -8,7 +8,7 @@ import ( "gopkg.in/yaml.v2" "github.com/prometheus/prometheus/config" - "github.com/prometheus/prometheus/pp/go/relabeler/remotewriter" + "github.com/prometheus/prometheus/pp/go/storage/remotewriter" ) func ApplyConfig(remoteWriter *remotewriter.RemoteWriter) func(promConfig *config.Config) error { diff --git a/pp-pkg/storage/adapter.go b/pp-pkg/storage/adapter.go index f6e7dc01a9..03bcc38334 100644 --- a/pp-pkg/storage/adapter.go +++ b/pp-pkg/storage/adapter.go @@ -25,11 +25,12 @@ var _ storage.Storage = (*Adapter)(nil) // Adapter for implementing the [Queryable] interface and append data. type Adapter struct { - proxy *pp_storage.ProxyHead - haTracker *hatracker.HighAvailabilityTracker - hashdexFactory cppbridge.HashdexFactory - hashdexLimits cppbridge.WALHashdexLimits - transparentState *cppbridge.StateV2 + proxy *pp_storage.ProxyHead + haTracker *hatracker.HighAvailabilityTracker + hashdexFactory cppbridge.HashdexFactory + hashdexLimits cppbridge.WALHashdexLimits + transparentState *cppbridge.StateV2 + mergeOutOfOrderChunks func() activeQuerierMetrics *querier.Metrics storageQuerierMetrics *querier.Metrics @@ -39,6 +40,7 @@ type Adapter struct { func NewAdapter( clock clockwork.Clock, proxy *pp_storage.ProxyHead, + mergeOutOfOrderChunks func(), registerer prometheus.Registerer, ) *Adapter { return &Adapter{ @@ -47,6 +49,7 @@ func NewAdapter( hashdexFactory: cppbridge.HashdexFactory{}, hashdexLimits: cppbridge.DefaultWALHashdexLimits(), transparentState: cppbridge.NewTransitionStateV2(), + mergeOutOfOrderChunks: mergeOutOfOrderChunks, activeQuerierMetrics: querier.NewMetrics(registerer, querier.QueryableAppenderSource), storageQuerierMetrics: querier.NewMetrics(registerer, querier.QueryableStorageSource), } @@ -165,14 +168,14 @@ func (ar *Adapter) Appender(ctx context.Context) storage.Appender { // ChunkQuerier provides querying access over time series data of a fixed time range. // Returns new Chunk Querier that merges results of given primary and secondary chunk queriers. func (ar *Adapter) ChunkQuerier(mint, maxt int64) (storage.ChunkQuerier, error) { - queriers := make([]storage.ChunkQuerier, 0, 2) + queriers := make([]storage.ChunkQuerier, 0, 1) //revive:disable-line:add-constant // the best way ahead := ar.proxy.Get() queriers = append( queriers, querier.NewChunkQuerier(ahead, querier.NewNoOpShardedDeduplicator, mint, maxt, nil), ) - for head := range ar.proxy.RangeQueriableHeads(mint, maxt) { + for _, head := range ar.proxy.Heads() { if ahead.ID() == head.ID() { continue } @@ -214,17 +217,27 @@ func (ar *Adapter) HeadStatus(ctx context.Context, limit int) (*querier.HeadStat return querier.QueryHeadStatus(ctx, ar.proxy.Get(), limit) } +// LowestSentTimestamp returns the lowest sent timestamp across all queues. +func (*Adapter) LowestSentTimestamp() int64 { + return 0 +} + +// MergeOutOfOrderChunks send signal to merge chunks with out of order data chunks. +func (ar *Adapter) MergeOutOfOrderChunks() { + ar.mergeOutOfOrderChunks() +} + // Querier calls f() with the given parameters. // Returns a [querier.MultiQuerier] combining of primary and secondary queriers. func (ar *Adapter) Querier(mint, maxt int64) (storage.Querier, error) { - queriers := make([]storage.Querier, 0, 2) + queriers := make([]storage.Querier, 0, 1) //revive:disable-line:add-constant // the best way ahead := ar.proxy.Get() queriers = append( queriers, querier.NewQuerier(ahead, querier.NewNoOpShardedDeduplicator, mint, maxt, nil, ar.activeQuerierMetrics), ) - for head := range ar.proxy.RangeQueriableHeads(mint, maxt) { + for _, head := range ar.proxy.Heads() { if ahead.ID() == head.ID() { continue } diff --git a/pp-pkg/tsdb/db.go b/pp-pkg/tsdb/db.go index f5ab64d89c..8d156c12ed 100644 --- a/pp-pkg/tsdb/db.go +++ b/pp-pkg/tsdb/db.go @@ -1,12 +1,13 @@ package tsdb import ( + "path/filepath" + "slices" + "github.com/oklog/ulid" - "github.com/prometheus/prometheus/pp/go/relabeler/head/catalog" + "github.com/prometheus/prometheus/pp/go/storage/catalog" "github.com/prometheus/prometheus/tsdb" "github.com/prometheus/prometheus/tsdb/fileutil" - "path/filepath" - "slices" ) // PPBlocksToDelete returns a filter which decides time based and size based @@ -104,7 +105,7 @@ func BeyondSizeRetention(db *tsdb.DB, dir string, catalog *catalog.Catalog, bloc func catalogHeadsSize(dir string, catalog *catalog.Catalog) (catalogSize int64) { catalogSize += catalog.OnDiskSize() - heads, _ := catalog.List(nil, nil) + heads := catalog.List(nil, nil) for _, h := range heads { catalogSize += headSize(filepath.Join(dir, h.Dir())) } diff --git a/pp/go/storage/head/keeper/keeper.go b/pp/go/storage/head/keeper/keeper.go index f649bb88c6..afbadb4e01 100644 --- a/pp/go/storage/head/keeper/keeper.go +++ b/pp/go/storage/head/keeper/keeper.go @@ -7,18 +7,18 @@ import ( "time" ) +type addPolicy = uint8 + const ( + // MinHeadConvertingQueueSize the minimum value of the [Keeper]'s queue. MinHeadConvertingQueueSize = 2 - Add = 0 - AddWithReplace = 1 -) - -var ( - ErrorNoSlots error = errors.New("keeper has no slots") + add addPolicy = 0 + addWithReplace addPolicy = 1 ) -type AddPolicy = uint8 +// ErrorNoSlots error when keeper has no slots. +var ErrorNoSlots = errors.New("keeper has no slots") type sortableHead[THead any] struct { head THead @@ -50,6 +50,7 @@ func (q *headSortedSlice[THead]) Pop() any { return item } +// Head the minimum required [Head] implementation for a [Keeper]. type Head interface { // ID returns id [Head]. ID() string @@ -58,56 +59,82 @@ type Head interface { Close() error } +// Keeper holds outdated heads until conversion. type Keeper[THead Head] struct { heads headSortedSlice[THead] - lock sync.Mutex + lock sync.RWMutex } +// NewKeeper init new [Keeper]. func NewKeeper[THead Head](queueSize int) *Keeper[THead] { return &Keeper[THead]{ heads: make(headSortedSlice[THead], 0, max(queueSize, MinHeadConvertingQueueSize)), } } -func (k *Keeper[THead]) Add(head THead, createdAt time.Duration, policy AddPolicy) error { +// Add the [Head] to the [Keeper] if there is a free slot. +func (k *Keeper[THead]) Add(head THead, createdAt time.Duration) error { k.lock.Lock() - result := k.addHead(head, createdAt, policy) + result := k.addHead(head, createdAt, add) k.lock.Unlock() + return result } -func (k *Keeper[THead]) addHead(head THead, createdAt time.Duration, policy AddPolicy) error { - if len(k.heads) < cap(k.heads) { - heap.Push(&k.heads, sortableHead[THead]{head: head, createdAt: createdAt}) +// AddWithReplace the [Head] to the [Keeper] with replace if the createdAt is earlier. +func (k *Keeper[THead]) AddWithReplace(head THead, createdAt time.Duration) error { + k.lock.Lock() + result := k.addHead(head, createdAt, addWithReplace) + k.lock.Unlock() + + return result +} + +// Close closes for the inability work with [Head]. +func (k *Keeper[THead]) Close() error { + k.lock.Lock() + if len(k.heads) == 0 { + k.lock.Unlock() return nil } - if policy == AddWithReplace && k.heads[0].createdAt < createdAt { - k.heads[0].head = head - k.heads[0].createdAt = createdAt - heap.Fix(&k.heads, 0) - return nil + errs := make([]error, 0, len(k.heads)) + for _, head := range k.heads { + errs = append(errs, head.head.Close()) } + k.lock.Unlock() - return ErrorNoSlots + return errors.Join(errs...) } -func (k *Keeper[THead]) setHeads(heads headSortedSlice[THead]) { - k.heads = heads - heap.Init(&k.heads) +// HasSlot returns the tru if there is a slot in the [Keeper]. +func (k *Keeper[THead]) HasSlot() bool { + k.lock.RLock() + result := cap(k.heads) > len(k.heads) + k.lock.RUnlock() + return result } +// Heads returns a slice of the [Head]s stored in the [Keeper]. func (k *Keeper[THead]) Heads() []THead { - k.lock.Lock() + k.lock.RLock() + + if len(k.heads) == 0 { + k.lock.RUnlock() + return nil + } + headsCopy := make([]THead, 0, len(k.heads)) for _, head := range k.heads { headsCopy = append(headsCopy, head.head) } - k.lock.Unlock() + + k.lock.RUnlock() return headsCopy } +// Remove removes [Head]s from the [Keeper]. func (k *Keeper[THead]) Remove(headsForRemove []THead) { if len(headsForRemove) == 0 { return @@ -137,9 +164,23 @@ func (k *Keeper[THead]) Remove(headsForRemove []THead) { } } -func (k *Keeper[THead]) HasSlot() bool { - k.lock.Lock() - result := cap(k.heads) > len(k.heads) - k.lock.Unlock() - return result +func (k *Keeper[THead]) addHead(head THead, createdAt time.Duration, policy addPolicy) error { + if len(k.heads) < cap(k.heads) { + heap.Push(&k.heads, sortableHead[THead]{head: head, createdAt: createdAt}) + return nil + } + + if policy == addWithReplace && k.heads[0].createdAt < createdAt { + k.heads[0].head = head + k.heads[0].createdAt = createdAt + heap.Fix(&k.heads, 0) + return nil + } + + return ErrorNoSlots +} + +func (k *Keeper[THead]) setHeads(heads headSortedSlice[THead]) { + k.heads = heads + heap.Init(&k.heads) } diff --git a/pp/go/storage/head/keeper/keeper_test.go b/pp/go/storage/head/keeper/keeper_test.go index 717f2a0c2d..f5ab01ab8a 100644 --- a/pp/go/storage/head/keeper/keeper_test.go +++ b/pp/go/storage/head/keeper/keeper_test.go @@ -18,7 +18,7 @@ func (h *headForTest) ID() string { return h.id } -func (h *headForTest) Close() error { +func (*headForTest) Close() error { return nil } @@ -46,9 +46,9 @@ func (s *KeeperSuite) TestAdd() { s.keeper = NewKeeper[*headForTest](2) // Act - _ = s.keeper.Add(newHeadForTest("d"), 4, Add) - _ = s.keeper.Add(newHeadForTest("c"), 3, Add) - err := s.keeper.Add(newHeadForTest("b"), 2, Add) + _ = s.keeper.Add(newHeadForTest("d"), 4) + _ = s.keeper.Add(newHeadForTest("c"), 3) + err := s.keeper.Add(newHeadForTest("b"), 2) // Assert s.Equal(sortedSlice{ @@ -63,9 +63,9 @@ func (s *KeeperSuite) TestAddWithReplaceNoReplace() { s.keeper = NewKeeper[*headForTest](2) // Act - _ = s.keeper.Add(newHeadForTest("d"), 4, Add) - _ = s.keeper.Add(newHeadForTest("c"), 3, Add) - err := s.keeper.Add(newHeadForTest("b"), 3, AddWithReplace) + _ = s.keeper.Add(newHeadForTest("d"), 4) + _ = s.keeper.Add(newHeadForTest("c"), 3) + err := s.keeper.AddWithReplace(newHeadForTest("b"), 3) // Assert s.Equal(sortedSlice{ @@ -80,9 +80,9 @@ func (s *KeeperSuite) TestAddWithReplace() { s.keeper = NewKeeper[*headForTest](2) // Act - _ = s.keeper.Add(newHeadForTest("d"), 4, Add) - _ = s.keeper.Add(newHeadForTest("c"), 3, Add) - err := s.keeper.Add(newHeadForTest("b"), 4, AddWithReplace) + _ = s.keeper.Add(newHeadForTest("d"), 4) + _ = s.keeper.Add(newHeadForTest("c"), 3) + err := s.keeper.AddWithReplace(newHeadForTest("b"), 4) // Assert s.Equal(sortedSlice{ @@ -97,11 +97,11 @@ func (s *KeeperSuite) TestRemove() { const Slots = 5 s.keeper = NewKeeper[*headForTest](Slots) - _ = s.keeper.Add(newHeadForTest("a"), 1, Add) - _ = s.keeper.Add(newHeadForTest("b"), 2, Add) - _ = s.keeper.Add(newHeadForTest("c"), 3, Add) - _ = s.keeper.Add(newHeadForTest("d"), 4, Add) - _ = s.keeper.Add(newHeadForTest("e"), 5, Add) + _ = s.keeper.Add(newHeadForTest("a"), 1) + _ = s.keeper.Add(newHeadForTest("b"), 2) + _ = s.keeper.Add(newHeadForTest("c"), 3) + _ = s.keeper.Add(newHeadForTest("d"), 4) + _ = s.keeper.Add(newHeadForTest("e"), 5) // Act s.keeper.Remove([]*headForTest{newHeadForTest("a"), newHeadForTest("c"), newHeadForTest("e")}) diff --git a/pp/go/storage/head/proxy/proxy.go b/pp/go/storage/head/proxy/proxy.go index a2ed229c70..6af058de69 100644 --- a/pp/go/storage/head/proxy/proxy.go +++ b/pp/go/storage/head/proxy/proxy.go @@ -3,6 +3,7 @@ package proxy import ( "context" "errors" + "time" ) // Head the minimum required [Head] implementation for a proxy. @@ -37,11 +38,25 @@ type ActiveHeadContainer[THead Head] interface { // Keeper // -// TODO Description +// Keeper holds outdated heads until conversion. type Keeper[THead Head] interface { - Add(head THead) + // Add the [Head] to the [Keeper] if there is a free slot. + Add(head THead, createdAt time.Duration) error + + // AddWithReplace the [Head] to the [Keeper] with replace if the createdAt is earlier. + AddWithReplace(head THead, createdAt time.Duration) error + + // Close closes for the inability work with [Head]. Close() error - RangeQueriableHeads(mint, maxt int64) func(func(THead) bool) + + // HasSlot returns the tru if there is a slot in the [Keeper]. + HasSlot() bool + + // Heads returns a slice of the [Head]s stored in the [Keeper]. + Heads() []THead + + // Remove removes [Head]s from the [Keeper]. + Remove(headsForRemove []THead) } // @@ -68,9 +83,14 @@ func NewProxy[THead Head]( } } -// Add the [Head] to the [Keeper]. -func (p *Proxy[THead]) Add(head THead) { - p.keeper.Add(head) +// Add the [Head] to the [Keeper] if there is a free slot. +func (p *Proxy[THead]) Add(head THead, createdAt time.Duration) error { + return p.keeper.Add(head, createdAt) +} + +// AddWithReplace the [Head] to the [Keeper] with replace if the createdAt is earlier. +func (p *Proxy[THead]) AddWithReplace(head THead, createdAt time.Duration) error { + return p.keeper.AddWithReplace(head, createdAt) } // Close closes [ActiveHeadContainer] and [Keeper] for the inability work with [Head]. @@ -91,30 +111,19 @@ func (p *Proxy[THead]) Get() THead { return p.activeHeadContainer.Get() } -// RangeQueriableHeadsWithActive returns the iterator to queriable [Head]s: -// the active [Head] and the [Head]s from the [Keeper]. -func (p *Proxy[THead]) RangeQueriableHeadsWithActive(mint, maxt int64) func(func(THead) bool) { - return func(yield func(h THead) bool) { - ahead := p.activeHeadContainer.Get() - if !yield(ahead) { - return - } - - for head := range p.keeper.RangeQueriableHeads(mint, maxt) { - if ahead.ID() == head.ID() { - continue - } - - if !yield(head) { - return - } - } - } +// HasSlot returns the tru if there is a slot in the [Keeper]. +func (p *Proxy[THead]) HasSlot() bool { + return p.keeper.HasSlot() +} + +// Heads returns a slice of the [Head]s stored in the [Keeper]. +func (p *Proxy[THead]) Heads() []THead { + return p.keeper.Heads() } -// RangeQueriableHeads returns the iterator to queriable [Head]s - the [Head]s only from the [Keeper]. -func (p *Proxy[THead]) RangeQueriableHeads(mint, maxt int64) func(func(THead) bool) { - return p.keeper.RangeQueriableHeads(mint, maxt) +// Remove removes [Head]s from the [Keeper]. +func (p *Proxy[THead]) Remove(headsForRemove []THead) { + p.keeper.Remove(headsForRemove) } // Replace the active [Head] with a new [Head]. diff --git a/pp/go/storage/head/services/interface.go b/pp/go/storage/head/services/interface.go index 1193293b4b..51fb8def21 100644 --- a/pp/go/storage/head/services/interface.go +++ b/pp/go/storage/head/services/interface.go @@ -5,7 +5,6 @@ import ( "time" "github.com/prometheus/prometheus/pp/go/cppbridge" - "github.com/prometheus/prometheus/pp/go/storage/head/keeper" ) // @@ -77,11 +76,14 @@ type HeadBuilder[ } // -// HeadStatusSetter +// HeadInformer // -// HeadStatusSetter sets status by headID in to catalog. -type HeadStatusSetter interface { +// HeadInformer sets status by headID in to catalog and get info. +type HeadInformer interface { + // CreatedAt returns the timestamp when the [Record]([Head]) was created. + CreatedAt(headID string) time.Duration + // SetActiveStatus sets the [catalog.StatusActive] status by headID. SetActiveStatus(headID string) error @@ -93,18 +95,26 @@ type HeadStatusSetter interface { // Keeper // +// Keeper holds outdated heads until conversion. type Keeper[ TTask Task, TShard, TGoShard Shard, THead Head[TTask, TShard, TGoShard], ] interface { - Add(head THead, createdAt time.Duration, policy keeper.AddPolicy) error + // Add the [Head] to the [Keeper] if there is a free slot. + Add(head THead, createdAt time.Duration) error + + // AddWithReplace the [Head] to the [Keeper] with replace if the createdAt is earlier. + AddWithReplace(head THead, createdAt time.Duration) error + + // HasSlot returns the tru if there is a slot in the [Keeper]. + HasSlot() bool + // Heads returns a slice of the [Head]s stored in the [Keeper]. Heads() []THead + // Remove removes [Head]s from the [Keeper]. Remove(headsForRemove []THead) - - HasSlot() bool } // @@ -127,17 +137,23 @@ type ProxyHead[ TShard, TGoShard Shard, THead Head[TTask, TShard, TGoShard], ] interface { - Add(head THead) + // Add the [Head] to the [Keeper] if there is a free slot. + Add(head THead, createdAt time.Duration) error + + // AddWithReplace the [Head] to the [Keeper] with replace if the createdAt is earlier. + AddWithReplace(head THead, createdAt time.Duration) error // Get the active [Head]. Get() THead - // RangeQueriableHeadsWithActive returns the iterator to queriable [Head]s: - // the active [Head] and the [Head]s from the [Keeper]. - RangeQueriableHeadsWithActive(mint int64, maxt int64) func(func(THead) bool) + // HasSlot returns the tru if there is a slot in the [Keeper]. + HasSlot() bool - // RangeQueriableHeads returns the iterator to queriable [Head]s - the [Head]s only from the [Keeper]. - RangeQueriableHeads(mint, maxt int64) func(func(THead) bool) + // Heads returns a slice of the [Head]s stored in the [Keeper]. + Heads() []THead + + // Remove removes [Head]s from the [Keeper]. + Remove(headsForRemove []THead) // Replace the active head [Head] with a new head. Replace(ctx context.Context, newHead THead) error diff --git a/pp/go/storage/head/services/metrics_updater.go b/pp/go/storage/head/services/metrics_updater.go index a8ae7a6d8c..92d6d13dac 100644 --- a/pp/go/storage/head/services/metrics_updater.go +++ b/pp/go/storage/head/services/metrics_updater.go @@ -2,7 +2,6 @@ package services import ( "context" - "math" "strconv" "github.com/prometheus/client_golang/prometheus" @@ -89,13 +88,13 @@ func NewMetricsUpdater[ // //revive:disable-next-line:confusing-naming // other type of Service. func (s *MetricsUpdater[TTask, TShard, TGoShard, THead, THeadStatus]) Execute(ctx context.Context) error { - logger.Infof("The Rotator is running.") + logger.Infof("The MetricsUpdater is running.") for range s.m.C() { s.collect(ctx) } - logger.Infof("The Rotator stopped.") + logger.Infof("The MetricsUpdater stopped.") return nil } @@ -105,7 +104,7 @@ func (s *MetricsUpdater[TTask, TShard, TGoShard, THead, THeadStatus]) collect(ct ahead := s.proxyHead.Get() status, err := s.queryHeadStatus(ctx, ahead, 0) - if err == nil { + if err != nil { // error may be only head is rotated, skip return } @@ -118,7 +117,7 @@ func (s *MetricsUpdater[TTask, TShard, TGoShard, THead, THeadStatus]) collect(ct s.collectFromShards(ahead, true) - for head := range s.proxyHead.RangeQueriableHeads(0, math.MaxInt64) { + for _, head := range s.proxyHead.Heads() { if head.ID() == ahead.ID() { continue } diff --git a/pp/go/storage/head/services/persistener.go b/pp/go/storage/head/services/persistener.go index 187b05db1e..84ad38dca4 100644 --- a/pp/go/storage/head/services/persistener.go +++ b/pp/go/storage/head/services/persistener.go @@ -8,7 +8,6 @@ import ( "github.com/prometheus/prometheus/pp/go/cppbridge" "github.com/prometheus/prometheus/pp/go/storage/block" "github.com/prometheus/prometheus/pp/go/storage/catalog" - "github.com/prometheus/prometheus/pp/go/storage/head/keeper" "github.com/prometheus/prometheus/pp/go/storage/logger" ) @@ -93,7 +92,9 @@ func (p *Persistener[TTask, TShard, TGoShard, THeadBlockWriter, THead]) Persist( return } -func (p *Persistener[TTask, TShard, TGoShard, THeadBlockWriter, THead]) headTimeInterval(head THead) cppbridge.TimeInterval { +func (p *Persistener[TTask, TShard, TGoShard, THeadBlockWriter, THead]) headTimeInterval( + head THead, +) cppbridge.TimeInterval { timeInterval := cppbridge.NewInvalidTimeInterval() for shard := range head.RangeShards() { interval := shard.TimeInterval(false) @@ -107,7 +108,9 @@ func (p *Persistener[TTask, TShard, TGoShard, THeadBlockWriter, THead]) HeadIsOu return p.clock.Since(time.UnixMilli(p.headTimeInterval(head).MaxT)) >= p.tsdbRetentionPeriod } -func (p *Persistener[TTask, TShard, TGoShard, THeadBlockWriter, THead]) persistedHeadIsOutdated(persistTimeMs int64) bool { +func (p *Persistener[TTask, TShard, TGoShard, THeadBlockWriter, THead]) persistedHeadIsOutdated( + persistTimeMs int64, +) bool { return p.clock.Since(time.UnixMilli(persistTimeMs)) >= p.retentionPeriod } @@ -162,36 +165,38 @@ func NewPersistenerService[ TKeeper Keeper[TTask, TShard, TGoShard, THead], TLoader Loader[TTask, TShard, TGoShard, THead], ]( - keeper TKeeper, + hkeeper TKeeper, loader TLoader, - catalog *catalog.Catalog, + hcatalog *catalog.Catalog, blockWriter THeadBlockWriter, clock clockwork.Clock, mediator Mediator, tsdbRetentionPeriod time.Duration, retentionPeriod time.Duration, -) PersistenerService[TTask, TShard, TGoShard, THeadBlockWriter, THead, TKeeper, TLoader] { - return PersistenerService[TTask, TShard, TGoShard, THeadBlockWriter, THead, TKeeper, TLoader]{ +) *PersistenerService[TTask, TShard, TGoShard, THeadBlockWriter, THead, TKeeper, TLoader] { + return &PersistenerService[TTask, TShard, TGoShard, THeadBlockWriter, THead, TKeeper, TLoader]{ persistener: NewPersistener[TTask, TShard, TGoShard, THeadBlockWriter, THead]( - catalog, + hcatalog, blockWriter, clock, tsdbRetentionPeriod, retentionPeriod, ), - keeper: keeper, + keeper: hkeeper, loader: loader, - catalog: catalog, + catalog: hcatalog, mediator: mediator, } } func (pg *PersistenerService[TTask, TShard, TGoShard, THeadBlockWriter, THead, TKeeper, TLoader]) Run() { - go func() { - for range pg.mediator.C() { - pg.ProcessHeads() - } - }() + logger.Infof("The PersistenerService is running.") + + for range pg.mediator.C() { + pg.ProcessHeads() + } + + logger.Infof("The PersistenerService stopped.") } func (pg *PersistenerService[TTask, TShard, TGoShard, THeadBlockWriter, THead, TKeeper, TLoader]) ProcessHeads() { @@ -200,11 +205,27 @@ func (pg *PersistenerService[TTask, TShard, TGoShard, THeadBlockWriter, THead, T pg.loadRotatedHeadsInKeeper(heads) } -func (pg *PersistenerService[TTask, TShard, TGoShard, THeadBlockWriter, THead, TKeeper, TLoader]) persistHeads(heads []THead) { +func (pg *PersistenerService[ + TTask, + TShard, + TGoShard, + THeadBlockWriter, + THead, + TKeeper, + TLoader, +]) persistHeads(heads []THead) { pg.keeper.Remove(pg.persistener.Persist(heads)) } -func (pg *PersistenerService[TTask, TShard, TGoShard, THeadBlockWriter, THead, TKeeper, TLoader]) loadRotatedHeadsInKeeper(keeperHeads []THead) { +func (pg *PersistenerService[ + TTask, + TShard, + TGoShard, + THeadBlockWriter, + THead, + TKeeper, + TLoader, +]) loadRotatedHeadsInKeeper(keeperHeads []THead) { if !pg.keeper.HasSlot() { return } @@ -226,9 +247,17 @@ func (pg *PersistenerService[TTask, TShard, TGoShard, THeadBlockWriter, THead, T } } -func (pg *PersistenerService[TTask, TShard, TGoShard, THeadBlockWriter, THead, TKeeper, TLoader]) loadAndAddHeadToKeeper(record *catalog.Record) bool { +func (pg *PersistenerService[ + TTask, + TShard, + TGoShard, + THeadBlockWriter, + THead, + TKeeper, + TLoader, +]) loadAndAddHeadToKeeper(record *catalog.Record) bool { head, _ := pg.loader.Load(record, 0) - if err := pg.keeper.Add(head, time.Duration(record.CreatedAt())*time.Millisecond, keeper.Add); err != nil { + if err := pg.keeper.Add(head, time.Duration(record.CreatedAt())*time.Millisecond); err != nil { _ = head.Close() return false } diff --git a/pp/go/storage/head/services/persistener_test.go b/pp/go/storage/head/services/persistener_test.go index f29271f43e..21c10d5df7 100644 --- a/pp/go/storage/head/services/persistener_test.go +++ b/pp/go/storage/head/services/persistener_test.go @@ -287,7 +287,7 @@ func (s *PersistenerSuite) TestPersistHeadErrorOnBlockWriterForSecondShard() { type PersistenerServiceSuite struct { GenericPersistenceSuite loader *storage.Loader - service services.PersistenerService[ + service *services.PersistenerService[ *task.Generic[*storage.PerGoroutineShard], *storage.ShardOnDisk, *storage.PerGoroutineShard, @@ -331,7 +331,7 @@ func (s *PersistenerServiceSuite) TestRemoveOutdatedHeadFromKeeper() { }) head.SetReadOnly() record, _ := s.catalog.SetStatus(head.ID(), catalog.StatusRotated) - _ = s.keeper.Add(head, time.Duration(s.clock.Now().Nanosecond()), keeper.Add) + _ = s.keeper.Add(head, time.Duration(s.clock.Now().Nanosecond())) // Act s.service.ProcessHeads() @@ -376,7 +376,7 @@ func (s *PersistenerServiceSuite) TestHeadAlreadyExistsInKeeper() { }, }) _, _ = s.catalog.SetStatus(head.ID(), catalog.StatusRotated) - _ = s.keeper.Add(head, 0, keeper.Add) + _ = s.keeper.Add(head, 0) // Act s.service.ProcessHeads() diff --git a/pp/go/storage/head/services/rotator.go b/pp/go/storage/head/services/rotator.go index 6727c69778..60d73e2ded 100644 --- a/pp/go/storage/head/services/rotator.go +++ b/pp/go/storage/head/services/rotator.go @@ -30,12 +30,12 @@ type Rotator[ TShard, TGoShard Shard, THead Head[TTask, TShard, TGoShard], ] struct { - proxyHead ProxyHead[TTask, TShard, TGoShard, THead] - headBuilder HeadBuilder[TTask, TShard, TGoShard, THead] - m Mediator - cfg RotatorConfig - headStatusSetter HeadStatusSetter - rotateCounter prometheus.Counter + proxyHead ProxyHead[TTask, TShard, TGoShard, THead] + headBuilder HeadBuilder[TTask, TShard, TGoShard, THead] + m Mediator + cfg RotatorConfig + headInformer HeadInformer + rotateCounter prometheus.Counter } // NewRotator init new [Rotator]. @@ -48,16 +48,16 @@ func NewRotator[ headBuilder HeadBuilder[TTask, TShard, TGoShard, THead], m Mediator, cfg RotatorConfig, - headStatusSetter HeadStatusSetter, + headInformer HeadInformer, r prometheus.Registerer, ) *Rotator[TTask, TShard, TGoShard, THead] { factory := util.NewUnconflictRegisterer(r) return &Rotator[TTask, TShard, TGoShard, THead]{ - proxyHead: proxyHead, - headBuilder: headBuilder, - m: m, - cfg: cfg, - headStatusSetter: headStatusSetter, + proxyHead: proxyHead, + headBuilder: headBuilder, + m: m, + cfg: cfg, + headInformer: headInformer, rotateCounter: factory.NewCounter( prometheus.CounterOpts{ Name: "prompp_rotator_rotate_count", @@ -101,14 +101,16 @@ func (s *Rotator[TTask, TShard, TGoShard, THead]) rotate( // TODO CopySeriesFrom only old nunber of shards == new // newHead.CopySeriesFrom(oldHead) - s.proxyHead.Add(oldHead) + if err = s.proxyHead.AddWithReplace(oldHead, s.headInformer.CreatedAt(oldHead.ID())); err != nil { + return fmt.Errorf("failed add to keeper old head: %w", err) + } // TODO if replace error? if err = s.proxyHead.Replace(ctx, newHead); err != nil { return fmt.Errorf("failed to replace old to new head: %w", err) } - if err = s.headStatusSetter.SetActiveStatus(newHead.ID()); err != nil { + if err = s.headInformer.SetActiveStatus(newHead.ID()); err != nil { logger.Warnf("failed set status active for head{%s}: %s", newHead.ID(), err) } @@ -120,7 +122,7 @@ func (s *Rotator[TTask, TShard, TGoShard, THead]) rotate( logger.Warnf("failed commit and flush to wal: %s", err) } - if err = s.headStatusSetter.SetRotatedStatus(oldHead.ID()); err != nil { + if err = s.headInformer.SetRotatedStatus(oldHead.ID()); err != nil { logger.Warnf("failed set status rotated for head{%s}: %s", oldHead.ID(), err) } oldHead.SetReadOnly() diff --git a/pp/go/storage/manager.go b/pp/go/storage/manager.go index ee93604a2b..32e5bd1eb4 100644 --- a/pp/go/storage/manager.go +++ b/pp/go/storage/manager.go @@ -4,7 +4,9 @@ import ( "context" "errors" "fmt" + "math" "os" + "path/filepath" "sync/atomic" "time" @@ -14,9 +16,12 @@ import ( "github.com/oklog/run" "github.com/prometheus/client_golang/prometheus" + "github.com/prometheus/prometheus/config" "github.com/prometheus/prometheus/pp/go/cppbridge" + "github.com/prometheus/prometheus/pp/go/storage/block" "github.com/prometheus/prometheus/pp/go/storage/catalog" "github.com/prometheus/prometheus/pp/go/storage/head/container" + "github.com/prometheus/prometheus/pp/go/storage/head/keeper" "github.com/prometheus/prometheus/pp/go/storage/head/proxy" "github.com/prometheus/prometheus/pp/go/storage/head/services" "github.com/prometheus/prometheus/pp/go/storage/logger" @@ -35,21 +40,40 @@ const ( // DefaultMetricWriteInterval default metric scrape interval. DefaultMetricWriteInterval = 15 * time.Second + + // DefaultPersistDuration the default interval for persisting [Head]. + DefaultPersistDuration = 5 * time.Minute + + // DefaultUnloadDataStorageInterval the default interval for unloading [DataStorage]. + DefaultUnloadDataStorageInterval = 5 * time.Minute ) -// DefaultNumberOfShards default number of shards. -var DefaultNumberOfShards uint16 = 2 +var ( + // CopySeriesOnRotate copy active series from the current head to the new head during rotation. + CopySeriesOnRotate = false + + // UnloadDataStorage flags for unloading [DataStorage]. + UnloadDataStorage = false + + // DefaultNumberOfShards default number of shards. + DefaultNumberOfShards uint16 = 2 +) // // Options // +// Options manager launch options. type Options struct { - Seed uint64 - BlockDuration time.Duration - CommitInterval time.Duration - MaxSegmentSize uint32 - NumberOfShards uint16 + Seed uint64 + BlockDuration time.Duration + CommitInterval time.Duration + MaxRetentionPeriod time.Duration + HeadRetentionPeriod time.Duration + QueueSize int + DataDir string + MaxSegmentSize uint32 + NumberOfShards uint16 } // @@ -103,35 +127,49 @@ type Manager struct { cgogc *cppbridge.CGOGC cfg *Config rotatorMediator *mediator.Mediator + mergerMediator *mediator.Mediator } // NewManager init new [Manager]. func NewManager( + o *Options, clock clockwork.Clock, - dataDir string, hcatalog *catalog.Catalog, - options Options, triggerNotifier *ReloadBlocksTriggerNotifier, readyNotifier ready.Notifier, r prometheus.Registerer, - unloadDataStorageInterval time.Duration, ) (*Manager, error) { - dirStat, err := os.Stat(dataDir) + if o == nil { + return nil, errors.New("manager options is nil") + } + + dataDir, err := filepath.Abs(o.DataDir) + if err != nil { + return nil, err + } + o.DataDir = dataDir + + dirStat, err := os.Stat(o.DataDir) if err != nil { return nil, fmt.Errorf("failed to stat dir: %w", err) } if !dirStat.IsDir() { - return nil, fmt.Errorf("%s is not directory", dataDir) + return nil, fmt.Errorf("%s is not directory", o.DataDir) } - builder := NewBuilder(hcatalog, dataDir, options.MaxSegmentSize, r, unloadDataStorageInterval) + var unloadDataStorageInterval time.Duration + if UnloadDataStorage { + unloadDataStorageInterval = DefaultUnloadDataStorageInterval + } + + builder := NewBuilder(hcatalog, o.DataDir, o.MaxSegmentSize, r, unloadDataStorageInterval) - loader := NewLoader(dataDir, options.MaxSegmentSize, r, unloadDataStorageInterval) + loader := NewLoader(o.DataDir, o.MaxSegmentSize, r, unloadDataStorageInterval) - cfg := NewConfig(options.NumberOfShards) + cfg := NewConfig(o.NumberOfShards) - h, err := uploadOrBuildHead(clock, hcatalog, builder, loader, options.BlockDuration, cfg.NumberOfShards()) + h, err := uploadOrBuildHead(clock, hcatalog, builder, loader, o.BlockDuration, cfg.NumberOfShards()) if err != nil { return nil, err } @@ -140,45 +178,55 @@ func NewManager( return nil, errors.Join(fmt.Errorf("failed to set active status: %w", err), h.Close()) } - readyNotifier.NotifyReady() - - // TODO implements - headKeeper := &NoopKeeper{} + hKeeper := keeper.NewKeeper[*HeadOnDisk](o.QueueSize) m := &Manager{ g: run.Group{}, closer: util.NewCloser(), - proxy: proxy.NewProxy(container.NewWeighted(h), headKeeper, services.CFSViaRange), + proxy: proxy.NewProxy(container.NewWeighted(h), hKeeper, services.CFSViaRange), cgogc: cppbridge.NewCGOGC(r), cfg: cfg, rotatorMediator: mediator.NewMediator( - mediator.NewRotateTimerWithSeed(clock, options.BlockDuration, options.Seed), + mediator.NewRotateTimerWithSeed(clock, o.BlockDuration, o.Seed), ), + mergerMediator: mediator.NewMediator(mediator.NewConstantIntervalTimer(clock, DefaultMergeDuration)), } - m.initServices(hcatalog, builder, clock, options.CommitInterval, r) + readyNotifier.NotifyReady() + + m.initServices(o, hcatalog, builder, loader, clock, r) - logger.Infof("[Manager] created") + logger.Infof("Head Manager created") return m, nil } // ApplyConfig update config. -func (m *Manager) ApplyConfig(numberOfShards uint16) error { +func (m *Manager) ApplyConfig(cfg *config.Config) error { logger.Infof("reconfiguration start") defer logger.Infof("reconfiguration completed") - if m.proxy.Get().NumberOfShards() == numberOfShards { + if m.proxy.Get().NumberOfShards() == cfg.PPNumberOfShards() { return nil } - if m.cfg.SetNumberOfShards(numberOfShards) { + if m.cfg.SetNumberOfShards(cfg.PPNumberOfShards()) { m.rotatorMediator.Trigger() } return nil } +// MergeOutOfOrderChunks send signal to merge chunks with out of order data chunks. +func (m *Manager) MergeOutOfOrderChunks() { + m.mergerMediator.Trigger() +} + +// Proxy returns proxy to the active [Head] and the keeper of old [Head]s. +func (m *Manager) Proxy() *proxy.Proxy[*HeadOnDisk] { + return m.proxy +} + // Run launches the [Manager]'s services. func (m *Manager) Run() error { defer m.closer.Done() @@ -188,17 +236,18 @@ func (m *Manager) Run() error { // Shutdown safe shutdown [Manager]: stop services and close [Head]'s. func (m *Manager) Shutdown(ctx context.Context) error { - _ = m.closer.Close() + m.close() return errors.Join(m.proxy.Close(), m.cgogc.Shutdown(ctx)) } // initServices initializes services for startup. func (m *Manager) initServices( + o *Options, hcatalog *catalog.Catalog, builder *Builder, + loader *Loader, clock clockwork.Clock, - commitInterval time.Duration, r prometheus.Registerer, ) { baseCtx := context.Background() @@ -211,7 +260,34 @@ func (m *Manager) initServices( return nil }, func(error) { - _ = m.closer.Close() + m.close() + }, + ) + + // Persistener + persistenerMediator := mediator.NewMediator(mediator.NewConstantIntervalTimer(clock, DefaultPersistDuration)) + m.g.Add( + func() error { + services.NewPersistenerService( + m.proxy, + loader, + hcatalog, + block.NewWriter[*ShardOnDisk]( + o.DataDir, + block.DefaultChunkSegmentSize, + o.BlockDuration, + r, + ), + clock, + persistenerMediator, + o.MaxRetentionPeriod, + o.HeadRetentionPeriod, + ).Run() + + return nil + }, + func(error) { + persistenerMediator.Close() }, ) @@ -224,7 +300,7 @@ func (m *Manager) initServices( builder, m.rotatorMediator, m.cfg, - &statusSetter{catalog: hcatalog}, + &headInformer{catalog: hcatalog}, r, ).Execute(rotatorCtx) }, @@ -234,6 +310,7 @@ func (m *Manager) initServices( }, ) + // checks if the head is new isNewHead := func(headID string) bool { rec, err := hcatalog.Get(headID) if err != nil { @@ -244,7 +321,7 @@ func (m *Manager) initServices( } // Committer - committerMediator := mediator.NewMediator(mediator.NewConstantIntervalTimer(clock, commitInterval)) + committerMediator := mediator.NewMediator(mediator.NewConstantIntervalTimer(clock, o.CommitInterval)) committerCtx, committerCancel := context.WithCancel(baseCtx) m.g.Add( func() error { @@ -257,14 +334,13 @@ func (m *Manager) initServices( ) // Merger - mergerMediator := mediator.NewMediator(mediator.NewConstantIntervalTimer(clock, DefaultMergeDuration)) mergerCtx, mergerCancel := context.WithCancel(baseCtx) m.g.Add( func() error { - return services.NewMerger(m.proxy, mergerMediator, isNewHead).Execute(mergerCtx) + return services.NewMerger(m.proxy, m.mergerMediator, isNewHead).Execute(mergerCtx) }, func(error) { - mergerMediator.Close() + m.mergerMediator.Close() mergerCancel() }, ) @@ -288,6 +364,14 @@ func (m *Manager) initServices( ) } +func (m *Manager) close() { + select { + case <-m.closer.Signal(): + default: + _ = m.closer.Close() + } +} + // InitLogHandler init log handler for pp. func InitLogHandler(l log.Logger) { l = log.With(l, "pp_caller", log.Caller(4)) @@ -310,23 +394,33 @@ func InitLogHandler(l log.Logger) { } // -// headStatusSetter +// headInformer // -// statusSetter wrapper over [catalog.Catalog] for set statuses. -type statusSetter struct { +// headInformer wrapper over [catalog.Catalog] for set statuses and get info. +type headInformer struct { catalog *catalog.Catalog } +// CreatedAt returns the timestamp when the [Record]([Head]) was created. +func (hi *headInformer) CreatedAt(headID string) time.Duration { + record, err := hi.catalog.Get(headID) + if err != nil { + return time.Duration(math.MaxInt64) + } + + return time.Duration(record.CreatedAt()) * time.Millisecond +} + // SetActiveStatus sets the [catalog.StatusActive] status by headID. -func (ha *statusSetter) SetActiveStatus(headID string) error { - _, err := ha.catalog.SetStatus(headID, catalog.StatusActive) +func (hi *headInformer) SetActiveStatus(headID string) error { + _, err := hi.catalog.SetStatus(headID, catalog.StatusActive) return err } // SetRotatedStatus sets the [catalog.StatusRotated] status by headID. -func (ha *statusSetter) SetRotatedStatus(headID string) error { - _, err := ha.catalog.SetStatus(headID, catalog.StatusRotated) +func (hi *headInformer) SetRotatedStatus(headID string) error { + _, err := hi.catalog.SetStatus(headID, catalog.StatusRotated) return err } diff --git a/pp/go/storage/querier/querier.go b/pp/go/storage/querier/querier.go index b456fec7f0..e8c7a9a351 100644 --- a/pp/go/storage/querier/querier.go +++ b/pp/go/storage/querier/querier.go @@ -340,8 +340,7 @@ func queryDataStorage[ _ = tDataStorageQuery.Wait() if err := loadAndQueryWaiter.Wait(); err != nil { - // TODO: Unrecoverable error - // q.head.UnrecoverableError(err) + SendUnrecoverableError(err) return nil } @@ -540,3 +539,34 @@ func queryLss[ return lssQueryResults, snapshots, nil } + +// UnrecoverableErrorChan channel singal for [UnrecoverableError]. +var UnrecoverableErrorChan = make(chan error) + +// SendUnrecoverableError send to terminate on [UnrecoverableError]. +func SendUnrecoverableError(err error) { + if err != nil { + logger.Warnf("Unrecoverable error: %v", err) + } + + select { + case UnrecoverableErrorChan <- UnrecoverableError{err}: + default: + } +} + +// UnrecoverableError error if Head get unrecoverable error. +type UnrecoverableError struct { + err error +} + +// Error implements error. +func (err UnrecoverableError) Error() string { + return fmt.Sprintf("Unrecoverable error: %v", err.err) +} + +// Is implements errors.Is interface. +func (UnrecoverableError) Is(target error) bool { + _, ok := target.(UnrecoverableError) + return ok +} diff --git a/pp/go/storage/remotewriter/remotewriter.go b/pp/go/storage/remotewriter/remotewriter.go index e99489b4e4..a05805fab7 100644 --- a/pp/go/storage/remotewriter/remotewriter.go +++ b/pp/go/storage/remotewriter/remotewriter.go @@ -22,7 +22,7 @@ type Catalog interface { List( filterFn func(record *catalog.Record) bool, sortLess func(lhs, rhs *catalog.Record) bool, - ) ([]*catalog.Record, error) + ) []*catalog.Record // SetCorrupted set corrupted flag for ID and returns [catalog.Record] if exist. SetCorrupted(id string) (*catalog.Record, error) diff --git a/pp/go/storage/remotewriter/writeloop.go b/pp/go/storage/remotewriter/writeloop.go index a97fabfe04..eb88ae9bf5 100644 --- a/pp/go/storage/remotewriter/writeloop.go +++ b/pp/go/storage/remotewriter/writeloop.go @@ -252,15 +252,12 @@ func nextHead(ctx context.Context, dataDir string, headCatalog Catalog, headID s return nil, err } - headRecords, err := headCatalog.List( + headRecords := headCatalog.List( nil, func(lhs, rhs *catalog.Record) bool { return lhs.CreatedAt() < rhs.CreatedAt() }, ) - if err != nil { - return nil, fmt.Errorf("list head records: %w", err) - } if len(headRecords) == 0 { return nil, fmt.Errorf("nextHead: no new heads: empty head records") @@ -277,7 +274,7 @@ func nextHead(ctx context.Context, dataDir string, headCatalog Catalog, headID s continue } - if err = validateHead(ctx, filepath.Join(dataDir, headRecord.Dir())); err != nil { + if err := validateHead(ctx, filepath.Join(dataDir, headRecord.Dir())); err != nil { if !errors.Is(err, os.ErrNotExist) { return nil, err } @@ -315,23 +312,19 @@ func scanForNextHead(ctx context.Context, dataDir string, headCatalog Catalog, d return nil, false, err } - headRecords, err := headCatalog.List( + headRecords := headCatalog.List( nil, func(lhs, rhs *catalog.Record) bool { return lhs.CreatedAt() > rhs.CreatedAt() }, ) - if err != nil { - return nil, false, fmt.Errorf("list head records: %w", err) - } if len(headRecords) == 0 { return nil, false, fmt.Errorf("scanForNextHead: no new heads: empty head records") } - var headFound bool for _, headRecord := range headRecords { - headFound, err = scanHeadForDestination(filepath.Join(dataDir, headRecord.Dir()), destinationName) + headFound, err := scanHeadForDestination(filepath.Join(dataDir, headRecord.Dir()), destinationName) if err != nil { if !headRecord.Corrupted() { logger.Errorf("head %s is corrupted: %v", headRecord.ID(), err) diff --git a/pp/go/storage/storagetest/fixtures.go b/pp/go/storage/storagetest/fixtures.go index d56d20ead9..b9dd342c35 100644 --- a/pp/go/storage/storagetest/fixtures.go +++ b/pp/go/storage/storagetest/fixtures.go @@ -25,9 +25,9 @@ func (s *TimeSeries) AppendSamples(samples ...cppbridge.Sample) { func (s *TimeSeries) toModelTimeSeries() []model.TimeSeries { lsBuilder := model.NewLabelSetBuilder() - for i := range s.Labels { - lsBuilder.Add(s.Labels[i].Name, s.Labels[i].Value) - } + s.Labels.Range(func(l labels.Label) { + lsBuilder.Add(l.Name, l.Value) + }) ls := lsBuilder.Build() diff --git a/web/api/v1/pp_api.go b/web/api/v1/pp_api.go index b6623d9c86..f595753402 100644 --- a/web/api/v1/pp_api.go +++ b/web/api/v1/pp_api.go @@ -83,7 +83,7 @@ func (api *API) queryHead(r *http.Request) apiFuncResult { matchers = append(matchers, selector...) } - q, err := api.adapter.HeadQuerier(ctx, start.UnixMilli(), end.UnixMilli()) + q, err := api.adapter.HeadQuerier(start.UnixMilli(), end.UnixMilli()) if err != nil { return apiFuncResult{nil, &apiError{errorBadData, err}, nil, nil} } @@ -122,7 +122,12 @@ func (api *API) serveHeadStatus(r *http.Request) apiFuncResult { } } - return apiFuncResult{api.adapter.HeadStatus(r.Context(), limit), nil, nil, nil} + hstatus, err := api.adapter.HeadStatus(r.Context(), limit) + if err != nil { + return apiFuncResult{nil, &apiError{errorExec, errors.New("limit must be a positive number")}, nil, nil} + } + + return apiFuncResult{hstatus, nil, nil, nil} } func (api *API) opRemoteWrite(middlewares ...middleware.Middleware) http.HandlerFunc { From 1e816ceb8c592b656b35b3ee8e94af8cf1e2b488 Mon Sep 17 00:00:00 2001 From: Alexandr Yudin Date: Thu, 25 Sep 2025 07:02:13 +0000 Subject: [PATCH 47/96] debug --- cmd/prometheus/main.go | 5 +++-- pp/go/cppbridge/head.go | 3 +++ pp/go/storage/head/services/rotator.go | 9 +++++++++ 3 files changed, 15 insertions(+), 2 deletions(-) diff --git a/cmd/prometheus/main.go b/cmd/prometheus/main.go index 541f893f04..24d931833a 100644 --- a/cmd/prometheus/main.go +++ b/cmd/prometheus/main.go @@ -759,8 +759,9 @@ func main() { hManagerReadyNotifier := ready.NewNotifiableNotifier() hManager, err := pp_storage.NewManager( &pp_storage.Options{ - Seed: cfgFile.GlobalConfig.ExternalLabels.Hash(), - BlockDuration: time.Duration(cfg.tsdb.MinBlockDuration), + Seed: cfgFile.GlobalConfig.ExternalLabels.Hash(), + // BlockDuration: time.Duration(cfg.tsdb.MinBlockDuration), + BlockDuration: 6 * time.Minute, CommitInterval: time.Duration(cfg.WalCommitInterval), MaxRetentionPeriod: time.Duration(cfg.tsdb.RetentionDuration), HeadRetentionPeriod: time.Duration(cfg.HeadRetentionTimeout), diff --git a/pp/go/cppbridge/head.go b/pp/go/cppbridge/head.go index 79e8d33c70..29c9716ee7 100644 --- a/pp/go/cppbridge/head.go +++ b/pp/go/cppbridge/head.go @@ -318,6 +318,9 @@ func (ds *HeadDataStorage) Query(query HeadDataStorageQuery) (*HeadDataStorageSe serializedChunks := &HeadDataStorageSerializedChunks{} result := seriesDataDataStorageQuery(ds.dataStorage, query, &serializedChunks.data) runtime.KeepAlive(ds) + runtime.SetFinalizer(serializedChunks, func(sc *HeadDataStorageSerializedChunks) { + freeBytes(sc.data) + }) return serializedChunks, result } diff --git a/pp/go/storage/head/services/rotator.go b/pp/go/storage/head/services/rotator.go index 60d73e2ded..e344b88762 100644 --- a/pp/go/storage/head/services/rotator.go +++ b/pp/go/storage/head/services/rotator.go @@ -91,8 +91,10 @@ func (s *Rotator[TTask, TShard, TGoShard, THead]) rotate( ctx context.Context, numberOfShards uint16, ) error { + fmt.Println("rotate") oldHead := s.proxyHead.Get() + fmt.Println("rotate newHead") newHead, err := s.headBuilder.Build(oldHead.Generation()+1, numberOfShards) if err != nil { return fmt.Errorf("failed to build a new head: %w", err) @@ -101,30 +103,37 @@ func (s *Rotator[TTask, TShard, TGoShard, THead]) rotate( // TODO CopySeriesFrom only old nunber of shards == new // newHead.CopySeriesFrom(oldHead) + fmt.Println("rotate AddWithReplace") if err = s.proxyHead.AddWithReplace(oldHead, s.headInformer.CreatedAt(oldHead.ID())); err != nil { return fmt.Errorf("failed add to keeper old head: %w", err) } // TODO if replace error? + fmt.Println("rotate Replace") if err = s.proxyHead.Replace(ctx, newHead); err != nil { return fmt.Errorf("failed to replace old to new head: %w", err) } + fmt.Println("rotate SetActiveStatus") if err = s.headInformer.SetActiveStatus(newHead.ID()); err != nil { logger.Warnf("failed set status active for head{%s}: %s", newHead.ID(), err) } + fmt.Println("rotate MergeOutOfOrderChunksWithHead") if err = MergeOutOfOrderChunksWithHead(oldHead); err != nil { logger.Warnf("failed merge out of order chunks in data storage: %s", err) } + fmt.Println("rotate CFSViaRange") if err = CFSViaRange(oldHead); err != nil { logger.Warnf("failed commit and flush to wal: %s", err) } + fmt.Println("rotate SetRotatedStatus") if err = s.headInformer.SetRotatedStatus(oldHead.ID()); err != nil { logger.Warnf("failed set status rotated for head{%s}: %s", oldHead.ID(), err) } + fmt.Println("rotate SetReadOnly") oldHead.SetReadOnly() return nil From 041b5bfdf0809cebf523dfb535bf5816ab25d3d1 Mon Sep 17 00:00:00 2001 From: Alexandr Yudin Date: Thu, 25 Sep 2025 14:11:20 +0000 Subject: [PATCH 48/96] add log, letrics --- cmd/prometheus/main.go | 10 +- pp-pkg/handler/processor/interface.go | 5 +- .../handler/processor/processor_moq_test.go | 21 - .../processor/stream_processor_test.go | 2 - pp-pkg/storage/adapter.go | 37 ++ pp/go/storage/appender/appender.go | 5 +- pp/go/storage/builder.go | 17 +- pp/go/storage/head/head/head.go | 39 +- pp/go/storage/head/keeper/keeper.go | 37 +- pp/go/storage/head/keeper/keeper_test.go | 29 +- pp/go/storage/head/services/interface.go | 38 ++ .../storage/head/services/mock/persistener.go | 55 ++ pp/go/storage/head/services/persistener.go | 122 ++-- .../storage/head/services/persistener_test.go | 56 +- pp/go/storage/head/services/rotator.go | 60 +- pp/go/storage/head/shard/wal/wal.go | 8 +- pp/go/storage/head/shard/wal/wal_moq_test.go | 465 +++++--------- pp/go/storage/head/shard/wal/wal_reader.go | 3 + .../head/shard/wal/wal_reader_moq_test.go | 108 ++++ pp/go/storage/head/shard/wal/wal_test.go | 2 - .../storage/head/shard/wal/writer/buffered.go | 3 + ...riter_moq_test.go => buffered_moq_test.go} | 606 ++++++------------ .../head/shard/wal/writer/buffered_test.go | 2 - .../storage/head/shard/wal/writer/segment.go | 3 + .../head/shard/wal/writer/segment_moq_test.go | 182 ++++++ pp/go/storage/loader.go | 14 +- pp/go/storage/manager.go | 17 +- pp/go/storage/mediator/mediator.go | 11 + pp/go/storage/mediator/mediator_moq_test.go | 46 +- pp/go/storage/mediator/mediator_test.go | 14 +- pp/go/storage/querier/metrics.go | 6 +- pp/go/storage/querier/querier.go | 3 +- 32 files changed, 1152 insertions(+), 874 deletions(-) create mode 100644 pp/go/storage/head/shard/wal/wal_reader_moq_test.go rename pp/go/storage/head/shard/wal/writer/{writer_moq_test.go => buffered_moq_test.go} (70%) create mode 100644 pp/go/storage/head/shard/wal/writer/segment_moq_test.go diff --git a/cmd/prometheus/main.go b/cmd/prometheus/main.go index 24d931833a..759f49998a 100644 --- a/cmd/prometheus/main.go +++ b/cmd/prometheus/main.go @@ -721,6 +721,8 @@ func main() { os.Exit(1) } + pp_storage.InitLogHandler(log.With(logger, "component", "pp_storage")) + reloadBlocksTriggerNotifier := pp_storage.NewReloadBlocksTriggerNotifier() cfg.tsdb.ReloadBlocksExternalTrigger = reloadBlocksTriggerNotifier @@ -754,14 +756,11 @@ func main() { os.Exit(1) } - pp_storage.InitLogHandler(log.With(logger, "component", "pp_storage")) - hManagerReadyNotifier := ready.NewNotifiableNotifier() hManager, err := pp_storage.NewManager( &pp_storage.Options{ - Seed: cfgFile.GlobalConfig.ExternalLabels.Hash(), - // BlockDuration: time.Duration(cfg.tsdb.MinBlockDuration), - BlockDuration: 6 * time.Minute, + Seed: cfgFile.GlobalConfig.ExternalLabels.Hash(), + BlockDuration: time.Duration(cfg.tsdb.MinBlockDuration), CommitInterval: time.Duration(cfg.WalCommitInterval), MaxRetentionPeriod: time.Duration(cfg.tsdb.RetentionDuration), HeadRetentionPeriod: time.Duration(cfg.HeadRetentionTimeout), @@ -2011,6 +2010,7 @@ func (opts tsdbOptions) ToTSDBOptions() tsdb.Options { OutOfOrderTimeWindow: opts.OutOfOrderTimeWindow, EnableDelayedCompaction: opts.EnableDelayedCompaction, EnableOverlappingCompaction: opts.EnableOverlappingCompaction, + ReloadBlocksExternalTrigger: opts.ReloadBlocksExternalTrigger, } } diff --git a/pp-pkg/handler/processor/interface.go b/pp-pkg/handler/processor/interface.go index 814bfb3f51..b659f63484 100644 --- a/pp-pkg/handler/processor/interface.go +++ b/pp-pkg/handler/processor/interface.go @@ -1,7 +1,5 @@ package processor -//go:generate -command moq go run github.com/matryer/moq -out processor_moq_test.go -pkg processor_test -rm . Adapter StatesStorage RemoteWrite MetricStream Refill - import ( "context" @@ -11,6 +9,9 @@ import ( "github.com/prometheus/prometheus/pp/go/cppbridge" ) +//go:generate -command moq go run github.com/matryer/moq --rm --skip-ensure --pkg processor_test --out +//go:generate moq processor_moq_test.go . Adapter StatesStorage RemoteWrite MetricStream Refill + type MetricStream interface { Metadata() model.Metadata Read(ctx context.Context) (*model.Segment, error) diff --git a/pp-pkg/handler/processor/processor_moq_test.go b/pp-pkg/handler/processor/processor_moq_test.go index 7a8974e9dc..915cabdb92 100644 --- a/pp-pkg/handler/processor/processor_moq_test.go +++ b/pp-pkg/handler/processor/processor_moq_test.go @@ -6,16 +6,11 @@ package processor_test import ( "context" "github.com/prometheus/prometheus/pp-pkg/handler/model" - "github.com/prometheus/prometheus/pp-pkg/handler/processor" pp_pkg_model "github.com/prometheus/prometheus/pp-pkg/model" "github.com/prometheus/prometheus/pp/go/cppbridge" "sync" ) -// Ensure, that AdapterMock does implement processor.Adapter. -// If this is not the case, regenerate this file with moq. -var _ processor.Adapter = &AdapterMock{} - // AdapterMock is a mock implementation of processor.Adapter. // // func TestSomethingThatUsesAdapter(t *testing.T) { @@ -319,10 +314,6 @@ func (mock *AdapterMock) MergeOutOfOrderChunksCalls() []struct { return calls } -// Ensure, that StatesStorageMock does implement processor.StatesStorage. -// If this is not the case, regenerate this file with moq. -var _ processor.StatesStorage = &StatesStorageMock{} - // StatesStorageMock is a mock implementation of processor.StatesStorage. // // func TestSomethingThatUsesStatesStorage(t *testing.T) { @@ -385,10 +376,6 @@ func (mock *StatesStorageMock) GetStateByIDCalls() []struct { return calls } -// Ensure, that RemoteWriteMock does implement processor.RemoteWrite. -// If this is not the case, regenerate this file with moq. -var _ processor.RemoteWrite = &RemoteWriteMock{} - // RemoteWriteMock is a mock implementation of processor.RemoteWrite. // // func TestSomethingThatUsesRemoteWrite(t *testing.T) { @@ -538,10 +525,6 @@ func (mock *RemoteWriteMock) WriteCalls() []struct { return calls } -// Ensure, that MetricStreamMock does implement processor.MetricStream. -// If this is not the case, regenerate this file with moq. -var _ processor.MetricStream = &MetricStreamMock{} - // MetricStreamMock is a mock implementation of processor.MetricStream. // // func TestSomethingThatUsesMetricStream(t *testing.T) { @@ -691,10 +674,6 @@ func (mock *MetricStreamMock) WriteCalls() []struct { return calls } -// Ensure, that RefillMock does implement processor.Refill. -// If this is not the case, regenerate this file with moq. -var _ processor.Refill = &RefillMock{} - // RefillMock is a mock implementation of processor.Refill. // // func TestSomethingThatUsesRefill(t *testing.T) { diff --git a/pp-pkg/handler/processor/stream_processor_test.go b/pp-pkg/handler/processor/stream_processor_test.go index 9333951a81..7e3a9bb8d0 100644 --- a/pp-pkg/handler/processor/stream_processor_test.go +++ b/pp-pkg/handler/processor/stream_processor_test.go @@ -22,8 +22,6 @@ import ( "github.com/prometheus/prometheus/util/pool" ) -// TODO //go:generate -command moq go run github.com/matryer/moq -out processor_moq_test.go -pkg processor_test -rm . Adapter StatesStorage RemoteWrite MetricStream Refill - type segmentContainer struct { timeSeries []coremodel.TimeSeries encoded model.Segment diff --git a/pp-pkg/storage/adapter.go b/pp-pkg/storage/adapter.go index 03bcc38334..9a077454b5 100644 --- a/pp-pkg/storage/adapter.go +++ b/pp-pkg/storage/adapter.go @@ -3,6 +3,7 @@ package storage import ( "context" "math" + "time" "github.com/jonboulle/clockwork" "github.com/prometheus/client_golang/prometheus" @@ -14,6 +15,7 @@ import ( "github.com/prometheus/prometheus/pp/go/storage/appender" "github.com/prometheus/prometheus/pp/go/storage/head/services" "github.com/prometheus/prometheus/pp/go/storage/querier" + "github.com/prometheus/prometheus/pp/go/util" "github.com/prometheus/prometheus/storage" ) @@ -32,8 +34,10 @@ type Adapter struct { transparentState *cppbridge.StateV2 mergeOutOfOrderChunks func() + // stat activeQuerierMetrics *querier.Metrics storageQuerierMetrics *querier.Metrics + appendDuration prometheus.Histogram } // NewAdapter init new [Adapter]. @@ -43,6 +47,7 @@ func NewAdapter( mergeOutOfOrderChunks func(), registerer prometheus.Registerer, ) *Adapter { + factory := util.NewUnconflictRegisterer(registerer) return &Adapter{ proxy: proxy, haTracker: hatracker.NewHighAvailabilityTracker(clock, registerer), @@ -52,6 +57,18 @@ func NewAdapter( mergeOutOfOrderChunks: mergeOutOfOrderChunks, activeQuerierMetrics: querier.NewMetrics(registerer, querier.QueryableAppenderSource), storageQuerierMetrics: querier.NewMetrics(registerer, querier.QueryableStorageSource), + appendDuration: factory.NewHistogram( + prometheus.HistogramOpts{ + Name: "prompp_adapter_append_duration", + Help: "Append to head duration in microseconds", + Buckets: []float64{ + 50, 100, 250, 500, 750, + 1000, 2500, 5000, 7500, + 10000, 25000, 50000, 75000, + 100000, 500000, + }, + }, + ), } } @@ -66,6 +83,11 @@ func (ar *Adapter) AppendHashdex( return nil } + start := time.Now() + defer func() { + ar.appendDuration.Observe(float64(time.Since(start).Microseconds())) + }() + return ar.proxy.With(ctx, func(h *pp_storage.HeadOnDisk) error { _, _, err := appender.New(h, services.CFViaRange).Append( ctx, @@ -85,6 +107,11 @@ func (ar *Adapter) AppendScraperHashdex( state *cppbridge.StateV2, commitToWal bool, ) (stats cppbridge.RelabelerStats, err error) { + start := time.Now() + defer func() { + ar.appendDuration.Observe(float64(time.Since(start).Microseconds())) + }() + _ = ar.proxy.With(ctx, func(h *pp_storage.HeadOnDisk) error { _, stats, err = appender.New(h, services.CFViaRange).Append( ctx, @@ -116,6 +143,11 @@ func (ar *Adapter) AppendSnappyProtobuf( return nil } + start := time.Now() + defer func() { + ar.appendDuration.Observe(float64(time.Since(start).Microseconds())) + }() + return ar.proxy.With(ctx, func(h *pp_storage.HeadOnDisk) error { _, _, err := appender.New(h, services.CFViaRange).Append( ctx, @@ -146,6 +178,11 @@ func (ar *Adapter) AppendTimeSeries( return stats, nil } + start := time.Now() + defer func() { + ar.appendDuration.Observe(float64(time.Since(start).Microseconds())) + }() + _ = ar.proxy.With(ctx, func(h *pp_storage.HeadOnDisk) error { _, stats, err = appender.New(h, services.CFViaRange).Append( ctx, diff --git a/pp/go/storage/appender/appender.go b/pp/go/storage/appender/appender.go index 0e3ea6c69d..a183f036af 100644 --- a/pp/go/storage/appender/appender.go +++ b/pp/go/storage/appender/appender.go @@ -119,7 +119,10 @@ func New[ TLSS LSS, TShard Shard[TLSS], THead Head[TTask, TLSS, TShard], -](head THead, commitAndFlush func(h THead) error) Appender[TTask, TLSS, TShard, THead] { +]( + head THead, + commitAndFlush func(h THead) error, +) Appender[TTask, TLSS, TShard, THead] { return Appender[TTask, TLSS, TShard, THead]{ head: head, commitAndFlush: commitAndFlush, diff --git a/pp/go/storage/builder.go b/pp/go/storage/builder.go index 488fcecc2e..b4c5476859 100644 --- a/pp/go/storage/builder.go +++ b/pp/go/storage/builder.go @@ -14,6 +14,7 @@ import ( "github.com/prometheus/prometheus/pp/go/storage/head/shard" "github.com/prometheus/prometheus/pp/go/storage/head/shard/wal" "github.com/prometheus/prometheus/pp/go/storage/head/shard/wal/writer" + "github.com/prometheus/prometheus/pp/go/util" ) // @@ -27,6 +28,8 @@ type Builder struct { maxSegmentSize uint32 registerer prometheus.Registerer unloadDataStorageInterval time.Duration + // stat + events *prometheus.CounterVec } // NewBuilder init new [Builder]. @@ -37,12 +40,20 @@ func NewBuilder( registerer prometheus.Registerer, unloadDataStorageInterval time.Duration, ) *Builder { + factory := util.NewUnconflictRegisterer(registerer) return &Builder{ catalog: hcatalog, dataDir: dataDir, maxSegmentSize: maxSegmentSize, registerer: registerer, unloadDataStorageInterval: unloadDataStorageInterval, + events: factory.NewCounterVec( + prometheus.CounterOpts{ + Name: "prompp_head_event_count", + Help: "Number of head events", + }, + []string{"type"}, + ), } } @@ -75,6 +86,7 @@ func (b *Builder) Build(generation uint64, numberOfShards uint16) (*HeadOnDisk, shards[shardID] = s } + b.events.With(prometheus.Labels{"type": "created"}).Inc() return head.NewHead( headRecord.ID(), shards, @@ -121,7 +133,10 @@ func (b *Builder) createShardOnDisk( var unloadedDataStorage *shard.UnloadedDataStorage var queriedSeriesStorage *shard.QueriedSeriesStorage if b.unloadDataStorageInterval != 0 { - unloadedDataStorage = shard.NewUnloadedDataStorage(shard.NewFileStorage(GetUnloadedDataStorageFilename(headDir, shardID))) + unloadedDataStorage = shard.NewUnloadedDataStorage( + shard.NewFileStorage(GetUnloadedDataStorageFilename(headDir, shardID)), + ) + queriedSeriesStorage = shard.NewQueriedSeriesStorage( shard.NewFileStorage(GetQueriedSeriesStorageFilename(headDir, shardID, 0)), shard.NewFileStorage(GetQueriedSeriesStorageFilename(headDir, shardID, 1)), diff --git a/pp/go/storage/head/head/head.go b/pp/go/storage/head/head/head.go index b4a9acff14..dc89a9958e 100644 --- a/pp/go/storage/head/head/head.go +++ b/pp/go/storage/head/head/head.go @@ -52,8 +52,9 @@ type Head[TShard Shard, TGorutineShard Shard] struct { taskChs []chan *task.Generic[TGorutineShard] querySemaphore *locker.Weighted - stopc chan struct{} - wg sync.WaitGroup + stopc chan struct{} + wg sync.WaitGroup + closeOnce sync.Once readOnly uint32 @@ -95,6 +96,7 @@ func NewHead[TShard Shard, TGoroutineShard Shard]( querySemaphore: locker.NewWeighted(2 * concurrency), // x2 for back pressure stopc: make(chan struct{}), wg: sync.WaitGroup{}, + closeOnce: sync.Once{}, // for clearing [Head] metrics memoryInUse: factory.NewGaugeVec(prometheus.GaugeOpts{ @@ -124,9 +126,11 @@ func NewHead[TShard Shard, TGoroutineShard Shard]( runtime.SetFinalizer(h, func(h *Head[TShard, TGoroutineShard]) { h.memoryInUse.DeletePartialMatch(prometheus.Labels{"head_id": h.id}) - logger.Debugf("head %s destroyed", h.String()) + logger.Debugf("[Head] %s destroyed.", h.String()) }) + logger.Debugf("[Head] %s created.", h.String()) + return h } @@ -138,22 +142,25 @@ func (h *Head[TShard, TGorutineShard]) AcquireQuery(ctx context.Context) (releas } // Close closes wals, query semaphore for the inability to get query and clear metrics. -func (h *Head[TShard, TGorutineShard]) Close() error { - if err := h.querySemaphore.Close(); err != nil { - return err - } +func (h *Head[TShard, TGorutineShard]) Close() (err error) { + h.closeOnce.Do(func() { + if err = h.querySemaphore.Close(); err != nil { + return + } - close(h.stopc) - h.wg.Wait() + close(h.stopc) + h.wg.Wait() - var err error - for _, s := range h.shards { - err = errors.Join(err, s.Close()) - } + for _, s := range h.shards { + err = errors.Join(err, s.Close()) + } - if h.releaseHeadFn != nil { - h.releaseHeadFn() - } + if h.releaseHeadFn != nil { + h.releaseHeadFn() + } + + logger.Debugf("[Head] %s is closed", h.String()) + }) return err } diff --git a/pp/go/storage/head/keeper/keeper.go b/pp/go/storage/head/keeper/keeper.go index afbadb4e01..0104194153 100644 --- a/pp/go/storage/head/keeper/keeper.go +++ b/pp/go/storage/head/keeper/keeper.go @@ -5,6 +5,8 @@ import ( "errors" "sync" "time" + + "github.com/prometheus/prometheus/pp/go/storage/logger" ) type addPolicy = uint8 @@ -51,29 +53,32 @@ func (q *headSortedSlice[THead]) Pop() any { } // Head the minimum required [Head] implementation for a [Keeper]. -type Head interface { +type Head[T any] interface { // ID returns id [Head]. ID() string // Close closes wals, query semaphore for the inability to get query and clear metrics. Close() error + + // for use as a pointer + *T } // Keeper holds outdated heads until conversion. -type Keeper[THead Head] struct { +type Keeper[T any, THead Head[T]] struct { heads headSortedSlice[THead] lock sync.RWMutex } // NewKeeper init new [Keeper]. -func NewKeeper[THead Head](queueSize int) *Keeper[THead] { - return &Keeper[THead]{ +func NewKeeper[T any, THead Head[T]](queueSize int) *Keeper[T, THead] { + return &Keeper[T, THead]{ heads: make(headSortedSlice[THead], 0, max(queueSize, MinHeadConvertingQueueSize)), } } // Add the [Head] to the [Keeper] if there is a free slot. -func (k *Keeper[THead]) Add(head THead, createdAt time.Duration) error { +func (k *Keeper[T, THead]) Add(head THead, createdAt time.Duration) error { k.lock.Lock() result := k.addHead(head, createdAt, add) k.lock.Unlock() @@ -82,7 +87,7 @@ func (k *Keeper[THead]) Add(head THead, createdAt time.Duration) error { } // AddWithReplace the [Head] to the [Keeper] with replace if the createdAt is earlier. -func (k *Keeper[THead]) AddWithReplace(head THead, createdAt time.Duration) error { +func (k *Keeper[T, THead]) AddWithReplace(head THead, createdAt time.Duration) error { k.lock.Lock() result := k.addHead(head, createdAt, addWithReplace) k.lock.Unlock() @@ -91,7 +96,7 @@ func (k *Keeper[THead]) AddWithReplace(head THead, createdAt time.Duration) erro } // Close closes for the inability work with [Head]. -func (k *Keeper[THead]) Close() error { +func (k *Keeper[T, THead]) Close() error { k.lock.Lock() if len(k.heads) == 0 { k.lock.Unlock() @@ -108,7 +113,7 @@ func (k *Keeper[THead]) Close() error { } // HasSlot returns the tru if there is a slot in the [Keeper]. -func (k *Keeper[THead]) HasSlot() bool { +func (k *Keeper[T, THead]) HasSlot() bool { k.lock.RLock() result := cap(k.heads) > len(k.heads) k.lock.RUnlock() @@ -116,7 +121,7 @@ func (k *Keeper[THead]) HasSlot() bool { } // Heads returns a slice of the [Head]s stored in the [Keeper]. -func (k *Keeper[THead]) Heads() []THead { +func (k *Keeper[T, THead]) Heads() []THead { k.lock.RLock() if len(k.heads) == 0 { @@ -135,12 +140,12 @@ func (k *Keeper[THead]) Heads() []THead { } // Remove removes [Head]s from the [Keeper]. -func (k *Keeper[THead]) Remove(headsForRemove []THead) { +func (k *Keeper[T, THead]) Remove(headsForRemove []THead) { if len(headsForRemove) == 0 { return } - headsMap := make(map[string]*THead, len(headsForRemove)) + headsMap := make(map[string]THead, len(headsForRemove)) for _, head := range headsForRemove { headsMap[head.ID()] = nil } @@ -149,7 +154,7 @@ func (k *Keeper[THead]) Remove(headsForRemove []THead) { newHeads := make([]sortableHead[THead], 0, cap(k.heads)) for _, head := range k.heads { if _, ok := headsMap[head.head.ID()]; ok { - headsMap[head.head.ID()] = &head.head + headsMap[head.head.ID()] = head.head } else { newHeads = append(newHeads, head) } @@ -159,18 +164,20 @@ func (k *Keeper[THead]) Remove(headsForRemove []THead) { for _, head := range headsMap { if head != nil { - _ = (*head).Close() + _ = head.Close() + logger.Infof("[Keeper]: head %s persisted, closed and removed", head.ID()) } } } -func (k *Keeper[THead]) addHead(head THead, createdAt time.Duration, policy addPolicy) error { +func (k *Keeper[T, THead]) addHead(head THead, createdAt time.Duration, policy addPolicy) error { if len(k.heads) < cap(k.heads) { heap.Push(&k.heads, sortableHead[THead]{head: head, createdAt: createdAt}) return nil } if policy == addWithReplace && k.heads[0].createdAt < createdAt { + _ = k.heads[0].head.Close() k.heads[0].head = head k.heads[0].createdAt = createdAt heap.Fix(&k.heads, 0) @@ -180,7 +187,7 @@ func (k *Keeper[THead]) addHead(head THead, createdAt time.Duration, policy addP return ErrorNoSlots } -func (k *Keeper[THead]) setHeads(heads headSortedSlice[THead]) { +func (k *Keeper[T, THead]) setHeads(heads headSortedSlice[THead]) { k.heads = heads heap.Init(&k.heads) } diff --git a/pp/go/storage/head/keeper/keeper_test.go b/pp/go/storage/head/keeper/keeper_test.go index f5ab01ab8a..42d5bb90d2 100644 --- a/pp/go/storage/head/keeper/keeper_test.go +++ b/pp/go/storage/head/keeper/keeper_test.go @@ -1,6 +1,7 @@ package keeper import ( + "container/heap" "testing" "github.com/stretchr/testify/suite" @@ -26,7 +27,7 @@ type sortedSlice = headSortedSlice[*headForTest] type KeeperSuite struct { suite.Suite - keeper *Keeper[*headForTest] + keeper *Keeper[headForTest, *headForTest] } func TestKeeperSuite(t *testing.T) { @@ -43,7 +44,7 @@ func (s *KeeperSuite) getHeads() []*headForTest { func (s *KeeperSuite) TestAdd() { // Arrange - s.keeper = NewKeeper[*headForTest](2) + s.keeper = NewKeeper[headForTest](2) // Act _ = s.keeper.Add(newHeadForTest("d"), 4) @@ -60,7 +61,7 @@ func (s *KeeperSuite) TestAdd() { func (s *KeeperSuite) TestAddWithReplaceNoReplace() { // Arrange - s.keeper = NewKeeper[*headForTest](2) + s.keeper = NewKeeper[headForTest](2) // Act _ = s.keeper.Add(newHeadForTest("d"), 4) @@ -77,7 +78,7 @@ func (s *KeeperSuite) TestAddWithReplaceNoReplace() { func (s *KeeperSuite) TestAddWithReplace() { // Arrange - s.keeper = NewKeeper[*headForTest](2) + s.keeper = NewKeeper[headForTest](2) // Act _ = s.keeper.Add(newHeadForTest("d"), 4) @@ -96,7 +97,7 @@ func (s *KeeperSuite) TestRemove() { // Arrange const Slots = 5 - s.keeper = NewKeeper[*headForTest](Slots) + s.keeper = NewKeeper[headForTest](Slots) _ = s.keeper.Add(newHeadForTest("a"), 1) _ = s.keeper.Add(newHeadForTest("b"), 2) _ = s.keeper.Add(newHeadForTest("c"), 3) @@ -113,3 +114,21 @@ func (s *KeeperSuite) TestRemove() { }, s.keeper.heads) s.Equal(Slots, cap(s.keeper.heads)) } + +func TestXxx(t *testing.T) { + ss := sortedSlice{ + {head: newHeadForTest("b"), createdAt: 2}, + {head: newHeadForTest("d"), createdAt: 4}, + } + + t.Log(ss) + + ss[0].head = newHeadForTest("b") + ss[0].createdAt = 5 + + t.Log(ss) + + heap.Fix(&ss, 0) + + t.Log(ss) +} diff --git a/pp/go/storage/head/services/interface.go b/pp/go/storage/head/services/interface.go index 51fb8def21..efaf29b44e 100644 --- a/pp/go/storage/head/services/interface.go +++ b/pp/go/storage/head/services/interface.go @@ -5,8 +5,13 @@ import ( "time" "github.com/prometheus/prometheus/pp/go/cppbridge" + "github.com/prometheus/prometheus/pp/go/storage/block" + "github.com/prometheus/prometheus/pp/go/storage/catalog" ) +//go:generate -command moq go run github.com/matryer/moq --rm --skip-ensure --pkg mock --out +//go:generate moq mock/persistener.go . HeadBlockWriter WriteNotifier + // // ActiveHeadContainer // @@ -208,3 +213,36 @@ type Task interface { // Wait for the task to complete on all shards. Wait() error } + +// +// WriteNotifier +// + +// WriteNotifier sends a notify that the writing is completed. +type WriteNotifier interface { + // NotifyWritten sends a notify that the writing is completed. + NotifyWritten() +} + +// +// Loader +// + +// Loader loads [Head] from [Wal]. +type Loader[ + TTask Task, + TShard, TGoShard Shard, + THead Head[TTask, TShard, TGoShard], +] interface { + // Load [Head] from [Wal] by head ID. + Load(headRecord *catalog.Record, generation uint64) (THead, bool) +} + +// +// HeadBlockWriter +// + +// HeadBlockWriter writes block on disk from [Head]. +type HeadBlockWriter[TShard Shard] interface { + Write(shard TShard) ([]block.WrittenBlock, error) +} diff --git a/pp/go/storage/head/services/mock/persistener.go b/pp/go/storage/head/services/mock/persistener.go index 2fd61c3bf9..219100712e 100644 --- a/pp/go/storage/head/services/mock/persistener.go +++ b/pp/go/storage/head/services/mock/persistener.go @@ -70,3 +70,58 @@ func (mock *HeadBlockWriterMock[TShard]) WriteCalls() []struct { mock.lockWrite.RUnlock() return calls } + +// WriteNotifierMock is a mock implementation of services.WriteNotifier. +// +// func TestSomethingThatUsesWriteNotifier(t *testing.T) { +// +// // make and configure a mocked services.WriteNotifier +// mockedWriteNotifier := &WriteNotifierMock{ +// NotifyWrittenFunc: func() { +// panic("mock out the NotifyWritten method") +// }, +// } +// +// // use mockedWriteNotifier in code that requires services.WriteNotifier +// // and then make assertions. +// +// } +type WriteNotifierMock struct { + // NotifyWrittenFunc mocks the NotifyWritten method. + NotifyWrittenFunc func() + + // calls tracks calls to the methods. + calls struct { + // NotifyWritten holds details about calls to the NotifyWritten method. + NotifyWritten []struct { + } + } + lockNotifyWritten sync.RWMutex +} + +// NotifyWritten calls NotifyWrittenFunc. +func (mock *WriteNotifierMock) NotifyWritten() { + if mock.NotifyWrittenFunc == nil { + panic("WriteNotifierMock.NotifyWrittenFunc: method is nil but WriteNotifier.NotifyWritten was just called") + } + callInfo := struct { + }{} + mock.lockNotifyWritten.Lock() + mock.calls.NotifyWritten = append(mock.calls.NotifyWritten, callInfo) + mock.lockNotifyWritten.Unlock() + mock.NotifyWrittenFunc() +} + +// NotifyWrittenCalls gets all the calls that were made to NotifyWritten. +// Check the length with: +// +// len(mockedWriteNotifier.NotifyWrittenCalls()) +func (mock *WriteNotifierMock) NotifyWrittenCalls() []struct { +} { + var calls []struct { + } + mock.lockNotifyWritten.RLock() + calls = mock.calls.NotifyWritten + mock.lockNotifyWritten.RUnlock() + return calls +} diff --git a/pp/go/storage/head/services/persistener.go b/pp/go/storage/head/services/persistener.go index 84ad38dca4..256274f3f1 100644 --- a/pp/go/storage/head/services/persistener.go +++ b/pp/go/storage/head/services/persistener.go @@ -5,74 +5,99 @@ import ( "time" "github.com/jonboulle/clockwork" + "github.com/prometheus/client_golang/prometheus" "github.com/prometheus/prometheus/pp/go/cppbridge" - "github.com/prometheus/prometheus/pp/go/storage/block" "github.com/prometheus/prometheus/pp/go/storage/catalog" "github.com/prometheus/prometheus/pp/go/storage/logger" + "github.com/prometheus/prometheus/pp/go/util" ) -//go:generate -command moq go run github.com/matryer/moq --rm --skip-ensure --pkg mock --out - -// -// HeadBlockWriter // - -// HeadBlockWriter writes block on disk from [Head]. +// Persistener // -//go:generate moq mock/persistener.go . HeadBlockWriter -type HeadBlockWriter[TShard Shard] interface { - Write(shard TShard) ([]block.WrittenBlock, error) -} +// Persistener converts and saves spent [Head]s. type Persistener[ TTask Task, TShard, TGoShard Shard, THeadBlockWriter HeadBlockWriter[TShard], THead Head[TTask, TShard, TGoShard], ] struct { - catalog *catalog.Catalog - blockWriter THeadBlockWriter + catalog *catalog.Catalog + blockWriter THeadBlockWriter + writeNotifier WriteNotifier clock clockwork.Clock tsdbRetentionPeriod time.Duration retentionPeriod time.Duration + // stat + events *prometheus.CounterVec + headPersistenceDuration prometheus.Histogram } +// NewPersistener init new [Persistener]. func NewPersistener[ TTask Task, TShard, TGoShard Shard, THeadBlockWriter HeadBlockWriter[TShard], THead Head[TTask, TShard, TGoShard], ]( - catalog *catalog.Catalog, + hcatalog *catalog.Catalog, blockWriter THeadBlockWriter, + writeNotifier WriteNotifier, clock clockwork.Clock, tsdbRetentionPeriod time.Duration, retentionPeriod time.Duration, + registerer prometheus.Registerer, ) *Persistener[TTask, TShard, TGoShard, THeadBlockWriter, THead] { + factory := util.NewUnconflictRegisterer(registerer) return &Persistener[TTask, TShard, TGoShard, THeadBlockWriter, THead]{ - catalog: catalog, + catalog: hcatalog, blockWriter: blockWriter, + writeNotifier: writeNotifier, clock: clock, tsdbRetentionPeriod: tsdbRetentionPeriod, retentionPeriod: retentionPeriod, + events: factory.NewCounterVec( + prometheus.CounterOpts{ + Name: "prompp_head_event_count", + Help: "Number of head events", + }, + []string{"type"}, + ), + headPersistenceDuration: factory.NewHistogram( + prometheus.HistogramOpts{ + Name: "prompp_head_persistence_duration", + Help: "Block write duration in milliseconds.", + Buckets: []float64{ + 500, 1000, 2500, 5000, 7500, + 10000, 25000, 50000, 75000, 100000, + }, + }, + ), } } +// Persist spent [Head]s. +// +//revive:disable-next-line:cognitive-complexity // long but readable. +//revive:disable-next-line:cyclomatic // long but readable. func (p *Persistener[TTask, TShard, TGoShard, THeadBlockWriter, THead]) Persist(heads []THead) (outdatedHeads []THead) { + shouldNotify := false for _, head := range heads { + start := p.clock.Now() if !head.IsReadOnly() { continue } - if record, err := p.catalog.Get(head.ID()); err == nil { - if record.Status() == catalog.StatusPersisted { - if p.persistedHeadIsOutdated(record.UpdatedAt()) { - outdatedHeads = append(outdatedHeads, head) - } - - continue + if record, err := p.catalog.Get(head.ID()); err != nil { + logger.Errorf("[Persistener]: failed get head %s from catalog: %v", head.ID(), err) + } else if record.Status() == catalog.StatusPersisted { + if p.persistedHeadIsOutdated(record.UpdatedAt()) { + outdatedHeads = append(outdatedHeads, head) } + + continue } if p.HeadIsOutdated(head) { @@ -80,19 +105,35 @@ func (p *Persistener[TTask, TShard, TGoShard, THeadBlockWriter, THead]) Persist( continue } - if err := p.flushHead(head); err == nil { - if err = p.persistHead(head); err == nil { - if _, err = p.catalog.SetStatus(head.ID(), catalog.StatusPersisted); err != nil { - logger.Errorf("keeper: set head status in catalog %s: %v", head.ID(), err) - } - } + if err := p.flushHead(head); err != nil { + logger.Errorf("[Persistener]: failed flush head %s: %v", head.ID(), err) + continue } + + if err := p.persistHead(head); err != nil { + logger.Errorf("[Persistener]: failed persist head %s: %v", head.ID(), err) + continue + } + + if _, err := p.catalog.SetStatus(head.ID(), catalog.StatusPersisted); err != nil { + logger.Errorf("[Persistener]: set head status in catalog %s: %v", head.ID(), err) + continue + } + + logger.Infof("[Persistener]: head %s persisted, duration: %v", head.ID(), p.clock.Since(start)) + p.events.With(prometheus.Labels{"type": "persisted"}).Inc() + p.headPersistenceDuration.Observe(float64(p.clock.Since(start).Milliseconds())) + shouldNotify = true + } + + if shouldNotify { + p.writeNotifier.NotifyWritten() } - return + return outdatedHeads } -func (p *Persistener[TTask, TShard, TGoShard, THeadBlockWriter, THead]) headTimeInterval( +func (*Persistener[TTask, TShard, TGoShard, THeadBlockWriter, THead]) headTimeInterval( head THead, ) cppbridge.TimeInterval { timeInterval := cppbridge.NewInvalidTimeInterval() @@ -104,6 +145,7 @@ func (p *Persistener[TTask, TShard, TGoShard, THeadBlockWriter, THead]) headTime return timeInterval } +// HeadIsOutdated check [Head] is outdated. func (p *Persistener[TTask, TShard, TGoShard, THeadBlockWriter, THead]) HeadIsOutdated(head THead) bool { return p.clock.Since(time.UnixMilli(p.headTimeInterval(head).MaxT)) >= p.tsdbRetentionPeriod } @@ -114,7 +156,7 @@ func (p *Persistener[TTask, TShard, TGoShard, THeadBlockWriter, THead]) persiste return p.clock.Since(time.UnixMilli(persistTimeMs)) >= p.retentionPeriod } -func (p *Persistener[TTask, TShard, TGoShard, THeadBlockWriter, THead]) flushHead(head THead) error { +func (*Persistener[TTask, TShard, TGoShard, THeadBlockWriter, THead]) flushHead(head THead) error { for shard := range head.RangeShards() { if err := shard.WalFlush(); err != nil { return err @@ -134,14 +176,11 @@ func (p *Persistener[TTask, TShard, TGoShard, THeadBlockWriter, THead]) persistH return nil } -type Loader[ - TTask Task, - TShard, TGoShard Shard, - THead Head[TTask, TShard, TGoShard], -] interface { - Load(headRecord *catalog.Record, generation uint64) (THead, bool) -} +// +// PersistenerService +// +// PersistenerService service for persist spent [Head]s. type PersistenerService[ TTask Task, TShard, TGoShard Shard, @@ -157,6 +196,7 @@ type PersistenerService[ mediator Mediator } +// NewPersistenerService init new [PersistenerService]. func NewPersistenerService[ TTask Task, TShard, TGoShard Shard, @@ -169,18 +209,22 @@ func NewPersistenerService[ loader TLoader, hcatalog *catalog.Catalog, blockWriter THeadBlockWriter, + writeNotifier WriteNotifier, clock clockwork.Clock, mediator Mediator, tsdbRetentionPeriod time.Duration, retentionPeriod time.Duration, + registerer prometheus.Registerer, ) *PersistenerService[TTask, TShard, TGoShard, THeadBlockWriter, THead, TKeeper, TLoader] { return &PersistenerService[TTask, TShard, TGoShard, THeadBlockWriter, THead, TKeeper, TLoader]{ persistener: NewPersistener[TTask, TShard, TGoShard, THeadBlockWriter, THead]( hcatalog, blockWriter, + writeNotifier, clock, tsdbRetentionPeriod, retentionPeriod, + registerer, ), keeper: hkeeper, loader: loader, @@ -189,6 +233,7 @@ func NewPersistenerService[ } } +// Run starts the [PersistenerService]. func (pg *PersistenerService[TTask, TShard, TGoShard, THeadBlockWriter, THead, TKeeper, TLoader]) Run() { logger.Infof("The PersistenerService is running.") @@ -199,6 +244,7 @@ func (pg *PersistenerService[TTask, TShard, TGoShard, THeadBlockWriter, THead, T logger.Infof("The PersistenerService stopped.") } +// ProcessHeads process persist [Head]s. func (pg *PersistenerService[TTask, TShard, TGoShard, THeadBlockWriter, THead, TKeeper, TLoader]) ProcessHeads() { heads := pg.keeper.Heads() pg.persistHeads(heads) diff --git a/pp/go/storage/head/services/persistener_test.go b/pp/go/storage/head/services/persistener_test.go index 21c10d5df7..d1e2521648 100644 --- a/pp/go/storage/head/services/persistener_test.go +++ b/pp/go/storage/head/services/persistener_test.go @@ -34,12 +34,12 @@ const ( type GenericPersistenceSuite struct { suite.Suite - dataDir string - clock *clockwork.FakeClock - catalog *catalog.Catalog - head *storage.HeadOnDisk - keeper *keeper.Keeper[*storage.HeadOnDisk] - blockWriter *mock.HeadBlockWriterMock[*storage.ShardOnDisk] + dataDir string + clock *clockwork.FakeClock + catalog *catalog.Catalog + keeper *keeper.Keeper[storage.HeadOnDisk, *storage.HeadOnDisk] + blockWriter *mock.HeadBlockWriterMock[*storage.ShardOnDisk] + writeNotifier *mock.WriteNotifierMock } func (s *GenericPersistenceSuite) SetupTest() { @@ -47,8 +47,9 @@ func (s *GenericPersistenceSuite) SetupTest() { s.clock = clockwork.NewFakeClockAt(time.UnixMilli(0)) s.createCatalog() - s.keeper = keeper.NewKeeper[*storage.HeadOnDisk](1) + s.keeper = keeper.NewKeeper[storage.HeadOnDisk](1) s.blockWriter = &mock.HeadBlockWriterMock[*storage.ShardOnDisk]{} + s.writeNotifier = &mock.WriteNotifierMock{NotifyWrittenFunc: func() {}} } func (s *GenericPersistenceSuite) createDataDirectory() string { @@ -107,7 +108,7 @@ func (s *PersistenerSuite) SetupTest() { *storage.PerGoroutineShard, *mock.HeadBlockWriterMock[*storage.ShardOnDisk], *storage.HeadOnDisk, - ](s.catalog, s.blockWriter, s.clock, tsdbRetentionPeriod, retentionPeriod) + ](s.catalog, s.blockWriter, s.writeNotifier, s.clock, tsdbRetentionPeriod, retentionPeriod, nil) } func TestPersistenerSuite(t *testing.T) { @@ -122,7 +123,7 @@ func (s *PersistenerSuite) TestNoHeads() { // Assert s.Equal([]*storage.HeadOnDisk(nil), outdated) - s.Equal(0, len(s.blockWriter.WriteCalls())) + s.Empty(s.blockWriter.WriteCalls()) } func (s *PersistenerSuite) TestNoPersistWritableHead() { @@ -134,7 +135,7 @@ func (s *PersistenerSuite) TestNoPersistWritableHead() { // Assert s.Equal([]*storage.HeadOnDisk(nil), outdated) - s.Equal(0, len(s.blockWriter.WriteCalls())) + s.Empty(s.blockWriter.WriteCalls()) } func (s *PersistenerSuite) TestNoPersistPersistedHead() { @@ -159,7 +160,7 @@ func (s *PersistenerSuite) TestNoPersistPersistedHead() { // Assert s.Equal([]*storage.HeadOnDisk(nil), outdated) - s.Equal(0, len(s.blockWriter.WriteCalls())) + s.Empty(s.blockWriter.WriteCalls()) } func (s *PersistenerSuite) TestOutdatedPersistedHead() { @@ -184,7 +185,7 @@ func (s *PersistenerSuite) TestOutdatedPersistedHead() { // Assert s.Equal([]*storage.HeadOnDisk{head}, outdated) - s.Equal(0, len(s.blockWriter.WriteCalls())) + s.Empty(s.blockWriter.WriteCalls()) } func (s *PersistenerSuite) TestOutdatedHead() { @@ -207,7 +208,7 @@ func (s *PersistenerSuite) TestOutdatedHead() { // Assert s.Equal([]*storage.HeadOnDisk{head}, outdated) - s.Equal(0, len(s.blockWriter.WriteCalls())) + s.Empty(s.blockWriter.WriteCalls()) } func (s *PersistenerSuite) TestPersistHeadSuccess() { @@ -240,7 +241,8 @@ func (s *PersistenerSuite) TestPersistHeadSuccess() { // Assert s.Equal([]*storage.HeadOnDisk(nil), outdated) - s.Equal(2, len(s.blockWriter.WriteCalls())) + s.Len(s.blockWriter.WriteCalls(), 2) + s.Len(s.writeNotifier.NotifyWrittenCalls(), 1) s.Require().NoError(err) s.Equal(catalog.StatusPersisted, record.Status()) } @@ -279,7 +281,8 @@ func (s *PersistenerSuite) TestPersistHeadErrorOnBlockWriterForSecondShard() { // Assert s.Equal([]*storage.HeadOnDisk(nil), outdated) - s.Equal(2, len(s.blockWriter.WriteCalls())) + s.Len(s.blockWriter.WriteCalls(), 2) + s.Empty(s.writeNotifier.NotifyWrittenCalls()) s.Require().NoError(err) s.Equal(catalog.StatusNew, record.Status()) } @@ -293,7 +296,7 @@ type PersistenerServiceSuite struct { *storage.PerGoroutineShard, *mock.HeadBlockWriterMock[*storage.ShardOnDisk], *storage.HeadOnDisk, - *keeper.Keeper[*storage.HeadOnDisk], + *keeper.Keeper[storage.HeadOnDisk, *storage.HeadOnDisk], *storage.Loader, ] } @@ -308,9 +311,20 @@ func (s *PersistenerServiceSuite) SetupTest() { *storage.PerGoroutineShard, *mock.HeadBlockWriterMock[*storage.ShardOnDisk], *storage.HeadOnDisk, - *keeper.Keeper[*storage.HeadOnDisk], + *keeper.Keeper[storage.HeadOnDisk, *storage.HeadOnDisk], *storage.Loader, - ](s.keeper, s.loader, s.catalog, s.blockWriter, s.clock, nil, tsdbRetentionPeriod, retentionPeriod) + ]( + s.keeper, + s.loader, + s.catalog, + s.blockWriter, + s.writeNotifier, + s.clock, + nil, + tsdbRetentionPeriod, + retentionPeriod, + nil, + ) } func TestPersistenerServiceSuite(t *testing.T) { @@ -337,7 +351,7 @@ func (s *PersistenerServiceSuite) TestRemoveOutdatedHeadFromKeeper() { s.service.ProcessHeads() // Assert - s.Equal(0, len(s.keeper.Heads())) + s.Empty(s.keeper.Heads()) s.Equal(catalog.StatusRotated, record.Status()) } @@ -359,7 +373,7 @@ func (s *PersistenerServiceSuite) TestLoadHeadsInKeeper() { s.service.ProcessHeads() // Assert - s.Require().Equal(1, len(s.keeper.Heads())) + s.Require().Len(s.keeper.Heads(), 1) s.Equal(head.ID(), s.keeper.Heads()[0].ID()) s.Equal(int64(0), record.CreatedAt()) } @@ -382,6 +396,6 @@ func (s *PersistenerServiceSuite) TestHeadAlreadyExistsInKeeper() { s.service.ProcessHeads() // Assert - s.Require().Equal(1, len(s.keeper.Heads())) + s.Require().Len(s.keeper.Heads(), 1) s.Equal(head.ID(), s.keeper.Heads()[0].ID()) } diff --git a/pp/go/storage/head/services/rotator.go b/pp/go/storage/head/services/rotator.go index e344b88762..816bf5bf8a 100644 --- a/pp/go/storage/head/services/rotator.go +++ b/pp/go/storage/head/services/rotator.go @@ -3,6 +3,7 @@ package services import ( "context" "fmt" + "time" "github.com/prometheus/client_golang/prometheus" "github.com/prometheus/prometheus/pp/go/storage/logger" @@ -30,12 +31,17 @@ type Rotator[ TShard, TGoShard Shard, THead Head[TTask, TShard, TGoShard], ] struct { - proxyHead ProxyHead[TTask, TShard, TGoShard, THead] - headBuilder HeadBuilder[TTask, TShard, TGoShard, THead] - m Mediator - cfg RotatorConfig - headInformer HeadInformer - rotateCounter prometheus.Counter + proxyHead ProxyHead[TTask, TShard, TGoShard, THead] + headBuilder HeadBuilder[TTask, TShard, TGoShard, THead] + m Mediator + cfg RotatorConfig + headInformer HeadInformer + + // stat + rotateCounter prometheus.Counter + events *prometheus.CounterVec + waitLockRotateDuration prometheus.Gauge + rotationDuration prometheus.Gauge } // NewRotator init new [Rotator]. @@ -49,9 +55,9 @@ func NewRotator[ m Mediator, cfg RotatorConfig, headInformer HeadInformer, - r prometheus.Registerer, + registerer prometheus.Registerer, ) *Rotator[TTask, TShard, TGoShard, THead] { - factory := util.NewUnconflictRegisterer(r) + factory := util.NewUnconflictRegisterer(registerer) return &Rotator[TTask, TShard, TGoShard, THead]{ proxyHead: proxyHead, headBuilder: headBuilder, @@ -64,6 +70,25 @@ func NewRotator[ Help: "Total counter of rotate rotatable object.", }, ), + events: factory.NewCounterVec( + prometheus.CounterOpts{ + Name: "prompp_head_event_count", + Help: "Number of head events", + }, + []string{"type"}, + ), + waitLockRotateDuration: factory.NewGauge( + prometheus.GaugeOpts{ + Name: "prompp_rotator_wait_lock_rotate_duration", + Help: "The duration of the lock wait for rotation in nanoseconds", + }, + ), + rotationDuration: factory.NewGauge( + prometheus.GaugeOpts{ + Name: "prompp_rotator_rotate_duration", + Help: "The duration of the rotate in nanoseconds", + }, + ), } } @@ -91,10 +116,8 @@ func (s *Rotator[TTask, TShard, TGoShard, THead]) rotate( ctx context.Context, numberOfShards uint16, ) error { - fmt.Println("rotate") + start := time.Now() oldHead := s.proxyHead.Get() - - fmt.Println("rotate newHead") newHead, err := s.headBuilder.Build(oldHead.Generation()+1, numberOfShards) if err != nil { return fmt.Errorf("failed to build a new head: %w", err) @@ -103,38 +126,39 @@ func (s *Rotator[TTask, TShard, TGoShard, THead]) rotate( // TODO CopySeriesFrom only old nunber of shards == new // newHead.CopySeriesFrom(oldHead) - fmt.Println("rotate AddWithReplace") if err = s.proxyHead.AddWithReplace(oldHead, s.headInformer.CreatedAt(oldHead.ID())); err != nil { return fmt.Errorf("failed add to keeper old head: %w", err) } + startWait := time.Now() // TODO if replace error? - fmt.Println("rotate Replace") if err = s.proxyHead.Replace(ctx, newHead); err != nil { + if errClose := newHead.Close(); errClose != nil { + logger.Errorf("failed close new head: %s : %v", newHead.ID(), errClose) + } + return fmt.Errorf("failed to replace old to new head: %w", err) } + s.waitLockRotateDuration.Set(float64(time.Since(startWait).Nanoseconds())) - fmt.Println("rotate SetActiveStatus") if err = s.headInformer.SetActiveStatus(newHead.ID()); err != nil { logger.Warnf("failed set status active for head{%s}: %s", newHead.ID(), err) } - fmt.Println("rotate MergeOutOfOrderChunksWithHead") if err = MergeOutOfOrderChunksWithHead(oldHead); err != nil { logger.Warnf("failed merge out of order chunks in data storage: %s", err) } - fmt.Println("rotate CFSViaRange") if err = CFSViaRange(oldHead); err != nil { logger.Warnf("failed commit and flush to wal: %s", err) } - fmt.Println("rotate SetRotatedStatus") if err = s.headInformer.SetRotatedStatus(oldHead.ID()); err != nil { logger.Warnf("failed set status rotated for head{%s}: %s", oldHead.ID(), err) } - fmt.Println("rotate SetReadOnly") oldHead.SetReadOnly() + s.events.With(prometheus.Labels{"type": "rotated"}).Inc() + s.rotationDuration.Set(float64(time.Since(start).Nanoseconds())) return nil } diff --git a/pp/go/storage/head/shard/wal/wal.go b/pp/go/storage/head/shard/wal/wal.go index 37c2651937..5b8b2ed45e 100644 --- a/pp/go/storage/head/shard/wal/wal.go +++ b/pp/go/storage/head/shard/wal/wal.go @@ -9,14 +9,16 @@ import ( "github.com/prometheus/prometheus/pp/go/cppbridge" ) +//go:generate -command moq go run github.com/matryer/moq --rm --skip-ensure --pkg wal_test --out +//go:generate moq wal_moq_test.go . SegmentWriter Encoder StatsSegment EncodedSegment + const ( // FileFormatVersion wal file version. FileFormatVersion = 1 ) -var ( - ErrWalIsCorrupted = errors.New("wal is corrupted") -) +// ErrWalIsCorrupted errror when wal is corrupted. +var ErrWalIsCorrupted = errors.New("wal is corrupted") // SegmentWriter writer for wal segments. type SegmentWriter[TSegment EncodedSegment] interface { diff --git a/pp/go/storage/head/shard/wal/wal_moq_test.go b/pp/go/storage/head/shard/wal/wal_moq_test.go index 924a724449..7012bb7246 100644 --- a/pp/go/storage/head/shard/wal/wal_moq_test.go +++ b/pp/go/storage/head/shard/wal/wal_moq_test.go @@ -10,290 +10,6 @@ import ( "sync" ) -// Ensure, that ReadSegmentMock does implement wal.ReadSegment. -// If this is not the case, regenerate this file with moq. -var _ wal.ReadSegment = &ReadSegmentMock{} - -// ReadSegmentMock is a mock implementation of wal.ReadSegment. -// -// func TestSomethingThatUsesReadSegment(t *testing.T) { -// -// // make and configure a mocked wal.ReadSegment -// mockedReadSegment := &ReadSegmentMock{ -// ReadFromFunc: func(r io.Reader) (int64, error) { -// panic("mock out the ReadFrom method") -// }, -// ResetFunc: func() { -// panic("mock out the Reset method") -// }, -// } -// -// // use mockedReadSegment in code that requires wal.ReadSegment -// // and then make assertions. -// -// } -type ReadSegmentMock struct { - // ReadFromFunc mocks the ReadFrom method. - ReadFromFunc func(r io.Reader) (int64, error) - - // ResetFunc mocks the Reset method. - ResetFunc func() - - // calls tracks calls to the methods. - calls struct { - // ReadFrom holds details about calls to the ReadFrom method. - ReadFrom []struct { - // R is the r argument value. - R io.Reader - } - // Reset holds details about calls to the Reset method. - Reset []struct { - } - } - lockReadFrom sync.RWMutex - lockReset sync.RWMutex -} - -// ReadFrom calls ReadFromFunc. -func (mock *ReadSegmentMock) ReadFrom(r io.Reader) (int64, error) { - if mock.ReadFromFunc == nil { - panic("ReadSegmentMock.ReadFromFunc: method is nil but ReadSegment.ReadFrom was just called") - } - callInfo := struct { - R io.Reader - }{ - R: r, - } - mock.lockReadFrom.Lock() - mock.calls.ReadFrom = append(mock.calls.ReadFrom, callInfo) - mock.lockReadFrom.Unlock() - return mock.ReadFromFunc(r) -} - -// ReadFromCalls gets all the calls that were made to ReadFrom. -// Check the length with: -// -// len(mockedReadSegment.ReadFromCalls()) -func (mock *ReadSegmentMock) ReadFromCalls() []struct { - R io.Reader -} { - var calls []struct { - R io.Reader - } - mock.lockReadFrom.RLock() - calls = mock.calls.ReadFrom - mock.lockReadFrom.RUnlock() - return calls -} - -// Reset calls ResetFunc. -func (mock *ReadSegmentMock) Reset() { - if mock.ResetFunc == nil { - panic("ReadSegmentMock.ResetFunc: method is nil but ReadSegment.Reset was just called") - } - callInfo := struct { - }{} - mock.lockReset.Lock() - mock.calls.Reset = append(mock.calls.Reset, callInfo) - mock.lockReset.Unlock() - mock.ResetFunc() -} - -// ResetCalls gets all the calls that were made to Reset. -// Check the length with: -// -// len(mockedReadSegment.ResetCalls()) -func (mock *ReadSegmentMock) ResetCalls() []struct { -} { - var calls []struct { - } - mock.lockReset.RLock() - calls = mock.calls.Reset - mock.lockReset.RUnlock() - return calls -} - -// Ensure, that EncodedSegmentMock does implement wal.EncodedSegment. -// If this is not the case, regenerate this file with moq. -var _ wal.EncodedSegment = &EncodedSegmentMock{} - -// EncodedSegmentMock is a mock implementation of wal.EncodedSegment. -// -// func TestSomethingThatUsesEncodedSegment(t *testing.T) { -// -// // make and configure a mocked wal.EncodedSegment -// mockedEncodedSegment := &EncodedSegmentMock{ -// CRC32Func: func() uint32 { -// panic("mock out the CRC32 method") -// }, -// SamplesFunc: func() uint32 { -// panic("mock out the Samples method") -// }, -// SizeFunc: func() int64 { -// panic("mock out the Size method") -// }, -// WriteToFunc: func(w io.Writer) (int64, error) { -// panic("mock out the WriteTo method") -// }, -// } -// -// // use mockedEncodedSegment in code that requires wal.EncodedSegment -// // and then make assertions. -// -// } -type EncodedSegmentMock struct { - // CRC32Func mocks the CRC32 method. - CRC32Func func() uint32 - - // SamplesFunc mocks the Samples method. - SamplesFunc func() uint32 - - // SizeFunc mocks the Size method. - SizeFunc func() int64 - - // WriteToFunc mocks the WriteTo method. - WriteToFunc func(w io.Writer) (int64, error) - - // calls tracks calls to the methods. - calls struct { - // CRC32 holds details about calls to the CRC32 method. - CRC32 []struct { - } - // Samples holds details about calls to the Samples method. - Samples []struct { - } - // Size holds details about calls to the Size method. - Size []struct { - } - // WriteTo holds details about calls to the WriteTo method. - WriteTo []struct { - // W is the w argument value. - W io.Writer - } - } - lockCRC32 sync.RWMutex - lockSamples sync.RWMutex - lockSize sync.RWMutex - lockWriteTo sync.RWMutex -} - -// CRC32 calls CRC32Func. -func (mock *EncodedSegmentMock) CRC32() uint32 { - if mock.CRC32Func == nil { - panic("EncodedSegmentMock.CRC32Func: method is nil but EncodedSegment.CRC32 was just called") - } - callInfo := struct { - }{} - mock.lockCRC32.Lock() - mock.calls.CRC32 = append(mock.calls.CRC32, callInfo) - mock.lockCRC32.Unlock() - return mock.CRC32Func() -} - -// CRC32Calls gets all the calls that were made to CRC32. -// Check the length with: -// -// len(mockedEncodedSegment.CRC32Calls()) -func (mock *EncodedSegmentMock) CRC32Calls() []struct { -} { - var calls []struct { - } - mock.lockCRC32.RLock() - calls = mock.calls.CRC32 - mock.lockCRC32.RUnlock() - return calls -} - -// Samples calls SamplesFunc. -func (mock *EncodedSegmentMock) Samples() uint32 { - if mock.SamplesFunc == nil { - panic("EncodedSegmentMock.SamplesFunc: method is nil but EncodedSegment.Samples was just called") - } - callInfo := struct { - }{} - mock.lockSamples.Lock() - mock.calls.Samples = append(mock.calls.Samples, callInfo) - mock.lockSamples.Unlock() - return mock.SamplesFunc() -} - -// SamplesCalls gets all the calls that were made to Samples. -// Check the length with: -// -// len(mockedEncodedSegment.SamplesCalls()) -func (mock *EncodedSegmentMock) SamplesCalls() []struct { -} { - var calls []struct { - } - mock.lockSamples.RLock() - calls = mock.calls.Samples - mock.lockSamples.RUnlock() - return calls -} - -// Size calls SizeFunc. -func (mock *EncodedSegmentMock) Size() int64 { - if mock.SizeFunc == nil { - panic("EncodedSegmentMock.SizeFunc: method is nil but EncodedSegment.Size was just called") - } - callInfo := struct { - }{} - mock.lockSize.Lock() - mock.calls.Size = append(mock.calls.Size, callInfo) - mock.lockSize.Unlock() - return mock.SizeFunc() -} - -// SizeCalls gets all the calls that were made to Size. -// Check the length with: -// -// len(mockedEncodedSegment.SizeCalls()) -func (mock *EncodedSegmentMock) SizeCalls() []struct { -} { - var calls []struct { - } - mock.lockSize.RLock() - calls = mock.calls.Size - mock.lockSize.RUnlock() - return calls -} - -// WriteTo calls WriteToFunc. -func (mock *EncodedSegmentMock) WriteTo(w io.Writer) (int64, error) { - if mock.WriteToFunc == nil { - panic("EncodedSegmentMock.WriteToFunc: method is nil but EncodedSegment.WriteTo was just called") - } - callInfo := struct { - W io.Writer - }{ - W: w, - } - mock.lockWriteTo.Lock() - mock.calls.WriteTo = append(mock.calls.WriteTo, callInfo) - mock.lockWriteTo.Unlock() - return mock.WriteToFunc(w) -} - -// WriteToCalls gets all the calls that were made to WriteTo. -// Check the length with: -// -// len(mockedEncodedSegment.WriteToCalls()) -func (mock *EncodedSegmentMock) WriteToCalls() []struct { - W io.Writer -} { - var calls []struct { - W io.Writer - } - mock.lockWriteTo.RLock() - calls = mock.calls.WriteTo - mock.lockWriteTo.RUnlock() - return calls -} - -// Ensure, that SegmentWriterMock does implement wal.SegmentWriter. -// If this is not the case, regenerate this file with moq. -var _ wal.SegmentWriter[wal.EncodedSegment] = &SegmentWriterMock[wal.EncodedSegment]{} - // SegmentWriterMock is a mock implementation of wal.SegmentWriter. // // func TestSomethingThatUsesSegmentWriter(t *testing.T) { @@ -504,10 +220,6 @@ func (mock *SegmentWriterMock[TSegment]) WriteCalls() []struct { return calls } -// Ensure, that EncoderMock does implement wal.Encoder. -// If this is not the case, regenerate this file with moq. -var _ wal.Encoder[wal.EncodedSegment, wal.StatsSegment] = &EncoderMock[wal.EncodedSegment, wal.StatsSegment]{} - // EncoderMock is a mock implementation of wal.Encoder. // // func TestSomethingThatUsesEncoder(t *testing.T) { @@ -607,10 +319,6 @@ func (mock *EncoderMock[TSegment, TStats]) FinalizeCalls() []struct { return calls } -// Ensure, that StatsSegmentMock does implement wal.StatsSegment. -// If this is not the case, regenerate this file with moq. -var _ wal.StatsSegment = &StatsSegmentMock{} - // StatsSegmentMock is a mock implementation of wal.StatsSegment. // // func TestSomethingThatUsesStatsSegment(t *testing.T) { @@ -665,3 +373,176 @@ func (mock *StatsSegmentMock) SamplesCalls() []struct { mock.lockSamples.RUnlock() return calls } + +// EncodedSegmentMock is a mock implementation of wal.EncodedSegment. +// +// func TestSomethingThatUsesEncodedSegment(t *testing.T) { +// +// // make and configure a mocked wal.EncodedSegment +// mockedEncodedSegment := &EncodedSegmentMock{ +// CRC32Func: func() uint32 { +// panic("mock out the CRC32 method") +// }, +// SamplesFunc: func() uint32 { +// panic("mock out the Samples method") +// }, +// SizeFunc: func() int64 { +// panic("mock out the Size method") +// }, +// WriteToFunc: func(w io.Writer) (int64, error) { +// panic("mock out the WriteTo method") +// }, +// } +// +// // use mockedEncodedSegment in code that requires wal.EncodedSegment +// // and then make assertions. +// +// } +type EncodedSegmentMock struct { + // CRC32Func mocks the CRC32 method. + CRC32Func func() uint32 + + // SamplesFunc mocks the Samples method. + SamplesFunc func() uint32 + + // SizeFunc mocks the Size method. + SizeFunc func() int64 + + // WriteToFunc mocks the WriteTo method. + WriteToFunc func(w io.Writer) (int64, error) + + // calls tracks calls to the methods. + calls struct { + // CRC32 holds details about calls to the CRC32 method. + CRC32 []struct { + } + // Samples holds details about calls to the Samples method. + Samples []struct { + } + // Size holds details about calls to the Size method. + Size []struct { + } + // WriteTo holds details about calls to the WriteTo method. + WriteTo []struct { + // W is the w argument value. + W io.Writer + } + } + lockCRC32 sync.RWMutex + lockSamples sync.RWMutex + lockSize sync.RWMutex + lockWriteTo sync.RWMutex +} + +// CRC32 calls CRC32Func. +func (mock *EncodedSegmentMock) CRC32() uint32 { + if mock.CRC32Func == nil { + panic("EncodedSegmentMock.CRC32Func: method is nil but EncodedSegment.CRC32 was just called") + } + callInfo := struct { + }{} + mock.lockCRC32.Lock() + mock.calls.CRC32 = append(mock.calls.CRC32, callInfo) + mock.lockCRC32.Unlock() + return mock.CRC32Func() +} + +// CRC32Calls gets all the calls that were made to CRC32. +// Check the length with: +// +// len(mockedEncodedSegment.CRC32Calls()) +func (mock *EncodedSegmentMock) CRC32Calls() []struct { +} { + var calls []struct { + } + mock.lockCRC32.RLock() + calls = mock.calls.CRC32 + mock.lockCRC32.RUnlock() + return calls +} + +// Samples calls SamplesFunc. +func (mock *EncodedSegmentMock) Samples() uint32 { + if mock.SamplesFunc == nil { + panic("EncodedSegmentMock.SamplesFunc: method is nil but EncodedSegment.Samples was just called") + } + callInfo := struct { + }{} + mock.lockSamples.Lock() + mock.calls.Samples = append(mock.calls.Samples, callInfo) + mock.lockSamples.Unlock() + return mock.SamplesFunc() +} + +// SamplesCalls gets all the calls that were made to Samples. +// Check the length with: +// +// len(mockedEncodedSegment.SamplesCalls()) +func (mock *EncodedSegmentMock) SamplesCalls() []struct { +} { + var calls []struct { + } + mock.lockSamples.RLock() + calls = mock.calls.Samples + mock.lockSamples.RUnlock() + return calls +} + +// Size calls SizeFunc. +func (mock *EncodedSegmentMock) Size() int64 { + if mock.SizeFunc == nil { + panic("EncodedSegmentMock.SizeFunc: method is nil but EncodedSegment.Size was just called") + } + callInfo := struct { + }{} + mock.lockSize.Lock() + mock.calls.Size = append(mock.calls.Size, callInfo) + mock.lockSize.Unlock() + return mock.SizeFunc() +} + +// SizeCalls gets all the calls that were made to Size. +// Check the length with: +// +// len(mockedEncodedSegment.SizeCalls()) +func (mock *EncodedSegmentMock) SizeCalls() []struct { +} { + var calls []struct { + } + mock.lockSize.RLock() + calls = mock.calls.Size + mock.lockSize.RUnlock() + return calls +} + +// WriteTo calls WriteToFunc. +func (mock *EncodedSegmentMock) WriteTo(w io.Writer) (int64, error) { + if mock.WriteToFunc == nil { + panic("EncodedSegmentMock.WriteToFunc: method is nil but EncodedSegment.WriteTo was just called") + } + callInfo := struct { + W io.Writer + }{ + W: w, + } + mock.lockWriteTo.Lock() + mock.calls.WriteTo = append(mock.calls.WriteTo, callInfo) + mock.lockWriteTo.Unlock() + return mock.WriteToFunc(w) +} + +// WriteToCalls gets all the calls that were made to WriteTo. +// Check the length with: +// +// len(mockedEncodedSegment.WriteToCalls()) +func (mock *EncodedSegmentMock) WriteToCalls() []struct { + W io.Writer +} { + var calls []struct { + W io.Writer + } + mock.lockWriteTo.RLock() + calls = mock.calls.WriteTo + mock.lockWriteTo.RUnlock() + return calls +} diff --git a/pp/go/storage/head/shard/wal/wal_reader.go b/pp/go/storage/head/shard/wal/wal_reader.go index 2aa637fe06..d4a1f31caa 100644 --- a/pp/go/storage/head/shard/wal/wal_reader.go +++ b/pp/go/storage/head/shard/wal/wal_reader.go @@ -7,6 +7,9 @@ import ( "io" ) +//go:generate -command moq go run github.com/matryer/moq --rm --skip-ensure --pkg wal_test --out +//go:generate moq wal_reader_moq_test.go . ReadSegment + // ReadSegment the minimum required [Segment] implementation for a [Wal]. type ReadSegment interface { // ReadFrom reads [ReadSegment] data from r [io.Reader]. The return value n is the number of bytes read. diff --git a/pp/go/storage/head/shard/wal/wal_reader_moq_test.go b/pp/go/storage/head/shard/wal/wal_reader_moq_test.go new file mode 100644 index 0000000000..4343b2e695 --- /dev/null +++ b/pp/go/storage/head/shard/wal/wal_reader_moq_test.go @@ -0,0 +1,108 @@ +// Code generated by moq; DO NOT EDIT. +// github.com/matryer/moq + +package wal_test + +import ( + "io" + "sync" +) + +// ReadSegmentMock is a mock implementation of wal.ReadSegment. +// +// func TestSomethingThatUsesReadSegment(t *testing.T) { +// +// // make and configure a mocked wal.ReadSegment +// mockedReadSegment := &ReadSegmentMock{ +// ReadFromFunc: func(r io.Reader) (int64, error) { +// panic("mock out the ReadFrom method") +// }, +// ResetFunc: func() { +// panic("mock out the Reset method") +// }, +// } +// +// // use mockedReadSegment in code that requires wal.ReadSegment +// // and then make assertions. +// +// } +type ReadSegmentMock struct { + // ReadFromFunc mocks the ReadFrom method. + ReadFromFunc func(r io.Reader) (int64, error) + + // ResetFunc mocks the Reset method. + ResetFunc func() + + // calls tracks calls to the methods. + calls struct { + // ReadFrom holds details about calls to the ReadFrom method. + ReadFrom []struct { + // R is the r argument value. + R io.Reader + } + // Reset holds details about calls to the Reset method. + Reset []struct { + } + } + lockReadFrom sync.RWMutex + lockReset sync.RWMutex +} + +// ReadFrom calls ReadFromFunc. +func (mock *ReadSegmentMock) ReadFrom(r io.Reader) (int64, error) { + if mock.ReadFromFunc == nil { + panic("ReadSegmentMock.ReadFromFunc: method is nil but ReadSegment.ReadFrom was just called") + } + callInfo := struct { + R io.Reader + }{ + R: r, + } + mock.lockReadFrom.Lock() + mock.calls.ReadFrom = append(mock.calls.ReadFrom, callInfo) + mock.lockReadFrom.Unlock() + return mock.ReadFromFunc(r) +} + +// ReadFromCalls gets all the calls that were made to ReadFrom. +// Check the length with: +// +// len(mockedReadSegment.ReadFromCalls()) +func (mock *ReadSegmentMock) ReadFromCalls() []struct { + R io.Reader +} { + var calls []struct { + R io.Reader + } + mock.lockReadFrom.RLock() + calls = mock.calls.ReadFrom + mock.lockReadFrom.RUnlock() + return calls +} + +// Reset calls ResetFunc. +func (mock *ReadSegmentMock) Reset() { + if mock.ResetFunc == nil { + panic("ReadSegmentMock.ResetFunc: method is nil but ReadSegment.Reset was just called") + } + callInfo := struct { + }{} + mock.lockReset.Lock() + mock.calls.Reset = append(mock.calls.Reset, callInfo) + mock.lockReset.Unlock() + mock.ResetFunc() +} + +// ResetCalls gets all the calls that were made to Reset. +// Check the length with: +// +// len(mockedReadSegment.ResetCalls()) +func (mock *ReadSegmentMock) ResetCalls() []struct { +} { + var calls []struct { + } + mock.lockReset.RLock() + calls = mock.calls.Reset + mock.lockReset.RUnlock() + return calls +} diff --git a/pp/go/storage/head/shard/wal/wal_test.go b/pp/go/storage/head/shard/wal/wal_test.go index e6e4fc00c5..551c987078 100644 --- a/pp/go/storage/head/shard/wal/wal_test.go +++ b/pp/go/storage/head/shard/wal/wal_test.go @@ -14,8 +14,6 @@ import ( "github.com/prometheus/prometheus/pp/go/storage/head/shard/wal/writer" ) -// TODO moq -out wal_moq_test.go -pkg wal_test -rm . ReadSegment EncodedSegment SegmentWriter Encoder StatsSegment - func TestXxx(t *testing.T) { shardID := uint16(0) tmpDir, err := os.MkdirTemp("", "shard") diff --git a/pp/go/storage/head/shard/wal/writer/buffered.go b/pp/go/storage/head/shard/wal/writer/buffered.go index a11dbc8459..5f06c2b7d7 100644 --- a/pp/go/storage/head/shard/wal/writer/buffered.go +++ b/pp/go/storage/head/shard/wal/writer/buffered.go @@ -8,6 +8,9 @@ import ( "sync/atomic" ) +//go:generate -command moq go run github.com/matryer/moq --rm --skip-ensure --pkg writer_test --out +//go:generate moq buffered_moq_test.go . FileInfo SegmentIsWrittenNotifier WriteSyncCloser + // FileInfo alias for [os.FileInfo]. type FileInfo = os.FileInfo diff --git a/pp/go/storage/head/shard/wal/writer/writer_moq_test.go b/pp/go/storage/head/shard/wal/writer/buffered_moq_test.go similarity index 70% rename from pp/go/storage/head/shard/wal/writer/writer_moq_test.go rename to pp/go/storage/head/shard/wal/writer/buffered_moq_test.go index 7c835b7f33..5ccb2ec06b 100644 --- a/pp/go/storage/head/shard/wal/writer/writer_moq_test.go +++ b/pp/go/storage/head/shard/wal/writer/buffered_moq_test.go @@ -5,134 +5,201 @@ package writer_test import ( "github.com/prometheus/prometheus/pp/go/storage/head/shard/wal/writer" - "io" "io/fs" "sync" "time" ) -// Ensure, that EncodedSegmentMock does implement writer.EncodedSegment. -// If this is not the case, regenerate this file with moq. -var _ writer.EncodedSegment = &EncodedSegmentMock{} - -// EncodedSegmentMock is a mock implementation of writer.EncodedSegment. +// FileInfoMock is a mock implementation of writer.FileInfo. // -// func TestSomethingThatUsesEncodedSegment(t *testing.T) { +// func TestSomethingThatUsesFileInfo(t *testing.T) { // -// // make and configure a mocked writer.EncodedSegment -// mockedEncodedSegment := &EncodedSegmentMock{ -// CRC32Func: func() uint32 { -// panic("mock out the CRC32 method") +// // make and configure a mocked writer.FileInfo +// mockedFileInfo := &FileInfoMock{ +// IsDirFunc: func() bool { +// panic("mock out the IsDir method") +// }, +// ModTimeFunc: func() time.Time { +// panic("mock out the ModTime method") +// }, +// ModeFunc: func() fs.FileMode { +// panic("mock out the Mode method") // }, -// SamplesFunc: func() uint32 { -// panic("mock out the Samples method") +// NameFunc: func() string { +// panic("mock out the Name method") // }, // SizeFunc: func() int64 { // panic("mock out the Size method") // }, -// WriteToFunc: func(w io.Writer) (int64, error) { -// panic("mock out the WriteTo method") +// SysFunc: func() any { +// panic("mock out the Sys method") // }, // } // -// // use mockedEncodedSegment in code that requires writer.EncodedSegment +// // use mockedFileInfo in code that requires writer.FileInfo // // and then make assertions. // // } -type EncodedSegmentMock struct { - // CRC32Func mocks the CRC32 method. - CRC32Func func() uint32 +type FileInfoMock struct { + // IsDirFunc mocks the IsDir method. + IsDirFunc func() bool + + // ModTimeFunc mocks the ModTime method. + ModTimeFunc func() time.Time - // SamplesFunc mocks the Samples method. - SamplesFunc func() uint32 + // ModeFunc mocks the Mode method. + ModeFunc func() fs.FileMode + + // NameFunc mocks the Name method. + NameFunc func() string // SizeFunc mocks the Size method. SizeFunc func() int64 - // WriteToFunc mocks the WriteTo method. - WriteToFunc func(w io.Writer) (int64, error) + // SysFunc mocks the Sys method. + SysFunc func() any // calls tracks calls to the methods. calls struct { - // CRC32 holds details about calls to the CRC32 method. - CRC32 []struct { + // IsDir holds details about calls to the IsDir method. + IsDir []struct { + } + // ModTime holds details about calls to the ModTime method. + ModTime []struct { } - // Samples holds details about calls to the Samples method. - Samples []struct { + // Mode holds details about calls to the Mode method. + Mode []struct { + } + // Name holds details about calls to the Name method. + Name []struct { } // Size holds details about calls to the Size method. Size []struct { } - // WriteTo holds details about calls to the WriteTo method. - WriteTo []struct { - // W is the w argument value. - W io.Writer + // Sys holds details about calls to the Sys method. + Sys []struct { } } - lockCRC32 sync.RWMutex - lockSamples sync.RWMutex + lockIsDir sync.RWMutex + lockModTime sync.RWMutex + lockMode sync.RWMutex + lockName sync.RWMutex lockSize sync.RWMutex - lockWriteTo sync.RWMutex + lockSys sync.RWMutex +} + +// IsDir calls IsDirFunc. +func (mock *FileInfoMock) IsDir() bool { + if mock.IsDirFunc == nil { + panic("FileInfoMock.IsDirFunc: method is nil but FileInfo.IsDir was just called") + } + callInfo := struct { + }{} + mock.lockIsDir.Lock() + mock.calls.IsDir = append(mock.calls.IsDir, callInfo) + mock.lockIsDir.Unlock() + return mock.IsDirFunc() +} + +// IsDirCalls gets all the calls that were made to IsDir. +// Check the length with: +// +// len(mockedFileInfo.IsDirCalls()) +func (mock *FileInfoMock) IsDirCalls() []struct { +} { + var calls []struct { + } + mock.lockIsDir.RLock() + calls = mock.calls.IsDir + mock.lockIsDir.RUnlock() + return calls +} + +// ModTime calls ModTimeFunc. +func (mock *FileInfoMock) ModTime() time.Time { + if mock.ModTimeFunc == nil { + panic("FileInfoMock.ModTimeFunc: method is nil but FileInfo.ModTime was just called") + } + callInfo := struct { + }{} + mock.lockModTime.Lock() + mock.calls.ModTime = append(mock.calls.ModTime, callInfo) + mock.lockModTime.Unlock() + return mock.ModTimeFunc() +} + +// ModTimeCalls gets all the calls that were made to ModTime. +// Check the length with: +// +// len(mockedFileInfo.ModTimeCalls()) +func (mock *FileInfoMock) ModTimeCalls() []struct { +} { + var calls []struct { + } + mock.lockModTime.RLock() + calls = mock.calls.ModTime + mock.lockModTime.RUnlock() + return calls } -// CRC32 calls CRC32Func. -func (mock *EncodedSegmentMock) CRC32() uint32 { - if mock.CRC32Func == nil { - panic("EncodedSegmentMock.CRC32Func: method is nil but EncodedSegment.CRC32 was just called") +// Mode calls ModeFunc. +func (mock *FileInfoMock) Mode() fs.FileMode { + if mock.ModeFunc == nil { + panic("FileInfoMock.ModeFunc: method is nil but FileInfo.Mode was just called") } callInfo := struct { }{} - mock.lockCRC32.Lock() - mock.calls.CRC32 = append(mock.calls.CRC32, callInfo) - mock.lockCRC32.Unlock() - return mock.CRC32Func() + mock.lockMode.Lock() + mock.calls.Mode = append(mock.calls.Mode, callInfo) + mock.lockMode.Unlock() + return mock.ModeFunc() } -// CRC32Calls gets all the calls that were made to CRC32. +// ModeCalls gets all the calls that were made to Mode. // Check the length with: // -// len(mockedEncodedSegment.CRC32Calls()) -func (mock *EncodedSegmentMock) CRC32Calls() []struct { +// len(mockedFileInfo.ModeCalls()) +func (mock *FileInfoMock) ModeCalls() []struct { } { var calls []struct { } - mock.lockCRC32.RLock() - calls = mock.calls.CRC32 - mock.lockCRC32.RUnlock() + mock.lockMode.RLock() + calls = mock.calls.Mode + mock.lockMode.RUnlock() return calls } -// Samples calls SamplesFunc. -func (mock *EncodedSegmentMock) Samples() uint32 { - if mock.SamplesFunc == nil { - panic("EncodedSegmentMock.SamplesFunc: method is nil but EncodedSegment.Samples was just called") +// Name calls NameFunc. +func (mock *FileInfoMock) Name() string { + if mock.NameFunc == nil { + panic("FileInfoMock.NameFunc: method is nil but FileInfo.Name was just called") } callInfo := struct { }{} - mock.lockSamples.Lock() - mock.calls.Samples = append(mock.calls.Samples, callInfo) - mock.lockSamples.Unlock() - return mock.SamplesFunc() + mock.lockName.Lock() + mock.calls.Name = append(mock.calls.Name, callInfo) + mock.lockName.Unlock() + return mock.NameFunc() } -// SamplesCalls gets all the calls that were made to Samples. +// NameCalls gets all the calls that were made to Name. // Check the length with: // -// len(mockedEncodedSegment.SamplesCalls()) -func (mock *EncodedSegmentMock) SamplesCalls() []struct { +// len(mockedFileInfo.NameCalls()) +func (mock *FileInfoMock) NameCalls() []struct { } { var calls []struct { } - mock.lockSamples.RLock() - calls = mock.calls.Samples - mock.lockSamples.RUnlock() + mock.lockName.RLock() + calls = mock.calls.Name + mock.lockName.RUnlock() return calls } // Size calls SizeFunc. -func (mock *EncodedSegmentMock) Size() int64 { +func (mock *FileInfoMock) Size() int64 { if mock.SizeFunc == nil { - panic("EncodedSegmentMock.SizeFunc: method is nil but EncodedSegment.Size was just called") + panic("FileInfoMock.SizeFunc: method is nil but FileInfo.Size was just called") } callInfo := struct { }{} @@ -145,8 +212,8 @@ func (mock *EncodedSegmentMock) Size() int64 { // SizeCalls gets all the calls that were made to Size. // Check the length with: // -// len(mockedEncodedSegment.SizeCalls()) -func (mock *EncodedSegmentMock) SizeCalls() []struct { +// len(mockedFileInfo.SizeCalls()) +func (mock *FileInfoMock) SizeCalls() []struct { } { var calls []struct { } @@ -156,42 +223,95 @@ func (mock *EncodedSegmentMock) SizeCalls() []struct { return calls } -// WriteTo calls WriteToFunc. -func (mock *EncodedSegmentMock) WriteTo(w io.Writer) (int64, error) { - if mock.WriteToFunc == nil { - panic("EncodedSegmentMock.WriteToFunc: method is nil but EncodedSegment.WriteTo was just called") +// Sys calls SysFunc. +func (mock *FileInfoMock) Sys() any { + if mock.SysFunc == nil { + panic("FileInfoMock.SysFunc: method is nil but FileInfo.Sys was just called") + } + callInfo := struct { + }{} + mock.lockSys.Lock() + mock.calls.Sys = append(mock.calls.Sys, callInfo) + mock.lockSys.Unlock() + return mock.SysFunc() +} + +// SysCalls gets all the calls that were made to Sys. +// Check the length with: +// +// len(mockedFileInfo.SysCalls()) +func (mock *FileInfoMock) SysCalls() []struct { +} { + var calls []struct { + } + mock.lockSys.RLock() + calls = mock.calls.Sys + mock.lockSys.RUnlock() + return calls +} + +// SegmentIsWrittenNotifierMock is a mock implementation of writer.SegmentIsWrittenNotifier. +// +// func TestSomethingThatUsesSegmentIsWrittenNotifier(t *testing.T) { +// +// // make and configure a mocked writer.SegmentIsWrittenNotifier +// mockedSegmentIsWrittenNotifier := &SegmentIsWrittenNotifierMock{ +// NotifySegmentIsWrittenFunc: func(shardID uint16) { +// panic("mock out the NotifySegmentIsWritten method") +// }, +// } +// +// // use mockedSegmentIsWrittenNotifier in code that requires writer.SegmentIsWrittenNotifier +// // and then make assertions. +// +// } +type SegmentIsWrittenNotifierMock struct { + // NotifySegmentIsWrittenFunc mocks the NotifySegmentIsWritten method. + NotifySegmentIsWrittenFunc func(shardID uint16) + + // calls tracks calls to the methods. + calls struct { + // NotifySegmentIsWritten holds details about calls to the NotifySegmentIsWritten method. + NotifySegmentIsWritten []struct { + // ShardID is the shardID argument value. + ShardID uint16 + } + } + lockNotifySegmentIsWritten sync.RWMutex +} + +// NotifySegmentIsWritten calls NotifySegmentIsWrittenFunc. +func (mock *SegmentIsWrittenNotifierMock) NotifySegmentIsWritten(shardID uint16) { + if mock.NotifySegmentIsWrittenFunc == nil { + panic("SegmentIsWrittenNotifierMock.NotifySegmentIsWrittenFunc: method is nil but SegmentIsWrittenNotifier.NotifySegmentIsWritten was just called") } callInfo := struct { - W io.Writer + ShardID uint16 }{ - W: w, + ShardID: shardID, } - mock.lockWriteTo.Lock() - mock.calls.WriteTo = append(mock.calls.WriteTo, callInfo) - mock.lockWriteTo.Unlock() - return mock.WriteToFunc(w) + mock.lockNotifySegmentIsWritten.Lock() + mock.calls.NotifySegmentIsWritten = append(mock.calls.NotifySegmentIsWritten, callInfo) + mock.lockNotifySegmentIsWritten.Unlock() + mock.NotifySegmentIsWrittenFunc(shardID) } -// WriteToCalls gets all the calls that were made to WriteTo. +// NotifySegmentIsWrittenCalls gets all the calls that were made to NotifySegmentIsWritten. // Check the length with: // -// len(mockedEncodedSegment.WriteToCalls()) -func (mock *EncodedSegmentMock) WriteToCalls() []struct { - W io.Writer +// len(mockedSegmentIsWrittenNotifier.NotifySegmentIsWrittenCalls()) +func (mock *SegmentIsWrittenNotifierMock) NotifySegmentIsWrittenCalls() []struct { + ShardID uint16 } { var calls []struct { - W io.Writer + ShardID uint16 } - mock.lockWriteTo.RLock() - calls = mock.calls.WriteTo - mock.lockWriteTo.RUnlock() + mock.lockNotifySegmentIsWritten.RLock() + calls = mock.calls.NotifySegmentIsWritten + mock.lockNotifySegmentIsWritten.RUnlock() return calls } -// Ensure, that WriteSyncCloserMock does implement writer.WriteSyncCloser. -// If this is not the case, regenerate this file with moq. -var _ writer.WriteSyncCloser = &WriteSyncCloserMock{} - // WriteSyncCloserMock is a mock implementation of writer.WriteSyncCloser. // // func TestSomethingThatUsesWriteSyncCloser(t *testing.T) { @@ -364,313 +484,3 @@ func (mock *WriteSyncCloserMock) WriteCalls() []struct { mock.lockWrite.RUnlock() return calls } - -// Ensure, that SegmentIsWrittenNotifierMock does implement writer.SegmentIsWrittenNotifier. -// If this is not the case, regenerate this file with moq. -var _ writer.SegmentIsWrittenNotifier = &SegmentIsWrittenNotifierMock{} - -// SegmentIsWrittenNotifierMock is a mock implementation of writer.SegmentIsWrittenNotifier. -// -// func TestSomethingThatUsesSegmentIsWrittenNotifier(t *testing.T) { -// -// // make and configure a mocked writer.SegmentIsWrittenNotifier -// mockedSegmentIsWrittenNotifier := &SegmentIsWrittenNotifierMock{ -// NotifySegmentIsWrittenFunc: func(shardID uint16) { -// panic("mock out the NotifySegmentIsWritten method") -// }, -// } -// -// // use mockedSegmentIsWrittenNotifier in code that requires writer.SegmentIsWrittenNotifier -// // and then make assertions. -// -// } -type SegmentIsWrittenNotifierMock struct { - // NotifySegmentIsWrittenFunc mocks the NotifySegmentIsWritten method. - NotifySegmentIsWrittenFunc func(shardID uint16) - - // calls tracks calls to the methods. - calls struct { - // NotifySegmentIsWritten holds details about calls to the NotifySegmentIsWritten method. - NotifySegmentIsWritten []struct { - // ShardID is the shardID argument value. - ShardID uint16 - } - } - lockNotifySegmentIsWritten sync.RWMutex -} - -// NotifySegmentIsWritten calls NotifySegmentIsWrittenFunc. -func (mock *SegmentIsWrittenNotifierMock) NotifySegmentIsWritten(shardID uint16) { - if mock.NotifySegmentIsWrittenFunc == nil { - panic("SegmentIsWrittenNotifierMock.NotifySegmentIsWrittenFunc: method is nil but SegmentIsWrittenNotifier.NotifySegmentIsWritten was just called") - } - callInfo := struct { - ShardID uint16 - }{ - ShardID: shardID, - } - mock.lockNotifySegmentIsWritten.Lock() - mock.calls.NotifySegmentIsWritten = append(mock.calls.NotifySegmentIsWritten, callInfo) - mock.lockNotifySegmentIsWritten.Unlock() - mock.NotifySegmentIsWrittenFunc(shardID) -} - -// NotifySegmentIsWrittenCalls gets all the calls that were made to NotifySegmentIsWritten. -// Check the length with: -// -// len(mockedSegmentIsWrittenNotifier.NotifySegmentIsWrittenCalls()) -func (mock *SegmentIsWrittenNotifierMock) NotifySegmentIsWrittenCalls() []struct { - ShardID uint16 -} { - var calls []struct { - ShardID uint16 - } - mock.lockNotifySegmentIsWritten.RLock() - calls = mock.calls.NotifySegmentIsWritten - mock.lockNotifySegmentIsWritten.RUnlock() - return calls -} - -// Ensure, that FileInfoMock does implement writer.FileInfo. -// If this is not the case, regenerate this file with moq. -var _ writer.FileInfo = &FileInfoMock{} - -// FileInfoMock is a mock implementation of writer.FileInfo. -// -// func TestSomethingThatUsesFileInfo(t *testing.T) { -// -// // make and configure a mocked writer.FileInfo -// mockedFileInfo := &FileInfoMock{ -// IsDirFunc: func() bool { -// panic("mock out the IsDir method") -// }, -// ModTimeFunc: func() time.Time { -// panic("mock out the ModTime method") -// }, -// ModeFunc: func() fs.FileMode { -// panic("mock out the Mode method") -// }, -// NameFunc: func() string { -// panic("mock out the Name method") -// }, -// SizeFunc: func() int64 { -// panic("mock out the Size method") -// }, -// SysFunc: func() any { -// panic("mock out the Sys method") -// }, -// } -// -// // use mockedFileInfo in code that requires writer.FileInfo -// // and then make assertions. -// -// } -type FileInfoMock struct { - // IsDirFunc mocks the IsDir method. - IsDirFunc func() bool - - // ModTimeFunc mocks the ModTime method. - ModTimeFunc func() time.Time - - // ModeFunc mocks the Mode method. - ModeFunc func() fs.FileMode - - // NameFunc mocks the Name method. - NameFunc func() string - - // SizeFunc mocks the Size method. - SizeFunc func() int64 - - // SysFunc mocks the Sys method. - SysFunc func() any - - // calls tracks calls to the methods. - calls struct { - // IsDir holds details about calls to the IsDir method. - IsDir []struct { - } - // ModTime holds details about calls to the ModTime method. - ModTime []struct { - } - // Mode holds details about calls to the Mode method. - Mode []struct { - } - // Name holds details about calls to the Name method. - Name []struct { - } - // Size holds details about calls to the Size method. - Size []struct { - } - // Sys holds details about calls to the Sys method. - Sys []struct { - } - } - lockIsDir sync.RWMutex - lockModTime sync.RWMutex - lockMode sync.RWMutex - lockName sync.RWMutex - lockSize sync.RWMutex - lockSys sync.RWMutex -} - -// IsDir calls IsDirFunc. -func (mock *FileInfoMock) IsDir() bool { - if mock.IsDirFunc == nil { - panic("FileInfoMock.IsDirFunc: method is nil but FileInfo.IsDir was just called") - } - callInfo := struct { - }{} - mock.lockIsDir.Lock() - mock.calls.IsDir = append(mock.calls.IsDir, callInfo) - mock.lockIsDir.Unlock() - return mock.IsDirFunc() -} - -// IsDirCalls gets all the calls that were made to IsDir. -// Check the length with: -// -// len(mockedFileInfo.IsDirCalls()) -func (mock *FileInfoMock) IsDirCalls() []struct { -} { - var calls []struct { - } - mock.lockIsDir.RLock() - calls = mock.calls.IsDir - mock.lockIsDir.RUnlock() - return calls -} - -// ModTime calls ModTimeFunc. -func (mock *FileInfoMock) ModTime() time.Time { - if mock.ModTimeFunc == nil { - panic("FileInfoMock.ModTimeFunc: method is nil but FileInfo.ModTime was just called") - } - callInfo := struct { - }{} - mock.lockModTime.Lock() - mock.calls.ModTime = append(mock.calls.ModTime, callInfo) - mock.lockModTime.Unlock() - return mock.ModTimeFunc() -} - -// ModTimeCalls gets all the calls that were made to ModTime. -// Check the length with: -// -// len(mockedFileInfo.ModTimeCalls()) -func (mock *FileInfoMock) ModTimeCalls() []struct { -} { - var calls []struct { - } - mock.lockModTime.RLock() - calls = mock.calls.ModTime - mock.lockModTime.RUnlock() - return calls -} - -// Mode calls ModeFunc. -func (mock *FileInfoMock) Mode() fs.FileMode { - if mock.ModeFunc == nil { - panic("FileInfoMock.ModeFunc: method is nil but FileInfo.Mode was just called") - } - callInfo := struct { - }{} - mock.lockMode.Lock() - mock.calls.Mode = append(mock.calls.Mode, callInfo) - mock.lockMode.Unlock() - return mock.ModeFunc() -} - -// ModeCalls gets all the calls that were made to Mode. -// Check the length with: -// -// len(mockedFileInfo.ModeCalls()) -func (mock *FileInfoMock) ModeCalls() []struct { -} { - var calls []struct { - } - mock.lockMode.RLock() - calls = mock.calls.Mode - mock.lockMode.RUnlock() - return calls -} - -// Name calls NameFunc. -func (mock *FileInfoMock) Name() string { - if mock.NameFunc == nil { - panic("FileInfoMock.NameFunc: method is nil but FileInfo.Name was just called") - } - callInfo := struct { - }{} - mock.lockName.Lock() - mock.calls.Name = append(mock.calls.Name, callInfo) - mock.lockName.Unlock() - return mock.NameFunc() -} - -// NameCalls gets all the calls that were made to Name. -// Check the length with: -// -// len(mockedFileInfo.NameCalls()) -func (mock *FileInfoMock) NameCalls() []struct { -} { - var calls []struct { - } - mock.lockName.RLock() - calls = mock.calls.Name - mock.lockName.RUnlock() - return calls -} - -// Size calls SizeFunc. -func (mock *FileInfoMock) Size() int64 { - if mock.SizeFunc == nil { - panic("FileInfoMock.SizeFunc: method is nil but FileInfo.Size was just called") - } - callInfo := struct { - }{} - mock.lockSize.Lock() - mock.calls.Size = append(mock.calls.Size, callInfo) - mock.lockSize.Unlock() - return mock.SizeFunc() -} - -// SizeCalls gets all the calls that were made to Size. -// Check the length with: -// -// len(mockedFileInfo.SizeCalls()) -func (mock *FileInfoMock) SizeCalls() []struct { -} { - var calls []struct { - } - mock.lockSize.RLock() - calls = mock.calls.Size - mock.lockSize.RUnlock() - return calls -} - -// Sys calls SysFunc. -func (mock *FileInfoMock) Sys() any { - if mock.SysFunc == nil { - panic("FileInfoMock.SysFunc: method is nil but FileInfo.Sys was just called") - } - callInfo := struct { - }{} - mock.lockSys.Lock() - mock.calls.Sys = append(mock.calls.Sys, callInfo) - mock.lockSys.Unlock() - return mock.SysFunc() -} - -// SysCalls gets all the calls that were made to Sys. -// Check the length with: -// -// len(mockedFileInfo.SysCalls()) -func (mock *FileInfoMock) SysCalls() []struct { -} { - var calls []struct { - } - mock.lockSys.RLock() - calls = mock.calls.Sys - mock.lockSys.RUnlock() - return calls -} diff --git a/pp/go/storage/head/shard/wal/writer/buffered_test.go b/pp/go/storage/head/shard/wal/writer/buffered_test.go index 1e061d5a21..ef18136208 100644 --- a/pp/go/storage/head/shard/wal/writer/buffered_test.go +++ b/pp/go/storage/head/shard/wal/writer/buffered_test.go @@ -13,8 +13,6 @@ import ( "github.com/prometheus/prometheus/pp/go/storage/head/shard/wal/writer" ) -// TODO moq -out writer_moq_test.go -pkg writer_test -rm . EncodedSegment WriteSyncCloser SegmentIsWrittenNotifier FileInfo - type BufferedSuite struct { suite.Suite } diff --git a/pp/go/storage/head/shard/wal/writer/segment.go b/pp/go/storage/head/shard/wal/writer/segment.go index a396f8537d..35ca87bf61 100644 --- a/pp/go/storage/head/shard/wal/writer/segment.go +++ b/pp/go/storage/head/shard/wal/writer/segment.go @@ -6,6 +6,9 @@ import ( "io" ) +//go:generate -command moq go run github.com/matryer/moq --rm --skip-ensure --pkg writer_test --out +//go:generate moq segment_moq_test.go . EncodedSegment + // EncodedSegment the minimum required Segment implementation for a [WriteSegment]. type EncodedSegment interface { CRC32() uint32 diff --git a/pp/go/storage/head/shard/wal/writer/segment_moq_test.go b/pp/go/storage/head/shard/wal/writer/segment_moq_test.go new file mode 100644 index 0000000000..33b8bd2971 --- /dev/null +++ b/pp/go/storage/head/shard/wal/writer/segment_moq_test.go @@ -0,0 +1,182 @@ +// Code generated by moq; DO NOT EDIT. +// github.com/matryer/moq + +package writer_test + +import ( + "io" + "sync" +) + +// EncodedSegmentMock is a mock implementation of writer.EncodedSegment. +// +// func TestSomethingThatUsesEncodedSegment(t *testing.T) { +// +// // make and configure a mocked writer.EncodedSegment +// mockedEncodedSegment := &EncodedSegmentMock{ +// CRC32Func: func() uint32 { +// panic("mock out the CRC32 method") +// }, +// SamplesFunc: func() uint32 { +// panic("mock out the Samples method") +// }, +// SizeFunc: func() int64 { +// panic("mock out the Size method") +// }, +// WriteToFunc: func(w io.Writer) (int64, error) { +// panic("mock out the WriteTo method") +// }, +// } +// +// // use mockedEncodedSegment in code that requires writer.EncodedSegment +// // and then make assertions. +// +// } +type EncodedSegmentMock struct { + // CRC32Func mocks the CRC32 method. + CRC32Func func() uint32 + + // SamplesFunc mocks the Samples method. + SamplesFunc func() uint32 + + // SizeFunc mocks the Size method. + SizeFunc func() int64 + + // WriteToFunc mocks the WriteTo method. + WriteToFunc func(w io.Writer) (int64, error) + + // calls tracks calls to the methods. + calls struct { + // CRC32 holds details about calls to the CRC32 method. + CRC32 []struct { + } + // Samples holds details about calls to the Samples method. + Samples []struct { + } + // Size holds details about calls to the Size method. + Size []struct { + } + // WriteTo holds details about calls to the WriteTo method. + WriteTo []struct { + // W is the w argument value. + W io.Writer + } + } + lockCRC32 sync.RWMutex + lockSamples sync.RWMutex + lockSize sync.RWMutex + lockWriteTo sync.RWMutex +} + +// CRC32 calls CRC32Func. +func (mock *EncodedSegmentMock) CRC32() uint32 { + if mock.CRC32Func == nil { + panic("EncodedSegmentMock.CRC32Func: method is nil but EncodedSegment.CRC32 was just called") + } + callInfo := struct { + }{} + mock.lockCRC32.Lock() + mock.calls.CRC32 = append(mock.calls.CRC32, callInfo) + mock.lockCRC32.Unlock() + return mock.CRC32Func() +} + +// CRC32Calls gets all the calls that were made to CRC32. +// Check the length with: +// +// len(mockedEncodedSegment.CRC32Calls()) +func (mock *EncodedSegmentMock) CRC32Calls() []struct { +} { + var calls []struct { + } + mock.lockCRC32.RLock() + calls = mock.calls.CRC32 + mock.lockCRC32.RUnlock() + return calls +} + +// Samples calls SamplesFunc. +func (mock *EncodedSegmentMock) Samples() uint32 { + if mock.SamplesFunc == nil { + panic("EncodedSegmentMock.SamplesFunc: method is nil but EncodedSegment.Samples was just called") + } + callInfo := struct { + }{} + mock.lockSamples.Lock() + mock.calls.Samples = append(mock.calls.Samples, callInfo) + mock.lockSamples.Unlock() + return mock.SamplesFunc() +} + +// SamplesCalls gets all the calls that were made to Samples. +// Check the length with: +// +// len(mockedEncodedSegment.SamplesCalls()) +func (mock *EncodedSegmentMock) SamplesCalls() []struct { +} { + var calls []struct { + } + mock.lockSamples.RLock() + calls = mock.calls.Samples + mock.lockSamples.RUnlock() + return calls +} + +// Size calls SizeFunc. +func (mock *EncodedSegmentMock) Size() int64 { + if mock.SizeFunc == nil { + panic("EncodedSegmentMock.SizeFunc: method is nil but EncodedSegment.Size was just called") + } + callInfo := struct { + }{} + mock.lockSize.Lock() + mock.calls.Size = append(mock.calls.Size, callInfo) + mock.lockSize.Unlock() + return mock.SizeFunc() +} + +// SizeCalls gets all the calls that were made to Size. +// Check the length with: +// +// len(mockedEncodedSegment.SizeCalls()) +func (mock *EncodedSegmentMock) SizeCalls() []struct { +} { + var calls []struct { + } + mock.lockSize.RLock() + calls = mock.calls.Size + mock.lockSize.RUnlock() + return calls +} + +// WriteTo calls WriteToFunc. +func (mock *EncodedSegmentMock) WriteTo(w io.Writer) (int64, error) { + if mock.WriteToFunc == nil { + panic("EncodedSegmentMock.WriteToFunc: method is nil but EncodedSegment.WriteTo was just called") + } + callInfo := struct { + W io.Writer + }{ + W: w, + } + mock.lockWriteTo.Lock() + mock.calls.WriteTo = append(mock.calls.WriteTo, callInfo) + mock.lockWriteTo.Unlock() + return mock.WriteToFunc(w) +} + +// WriteToCalls gets all the calls that were made to WriteTo. +// Check the length with: +// +// len(mockedEncodedSegment.WriteToCalls()) +func (mock *EncodedSegmentMock) WriteToCalls() []struct { + W io.Writer +} { + var calls []struct { + W io.Writer + } + mock.lockWriteTo.RLock() + calls = mock.calls.WriteTo + mock.lockWriteTo.RUnlock() + return calls +} diff --git a/pp/go/storage/loader.go b/pp/go/storage/loader.go index 03b60ff7c9..1286d548ee 100644 --- a/pp/go/storage/loader.go +++ b/pp/go/storage/loader.go @@ -47,7 +47,7 @@ func NewLoader( } } -// Load upload [HeadOnDisk] from [WalOnDisk] by head ID. +// Load [HeadOnDisk] from [WalOnDisk] by head ID. func (l *Loader) Load( headRecord *catalog.Record, generation uint64, @@ -199,7 +199,7 @@ func (l *ShardDataLoader) Load() (err error) { ](), } - shardWalFile, err := os.OpenFile(GetShardWalFilename(l.dir, l.shardID), os.O_RDWR, 0666) + shardWalFile, err := os.OpenFile(GetShardWalFilename(l.dir, l.shardID), os.O_RDWR, 0o666) if err != nil { return err } @@ -364,11 +364,9 @@ func GetQueriedSeriesStorageFilename(dir string, shardID uint16, index uint8) st // isNumberOfSegmentsMismatched check number of segments loaded and last appended to record. func isNumberOfSegmentsMismatched(record *catalog.Record, loadedSegments uint32) bool { - return false + if record.LastAppendedSegmentID() == nil { + return loadedSegments != 0 + } - // TODO: uncomment this code block - //if record.LastAppendedSegmentID() == nil { - // return loadedSegments != 0 - //} - //return *record.LastAppendedSegmentID()+1 != loadedSegments + return *record.LastAppendedSegmentID()+1 != loadedSegments } diff --git a/pp/go/storage/manager.go b/pp/go/storage/manager.go index 32e5bd1eb4..0dc1a6b621 100644 --- a/pp/go/storage/manager.go +++ b/pp/go/storage/manager.go @@ -178,7 +178,7 @@ func NewManager( return nil, errors.Join(fmt.Errorf("failed to set active status: %w", err), h.Close()) } - hKeeper := keeper.NewKeeper[*HeadOnDisk](o.QueueSize) + hKeeper := keeper.NewKeeper[HeadOnDisk](o.QueueSize) m := &Manager{ g: run.Group{}, @@ -194,9 +194,9 @@ func NewManager( readyNotifier.NotifyReady() - m.initServices(o, hcatalog, builder, loader, clock, r) + m.initServices(o, hcatalog, builder, loader, triggerNotifier, clock, r) - logger.Infof("Head Manager created") + logger.Infof("[Head Manager] created") return m, nil } @@ -247,6 +247,7 @@ func (m *Manager) initServices( hcatalog *catalog.Catalog, builder *Builder, loader *Loader, + triggerNotifier *ReloadBlocksTriggerNotifier, clock clockwork.Clock, r prometheus.Registerer, ) { @@ -278,10 +279,12 @@ func (m *Manager) initServices( o.BlockDuration, r, ), + triggerNotifier, clock, persistenerMediator, o.MaxRetentionPeriod, o.HeadRetentionPeriod, + r, ).Run() return nil @@ -428,18 +431,22 @@ func (hi *headInformer) SetRotatedStatus(headID string) error { // ReloadBlocksTriggerNotifier // +// ReloadBlocksTriggerNotifier for notifications about the appearance of new blocks. type ReloadBlocksTriggerNotifier struct { c chan struct{} } +// NewReloadBlocksTriggerNotifier init new [ReloadBlocksTriggerNotifier]. func NewReloadBlocksTriggerNotifier() *ReloadBlocksTriggerNotifier { return &ReloadBlocksTriggerNotifier{c: make(chan struct{}, 1)} } +// Chan returns channel with notifications. func (tn *ReloadBlocksTriggerNotifier) Chan() <-chan struct{} { return tn.c } +// NotifyWritten sends a notify that the writing is completed. func (tn *ReloadBlocksTriggerNotifier) NotifyWritten() { select { case tn.c <- struct{}{}: @@ -481,7 +488,6 @@ func uploadOrBuildHead( var generation uint64 if len(headRecords) == 0 { - // TODO // m.counter.With(prometheus.Labels{"type": "created"}).Inc() return builder.Build(generation, numberOfShards) } @@ -492,7 +498,7 @@ func uploadOrBuildHead( logger.Errorf("failed to set corrupted state, head id: %s: %v", headRecords[0].ID(), setCorruptedErr) } } - // TODO // m.counter.With(prometheus.Labels{"type": "corrupted"}).Inc() + logger.Warnf("[Head Manager] upload corrupted head, building new: %s", headRecords[0].ID()) if _, err := hcatalog.SetStatus(headRecords[0].ID(), catalog.StatusRotated); err != nil { logger.Warnf("failed to set rotated status for head {%s}: %s", headRecords[0].ID(), err) @@ -500,7 +506,6 @@ func uploadOrBuildHead( _ = h.Close() - // TODO // m.counter.With(prometheus.Labels{"type": "created"}).Inc() return builder.Build(generation, numberOfShards) } diff --git a/pp/go/storage/mediator/mediator.go b/pp/go/storage/mediator/mediator.go index 7d12912946..a200fc162f 100644 --- a/pp/go/storage/mediator/mediator.go +++ b/pp/go/storage/mediator/mediator.go @@ -7,13 +7,23 @@ import ( "github.com/prometheus/prometheus/pp/go/util" ) +//go:generate -command moq go run github.com/matryer/moq --rm --skip-ensure --pkg mediator_test --out + // // Timer // // Timer implementation timer. +// +//go:generate moq mediator_moq_test.go . Timer type Timer interface { + // Chan returns chan with ticker time. Chan() <-chan time.Time + + // Reset changes the timer to expire after duration Block and clearing channels. + Reset() + + // Stop prevents the Timer from firing. Stop() } @@ -73,6 +83,7 @@ func (m *Mediator) loop() { select { case <-m.timer.Chan(): m.Trigger() + m.timer.Reset() case <-m.closer.Signal(): return } diff --git a/pp/go/storage/mediator/mediator_moq_test.go b/pp/go/storage/mediator/mediator_moq_test.go index bc3b50e51b..0d5a5ce0b7 100644 --- a/pp/go/storage/mediator/mediator_moq_test.go +++ b/pp/go/storage/mediator/mediator_moq_test.go @@ -4,15 +4,10 @@ package mediator_test import ( - "github.com/prometheus/prometheus/pp/go/storage/mediator" "sync" "time" ) -// Ensure, that TimerMock does implement mediator.Timer. -// If this is not the case, regenerate this file with moq. -var _ mediator.Timer = &TimerMock{} - // TimerMock is a mock implementation of mediator.Timer. // // func TestSomethingThatUsesTimer(t *testing.T) { @@ -22,6 +17,9 @@ var _ mediator.Timer = &TimerMock{} // ChanFunc: func() <-chan time.Time { // panic("mock out the Chan method") // }, +// ResetFunc: func() { +// panic("mock out the Reset method") +// }, // StopFunc: func() { // panic("mock out the Stop method") // }, @@ -35,6 +33,9 @@ type TimerMock struct { // ChanFunc mocks the Chan method. ChanFunc func() <-chan time.Time + // ResetFunc mocks the Reset method. + ResetFunc func() + // StopFunc mocks the Stop method. StopFunc func() @@ -43,12 +44,16 @@ type TimerMock struct { // Chan holds details about calls to the Chan method. Chan []struct { } + // Reset holds details about calls to the Reset method. + Reset []struct { + } // Stop holds details about calls to the Stop method. Stop []struct { } } - lockChan sync.RWMutex - lockStop sync.RWMutex + lockChan sync.RWMutex + lockReset sync.RWMutex + lockStop sync.RWMutex } // Chan calls ChanFunc. @@ -78,6 +83,33 @@ func (mock *TimerMock) ChanCalls() []struct { return calls } +// Reset calls ResetFunc. +func (mock *TimerMock) Reset() { + if mock.ResetFunc == nil { + panic("TimerMock.ResetFunc: method is nil but Timer.Reset was just called") + } + callInfo := struct { + }{} + mock.lockReset.Lock() + mock.calls.Reset = append(mock.calls.Reset, callInfo) + mock.lockReset.Unlock() + mock.ResetFunc() +} + +// ResetCalls gets all the calls that were made to Reset. +// Check the length with: +// +// len(mockedTimer.ResetCalls()) +func (mock *TimerMock) ResetCalls() []struct { +} { + var calls []struct { + } + mock.lockReset.RLock() + calls = mock.calls.Reset + mock.lockReset.RUnlock() + return calls +} + // Stop calls StopFunc. func (mock *TimerMock) Stop() { if mock.StopFunc == nil { diff --git a/pp/go/storage/mediator/mediator_test.go b/pp/go/storage/mediator/mediator_test.go index 002c6d4356..b819b5dbe6 100644 --- a/pp/go/storage/mediator/mediator_test.go +++ b/pp/go/storage/mediator/mediator_test.go @@ -10,8 +10,6 @@ import ( "github.com/stretchr/testify/suite" ) -// TODO moq -out mediator_moq_test.go -pkg mediator_test -rm . Timer - type MediatorSuite struct { suite.Suite } @@ -27,7 +25,8 @@ func (s *MediatorSuite) TestC() { ChanFunc: func() <-chan time.Time { return chTimer }, - StopFunc: func() {}, + ResetFunc: func() {}, + StopFunc: func() {}, } m := mediator.NewMediator(timer) @@ -69,7 +68,8 @@ func (s *MediatorSuite) TestClose() { ChanFunc: func() <-chan time.Time { return chTimer }, - StopFunc: func() { stopCounter++ }, + ResetFunc: func() {}, + StopFunc: func() { stopCounter++ }, } m := mediator.NewMediator(timer) @@ -114,7 +114,8 @@ func (s *MediatorSuite) TestTrigger() { ChanFunc: func() <-chan time.Time { return chTimer }, - StopFunc: func() {}, + ResetFunc: func() {}, + StopFunc: func() {}, } m := mediator.NewMediator(timer) @@ -155,7 +156,8 @@ func (s *MediatorSuite) TestTriggerSkip() { ChanFunc: func() <-chan time.Time { return chTimer }, - StopFunc: func() {}, + ResetFunc: func() {}, + StopFunc: func() {}, } m := mediator.NewMediator(timer) diff --git a/pp/go/storage/querier/metrics.go b/pp/go/storage/querier/metrics.go index c31abaee9c..71e968890a 100644 --- a/pp/go/storage/querier/metrics.go +++ b/pp/go/storage/querier/metrics.go @@ -25,7 +25,7 @@ func NewMetrics(registerer prometheus.Registerer, source string) *Metrics { return &Metrics{ LabelNamesDuration: factory.NewHistogram( prometheus.HistogramOpts{ - Name: "prompp_head_query_label_names_duration", + Name: "prompp_querier_query_label_names_duration", Help: "Label names query from head duration in microseconds", Buckets: []float64{ 50, 100, 250, 500, 750, @@ -38,7 +38,7 @@ func NewMetrics(registerer prometheus.Registerer, source string) *Metrics { ), LabelValuesDuration: factory.NewHistogram( prometheus.HistogramOpts{ - Name: "prompp_head_query_label_values_duration", + Name: "prompp_querier_query_label_values_duration", Help: "Label values query from head duration in microseconds", Buckets: []float64{ 50, 100, 250, 500, 750, @@ -51,7 +51,7 @@ func NewMetrics(registerer prometheus.Registerer, source string) *Metrics { ), SelectDuration: factory.NewHistogramVec( prometheus.HistogramOpts{ - Name: "prompp_head_query_select_duration", + Name: "prompp_querier_query_select_duration", Help: "Select query from head duration in microseconds", Buckets: []float64{ 50, 100, 250, 500, 750, diff --git a/pp/go/storage/querier/querier.go b/pp/go/storage/querier/querier.go index e8c7a9a351..396bdbbcaa 100644 --- a/pp/go/storage/querier/querier.go +++ b/pp/go/storage/querier/querier.go @@ -221,8 +221,7 @@ func (q *Querier[TTask, TDataStorage, TLSS, TShard, THead]) selectInstant( _ = tDataStorageQuery.Wait() if err := loadAndQueryWaiter.Wait(); err != nil { - // TODO: Unrecoverable error - // q.head.UnrecoverableError(err) + SendUnrecoverableError(err) return storage.ErrSeriesSet(err) } From 6c0f8c4db07f3c621c46959664e3a3e900a964cd Mon Sep 17 00:00:00 2001 From: Alexandr Yudin Date: Thu, 25 Sep 2025 14:26:18 +0000 Subject: [PATCH 49/96] smal fix --- pp/go/storage/loader.go | 26 +++++++++++++++++++------- pp/go/storage/loader_test.go | 2 -- pp/go/storage/querier/interface.go | 14 ++++++++++---- 3 files changed, 29 insertions(+), 13 deletions(-) diff --git a/pp/go/storage/loader.go b/pp/go/storage/loader.go index 1286d548ee..6816a402f9 100644 --- a/pp/go/storage/loader.go +++ b/pp/go/storage/loader.go @@ -105,7 +105,18 @@ func (l *Loader) Load( if numberOfSegmentsRead.Value() > 0 { headRecord.SetLastAppendedSegmentID(numberOfSegmentsRead.Value() - 1) } - logger.Errorf("head: %s number of segments mismatched", headRecord.ID()) + + lastAppendedSegmentID := uint32(0) + if headRecord.LastAppendedSegmentID() != nil { + lastAppendedSegmentID = *headRecord.LastAppendedSegmentID() + } + + logger.Errorf( + "head: %s number of segments mismatched", + headRecord.ID(), + lastAppendedSegmentID, + numberOfSegmentsRead.Value(), + ) } h := head.NewHead( @@ -154,7 +165,6 @@ type ShardLoadResult struct { } type ShardData struct { - notifier *writer.SegmentWriteNotifier lss *shard.LSS dataStorage *shard.DataStorage wal *WalOnDisk @@ -199,7 +209,11 @@ func (l *ShardDataLoader) Load() (err error) { ](), } - shardWalFile, err := os.OpenFile(GetShardWalFilename(l.dir, l.shardID), os.O_RDWR, 0o666) + shardWalFile, err := os.OpenFile( //nolint:gosec // need this permissions + GetShardWalFilename(l.dir, l.shardID), + os.O_RDWR, + 0o666, + ) if err != nil { return err } @@ -341,10 +355,8 @@ func (l *ShardDataLoader) loadQueriedSeries() (bool, error) { } logger.Warnf("error loading queried series: %v", err) - } else { - if !l.shardData.dataStorage.SetQueriedSeriesBitset(queriedSeries) { - logger.Warnf("error set queried series in storage: %v", err) - } + } else if !l.shardData.dataStorage.SetQueriedSeriesBitset(queriedSeries) { + logger.Warnf("error set queried series in storage: %v", err) } return false, nil diff --git a/pp/go/storage/loader_test.go b/pp/go/storage/loader_test.go index c339f02857..5f50574fe1 100644 --- a/pp/go/storage/loader_test.go +++ b/pp/go/storage/loader_test.go @@ -1,7 +1,6 @@ package storage_test import ( - "context" "os" "path/filepath" "testing" @@ -48,7 +47,6 @@ func (g *idGeneratorStub) last() string { type HeadLoadSuite struct { suite.Suite dataDir string - ctx context.Context clock clockwork.Clock headIdGenerator *idGeneratorStub catalog *catalog.Catalog diff --git a/pp/go/storage/querier/interface.go b/pp/go/storage/querier/interface.go index ca1ed3a0ef..5cb1fcf6d0 100644 --- a/pp/go/storage/querier/interface.go +++ b/pp/go/storage/querier/interface.go @@ -15,7 +15,8 @@ import ( // Deduplicator accumulates and deduplicates incoming values. type Deduplicator interface { // Add values to deduplicator by shard ID. - Add(shard uint16, snapshot *cppbridge.LabelSetSnapshot, values []string) + Add(shardID uint16, snapshot *cppbridge.LabelSetSnapshot, values []string) + // Values returns collected values. Values() []string } @@ -40,10 +41,15 @@ type Task interface { // DataStorage the minimum required [DataStorage] implementation. type DataStorage interface { // InstantQuery returns samples for instant query from data storage. - InstantQuery(maxt, valueNotFoundTimestampValue int64, ids []uint32) ([]cppbridge.Sample, cppbridge.DataStorageQueryResult) + InstantQuery( + maxt, valueNotFoundTimestampValue int64, + ids []uint32, + ) ([]cppbridge.Sample, cppbridge.DataStorageQueryResult) // QueryDataStorage returns serialized chunks from data storage. - Query(query cppbridge.HeadDataStorageQuery) (*cppbridge.HeadDataStorageSerializedChunks, cppbridge.DataStorageQueryResult) + Query( + query cppbridge.HeadDataStorageQuery, + ) (*cppbridge.HeadDataStorageSerializedChunks, cppbridge.DataStorageQueryResult) // WithRLock calls fn on raw [cppbridge.HeadDataStorage] with read lock. WithRLock(fn func(ds *cppbridge.HeadDataStorage) error) error @@ -115,7 +121,7 @@ type Head[ AcquireQuery(ctx context.Context) (release func(), err error) // CreateTask create a task for operations on the [Head] shards. - CreateTask(taskName string, shardFn func(shard TShard) error) TGenericTask + CreateTask(taskName string, shardFn func(s TShard) error) TGenericTask // Enqueue the task to be executed on shards [Head]. Enqueue(t TGenericTask) From 2a4d346b7cccd5999b6a0b019e43e468456d30c9 Mon Sep 17 00:00:00 2001 From: Alexandr Yudin Date: Thu, 25 Sep 2025 15:14:45 +0000 Subject: [PATCH 50/96] small fix --- cmd/prometheus/main.go | 33 +------- cmd/prompptool/main.go | 10 +-- cmd/prompptool/walpp.go | 77 +++++++------------ pp-pkg/logger/logger.go | 31 ++++++++ pp/go/{storage => }/logger/logger.go | 0 pp/go/storage/appender/appender.go | 2 +- pp/go/storage/block/block_writer.go | 6 +- pp/go/storage/catalog/catalog.go | 2 +- pp/go/storage/catalog/gc.go | 2 +- pp/go/storage/catalog/log.go | 2 +- pp/go/storage/catalog/log_migrate.go | 2 +- pp/go/storage/head/head/head.go | 3 +- pp/go/storage/head/keeper/keeper.go | 2 +- pp/go/storage/head/services/commiter.go | 2 +- pp/go/storage/head/services/merger.go | 2 +- .../storage/head/services/metrics_updater.go | 2 +- pp/go/storage/head/services/persistener.go | 3 +- pp/go/storage/head/services/rotator.go | 3 +- .../head/shard/unloaded_data_storage.go | 2 +- pp/go/storage/loader.go | 30 +++++--- pp/go/storage/manager.go | 26 +------ pp/go/storage/querier/chunk_querier.go | 7 +- pp/go/storage/querier/querier.go | 8 +- pp/go/storage/querier/status_querier.go | 2 +- pp/go/storage/remotewriter/datasource.go | 2 +- pp/go/storage/remotewriter/iterator.go | 5 +- pp/go/storage/remotewriter/writeloop.go | 2 +- 27 files changed, 123 insertions(+), 145 deletions(-) create mode 100644 pp-pkg/logger/logger.go rename pp/go/{storage => }/logger/logger.go (100%) diff --git a/cmd/prometheus/main.go b/cmd/prometheus/main.go index 42c214e522..cde9eb0bb2 100644 --- a/cmd/prometheus/main.go +++ b/cmd/prometheus/main.go @@ -59,6 +59,7 @@ import ( pp_pkg_handler "github.com/prometheus/prometheus/pp-pkg/handler" rwprocessor "github.com/prometheus/prometheus/pp-pkg/handler/processor" + pp_pkg_logger "github.com/prometheus/prometheus/pp-pkg/logger" "github.com/prometheus/prometheus/pp-pkg/receiver" // PP_CHANGES.md: rebuild on cpp "github.com/prometheus/prometheus/pp-pkg/remote" // PP_CHANGES.md: rebuild on cpp "github.com/prometheus/prometheus/pp-pkg/scrape" // PP_CHANGES.md: rebuild on cpp @@ -721,7 +722,7 @@ func main() { os.Exit(1) } - pp_storage.InitLogHandler(log.With(logger, "component", "pp_storage")) + pp_pkg_logger.InitLogHandler(log.With(logger, "component", "pp")) reloadBlocksTriggerNotifier := pp_storage.NewReloadBlocksTriggerNotifier() cfg.tsdb.ReloadBlocksExternalTrigger = reloadBlocksTriggerNotifier @@ -780,36 +781,6 @@ func main() { os.Exit(1) } - // receiverReadyNotifier := ready.NewNotifiableNotifier() - // // create receiver - // receiver, err := receiver.NewReceiver( - // ctxReceiver, - // log.With(logger, "component", "receiver"), - // prometheus.DefaultRegisterer, - // receiverConfig, - // localStoragePath, - // cfgFile.RemoteWriteConfigs, - // localStoragePath, - // receiver.RotationInfo{ - // BlockDuration: time.Duration(cfg.tsdb.MinBlockDuration), - // Seed: cfgFile.GlobalConfig.ExternalLabels.Hash(), - // }, - // headCatalog, - // reloadBlocksTriggerNotifier, - // receiverReadyNotifier, - // time.Duration(cfg.WalCommitInterval), - // time.Duration(cfg.tsdb.RetentionDuration), - // time.Duration(cfg.HeadRetentionTimeout), - // // x3 ScrapeInterval timeout for write block - // time.Duration(cfgFile.GlobalConfig.ScrapeInterval*3), - // cfg.WalMaxSamplesPerSegment, - // appender.UnloadDataStorage, - // ) - // if err != nil { - // level.Error(logger).Log("msg", "failed to create a receiver", "err", err) - // os.Exit(1) - // } - remoteWriterReadyNotifier := ready.NewNotifiableNotifier() remoteWriter := remotewriter.New( dataDir, diff --git a/cmd/prompptool/main.go b/cmd/prompptool/main.go index 9d380ae4f9..532a7c4441 100644 --- a/cmd/prompptool/main.go +++ b/cmd/prompptool/main.go @@ -14,7 +14,7 @@ import ( "github.com/go-kit/log" "github.com/go-kit/log/level" - rlogger "github.com/prometheus/prometheus/pp/go/relabeler/logger" + pplogger "github.com/prometheus/prometheus/pp/go/logger" ) func main() { @@ -81,16 +81,16 @@ func initLogger(verbose bool) log.Logger { func initLogHandler(logger log.Logger) { logger = log.With(logger, "op_caller", log.Caller(4)) - rlogger.Debugf = func(template string, args ...interface{}) { + pplogger.Debugf = func(template string, args ...interface{}) { level.Debug(logger).Log("msg", fmt.Sprintf(template, args...)) } - rlogger.Infof = func(template string, args ...interface{}) { + pplogger.Infof = func(template string, args ...interface{}) { level.Info(logger).Log("msg", fmt.Sprintf(template, args...)) } - rlogger.Warnf = func(template string, args ...interface{}) { + pplogger.Warnf = func(template string, args ...interface{}) { level.Warn(logger).Log("msg", fmt.Sprintf(template, args...)) } - rlogger.Errorf = func(template string, args ...interface{}) { + pplogger.Errorf = func(template string, args ...interface{}) { level.Error(logger).Log("msg", fmt.Sprintf(template, args...)) } } diff --git a/cmd/prompptool/walpp.go b/cmd/prompptool/walpp.go index e857aa2d4b..df91dd8a3c 100644 --- a/cmd/prompptool/walpp.go +++ b/cmd/prompptool/walpp.go @@ -14,11 +14,9 @@ import ( "github.com/prometheus/client_golang/prometheus" "github.com/prometheus/common/model" - "github.com/prometheus/prometheus/pp/go/relabeler" - "github.com/prometheus/prometheus/pp/go/relabeler/block" - "github.com/prometheus/prometheus/pp/go/relabeler/config" - "github.com/prometheus/prometheus/pp/go/relabeler/head" - "github.com/prometheus/prometheus/pp/go/relabeler/head/catalog" + "github.com/prometheus/prometheus/pp/go/storage" + "github.com/prometheus/prometheus/pp/go/storage/block" + "github.com/prometheus/prometheus/pp/go/storage/catalog" ) type cmdWALPPToBlock struct { @@ -69,7 +67,7 @@ func (cmd *cmdWALPPToBlock) Do( if err != nil { return fmt.Errorf("failed init head catalog: %w", err) } - headRecords, err := headCatalog.List( + headRecords := headCatalog.List( func(record *catalog.Record) bool { return record.DeletedAt() == 0 && (record.Status() == catalog.StatusNew || record.Status() == catalog.StatusActive || record.Status() == catalog.StatusRotated) @@ -78,53 +76,37 @@ func (cmd *cmdWALPPToBlock) Do( return lhs.CreatedAt() < rhs.CreatedAt() }, ) - if err != nil { - return fmt.Errorf("failed listed head catalog: %w", err) - } + level.Debug(logger).Log("msg", "catalog records", "len", len(headRecords)) - var inputRelabelerConfig []*config.InputRelabelerConfig - bw := block.NewWriter(workingDir, block.DefaultChunkSegmentSize, time.Duration(cmd.blockDuration), registerer) + bw := block.NewWriter[*storage.ShardOnDisk]( + workingDir, + block.DefaultChunkSegmentSize, + time.Duration(cmd.blockDuration), + registerer, + ) + + loader := storage.NewLoader(workingDir, 0, registerer, time.Duration(0)) + for _, headRecord := range headRecords { if err := ctx.Err(); err != nil { return err } level.Debug(logger).Log("msg", "load head", "id", headRecord.ID(), "dir", headRecord.Dir()) - h, _, _, err := head.Load( - headRecord.ID(), - 0, - filepath.Join(workingDir, headRecord.Dir()), - inputRelabelerConfig, - headRecord.NumberOfShards(), - 0, - head.NoOpLastAppendedSegmentIDSetter{}, - registerer, - time.Duration(0), - ) - if err != nil { - level.Error(logger).Log( - "msg", "failed to load head", - "id", headRecord.ID(), - "dir", headRecord.Dir(), - "err", err, - ) - return err - } - h.Stop() + h, _ := loader.Load(headRecord, 0) level.Debug(logger).Log("msg", "write block", "id", headRecord.ID(), "dir", headRecord.Dir()) - tBlockWrite := h.CreateTask( - relabeler.BlockWrite, - func(shard relabeler.Shard) error { - _, err := bw.Write(shard) - return err - }, - relabeler.ForLSSTask, - ) - h.Enqueue(tBlockWrite) - if err = tBlockWrite.Wait(); err != nil { - return fmt.Errorf("failed to write tsdb block [id: %s, dir: %s]: %w", headRecord.ID(), headRecord.Dir(), err) + for shard := range h.RangeShards() { + if _, err := bw.Write(shard); err != nil { + _ = h.Close() + return fmt.Errorf( + "failed to write tsdb block [id: %s, dir: %s]: %w", + headRecord.ID(), + headRecord.Dir(), + err, + ) + } } if cmd.updateCatalog { @@ -169,7 +151,7 @@ func (cmd *cmdWALPPToBlock) clearing( level.Debug(logger).Log("msg", "catalog clearing: ended") }() - records, err := headCatalog.List( + records := headCatalog.List( func(record *catalog.Record) bool { return record.DeletedAt() == 0 && record.Status() == catalog.StatusPersisted }, @@ -177,9 +159,6 @@ func (cmd *cmdWALPPToBlock) clearing( return lhs.CreatedAt() < rhs.CreatedAt() }, ) - if err != nil { - return fmt.Errorf("failed listed head catalog: %w", err) - } for _, record := range records { if err := ctx.Err(); err != nil { @@ -199,7 +178,7 @@ func (cmd *cmdWALPPToBlock) clearing( ) } - if err = os.RemoveAll(filepath.Join(workingDir, record.Dir())); err != nil { + if err := os.RemoveAll(filepath.Join(workingDir, record.Dir())); err != nil { level.Error(logger).Log( "msg", "failed to delete head directory", "id", record.ID(), @@ -209,7 +188,7 @@ func (cmd *cmdWALPPToBlock) clearing( continue } - if err = headCatalog.Delete(record.ID()); err != nil { + if err := headCatalog.Delete(record.ID()); err != nil { level.Error(logger).Log( "msg", "failed to delete head record", "id", record.ID(), diff --git a/pp-pkg/logger/logger.go b/pp-pkg/logger/logger.go new file mode 100644 index 0000000000..072b916c06 --- /dev/null +++ b/pp-pkg/logger/logger.go @@ -0,0 +1,31 @@ +package logger + +import ( + "fmt" + + "github.com/go-kit/log" + "github.com/go-kit/log/level" + + "github.com/prometheus/prometheus/pp/go/logger" +) + +// InitLogHandler init log handler for pp. +func InitLogHandler(l log.Logger) { + l = log.With(l, "pp_caller", log.Caller(4)) + + logger.Debugf = func(template string, args ...any) { + _ = level.Debug(l).Log("msg", fmt.Sprintf(template, args...)) + } + + logger.Infof = func(template string, args ...any) { + _ = level.Info(l).Log("msg", fmt.Sprintf(template, args...)) + } + + logger.Warnf = func(template string, args ...any) { + _ = level.Warn(l).Log("msg", fmt.Sprintf(template, args...)) + } + + logger.Errorf = func(template string, args ...any) { + _ = level.Error(l).Log("msg", fmt.Sprintf(template, args...)) + } +} diff --git a/pp/go/storage/logger/logger.go b/pp/go/logger/logger.go similarity index 100% rename from pp/go/storage/logger/logger.go rename to pp/go/logger/logger.go diff --git a/pp/go/storage/appender/appender.go b/pp/go/storage/appender/appender.go index a183f036af..3ab29d4399 100644 --- a/pp/go/storage/appender/appender.go +++ b/pp/go/storage/appender/appender.go @@ -7,8 +7,8 @@ import ( "sync/atomic" "github.com/prometheus/prometheus/pp/go/cppbridge" + "github.com/prometheus/prometheus/pp/go/logger" "github.com/prometheus/prometheus/pp/go/storage/head/task" - "github.com/prometheus/prometheus/pp/go/storage/logger" ) const ( diff --git a/pp/go/storage/block/block_writer.go b/pp/go/storage/block/block_writer.go index 0e8d695295..0090ba4863 100644 --- a/pp/go/storage/block/block_writer.go +++ b/pp/go/storage/block/block_writer.go @@ -9,10 +9,12 @@ import ( "path/filepath" "github.com/oklog/ulid" - "github.com/prometheus/prometheus/pp/go/storage/logger" - "github.com/prometheus/prometheus/pp/go/util" + "github.com/prometheus/prometheus/tsdb" "github.com/prometheus/prometheus/tsdb/fileutil" + + "github.com/prometheus/prometheus/pp/go/logger" + "github.com/prometheus/prometheus/pp/go/util" ) const ( diff --git a/pp/go/storage/catalog/catalog.go b/pp/go/storage/catalog/catalog.go index 6eee902384..4e4e626e40 100644 --- a/pp/go/storage/catalog/catalog.go +++ b/pp/go/storage/catalog/catalog.go @@ -11,7 +11,7 @@ import ( "github.com/jonboulle/clockwork" "github.com/prometheus/client_golang/prometheus" - "github.com/prometheus/prometheus/pp/go/storage/logger" + "github.com/prometheus/prometheus/pp/go/logger" "github.com/prometheus/prometheus/pp/go/util" ) diff --git a/pp/go/storage/catalog/gc.go b/pp/go/storage/catalog/gc.go index 6b5192d9f0..845627611d 100644 --- a/pp/go/storage/catalog/gc.go +++ b/pp/go/storage/catalog/gc.go @@ -7,7 +7,7 @@ import ( "path/filepath" "time" - "github.com/prometheus/prometheus/pp/go/storage/logger" + "github.com/prometheus/prometheus/pp/go/logger" ) // diff --git a/pp/go/storage/catalog/log.go b/pp/go/storage/catalog/log.go index 7e71033a88..549da10fbb 100644 --- a/pp/go/storage/catalog/log.go +++ b/pp/go/storage/catalog/log.go @@ -8,7 +8,7 @@ import ( "os" "strings" - "github.com/prometheus/prometheus/pp/go/storage/logger" + "github.com/prometheus/prometheus/pp/go/logger" ) const ( diff --git a/pp/go/storage/catalog/log_migrate.go b/pp/go/storage/catalog/log_migrate.go index 82cce04fc0..b9e8a21763 100644 --- a/pp/go/storage/catalog/log_migrate.go +++ b/pp/go/storage/catalog/log_migrate.go @@ -6,7 +6,7 @@ import ( "io" "os" - "github.com/prometheus/prometheus/pp/go/storage/logger" + "github.com/prometheus/prometheus/pp/go/logger" "github.com/prometheus/prometheus/pp/go/util/optional" ) diff --git a/pp/go/storage/head/head/head.go b/pp/go/storage/head/head/head.go index dc89a9958e..fa6729cc6e 100644 --- a/pp/go/storage/head/head/head.go +++ b/pp/go/storage/head/head/head.go @@ -9,8 +9,9 @@ import ( "sync/atomic" "github.com/prometheus/client_golang/prometheus" + + "github.com/prometheus/prometheus/pp/go/logger" "github.com/prometheus/prometheus/pp/go/storage/head/task" - "github.com/prometheus/prometheus/pp/go/storage/logger" "github.com/prometheus/prometheus/pp/go/util" "github.com/prometheus/prometheus/pp/go/util/locker" ) diff --git a/pp/go/storage/head/keeper/keeper.go b/pp/go/storage/head/keeper/keeper.go index 0104194153..32c7af1570 100644 --- a/pp/go/storage/head/keeper/keeper.go +++ b/pp/go/storage/head/keeper/keeper.go @@ -6,7 +6,7 @@ import ( "sync" "time" - "github.com/prometheus/prometheus/pp/go/storage/logger" + "github.com/prometheus/prometheus/pp/go/logger" ) type addPolicy = uint8 diff --git a/pp/go/storage/head/services/commiter.go b/pp/go/storage/head/services/commiter.go index 38711a121b..eb6609c13d 100644 --- a/pp/go/storage/head/services/commiter.go +++ b/pp/go/storage/head/services/commiter.go @@ -3,7 +3,7 @@ package services import ( "context" - "github.com/prometheus/prometheus/pp/go/storage/logger" + "github.com/prometheus/prometheus/pp/go/logger" ) // diff --git a/pp/go/storage/head/services/merger.go b/pp/go/storage/head/services/merger.go index 6a2b6d95df..11627a7b3f 100644 --- a/pp/go/storage/head/services/merger.go +++ b/pp/go/storage/head/services/merger.go @@ -3,7 +3,7 @@ package services import ( "context" - "github.com/prometheus/prometheus/pp/go/storage/logger" + "github.com/prometheus/prometheus/pp/go/logger" ) // diff --git a/pp/go/storage/head/services/metrics_updater.go b/pp/go/storage/head/services/metrics_updater.go index 92d6d13dac..9301b7dfec 100644 --- a/pp/go/storage/head/services/metrics_updater.go +++ b/pp/go/storage/head/services/metrics_updater.go @@ -6,7 +6,7 @@ import ( "github.com/prometheus/client_golang/prometheus" - "github.com/prometheus/prometheus/pp/go/storage/logger" + "github.com/prometheus/prometheus/pp/go/logger" "github.com/prometheus/prometheus/pp/go/util" ) diff --git a/pp/go/storage/head/services/persistener.go b/pp/go/storage/head/services/persistener.go index 256274f3f1..e9292a4b87 100644 --- a/pp/go/storage/head/services/persistener.go +++ b/pp/go/storage/head/services/persistener.go @@ -6,9 +6,10 @@ import ( "github.com/jonboulle/clockwork" "github.com/prometheus/client_golang/prometheus" + "github.com/prometheus/prometheus/pp/go/cppbridge" + "github.com/prometheus/prometheus/pp/go/logger" "github.com/prometheus/prometheus/pp/go/storage/catalog" - "github.com/prometheus/prometheus/pp/go/storage/logger" "github.com/prometheus/prometheus/pp/go/util" ) diff --git a/pp/go/storage/head/services/rotator.go b/pp/go/storage/head/services/rotator.go index 816bf5bf8a..1ded614c04 100644 --- a/pp/go/storage/head/services/rotator.go +++ b/pp/go/storage/head/services/rotator.go @@ -6,7 +6,8 @@ import ( "time" "github.com/prometheus/client_golang/prometheus" - "github.com/prometheus/prometheus/pp/go/storage/logger" + + "github.com/prometheus/prometheus/pp/go/logger" "github.com/prometheus/prometheus/pp/go/util" ) diff --git a/pp/go/storage/head/shard/unloaded_data_storage.go b/pp/go/storage/head/shard/unloaded_data_storage.go index 07f76e6cfe..100495441a 100644 --- a/pp/go/storage/head/shard/unloaded_data_storage.go +++ b/pp/go/storage/head/shard/unloaded_data_storage.go @@ -8,7 +8,7 @@ import ( "os" "unsafe" - "github.com/prometheus/prometheus/pp/go/storage/logger" + "github.com/prometheus/prometheus/pp/go/logger" ) const ( diff --git a/pp/go/storage/loader.go b/pp/go/storage/loader.go index 6816a402f9..ddd97ac5c5 100644 --- a/pp/go/storage/loader.go +++ b/pp/go/storage/loader.go @@ -13,6 +13,7 @@ import ( "github.com/prometheus/client_golang/prometheus" "github.com/prometheus/prometheus/pp/go/cppbridge" + "github.com/prometheus/prometheus/pp/go/logger" "github.com/prometheus/prometheus/pp/go/storage/catalog" "github.com/prometheus/prometheus/pp/go/storage/head/head" "github.com/prometheus/prometheus/pp/go/storage/head/services" @@ -20,7 +21,6 @@ import ( "github.com/prometheus/prometheus/pp/go/storage/head/shard/wal" "github.com/prometheus/prometheus/pp/go/storage/head/shard/wal/reader" "github.com/prometheus/prometheus/pp/go/storage/head/shard/wal/writer" - "github.com/prometheus/prometheus/pp/go/storage/logger" "github.com/prometheus/prometheus/pp/go/util/optional" ) @@ -48,6 +48,10 @@ func NewLoader( } // Load [HeadOnDisk] from [WalOnDisk] by head ID. +// +//revive:disable-next-line:cognitive-complexity // function is not complicated +//revive:disable-next-line:function-length // long but readable. +//revive:disable-next-line:cyclomatic // but readable func (l *Loader) Load( headRecord *catalog.Record, generation uint64, @@ -112,7 +116,7 @@ func (l *Loader) Load( } logger.Errorf( - "head: %s number of segments mismatched", + "head: %s number of segments mismatched: last appended=%d, number of segments read=%d", headRecord.ID(), lastAppendedSegmentID, numberOfSegmentsRead.Value(), @@ -212,7 +216,7 @@ func (l *ShardDataLoader) Load() (err error) { shardWalFile, err := os.OpenFile( //nolint:gosec // need this permissions GetShardWalFilename(l.dir, l.shardID), os.O_RDWR, - 0o666, + 0o666, //revive:disable-line:add-constant // file permissions simple readable as octa-number ) if err != nil { return err @@ -226,7 +230,9 @@ func (l *ShardDataLoader) Load() (err error) { queriedSeriesStorageIsEmpty := true if l.unloadDataStorageInterval > 0 { - l.shardData.unloadedDataStorage = shard.NewUnloadedDataStorage(shard.NewFileStorage(GetUnloadedDataStorageFilename(l.dir, l.shardID))) + l.shardData.unloadedDataStorage = shard.NewUnloadedDataStorage( + shard.NewFileStorage(GetUnloadedDataStorageFilename(l.dir, l.shardID)), + ) queriedSeriesStorageIsEmpty, _ = l.loadQueriedSeries() } @@ -235,11 +241,7 @@ func (l *ShardDataLoader) Load() (err error) { return err } - if err = l.createShardWal(shardWalFile, decoder); err != nil { - return err - } - - return nil + return l.createShardWal(shardWalFile, decoder) } func (l *ShardDataLoader) loadWalFile( @@ -272,7 +274,12 @@ func (l *ShardDataLoader) loadWalFile( } func (l *ShardDataLoader) createShardWal(shardWalFile *os.File, walDecoder *cppbridge.HeadWalDecoder) error { - if sw, err := writer.NewBuffered(l.shardID, shardWalFile, writer.WriteSegment[*cppbridge.EncodedSegment], l.notifier); err != nil { + if sw, err := writer.NewBuffered( + l.shardID, + shardWalFile, + writer.WriteSegment[*cppbridge.EncodedSegment], + l.notifier, + ); err != nil { return err } else { l.notifier.Set(l.shardID, l.shardData.numberOfSegments) @@ -362,14 +369,17 @@ func (l *ShardDataLoader) loadQueriedSeries() (bool, error) { return false, nil } +// GetShardWalFilename returns shard's Wal file name. func GetShardWalFilename(dir string, shardID uint16) string { return filepath.Join(dir, fmt.Sprintf("shard_%d.wal", shardID)) } +// GetUnloadedDataStorageFilename returns unloaded DataStorage file name. func GetUnloadedDataStorageFilename(dir string, shardID uint16) string { return filepath.Join(dir, fmt.Sprintf("unloaded_%d.ds", shardID)) } +// GetQueriedSeriesStorageFilename returns queried series storage file name. func GetQueriedSeriesStorageFilename(dir string, shardID uint16, index uint8) string { return filepath.Join(dir, fmt.Sprintf("queried_series_%d_%d.ds", shardID, index)) } diff --git a/pp/go/storage/manager.go b/pp/go/storage/manager.go index 0dc1a6b621..73ce7f406f 100644 --- a/pp/go/storage/manager.go +++ b/pp/go/storage/manager.go @@ -10,21 +10,20 @@ import ( "sync/atomic" "time" - "github.com/go-kit/log" - "github.com/go-kit/log/level" "github.com/jonboulle/clockwork" "github.com/oklog/run" "github.com/prometheus/client_golang/prometheus" "github.com/prometheus/prometheus/config" + "github.com/prometheus/prometheus/pp/go/cppbridge" + "github.com/prometheus/prometheus/pp/go/logger" "github.com/prometheus/prometheus/pp/go/storage/block" "github.com/prometheus/prometheus/pp/go/storage/catalog" "github.com/prometheus/prometheus/pp/go/storage/head/container" "github.com/prometheus/prometheus/pp/go/storage/head/keeper" "github.com/prometheus/prometheus/pp/go/storage/head/proxy" "github.com/prometheus/prometheus/pp/go/storage/head/services" - "github.com/prometheus/prometheus/pp/go/storage/logger" "github.com/prometheus/prometheus/pp/go/storage/mediator" "github.com/prometheus/prometheus/pp/go/storage/querier" "github.com/prometheus/prometheus/pp/go/storage/ready" @@ -375,27 +374,6 @@ func (m *Manager) close() { } } -// InitLogHandler init log handler for pp. -func InitLogHandler(l log.Logger) { - l = log.With(l, "pp_caller", log.Caller(4)) - - logger.Debugf = func(template string, args ...any) { - _ = level.Debug(l).Log("msg", fmt.Sprintf(template, args...)) - } - - logger.Infof = func(template string, args ...any) { - _ = level.Info(l).Log("msg", fmt.Sprintf(template, args...)) - } - - logger.Warnf = func(template string, args ...any) { - _ = level.Warn(l).Log("msg", fmt.Sprintf(template, args...)) - } - - logger.Errorf = func(template string, args ...any) { - _ = level.Error(l).Log("msg", fmt.Sprintf(template, args...)) - } -} - // // headInformer // diff --git a/pp/go/storage/querier/chunk_querier.go b/pp/go/storage/querier/chunk_querier.go index db1ef151b5..5a4b3b9049 100644 --- a/pp/go/storage/querier/chunk_querier.go +++ b/pp/go/storage/querier/chunk_querier.go @@ -5,11 +5,12 @@ import ( "errors" "github.com/prometheus/prometheus/model/labels" - "github.com/prometheus/prometheus/pp/go/cppbridge" - "github.com/prometheus/prometheus/pp/go/storage/logger" - "github.com/prometheus/prometheus/pp/go/util/locker" "github.com/prometheus/prometheus/storage" "github.com/prometheus/prometheus/util/annotations" + + "github.com/prometheus/prometheus/pp/go/cppbridge" + "github.com/prometheus/prometheus/pp/go/logger" + "github.com/prometheus/prometheus/pp/go/util/locker" ) const ( diff --git a/pp/go/storage/querier/querier.go b/pp/go/storage/querier/querier.go index 396bdbbcaa..8b83a910ec 100644 --- a/pp/go/storage/querier/querier.go +++ b/pp/go/storage/querier/querier.go @@ -8,13 +8,15 @@ import ( "time" "github.com/prometheus/client_golang/prometheus" + "github.com/prometheus/prometheus/model/labels" + "github.com/prometheus/prometheus/storage" + "github.com/prometheus/prometheus/util/annotations" + "github.com/prometheus/prometheus/pp/go/cppbridge" + "github.com/prometheus/prometheus/pp/go/logger" "github.com/prometheus/prometheus/pp/go/model" - "github.com/prometheus/prometheus/pp/go/storage/logger" "github.com/prometheus/prometheus/pp/go/util/locker" - "github.com/prometheus/prometheus/storage" - "github.com/prometheus/prometheus/util/annotations" ) const ( diff --git a/pp/go/storage/querier/status_querier.go b/pp/go/storage/querier/status_querier.go index a46022af64..893ddd0e7c 100644 --- a/pp/go/storage/querier/status_querier.go +++ b/pp/go/storage/querier/status_querier.go @@ -7,8 +7,8 @@ import ( "sort" "github.com/prometheus/prometheus/pp/go/cppbridge" + "github.com/prometheus/prometheus/pp/go/logger" "github.com/prometheus/prometheus/pp/go/storage/head/task" - "github.com/prometheus/prometheus/pp/go/storage/logger" "github.com/prometheus/prometheus/pp/go/util/locker" ) diff --git a/pp/go/storage/remotewriter/datasource.go b/pp/go/storage/remotewriter/datasource.go index a2e00b9f95..4bdae97163 100644 --- a/pp/go/storage/remotewriter/datasource.go +++ b/pp/go/storage/remotewriter/datasource.go @@ -15,8 +15,8 @@ import ( "github.com/prometheus/prometheus/model/labels" "github.com/prometheus/prometheus/model/relabel" "github.com/prometheus/prometheus/pp/go/cppbridge" + "github.com/prometheus/prometheus/pp/go/logger" "github.com/prometheus/prometheus/pp/go/storage/catalog" - "github.com/prometheus/prometheus/pp/go/storage/logger" "github.com/prometheus/prometheus/pp/go/util/optional" ) diff --git a/pp/go/storage/remotewriter/iterator.go b/pp/go/storage/remotewriter/iterator.go index ac9c766f8e..368340799d 100644 --- a/pp/go/storage/remotewriter/iterator.go +++ b/pp/go/storage/remotewriter/iterator.go @@ -12,9 +12,10 @@ import ( "github.com/jonboulle/clockwork" "github.com/prometheus/prometheus/config" - "github.com/prometheus/prometheus/pp/go/cppbridge" - "github.com/prometheus/prometheus/pp/go/storage/logger" "github.com/prometheus/prometheus/storage/remote" + + "github.com/prometheus/prometheus/pp/go/cppbridge" + "github.com/prometheus/prometheus/pp/go/logger" ) type DataSource interface { diff --git a/pp/go/storage/remotewriter/writeloop.go b/pp/go/storage/remotewriter/writeloop.go index eb88ae9bf5..dde7d95e83 100644 --- a/pp/go/storage/remotewriter/writeloop.go +++ b/pp/go/storage/remotewriter/writeloop.go @@ -12,8 +12,8 @@ import ( "github.com/prometheus/prometheus/storage/remote" "github.com/prometheus/prometheus/pp/go/cppbridge" + "github.com/prometheus/prometheus/pp/go/logger" "github.com/prometheus/prometheus/pp/go/storage/catalog" - "github.com/prometheus/prometheus/pp/go/storage/logger" ) const defaultDelay = time.Second * 5 From 0527236149cc238a177c1463a261a38e9ed9bc71 Mon Sep 17 00:00:00 2001 From: Alexandr Yudin Date: Thu, 25 Sep 2025 16:11:42 +0000 Subject: [PATCH 51/96] small fix --- cmd/prometheus/main.go | 15 +- pp-pkg/logger/logger.go | 13 +- pp-pkg/receiver/appender.go | 162 +- pp-pkg/receiver/receiver.go | 1818 ++++++++--------- pp/go/storage/builder.go | 2 + pp/go/storage/catalog/catalog.go | 6 +- pp/go/storage/catalog/record.go | 1 + pp/go/storage/head/keeper/keeper_test.go | 27 - pp/go/storage/head/services/persistener.go | 13 +- pp/go/storage/head/shard/file_storage.go | 71 +- .../head/shard/unloaded_data_storage.go | 5 +- .../head/shard/unloaded_data_storage_test.go | 80 +- pp/go/storage/loader.go | 2 + pp/go/storage/loader_test.go | 4 +- pp/go/storage/manager.go | 27 +- pp/go/storage/storagetest/fixtures.go | 11 +- pp/go/storage/types.go | 9 - 17 files changed, 1126 insertions(+), 1140 deletions(-) diff --git a/cmd/prometheus/main.go b/cmd/prometheus/main.go index cde9eb0bb2..436e4f9f5c 100644 --- a/cmd/prometheus/main.go +++ b/cmd/prometheus/main.go @@ -59,8 +59,7 @@ import ( pp_pkg_handler "github.com/prometheus/prometheus/pp-pkg/handler" rwprocessor "github.com/prometheus/prometheus/pp-pkg/handler/processor" - pp_pkg_logger "github.com/prometheus/prometheus/pp-pkg/logger" - "github.com/prometheus/prometheus/pp-pkg/receiver" // PP_CHANGES.md: rebuild on cpp + pp_pkg_logger "github.com/prometheus/prometheus/pp-pkg/logger" // PP_CHANGES.md: rebuild on cpp "github.com/prometheus/prometheus/pp-pkg/remote" // PP_CHANGES.md: rebuild on cpp "github.com/prometheus/prometheus/pp-pkg/scrape" // PP_CHANGES.md: rebuild on cpp pp_pkg_storage "github.com/prometheus/prometheus/pp-pkg/storage" // PP_CHANGES.md: rebuild on cpp @@ -2107,7 +2106,7 @@ func readPromPPFeatures(logger log.Logger) { if fvalue == "" { level.Error(logger).Log( "msg", "[FEATURE] The default number of shards is empty, no changes.", - "default_number_of_shards", receiver.DefaultNumberOfShards, + "default_number_of_shards", pp_storage.DefaultNumberOfShards, ) continue @@ -2118,27 +2117,27 @@ func readPromPPFeatures(logger log.Logger) { case err != nil: level.Error(logger).Log( "msg", "[FEATURE] Error parsing head_numbehead_default_number_of_shardsr_of_shards value", - "default_number_of_shards", receiver.DefaultNumberOfShards, + "default_number_of_shards", pp_storage.DefaultNumberOfShards, "err", err, ) case v > math.MaxUint16: level.Error(logger).Log( "msg", "[FEATURE] The default number of shards is overflow(max 65535), no changes.", - "default_number_of_shards", receiver.DefaultNumberOfShards, + "default_number_of_shards", pp_storage.DefaultNumberOfShards, ) case v < 1: level.Error(logger).Log( "msg", "[FEATURE] The default number of shards is incorrect(min 1), no changes.", - "default_number_of_shards", receiver.DefaultNumberOfShards, + "default_number_of_shards", pp_storage.DefaultNumberOfShards, ) default: - receiver.DefaultNumberOfShards = uint16(v) + pp_storage.DefaultNumberOfShards = uint16(v) level.Info(logger).Log( "msg", "[FEATURE] Changed default number of shards.", - "default_number_of_shards", receiver.DefaultNumberOfShards, + "default_number_of_shards", pp_storage.DefaultNumberOfShards, ) } diff --git a/pp-pkg/logger/logger.go b/pp-pkg/logger/logger.go index 072b916c06..8186357244 100644 --- a/pp-pkg/logger/logger.go +++ b/pp-pkg/logger/logger.go @@ -9,23 +9,26 @@ import ( "github.com/prometheus/prometheus/pp/go/logger" ) +const msg = "msg" + // InitLogHandler init log handler for pp. func InitLogHandler(l log.Logger) { - l = log.With(l, "pp_caller", log.Caller(4)) + l = log.With(l, "pp_caller", log.Caller(4)) //revive:disable-line:add-constant // caller id logger.Debugf = func(template string, args ...any) { - _ = level.Debug(l).Log("msg", fmt.Sprintf(template, args...)) + // _ = level.Debug(l).Log(msg, fmt.Sprintf(template, args...)) + _ = level.Info(l).Log(msg, fmt.Sprintf(template, args...)) } logger.Infof = func(template string, args ...any) { - _ = level.Info(l).Log("msg", fmt.Sprintf(template, args...)) + _ = level.Info(l).Log(msg, fmt.Sprintf(template, args...)) } logger.Warnf = func(template string, args ...any) { - _ = level.Warn(l).Log("msg", fmt.Sprintf(template, args...)) + _ = level.Warn(l).Log(msg, fmt.Sprintf(template, args...)) } logger.Errorf = func(template string, args ...any) { - _ = level.Error(l).Log("msg", fmt.Sprintf(template, args...)) + _ = level.Error(l).Log(msg, fmt.Sprintf(template, args...)) } } diff --git a/pp-pkg/receiver/appender.go b/pp-pkg/receiver/appender.go index 9df2268890..42972027c3 100644 --- a/pp-pkg/receiver/appender.go +++ b/pp-pkg/receiver/appender.go @@ -1,83 +1,83 @@ package receiver -import ( - "context" - - "github.com/prometheus/prometheus/model/exemplar" - "github.com/prometheus/prometheus/model/histogram" - "github.com/prometheus/prometheus/model/labels" - "github.com/prometheus/prometheus/model/metadata" - "github.com/prometheus/prometheus/pp/go/model" - "github.com/prometheus/prometheus/storage" -) - -type timeSeriesData struct { - timeSeries []model.TimeSeries -} - -func (d *timeSeriesData) TimeSeries() []model.TimeSeries { - return d.timeSeries -} - -func (d *timeSeriesData) Destroy() { - d.timeSeries = nil -} - -type promAppender struct { - ctx context.Context - receiver *Receiver - relabelerID string - data *timeSeriesData -} - -func newPromAppender(ctx context.Context, receiver *Receiver, relabelerID string) *promAppender { - return &promAppender{ - ctx: ctx, - receiver: receiver, - relabelerID: relabelerID, - data: &timeSeriesData{}, - } -} - -func (a *promAppender) Append(ref storage.SeriesRef, l labels.Labels, t int64, v float64) (storage.SeriesRef, error) { - lsb := model.NewLabelSetBuilder() - l.Range(func(label labels.Label) { - lsb.Add(label.Name, label.Value) - }) - - a.data.timeSeries = append(a.data.timeSeries, model.TimeSeries{ - LabelSet: lsb.Build(), - Timestamp: uint64(t), - Value: v, - }) - return 0, nil -} - -func (a *promAppender) Commit() error { - if len(a.data.timeSeries) == 0 { - return nil - } - - _, err := a.receiver.AppendTimeSeries(a.ctx, a.data, nil, a.relabelerID, false) - return err -} - -func (a *promAppender) Rollback() error { - return nil -} - -func (a *promAppender) AppendExemplar(ref storage.SeriesRef, l labels.Labels, e exemplar.Exemplar) (storage.SeriesRef, error) { - return 0, nil -} - -func (a *promAppender) AppendHistogram(ref storage.SeriesRef, l labels.Labels, t int64, h *histogram.Histogram, fh *histogram.FloatHistogram) (storage.SeriesRef, error) { - return 0, nil -} - -func (a *promAppender) UpdateMetadata(ref storage.SeriesRef, l labels.Labels, m metadata.Metadata) (storage.SeriesRef, error) { - return 0, nil -} - -func (a *promAppender) AppendCTZeroSample(ref storage.SeriesRef, l labels.Labels, t, ct int64) (storage.SeriesRef, error) { - return 0, nil -} +// import ( +// "context" + +// "github.com/prometheus/prometheus/model/exemplar" +// "github.com/prometheus/prometheus/model/histogram" +// "github.com/prometheus/prometheus/model/labels" +// "github.com/prometheus/prometheus/model/metadata" +// "github.com/prometheus/prometheus/pp/go/model" +// "github.com/prometheus/prometheus/storage" +// ) + +// type timeSeriesData struct { +// timeSeries []model.TimeSeries +// } + +// func (d *timeSeriesData) TimeSeries() []model.TimeSeries { +// return d.timeSeries +// } + +// func (d *timeSeriesData) Destroy() { +// d.timeSeries = nil +// } + +// type promAppender struct { +// ctx context.Context +// receiver *Receiver +// relabelerID string +// data *timeSeriesData +// } + +// func newPromAppender(ctx context.Context, receiver *Receiver, relabelerID string) *promAppender { +// return &promAppender{ +// ctx: ctx, +// receiver: receiver, +// relabelerID: relabelerID, +// data: &timeSeriesData{}, +// } +// } + +// func (a *promAppender) Append(ref storage.SeriesRef, l labels.Labels, t int64, v float64) (storage.SeriesRef, error) { +// lsb := model.NewLabelSetBuilder() +// l.Range(func(label labels.Label) { +// lsb.Add(label.Name, label.Value) +// }) + +// a.data.timeSeries = append(a.data.timeSeries, model.TimeSeries{ +// LabelSet: lsb.Build(), +// Timestamp: uint64(t), +// Value: v, +// }) +// return 0, nil +// } + +// func (a *promAppender) Commit() error { +// if len(a.data.timeSeries) == 0 { +// return nil +// } + +// _, err := a.receiver.AppendTimeSeries(a.ctx, a.data, nil, a.relabelerID, false) +// return err +// } + +// func (a *promAppender) Rollback() error { +// return nil +// } + +// func (a *promAppender) AppendExemplar(ref storage.SeriesRef, l labels.Labels, e exemplar.Exemplar) (storage.SeriesRef, error) { +// return 0, nil +// } + +// func (a *promAppender) AppendHistogram(ref storage.SeriesRef, l labels.Labels, t int64, h *histogram.Histogram, fh *histogram.FloatHistogram) (storage.SeriesRef, error) { +// return 0, nil +// } + +// func (a *promAppender) UpdateMetadata(ref storage.SeriesRef, l labels.Labels, m metadata.Metadata) (storage.SeriesRef, error) { +// return 0, nil +// } + +// func (a *promAppender) AppendCTZeroSample(ref storage.SeriesRef, l labels.Labels, t, ct int64) (storage.SeriesRef, error) { +// return 0, nil +// } diff --git a/pp-pkg/receiver/receiver.go b/pp-pkg/receiver/receiver.go index 45acb7eb32..c0cb546c58 100644 --- a/pp-pkg/receiver/receiver.go +++ b/pp-pkg/receiver/receiver.go @@ -2,912 +2,912 @@ package receiver -import ( - "context" - "errors" - "fmt" - "os" - "path" - "path/filepath" - "time" - - "github.com/go-kit/log" - "github.com/go-kit/log/level" - "github.com/google/uuid" - "github.com/jonboulle/clockwork" - "github.com/prometheus/client_golang/prometheus" - common_config "github.com/prometheus/common/config" - "go.uber.org/atomic" - "gopkg.in/yaml.v2" - - prom_config "github.com/prometheus/prometheus/config" - "github.com/prometheus/prometheus/model/labels" - "github.com/prometheus/prometheus/model/relabel" - pp_pkg_config "github.com/prometheus/prometheus/pp-pkg/config" - "github.com/prometheus/prometheus/pp-pkg/dialer" - "github.com/prometheus/prometheus/pp/go/cppbridge" - "github.com/prometheus/prometheus/pp/go/relabeler" - "github.com/prometheus/prometheus/pp/go/relabeler/appender" - "github.com/prometheus/prometheus/pp/go/relabeler/block" - "github.com/prometheus/prometheus/pp/go/relabeler/config" - "github.com/prometheus/prometheus/pp/go/relabeler/distributor" - "github.com/prometheus/prometheus/pp/go/relabeler/head/catalog" - headmanager "github.com/prometheus/prometheus/pp/go/relabeler/head/manager" - "github.com/prometheus/prometheus/pp/go/relabeler/head/ready" - "github.com/prometheus/prometheus/pp/go/relabeler/headcontainer" - rlogger "github.com/prometheus/prometheus/pp/go/relabeler/logger" - "github.com/prometheus/prometheus/pp/go/relabeler/querier" - "github.com/prometheus/prometheus/pp/go/util" - "github.com/prometheus/prometheus/storage" - "github.com/prometheus/prometheus/util/annotations" -) - -const defaultShutdownTimeout = 40 * time.Second - -var DefaultNumberOfShards uint16 = 2 - -type HeadConfig struct { - inputRelabelerConfigs []*config.InputRelabelerConfig - numberOfShards uint16 -} - -type HeadConfigStorage struct { - ptr atomic.Pointer[HeadConfig] -} - -func (s *HeadConfigStorage) Load() *HeadConfig { - return s.ptr.Load() -} - -func (s *HeadConfigStorage) Get() ([]*config.InputRelabelerConfig, uint16) { - cfg := s.ptr.Load() - return cfg.inputRelabelerConfigs, cfg.numberOfShards -} - -func (s *HeadConfigStorage) Store(headConfig *HeadConfig) { - s.ptr.Store(headConfig) -} - -type Receiver struct { - ctx context.Context - - distributor *distributor.Distributor - // appender *appender.QueryableAppender - activeHead *headcontainer.Active - storage *appender.QueryableStorage - rotator *appender.RotateCommiter - metricsWriteTrigger *appender.MetricsWriteTrigger - - headConfigStorage *HeadConfigStorage - hashdexFactory relabeler.HashdexFactory - hashdexLimits cppbridge.WALHashdexLimits - haTracker relabeler.HATracker - clock clockwork.Clock - registerer prometheus.Registerer - logger log.Logger - workingDir string - clientID string - cgogc *cppbridge.CGOGC - shutdowner *util.GracefulShutdowner - - activeQuerierMetrics *querier.Metrics - storageQuerierMetrics *querier.Metrics -} - -type RotationInfo struct { - BlockDuration time.Duration - Seed uint64 -} - -type HeadActivator struct { - catalog *catalog.Catalog -} - -func newHeadActivator(catalog *catalog.Catalog) *HeadActivator { - return &HeadActivator{catalog: catalog} -} - -func (ha *HeadActivator) Activate(headID string) error { - _, err := ha.catalog.SetStatus(headID, catalog.StatusActive) - return err -} - -func NewReceiver( - ctx context.Context, - logger log.Logger, - registerer prometheus.Registerer, - receiverCfg *pp_pkg_config.RemoteWriteReceiverConfig, - workingDir string, - remoteWriteCfgs []*prom_config.PPRemoteWriteConfig, - dataDir string, - rotationInfo RotationInfo, - headCatalog *catalog.Catalog, - triggerNotifier *ReloadBlocksTriggerNotifier, - readyNotifier ready.Notifier, - commitInterval time.Duration, - maxRetentionDuration time.Duration, - headRetentionTimeout time.Duration, - writeTimeout time.Duration, - maxSegmentSize uint32, - unloadDataStorage bool, -) (*Receiver, error) { - if logger == nil { - logger = log.NewNopLogger() - } - - clientID, err := readClientID(logger, workingDir) - if err != nil { - level.Error(logger).Log("msg", "failed read client id", "err", err) - return nil, err - } - - initLogHandler(logger) - clock := clockwork.NewRealClock() - - numberOfShards := receiverCfg.NumberOfShards - if numberOfShards == 0 { - numberOfShards = DefaultNumberOfShards - } - - destinationGroups, err := makeDestinationGroups( - ctx, - clock, - registerer, - workingDir, - clientID, - remoteWriteCfgs, - numberOfShards, - ) - if err != nil { - level.Error(logger).Log("msg", "failed to init DestinationGroups", "err", err) - return nil, err - } - - headConfigStorage := &HeadConfigStorage{} - - headConfigStorage.Store(&HeadConfig{ - inputRelabelerConfigs: receiverCfg.Configs, - numberOfShards: numberOfShards, - }) - - dataDir, err = filepath.Abs(dataDir) - if err != nil { - return nil, err - } - - var unloadDataStorageInterval time.Duration - if unloadDataStorage { - unloadDataStorageInterval = appender.DefaultMergeDuration - } - - headManager, err := headmanager.New( - dataDir, - clock, - headConfigStorage, - headCatalog, - maxSegmentSize, - registerer, - unloadDataStorageInterval, - ) - if err != nil { - return nil, fmt.Errorf("failed to create head manager: %w", err) - } - - currentHead, rotatedHeads, err := headManager.Restore(rotationInfo.BlockDuration, unloadDataStorageInterval) - if err != nil { - return nil, fmt.Errorf("failed to restore heads: %w", err) - } - readyNotifier.NotifyReady() - storageQuerierMetrics := querier.NewMetrics(registerer, querier.QueryableStorageSource) - queryableStorage := appender.NewQueryableStorageWithWriteNotifier( - block.NewWriter(dataDir, block.DefaultChunkSegmentSize, rotationInfo.BlockDuration, registerer), - registerer, - storageQuerierMetrics, - triggerNotifier, - clock, - maxRetentionDuration, - headRetentionTimeout, - writeTimeout, - rotatedHeads..., - ) - - var containeredHead relabeler.Head - containeredHead = headcontainer.NewRotatable(currentHead, queryableStorage, headManager, newHeadActivator(headCatalog)) - - if len(os.Getenv("OPCORE_ROTATION_HEAP_DEBUG")) > 0 { - containeredHead = headcontainer.NewHeapProfileWritable( - containeredHead, - util.NewHeapProfileWriter(filepath.Join(dataDir, "heap_profiles")), - ) - } - - dstrb := distributor.NewDistributor(*destinationGroups) - activeQuerierMetrics := querier.NewMetrics(registerer, querier.QueryableAppenderSource) - activeHead := headcontainer.NewActive(containeredHead, registerer) - // app := appender.NewQueryableAppender( - // ctx, - // appenderHead, - // dstrb, - // activeQuerierMetrics, - // registerer, - // ) - mwt := appender.NewMetricsWriteTrigger(ctx, appender.DefaultMetricWriteInterval, activeHead, queryableStorage) - - r := &Receiver{ - ctx: ctx, - distributor: dstrb, - // appender: app, - activeHead: activeHead, - storage: queryableStorage, - headConfigStorage: headConfigStorage, - rotator: appender.NewRotateCommiter( - ctx, - activeHead, - relabeler.NewRotateTimerWithSeed(clock, rotationInfo.BlockDuration, rotationInfo.Seed), - appender.NewConstantIntervalTimer(clock, commitInterval), - appender.NewConstantIntervalTimer(clock, appender.DefaultMergeDuration), - unloadDataStorage, - registerer, - ), - - metricsWriteTrigger: mwt, - hashdexFactory: cppbridge.HashdexFactory{}, - hashdexLimits: cppbridge.DefaultWALHashdexLimits(), - haTracker: relabeler.NewHighAvailabilityTracker(ctx, registerer, clock), - clock: clock, - registerer: registerer, - logger: logger, - workingDir: workingDir, - clientID: clientID, - cgogc: cppbridge.NewCGOGC(registerer), - shutdowner: util.NewGracefulShutdowner(), - - activeQuerierMetrics: activeQuerierMetrics, - storageQuerierMetrics: storageQuerierMetrics, - } - - level.Info(logger).Log("msg", "created") - - return r, nil -} - -// AppendHashdex append incoming Hashdex data to relabeling. -func (rr *Receiver) AppendHashdex( - ctx context.Context, - hashdex cppbridge.ShardedData, - relabelerID string, - commitToWal bool, -) error { - if rr.haTracker.IsDrop(hashdex.Cluster(), hashdex.Replica()) { - return nil - } - incomingData := &relabeler.IncomingData{Hashdex: hashdex} - _, err := rr.activeHead.Append(ctx, incomingData, nil, relabelerID, commitToWal) - return err -} - -// AppendSnappyProtobuf append compressed via snappy Protobuf data to relabeling hashdex data. -func (rr *Receiver) AppendSnappyProtobuf( - ctx context.Context, - compressedData relabeler.ProtobufData, - relabelerID string, - commitToWal bool, -) error { - hx, err := cppbridge.NewWALSnappyProtobufHashdex(compressedData.Bytes(), rr.hashdexLimits) - compressedData.Destroy() - if err != nil { - return err - } - - if rr.haTracker.IsDrop(hx.Cluster(), hx.Replica()) { - return nil - } - - incomingData := &relabeler.IncomingData{Hashdex: hx} - _, err = rr.activeHead.Append(ctx, incomingData, nil, relabelerID, commitToWal) - return err -} - -// AppendTimeSeries append TimeSeries data to relabeling hashdex data. -func (rr *Receiver) AppendTimeSeries( - ctx context.Context, - data relabeler.TimeSeriesData, - state *cppbridge.State, - relabelerID string, - commitToWal bool, -) (cppbridge.RelabelerStats, error) { - hx, err := rr.hashdexFactory.GoModel(data.TimeSeries(), rr.hashdexLimits) - if err != nil { - data.Destroy() - return cppbridge.RelabelerStats{}, err - } - - if rr.haTracker.IsDrop(hx.Cluster(), hx.Replica()) { - data.Destroy() - return cppbridge.RelabelerStats{}, nil - } - incomingData := &relabeler.IncomingData{Hashdex: hx, Data: data} - return rr.activeHead.Append( - ctx, - incomingData, - state, - relabelerID, - commitToWal, - ) -} - -func (rr *Receiver) AppendTimeSeriesHashdex( - ctx context.Context, - hashdex cppbridge.ShardedData, - state *cppbridge.State, - relabelerID string, - commitToWal bool, -) (cppbridge.RelabelerStats, error) { - return rr.activeHead.Append( - ctx, - &relabeler.IncomingData{Hashdex: hashdex}, - state, - relabelerID, - commitToWal, - ) -} - -// Appender create a new appender for head. -func (rr *Receiver) Appender(ctx context.Context) storage.Appender { - return newPromAppender(ctx, rr, prom_config.TransparentRelabeler) -} - -// ApplyConfig update config. -func (rr *Receiver) ApplyConfig(cfg *prom_config.Config) error { - level.Info(rr.logger).Log("msg", "reconfiguration start") - defer level.Info(rr.logger).Log("msg", "reconfiguration completed") - - rCfg, err := cfg.GetReceiverConfig() - if err != nil { - return err - } - - numberOfShards := rCfg.NumberOfShards - if numberOfShards == 0 { - numberOfShards = DefaultNumberOfShards - } - - rr.headConfigStorage.Store(&HeadConfig{ - inputRelabelerConfigs: rCfg.Configs, - numberOfShards: numberOfShards, - }) - - err = rr.activeHead.Reconfigure( - rr.ctx, - HeadConfigureFunc(func(head relabeler.Head) error { - return head.Reconfigure(rr.ctx, rCfg.Configs, numberOfShards) - }), - // DistributorConfigureFunc(func(dstrb relabeler.Distributor) error { - // mxdgupds := new(sync.Mutex) - // dgupds, err := makeDestinationGroupUpdates( - // cfg.RemoteWriteConfigs, - // rr.workingDir, - // rr.clientID, - // numberOfShards, - // ) - // if err != nil { - // level.Error(rr.logger).Log("msg", "failed to init destination group update", "err", err) - // return err - // } - // mxDelete := new(sync.Mutex) - // toDelete := []int{} - - // dgs := dstrb.DestinationGroups() - // if err = dgs.RangeGo(func(destinationGroupID int, dg *relabeler.DestinationGroup) error { - // var rangeErr error - // dgu, ok := dgupds[dg.Name()] - // if !ok { - // mxDelete.Lock() - // toDelete = append(toDelete, destinationGroupID) - // mxDelete.Unlock() - // ctxShutdown, cancel := context.WithTimeout(rr.ctx, defaultShutdownTimeout) - // if rangeErr = dg.Shutdown(ctxShutdown); err != nil { - // level.Error(rr.logger).Log("msg", "failed shutdown DestinationGroup", "err", rangeErr) - // } - // cancel() - // return nil - // } - - // if !dg.Equal(dgu.DestinationGroupConfig) || - // !dg.EqualDialers(dgu.DialersConfigs) { - // var dialers []relabeler.Dialer - // if !dg.EqualDialers(dgu.DialersConfigs) { - // dialers, rangeErr = makeDialers(rr.clock, rr.registerer, dgu.DialersConfigs) - // if rangeErr != nil { - // return rangeErr - // } - // } - - // if rangeErr = dg.ResetTo(dgu.DestinationGroupConfig, dialers); err != nil { - // return rangeErr - // } - // } - // mxdgupds.Lock() - // delete(dgupds, dg.Name()) - // mxdgupds.Unlock() - // return nil - // }); err != nil { - // level.Error(rr.logger).Log("msg", "failed to apply config DestinationGroups", "err", err) - // return err - // } - // // delete unused DestinationGroup - // dgs.RemoveByID(toDelete) - - // // create new DestinationGroup - // for _, dgupd := range dgupds { - // dialers, err := makeDialers(rr.clock, rr.registerer, dgupd.DialersConfigs) - // if err != nil { - // level.Error(rr.logger).Log("msg", "failed to make new dialers", "err", err) - // return err - // } - - // dg, err := relabeler.NewDestinationGroup( - // rr.ctx, - // dgupd.DestinationGroupConfig, - // encoderSelector, - // refillCtor, - // refillSenderCtor, - // rr.clock, - // dialers, - // rr.registerer, - // ) - // if err != nil { - // level.Error(rr.logger).Log("msg", "failed to init DestinationGroup", "err", err) - // return err - // } - - // dgs.Add(dg) - // } - // dstrb.SetDestinationGroups(dgs) - - // return nil - // }), - ) - if err != nil { - return err - } - - return nil -} - -// GetState create new state. -func (rr *Receiver) GetState() *cppbridge.State { - return cppbridge.NewState(rr.headConfigStorage.Load().numberOfShards) -} - -// HeadQuerier returns [storage.Querier] from active head. -func (rr *Receiver) HeadQuerier(ctx context.Context, mint, maxt int64) (storage.Querier, error) { - return rr.activeHead.Querier(ctx, rr.activeQuerierMetrics, mint, maxt) -} - -func (rr *Receiver) HeadStatus(ctx context.Context, limit int) relabeler.HeadStatus { - return rr.activeHead.HeadStatus(ctx, limit) -} - -// LowestSentTimestamp returns the lowest sent timestamp across all queues. -func (*Receiver) LowestSentTimestamp() int64 { - return 0 -} - -// MergeOutOfOrderChunks merge chunks with out of order data chunks. -func (rr *Receiver) MergeOutOfOrderChunks(ctx context.Context) { - rr.activeHead.MergeOutOfOrderChunks(ctx) -} - -// Querier calls f() with the given parameters. -// Returns a querier.MultiQuerier combining of appenderQuerier and storageQuerier. -func (rr *Receiver) Querier(mint, maxt int64) (storage.Querier, error) { - activeQuerier, err := rr.activeHead.Querier(rr.ctx, rr.activeQuerierMetrics, mint, maxt) - if err != nil { - return nil, err - } - - storageQuerier, err := rr.storage.Querier(mint, maxt) - if err != nil { - return nil, errors.Join(err, activeQuerier.Close()) - } - - return querier.NewMultiQuerier([]storage.Querier{activeQuerier, storageQuerier}, nil), nil -} - -func (rr *Receiver) ChunkQuerier(mint, maxt int64) (storage.ChunkQuerier, error) { - appenderQuerier, err := rr.activeHead.ChunkQuerier(rr.ctx, mint, maxt) - if err != nil { - return nil, err - } - - storageQuerier, err := rr.storage.ChunkQuerier(mint, maxt) - if err != nil { - return nil, errors.Join(err, appenderQuerier.Close()) - } - - return storage.NewMergeChunkQuerier( - nil, - []storage.ChunkQuerier{appenderQuerier, storageQuerier}, - storage.NewConcatenatingChunkSeriesMerger(), - ), nil -} - -// RelabelerIDIsExist check on exist relabelerID. -func (rr *Receiver) RelabelerIDIsExist(relabelerID string) bool { - cs := rr.headConfigStorage.Load() - for _, cfg := range cs.inputRelabelerConfigs { - if cfg.Name == relabelerID { - return true - } - } - - return false -} - -// Run main loop. -func (rr *Receiver) Run(_ context.Context) (err error) { - defer rr.shutdowner.Done(err) - rr.storage.Run() - rr.rotator.Run() - <-rr.shutdowner.Signal() - return nil -} - -// Shutdown safe shutdown Receiver. -func (rr *Receiver) Shutdown(ctx context.Context) error { - cgogcErr := rr.cgogc.Shutdown(ctx) - metricWriteErr := rr.metricsWriteTrigger.Close() - rotatorErr := rr.rotator.Close() - storageErr := rr.storage.Close() - distributorErr := rr.distributor.Shutdown(ctx) - activeHeadErr := rr.activeHead.Close(ctx) - err := rr.shutdowner.Shutdown(ctx) - return errors.Join(cgogcErr, metricWriteErr, rotatorErr, storageErr, distributorErr, activeHeadErr, err) -} - -// makeDestinationGroups create DestinationGroups from configs. -func makeDestinationGroups( - ctx context.Context, - clock clockwork.Clock, - registerer prometheus.Registerer, - workingDir, clientID string, - rwCfgs []*prom_config.PPRemoteWriteConfig, - numberOfShards uint16, -) (*relabeler.DestinationGroups, error) { - dgs := make(relabeler.DestinationGroups, 0, len(rwCfgs)) - - for _, rwCfg := range rwCfgs { - if rwCfg.IsPrometheusProtocol() { - continue - } - - dgCfg, err := convertingDestinationGroupConfig(rwCfg, workingDir, numberOfShards) - if err != nil { - return nil, err - } - - dialersConfigs, err := convertingConfigDialers(clientID, rwCfg.Destinations) - if err != nil { - return nil, err - } - dialers, err := makeDialers(clock, registerer, dialersConfigs) - if err != nil { - return nil, err - } - - dg, err := relabeler.NewDestinationGroup( - ctx, - dgCfg, - encoderSelector, - refillCtor, - refillSenderCtor, - clock, - dialers, - registerer, - ) - if err != nil { - return nil, err - } - - dgs = append(dgs, dg) - } - - return &dgs, nil -} - -// makeDestinationGroupUpdates create update for DestinationGroups. -func makeDestinationGroupUpdates( - rwCfgs []*prom_config.PPRemoteWriteConfig, - workingDir, clientID string, - numberOfShards uint16, -) (map[string]*relabeler.DestinationGroupUpdate, error) { - dgus := make(map[string]*relabeler.DestinationGroupUpdate, len(rwCfgs)) - - for _, rwCfg := range rwCfgs { - if rwCfg.IsPrometheusProtocol() { - continue - } - - dgCfg, err := convertingDestinationGroupConfig(rwCfg, workingDir, numberOfShards) - if err != nil { - return nil, err - } - - dialersConfigs, err := convertingConfigDialers(clientID, rwCfg.Destinations) - if err != nil { - return nil, err - } - - dgus[rwCfg.Name] = &relabeler.DestinationGroupUpdate{ - DestinationGroupConfig: dgCfg, - DialersConfigs: dialersConfigs, - } - } - - return dgus, nil -} - -// convertingDestinationGroupConfig converting incoming config to internal DestinationGroupConfig. -func convertingDestinationGroupConfig( - rwCfg *prom_config.PPRemoteWriteConfig, - workingDir string, - numberOfShards uint16, -) (*relabeler.DestinationGroupConfig, error) { - rCfgs, err := convertingRelabelersConfig(rwCfg.WriteRelabelConfigs) - if err != nil { - return nil, err - } - - dgcfg := relabeler.NewDestinationGroupConfig( - rwCfg.Name, - workingDir, - rCfgs, - numberOfShards, - ) - - return dgcfg, nil -} - -// convertingRelabelersConfig converting incoming relabel config to internal relabel config. -func convertingRelabelersConfig(rCfgs []*relabel.Config) ([]*cppbridge.RelabelConfig, error) { - var crCfgs []*cppbridge.RelabelConfig - raw, err := yaml.Marshal(rCfgs) - if err != nil { - return nil, err - } - - if err = yaml.Unmarshal(raw, &crCfgs); err != nil { - return nil, err - } - - return crCfgs, nil -} - -// convertingConfigDialers converting and make internal dialer configs. -func convertingConfigDialers( - clientID string, - sCfgs []*prom_config.PPDestinationConfig, -) ([]*relabeler.DialersConfig, error) { - dialersConfigs := make([]*relabeler.DialersConfig, 0, len(sCfgs)) - for _, sCfg := range sCfgs { - tlsCfg, err := common_config.NewTLSConfig(&sCfg.HTTPClientConfig.TLSConfig) - if err != nil { - return nil, err - } - - ccfg, err := dialer.NewCommonConfig( - sCfg.URL.URL, - tlsCfg, - sCfg.Name, - ) - if err != nil { - return nil, err - } - - dialersConfigs = append( - dialersConfigs, - &relabeler.DialersConfig{ - DialerConfig: relabeler.NewDialerConfig( - sCfg.URL.URL, - clientID, - extractAccessToken(sCfg.HTTPClientConfig.Authorization), - ), - ConnDialerConfig: ccfg, - }, - ) - } - - return dialersConfigs, nil -} - -// extractAccessToken extract access token from Authorization config. -func extractAccessToken(authorization *common_config.Authorization) string { - if authorization == nil { - return "" - } - - return string(authorization.Credentials) -} - -// makeDialers create dialers from main config according to the specified parameters. -func makeDialers( - clock clockwork.Clock, - registerer prometheus.Registerer, - dialersConfig []*relabeler.DialersConfig, -) ([]relabeler.Dialer, error) { - dialers := make([]relabeler.Dialer, 0, len(dialersConfig)) - for i := range dialersConfig { - ccfg, ok := dialersConfig[i].ConnDialerConfig.(*dialer.CommonConfig) - if !ok { - return nil, fmt.Errorf("invalid dialer CommonConfig: %v", dialersConfig[i].ConnDialerConfig) - } - - d, err := dialer.DefaultDialer(ccfg, registerer) - if err != nil { - return nil, err - } - - tcpDialer := relabeler.NewWebSocketDialer( - d, - dialersConfig[i].DialerConfig, - clock, - registerer, - ) - dialers = append(dialers, tcpDialer) - } - - return dialers, nil -} - -// encoderSelector selector for constructors for encoders. -func encoderSelector(isShrinkable bool) relabeler.ManagerEncoderCtor { - if isShrinkable { - return func(shardID uint16, shardsNumberPower uint8) relabeler.ManagerEncoder { - return cppbridge.NewWALEncoderLightweight(shardID, shardsNumberPower) - } - } - - return func(shardID uint16, shardsNumberPower uint8) relabeler.ManagerEncoder { - return cppbridge.NewWALEncoder(shardID, shardsNumberPower) - } -} - -// refillCtor default contructor for refill. -func refillCtor( - workinDir string, - blockID uuid.UUID, - destinations []string, - shardsNumberPower uint8, - segmentEncodingVersion uint8, - alwaysToRefill bool, - name string, - registerer prometheus.Registerer, -) (relabeler.ManagerRefill, error) { - return relabeler.NewRefill( - workinDir, - shardsNumberPower, - segmentEncodingVersion, - blockID, - alwaysToRefill, - name, - registerer, - destinations..., - ) -} - -// refillSenderCtor default contructor for manager sender. -func refillSenderCtor( - rsmCfg relabeler.RefillSendManagerConfig, - workingDir string, - dialers []relabeler.Dialer, - clock clockwork.Clock, - name string, - registerer prometheus.Registerer, -) (relabeler.ManagerRefillSender, error) { - return relabeler.NewRefillSendManager(rsmCfg, workingDir, dialers, clock, name, registerer) -} - -// initLogHandler init log handler for ManagerKeeper. -func initLogHandler(logger log.Logger) { - logger = log.With(logger, "pp_caller", log.Caller(4)) - rlogger.Debugf = func(template string, args ...any) { - level.Debug(logger).Log("msg", fmt.Sprintf(template, args...)) - } - rlogger.Infof = func(template string, args ...any) { - level.Info(logger).Log("msg", fmt.Sprintf(template, args...)) - } - rlogger.Warnf = func(template string, args ...any) { - level.Warn(logger).Log("msg", fmt.Sprintf(template, args...)) - } - rlogger.Errorf = func(template string, args ...any) { - level.Error(logger).Log("msg", fmt.Sprintf(template, args...)) - } -} - -// readClientID read ClientID. -func readClientID(logger log.Logger, dir string) (string, error) { - if err := os.MkdirAll(dir, 0o700); err != nil { - return "", fmt.Errorf("mkdir %s: %w", filepath.Dir(dir), err) - } - clientIDPath := path.Join(dir, "client_id.uuid") - // Try reading UUID from the file. If not present, generate new one and write to file - data, err := os.ReadFile(clientIDPath) - switch { - case os.IsNotExist(err): - proxyUUID := uuid.NewString() - //revive:disable-next-line:add-constant file permissions simple readable as octa-number - if err = os.WriteFile(clientIDPath, []byte(proxyUUID), 0o644); err != nil { // #nosec G306 - return "", fmt.Errorf("failed to write proxy id: %w", err) - } - - level.Info(logger).Log("msg", "create new client id") - return proxyUUID, nil - - case err == nil: - //revive:disable-next-line:add-constant uuid len - if len(data) < 36 { - return "", fmt.Errorf("short client id: %d", len(data)) - } - - return string(data[:36]), nil - - default: - return "", fmt.Errorf("failed to read client id: %w", err) - } -} - -// -// NoopQuerier -// - -type NoopQuerier struct{} - -var _ storage.Querier = (*NoopQuerier)(nil) - -func (*NoopQuerier) Select(_ context.Context, _ bool, _ *storage.SelectHints, _ ...*labels.Matcher) storage.SeriesSet { - return &NoopSeriesSet{} -} - -func (q *NoopQuerier) LabelValues( - ctx context.Context, - name string, - hints *storage.LabelHints, - matchers ...*labels.Matcher, -) ([]string, annotations.Annotations, error) { - return []string{}, *annotations.New(), nil -} - -func (q *NoopQuerier) LabelNames( - ctx context.Context, - hints *storage.LabelHints, - matchers ...*labels.Matcher, -) ([]string, annotations.Annotations, error) { - return []string{}, *annotations.New(), nil -} - -func (*NoopQuerier) Close() error { - return nil -} - -// -// NoopSeriesSet -// - -type NoopSeriesSet struct{} - -func (*NoopSeriesSet) Next() bool { - return false -} - -func (*NoopSeriesSet) At() storage.Series { - return nil -} - -func (*NoopSeriesSet) Err() error { - return nil -} - -func (*NoopSeriesSet) Warnings() annotations.Annotations { - return nil -} +// import ( +// "context" +// "errors" +// "fmt" +// "os" +// "path" +// "path/filepath" +// "time" + +// "github.com/go-kit/log" +// "github.com/go-kit/log/level" +// "github.com/google/uuid" +// "github.com/jonboulle/clockwork" +// "github.com/prometheus/client_golang/prometheus" +// common_config "github.com/prometheus/common/config" +// "go.uber.org/atomic" +// "gopkg.in/yaml.v2" + +// prom_config "github.com/prometheus/prometheus/config" +// "github.com/prometheus/prometheus/model/labels" +// "github.com/prometheus/prometheus/model/relabel" +// pp_pkg_config "github.com/prometheus/prometheus/pp-pkg/config" +// "github.com/prometheus/prometheus/pp-pkg/dialer" +// "github.com/prometheus/prometheus/pp/go/cppbridge" +// "github.com/prometheus/prometheus/pp/go/relabeler" +// "github.com/prometheus/prometheus/pp/go/relabeler/appender" +// "github.com/prometheus/prometheus/pp/go/relabeler/block" +// "github.com/prometheus/prometheus/pp/go/relabeler/config" +// "github.com/prometheus/prometheus/pp/go/relabeler/distributor" +// "github.com/prometheus/prometheus/pp/go/relabeler/head/catalog" +// headmanager "github.com/prometheus/prometheus/pp/go/relabeler/head/manager" +// "github.com/prometheus/prometheus/pp/go/relabeler/head/ready" +// "github.com/prometheus/prometheus/pp/go/relabeler/headcontainer" +// rlogger "github.com/prometheus/prometheus/pp/go/relabeler/logger" +// "github.com/prometheus/prometheus/pp/go/relabeler/querier" +// "github.com/prometheus/prometheus/pp/go/util" +// "github.com/prometheus/prometheus/storage" +// "github.com/prometheus/prometheus/util/annotations" +// ) + +// const defaultShutdownTimeout = 40 * time.Second + +// var DefaultNumberOfShards uint16 = 2 + +// type HeadConfig struct { +// inputRelabelerConfigs []*config.InputRelabelerConfig +// numberOfShards uint16 +// } + +// type HeadConfigStorage struct { +// ptr atomic.Pointer[HeadConfig] +// } + +// func (s *HeadConfigStorage) Load() *HeadConfig { +// return s.ptr.Load() +// } + +// func (s *HeadConfigStorage) Get() ([]*config.InputRelabelerConfig, uint16) { +// cfg := s.ptr.Load() +// return cfg.inputRelabelerConfigs, cfg.numberOfShards +// } + +// func (s *HeadConfigStorage) Store(headConfig *HeadConfig) { +// s.ptr.Store(headConfig) +// } + +// type Receiver struct { +// ctx context.Context + +// distributor *distributor.Distributor +// // appender *appender.QueryableAppender +// activeHead *headcontainer.Active +// storage *appender.QueryableStorage +// rotator *appender.RotateCommiter +// metricsWriteTrigger *appender.MetricsWriteTrigger + +// headConfigStorage *HeadConfigStorage +// hashdexFactory relabeler.HashdexFactory +// hashdexLimits cppbridge.WALHashdexLimits +// haTracker relabeler.HATracker +// clock clockwork.Clock +// registerer prometheus.Registerer +// logger log.Logger +// workingDir string +// clientID string +// cgogc *cppbridge.CGOGC +// shutdowner *util.GracefulShutdowner + +// activeQuerierMetrics *querier.Metrics +// storageQuerierMetrics *querier.Metrics +// } + +// type RotationInfo struct { +// BlockDuration time.Duration +// Seed uint64 +// } + +// type HeadActivator struct { +// catalog *catalog.Catalog +// } + +// func newHeadActivator(catalog *catalog.Catalog) *HeadActivator { +// return &HeadActivator{catalog: catalog} +// } + +// func (ha *HeadActivator) Activate(headID string) error { +// _, err := ha.catalog.SetStatus(headID, catalog.StatusActive) +// return err +// } + +// func NewReceiver( +// ctx context.Context, +// logger log.Logger, +// registerer prometheus.Registerer, +// receiverCfg *pp_pkg_config.RemoteWriteReceiverConfig, +// workingDir string, +// remoteWriteCfgs []*prom_config.PPRemoteWriteConfig, +// dataDir string, +// rotationInfo RotationInfo, +// headCatalog *catalog.Catalog, +// triggerNotifier *ReloadBlocksTriggerNotifier, +// readyNotifier ready.Notifier, +// commitInterval time.Duration, +// maxRetentionDuration time.Duration, +// headRetentionTimeout time.Duration, +// writeTimeout time.Duration, +// maxSegmentSize uint32, +// unloadDataStorage bool, +// ) (*Receiver, error) { +// if logger == nil { +// logger = log.NewNopLogger() +// } + +// clientID, err := readClientID(logger, workingDir) +// if err != nil { +// level.Error(logger).Log("msg", "failed read client id", "err", err) +// return nil, err +// } + +// initLogHandler(logger) +// clock := clockwork.NewRealClock() + +// numberOfShards := receiverCfg.NumberOfShards +// if numberOfShards == 0 { +// numberOfShards = DefaultNumberOfShards +// } + +// destinationGroups, err := makeDestinationGroups( +// ctx, +// clock, +// registerer, +// workingDir, +// clientID, +// remoteWriteCfgs, +// numberOfShards, +// ) +// if err != nil { +// level.Error(logger).Log("msg", "failed to init DestinationGroups", "err", err) +// return nil, err +// } + +// headConfigStorage := &HeadConfigStorage{} + +// headConfigStorage.Store(&HeadConfig{ +// inputRelabelerConfigs: receiverCfg.Configs, +// numberOfShards: numberOfShards, +// }) + +// dataDir, err = filepath.Abs(dataDir) +// if err != nil { +// return nil, err +// } + +// var unloadDataStorageInterval time.Duration +// if unloadDataStorage { +// unloadDataStorageInterval = appender.DefaultMergeDuration +// } + +// headManager, err := headmanager.New( +// dataDir, +// clock, +// headConfigStorage, +// headCatalog, +// maxSegmentSize, +// registerer, +// unloadDataStorageInterval, +// ) +// if err != nil { +// return nil, fmt.Errorf("failed to create head manager: %w", err) +// } + +// currentHead, rotatedHeads, err := headManager.Restore(rotationInfo.BlockDuration, unloadDataStorageInterval) +// if err != nil { +// return nil, fmt.Errorf("failed to restore heads: %w", err) +// } +// readyNotifier.NotifyReady() +// storageQuerierMetrics := querier.NewMetrics(registerer, querier.QueryableStorageSource) +// queryableStorage := appender.NewQueryableStorageWithWriteNotifier( +// block.NewWriter(dataDir, block.DefaultChunkSegmentSize, rotationInfo.BlockDuration, registerer), +// registerer, +// storageQuerierMetrics, +// triggerNotifier, +// clock, +// maxRetentionDuration, +// headRetentionTimeout, +// writeTimeout, +// rotatedHeads..., +// ) + +// var containeredHead relabeler.Head +// containeredHead = headcontainer.NewRotatable(currentHead, queryableStorage, headManager, newHeadActivator(headCatalog)) + +// if len(os.Getenv("OPCORE_ROTATION_HEAP_DEBUG")) > 0 { +// containeredHead = headcontainer.NewHeapProfileWritable( +// containeredHead, +// util.NewHeapProfileWriter(filepath.Join(dataDir, "heap_profiles")), +// ) +// } + +// dstrb := distributor.NewDistributor(*destinationGroups) +// activeQuerierMetrics := querier.NewMetrics(registerer, querier.QueryableAppenderSource) +// activeHead := headcontainer.NewActive(containeredHead, registerer) +// // app := appender.NewQueryableAppender( +// // ctx, +// // appenderHead, +// // dstrb, +// // activeQuerierMetrics, +// // registerer, +// // ) +// mwt := appender.NewMetricsWriteTrigger(ctx, appender.DefaultMetricWriteInterval, activeHead, queryableStorage) + +// r := &Receiver{ +// ctx: ctx, +// distributor: dstrb, +// // appender: app, +// activeHead: activeHead, +// storage: queryableStorage, +// headConfigStorage: headConfigStorage, +// rotator: appender.NewRotateCommiter( +// ctx, +// activeHead, +// relabeler.NewRotateTimerWithSeed(clock, rotationInfo.BlockDuration, rotationInfo.Seed), +// appender.NewConstantIntervalTimer(clock, commitInterval), +// appender.NewConstantIntervalTimer(clock, appender.DefaultMergeDuration), +// unloadDataStorage, +// registerer, +// ), + +// metricsWriteTrigger: mwt, +// hashdexFactory: cppbridge.HashdexFactory{}, +// hashdexLimits: cppbridge.DefaultWALHashdexLimits(), +// haTracker: relabeler.NewHighAvailabilityTracker(ctx, registerer, clock), +// clock: clock, +// registerer: registerer, +// logger: logger, +// workingDir: workingDir, +// clientID: clientID, +// cgogc: cppbridge.NewCGOGC(registerer), +// shutdowner: util.NewGracefulShutdowner(), + +// activeQuerierMetrics: activeQuerierMetrics, +// storageQuerierMetrics: storageQuerierMetrics, +// } + +// level.Info(logger).Log("msg", "created") + +// return r, nil +// } + +// // AppendHashdex append incoming Hashdex data to relabeling. +// func (rr *Receiver) AppendHashdex( +// ctx context.Context, +// hashdex cppbridge.ShardedData, +// relabelerID string, +// commitToWal bool, +// ) error { +// if rr.haTracker.IsDrop(hashdex.Cluster(), hashdex.Replica()) { +// return nil +// } +// incomingData := &relabeler.IncomingData{Hashdex: hashdex} +// _, err := rr.activeHead.Append(ctx, incomingData, nil, relabelerID, commitToWal) +// return err +// } + +// // AppendSnappyProtobuf append compressed via snappy Protobuf data to relabeling hashdex data. +// func (rr *Receiver) AppendSnappyProtobuf( +// ctx context.Context, +// compressedData relabeler.ProtobufData, +// relabelerID string, +// commitToWal bool, +// ) error { +// hx, err := cppbridge.NewWALSnappyProtobufHashdex(compressedData.Bytes(), rr.hashdexLimits) +// compressedData.Destroy() +// if err != nil { +// return err +// } + +// if rr.haTracker.IsDrop(hx.Cluster(), hx.Replica()) { +// return nil +// } + +// incomingData := &relabeler.IncomingData{Hashdex: hx} +// _, err = rr.activeHead.Append(ctx, incomingData, nil, relabelerID, commitToWal) +// return err +// } + +// // AppendTimeSeries append TimeSeries data to relabeling hashdex data. +// func (rr *Receiver) AppendTimeSeries( +// ctx context.Context, +// data relabeler.TimeSeriesData, +// state *cppbridge.State, +// relabelerID string, +// commitToWal bool, +// ) (cppbridge.RelabelerStats, error) { +// hx, err := rr.hashdexFactory.GoModel(data.TimeSeries(), rr.hashdexLimits) +// if err != nil { +// data.Destroy() +// return cppbridge.RelabelerStats{}, err +// } + +// if rr.haTracker.IsDrop(hx.Cluster(), hx.Replica()) { +// data.Destroy() +// return cppbridge.RelabelerStats{}, nil +// } +// incomingData := &relabeler.IncomingData{Hashdex: hx, Data: data} +// return rr.activeHead.Append( +// ctx, +// incomingData, +// state, +// relabelerID, +// commitToWal, +// ) +// } + +// func (rr *Receiver) AppendTimeSeriesHashdex( +// ctx context.Context, +// hashdex cppbridge.ShardedData, +// state *cppbridge.State, +// relabelerID string, +// commitToWal bool, +// ) (cppbridge.RelabelerStats, error) { +// return rr.activeHead.Append( +// ctx, +// &relabeler.IncomingData{Hashdex: hashdex}, +// state, +// relabelerID, +// commitToWal, +// ) +// } + +// // Appender create a new appender for head. +// func (rr *Receiver) Appender(ctx context.Context) storage.Appender { +// return newPromAppender(ctx, rr, prom_config.TransparentRelabeler) +// } + +// // ApplyConfig update config. +// func (rr *Receiver) ApplyConfig(cfg *prom_config.Config) error { +// level.Info(rr.logger).Log("msg", "reconfiguration start") +// defer level.Info(rr.logger).Log("msg", "reconfiguration completed") + +// rCfg, err := cfg.GetReceiverConfig() +// if err != nil { +// return err +// } + +// numberOfShards := rCfg.NumberOfShards +// if numberOfShards == 0 { +// numberOfShards = DefaultNumberOfShards +// } + +// rr.headConfigStorage.Store(&HeadConfig{ +// inputRelabelerConfigs: rCfg.Configs, +// numberOfShards: numberOfShards, +// }) + +// err = rr.activeHead.Reconfigure( +// rr.ctx, +// HeadConfigureFunc(func(head relabeler.Head) error { +// return head.Reconfigure(rr.ctx, rCfg.Configs, numberOfShards) +// }), +// // DistributorConfigureFunc(func(dstrb relabeler.Distributor) error { +// // mxdgupds := new(sync.Mutex) +// // dgupds, err := makeDestinationGroupUpdates( +// // cfg.RemoteWriteConfigs, +// // rr.workingDir, +// // rr.clientID, +// // numberOfShards, +// // ) +// // if err != nil { +// // level.Error(rr.logger).Log("msg", "failed to init destination group update", "err", err) +// // return err +// // } +// // mxDelete := new(sync.Mutex) +// // toDelete := []int{} + +// // dgs := dstrb.DestinationGroups() +// // if err = dgs.RangeGo(func(destinationGroupID int, dg *relabeler.DestinationGroup) error { +// // var rangeErr error +// // dgu, ok := dgupds[dg.Name()] +// // if !ok { +// // mxDelete.Lock() +// // toDelete = append(toDelete, destinationGroupID) +// // mxDelete.Unlock() +// // ctxShutdown, cancel := context.WithTimeout(rr.ctx, defaultShutdownTimeout) +// // if rangeErr = dg.Shutdown(ctxShutdown); err != nil { +// // level.Error(rr.logger).Log("msg", "failed shutdown DestinationGroup", "err", rangeErr) +// // } +// // cancel() +// // return nil +// // } + +// // if !dg.Equal(dgu.DestinationGroupConfig) || +// // !dg.EqualDialers(dgu.DialersConfigs) { +// // var dialers []relabeler.Dialer +// // if !dg.EqualDialers(dgu.DialersConfigs) { +// // dialers, rangeErr = makeDialers(rr.clock, rr.registerer, dgu.DialersConfigs) +// // if rangeErr != nil { +// // return rangeErr +// // } +// // } + +// // if rangeErr = dg.ResetTo(dgu.DestinationGroupConfig, dialers); err != nil { +// // return rangeErr +// // } +// // } +// // mxdgupds.Lock() +// // delete(dgupds, dg.Name()) +// // mxdgupds.Unlock() +// // return nil +// // }); err != nil { +// // level.Error(rr.logger).Log("msg", "failed to apply config DestinationGroups", "err", err) +// // return err +// // } +// // // delete unused DestinationGroup +// // dgs.RemoveByID(toDelete) + +// // // create new DestinationGroup +// // for _, dgupd := range dgupds { +// // dialers, err := makeDialers(rr.clock, rr.registerer, dgupd.DialersConfigs) +// // if err != nil { +// // level.Error(rr.logger).Log("msg", "failed to make new dialers", "err", err) +// // return err +// // } + +// // dg, err := relabeler.NewDestinationGroup( +// // rr.ctx, +// // dgupd.DestinationGroupConfig, +// // encoderSelector, +// // refillCtor, +// // refillSenderCtor, +// // rr.clock, +// // dialers, +// // rr.registerer, +// // ) +// // if err != nil { +// // level.Error(rr.logger).Log("msg", "failed to init DestinationGroup", "err", err) +// // return err +// // } + +// // dgs.Add(dg) +// // } +// // dstrb.SetDestinationGroups(dgs) + +// // return nil +// // }), +// ) +// if err != nil { +// return err +// } + +// return nil +// } + +// // GetState create new state. +// func (rr *Receiver) GetState() *cppbridge.State { +// return cppbridge.NewState(rr.headConfigStorage.Load().numberOfShards) +// } + +// // HeadQuerier returns [storage.Querier] from active head. +// func (rr *Receiver) HeadQuerier(ctx context.Context, mint, maxt int64) (storage.Querier, error) { +// return rr.activeHead.Querier(ctx, rr.activeQuerierMetrics, mint, maxt) +// } + +// func (rr *Receiver) HeadStatus(ctx context.Context, limit int) relabeler.HeadStatus { +// return rr.activeHead.HeadStatus(ctx, limit) +// } + +// // LowestSentTimestamp returns the lowest sent timestamp across all queues. +// func (*Receiver) LowestSentTimestamp() int64 { +// return 0 +// } + +// // MergeOutOfOrderChunks merge chunks with out of order data chunks. +// func (rr *Receiver) MergeOutOfOrderChunks(ctx context.Context) { +// rr.activeHead.MergeOutOfOrderChunks(ctx) +// } + +// // Querier calls f() with the given parameters. +// // Returns a querier.MultiQuerier combining of appenderQuerier and storageQuerier. +// func (rr *Receiver) Querier(mint, maxt int64) (storage.Querier, error) { +// activeQuerier, err := rr.activeHead.Querier(rr.ctx, rr.activeQuerierMetrics, mint, maxt) +// if err != nil { +// return nil, err +// } + +// storageQuerier, err := rr.storage.Querier(mint, maxt) +// if err != nil { +// return nil, errors.Join(err, activeQuerier.Close()) +// } + +// return querier.NewMultiQuerier([]storage.Querier{activeQuerier, storageQuerier}, nil), nil +// } + +// func (rr *Receiver) ChunkQuerier(mint, maxt int64) (storage.ChunkQuerier, error) { +// appenderQuerier, err := rr.activeHead.ChunkQuerier(rr.ctx, mint, maxt) +// if err != nil { +// return nil, err +// } + +// storageQuerier, err := rr.storage.ChunkQuerier(mint, maxt) +// if err != nil { +// return nil, errors.Join(err, appenderQuerier.Close()) +// } + +// return storage.NewMergeChunkQuerier( +// nil, +// []storage.ChunkQuerier{appenderQuerier, storageQuerier}, +// storage.NewConcatenatingChunkSeriesMerger(), +// ), nil +// } + +// // RelabelerIDIsExist check on exist relabelerID. +// func (rr *Receiver) RelabelerIDIsExist(relabelerID string) bool { +// cs := rr.headConfigStorage.Load() +// for _, cfg := range cs.inputRelabelerConfigs { +// if cfg.Name == relabelerID { +// return true +// } +// } + +// return false +// } + +// // Run main loop. +// func (rr *Receiver) Run(_ context.Context) (err error) { +// defer rr.shutdowner.Done(err) +// rr.storage.Run() +// rr.rotator.Run() +// <-rr.shutdowner.Signal() +// return nil +// } + +// // Shutdown safe shutdown Receiver. +// func (rr *Receiver) Shutdown(ctx context.Context) error { +// cgogcErr := rr.cgogc.Shutdown(ctx) +// metricWriteErr := rr.metricsWriteTrigger.Close() +// rotatorErr := rr.rotator.Close() +// storageErr := rr.storage.Close() +// distributorErr := rr.distributor.Shutdown(ctx) +// activeHeadErr := rr.activeHead.Close(ctx) +// err := rr.shutdowner.Shutdown(ctx) +// return errors.Join(cgogcErr, metricWriteErr, rotatorErr, storageErr, distributorErr, activeHeadErr, err) +// } + +// // makeDestinationGroups create DestinationGroups from configs. +// func makeDestinationGroups( +// ctx context.Context, +// clock clockwork.Clock, +// registerer prometheus.Registerer, +// workingDir, clientID string, +// rwCfgs []*prom_config.PPRemoteWriteConfig, +// numberOfShards uint16, +// ) (*relabeler.DestinationGroups, error) { +// dgs := make(relabeler.DestinationGroups, 0, len(rwCfgs)) + +// for _, rwCfg := range rwCfgs { +// if rwCfg.IsPrometheusProtocol() { +// continue +// } + +// dgCfg, err := convertingDestinationGroupConfig(rwCfg, workingDir, numberOfShards) +// if err != nil { +// return nil, err +// } + +// dialersConfigs, err := convertingConfigDialers(clientID, rwCfg.Destinations) +// if err != nil { +// return nil, err +// } +// dialers, err := makeDialers(clock, registerer, dialersConfigs) +// if err != nil { +// return nil, err +// } + +// dg, err := relabeler.NewDestinationGroup( +// ctx, +// dgCfg, +// encoderSelector, +// refillCtor, +// refillSenderCtor, +// clock, +// dialers, +// registerer, +// ) +// if err != nil { +// return nil, err +// } + +// dgs = append(dgs, dg) +// } + +// return &dgs, nil +// } + +// // makeDestinationGroupUpdates create update for DestinationGroups. +// func makeDestinationGroupUpdates( +// rwCfgs []*prom_config.PPRemoteWriteConfig, +// workingDir, clientID string, +// numberOfShards uint16, +// ) (map[string]*relabeler.DestinationGroupUpdate, error) { +// dgus := make(map[string]*relabeler.DestinationGroupUpdate, len(rwCfgs)) + +// for _, rwCfg := range rwCfgs { +// if rwCfg.IsPrometheusProtocol() { +// continue +// } + +// dgCfg, err := convertingDestinationGroupConfig(rwCfg, workingDir, numberOfShards) +// if err != nil { +// return nil, err +// } + +// dialersConfigs, err := convertingConfigDialers(clientID, rwCfg.Destinations) +// if err != nil { +// return nil, err +// } + +// dgus[rwCfg.Name] = &relabeler.DestinationGroupUpdate{ +// DestinationGroupConfig: dgCfg, +// DialersConfigs: dialersConfigs, +// } +// } + +// return dgus, nil +// } + +// // convertingDestinationGroupConfig converting incoming config to internal DestinationGroupConfig. +// func convertingDestinationGroupConfig( +// rwCfg *prom_config.PPRemoteWriteConfig, +// workingDir string, +// numberOfShards uint16, +// ) (*relabeler.DestinationGroupConfig, error) { +// rCfgs, err := convertingRelabelersConfig(rwCfg.WriteRelabelConfigs) +// if err != nil { +// return nil, err +// } + +// dgcfg := relabeler.NewDestinationGroupConfig( +// rwCfg.Name, +// workingDir, +// rCfgs, +// numberOfShards, +// ) + +// return dgcfg, nil +// } + +// // convertingRelabelersConfig converting incoming relabel config to internal relabel config. +// func convertingRelabelersConfig(rCfgs []*relabel.Config) ([]*cppbridge.RelabelConfig, error) { +// var crCfgs []*cppbridge.RelabelConfig +// raw, err := yaml.Marshal(rCfgs) +// if err != nil { +// return nil, err +// } + +// if err = yaml.Unmarshal(raw, &crCfgs); err != nil { +// return nil, err +// } + +// return crCfgs, nil +// } + +// // convertingConfigDialers converting and make internal dialer configs. +// func convertingConfigDialers( +// clientID string, +// sCfgs []*prom_config.PPDestinationConfig, +// ) ([]*relabeler.DialersConfig, error) { +// dialersConfigs := make([]*relabeler.DialersConfig, 0, len(sCfgs)) +// for _, sCfg := range sCfgs { +// tlsCfg, err := common_config.NewTLSConfig(&sCfg.HTTPClientConfig.TLSConfig) +// if err != nil { +// return nil, err +// } + +// ccfg, err := dialer.NewCommonConfig( +// sCfg.URL.URL, +// tlsCfg, +// sCfg.Name, +// ) +// if err != nil { +// return nil, err +// } + +// dialersConfigs = append( +// dialersConfigs, +// &relabeler.DialersConfig{ +// DialerConfig: relabeler.NewDialerConfig( +// sCfg.URL.URL, +// clientID, +// extractAccessToken(sCfg.HTTPClientConfig.Authorization), +// ), +// ConnDialerConfig: ccfg, +// }, +// ) +// } + +// return dialersConfigs, nil +// } + +// // extractAccessToken extract access token from Authorization config. +// func extractAccessToken(authorization *common_config.Authorization) string { +// if authorization == nil { +// return "" +// } + +// return string(authorization.Credentials) +// } + +// // makeDialers create dialers from main config according to the specified parameters. +// func makeDialers( +// clock clockwork.Clock, +// registerer prometheus.Registerer, +// dialersConfig []*relabeler.DialersConfig, +// ) ([]relabeler.Dialer, error) { +// dialers := make([]relabeler.Dialer, 0, len(dialersConfig)) +// for i := range dialersConfig { +// ccfg, ok := dialersConfig[i].ConnDialerConfig.(*dialer.CommonConfig) +// if !ok { +// return nil, fmt.Errorf("invalid dialer CommonConfig: %v", dialersConfig[i].ConnDialerConfig) +// } + +// d, err := dialer.DefaultDialer(ccfg, registerer) +// if err != nil { +// return nil, err +// } + +// tcpDialer := relabeler.NewWebSocketDialer( +// d, +// dialersConfig[i].DialerConfig, +// clock, +// registerer, +// ) +// dialers = append(dialers, tcpDialer) +// } + +// return dialers, nil +// } + +// // encoderSelector selector for constructors for encoders. +// func encoderSelector(isShrinkable bool) relabeler.ManagerEncoderCtor { +// if isShrinkable { +// return func(shardID uint16, shardsNumberPower uint8) relabeler.ManagerEncoder { +// return cppbridge.NewWALEncoderLightweight(shardID, shardsNumberPower) +// } +// } + +// return func(shardID uint16, shardsNumberPower uint8) relabeler.ManagerEncoder { +// return cppbridge.NewWALEncoder(shardID, shardsNumberPower) +// } +// } + +// // refillCtor default contructor for refill. +// func refillCtor( +// workinDir string, +// blockID uuid.UUID, +// destinations []string, +// shardsNumberPower uint8, +// segmentEncodingVersion uint8, +// alwaysToRefill bool, +// name string, +// registerer prometheus.Registerer, +// ) (relabeler.ManagerRefill, error) { +// return relabeler.NewRefill( +// workinDir, +// shardsNumberPower, +// segmentEncodingVersion, +// blockID, +// alwaysToRefill, +// name, +// registerer, +// destinations..., +// ) +// } + +// // refillSenderCtor default contructor for manager sender. +// func refillSenderCtor( +// rsmCfg relabeler.RefillSendManagerConfig, +// workingDir string, +// dialers []relabeler.Dialer, +// clock clockwork.Clock, +// name string, +// registerer prometheus.Registerer, +// ) (relabeler.ManagerRefillSender, error) { +// return relabeler.NewRefillSendManager(rsmCfg, workingDir, dialers, clock, name, registerer) +// } + +// // initLogHandler init log handler for ManagerKeeper. +// func initLogHandler(logger log.Logger) { +// logger = log.With(logger, "pp_caller", log.Caller(4)) +// rlogger.Debugf = func(template string, args ...any) { +// level.Debug(logger).Log("msg", fmt.Sprintf(template, args...)) +// } +// rlogger.Infof = func(template string, args ...any) { +// level.Info(logger).Log("msg", fmt.Sprintf(template, args...)) +// } +// rlogger.Warnf = func(template string, args ...any) { +// level.Warn(logger).Log("msg", fmt.Sprintf(template, args...)) +// } +// rlogger.Errorf = func(template string, args ...any) { +// level.Error(logger).Log("msg", fmt.Sprintf(template, args...)) +// } +// } + +// // readClientID read ClientID. +// func readClientID(logger log.Logger, dir string) (string, error) { +// if err := os.MkdirAll(dir, 0o700); err != nil { +// return "", fmt.Errorf("mkdir %s: %w", filepath.Dir(dir), err) +// } +// clientIDPath := path.Join(dir, "client_id.uuid") +// // Try reading UUID from the file. If not present, generate new one and write to file +// data, err := os.ReadFile(clientIDPath) +// switch { +// case os.IsNotExist(err): +// proxyUUID := uuid.NewString() +// //revive:disable-next-line:add-constant file permissions simple readable as octa-number +// if err = os.WriteFile(clientIDPath, []byte(proxyUUID), 0o644); err != nil { // #nosec G306 +// return "", fmt.Errorf("failed to write proxy id: %w", err) +// } + +// level.Info(logger).Log("msg", "create new client id") +// return proxyUUID, nil + +// case err == nil: +// //revive:disable-next-line:add-constant uuid len +// if len(data) < 36 { +// return "", fmt.Errorf("short client id: %d", len(data)) +// } + +// return string(data[:36]), nil + +// default: +// return "", fmt.Errorf("failed to read client id: %w", err) +// } +// } + +// // +// // NoopQuerier +// // + +// type NoopQuerier struct{} + +// var _ storage.Querier = (*NoopQuerier)(nil) + +// func (*NoopQuerier) Select(_ context.Context, _ bool, _ *storage.SelectHints, _ ...*labels.Matcher) storage.SeriesSet { +// return &NoopSeriesSet{} +// } + +// func (q *NoopQuerier) LabelValues( +// ctx context.Context, +// name string, +// hints *storage.LabelHints, +// matchers ...*labels.Matcher, +// ) ([]string, annotations.Annotations, error) { +// return []string{}, *annotations.New(), nil +// } + +// func (q *NoopQuerier) LabelNames( +// ctx context.Context, +// hints *storage.LabelHints, +// matchers ...*labels.Matcher, +// ) ([]string, annotations.Annotations, error) { +// return []string{}, *annotations.New(), nil +// } + +// func (*NoopQuerier) Close() error { +// return nil +// } + +// // +// // NoopSeriesSet +// // + +// type NoopSeriesSet struct{} + +// func (*NoopSeriesSet) Next() bool { +// return false +// } + +// func (*NoopSeriesSet) At() storage.Series { +// return nil +// } + +// func (*NoopSeriesSet) Err() error { +// return nil +// } + +// func (*NoopSeriesSet) Warnings() annotations.Annotations { +// return nil +// } diff --git a/pp/go/storage/builder.go b/pp/go/storage/builder.go index b4c5476859..38dc492609 100644 --- a/pp/go/storage/builder.go +++ b/pp/go/storage/builder.go @@ -9,6 +9,7 @@ import ( "github.com/prometheus/client_golang/prometheus" "github.com/prometheus/prometheus/pp/go/cppbridge" + "github.com/prometheus/prometheus/pp/go/logger" "github.com/prometheus/prometheus/pp/go/storage/catalog" "github.com/prometheus/prometheus/pp/go/storage/head/head" "github.com/prometheus/prometheus/pp/go/storage/head/shard" @@ -87,6 +88,7 @@ func (b *Builder) Build(generation uint64, numberOfShards uint16) (*HeadOnDisk, } b.events.With(prometheus.Labels{"type": "created"}).Inc() + logger.Debugf("[Builder] builded head: %s", headRecord.ID()) return head.NewHead( headRecord.ID(), shards, diff --git a/pp/go/storage/catalog/catalog.go b/pp/go/storage/catalog/catalog.go index 4e4e626e40..2470d29a41 100644 --- a/pp/go/storage/catalog/catalog.go +++ b/pp/go/storage/catalog/catalog.go @@ -195,7 +195,7 @@ func (c *Catalog) Get(id string) (*Record, error) { // List returns slice of records with filter and sort. func (c *Catalog) List(filterFn func(record *Record) bool, sortLess func(lhs, rhs *Record) bool) []*Record { - records := c.list(filterFn) + records := c.listWithFilter(filterFn) if sortLess != nil { sort.Slice(records, func(i, j int) bool { @@ -206,8 +206,8 @@ func (c *Catalog) List(filterFn func(record *Record) bool, sortLess func(lhs, rh return records } -// list returns slice of filtered records -func (c *Catalog) list(filterFn func(record *Record) bool) []*Record { +// listWithFilter returns slice of filtered records +func (c *Catalog) listWithFilter(filterFn func(record *Record) bool) []*Record { c.mtx.Lock() defer c.mtx.Unlock() diff --git a/pp/go/storage/catalog/record.go b/pp/go/storage/catalog/record.go index 12b89ed828..df55a4c3e5 100644 --- a/pp/go/storage/catalog/record.go +++ b/pp/go/storage/catalog/record.go @@ -209,6 +209,7 @@ func applyRecordChanges(r, changed *Record) { r.maxt = changed.maxt } +// LessByUpdateAt less [Record] by UpdateAt. func LessByUpdateAt(lhs, rhs *Record) bool { return lhs.UpdatedAt() < rhs.UpdatedAt() } diff --git a/pp/go/storage/head/keeper/keeper_test.go b/pp/go/storage/head/keeper/keeper_test.go index 42d5bb90d2..2bc1cb6b2a 100644 --- a/pp/go/storage/head/keeper/keeper_test.go +++ b/pp/go/storage/head/keeper/keeper_test.go @@ -1,7 +1,6 @@ package keeper import ( - "container/heap" "testing" "github.com/stretchr/testify/suite" @@ -34,14 +33,6 @@ func TestKeeperSuite(t *testing.T) { suite.Run(t, new(KeeperSuite)) } -func (s *KeeperSuite) getHeads() []*headForTest { - heads := make([]*headForTest, 0, len(s.keeper.heads)) - for _, head := range s.keeper.heads { - heads = append(heads, head.head) - } - return heads -} - func (s *KeeperSuite) TestAdd() { // Arrange s.keeper = NewKeeper[headForTest](2) @@ -114,21 +105,3 @@ func (s *KeeperSuite) TestRemove() { }, s.keeper.heads) s.Equal(Slots, cap(s.keeper.heads)) } - -func TestXxx(t *testing.T) { - ss := sortedSlice{ - {head: newHeadForTest("b"), createdAt: 2}, - {head: newHeadForTest("d"), createdAt: 4}, - } - - t.Log(ss) - - ss[0].head = newHeadForTest("b") - ss[0].createdAt = 5 - - t.Log(ss) - - heap.Fix(&ss, 0) - - t.Log(ss) -} diff --git a/pp/go/storage/head/services/persistener.go b/pp/go/storage/head/services/persistener.go index e9292a4b87..698237ee00 100644 --- a/pp/go/storage/head/services/persistener.go +++ b/pp/go/storage/head/services/persistener.go @@ -252,15 +252,9 @@ func (pg *PersistenerService[TTask, TShard, TGoShard, THeadBlockWriter, THead, T pg.loadRotatedHeadsInKeeper(heads) } -func (pg *PersistenerService[ - TTask, - TShard, - TGoShard, - THeadBlockWriter, - THead, - TKeeper, - TLoader, -]) persistHeads(heads []THead) { +func (pg *PersistenerService[TTask, TShard, TGoShard, THeadBlockWriter, THead, TKeeper, TLoader]) persistHeads( + heads []THead, +) { pg.keeper.Remove(pg.persistener.Persist(heads)) } @@ -304,6 +298,7 @@ func (pg *PersistenerService[ TLoader, ]) loadAndAddHeadToKeeper(record *catalog.Record) bool { head, _ := pg.loader.Load(record, 0) + head.SetReadOnly() if err := pg.keeper.Add(head, time.Duration(record.CreatedAt())*time.Millisecond); err != nil { _ = head.Close() return false diff --git a/pp/go/storage/head/shard/file_storage.go b/pp/go/storage/head/shard/file_storage.go index aaaeeec5e2..d00e30c318 100644 --- a/pp/go/storage/head/shard/file_storage.go +++ b/pp/go/storage/head/shard/file_storage.go @@ -2,61 +2,88 @@ package shard import "os" +// FileStorage wrapper over [os.File] for convenient operation. type FileStorage struct { fileName string file *os.File } +// NewFileStorage init new [FileStorage]. func NewFileStorage(fileName string) *FileStorage { return &FileStorage{fileName: fileName} } -func (q *FileStorage) ReadAt(p []byte, off int64) (n int, err error) { - return q.file.ReadAt(p, off) -} - -func (q *FileStorage) Open(flags int) (err error) { - if q.file == nil { - q.file, err = os.OpenFile(q.fileName, flags, 0666) +// Close closes the [File], rendering it unusable for I/O. On files that support [File.SetDeadline], +// any pending I/O operations will be canceled and return immediately with an [ErrClosed] error. +// Close will return an error if it has already been called. +func (q *FileStorage) Close() error { + if q.file != nil { + return q.file.Close() } - return + return nil } -func (q *FileStorage) Write(p []byte) (n int, err error) { - return q.file.Write(p) +// IsEmpty returns true if file is empty. +func (q *FileStorage) IsEmpty() bool { + if q.file != nil { + if info, err := q.file.Stat(); err == nil { + return info.Size() == 0 + } + } + + return true } -func (q *FileStorage) Close() error { - if q.file != nil { - return q.file.Close() +// Open open file for [FileStorage] with flags. +func (q *FileStorage) Open(flags int) (err error) { + if q.file == nil { + q.file, err = os.OpenFile( //nolint:gosec // need this permissions + q.fileName, + flags, + 0o666, //revive:disable-line:add-constant // file permissions simple readable as octa-number + ) } - return nil + return err } +// Read reads up to len(b) bytes from the File and stores them in b. +// It returns the number of bytes read and any error encountered. At end of file, Read returns 0, io.EOF. func (q *FileStorage) Read(p []byte) (n int, err error) { return q.file.Read(p) } +// ReadAt reads len(b) bytes from the File starting at byte offset off. +// It returns the number of bytes read and the error, if any. +// ReadAt always returns a non-nil error when n < len(b). At end of file, that error is io.EOF. +func (q *FileStorage) ReadAt(p []byte, off int64) (n int, err error) { + return q.file.ReadAt(p, off) +} + +// Seek sets the offset for the next Read or Write on file to offset, +// interpreted according to whence: 0 means relative to the origin of the file, +// 1 means relative to the current offset, and 2 means relative to the end. +// It returns the new offset and an error, if any. +// The behavior of Seek on a file opened with [O_APPEND] is not specified. func (q *FileStorage) Seek(offset int64, whence int) (int64, error) { return q.file.Seek(offset, whence) } +// Sync commits the current contents of the file to stable storage. +// Typically, this means flushing the file system's in-memory copy of recently written data to disk. func (q *FileStorage) Sync() error { return q.file.Sync() } +// Truncate changes the size of the file. It does not change the I/O offset. +// If there is an error, it will be of type [*PathError]. func (q *FileStorage) Truncate(size int64) error { return q.file.Truncate(size) } -func (q *FileStorage) IsEmpty() bool { - if q.file != nil { - if info, err := q.file.Stat(); err == nil { - return info.Size() == 0 - } - } - - return true +// Write writes len(b) bytes from b to the File. It returns the number of bytes written and an error, if any. +// Write returns a non-nil error when n != len(b). +func (q *FileStorage) Write(p []byte) (n int, err error) { + return q.file.Write(p) } diff --git a/pp/go/storage/head/shard/unloaded_data_storage.go b/pp/go/storage/head/shard/unloaded_data_storage.go index 100495441a..304eaf68ce 100644 --- a/pp/go/storage/head/shard/unloaded_data_storage.go +++ b/pp/go/storage/head/shard/unloaded_data_storage.go @@ -32,7 +32,10 @@ type UnloadedDataSnapshotHeader struct { } func NewUnloadedDataSnapshotHeader(snapshot []byte) UnloadedDataSnapshotHeader { - return UnloadedDataSnapshotHeader{Crc32: crc32.ChecksumIEEE(snapshot), SnapshotSize: uint32(len(snapshot))} + return UnloadedDataSnapshotHeader{ + Crc32: crc32.ChecksumIEEE(snapshot), + SnapshotSize: uint32(len(snapshot)), // #nosec G115 // no overflow + } } func (h UnloadedDataSnapshotHeader) IsValid(snapshot []byte) bool { diff --git a/pp/go/storage/head/shard/unloaded_data_storage_test.go b/pp/go/storage/head/shard/unloaded_data_storage_test.go index 6bbd919ad6..856f0c3aab 100644 --- a/pp/go/storage/head/shard/unloaded_data_storage_test.go +++ b/pp/go/storage/head/shard/unloaded_data_storage_test.go @@ -12,44 +12,54 @@ import ( "github.com/stretchr/testify/suite" ) +// BufferReaderAtWriterCloser implementation [FileStorage]. type BufferReaderAtWriterCloser struct { buffer []byte } -func (s *BufferReaderAtWriterCloser) IsEmpty() bool { +// IsEmpty implementation [FileStorage]. +func (*BufferReaderAtWriterCloser) IsEmpty() bool { return true } -func (s *BufferReaderAtWriterCloser) Open(flags int) error { +// Open implementation [FileStorage]. +func (*BufferReaderAtWriterCloser) Open(int) error { return nil } -func (s *BufferReaderAtWriterCloser) Read(p []byte) (n int, err error) { +// Read implementation [FileStorage]. +func (*BufferReaderAtWriterCloser) Read([]byte) (n int, err error) { return 0, nil } -func (s *BufferReaderAtWriterCloser) Seek(offset int64, whence int) (int64, error) { +// Seek implementation [FileStorage]. +func (*BufferReaderAtWriterCloser) Seek(int64, int) (int64, error) { return 0, nil } -func (s *BufferReaderAtWriterCloser) Sync() error { +// Sync implementation [FileStorage]. +func (*BufferReaderAtWriterCloser) Sync() error { return nil } -func (s *BufferReaderAtWriterCloser) Truncate(size int64) error { +// Truncate implementation [FileStorage]. +func (*BufferReaderAtWriterCloser) Truncate(int64) error { return nil } +// ReadAt implementation [FileStorage]. func (s *BufferReaderAtWriterCloser) ReadAt(p []byte, off int64) (n int, err error) { return bytes.NewReader(s.buffer).ReadAt(p, off) } +// Write implementation [FileStorage]. func (s *BufferReaderAtWriterCloser) Write(p []byte) (n int, err error) { s.buffer = append(s.buffer, p...) return len(p), nil } -func (s *BufferReaderAtWriterCloser) Close() error { +// Close implementation [FileStorage]. +func (*BufferReaderAtWriterCloser) Close() error { return nil } @@ -75,7 +85,7 @@ func (s *UnloadedDataStorageSuite) Write(snapshot []byte) { func (s *UnloadedDataStorageSuite) readSnapshots() ([]string, error) { var snapshots []string - return snapshots, s.storage.ForEachSnapshot(func(snapshot []byte, isLast bool) { + return snapshots, s.storage.ForEachSnapshot(func(snapshot []byte, _ bool) { snapshots = append(snapshots, string(snapshot)) }) } @@ -99,8 +109,8 @@ func (s *UnloadedDataStorageSuite) TestReadEmptySnapshots() { snapshots, err := s.readSnapshots() // Assert + s.Require().NoError(err) s.Equal([]string(nil), snapshots) - s.Equal(nil, err) } func (s *UnloadedDataStorageSuite) TestReadOneSnapshot() { @@ -111,8 +121,8 @@ func (s *UnloadedDataStorageSuite) TestReadOneSnapshot() { snapshots, err := s.readSnapshots() // Assert + s.Require().NoError(err) s.Equal([]string{"12345"}, snapshots) - s.Equal(nil, err) } func (s *UnloadedDataStorageSuite) TestReadMultipleSnapshots() { @@ -125,8 +135,8 @@ func (s *UnloadedDataStorageSuite) TestReadMultipleSnapshots() { snapshots, err := s.readSnapshots() // Assert + s.Require().NoError(err) s.Equal([]string{"123", "45678", "90"}, snapshots) - s.Equal(nil, err) } func (s *UnloadedDataStorageSuite) TestReadEof() { @@ -138,8 +148,8 @@ func (s *UnloadedDataStorageSuite) TestReadEof() { snapshots, err := s.readSnapshots() // Assert + s.Require().ErrorIs(err, io.EOF) s.Equal([]string(nil), snapshots) - s.Equal(fmt.Errorf("EOF"), err) } func (s *UnloadedDataStorageSuite) TestReadVersionError() { @@ -151,8 +161,8 @@ func (s *UnloadedDataStorageSuite) TestReadVersionError() { snapshots, err := s.readSnapshots() // Assert + s.Require().Equal(io.EOF, err) s.Equal([]string(nil), snapshots) - s.Equal(io.EOF, err) } func (s *UnloadedDataStorageSuite) TestInvalidVersion() { @@ -229,7 +239,7 @@ func (s *QueriedSeriesStorageSuite) TestOpenErrorOnWrite() { err := s.storage.Write([]byte("12345"), 1234567890) // Assert - s.Error(err) + s.Require().Error(err) s.Nil(s.storage.validStorage) s.Equal(s.file2, s.storage.storages[0]) } @@ -241,11 +251,11 @@ func (s *QueriedSeriesStorageSuite) TestWriteInFirstStorage() { err := s.storage.Write([]byte("12345"), 1234567890) // Assert - s.NoError(err) + s.Require().NoError(err) s.Equal([]byte{ QueriedSeriesStorageVersion, // version 0xd2, 0x02, 0x96, 0x49, 0x00, 0x00, 0x00, 0x00, // timestamp - 0x4e, 0x78, 0xf9, 0xf3, //crc32 + 0x4e, 0x78, 0xf9, 0xf3, // crc32 0x05, 0x00, 0x00, 0x00, // size '1', '2', '3', '4', '5', // content }, s.readFile(s.file1)) @@ -265,14 +275,14 @@ func (s *QueriedSeriesStorageSuite) TestWriteInAllStorages() { s.Equal([]byte{ QueriedSeriesStorageVersion, // version 0xd2, 0x02, 0x96, 0x49, 0x00, 0x00, 0x00, 0x00, // timestamp - 0x4e, 0x78, 0xf9, 0xf3, //crc32 + 0x4e, 0x78, 0xf9, 0xf3, // crc32 0x05, 0x00, 0x00, 0x00, // size '1', '2', '3', '4', '5', // content }, s.readFile(s.file1)) s.Equal([]byte{ QueriedSeriesStorageVersion, // version 0xb1, 0x68, 0xde, 0x3a, 0x00, 0x00, 0x00, 0x00, // timestamp - 0x21, 0x33, 0xf7, 0xb8, //crc32 + 0x21, 0x33, 0xf7, 0xb8, // crc32 0x05, 0x00, 0x00, 0x00, // size '6', '7', '8', '9', '0', // content }, s.readFile(s.file2)) @@ -291,14 +301,14 @@ func (s *QueriedSeriesStorageSuite) TestMultipleWriteInFirstStorage() { s.Equal([]byte{ QueriedSeriesStorageVersion, // version 0xb1, 0x68, 0xde, 0x3a, 0x00, 0x00, 0x00, 0x00, // timestamp - 0x21, 0x33, 0xf7, 0xb8, //crc32 + 0x21, 0x33, 0xf7, 0xb8, // crc32 0x05, 0x00, 0x00, 0x00, // size '6', '7', '8', '9', '0', // content }, s.readFile(s.file1)) s.Equal([]byte{ QueriedSeriesStorageVersion, // version 0xd2, 0x02, 0x96, 0x49, 0x00, 0x00, 0x00, 0x00, // timestamp - 0x4e, 0x78, 0xf9, 0xf3, //crc32 + 0x4e, 0x78, 0xf9, 0xf3, // crc32 0x05, 0x00, 0x00, 0x00, // size '1', '2', '3', '4', '5', // content }, s.readFile(s.file2)) @@ -325,12 +335,12 @@ func (s *QueriedSeriesStorageSuite) TestChangeActiveFileOnOpenErrorWithoutValidF writeErr2 := s.storage.Write([]byte("12345"), 1234567890) // Assert - s.Error(writeErr1) - s.NoError(writeErr2) + s.Require().Error(writeErr1) + s.Require().NoError(writeErr2) s.Equal([]byte{ QueriedSeriesStorageVersion, // version 0xd2, 0x02, 0x96, 0x49, 0x00, 0x00, 0x00, 0x00, // timestamp - 0x4e, 0x78, 0xf9, 0xf3, //crc32 + 0x4e, 0x78, 0xf9, 0xf3, // crc32 0x05, 0x00, 0x00, 0x00, // size '1', '2', '3', '4', '5', // content }, s.readFile(s.file2)) @@ -342,7 +352,7 @@ func (s *QueriedSeriesStorageSuite) TestNoChangeActiveFileOnOpenErrorWithValidFi s.writeFile(s.file1, []byte{ QueriedSeriesStorageVersion, // version 0xd2, 0x02, 0x96, 0x49, 0x00, 0x00, 0x00, 0x00, // timestamp - 0x4e, 0x78, 0xf9, 0xf3, //crc32 + 0x4e, 0x78, 0xf9, 0xf3, // crc32 0x05, 0x00, 0x00, 0x00, '1', '2', '3', '4', '5', }) @@ -355,8 +365,8 @@ func (s *QueriedSeriesStorageSuite) TestNoChangeActiveFileOnOpenErrorWithValidFi // Assert s.Require().NoError(readErr) s.Equal([]byte("12345"), data) - s.Error(writeErr1) - s.Error(writeErr2) + s.Require().Error(writeErr1) + s.Require().Error(writeErr2) } func (s *QueriedSeriesStorageSuite) TestReadEmptyFiles() { @@ -388,7 +398,7 @@ func (s *QueriedSeriesStorageSuite) TestInvalidHeaderInAllStorages() { invalidHeader := []byte{ QueriedSeriesStorageVersion, // version 0xd2, 0x02, 0x96, 0x49, 0x00, 0x00, 0x00, 0x00, // timestamp - 0x4e, 0x78, 0xf9, 0xf3, //crc32 + 0x4e, 0x78, 0xf9, 0xf3, // crc32 0x05, 0x00, 0x00, } s.writeFile(s.file1, invalidHeader) @@ -407,7 +417,7 @@ func (s *QueriedSeriesStorageSuite) TestInvalidDataInAllStorages() { invalidData := []byte{ QueriedSeriesStorageVersion, // version 0xd2, 0x02, 0x96, 0x49, 0x00, 0x00, 0x00, 0x00, // timestamp - 0x4e, 0x78, 0xf9, 0xf3, //crc32 + 0x4e, 0x78, 0xf9, 0xf3, // crc32 0x05, 0x00, 0x00, 0x00, '1', '2', '3', '4', } @@ -427,7 +437,7 @@ func (s *QueriedSeriesStorageSuite) TestInvalidCrc32InAllStorages() { invalidCrc32 := []byte{ QueriedSeriesStorageVersion, // version 0xd2, 0x02, 0x96, 0x49, 0x00, 0x00, 0x00, 0x00, // timestamp - 0x4e, 0x78, 0xf9, 0xf2, //crc32 + 0x4e, 0x78, 0xf9, 0xf2, // crc32 0x05, 0x00, 0x00, 0x00, '1', '2', '3', '4', '5', } @@ -447,7 +457,7 @@ func (s *QueriedSeriesStorageSuite) TestReadFromFirstStorageAndChangeActiveStora s.writeFile(s.file1, []byte{ QueriedSeriesStorageVersion, // version 0xd2, 0x02, 0x96, 0x49, 0x00, 0x00, 0x00, 0x00, // timestamp - 0x4e, 0x78, 0xf9, 0xf3, //crc32 + 0x4e, 0x78, 0xf9, 0xf3, // crc32 0x05, 0x00, 0x00, 0x00, '1', '2', '3', '4', '5', }) @@ -462,7 +472,7 @@ func (s *QueriedSeriesStorageSuite) TestReadFromFirstStorageAndChangeActiveStora s.Equal([]byte{ QueriedSeriesStorageVersion, // version 0xb1, 0x68, 0xde, 0x3a, 0x00, 0x00, 0x00, 0x00, // timestamp - 0x21, 0x33, 0xf7, 0xb8, //crc32 + 0x21, 0x33, 0xf7, 0xb8, // crc32 0x05, 0x00, 0x00, 0x00, // size '6', '7', '8', '9', '0', // content }, s.readFile(s.file2)) @@ -473,7 +483,7 @@ func (s *QueriedSeriesStorageSuite) TestReadFromSecondStorage() { s.writeFile(s.file2, []byte{ QueriedSeriesStorageVersion, // version 0xd2, 0x02, 0x96, 0x49, 0x00, 0x00, 0x00, 0x00, // timestamp - 0x4e, 0x78, 0xf9, 0xf3, //crc32 + 0x4e, 0x78, 0xf9, 0xf3, // crc32 0x05, 0x00, 0x00, 0x00, '1', '2', '3', '4', '5', }) @@ -491,14 +501,14 @@ func (s *QueriedSeriesStorageSuite) TestReadFromStorageWithMaxTimestamp() { s.writeFile(s.file1, []byte{ QueriedSeriesStorageVersion, // version 0xb1, 0x68, 0xde, 0x3a, 0x00, 0x00, 0x00, 0x00, // timestamp - 0x21, 0x33, 0xf7, 0xb8, //crc32 + 0x21, 0x33, 0xf7, 0xb8, // crc32 0x05, 0x00, 0x00, 0x00, // size '6', '7', '8', '9', '0', // content }) s.writeFile(s.file2, []byte{ QueriedSeriesStorageVersion, // version 0xd3, 0x02, 0x96, 0x49, 0x00, 0x00, 0x00, 0x00, // timestamp - 0xfd, 0x12, 0xf7, 0xe0, //crc32 + 0xfd, 0x12, 0xf7, 0xe0, // crc32 0x04, 0x00, 0x00, 0x00, '6', '7', '8', '9', }) @@ -516,7 +526,7 @@ func (s *QueriedSeriesStorageSuite) TestReadEmptyContent() { s.writeFile(s.file1, []byte{ QueriedSeriesStorageVersion, // version 0xb1, 0x68, 0xde, 0x3a, 0x00, 0x00, 0x00, 0x00, // timestamp - 0x41, 0x01, 0x44, 0x30, //crc32 + 0x41, 0x01, 0x44, 0x30, // crc32 0x00, 0x00, 0x00, 0x00, // size }) diff --git a/pp/go/storage/loader.go b/pp/go/storage/loader.go index ddd97ac5c5..2e647d2e49 100644 --- a/pp/go/storage/loader.go +++ b/pp/go/storage/loader.go @@ -136,6 +136,8 @@ func (l *Loader) Load( corrupted = true } + logger.Debugf("[Loader] loaded head: %s, corrupted: %t", headRecord.ID(), corrupted) + return h, corrupted } diff --git a/pp/go/storage/loader_test.go b/pp/go/storage/loader_test.go index 5f50574fe1..bdf963a5a8 100644 --- a/pp/go/storage/loader_test.go +++ b/pp/go/storage/loader_test.go @@ -127,12 +127,12 @@ func (s *HeadLoadSuite) appendTimeSeries(head *storage.HeadOnDisk, timeSeries [] storagetest.MustAppendTimeSeries(&s.Suite, head, timeSeries) } -func (s *HeadLoadSuite) shards(head *storage.HeadOnDisk) (result []*storage.ShardOnDisk) { +func (*HeadLoadSuite) shards(head *storage.HeadOnDisk) (result []*storage.ShardOnDisk) { for shard := range head.RangeShards() { result = append(result, shard) } - return + return result } func (s *HeadLoadSuite) TestErrorCreateShardFileInOneShard() { diff --git a/pp/go/storage/manager.go b/pp/go/storage/manager.go index 73ce7f406f..4820446283 100644 --- a/pp/go/storage/manager.go +++ b/pp/go/storage/manager.go @@ -119,6 +119,7 @@ func (c *Config) SetNumberOfShards(numberOfShards uint16) bool { // Manager // +// Manager manages services for the work of the heads. type Manager struct { g run.Group closer *util.Closer @@ -163,11 +164,8 @@ func NewManager( } builder := NewBuilder(hcatalog, o.DataDir, o.MaxSegmentSize, r, unloadDataStorageInterval) - loader := NewLoader(o.DataDir, o.MaxSegmentSize, r, unloadDataStorageInterval) - cfg := NewConfig(o.NumberOfShards) - h, err := uploadOrBuildHead(clock, hcatalog, builder, loader, o.BlockDuration, cfg.NumberOfShards()) if err != nil { return nil, err @@ -178,7 +176,6 @@ func NewManager( } hKeeper := keeper.NewKeeper[HeadOnDisk](o.QueueSize) - m := &Manager{ g: run.Group{}, closer: util.NewCloser(), @@ -192,9 +189,7 @@ func NewManager( } readyNotifier.NotifyReady() - m.initServices(o, hcatalog, builder, loader, triggerNotifier, clock, r) - logger.Infof("[Head Manager] created") return m, nil @@ -489,23 +484,3 @@ func uploadOrBuildHead( return h, nil } - -// -// NoopKeeper -// - -// NoopKeeper implements Keeper. -type NoopKeeper struct{} - -// Add implements Keeper. -func (*NoopKeeper) Add(*HeadOnDisk) {} - -// Close implements Keeper. -func (*NoopKeeper) Close() error { return nil } - -// RangeQueriableHeads implements Keeper. -func (k *NoopKeeper) RangeQueriableHeads( - mint, maxt int64, -) func(func(*HeadOnDisk) bool) { - return func(func(*HeadOnDisk) bool) {} -} diff --git a/pp/go/storage/storagetest/fixtures.go b/pp/go/storage/storagetest/fixtures.go index b9dd342c35..db0fc2f7cf 100644 --- a/pp/go/storage/storagetest/fixtures.go +++ b/pp/go/storage/storagetest/fixtures.go @@ -14,11 +14,13 @@ import ( "github.com/stretchr/testify/suite" ) +// TimeSeries test data. type TimeSeries struct { Labels labels.Labels Samples []cppbridge.Sample } +// AppendSamples add samples to time series. func (s *TimeSeries) AppendSamples(samples ...cppbridge.Sample) { s.Samples = append(s.Samples, samples...) } @@ -35,7 +37,7 @@ func (s *TimeSeries) toModelTimeSeries() []model.TimeSeries { for i := range s.Samples { timeSeries = append(timeSeries, model.TimeSeries{ LabelSet: ls, - Timestamp: uint64(s.Samples[i].Timestamp), + Timestamp: uint64(s.Samples[i].Timestamp), // #nosec G115 // no overflow Value: s.Samples[i].Value, }) } @@ -55,6 +57,7 @@ func (tsd *timeSeriesDataSlice) Destroy() { tsd.timeSeries = nil } +// MustAppendTimeSeries add time series to head. func MustAppendTimeSeries(s *suite.Suite, head *storage.HeadOnDisk, timeSeries []TimeSeries) { headAppender := appender.New(head, services.CFViaRange) @@ -67,7 +70,7 @@ func MustAppendTimeSeries(s *suite.Suite, head *storage.HeadOnDisk, timeSeries [ for i := range timeSeries { tsd := timeSeriesDataSlice{timeSeries: timeSeries[i].toModelTimeSeries()} hx, err := (cppbridge.HashdexFactory{}).GoModel(tsd.TimeSeries(), cppbridge.DefaultWALHashdexLimits()) - s.NoError(err) + s.Require().NoError(err) _, _, err = headAppender.Append( context.Background(), @@ -78,8 +81,10 @@ func MustAppendTimeSeries(s *suite.Suite, head *storage.HeadOnDisk, timeSeries [ } } +// SamplesMap samples map with series ID as key. type SamplesMap map[uint32][]cppbridge.Sample +// GetSamplesFromSerializedChunks returns sample from serialized chunks. func GetSamplesFromSerializedChunks(chunks *cppbridge.HeadDataStorageSerializedChunks) SamplesMap { result := make(SamplesMap) @@ -93,13 +98,13 @@ func GetSamplesFromSerializedChunks(chunks *cppbridge.HeadDataStorageSerializedC for iterator.Next() { ts, value := iterator.Sample() result[seriesId] = append(result[seriesId], cppbridge.Sample{Timestamp: ts, Value: value}) - } } return result } +// TimeSeriesFromSeriesSet converting seriesset to slice timeseries. func TimeSeriesFromSeriesSet(seriesSet promstorage.SeriesSet) []TimeSeries { var timeSeries []TimeSeries for seriesSet.Next() { diff --git a/pp/go/storage/types.go b/pp/go/storage/types.go index 65967639b1..246c16fead 100644 --- a/pp/go/storage/types.go +++ b/pp/go/storage/types.go @@ -2,13 +2,11 @@ package storage import ( "github.com/prometheus/prometheus/pp/go/cppbridge" - "github.com/prometheus/prometheus/pp/go/storage/appender" "github.com/prometheus/prometheus/pp/go/storage/head/head" "github.com/prometheus/prometheus/pp/go/storage/head/proxy" "github.com/prometheus/prometheus/pp/go/storage/head/shard" "github.com/prometheus/prometheus/pp/go/storage/head/shard/wal" "github.com/prometheus/prometheus/pp/go/storage/head/shard/wal/writer" - "github.com/prometheus/prometheus/pp/go/storage/head/task" ) // WalOnDisk wal on disk. @@ -29,10 +27,3 @@ type HeadOnDisk = head.Head[*ShardOnDisk, *PerGoroutineShard] // ProxyHead [proxy.Proxy] for [HeadOnDisk]s. type ProxyHead = proxy.Proxy[*HeadOnDisk] - -type AppenderHead = appender.Appender[ - *task.Generic[*PerGoroutineShard], - *shard.LSS, - *PerGoroutineShard, - *HeadOnDisk, -] From bb81c0a1047fa3c534ac6f20e1ada391e92fb8ea Mon Sep 17 00:00:00 2001 From: Alexandr Yudin Date: Fri, 26 Sep 2025 14:17:30 +0000 Subject: [PATCH 52/96] some fix --- cmd/prometheus/main.go | 7 +- pp-pkg/logger/logger.go | 3 +- pp/go/storage/builder.go | 7 +- pp/go/storage/catalog/gc.go | 58 +++++++++---- pp/go/storage/head/head/head.go | 4 +- pp/go/storage/head/services/persistener.go | 77 ++++++++++++---- .../storage/head/services/persistener_test.go | 39 ++++++--- .../head/shard/unloaded_data_storage.go | 9 +- pp/go/storage/loader.go | 58 +++++++++---- pp/go/storage/manager.go | 39 +++++++-- pp/go/storage/mediator/timer.go | 4 +- pp/go/storage/remotewriter/datasource.go | 2 +- pp/go/storage/remotewriter/walreader.go | 8 +- pp/go/storage/remotewriter/writeloop.go | 4 +- pp/go/storage/remotewriter/writer.go | 10 +-- web/web_fuzzy_test.go | 87 ++++++++++--------- 16 files changed, 284 insertions(+), 132 deletions(-) diff --git a/cmd/prometheus/main.go b/cmd/prometheus/main.go index 436e4f9f5c..14a9411dfd 100644 --- a/cmd/prometheus/main.go +++ b/cmd/prometheus/main.go @@ -742,6 +742,11 @@ func main() { level.Error(logger).Log("msg", "failed to create file log", "err", err) os.Exit(1) } + defer func() { + if err := fileLog.Close(); err != nil { + level.Error(logger).Log("msg", "failed to close file log", "err", err) + } + }() clock := clockwork.NewRealClock() headCatalog, err := catalog.New( @@ -1162,7 +1167,7 @@ func main() { ).Add( remoteWriterReadyNotifier, ).Build() - opGC := catalog.NewGC(dataDir, headCatalog, multiNotifiable) + opGC := catalog.NewGC(dataDir, headCatalog, clock, multiNotifiable, time.Duration(cfg.tsdb.RetentionDuration)) var g run.Group { diff --git a/pp-pkg/logger/logger.go b/pp-pkg/logger/logger.go index 8186357244..6f7e020067 100644 --- a/pp-pkg/logger/logger.go +++ b/pp-pkg/logger/logger.go @@ -16,8 +16,7 @@ func InitLogHandler(l log.Logger) { l = log.With(l, "pp_caller", log.Caller(4)) //revive:disable-line:add-constant // caller id logger.Debugf = func(template string, args ...any) { - // _ = level.Debug(l).Log(msg, fmt.Sprintf(template, args...)) - _ = level.Info(l).Log(msg, fmt.Sprintf(template, args...)) + _ = level.Debug(l).Log(msg, fmt.Sprintf(template, args...)) } logger.Infof = func(template string, args ...any) { diff --git a/pp/go/storage/builder.go b/pp/go/storage/builder.go index 38dc492609..c9131e488c 100644 --- a/pp/go/storage/builder.go +++ b/pp/go/storage/builder.go @@ -106,7 +106,11 @@ func (b *Builder) createShardOnDisk( shardID uint16, ) (*ShardOnDisk, error) { headDir = filepath.Clean(headDir) - shardFile, err := os.Create(GetShardWalFilename(headDir, shardID)) + shardFile, err := os.OpenFile( //nolint:gosec // need this permissions + GetShardWalFilename(headDir, shardID), + os.O_WRONLY|os.O_CREATE, + 0o666, //revive:disable-line:add-constant // file permissions simple readable as octa-number + ) if err != nil { return nil, fmt.Errorf("failed to create shard wal file id %d: %w", shardID, err) } @@ -115,6 +119,7 @@ func (b *Builder) createShardOnDisk( if err == nil { return } + _ = shardFile.Close() }() diff --git a/pp/go/storage/catalog/gc.go b/pp/go/storage/catalog/gc.go index 845627611d..bf93394dcf 100644 --- a/pp/go/storage/catalog/gc.go +++ b/pp/go/storage/catalog/gc.go @@ -7,6 +7,7 @@ import ( "path/filepath" "time" + "github.com/jonboulle/clockwork" "github.com/prometheus/prometheus/pp/go/logger" ) @@ -39,21 +40,31 @@ type Notifiable interface { // GC garbage collector for old [Head]. type GC struct { - dataDir string - catalog HeadsCatalog - readyNotifiable Notifiable - stop chan struct{} - stopped chan struct{} + dataDir string + catalog HeadsCatalog + clock clockwork.Clock + readyNotifiable Notifiable + maxRetentionPeriod time.Duration + stop chan struct{} + stopped chan struct{} } // NewGC init new [GC]. -func NewGC(dataDir string, catalog HeadsCatalog, readyNotifiable Notifiable) *GC { +func NewGC( + dataDir string, + catalog HeadsCatalog, + clock clockwork.Clock, + readyNotifiable Notifiable, + maxRetentionPeriod time.Duration, +) *GC { return &GC{ - dataDir: dataDir, - catalog: catalog, - readyNotifiable: readyNotifiable, - stop: make(chan struct{}), - stopped: make(chan struct{}), + dataDir: dataDir, + catalog: catalog, + clock: clock, + readyNotifiable: readyNotifiable, + maxRetentionPeriod: maxRetentionPeriod, + stop: make(chan struct{}), + stopped: make(chan struct{}), } } @@ -63,18 +74,17 @@ func (gc *GC) Iterate() { defer logger.Debugf("catalog gc iteration: head ended") records := gc.catalog.List( - func(record *Record) bool { - return record.DeletedAt() == 0 - }, + gc.possibleRemoval, func(lhs, rhs *Record) bool { return lhs.CreatedAt() < rhs.CreatedAt() }, ) for _, record := range records { - if record.deletedAt != 0 { + if record.DeletedAt() != 0 { continue } + logger.Debugf("catalog gc iteration: head: %s", record.ID()) if record.ReferenceCount() > 0 { return @@ -128,3 +138,21 @@ func (gc *GC) Stop() { close(gc.stop) <-gc.stopped } + +// possibleRemoval a filter to remove unwanted wals. +func (gc *GC) possibleRemoval(record *Record) bool { + if record.DeletedAt() != 0 { + return false + } + + // the head is outdated and data on it is no longer required + if gc.clock.Since(time.UnixMilli(record.CreatedAt())) >= gc.maxRetentionPeriod { + return true + } + + if record.Status() != StatusPersisted { + return false + } + + return true +} diff --git a/pp/go/storage/head/head/head.go b/pp/go/storage/head/head/head.go index fa6729cc6e..23dba6a3ea 100644 --- a/pp/go/storage/head/head/head.go +++ b/pp/go/storage/head/head/head.go @@ -127,10 +127,10 @@ func NewHead[TShard Shard, TGoroutineShard Shard]( runtime.SetFinalizer(h, func(h *Head[TShard, TGoroutineShard]) { h.memoryInUse.DeletePartialMatch(prometheus.Labels{"head_id": h.id}) - logger.Debugf("[Head] %s destroyed.", h.String()) + logger.Debugf("[Head] %s destroyed", h.String()) }) - logger.Debugf("[Head] %s created.", h.String()) + logger.Debugf("[Head] %s created", h.String()) return h } diff --git a/pp/go/storage/head/services/persistener.go b/pp/go/storage/head/services/persistener.go index 698237ee00..2cb957adfa 100644 --- a/pp/go/storage/head/services/persistener.go +++ b/pp/go/storage/head/services/persistener.go @@ -13,6 +13,10 @@ import ( "github.com/prometheus/prometheus/pp/go/util" ) +// defaultCoolingInterval the interval after which the rotation should have +// taken place to eliminate errors in the selection from the catalog. +const defaultCoolingInterval = 60 * time.Second + // // Persistener // @@ -81,6 +85,7 @@ func NewPersistener[ // Persist spent [Head]s. // +//revive:disable-next-line:function-length // long but readable. //revive:disable-next-line:cognitive-complexity // long but readable. //revive:disable-next-line:cyclomatic // long but readable. func (p *Persistener[TTask, TShard, TGoShard, THeadBlockWriter, THead]) Persist(heads []THead) (outdatedHeads []THead) { @@ -91,6 +96,7 @@ func (p *Persistener[TTask, TShard, TGoShard, THeadBlockWriter, THead]) Persist( continue } + logger.Debugf("[Persistener]: head %s start persist", head.ID()) if record, err := p.catalog.Get(head.ID()); err != nil { logger.Errorf("[Persistener]: failed get head %s from catalog: %v", head.ID(), err) } else if record.Status() == catalog.StatusPersisted { @@ -102,6 +108,12 @@ func (p *Persistener[TTask, TShard, TGoShard, THeadBlockWriter, THead]) Persist( } if p.HeadIsOutdated(head) { + // the head is outdated and data on it is no longer required + if _, err := p.catalog.SetStatus(head.ID(), catalog.StatusPersisted); err != nil { + logger.Errorf("[Persistener]: set head status in catalog %s: %v", head.ID(), err) + continue + } + outdatedHeads = append(outdatedHeads, head) continue } @@ -187,14 +199,16 @@ type PersistenerService[ TShard, TGoShard Shard, THeadBlockWriter HeadBlockWriter[TShard], THead Head[TTask, TShard, TGoShard], - TKeeper Keeper[TTask, TShard, TGoShard, THead], + TProxyHead ProxyHead[TTask, TShard, TGoShard, THead], TLoader Loader[TTask, TShard, TGoShard, THead], ] struct { - persistener *Persistener[TTask, TShard, TGoShard, THeadBlockWriter, THead] - keeper TKeeper - loader TLoader - catalog *catalog.Catalog - mediator Mediator + persistener *Persistener[TTask, TShard, TGoShard, THeadBlockWriter, THead] + proxy TProxyHead + loader TLoader + catalog *catalog.Catalog + mediator Mediator + clock clockwork.Clock + tsdbRetentionPeriod time.Duration } // NewPersistenerService init new [PersistenerService]. @@ -203,10 +217,10 @@ func NewPersistenerService[ TShard, TGoShard Shard, THeadBlockWriter HeadBlockWriter[TShard], THead Head[TTask, TShard, TGoShard], - TKeeper Keeper[TTask, TShard, TGoShard, THead], + TProxyHead ProxyHead[TTask, TShard, TGoShard, THead], TLoader Loader[TTask, TShard, TGoShard, THead], ]( - hkeeper TKeeper, + proxy TProxyHead, loader TLoader, hcatalog *catalog.Catalog, blockWriter THeadBlockWriter, @@ -216,8 +230,8 @@ func NewPersistenerService[ tsdbRetentionPeriod time.Duration, retentionPeriod time.Duration, registerer prometheus.Registerer, -) *PersistenerService[TTask, TShard, TGoShard, THeadBlockWriter, THead, TKeeper, TLoader] { - return &PersistenerService[TTask, TShard, TGoShard, THeadBlockWriter, THead, TKeeper, TLoader]{ +) *PersistenerService[TTask, TShard, TGoShard, THeadBlockWriter, THead, TProxyHead, TLoader] { + return &PersistenerService[TTask, TShard, TGoShard, THeadBlockWriter, THead, TProxyHead, TLoader]{ persistener: NewPersistener[TTask, TShard, TGoShard, THeadBlockWriter, THead]( hcatalog, blockWriter, @@ -227,10 +241,12 @@ func NewPersistenerService[ retentionPeriod, registerer, ), - keeper: hkeeper, - loader: loader, - catalog: hcatalog, - mediator: mediator, + proxy: proxy, + loader: loader, + catalog: hcatalog, + mediator: mediator, + clock: clock, + tsdbRetentionPeriod: tsdbRetentionPeriod, } } @@ -247,7 +263,7 @@ func (pg *PersistenerService[TTask, TShard, TGoShard, THeadBlockWriter, THead, T // ProcessHeads process persist [Head]s. func (pg *PersistenerService[TTask, TShard, TGoShard, THeadBlockWriter, THead, TKeeper, TLoader]) ProcessHeads() { - heads := pg.keeper.Heads() + heads := pg.proxy.Heads() pg.persistHeads(heads) pg.loadRotatedHeadsInKeeper(heads) } @@ -255,9 +271,12 @@ func (pg *PersistenerService[TTask, TShard, TGoShard, THeadBlockWriter, THead, T func (pg *PersistenerService[TTask, TShard, TGoShard, THeadBlockWriter, THead, TKeeper, TLoader]) persistHeads( heads []THead, ) { - pg.keeper.Remove(pg.persistener.Persist(heads)) + pg.proxy.Remove(pg.persistener.Persist(heads)) } +// loadRotatedHeadsInKeeper loads rotated or unused [Head]s and adds them to the [Keeper]. +// +//revive:disable-next-line:cyclomatic // but readable func (pg *PersistenerService[ TTask, TShard, @@ -267,7 +286,7 @@ func (pg *PersistenerService[ TKeeper, TLoader, ]) loadRotatedHeadsInKeeper(keeperHeads []THead) { - if !pg.keeper.HasSlot() { + if !pg.proxy.HasSlot() { return } @@ -278,16 +297,36 @@ func (pg *PersistenerService[ } records := pg.catalog.List(func(record *catalog.Record) bool { - return record.Status() == catalog.StatusRotated && !headExists(record.ID()) + // in case the rotated status was not set due to an error + statusIsAppropriate := record.Status() == catalog.StatusNew || + record.Status() == catalog.StatusRotated || + record.Status() == catalog.StatusActive + + isOutdated := pg.clock.Since(time.UnixMilli(record.CreatedAt())) >= pg.tsdbRetentionPeriod + + return statusIsAppropriate && !headExists(record.ID()) && record.DeletedAt() == 0 && !isOutdated }, catalog.LessByUpdateAt) + aheadID := pg.proxy.Get().ID() for _, record := range records { + // skip active head + if aheadID == record.ID() { + continue + } + + // skip the newly created head + if (record.Status() == catalog.StatusNew || record.Status() == catalog.StatusActive) && + pg.clock.Since(time.UnixMilli(record.CreatedAt())) < defaultCoolingInterval { + continue + } + if !pg.loadAndAddHeadToKeeper(record) { break } } } +// loadAndAddHeadToKeeper loads [Head] and adds them to the [Keeper]. func (pg *PersistenerService[ TTask, TShard, @@ -299,7 +338,7 @@ func (pg *PersistenerService[ ]) loadAndAddHeadToKeeper(record *catalog.Record) bool { head, _ := pg.loader.Load(record, 0) head.SetReadOnly() - if err := pg.keeper.Add(head, time.Duration(record.CreatedAt())*time.Millisecond); err != nil { + if err := pg.proxy.Add(head, time.Duration(record.CreatedAt())*time.Millisecond); err != nil { _ = head.Close() return false } diff --git a/pp/go/storage/head/services/persistener_test.go b/pp/go/storage/head/services/persistener_test.go index d1e2521648..90cc75e96b 100644 --- a/pp/go/storage/head/services/persistener_test.go +++ b/pp/go/storage/head/services/persistener_test.go @@ -14,7 +14,9 @@ import ( "github.com/prometheus/prometheus/pp/go/storage" "github.com/prometheus/prometheus/pp/go/storage/block" "github.com/prometheus/prometheus/pp/go/storage/catalog" + "github.com/prometheus/prometheus/pp/go/storage/head/container" "github.com/prometheus/prometheus/pp/go/storage/head/keeper" + "github.com/prometheus/prometheus/pp/go/storage/head/proxy" "github.com/prometheus/prometheus/pp/go/storage/head/services" "github.com/prometheus/prometheus/pp/go/storage/head/services/mock" "github.com/prometheus/prometheus/pp/go/storage/head/task" @@ -37,17 +39,20 @@ type GenericPersistenceSuite struct { dataDir string clock *clockwork.FakeClock catalog *catalog.Catalog - keeper *keeper.Keeper[storage.HeadOnDisk, *storage.HeadOnDisk] + proxy *proxy.Proxy[*storage.HeadOnDisk] blockWriter *mock.HeadBlockWriterMock[*storage.ShardOnDisk] writeNotifier *mock.WriteNotifierMock } func (s *GenericPersistenceSuite) SetupTest() { s.dataDir = s.createDataDirectory() - s.clock = clockwork.NewFakeClockAt(time.UnixMilli(0)) s.createCatalog() - s.keeper = keeper.NewKeeper[storage.HeadOnDisk](1) + + h := s.mustCreateHead() + activeHeadContainer := container.NewWeighted(h) + hKeeper := keeper.NewKeeper[storage.HeadOnDisk](1) + s.proxy = proxy.NewProxy(activeHeadContainer, hKeeper, func(*storage.HeadOnDisk) error { return nil }) s.blockWriter = &mock.HeadBlockWriterMock[*storage.ShardOnDisk]{} s.writeNotifier = &mock.WriteNotifierMock{NotifyWrittenFunc: func() {}} } @@ -99,6 +104,10 @@ type PersistenerSuite struct { ] } +// func (s *PersistenerSuite) SetupSuite() { +// s.GenericPersistenceSuite.SetupSuite() +// } + func (s *PersistenerSuite) SetupTest() { s.GenericPersistenceSuite.SetupTest() @@ -296,11 +305,15 @@ type PersistenerServiceSuite struct { *storage.PerGoroutineShard, *mock.HeadBlockWriterMock[*storage.ShardOnDisk], *storage.HeadOnDisk, - *keeper.Keeper[storage.HeadOnDisk, *storage.HeadOnDisk], + *proxy.Proxy[*storage.HeadOnDisk], *storage.Loader, ] } +// func (s *PersistenerServiceSuite) SetupSuite() { +// s.GenericPersistenceSuite.SetupSuite() +// } + func (s *PersistenerServiceSuite) SetupTest() { s.GenericPersistenceSuite.SetupTest() @@ -311,10 +324,10 @@ func (s *PersistenerServiceSuite) SetupTest() { *storage.PerGoroutineShard, *mock.HeadBlockWriterMock[*storage.ShardOnDisk], *storage.HeadOnDisk, - *keeper.Keeper[storage.HeadOnDisk, *storage.HeadOnDisk], + *proxy.Proxy[*storage.HeadOnDisk], *storage.Loader, ]( - s.keeper, + s.proxy, s.loader, s.catalog, s.blockWriter, @@ -345,13 +358,13 @@ func (s *PersistenerServiceSuite) TestRemoveOutdatedHeadFromKeeper() { }) head.SetReadOnly() record, _ := s.catalog.SetStatus(head.ID(), catalog.StatusRotated) - _ = s.keeper.Add(head, time.Duration(s.clock.Now().Nanosecond())) + _ = s.proxy.Add(head, time.Duration(s.clock.Now().Nanosecond())) // Act s.service.ProcessHeads() // Assert - s.Empty(s.keeper.Heads()) + s.Empty(s.proxy.Heads()) s.Equal(catalog.StatusRotated, record.Status()) } @@ -373,8 +386,8 @@ func (s *PersistenerServiceSuite) TestLoadHeadsInKeeper() { s.service.ProcessHeads() // Assert - s.Require().Len(s.keeper.Heads(), 1) - s.Equal(head.ID(), s.keeper.Heads()[0].ID()) + s.Require().Len(s.proxy.Heads(), 1) + s.Equal(head.ID(), s.proxy.Heads()[0].ID()) s.Equal(int64(0), record.CreatedAt()) } @@ -390,12 +403,12 @@ func (s *PersistenerServiceSuite) TestHeadAlreadyExistsInKeeper() { }, }) _, _ = s.catalog.SetStatus(head.ID(), catalog.StatusRotated) - _ = s.keeper.Add(head, 0) + _ = s.proxy.Add(head, 0) // Act s.service.ProcessHeads() // Assert - s.Require().Len(s.keeper.Heads(), 1) - s.Equal(head.ID(), s.keeper.Heads()[0].ID()) + s.Require().Len(s.proxy.Heads(), 1) + s.Equal(head.ID(), s.proxy.Heads()[0].ID()) } diff --git a/pp/go/storage/head/shard/unloaded_data_storage.go b/pp/go/storage/head/shard/unloaded_data_storage.go index 304eaf68ce..2383934ff4 100644 --- a/pp/go/storage/head/shard/unloaded_data_storage.go +++ b/pp/go/storage/head/shard/unloaded_data_storage.go @@ -12,10 +12,14 @@ import ( ) const ( - UnloadedDataStorageVersion = 1 + // UnloadedDataStorageVersion file version for [UnloadedDataStorageVersion]. + UnloadedDataStorageVersion = 1 + + // QueriedSeriesStorageVersion file version for [QueriedSeriesStorage]. QueriedSeriesStorageVersion = 1 ) +// StorageFile wrapper over [os.File] for convenient operation. type StorageFile interface { Open(flags int) error io.WriteCloser @@ -26,11 +30,13 @@ type StorageFile interface { IsEmpty() bool } +// UnloadedDataSnapshotHeader stubs for recording snapshots. type UnloadedDataSnapshotHeader struct { Crc32 uint32 SnapshotSize uint32 } +// NewUnloadedDataSnapshotHeader init new [UnloadedDataSnapshotHeader]. func NewUnloadedDataSnapshotHeader(snapshot []byte) UnloadedDataSnapshotHeader { return UnloadedDataSnapshotHeader{ Crc32: crc32.ChecksumIEEE(snapshot), @@ -38,6 +44,7 @@ func NewUnloadedDataSnapshotHeader(snapshot []byte) UnloadedDataSnapshotHeader { } } +// IsValid checks checksum if the header is valid. func (h UnloadedDataSnapshotHeader) IsValid(snapshot []byte) bool { return h.Crc32 == crc32.ChecksumIEEE(snapshot) } diff --git a/pp/go/storage/loader.go b/pp/go/storage/loader.go index 2e647d2e49..12c7f2a2b6 100644 --- a/pp/go/storage/loader.go +++ b/pp/go/storage/loader.go @@ -141,7 +141,7 @@ func (l *Loader) Load( return h, corrupted } -func (l *Loader) loadShard( +func (*Loader) loadShard( shardID uint16, dir string, maxSegmentSize uint32, @@ -164,12 +164,14 @@ func (l *Loader) loadShard( } } +// ShardLoadResult the result of loading a shard from a wal file. type ShardLoadResult struct { shard *ShardOnDisk numberOfSegments uint32 corrupted bool } +// ShardData data for creating a shard. type ShardData struct { lss *shard.LSS dataStorage *shard.DataStorage @@ -179,6 +181,7 @@ type ShardData struct { numberOfSegments uint32 } +// ShardDataLoader loads shard data from a file and creates a shard. type ShardDataLoader struct { shardID uint16 dir string @@ -188,6 +191,7 @@ type ShardDataLoader struct { unloadDataStorageInterval time.Duration } +// NewShardDataLoader init new [ShardDataLoader]. func NewShardDataLoader( shardID uint16, dir string, @@ -204,7 +208,8 @@ func NewShardDataLoader( } } -func (l *ShardDataLoader) Load() (err error) { +// Load loads shard data from a file and creates a shard. +func (l *ShardDataLoader) Load() error { l.shardData = ShardData{ lss: shard.NewLSS(), dataStorage: shard.NewDataStorage(), @@ -217,17 +222,14 @@ func (l *ShardDataLoader) Load() (err error) { shardWalFile, err := os.OpenFile( //nolint:gosec // need this permissions GetShardWalFilename(l.dir, l.shardID), - os.O_RDWR, + os.O_RDONLY, 0o666, //revive:disable-line:add-constant // file permissions simple readable as octa-number ) if err != nil { return err } - defer func() { - if err != nil { - _ = shardWalFile.Close() - } + _ = shardWalFile.Close() }() queriedSeriesStorageIsEmpty := true @@ -243,9 +245,16 @@ func (l *ShardDataLoader) Load() (err error) { return err } - return l.createShardWal(shardWalFile, decoder) + if err = l.createShardWal(shardWalFile.Name(), decoder); err != nil { + return err + } + + return nil } +// loadWalFile loads and decode wal file. +// +//revive:disable-next-line:flag-parameter this is a flag, but it's more convenient this way func (l *ShardDataLoader) loadWalFile( rd io.Reader, queriedSeriesStorageIsEmpty bool, @@ -272,22 +281,40 @@ func (l *ShardDataLoader) loadWalFile( l.shardData.dataStorage, unloader, ) + return decoder, err } -func (l *ShardDataLoader) createShardWal(shardWalFile *os.File, walDecoder *cppbridge.HeadWalDecoder) error { - if sw, err := writer.NewBuffered( +// createShardWal creates a wal for a shard. +func (l *ShardDataLoader) createShardWal(fileName string, walDecoder *cppbridge.HeadWalDecoder) error { + shardWalFile, err := os.OpenFile( //nolint:gosec // need this permissions + fileName, + os.O_WRONLY, + 0o666, //revive:disable-line:add-constant // file permissions simple readable as octa-number + ) + if err != nil { + return err + } + if _, err = shardWalFile.Seek(0, io.SeekEnd); err != nil { + _ = shardWalFile.Close() + return err + } + + sw, err := writer.NewBuffered( l.shardID, shardWalFile, writer.WriteSegment[*cppbridge.EncodedSegment], l.notifier, - ); err != nil { + ) + if err != nil { + _ = shardWalFile.Close() return err - } else { - l.notifier.Set(l.shardID, l.shardData.numberOfSegments) - l.shardData.wal = wal.NewWal(walDecoder.CreateEncoder(), sw, l.maxSegmentSize) - return nil } + + l.notifier.Set(l.shardID, l.shardData.numberOfSegments) + l.shardData.wal = wal.NewWal(walDecoder.CreateEncoder(), sw, l.maxSegmentSize) + + return nil } type dataUnloader struct { @@ -321,6 +348,7 @@ func (d *dataUnloader) Unload(createTs, encodeTs time.Duration) error { return nil } +// loadSegments loads and decode segments from wal file. func (l *ShardDataLoader) loadSegments( rd io.Reader, walDecoder *cppbridge.HeadWalDecoder, diff --git a/pp/go/storage/manager.go b/pp/go/storage/manager.go index 4820446283..2bc65258d2 100644 --- a/pp/go/storage/manager.go +++ b/pp/go/storage/manager.go @@ -45,6 +45,9 @@ const ( // DefaultUnloadDataStorageInterval the default interval for unloading [DataStorage]. DefaultUnloadDataStorageInterval = 5 * time.Minute + + // defaultStartPersistnerInterval the default interval for start [Persistener] timer. + defaultStartPersistnerInterval = 15 * time.Second ) var ( @@ -131,6 +134,8 @@ type Manager struct { } // NewManager init new [Manager]. +// +//revive:disable-next-line:function-length // this is contructor. func NewManager( o *Options, clock clockwork.Clock, @@ -185,11 +190,12 @@ func NewManager( rotatorMediator: mediator.NewMediator( mediator.NewRotateTimerWithSeed(clock, o.BlockDuration, o.Seed), ), - mergerMediator: mediator.NewMediator(mediator.NewConstantIntervalTimer(clock, DefaultMergeDuration)), + mergerMediator: mediator.NewMediator( + mediator.NewConstantIntervalTimer(clock, DefaultMergeDuration, DefaultMergeDuration), + ), } - readyNotifier.NotifyReady() - m.initServices(o, hcatalog, builder, loader, triggerNotifier, clock, r) + m.initServices(o, hcatalog, builder, loader, triggerNotifier, readyNotifier, clock, r) logger.Infof("[Head Manager] created") return m, nil @@ -236,12 +242,15 @@ func (m *Manager) Shutdown(ctx context.Context) error { } // initServices initializes services for startup. +// +//revive:disable-next-line:function-length // init contructor. func (m *Manager) initServices( o *Options, hcatalog *catalog.Catalog, builder *Builder, loader *Loader, triggerNotifier *ReloadBlocksTriggerNotifier, + readyNotifier ready.Notifier, clock clockwork.Clock, r prometheus.Registerer, ) { @@ -250,6 +259,7 @@ func (m *Manager) initServices( // Termination handler. m.g.Add( func() error { + readyNotifier.NotifyReady() <-m.closer.Signal() return nil @@ -260,7 +270,9 @@ func (m *Manager) initServices( ) // Persistener - persistenerMediator := mediator.NewMediator(mediator.NewConstantIntervalTimer(clock, DefaultPersistDuration)) + persistenerMediator := mediator.NewMediator( + mediator.NewConstantIntervalTimer(clock, defaultStartPersistnerInterval, DefaultPersistDuration), + ) m.g.Add( func() error { services.NewPersistenerService( @@ -318,7 +330,9 @@ func (m *Manager) initServices( } // Committer - committerMediator := mediator.NewMediator(mediator.NewConstantIntervalTimer(clock, o.CommitInterval)) + committerMediator := mediator.NewMediator( + mediator.NewConstantIntervalTimer(clock, o.CommitInterval, o.CommitInterval), + ) committerCtx, committerCancel := context.WithCancel(baseCtx) m.g.Add( func() error { @@ -343,7 +357,9 @@ func (m *Manager) initServices( ) // MetricsUpdater - metricsUpdaterMediator := mediator.NewMediator(mediator.NewConstantIntervalTimer(clock, DefaultMetricWriteInterval)) + metricsUpdaterMediator := mediator.NewMediator( + mediator.NewConstantIntervalTimer(clock, DefaultMetricWriteInterval, DefaultMetricWriteInterval), + ) metricsUpdaterCtx, metricsUpdaterCancel := context.WithCancel(baseCtx) m.g.Add( func() error { @@ -461,6 +477,7 @@ func uploadOrBuildHead( var generation uint64 if len(headRecords) == 0 { + logger.Debugf("[Head Manager] no suitable heads were found, building new") return builder.Build(generation, numberOfShards) } @@ -468,10 +485,10 @@ func uploadOrBuildHead( if corrupted { if !headRecords[0].Corrupted() { if _, setCorruptedErr := hcatalog.SetCorrupted(headRecords[0].ID()); setCorruptedErr != nil { - logger.Errorf("failed to set corrupted state, head id: %s: %v", headRecords[0].ID(), setCorruptedErr) + logger.Errorf("failed to set corrupted state, head {%s}: %v", headRecords[0].ID(), setCorruptedErr) } } - logger.Warnf("[Head Manager] upload corrupted head, building new: %s", headRecords[0].ID()) + logger.Warnf("[Head Manager] upload corrupted head {%s}, building new...", headRecords[0].ID()) if _, err := hcatalog.SetStatus(headRecords[0].ID(), catalog.StatusRotated); err != nil { logger.Warnf("failed to set rotated status for head {%s}: %s", headRecords[0].ID(), err) @@ -482,5 +499,11 @@ func uploadOrBuildHead( return builder.Build(generation, numberOfShards) } + if _, err := hcatalog.SetStatus(headRecords[0].ID(), catalog.StatusActive); err != nil { + logger.Warnf("failed to set active status for head {%s}: %s", headRecords[0].ID(), err) + + return builder.Build(generation, numberOfShards) + } + return h, nil } diff --git a/pp/go/storage/mediator/timer.go b/pp/go/storage/mediator/timer.go index f6dcd7e19a..1d7d7303b4 100644 --- a/pp/go/storage/mediator/timer.go +++ b/pp/go/storage/mediator/timer.go @@ -111,9 +111,9 @@ type ConstantIntervalTimer struct { } // NewConstantIntervalTimer init new [ConstantIntervalTimer]. -func NewConstantIntervalTimer(clock clockwork.Clock, interval time.Duration) *ConstantIntervalTimer { +func NewConstantIntervalTimer(clock clockwork.Clock, start, interval time.Duration) *ConstantIntervalTimer { return &ConstantIntervalTimer{ - timer: clock.NewTimer(interval), + timer: clock.NewTimer(start), interval: interval, } } diff --git a/pp/go/storage/remotewriter/datasource.go b/pp/go/storage/remotewriter/datasource.go index 4bdae97163..db9c04ea29 100644 --- a/pp/go/storage/remotewriter/datasource.go +++ b/pp/go/storage/remotewriter/datasource.go @@ -301,7 +301,7 @@ func newDataSource(dataDir string, go b.cacheWriteLoop() - for shardID := uint16(0); shardID < numberOfShards; shardID++ { + for shardID := range numberOfShards { shardFileName := filepath.Join(dataDir, fmt.Sprintf("shard_%d.wal", shardID)) decoderStateFileName := filepath.Join(dataDir, fmt.Sprintf("%s_shard_%d.state", config.Name, shardID)) var s *shard diff --git a/pp/go/storage/remotewriter/walreader.go b/pp/go/storage/remotewriter/walreader.go index 86acf5f748..d773bf2314 100644 --- a/pp/go/storage/remotewriter/walreader.go +++ b/pp/go/storage/remotewriter/walreader.go @@ -17,7 +17,11 @@ type walReader struct { } func newWalReader(fileName string) (*walReader, uint8, error) { - file, err := os.Open(fileName) + file, err := os.OpenFile( //nolint:gosec // need this permissions + fileName, + os.O_RDONLY, + 0o666, //revive:disable-line:add-constant // file permissions simple readable as octa-number + ) if err != nil { return nil, 0, fmt.Errorf("failed to read wal file: %w", err) } @@ -29,7 +33,7 @@ func newWalReader(fileName string) (*walReader, uint8, error) { return &walReader{ file: file, - reader: bufio.NewReaderSize(file, 4096), + reader: bufio.NewReaderSize(file, 4096), //revive:disable-line:add-constant // 4kb }, encoderVersion, nil } diff --git a/pp/go/storage/remotewriter/writeloop.go b/pp/go/storage/remotewriter/writeloop.go index dde7d95e83..d639d43463 100644 --- a/pp/go/storage/remotewriter/writeloop.go +++ b/pp/go/storage/remotewriter/writeloop.go @@ -274,7 +274,7 @@ func nextHead(ctx context.Context, dataDir string, headCatalog Catalog, headID s continue } - if err := validateHead(ctx, filepath.Join(dataDir, headRecord.Dir())); err != nil { + if err := validateHead(filepath.Join(dataDir, headRecord.Dir())); err != nil { if !errors.Is(err, os.ErrNotExist) { return nil, err } @@ -298,7 +298,7 @@ func nextHead(ctx context.Context, dataDir string, headCatalog Catalog, headID s return nil, fmt.Errorf("nextHead: no new heads: appropriate head not found") } -func validateHead(ctx context.Context, headDir string) error { +func validateHead(headDir string) error { dir, err := os.Open(headDir) if err != nil { return err diff --git a/pp/go/storage/remotewriter/writer.go b/pp/go/storage/remotewriter/writer.go index b5483d9e11..ef1b0d03c0 100644 --- a/pp/go/storage/remotewriter/writer.go +++ b/pp/go/storage/remotewriter/writer.go @@ -19,6 +19,11 @@ func newProtobufWriter(client remote.WriteClient) *protobufWriter { } } +// Close implementation [io.Closer]. +func (*protobufWriter) Close() error { + return nil +} + // Write [cppbridge.SnappyProtobufEncodedData] to [remote.WriteClient] func (w *protobufWriter) Write(ctx context.Context, protobuf *cppbridge.SnappyProtobufEncodedData) error { return protobuf.Do(func(buf []byte) error { @@ -31,8 +36,3 @@ func (w *protobufWriter) Write(ctx context.Context, protobuf *cppbridge.SnappyPr return err }) } - -// Close implementation [io.Closer]. -func (*protobufWriter) Close() error { - return nil -} diff --git a/web/web_fuzzy_test.go b/web/web_fuzzy_test.go index c102d874b6..6386869a78 100644 --- a/web/web_fuzzy_test.go +++ b/web/web_fuzzy_test.go @@ -14,13 +14,13 @@ import ( "testing" "time" + "github.com/go-kit/log" + "github.com/jonboulle/clockwork" + "github.com/prometheus/client_golang/prometheus" + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" + "github.com/prometheus/prometheus/model/labels" - "github.com/prometheus/prometheus/pp-pkg/config" - "github.com/prometheus/prometheus/pp-pkg/receiver" - "github.com/prometheus/prometheus/pp-pkg/scrape" - pp_pkg_storage "github.com/prometheus/prometheus/pp-pkg/storage" - relabeler_config "github.com/prometheus/prometheus/pp/go/relabeler/config" - "github.com/prometheus/prometheus/pp/go/relabeler/head/catalog" "github.com/prometheus/prometheus/promql" "github.com/prometheus/prometheus/rules" "github.com/prometheus/prometheus/storage" @@ -29,11 +29,11 @@ import ( apiV1 "github.com/prometheus/prometheus/web/api/v1" "github.com/prometheus/prometheus/web/mock" - "github.com/go-kit/log" - "github.com/jonboulle/clockwork" - "github.com/prometheus/client_golang/prometheus" - "github.com/stretchr/testify/assert" - "github.com/stretchr/testify/require" + "github.com/prometheus/prometheus/pp-pkg/scrape" + pp_pkg_storage "github.com/prometheus/prometheus/pp-pkg/storage" + + pp_storage "github.com/prometheus/prometheus/pp/go/storage" + "github.com/prometheus/prometheus/pp/go/storage/catalog" ) func FuzzWeb(f *testing.F) { @@ -101,15 +101,22 @@ func startService(ctx context.Context, t TestingT) net.Listener { logger := log.NewLogfmtLogger(os.Stderr) clock := clockwork.NewRealClock() headCatalog := makeCatalog(t, clock, dbDir) - receiver := makeReceiver(ctx, t, logger, dbDir, headCatalog) + hManager := makeManager(t, clock, dbDir, headCatalog) + adapter := pp_pkg_storage.NewAdapter( + clock, + hManager.Proxy(), + hManager.MergeOutOfOrderChunks, + prometheus.DefaultRegisterer, + ) + adminStats := makeTSDBAdminStats(t) listener, err := net.Listen("tcp", ":0") require.NoError(t, err, "Listen on random port") - fanoutStorage := storage.NewFanout(logger, pp_pkg_storage.NewQueryableStorage(receiver)) + fanoutStorage := storage.NewFanout(logger, adapter) opts := makeOptions(t, adminStats, fanoutStorage, dbDir, listener.Addr().String()) - webHandler := web.New(logger, opts, receiver) + webHandler := web.New(logger, opts, adapter) webHandler.SetReady(true) go func() { @@ -134,41 +141,35 @@ func makeCatalog(t TestingT, clock clockwork.Clock, dbDir string) *catalog.Catal return headCatalog } -func makeReceiver(ctx context.Context, t TestingT, logger log.Logger, dbDir string, headCatalog *catalog.Catalog) *receiver.Receiver { +func makeManager( + t TestingT, + clock clockwork.Clock, + dbDir string, + headCatalog *catalog.Catalog, +) *pp_storage.Manager { t.Helper() - transparent := &relabeler_config.InputRelabelerConfig{ - Name: "transparent_relabeler", - } - - unloadDataStorage := false - receiver, err := receiver.NewReceiver( - ctx, - log.With(logger, "component", "receiver"), - nil, - &config.RemoteWriteReceiverConfig{ - NumberOfShards: 2, - Configs: []*relabeler_config.InputRelabelerConfig{transparent}, - }, - dbDir, - nil, - dbDir, - receiver.RotationInfo{ - BlockDuration: 2 * time.Hour, - Seed: 0, + hManager, err := pp_storage.NewManager( + &pp_storage.Options{ + Seed: 0, + BlockDuration: 2 * time.Hour, + CommitInterval: 5 * time.Second, + MaxRetentionPeriod: 24 * time.Hour, + HeadRetentionPeriod: 4 * time.Hour, + QueueSize: 2, + DataDir: dbDir, + MaxSegmentSize: 100e3, + NumberOfShards: 2, }, + clock, headCatalog, - receiver.NewReloadBlocksTriggerNotifier(), + pp_storage.NewReloadBlocksTriggerNotifier(), &mock.ReadyNotifierMock{NotifyReadyFunc: func() {}}, - 5*time.Second, - 24*time.Hour, - 4*time.Hour, - 90*time.Second, - 100e3, - unloadDataStorage, + prometheus.DefaultRegisterer, ) - require.NoError(t, err, "create a receiver") - return receiver + require.NoError(t, err, "create a head manager") + + return hManager } func makeTSDBAdminStats(t TestingT) apiV1.TSDBAdminStats { From 02d56d74f4f44cb74b382d8bf45481136fb58d42 Mon Sep 17 00:00:00 2001 From: Alexandr Yudin Date: Fri, 26 Sep 2025 14:35:44 +0000 Subject: [PATCH 53/96] fix metrics --- pp/go/cppbridge/entrypoint.go | 110 +++++++++++++++++++++++++++++++--- 1 file changed, 102 insertions(+), 8 deletions(-) diff --git a/pp/go/cppbridge/entrypoint.go b/pp/go/cppbridge/entrypoint.go index e3f5888f28..0a2925cedc 100644 --- a/pp/go/cppbridge/entrypoint.go +++ b/pp/go/cppbridge/entrypoint.go @@ -98,6 +98,100 @@ var ( }, ) + // per_goroutine_relabeler input_relabeling + perGoroutineRelabelerInputRelabelingSum = util.NewUnconflictRegisterer(prometheus.DefaultRegisterer).NewCounter( + prometheus.CounterOpts{ + Name: "prompp_cppbridge_unsafecall_nanoseconds_sum", + Help: "The time duration cpp call.", + ConstLabels: prometheus.Labels{"object": "per_goroutine_relabeler", "method": "input_relabeling"}, + }, + ) + perGoroutineRelabelerInputRelabelingCount = util.NewUnconflictRegisterer(prometheus.DefaultRegisterer).NewCounter( + prometheus.CounterOpts{ + Name: "prompp_cppbridge_unsafecall_nanoseconds_count", + Help: "The time duration cpp call.", + ConstLabels: prometheus.Labels{"object": "per_goroutine_relabeler", "method": "input_relabeling"}, + }, + ) + + // per_goroutine_relabeler input_relabeling_from_cache + perGoroutineRelabelerInputRelabelingFromCacheSum = util.NewUnconflictRegisterer( + prometheus.DefaultRegisterer, + ).NewCounter( + prometheus.CounterOpts{ + Name: "prompp_cppbridge_unsafecall_nanoseconds_sum", + Help: "The time duration cpp call.", + ConstLabels: prometheus.Labels{ + "object": "per_goroutine_relabeler", + "method": "input_relabeling_from_cache", + }, + }, + ) + perGoroutineRelabelerInputRelabelingFromCacheCount = util.NewUnconflictRegisterer( + prometheus.DefaultRegisterer, + ).NewCounter( + prometheus.CounterOpts{ + Name: "prompp_cppbridge_unsafecall_nanoseconds_count", + Help: "The time duration cpp call.", + ConstLabels: prometheus.Labels{ + "object": "per_goroutine_relabeler", + "method": "input_relabeling_from_cache", + }, + }, + ) + + // per_goroutine_relabeler relabeling_with_stalenans + perGoroutineRelabelerInputRelabelingWithStalenansSum = util.NewUnconflictRegisterer( + prometheus.DefaultRegisterer, + ).NewCounter( + prometheus.CounterOpts{ + Name: "prompp_cppbridge_unsafecall_nanoseconds_sum", + Help: "The time duration cpp call.", + ConstLabels: prometheus.Labels{ + "object": "per_goroutine_relabeler", + "method": "relabeling_with_stalenans", + }, + }, + ) + perGoroutineRelabelerInputRelabelingWithStalenansCount = util.NewUnconflictRegisterer( + prometheus.DefaultRegisterer, + ).NewCounter( + prometheus.CounterOpts{ + Name: "prompp_cppbridge_unsafecall_nanoseconds_count", + Help: "The time duration cpp call.", + ConstLabels: prometheus.Labels{ + "object": "per_goroutine_relabeler", + "method": "relabeling_with_stalenans", + }, + }, + ) + + // per_goroutine_relabeler relabeling_with_stalenans_from_cache + perGoroutineRelabelerInputRelabelingWithStalenansFromCacheSum = util.NewUnconflictRegisterer( + prometheus.DefaultRegisterer, + ).NewCounter( + prometheus.CounterOpts{ + Name: "prompp_cppbridge_unsafecall_nanoseconds_sum", + Help: "The time duration cpp call.", + ConstLabels: prometheus.Labels{ + "object": "per_goroutine_relabeler", + "method": "relabeling_with_stalenans_from_cache", + }, + }, + ) + perGoroutineRelabelerInputRelabelingWithStalenansFromCacheCount = util.NewUnconflictRegisterer( + prometheus.DefaultRegisterer, + ).NewCounter( + prometheus.CounterOpts{ + Name: "prompp_cppbridge_unsafecall_nanoseconds_count", + Help: "The time duration cpp call.", + ConstLabels: prometheus.Labels{ + "object": "per_goroutine_relabeler", + "method": "relabeling_with_stalenans_from_cache", + }, + }, + ) + // per_goroutine_relabeler input_transition_relabeling perGoroutineRelabelerInputTransitionRelabelingSum = util.NewUnconflictRegisterer( prometheus.DefaultRegisterer, @@ -3327,8 +3421,8 @@ func prometheusPerGoroutineRelabelerInputRelabeling( uintptr(unsafe.Pointer(&args)), uintptr(unsafe.Pointer(&res)), ) - inputRelabelerInputRelabelingSum.Add(float64(time.Now().UnixNano() - start)) - inputRelabelerInputRelabelingCount.Inc() + perGoroutineRelabelerInputRelabelingSum.Add(float64(time.Now().UnixNano() - start)) + perGoroutineRelabelerInputRelabelingCount.Inc() return res.RelabelerStats, res.exception, res.targetLssHasReallocations } @@ -3361,8 +3455,8 @@ func prometheusPerGoroutineRelabelerInputRelabelingFromCache( uintptr(unsafe.Pointer(&args)), uintptr(unsafe.Pointer(&res)), ) - inputRelabelerInputRelabelingFromCacheSum.Add(float64(time.Now().UnixNano() - start)) - inputRelabelerInputRelabelingFromCacheCount.Inc() + perGoroutineRelabelerInputRelabelingFromCacheSum.Add(float64(time.Now().UnixNano() - start)) + perGoroutineRelabelerInputRelabelingFromCacheCount.Inc() return res.RelabelerStats, res.exception, res.ok } @@ -3413,8 +3507,8 @@ func prometheusPerGoroutineRelabelerInputRelabelingWithStalenans( uintptr(unsafe.Pointer(&args)), uintptr(unsafe.Pointer(&res)), ) - inputRelabelerRelabelingWithStalenansSum.Add(float64(time.Now().UnixNano() - start)) - inputRelabelerRelabelingWithStalenansCount.Inc() + perGoroutineRelabelerInputRelabelingWithStalenansSum.Add(float64(time.Now().UnixNano() - start)) + perGoroutineRelabelerInputRelabelingWithStalenansCount.Inc() return res.RelabelerStats, res.exception, res.targetLssHasReallocations } @@ -3460,8 +3554,8 @@ func prometheusPerGoroutineRelabelerInputRelabelingWithStalenansFromCache( uintptr(unsafe.Pointer(&args)), uintptr(unsafe.Pointer(&res)), ) - inputRelabelerRelabelingWithStalenansFromCacheSum.Add(float64(time.Now().UnixNano() - start)) - inputRelabelerRelabelingWithStalenansFromCacheCount.Inc() + perGoroutineRelabelerInputRelabelingWithStalenansFromCacheSum.Add(float64(time.Now().UnixNano() - start)) + perGoroutineRelabelerInputRelabelingWithStalenansFromCacheCount.Inc() return res.RelabelerStats, res.exception, res.ok } From cca12a7eb078fe1be6b0d9783fd3d383139d193a Mon Sep 17 00:00:00 2001 From: Alexandr Yudin Date: Fri, 26 Sep 2025 15:25:44 +0000 Subject: [PATCH 54/96] fix state --- pp/go/cppbridge/entrypoint.go | 24 ++++++++++++++++++++++-- pp/go/cppbridge/prometheus_relabeler.go | 2 ++ 2 files changed, 24 insertions(+), 2 deletions(-) diff --git a/pp/go/cppbridge/entrypoint.go b/pp/go/cppbridge/entrypoint.go index 0a2925cedc..e69c0217ad 100644 --- a/pp/go/cppbridge/entrypoint.go +++ b/pp/go/cppbridge/entrypoint.go @@ -244,6 +244,26 @@ var ( }, ) + // per_goroutine_relabeler append_relabeler_series + perGoroutineRelabelerAppendRelabelerSeriesSum = util.NewUnconflictRegisterer( + prometheus.DefaultRegisterer, + ).NewCounter( + prometheus.CounterOpts{ + Name: "prompp_cppbridge_unsafecall_nanoseconds_sum", + Help: "The time duration cpp call.", + ConstLabels: prometheus.Labels{"object": "per_goroutine_relabeler", "method": "append_relabeler_series"}, + }, + ) + perGoroutineRelabelerAppendRelabelerSeriesCount = util.NewUnconflictRegisterer( + prometheus.DefaultRegisterer, + ).NewCounter( + prometheus.CounterOpts{ + Name: "prompp_cppbridge_unsafecall_nanoseconds_count", + Help: "The time duration cpp call.", + ConstLabels: prometheus.Labels{"object": "per_goroutine_relabeler", "method": "append_relabeler_series"}, + }, + ) + // input_relabeler append_relabeler_series inputRelabelerAppendRelabelerSeriesSum = util.NewUnconflictRegisterer(prometheus.DefaultRegisterer).NewCounter( prometheus.CounterOpts{ @@ -3651,8 +3671,8 @@ func prometheusPerGoroutineRelabelerAppendRelabelerSeries( uintptr(unsafe.Pointer(&args)), uintptr(unsafe.Pointer(&res)), ) - inputRelabelerAppendRelabelerSeriesSum.Add(float64(time.Now().UnixNano() - start)) - inputRelabelerAppendRelabelerSeriesCount.Inc() + perGoroutineRelabelerAppendRelabelerSeriesSum.Add(float64(time.Now().UnixNano() - start)) + perGoroutineRelabelerAppendRelabelerSeriesCount.Inc() return res.exception, res.targetLssHasReallocations } diff --git a/pp/go/cppbridge/prometheus_relabeler.go b/pp/go/cppbridge/prometheus_relabeler.go index 4b5e5fb551..da4ae09eaa 100644 --- a/pp/go/cppbridge/prometheus_relabeler.go +++ b/pp/go/cppbridge/prometheus_relabeler.go @@ -1620,6 +1620,8 @@ func (s *StateV2) Reconfigure( s.resetCaches(numberOfShards) s.resetStaleNansStates(numberOfShards) + s.status |= inited + s.generationHead = generationHead s.locker.Unlock() } From 55a2b5f4428006d225c0df516a45fa64daf17a9b Mon Sep 17 00:00:00 2001 From: Alexandr Yudin Date: Mon, 29 Sep 2025 08:33:40 +0000 Subject: [PATCH 55/96] smal fix --- cmd/prometheus/main.go | 2 +- pp/go/cppbridge/c_garbage_collector.go | 3 +- pp/go/cppbridge/prometheus_relabeler_test.go | 15 ++ pp/go/storage/block/block.go | 75 ++++--- pp/go/storage/block/block_writer.go | 23 +- pp/go/storage/block/chunk_writer.go | 198 ++++++++++-------- pp/go/storage/block/file_writer.go | 20 +- pp/go/storage/block/writer.go | 52 +++-- pp/go/storage/block/writer_test.go | 2 +- pp/go/storage/builder.go | 2 + pp/go/storage/catalog/log_file_handler.go | 2 +- pp/go/storage/head/head/head_test.go | 3 +- pp/go/storage/head/shard/shard.go | 13 +- .../head/shard/unloaded_data_storage.go | 25 +-- pp/go/storage/head/shard/wal/wal_test.go | 3 +- 15 files changed, 260 insertions(+), 178 deletions(-) diff --git a/cmd/prometheus/main.go b/cmd/prometheus/main.go index 14a9411dfd..fdc5ee6a34 100644 --- a/cmd/prometheus/main.go +++ b/cmd/prometheus/main.go @@ -417,7 +417,7 @@ func main() { Default("100000").Uint32Var(&cfg.WalMaxSamplesPerSegment) serverOnlyFlag(a, "storage.head-retention-timeout", "Timeout before inactive heads are shrieked."). - Default("5m").SetValue(&cfg.HeadRetentionTimeout) + Default("3m").SetValue(&cfg.HeadRetentionTimeout) // TODO: Remove in Prometheus 3.0. var b bool diff --git a/pp/go/cppbridge/c_garbage_collector.go b/pp/go/cppbridge/c_garbage_collector.go index bd92f09afe..323a2e78e1 100644 --- a/pp/go/cppbridge/c_garbage_collector.go +++ b/pp/go/cppbridge/c_garbage_collector.go @@ -5,8 +5,9 @@ import ( "runtime" "time" - "github.com/prometheus/prometheus/pp/go/util" "github.com/prometheus/client_golang/prometheus" + + "github.com/prometheus/prometheus/pp/go/util" ) // garbage collector for objects initiated in GO but filled in C/C++, diff --git a/pp/go/cppbridge/prometheus_relabeler_test.go b/pp/go/cppbridge/prometheus_relabeler_test.go index 98eb537b1c..b268074bd2 100644 --- a/pp/go/cppbridge/prometheus_relabeler_test.go +++ b/pp/go/cppbridge/prometheus_relabeler_test.go @@ -1452,3 +1452,18 @@ func (s *RelabelerSuite) TestPerGoroutineRelabelerFromCachePartiallyTransition() s.Equal(cppbridge.RelabelerStats{1, 1, 0}, stats) s.Equal(uint64(5), shardsInnerSeries[0].Size()) } + +// +// StateV2Suite +// + +type StateV2Suite struct { + suite.Suite +} + +func TestStateV2Suite(t *testing.T) { + suite.Run(t, new(StateV2Suite)) +} + +func (s *StateV2Suite) TestHappyPath() { +} diff --git a/pp/go/storage/block/block.go b/pp/go/storage/block/block.go index 6ba726d949..3003692fd1 100644 --- a/pp/go/storage/block/block.go +++ b/pp/go/storage/block/block.go @@ -26,7 +26,7 @@ func (c *Chunk) SeriesID() uint32 { return c.rc.SeriesId } -func (c *Chunk) Encoding() chunkenc.Encoding { +func (*Chunk) Encoding() chunkenc.Encoding { return chunkenc.EncXOR } @@ -43,8 +43,16 @@ type ChunkIterator struct { rc *cppbridge.RecodedChunk } -func NewChunkIterator(lss *cppbridge.LabelSetStorage, lsIdBatchSize uint32, ds *cppbridge.HeadDataStorage, minT, maxT int64) ChunkIterator { - return ChunkIterator{r: cppbridge.NewChunkRecoder(lss, lsIdBatchSize, ds, cppbridge.TimeInterval{MinT: minT, MaxT: maxT})} +// NewChunkIterator init new [ChunkIterator]. +func NewChunkIterator( + lss *cppbridge.LabelSetStorage, + lsIdBatchSize uint32, + ds *cppbridge.HeadDataStorage, + minT, maxT int64, +) ChunkIterator { + return ChunkIterator{ + r: cppbridge.NewChunkRecoder(lss, lsIdBatchSize, ds, cppbridge.TimeInterval{MinT: minT, MaxT: maxT}), + } } func (i *ChunkIterator) Next() bool { @@ -66,35 +74,18 @@ func (i *ChunkIterator) At() Chunk { return Chunk{rc: i.rc} } +// +// IndexWriter +// + type IndexWriter struct { cppIndexWriter *cppbridge.IndexWriter isPrefixWritten bool } -func (iw *IndexWriter) WriteSeriesTo(id uint32, chunks []ChunkMetadata, w io.Writer) (n int64, err error) { - if !iw.isPrefixWritten { - var bytesWritten int - bytesWritten, err = w.Write(iw.cppIndexWriter.WriteHeader()) - n += int64(bytesWritten) - if err != nil { - return n, fmt.Errorf("failed to write header: %w", err) - } - - bytesWritten, err = w.Write(iw.cppIndexWriter.WriteSymbols()) - n += int64(bytesWritten) - if err != nil { - return n, fmt.Errorf("failed to write symbols: %w", err) - } - iw.isPrefixWritten = true - } - - bytesWritten, err := w.Write(iw.cppIndexWriter.WriteSeries(id, *(*[]cppbridge.ChunkMetadata)(unsafe.Pointer(&chunks)))) - n += int64(bytesWritten) - if err != nil { - return n, fmt.Errorf("failed to write series: %w", err) - } - - return n, nil +// NewIndexWriter init new [IndexWriter]. +func NewIndexWriter(lss *cppbridge.LabelSetStorage) IndexWriter { + return IndexWriter{cppIndexWriter: cppbridge.NewIndexWriter(lss)} } func (iw *IndexWriter) WriteRestTo(w io.Writer) (n int64, err error) { @@ -137,6 +128,32 @@ func (iw *IndexWriter) WriteRestTo(w io.Writer) (n int64, err error) { return n, nil } -func NewIndexWriter(lss *cppbridge.LabelSetStorage) IndexWriter { - return IndexWriter{cppIndexWriter: cppbridge.NewIndexWriter(lss)} +// WriteSeriesTo write series(id and chunks) to [io.Writer]. +func (iw *IndexWriter) WriteSeriesTo(id uint32, chunks []ChunkMetadata, w io.Writer) (n int64, err error) { + if !iw.isPrefixWritten { + var bytesWritten int + bytesWritten, err = w.Write(iw.cppIndexWriter.WriteHeader()) + n += int64(bytesWritten) + if err != nil { + return n, fmt.Errorf("failed to write header: %w", err) + } + + bytesWritten, err = w.Write(iw.cppIndexWriter.WriteSymbols()) + n += int64(bytesWritten) + if err != nil { + return n, fmt.Errorf("failed to write symbols: %w", err) + } + iw.isPrefixWritten = true + } + + bytesWritten, err := w.Write(iw.cppIndexWriter.WriteSeries( + id, + *(*[]cppbridge.ChunkMetadata)(unsafe.Pointer(&chunks)), // #nosec G103 // it's meant to be that way + )) + n += int64(bytesWritten) + if err != nil { + return n, fmt.Errorf("failed to write series: %w", err) + } + + return n, nil } diff --git a/pp/go/storage/block/block_writer.go b/pp/go/storage/block/block_writer.go index 0090ba4863..73bd9513aa 100644 --- a/pp/go/storage/block/block_writer.go +++ b/pp/go/storage/block/block_writer.go @@ -53,16 +53,21 @@ type blockWriter struct { chunkRecoder chunkRecoder } -func newBlockWriter(dir string, maxBlockChunkSegmentSize int64, indexWriter IndexWriter, chunkIterator ChunkIterator) (writer blockWriter, err error) { +func newBlockWriter( + dir string, + maxBlockChunkSegmentSize int64, + indexWriter IndexWriter, + chunkIterator ChunkIterator, +) (writer blockWriter, err error) { uid := ulid.MustNew(ulid.Now(), rand.Reader) writer.Dir = filepath.Join(dir, uid.String()) + tmpForCreationBlockDirSuffix if err = createTmpDir(writer.Dir); err != nil { - return + return writer, err } if err = writer.createWriters(maxBlockChunkSegmentSize); err != nil { - return + return writer, err } writer.Meta = tsdb.BlockMeta{ @@ -78,7 +83,8 @@ func newBlockWriter(dir string, maxBlockChunkSegmentSize int64, indexWriter Inde writer.indexWriter = indexWriter writer.chunkRecoder = newChunkRecoder(chunkIterator) - return + + return writer, err } func (writer *blockWriter) createWriters(maxBlockChunkSegmentSize int64) error { @@ -125,7 +131,7 @@ func (writer *blockWriter) writeIndex() error { return fmt.Errorf("failed to write index: %w", err) } - writer.Meta.MaxTime += 1 + writer.Meta.MaxTime++ if _, err := writeBlockMetaFile(writer.MetaFilename(), &writer.Meta); err != nil { return fmt.Errorf("failed to write block meta file: %w", err) } @@ -262,7 +268,7 @@ func writeBlockMetaFile(fileName string, blockMeta *tsdb.BlockMeta) (int64, erro } }() - metaFile, err := os.Create(tmp) + metaFile, err := os.Create(tmp) // #nosec G304 // it's meant to be that way if err != nil { return 0, fmt.Errorf("failed to create block meta file: %w", err) } @@ -301,7 +307,10 @@ func createTmpDir(dir string) error { return err } - return os.MkdirAll(dir, 0o777) + return os.MkdirAll( //nolint:gosec // need this permissions + dir, + 0o777, //revive:disable-line:add-constant // file permissions simple readable as octa-number + ) } func syncDir(dir string) error { diff --git a/pp/go/storage/block/chunk_writer.go b/pp/go/storage/block/chunk_writer.go index ed3bb0f509..d1a531c884 100644 --- a/pp/go/storage/block/chunk_writer.go +++ b/pp/go/storage/block/chunk_writer.go @@ -19,12 +19,14 @@ const ( chunksFormatV1 = 1 ) +// ChunkMetadata meta information for the chunk. type ChunkMetadata struct { MinT int64 MaxT int64 Ref uint64 } +// ChunkWriter a writer for encoding and writing chunks. type ChunkWriter struct { dirFile *os.File files []*os.File @@ -35,12 +37,16 @@ type ChunkWriter struct { buf [binary.MaxVarintLen32]byte } +// NewChunkWriter init new [ChunkWriter]. func NewChunkWriter(dir string, segmentSize int64) (*ChunkWriter, error) { if segmentSize < 0 { segmentSize = DefaultChunkSegmentSize } - if err := os.MkdirAll(dir, 0o777); err != nil { + if err := os.MkdirAll( //nolint:gosec // need this permissions + dir, + 0o777, //revive:disable-line:add-constant // file permissions simple readable as octa-number + ); err != nil { return nil, fmt.Errorf("failed to create all dirs: %w", err) } @@ -56,6 +62,16 @@ func NewChunkWriter(dir string, segmentSize int64) (*ChunkWriter, error) { }, nil } +// Close writes all pending data to the current tail file and closes chunk's files. +func (w *ChunkWriter) Close() (err error) { + if err = w.finalizeTail(); err != nil { + return fmt.Errorf("failed to finalize tail on close: %w", err) + } + + return w.dirFile.Close() +} + +// Write encoding and write to buffer chunk. func (w *ChunkWriter) Write(chunk Chunk) (meta ChunkMetadata, err error) { // calculate chunk size chunkSize := int64(chunks.MaxChunkLengthFieldSize) @@ -74,67 +90,26 @@ func (w *ChunkWriter) Write(chunk Chunk) (meta ChunkMetadata, err error) { return w.writeChunk(chunk) } -func (w *ChunkWriter) writeChunk(chunk Chunk) (meta ChunkMetadata, err error) { - meta.Ref = uint64(chunks.NewBlockChunkRef(uint64(w.seq()), uint64(w.n))) - - n := binary.PutUvarint(w.buf[:], uint64(len(chunk.Bytes()))) - if err = w.write(w.buf[:n]); err != nil { - return meta, err - } - - w.buf[0] = byte(chunk.Encoding()) - if err = w.write(w.buf[:1]); err != nil { - return meta, err - } - - if err = w.write(chunk.Bytes()); err != nil { - return meta, err - } - - w.crc32.Reset() - - buf := append(w.buf[:0], byte(chunk.Encoding())) - if _, err = w.crc32.Write(buf[:1]); err != nil { - return meta, err - } - - if _, err = w.crc32.Write(chunk.Bytes()); err != nil { - return meta, err +func (w *ChunkWriter) cut() error { + // Sync current tail to disk and close. + if err := w.finalizeTail(); err != nil { + return err } - if err = w.write(w.crc32.Sum(w.buf[:0])); err != nil { - return meta, err + f, n, err := cutSegmentFile(w.dirFile, w.seq(), chunks.MagicChunks, chunksFormatV1, w.segmentSize) + if err != nil { + return err } + w.n = int64(n) - meta.MinT = chunk.MinT() - meta.MaxT = chunk.MaxT() - - return meta, nil -} - -func (w *ChunkWriter) Close() (err error) { - if err = w.finalizeTail(); err != nil { - return fmt.Errorf("failed to finalize tail on close: %w", err) + w.files = append(w.files, f) + if w.wbuf != nil { + w.wbuf.Reset(f) + } else { + w.wbuf = bufio.NewWriterSize(f, 8*1024*1024) } - return w.dirFile.Close() -} - -func (w *ChunkWriter) write(b []byte) error { - n, err := w.wbuf.Write(b) - w.n += int64(n) - return err -} - -func (w *ChunkWriter) seq() int { - return len(w.files) - 1 -} - -func (w *ChunkWriter) tail() *os.File { - if len(w.files) == 0 { - return nil - } - return w.files[len(w.files)-1] + return nil } // finalizeTail writes all pending data to the current tail file, @@ -164,37 +139,78 @@ func (w *ChunkWriter) finalizeTail() error { return tf.Close() } -func (w *ChunkWriter) cut() error { - // Sync current tail to disk and close. - if err := w.finalizeTail(); err != nil { - return err +func (w *ChunkWriter) seq() int { + return len(w.files) - 1 +} + +func (w *ChunkWriter) tail() *os.File { + if len(w.files) == 0 { + return nil } + return w.files[len(w.files)-1] +} - n, f, _, err := cutSegmentFile(w.dirFile, w.seq(), chunks.MagicChunks, chunksFormatV1, w.segmentSize) - if err != nil { - return err +func (w *ChunkWriter) writeChunk(chunk Chunk) (meta ChunkMetadata, err error) { + meta.Ref = uint64(chunks.NewBlockChunkRef(uint64(w.seq()), uint64(w.n))) // #nosec G115 // no overflow + + n := binary.PutUvarint(w.buf[:], uint64(len(chunk.Bytes()))) + if err = w.writeToBuf(w.buf[:n]); err != nil { + return meta, err } - w.n = int64(n) - w.files = append(w.files, f) - if w.wbuf != nil { - w.wbuf.Reset(f) - } else { - w.wbuf = bufio.NewWriterSize(f, 8*1024*1024) + w.buf[0] = byte(chunk.Encoding()) + if err = w.writeToBuf(w.buf[:1]); err != nil { + return meta, err } - return nil + if err = w.writeToBuf(chunk.Bytes()); err != nil { + return meta, err + } + + w.crc32.Reset() + + buf := append(w.buf[:0], byte(chunk.Encoding())) + if _, err = w.crc32.Write(buf[:1]); err != nil { + return meta, err + } + + if _, err = w.crc32.Write(chunk.Bytes()); err != nil { + return meta, err + } + + if err = w.writeToBuf(w.crc32.Sum(w.buf[:0])); err != nil { + return meta, err + } + + meta.MinT = chunk.MinT() + meta.MaxT = chunk.MaxT() + + return meta, nil +} + +func (w *ChunkWriter) writeToBuf(b []byte) error { + n, err := w.wbuf.Write(b) + w.n += int64(n) + return err } -func cutSegmentFile(dirFile *os.File, currentSeq int, magicNumber uint32, chunksFormat byte, allocSize int64) (headerSize int, newFile *os.File, seq int, returnErr error) { - p, seq, err := nextSequenceFile(dirFile.Name(), currentSeq) +//revive:disable-next-line:function-length // long but readable. +//revive:disable-next-line:cyclomatic // but readable +func cutSegmentFile( + dirFile *os.File, + currentSeq int, + magicNumber uint32, + chunksFormat byte, + allocSize int64, +) (newFile *os.File, headerSize int, returnErr error) { + p, err := nextSequenceFile(dirFile.Name(), currentSeq) if err != nil { - return 0, nil, 0, fmt.Errorf("next sequence file: %w", err) + return nil, 0, fmt.Errorf("next sequence file: %w", err) } ptmp := p + ".tmp" - f, err := os.Create(ptmp) + f, err := os.Create(ptmp) // #nosec G304 // it's meant to be that way if err != nil { - return 0, nil, 0, fmt.Errorf("open temp file: %w", err) + return nil, 0, fmt.Errorf("open temp file: %w", err) } defer func() { if returnErr != nil { @@ -207,45 +223,49 @@ func cutSegmentFile(dirFile *os.File, currentSeq int, magicNumber uint32, chunks }() if allocSize > 0 { if err = fileutil.Preallocate(f, allocSize, true); err != nil { - return 0, nil, 0, fmt.Errorf("preallocate: %w", err) + return nil, 0, fmt.Errorf("preallocate: %w", err) } } if err = dirFile.Sync(); err != nil { - return 0, nil, 0, fmt.Errorf("sync directory: %w", err) + return nil, 0, fmt.Errorf("sync directory: %w", err) } // Write header metadata for new file. metab := make([]byte, chunks.SegmentHeaderSize) binary.BigEndian.PutUint32(metab[:chunks.MagicChunksSize], magicNumber) - metab[4] = chunksFormat + metab[4] = chunksFormat //revive:disable-line:add-constant // 4 byte for chunksFormat n, err := f.Write(metab) if err != nil { - return 0, nil, 0, fmt.Errorf("write header: %w", err) + return nil, 0, fmt.Errorf("write header: %w", err) } - if err := f.Close(); err != nil { - return 0, nil, 0, fmt.Errorf("close temp file: %w", err) + if err = f.Close(); err != nil { + return nil, 0, fmt.Errorf("close temp file: %w", err) } f = nil - if err := fileutil.Rename(ptmp, p); err != nil { - return 0, nil, 0, fmt.Errorf("replace file: %w", err) + if err = fileutil.Rename(ptmp, p); err != nil { + return nil, 0, fmt.Errorf("replace file: %w", err) } - f, err = os.OpenFile(p, os.O_WRONLY, 0o666) + f, err = os.OpenFile( //nolint:gosec // need this permissions + p, + os.O_WRONLY, + 0o666, //revive:disable-line:add-constant // file permissions simple readable as octa-number + ) if err != nil { - return 0, nil, 0, fmt.Errorf("open final file: %w", err) + return nil, 0, fmt.Errorf("open final file: %w", err) } // Skip header for further writes. if _, err := f.Seek(int64(n), 0); err != nil { - return 0, nil, 0, fmt.Errorf("seek in final file: %w", err) + return nil, 0, fmt.Errorf("seek in final file: %w", err) } - return n, f, seq, nil + return f, n, nil } -func nextSequenceFile(dir string, currentSeq int) (string, int, error) { - return segmentFile(dir, currentSeq+1), currentSeq + 1, nil +func nextSequenceFile(dir string, currentSeq int) (string, error) { + return segmentFile(dir, currentSeq+1), nil } func segmentFile(baseDir string, index int) string { diff --git a/pp/go/storage/block/file_writer.go b/pp/go/storage/block/file_writer.go index 1b93d572f4..e658eec23f 100644 --- a/pp/go/storage/block/file_writer.go +++ b/pp/go/storage/block/file_writer.go @@ -9,11 +9,13 @@ import ( "github.com/prometheus/prometheus/tsdb/fileutil" ) +// FileWriter a buffered file writer. type FileWriter struct { file *os.File writeBuffer *bufio.Writer } +// NewFileWriter init new [FileWriter]. func NewFileWriter(fileName string) (*FileWriter, error) { dir := filepath.Dir(fileName) df, err := fileutil.OpenDir(dir) @@ -22,11 +24,15 @@ func NewFileWriter(fileName string) (*FileWriter, error) { } defer func() { _ = df.Close() }() - if err := os.RemoveAll(fileName); err != nil { + if err = os.RemoveAll(fileName); err != nil { return nil, fmt.Errorf("failed to cleanup {%s}: %w", fileName, err) } - indexFile, err := os.OpenFile(fileName, os.O_CREATE|os.O_RDWR, 0o666) + indexFile, err := os.OpenFile( //nolint:gosec // need this permissions + fileName, + os.O_CREATE|os.O_RDWR, + 0o666, //revive:disable-line:add-constant // file permissions simple readable as octa-number + ) if err != nil { return nil, fmt.Errorf(" failed to open file {%s}: %w", fileName, err) } @@ -37,10 +43,7 @@ func NewFileWriter(fileName string) (*FileWriter, error) { }, nil } -func (w *FileWriter) Write(p []byte) (n int, err error) { - return w.writeBuffer.Write(p) -} - +// Close flush buffer to file and sync and closes file. func (w *FileWriter) Close() error { if err := w.writeBuffer.Flush(); err != nil { return fmt.Errorf("failed to flush write buffer: %w", err) @@ -52,3 +55,8 @@ func (w *FileWriter) Close() error { return w.file.Close() } + +// Write writes the contents of p into the buffer. +func (w *FileWriter) Write(p []byte) (n int, err error) { + return w.writeBuffer.Write(p) +} diff --git a/pp/go/storage/block/writer.go b/pp/go/storage/block/writer.go index 3bceb88478..923cea5339 100644 --- a/pp/go/storage/block/writer.go +++ b/pp/go/storage/block/writer.go @@ -52,10 +52,10 @@ func NewWriter[TShard Shard]( } } -func (w *Writer[TShard]) Write(shard TShard) (writtenBlocks []WrittenBlock, err error) { - _ = shard.LSS().WithRLock(func(_, _ *cppbridge.LabelSetStorage) error { +func (w *Writer[TShard]) Write(sd TShard) (writtenBlocks []WrittenBlock, err error) { + _ = sd.LSS().WithRLock(func(_, _ *cppbridge.LabelSetStorage) error { var writers blockWriters - writers, err = w.createWriters(shard) + writers, err = w.createWriters(sd) if err != nil { return err } @@ -64,20 +64,22 @@ func (w *Writer[TShard]) Write(shard TShard) (writtenBlocks []WrittenBlock, err writers.close() }() - if err = w.recodeAndWriteChunks(shard, writers); err != nil { + if err = w.recodeAndWriteChunks(sd, writers); err != nil { return err } writtenBlocks, err = writers.writeIndexAndMoveTmpDirToDir() + return nil }) - return + + return writtenBlocks, err } -func (w *Writer[TShard]) createWriters(shard TShard) (blockWriters, error) { +func (w *Writer[TShard]) createWriters(sd TShard) (blockWriters, error) { var writers blockWriters - timeInterval := shard.DataStorage().TimeInterval(false) + timeInterval := sd.DataStorage().TimeInterval(false) quantStart := (timeInterval.MinT / w.blockDurationMs) * w.blockDurationMs for ; quantStart <= timeInterval.MaxT; quantStart += w.blockDurationMs { @@ -90,26 +92,32 @@ func (w *Writer[TShard]) createWriters(shard TShard) (blockWriters, error) { } var chunkIterator ChunkIterator - _ = shard.DataStorage().WithRLock(func(ds *cppbridge.HeadDataStorage) error { - chunkIterator = NewChunkIterator(shard.LSS().Target(), LsIdBatchSize, shard.DataStorage().Raw(), minT, maxT) + _ = sd.DataStorage().WithRLock(func(*cppbridge.HeadDataStorage) error { + chunkIterator = NewChunkIterator(sd.LSS().Target(), LsIdBatchSize, sd.DataStorage().Raw(), minT, maxT) return nil }) - if writer, err := newBlockWriter(w.dataDir, w.maxBlockChunkSegmentSize, NewIndexWriter(shard.LSS().Target()), chunkIterator); err == nil { - writers.append(writer) - } else { + writer, err := newBlockWriter( + w.dataDir, + w.maxBlockChunkSegmentSize, + NewIndexWriter(sd.LSS().Target()), + chunkIterator, + ) + if err != nil { writers.close() return blockWriters{}, err } + + writers.append(writer) } return writers, nil } -func (w *Writer[TShard]) recodeAndWriteChunks(shard TShard, writers blockWriters) error { +func (*Writer[TShard]) recodeAndWriteChunks(sd TShard, writers blockWriters) error { var loader *cppbridge.UnloadedDataRevertableLoader - _ = shard.DataStorage().WithRLock(func(ds *cppbridge.HeadDataStorage) error { - loader = shard.DataStorage().CreateRevertableLoader(shard.LSS().Target(), LsIdBatchSize) + _ = sd.DataStorage().WithRLock(func(*cppbridge.HeadDataStorage) error { + loader = sd.DataStorage().CreateRevertableLoader(sd.LSS().Target(), LsIdBatchSize) return nil }) @@ -118,23 +126,21 @@ func (w *Writer[TShard]) recodeAndWriteChunks(shard TShard, writers blockWriters loadData := func() (bool, error) { if isFirstBatch { isFirstBatch = false - } else { - if !loader.NextBatch() { - return false, nil - } + } else if !loader.NextBatch() { + return false, nil } - if shard.UnloadedDataStorage() == nil { + if sd.UnloadedDataStorage() == nil { return true, nil } - return true, shard.UnloadedDataStorage().ForEachSnapshot(loader.Load) + return true, sd.UnloadedDataStorage().ForEachSnapshot(loader.Load) } for { var hasMoreData bool var err error - _ = shard.DataStorage().WithLock(func(ds *cppbridge.HeadDataStorage) error { + _ = sd.DataStorage().WithLock(func(*cppbridge.HeadDataStorage) error { hasMoreData, err = loadData() return nil }) @@ -147,7 +153,7 @@ func (w *Writer[TShard]) recodeAndWriteChunks(shard TShard, writers blockWriters return err } - if err = shard.DataStorage().WithRLock(func(ds *cppbridge.HeadDataStorage) error { + if err = sd.DataStorage().WithRLock(func(*cppbridge.HeadDataStorage) error { return writers.recodeAndWriteChunksBatch() }); err != nil { return err diff --git a/pp/go/storage/block/writer_test.go b/pp/go/storage/block/writer_test.go index 572d195b4d..cff4da6c95 100644 --- a/pp/go/storage/block/writer_test.go +++ b/pp/go/storage/block/writer_test.go @@ -130,7 +130,7 @@ func (s *WriterSuite) fillHead() { func (s *WriterSuite) assertWrittenBlocks(blocks []block.WrittenBlock, err error) { s.Require().NoError(err) - s.Require().Equal(2, len(blocks)) + s.Require().Len(blocks, 2) meta1 := s.mustReadBlockMeta(blocks[0].MetaFilename()) s.Equal(tsdb.BlockMeta{ diff --git a/pp/go/storage/builder.go b/pp/go/storage/builder.go index c9131e488c..8573062aa0 100644 --- a/pp/go/storage/builder.go +++ b/pp/go/storage/builder.go @@ -100,6 +100,8 @@ func (b *Builder) Build(generation uint64, numberOfShards uint16) (*HeadOnDisk, } // createShardOnDisk create [shard.Shard] with [wal.Wal] which is written to disk. +// +//revive:disable-next-line:function-length // long but readable. func (b *Builder) createShardOnDisk( headDir string, swn *writer.SegmentWriteNotifier, diff --git a/pp/go/storage/catalog/log_file_handler.go b/pp/go/storage/catalog/log_file_handler.go index f487fd0dcb..d111fb5130 100644 --- a/pp/go/storage/catalog/log_file_handler.go +++ b/pp/go/storage/catalog/log_file_handler.go @@ -26,7 +26,7 @@ func NewFileHandler(filePath string) (*FileHandler, error) { // NewFileHandlerWithOpts init new [FileHandler] with opts. func NewFileHandlerWithOpts(filePath string, flag int, perm os.FileMode) (*FileHandler, error) { - file, err := os.OpenFile(filePath, flag, perm) // #nosec G304 // it's meant to be that way + file, err := os.OpenFile(filePath, flag, perm) //#nosec G304 // it's meant to be that way if err != nil { return nil, fmt.Errorf("open file: %w", err) } diff --git a/pp/go/storage/head/head/head_test.go b/pp/go/storage/head/head/head_test.go index 994808d147..e09ea4ded7 100644 --- a/pp/go/storage/head/head/head_test.go +++ b/pp/go/storage/head/head/head_test.go @@ -24,7 +24,8 @@ func TestXxx(t *testing.T) { generation, nil, ) - _ = h + + t.Log(h) } // testWal test implementation wal. diff --git a/pp/go/storage/head/shard/shard.go b/pp/go/storage/head/shard/shard.go index 1426846d96..2e660b5d1a 100644 --- a/pp/go/storage/head/shard/shard.go +++ b/pp/go/storage/head/shard/shard.go @@ -177,7 +177,7 @@ func (s *Shard[TWal]) UnloadUnusedSeriesData() error { unloader := s.DataStorage().CreateUnusedSeriesDataUnloader() var snapshot, queriedSeries []byte - _ = s.DataStorage().WithRLock(func(ds *cppbridge.HeadDataStorage) error { + _ = s.DataStorage().WithRLock(func(*cppbridge.HeadDataStorage) error { snapshot = unloader.CreateSnapshot() queriedSeries = s.DataStorage().GetQueriedSeriesBitset() return nil @@ -188,7 +188,7 @@ func (s *Shard[TWal]) UnloadUnusedSeriesData() error { return fmt.Errorf("unable to write unloaded series data snapshot: %v", err) } - _ = s.DataStorage().WithLock(func(ds *cppbridge.HeadDataStorage) error { + _ = s.DataStorage().WithLock(func(*cppbridge.HeadDataStorage) error { s.UnloadedDataStorage().WriteIndex(header) unloader.Unload() return nil @@ -201,22 +201,23 @@ func (s *Shard[TWal]) UnloadUnusedSeriesData() error { return nil } +// LoadAndQuerySeriesData loads the data and queries the series from the [DataStorage]. func (s *Shard[TWal]) LoadAndQuerySeriesData() (err error) { var queriers []uintptr s.loadAndQueryTask.Release(func(q []uintptr) { queriers = q - err = s.DataStorage().WithLock(func(ds *cppbridge.HeadDataStorage) error { + err = s.DataStorage().WithLock(func(*cppbridge.HeadDataStorage) error { loader := s.DataStorage().CreateLoader(queriers) return s.UnloadedDataStorage().ForEachSnapshot(loader.Load) }) }) - if err != nil { - return + return err } s.DataStorage().QueryFinal(queriers) - return + + return nil } // diff --git a/pp/go/storage/head/shard/unloaded_data_storage.go b/pp/go/storage/head/shard/unloaded_data_storage.go index 2383934ff4..1b343f9fce 100644 --- a/pp/go/storage/head/shard/unloaded_data_storage.go +++ b/pp/go/storage/head/shard/unloaded_data_storage.go @@ -107,7 +107,7 @@ func (s *UnloadedDataStorage) ForEachSnapshot(f func(snapshot []byte, isLast boo for index, header := range s.snapshots { snapshot = snapshot[:header.SnapshotSize] size, err := s.storage.ReadAt(snapshot, offset) - if uint32(size) != header.SnapshotSize { + if size != int(header.SnapshotSize) { return err } offset += int64(size) @@ -141,7 +141,7 @@ func (s *UnloadedDataStorage) Close() (err error) { s.storage = nil } - return + return err } func (s *UnloadedDataStorage) IsEmpty() bool { @@ -155,7 +155,7 @@ type QueriedSeriesStorage struct { func NewQueriedSeriesStorage(storage1, storage2 StorageFile) *QueriedSeriesStorage { return &QueriedSeriesStorage{ - storages: [2]StorageFile{storage1, storage2}, + storages: [2]StorageFile{storage1, storage2}, //revive:disable-line:add-constant // 2 working files } } @@ -166,7 +166,9 @@ type queriedSeriesStorageHeader struct { } func (h *queriedSeriesStorageHeader) toSlice() []byte { - return (*(*[unsafe.Sizeof(queriedSeriesStorageHeader{})]byte)(unsafe.Pointer(h)))[:] + return (*(*[unsafe.Sizeof(queriedSeriesStorageHeader{})]byte)( + unsafe.Pointer(h), + ))[:] // #nosec G103 // it's meant to be that way } func (h *queriedSeriesStorageHeader) CalculateCrc32(queriedSeriesBitset []byte) uint32 { @@ -190,9 +192,9 @@ func (s *QueriedSeriesStorage) Write(queriedSeriesBitset []byte, timestamp int64 var headerBuffer [1 + unsafe.Sizeof(queriedSeriesStorageHeader{})]byte headerBuffer[0] = UnloadedDataStorageVersion - header := (*queriedSeriesStorageHeader)(unsafe.Pointer(&headerBuffer[1])) + header := (*queriedSeriesStorageHeader)(unsafe.Pointer(&headerBuffer[1])) // #nosec G103 it's meant to be that way header.timestamp = timestamp - header.size = uint32(len(queriedSeriesBitset)) + header.size = uint32(len(queriedSeriesBitset)) // #nosec G115 // no overflow header.CalculateCrc32(queriedSeriesBitset) if err := s.writeToStorage(storage, headerBuffer[:], queriedSeriesBitset); err != nil { @@ -205,7 +207,7 @@ func (s *QueriedSeriesStorage) Write(queriedSeriesBitset []byte, timestamp int64 return nil } -func (s *QueriedSeriesStorage) writeToStorage(storage StorageFile, headerBuffer, queriedSeriesBitset []byte) error { +func (*QueriedSeriesStorage) writeToStorage(storage StorageFile, headerBuffer, queriedSeriesBitset []byte) error { if _, err := storage.Seek(0, io.SeekStart); err != nil { return err } @@ -272,18 +274,17 @@ func (s *QueriedSeriesStorage) readStorageHeaders() (result []storageHeaderReade if err := reader.read(); err == nil { result = append(result, reader) maxSize = max(maxSize, reader.size) - } else { - if !os.IsNotExist(err) && !errors.Is(err, io.EOF) { - logger.Warnf("failed to read header: %v", err) - } + } else if !os.IsNotExist(err) && !errors.Is(err, io.EOF) { + logger.Warnf("failed to read header: %v", err) } } + //revive:disable-next-line:add-constant // 2 working files if len(result) == 2 && result[0].timestamp < result[1].timestamp { result[0], result[1] = result[1], result[0] } - return + return result, maxSize } func (s *QueriedSeriesStorage) Close() error { diff --git a/pp/go/storage/head/shard/wal/wal_test.go b/pp/go/storage/head/shard/wal/wal_test.go index 551c987078..0f00fb2456 100644 --- a/pp/go/storage/head/shard/wal/wal_test.go +++ b/pp/go/storage/head/shard/wal/wal_test.go @@ -82,5 +82,6 @@ func (s *WalSuite) TestCurrentSize2() { segmentWriter := &SegmentWriterMock[*EncodedSegmentMock]{} wl := wal.NewWal(enc, segmentWriter, maxSegmentSize) - _ = wl + + s.T().Log(wl) } From 00581e0ed8d5dd9d3d6365e42a62130d2c404cb5 Mon Sep 17 00:00:00 2001 From: Alexandr Yudin Date: Mon, 29 Sep 2025 10:42:26 +0000 Subject: [PATCH 56/96] for save --- cmd/prometheus/main.go | 10 +++++----- pp/go/storage/block/file_writer.go | 2 +- pp/go/storage/head/proxy/proxy.go | 3 --- 3 files changed, 6 insertions(+), 9 deletions(-) diff --git a/cmd/prometheus/main.go b/cmd/prometheus/main.go index fdc5ee6a34..eb841e3788 100644 --- a/cmd/prometheus/main.go +++ b/cmd/prometheus/main.go @@ -277,10 +277,10 @@ func (c *flagConfig) setFeatureListOptions(logger log.Logger) error { } func main() { - if os.Getenv("DEBUG") != "" { - runtime.SetBlockProfileRate(20) - runtime.SetMutexProfileFraction(20) - } + // if os.Getenv("DEBUG") != "" { + runtime.SetBlockProfileRate(20) + runtime.SetMutexProfileFraction(20) + // } var ( oldFlagRetentionDuration model.Duration @@ -417,7 +417,7 @@ func main() { Default("100000").Uint32Var(&cfg.WalMaxSamplesPerSegment) serverOnlyFlag(a, "storage.head-retention-timeout", "Timeout before inactive heads are shrieked."). - Default("3m").SetValue(&cfg.HeadRetentionTimeout) + Default("2m").SetValue(&cfg.HeadRetentionTimeout) // TODO: Remove in Prometheus 3.0. var b bool diff --git a/pp/go/storage/block/file_writer.go b/pp/go/storage/block/file_writer.go index e658eec23f..199c90fc5c 100644 --- a/pp/go/storage/block/file_writer.go +++ b/pp/go/storage/block/file_writer.go @@ -30,7 +30,7 @@ func NewFileWriter(fileName string) (*FileWriter, error) { indexFile, err := os.OpenFile( //nolint:gosec // need this permissions fileName, - os.O_CREATE|os.O_RDWR, + os.O_CREATE|os.O_WRONLY, 0o666, //revive:disable-line:add-constant // file permissions simple readable as octa-number ) if err != nil { diff --git a/pp/go/storage/head/proxy/proxy.go b/pp/go/storage/head/proxy/proxy.go index 6af058de69..9818e7b2b3 100644 --- a/pp/go/storage/head/proxy/proxy.go +++ b/pp/go/storage/head/proxy/proxy.go @@ -10,9 +10,6 @@ import ( type Head interface { // Close closes wals, query semaphore for the inability to get query and clear metrics. Close() error - - // ID returns id [Head]. - ID() string } // From 2b46965b0e286f42c4f666ddcbc3e4da27e9eddb Mon Sep 17 00:00:00 2001 From: Alexandr Yudin Date: Mon, 29 Sep 2025 14:58:04 +0000 Subject: [PATCH 57/96] some fix --- pp-pkg/storage/adapter.go | 4 +- pp/go/storage/builder.go | 2 +- pp/go/storage/head/services/functions.go | 47 +++---------- pp/go/storage/head/services/interface.go | 6 ++ pp/go/storage/loader.go | 4 +- pp/go/storage/manager.go | 17 ++--- pp/go/storage/proxy.go | 87 ++++++++++++++++++++++++ 7 files changed, 116 insertions(+), 51 deletions(-) create mode 100644 pp/go/storage/proxy.go diff --git a/pp-pkg/storage/adapter.go b/pp-pkg/storage/adapter.go index 9a077454b5..94d6a3beaa 100644 --- a/pp-pkg/storage/adapter.go +++ b/pp-pkg/storage/adapter.go @@ -27,7 +27,7 @@ var _ storage.Storage = (*Adapter)(nil) // Adapter for implementing the [Queryable] interface and append data. type Adapter struct { - proxy *pp_storage.ProxyHead + proxy *pp_storage.Proxy haTracker *hatracker.HighAvailabilityTracker hashdexFactory cppbridge.HashdexFactory hashdexLimits cppbridge.WALHashdexLimits @@ -43,7 +43,7 @@ type Adapter struct { // NewAdapter init new [Adapter]. func NewAdapter( clock clockwork.Clock, - proxy *pp_storage.ProxyHead, + proxy *pp_storage.Proxy, mergeOutOfOrderChunks func(), registerer prometheus.Registerer, ) *Adapter { diff --git a/pp/go/storage/builder.go b/pp/go/storage/builder.go index 8573062aa0..04d8cd8ef4 100644 --- a/pp/go/storage/builder.go +++ b/pp/go/storage/builder.go @@ -110,7 +110,7 @@ func (b *Builder) createShardOnDisk( headDir = filepath.Clean(headDir) shardFile, err := os.OpenFile( //nolint:gosec // need this permissions GetShardWalFilename(headDir, shardID), - os.O_WRONLY|os.O_CREATE, + os.O_WRONLY|os.O_CREATE|os.O_APPEND, 0o666, //revive:disable-line:add-constant // file permissions simple readable as octa-number ) if err != nil { diff --git a/pp/go/storage/head/services/functions.go b/pp/go/storage/head/services/functions.go index a6ec3cea12..871aaa2332 100644 --- a/pp/go/storage/head/services/functions.go +++ b/pp/go/storage/head/services/functions.go @@ -20,11 +20,11 @@ const ( // CFViaRange finalize segment from encoder and add to wal // and flush wal segment writer, write all buffered data to storage without sync, do via range. func CFViaRange[ - TTask Task, - TShard, TGoShard Shard, - THead Head[TTask, TShard, TGoShard], + TShard Shard, + THead RangeHead[TShard], ](h THead) error { - errs := make([]error, 0, h.NumberOfShards()*2) + // we hope that there will be no mistakes, positive expectations + var errs []error for shard := range h.RangeShards() { if err := shard.WalCommit(); err != nil { errs = append(errs, fmt.Errorf("commit shard id %d: %w", shard.ShardID(), err)) @@ -41,11 +41,11 @@ func CFViaRange[ // CFSViaRange finalize segment from encoder and add to wal // and flush wal segment writer, write all buffered data to storage and sync, do via range. func CFSViaRange[ - TTask Task, - TShard, TGoShard Shard, - THead Head[TTask, TShard, TGoShard], + TShard Shard, + THead RangeHead[TShard], ](h THead) error { - errs := make([]error, 0, h.NumberOfShards()*3) + // we hope that there will be no mistakes, positive expectations + var errs []error for shard := range h.RangeShards() { if err := shard.WalCommit(); err != nil { errs = append(errs, fmt.Errorf("commit shard id %d: %w", shard.ShardID(), err)) @@ -54,7 +54,7 @@ func CFSViaRange[ if err := shard.WalFlush(); err != nil { errs = append(errs, fmt.Errorf("flush shard id %d: %w", shard.ShardID(), err)) - // if the flush operation fails, skip the Sinc + // if the flush operation fails, skip the Sync continue } @@ -109,32 +109,3 @@ func MergeOutOfOrderChunksWithHead[ return t.Wait() } - -// // commitAndFlushViaTask finalize segment from encoder and add to wal -// // and flush wal segment writer, write all buffered data to storage, do via task. -// func commitAndFlushViaTask[ -// TTask Task, -// TDataStorage DataStorage, -// TLSS LSS, -// TShard, TGoShard Shard[TDataStorage, TLSS], -// THead Head[TTask, TDataStorage, TLSS, TShard, TGoShard], -// ](h THead) error { -// t := h.CreateTask( -// WalCommit, -// func(shard TGoShard) error { -// swal := shard.Wal() - -// // wal contains LSS and it is necessary to lock the LSS for reading for the commit. -// if err := shard.LSS().WithRLock(func(_, _ *cppbridge.LabelSetStorage) error { -// return swal.Commit() -// }); err != nil { -// return err -// } - -// return swal.Flush() -// }, -// ) -// h.Enqueue(t) - -// return t.Wait() -// } diff --git a/pp/go/storage/head/services/interface.go b/pp/go/storage/head/services/interface.go index efaf29b44e..7c29f98442 100644 --- a/pp/go/storage/head/services/interface.go +++ b/pp/go/storage/head/services/interface.go @@ -30,6 +30,12 @@ type ActiveHeadContainer[ // Head // +// RangeHead the minimum required [Head] implementation. +type RangeHead[TShard Shard] interface { + // RangeShards returns an iterator over the [Head] [Shard]s, through which the shard can be directly accessed. + RangeShards() func(func(TShard) bool) +} + // Head the minimum required [Head] implementation. type Head[ TTask Task, diff --git a/pp/go/storage/loader.go b/pp/go/storage/loader.go index 12c7f2a2b6..9dca1e2a98 100644 --- a/pp/go/storage/loader.go +++ b/pp/go/storage/loader.go @@ -240,7 +240,7 @@ func (l *ShardDataLoader) Load() error { queriedSeriesStorageIsEmpty, _ = l.loadQueriedSeries() } - decoder, err := l.loadWalFile(bufio.NewReaderSize(shardWalFile, 1024*1024*4), queriedSeriesStorageIsEmpty) + decoder, err := l.loadWalFile(bufio.NewReaderSize(shardWalFile, 1024*1024*10), queriedSeriesStorageIsEmpty) if err != nil { return err } @@ -289,7 +289,7 @@ func (l *ShardDataLoader) loadWalFile( func (l *ShardDataLoader) createShardWal(fileName string, walDecoder *cppbridge.HeadWalDecoder) error { shardWalFile, err := os.OpenFile( //nolint:gosec // need this permissions fileName, - os.O_WRONLY, + os.O_WRONLY|os.O_APPEND, 0o666, //revive:disable-line:add-constant // file permissions simple readable as octa-number ) if err != nil { diff --git a/pp/go/storage/manager.go b/pp/go/storage/manager.go index 2bc65258d2..cece1f4115 100644 --- a/pp/go/storage/manager.go +++ b/pp/go/storage/manager.go @@ -22,7 +22,6 @@ import ( "github.com/prometheus/prometheus/pp/go/storage/catalog" "github.com/prometheus/prometheus/pp/go/storage/head/container" "github.com/prometheus/prometheus/pp/go/storage/head/keeper" - "github.com/prometheus/prometheus/pp/go/storage/head/proxy" "github.com/prometheus/prometheus/pp/go/storage/head/services" "github.com/prometheus/prometheus/pp/go/storage/mediator" "github.com/prometheus/prometheus/pp/go/storage/querier" @@ -124,9 +123,10 @@ func (c *Config) SetNumberOfShards(numberOfShards uint16) bool { // Manager manages services for the work of the heads. type Manager struct { - g run.Group - closer *util.Closer - proxy *proxy.Proxy[*HeadOnDisk] + g run.Group + closer *util.Closer + // proxy *proxy.Proxy[*HeadOnDisk] + proxy *Proxy cgogc *cppbridge.CGOGC cfg *Config rotatorMediator *mediator.Mediator @@ -184,9 +184,10 @@ func NewManager( m := &Manager{ g: run.Group{}, closer: util.NewCloser(), - proxy: proxy.NewProxy(container.NewWeighted(h), hKeeper, services.CFSViaRange), - cgogc: cppbridge.NewCGOGC(r), - cfg: cfg, + // proxy: proxy.NewProxy(container.NewWeighted(h), hKeeper, services.CFSViaRange), + proxy: NewProxy(container.NewWeighted(h), hKeeper, services.CFSViaRange), + cgogc: cppbridge.NewCGOGC(r), + cfg: cfg, rotatorMediator: mediator.NewMediator( mediator.NewRotateTimerWithSeed(clock, o.BlockDuration, o.Seed), ), @@ -223,7 +224,7 @@ func (m *Manager) MergeOutOfOrderChunks() { } // Proxy returns proxy to the active [Head] and the keeper of old [Head]s. -func (m *Manager) Proxy() *proxy.Proxy[*HeadOnDisk] { +func (m *Manager) Proxy() *Proxy { return m.proxy } diff --git a/pp/go/storage/proxy.go b/pp/go/storage/proxy.go new file mode 100644 index 0000000000..6e8a23f063 --- /dev/null +++ b/pp/go/storage/proxy.go @@ -0,0 +1,87 @@ +package storage + +import ( + "context" + "errors" + "time" + + "github.com/prometheus/prometheus/pp/go/storage/head/container" + "github.com/prometheus/prometheus/pp/go/storage/head/keeper" +) + +// +// Proxy +// + +// Proxy it proxies requests to the active [Head] and the keeper of old [Head]s. +type Proxy struct { + activeHeadContainer *container.Weighted[HeadOnDisk, *HeadOnDisk] + keeper *keeper.Keeper[HeadOnDisk, *HeadOnDisk] + onClose func(h *HeadOnDisk) error +} + +// NewProxy init new [Proxy]. +func NewProxy( + activeHeadContainer *container.Weighted[HeadOnDisk, *HeadOnDisk], + hKeeper *keeper.Keeper[HeadOnDisk, *HeadOnDisk], + onClose func(h *HeadOnDisk) error, +) *Proxy { + return &Proxy{ + activeHeadContainer: activeHeadContainer, + keeper: hKeeper, + onClose: onClose, + } +} + +// Add the [Head] to the [Keeper] if there is a free slot. +func (p *Proxy) Add(head *HeadOnDisk, createdAt time.Duration) error { + return p.keeper.Add(head, createdAt) +} + +// AddWithReplace the [Head] to the [Keeper] with replace if the createdAt is earlier. +func (p *Proxy) AddWithReplace(head *HeadOnDisk, createdAt time.Duration) error { + return p.keeper.AddWithReplace(head, createdAt) +} + +// Close closes [ActiveHeadContainer] and [Keeper] for the inability work with [Head]. +func (p *Proxy) Close() error { + ahErr := p.activeHeadContainer.Close() + + h := p.activeHeadContainer.Get() + onCloseErr := p.onClose(h) + headCloseErr := h.Close() + + keeperErr := p.keeper.Close() + + return errors.Join(ahErr, onCloseErr, headCloseErr, keeperErr) +} + +// Get the active [Head]. +func (p *Proxy) Get() *HeadOnDisk { + return p.activeHeadContainer.Get() +} + +// HasSlot returns the tru if there is a slot in the [Keeper]. +func (p *Proxy) HasSlot() bool { + return p.keeper.HasSlot() +} + +// Heads returns a slice of the [Head]s stored in the [Keeper]. +func (p *Proxy) Heads() []*HeadOnDisk { + return p.keeper.Heads() +} + +// Remove removes [Head]s from the [Keeper]. +func (p *Proxy) Remove(headsForRemove []*HeadOnDisk) { + p.keeper.Remove(headsForRemove) +} + +// Replace the active [Head] with a new [Head]. +func (p *Proxy) Replace(ctx context.Context, newHead *HeadOnDisk) error { + return p.activeHeadContainer.Replace(ctx, newHead) +} + +// With calls fn(h Head) on active [Head]. +func (p *Proxy) With(ctx context.Context, fn func(h *HeadOnDisk) error) error { + return p.activeHeadContainer.With(ctx, fn) +} From b4c1f68d74ac7e4a286c4e49558dc7180f380c36 Mon Sep 17 00:00:00 2001 From: Alexandr Yudin Date: Mon, 29 Sep 2025 15:43:49 +0000 Subject: [PATCH 58/96] some rebuild --- pp/go/storage/appender/appender.go | 80 ++++++++++--------------- pp/go/storage/appender/appender_test.go | 2 +- pp/go/storage/head/head/head.go | 2 - pp/go/storage/head/head/head_test.go | 2 +- pp/go/storage/head/shard/shard.go | 80 ++++++++++++++----------- pp/go/storage/types.go | 8 +-- 6 files changed, 83 insertions(+), 91 deletions(-) diff --git a/pp/go/storage/appender/appender.go b/pp/go/storage/appender/appender.go index 3ab29d4399..dc04fd51e7 100644 --- a/pp/go/storage/appender/appender.go +++ b/pp/go/storage/appender/appender.go @@ -37,33 +37,23 @@ type Task interface { Wait() error } -// -// LSS -// - -// LSS the minimum required [LSS] implementation. -type LSS interface { - // WithLock calls fn on raws [cppbridge.LabelSetStorage] with write lock. - WithLock(fn func(target, input *cppbridge.LabelSetStorage) error) error - - // WithRLock calls fn on raws [cppbridge.LabelSetStorage] with read lock. - WithRLock(fn func(target, input *cppbridge.LabelSetStorage) error) error - - // ResetSnapshot resets the current snapshot. Use only WithLock. - ResetSnapshot() -} - // // Shard // // Shard the minimum required head [Shard] implementation. -type Shard[TLSS LSS] interface { +type Shard interface { // AppendInnerSeriesSlice add InnerSeries to [DataStorage]. AppendInnerSeriesSlice(innerSeriesSlice []*cppbridge.InnerSeries) - // LSS returns shard labelset storage [LSS]. - LSS() TLSS + // LSSWithLock calls fn on raws [cppbridge.LabelSetStorage] with write lock. + LSSWithLock(fn func(target, input *cppbridge.LabelSetStorage) error) error + + // LSSWithRLock calls fn on raws [cppbridge.LabelSetStorage] with read lock. + LSSWithRLock(fn func(target, input *cppbridge.LabelSetStorage) error) error + + // LSSResetSnapshot resets the current snapshot. Use only WithLock. + LSSResetSnapshot() // Relabeler returns relabeler for shard goroutines. Relabeler() *cppbridge.PerGoroutineRelabeler @@ -82,8 +72,7 @@ type Shard[TLSS LSS] interface { // Head the minimum required [Head] implementation. type Head[ TTask Task, - TLSS LSS, - TShard Shard[TLSS], + TShard Shard, ] interface { // CreateTask create a task for operations on the [Head] shards. CreateTask(taskName string, shardFn func(shard TShard) error) TTask @@ -105,9 +94,8 @@ type Head[ // Appender adds incoming data to the [Head]. type Appender[ TTask Task, - TLSS LSS, - TShard Shard[TLSS], - THead Head[TTask, TLSS, TShard], + TShard Shard, + THead Head[TTask, TShard], ] struct { head THead commitAndFlush func(h THead) error @@ -116,14 +104,13 @@ type Appender[ // New init new [Appender]. func New[ TTask Task, - TLSS LSS, - TShard Shard[TLSS], - THead Head[TTask, TLSS, TShard], + TShard Shard, + THead Head[TTask, TShard], ]( head THead, commitAndFlush func(h THead) error, -) Appender[TTask, TLSS, TShard, THead] { - return Appender[TTask, TLSS, TShard, THead]{ +) Appender[TTask, TShard, THead] { + return Appender[TTask, TShard, THead]{ head: head, commitAndFlush: commitAndFlush, } @@ -132,7 +119,7 @@ func New[ // Append incoming data to [Head]. // //revive:disable-next-line:flag-parameter this is a flag, but it's more convenient this way -func (a Appender[TTask, TLSS, TShard, THead]) Append( +func (a Appender[TTask, TShard, THead]) Append( ctx context.Context, incomingData *IncomingData, state *cppbridge.StateV2, @@ -142,13 +129,13 @@ func (a Appender[TTask, TLSS, TShard, THead]) Append( return nil, cppbridge.RelabelerStats{}, err } - shardedInnerSeries := NewShardedInnerSeries(a.head.NumberOfShards()) - shardedRelabeledSeries := NewShardedRelabeledSeries(a.head.NumberOfShards()) - + numberOfShards := a.head.NumberOfShards() + shardedInnerSeries := NewShardedInnerSeries(numberOfShards) + shardedRelabeledSeries := NewShardedRelabeledSeries(numberOfShards) stats, err := a.inputRelabelingStage( ctx, state, - NewDestructibleIncomingData(incomingData, int(a.head.NumberOfShards())), + NewDestructibleIncomingData(incomingData, int(numberOfShards)), shardedInnerSeries, shardedRelabeledSeries, ) @@ -157,7 +144,7 @@ func (a Appender[TTask, TLSS, TShard, THead]) Append( } if !shardedRelabeledSeries.IsEmpty() { - shardedStateUpdates := NewShardedStateUpdates(a.head.NumberOfShards()) + shardedStateUpdates := NewShardedStateUpdates(numberOfShards) if err = a.appendRelabelerSeriesStage( ctx, shardedInnerSeries, @@ -193,7 +180,7 @@ func (a Appender[TTask, TLSS, TShard, THead]) Append( // inputRelabelingStage first stage - relabeling. // //revive:disable-next-line:function-length long but this is first stage. -func (a Appender[TTask, TLSS, TShard, THead]) inputRelabelingStage( +func (a Appender[TTask, TShard, THead]) inputRelabelingStage( ctx context.Context, state *cppbridge.StateV2, incomingData *DestructibleIncomingData, @@ -205,13 +192,12 @@ func (a Appender[TTask, TLSS, TShard, THead]) inputRelabelingStage( lssInputRelabeling, func(shard TShard) error { var ( - lss = shard.LSS() relabeler = shard.Relabeler() shardID = shard.ShardID() ok bool ) - if err := lss.WithRLock(func(target, input *cppbridge.LabelSetStorage) (rErr error) { + if err := shard.LSSWithRLock(func(target, input *cppbridge.LabelSetStorage) (rErr error) { stats[shardID], ok, rErr = relabeler.RelabelingFromCache( ctx, input, @@ -236,7 +222,7 @@ func (a Appender[TTask, TLSS, TShard, THead]) inputRelabelingStage( hasReallocations bool rstats = cppbridge.RelabelerStats{} ) - err := lss.WithLock(func(target, input *cppbridge.LabelSetStorage) (rErr error) { + err := shard.LSSWithLock(func(target, input *cppbridge.LabelSetStorage) (rErr error) { rstats, hasReallocations, rErr = relabeler.Relabeling( ctx, input, @@ -248,7 +234,7 @@ func (a Appender[TTask, TLSS, TShard, THead]) inputRelabelingStage( ) if hasReallocations { - lss.ResetSnapshot() + shard.LSSResetSnapshot() } return rErr @@ -277,7 +263,7 @@ func (a Appender[TTask, TLSS, TShard, THead]) inputRelabelingStage( } // appendRelabelerSeriesStage second stage - append to lss relabeling ls. -func (a Appender[TTask, TLSS, TShard, THead]) appendRelabelerSeriesStage( +func (a Appender[TTask, TShard, THead]) appendRelabelerSeriesStage( ctx context.Context, shardedInnerSeries *ShardedInnerSeries, shardedRelabeledSeries *ShardedRelabeledSeries, @@ -293,9 +279,7 @@ func (a Appender[TTask, TLSS, TShard, THead]) appendRelabelerSeriesStage( return nil } - lss := shard.LSS() - - return lss.WithLock(func(target, _ *cppbridge.LabelSetStorage) error { + return shard.LSSWithLock(func(target, _ *cppbridge.LabelSetStorage) error { hasReallocations, err := shard.Relabeler().AppendRelabelerSeries( ctx, target, @@ -308,7 +292,7 @@ func (a Appender[TTask, TLSS, TShard, THead]) appendRelabelerSeriesStage( } if hasReallocations { - lss.ResetSnapshot() + shard.LSSResetSnapshot() } return nil @@ -321,7 +305,7 @@ func (a Appender[TTask, TLSS, TShard, THead]) appendRelabelerSeriesStage( } // updateRelabelerStateStage third stage - update state cache. -func (a Appender[TTask, TLSS, TShard, THead]) updateRelabelerStateStage( +func (a Appender[TTask, TShard, THead]) updateRelabelerStateStage( ctx context.Context, state *cppbridge.StateV2, shardedStateUpdates *ShardedStateUpdates, @@ -342,7 +326,7 @@ func (a Appender[TTask, TLSS, TShard, THead]) updateRelabelerStateStage( } // appendInnerSeriesAndWriteToWal append [cppbridge.InnerSeries] to [Shard]'s to [DataStorage] and write to [Wal]. -func (a Appender[TTask, TLSS, TShard, THead]) appendInnerSeriesAndWriteToWal( +func (a Appender[TTask, TShard, THead]) appendInnerSeriesAndWriteToWal( shardedInnerSeries *ShardedInnerSeries, ) (uint32, error) { tw := task.NewTaskWaiter[TTask](2) //revive:disable-line:add-constant // 2 task for wait @@ -381,7 +365,7 @@ func (a Appender[TTask, TLSS, TShard, THead]) appendInnerSeriesAndWriteToWal( return atomicLimitExhausted, tw.Wait() } -func (a Appender[TTask, TLSS, TShard, THead]) resolveState(state *cppbridge.StateV2) error { +func (a Appender[TTask, TShard, THead]) resolveState(state *cppbridge.StateV2) error { if state == nil { return errNilState } diff --git a/pp/go/storage/appender/appender_test.go b/pp/go/storage/appender/appender_test.go index c96fef8e2c..15d69f1287 100644 --- a/pp/go/storage/appender/appender_test.go +++ b/pp/go/storage/appender/appender_test.go @@ -20,7 +20,7 @@ func TestXxx(t *testing.T) { h := head.NewHead( id, - []*shard.Shard[*testWal]{sd}, + []*shard.Shard{sd}, shard.NewPerGoroutineShard[*testWal], nil, generation, diff --git a/pp/go/storage/head/head/head.go b/pp/go/storage/head/head/head.go index 23dba6a3ea..8b89ce976d 100644 --- a/pp/go/storage/head/head/head.go +++ b/pp/go/storage/head/head/head.go @@ -28,8 +28,6 @@ const defaultNumberOfWorkers = 2 // Shard the minimum required head Shard implementation. type Shard interface { - // LSS() *LSS - // ShardID returns the shard ID. ShardID() uint16 diff --git a/pp/go/storage/head/head/head_test.go b/pp/go/storage/head/head/head_test.go index e09ea4ded7..f3d718c6f3 100644 --- a/pp/go/storage/head/head/head_test.go +++ b/pp/go/storage/head/head/head_test.go @@ -18,7 +18,7 @@ func TestXxx(t *testing.T) { h := head.NewHead( id, - []*shard.Shard[*testWal]{sd}, + []*shard.Shard{sd}, shard.NewPerGoroutineShard[*testWal], nil, generation, diff --git a/pp/go/storage/head/shard/shard.go b/pp/go/storage/head/shard/shard.go index 2e660b5d1a..f3a47953b7 100644 --- a/pp/go/storage/head/shard/shard.go +++ b/pp/go/storage/head/shard/shard.go @@ -36,26 +36,26 @@ type Wal interface { // // Shard bridge to labelset storage, data storage and wal. -type Shard[TWal Wal] struct { +type Shard struct { lss *LSS dataStorage *DataStorage unloadedDataStorage *UnloadedDataStorage queriedSeriesStorage *QueriedSeriesStorage loadAndQueryTask *LoadAndQuerySeriesDataTask - wal TWal + wal Wal id uint16 } // NewShard init new [Shard]. -func NewShard[TWal Wal]( +func NewShard( lss *LSS, dataStorage *DataStorage, unloadedDataStorage *UnloadedDataStorage, queriedSeriesStorage *QueriedSeriesStorage, - wal TWal, + wal Wal, shardID uint16, -) *Shard[TWal] { - return &Shard[TWal]{ +) *Shard { + return &Shard{ id: shardID, lss: lss, dataStorage: dataStorage, @@ -67,12 +67,12 @@ func NewShard[TWal Wal]( } // AppendInnerSeriesSlice add InnerSeries to [DataStorage]. -func (s *Shard[TWal]) AppendInnerSeriesSlice(innerSeriesSlice []*cppbridge.InnerSeries) { +func (s *Shard) AppendInnerSeriesSlice(innerSeriesSlice []*cppbridge.InnerSeries) { s.dataStorage.AppendInnerSeriesSlice(innerSeriesSlice) } // Close closes the wal segmentWriter. -func (s *Shard[TWal]) Close() error { +func (s *Shard) Close() error { err := s.wal.Close() if s.unloadedDataStorage != nil { @@ -87,89 +87,99 @@ func (s *Shard[TWal]) Close() error { } // DSAllocatedMemory return size of allocated memory for [DataStorage]. -func (s *Shard[TWal]) DSAllocatedMemory() uint64 { +func (s *Shard) DSAllocatedMemory() uint64 { return s.dataStorage.AllocatedMemory() } // DataStorage returns shard [DataStorage]. -func (s *Shard[TWal]) DataStorage() *DataStorage { +func (s *Shard) DataStorage() *DataStorage { return s.dataStorage } // LSS returns shard labelset storage [LSS]. -func (s *Shard[TWal]) LSS() *LSS { +func (s *Shard) LSS() *LSS { return s.lss } // LSSAllocatedMemory return size of allocated memory for labelset storages. -func (s *Shard[TWal]) LSSAllocatedMemory() uint64 { +func (s *Shard) LSSAllocatedMemory() uint64 { return s.lss.AllocatedMemory() } +// LSSWithLock calls fn on raws [cppbridge.LabelSetStorage] with write lock. +func (s *Shard) LSSWithLock(fn func(target, input *cppbridge.LabelSetStorage) error) error { + return s.lss.WithLock(fn) +} + +// LSSWithRLock calls fn on raws [cppbridge.LabelSetStorage] with read lock. +func (s *Shard) LSSWithRLock(fn func(target, input *cppbridge.LabelSetStorage) error) error { + return s.lss.WithRLock(fn) +} + +// LSSResetSnapshot resets the current snapshot. Use only WithLock. +func (s *Shard) LSSResetSnapshot() { + s.lss.ResetSnapshot() +} + // MergeOutOfOrderChunks merge chunks with out of order data chunks in [DataStorage]. -func (s *Shard[TWal]) MergeOutOfOrderChunks() { +func (s *Shard) MergeOutOfOrderChunks() { s.dataStorage.MergeOutOfOrderChunks() } // ShardID returns the shard ID. -func (s *Shard[TWal]) ShardID() uint16 { +func (s *Shard) ShardID() uint16 { return s.id } -// Wal returns write-ahead log. -func (s *Shard[TWal]) Wal() TWal { - return s.wal -} - // WalCommit finalize segment from encoder and write to wal. -func (s *Shard[TWal]) WalCommit() error { +func (s *Shard) WalCommit() error { return s.lss.WithRLock(func(_, _ *cppbridge.LabelSetStorage) error { return s.wal.Commit() }) } // WalCurrentSize returns current [Wal] size. -func (s *Shard[TWal]) WalCurrentSize() int64 { +func (s *Shard) WalCurrentSize() int64 { return s.wal.CurrentSize() } // WalFlush flush all contetnt into wal. -func (s *Shard[TWal]) WalFlush() error { +func (s *Shard) WalFlush() error { return s.wal.Flush() } // WalSync commits the current contents of the [Wal]. -func (s *Shard[TWal]) WalSync() error { +func (s *Shard) WalSync() error { return s.wal.Sync() } // WalWrite append the incoming inner series to wal encoder. -func (s *Shard[TWal]) WalWrite(innerSeriesSlice []*cppbridge.InnerSeries) (bool, error) { +func (s *Shard) WalWrite(innerSeriesSlice []*cppbridge.InnerSeries) (bool, error) { return s.wal.Write(innerSeriesSlice) } // TimeInterval get time interval from [DataStorage]. -func (s *Shard[TWal]) TimeInterval(invalidateCache bool) cppbridge.TimeInterval { +func (s *Shard) TimeInterval(invalidateCache bool) cppbridge.TimeInterval { return s.dataStorage.TimeInterval(invalidateCache) } // UnloadedDataStorage get unloaded data storage -func (s *Shard[TWal]) UnloadedDataStorage() *UnloadedDataStorage { +func (s *Shard) UnloadedDataStorage() *UnloadedDataStorage { return s.unloadedDataStorage } // QueriedSeriesStorage get queried series storage -func (s *Shard[TWal]) QueriedSeriesStorage() *QueriedSeriesStorage { +func (s *Shard) QueriedSeriesStorage() *QueriedSeriesStorage { return s.queriedSeriesStorage } // LoadAndQuerySeriesDataTask get load and query series data task -func (s *Shard[TWal]) LoadAndQuerySeriesDataTask() *LoadAndQuerySeriesDataTask { +func (s *Shard) LoadAndQuerySeriesDataTask() *LoadAndQuerySeriesDataTask { return s.loadAndQueryTask } // UnloadUnusedSeriesData unload unused series data -func (s *Shard[TWal]) UnloadUnusedSeriesData() error { +func (s *Shard) UnloadUnusedSeriesData() error { if s.UnloadedDataStorage() == nil { return nil } @@ -202,7 +212,7 @@ func (s *Shard[TWal]) UnloadUnusedSeriesData() error { } // LoadAndQuerySeriesData loads the data and queries the series from the [DataStorage]. -func (s *Shard[TWal]) LoadAndQuerySeriesData() (err error) { +func (s *Shard) LoadAndQuerySeriesData() (err error) { var queriers []uintptr s.loadAndQueryTask.Release(func(q []uintptr) { queriers = q @@ -225,20 +235,20 @@ func (s *Shard[TWal]) LoadAndQuerySeriesData() (err error) { // // PerGoroutineShard wrapper of shard with [PerGoroutineRelabeler] for goroutines. -type PerGoroutineShard[TWal Wal] struct { +type PerGoroutineShard struct { relabeler *cppbridge.PerGoroutineRelabeler - *Shard[TWal] + *Shard } // NewPerGoroutineShard init new [PerGoroutineShard]. -func NewPerGoroutineShard[TWal Wal](s *Shard[TWal], numberOfShards uint16) *PerGoroutineShard[TWal] { - return &PerGoroutineShard[TWal]{ +func NewPerGoroutineShard[TWal Wal](s *Shard, numberOfShards uint16) *PerGoroutineShard { + return &PerGoroutineShard{ relabeler: cppbridge.NewPerGoroutineRelabeler(numberOfShards, s.ShardID()), Shard: s, } } // Relabeler returns relabeler for shard goroutines. -func (s *PerGoroutineShard[TWal]) Relabeler() *cppbridge.PerGoroutineRelabeler { +func (s *PerGoroutineShard) Relabeler() *cppbridge.PerGoroutineRelabeler { return s.relabeler } diff --git a/pp/go/storage/types.go b/pp/go/storage/types.go index 246c16fead..1f12b44065 100644 --- a/pp/go/storage/types.go +++ b/pp/go/storage/types.go @@ -16,11 +16,11 @@ type WalOnDisk = wal.Wal[ *writer.Buffered[*cppbridge.EncodedSegment], ] -// ShardOnDisk [shard.Shard] with [WalOnDisk]. -type ShardOnDisk = shard.Shard[*WalOnDisk] +// ShardOnDisk [shard.Shard]. +type ShardOnDisk = shard.Shard -// PerGoroutineShard [shard.PerGoroutineShard] with [WalOnDisk]. -type PerGoroutineShard = shard.PerGoroutineShard[*WalOnDisk] +// PerGoroutineShard [shard.PerGoroutineShard]. +type PerGoroutineShard = shard.PerGoroutineShard // HeadOnDisk [head.Head] with [ShardOnDisk]. type HeadOnDisk = head.Head[*ShardOnDisk, *PerGoroutineShard] From a780b29d400be9ed4395b526141ec84a9493f231 Mon Sep 17 00:00:00 2001 From: Alexandr Yudin Date: Mon, 29 Sep 2025 16:24:36 +0000 Subject: [PATCH 59/96] for save --- pp/go/storage/appender/appender.go | 25 +++++++++++++++---------- 1 file changed, 15 insertions(+), 10 deletions(-) diff --git a/pp/go/storage/appender/appender.go b/pp/go/storage/appender/appender.go index dc04fd51e7..ab44a538e2 100644 --- a/pp/go/storage/appender/appender.go +++ b/pp/go/storage/appender/appender.go @@ -192,9 +192,11 @@ func (a Appender[TTask, TShard, THead]) inputRelabelingStage( lssInputRelabeling, func(shard TShard) error { var ( - relabeler = shard.Relabeler() - shardID = shard.ShardID() - ok bool + relabeler = shard.Relabeler() + shardID = shard.ShardID() + ok bool + innerSeries = shardedInnerSeries.DataBySourceShard(shardID) + shardedData = incomingData.ShardedData() ) if err := shard.LSSWithRLock(func(target, input *cppbridge.LabelSetStorage) (rErr error) { @@ -203,8 +205,8 @@ func (a Appender[TTask, TShard, THead]) inputRelabelingStage( input, target, state, - incomingData.ShardedData(), - shardedInnerSeries.DataBySourceShard(shardID), + shardedData, + innerSeries, ) return rErr @@ -221,6 +223,7 @@ func (a Appender[TTask, TShard, THead]) inputRelabelingStage( var ( hasReallocations bool rstats = cppbridge.RelabelerStats{} + relabeledSeries = shardedRelabeledSeries.DataByShard(shardID) ) err := shard.LSSWithLock(func(target, input *cppbridge.LabelSetStorage) (rErr error) { rstats, hasReallocations, rErr = relabeler.Relabeling( @@ -228,9 +231,9 @@ func (a Appender[TTask, TShard, THead]) inputRelabelingStage( input, target, state, - incomingData.ShardedData(), - shardedInnerSeries.DataBySourceShard(shardID), - shardedRelabeledSeries.DataByShard(shardID), + shardedData, + innerSeries, + relabeledSeries, ) if hasReallocations { @@ -279,13 +282,15 @@ func (a Appender[TTask, TShard, THead]) appendRelabelerSeriesStage( return nil } + innerSeries := shardedInnerSeries.DataByShard(shardID) + stateUpdates := shardedStateUpdates.DataByShard(shardID) return shard.LSSWithLock(func(target, _ *cppbridge.LabelSetStorage) error { hasReallocations, err := shard.Relabeler().AppendRelabelerSeries( ctx, target, - shardedInnerSeries.DataByShard(shardID), + innerSeries, relabeledSeries, - shardedStateUpdates.DataByShard(shardID), + stateUpdates, ) if err != nil { return fmt.Errorf("shard %d: %w", shardID, err) From 5a804a13ae4861c706da291d39f9994577bb7d81 Mon Sep 17 00:00:00 2001 From: Alexandr Yudin Date: Tue, 30 Sep 2025 05:35:16 +0000 Subject: [PATCH 60/96] some fix --- cmd/prometheus/main.go | 5 ++--- pp/go/storage/manager.go | 4 ++-- web/web_fuzzy_test.go | 2 +- 3 files changed, 5 insertions(+), 6 deletions(-) diff --git a/cmd/prometheus/main.go b/cmd/prometheus/main.go index eb841e3788..4ebb137253 100644 --- a/cmd/prometheus/main.go +++ b/cmd/prometheus/main.go @@ -769,7 +769,7 @@ func main() { CommitInterval: time.Duration(cfg.WalCommitInterval), MaxRetentionPeriod: time.Duration(cfg.tsdb.RetentionDuration), HeadRetentionPeriod: time.Duration(cfg.HeadRetentionTimeout), - QueueSize: 2, + KeeperCapacity: 2, DataDir: localStoragePath, MaxSegmentSize: cfg.WalMaxSamplesPerSegment, NumberOfShards: receiverConfig.NumberOfShards, @@ -1012,7 +1012,6 @@ func main() { } // Depends on cfg.web.ScrapeManager so needs to be after cfg.web.ScrapeManager = scrapeManager. - // TODO receiver adapter webHandler := web.New(log.With(logger, "component", "web"), &cfg.web, adapter) // PP_CHANGES.md: rebuild on cpp // Monitor outgoing connections on default transport with conntrack. @@ -1414,7 +1413,7 @@ func main() { db, err := agent.Open( logger, prometheus.DefaultRegisterer, - adapter, // TODO RW // PP_CHANGES.md: rebuild on cpp + adapter, // PP_CHANGES.md: rebuild on cpp localStoragePath, &opts, ) diff --git a/pp/go/storage/manager.go b/pp/go/storage/manager.go index cece1f4115..09d31f1c8a 100644 --- a/pp/go/storage/manager.go +++ b/pp/go/storage/manager.go @@ -71,7 +71,7 @@ type Options struct { CommitInterval time.Duration MaxRetentionPeriod time.Duration HeadRetentionPeriod time.Duration - QueueSize int + KeeperCapacity int DataDir string MaxSegmentSize uint32 NumberOfShards uint16 @@ -180,7 +180,7 @@ func NewManager( return nil, errors.Join(fmt.Errorf("failed to set active status: %w", err), h.Close()) } - hKeeper := keeper.NewKeeper[HeadOnDisk](o.QueueSize) + hKeeper := keeper.NewKeeper[HeadOnDisk](o.KeeperCapacity) m := &Manager{ g: run.Group{}, closer: util.NewCloser(), diff --git a/web/web_fuzzy_test.go b/web/web_fuzzy_test.go index 6386869a78..2a2ac14da4 100644 --- a/web/web_fuzzy_test.go +++ b/web/web_fuzzy_test.go @@ -156,7 +156,7 @@ func makeManager( CommitInterval: 5 * time.Second, MaxRetentionPeriod: 24 * time.Hour, HeadRetentionPeriod: 4 * time.Hour, - QueueSize: 2, + KeeperCapacity: 2, DataDir: dbDir, MaxSegmentSize: 100e3, NumberOfShards: 2, From 061fa320c05f3c28303f36daa62e2af1775f7e03 Mon Sep 17 00:00:00 2001 From: Alexandr Yudin Date: Tue, 30 Sep 2025 15:18:47 +0000 Subject: [PATCH 61/96] for save --- cmd/prometheus/main.go | 22 ++++-- pp/bare_bones/concepts.h | 20 ++++++ pp/entrypoint/head_wal.cpp | 12 +--- pp/entrypoint/head_wal.h | 14 +--- pp/go/cppbridge/entrypoint.go | 12 ++-- pp/go/cppbridge/entrypoint.h | 14 +--- pp/go/cppbridge/head_wal.go | 71 +++++++++++++++++-- pp/go/cppbridge/prometheus_relabeler_test.go | 3 + pp/go/relabeler/head/wal.go | 6 +- pp/go/storage/block/block.go | 5 ++ pp/go/storage/block/block_writer.go | 14 ++++ pp/go/storage/builder.go | 2 +- pp/go/storage/head/keeper/keeper.go | 20 ++++-- pp/go/storage/head/keeper/keeper_test.go | 20 ++++-- pp/go/storage/head/services/persistener.go | 6 ++ .../storage/head/services/persistener_test.go | 12 +--- pp/go/storage/head/shard/wal/wal.go | 44 +++++------- pp/go/storage/head/shard/wal/wal_moq_test.go | 69 ++---------------- pp/go/storage/head/shard/wal/wal_test.go | 6 +- pp/go/storage/loader.go | 7 +- pp/go/storage/manager.go | 30 ++++---- pp/go/storage/types.go | 5 +- pp/wal/encoder.h | 19 +++-- 23 files changed, 242 insertions(+), 191 deletions(-) diff --git a/cmd/prometheus/main.go b/cmd/prometheus/main.go index 4ebb137253..23f299002c 100644 --- a/cmd/prometheus/main.go +++ b/cmd/prometheus/main.go @@ -417,7 +417,7 @@ func main() { Default("100000").Uint32Var(&cfg.WalMaxSamplesPerSegment) serverOnlyFlag(a, "storage.head-retention-timeout", "Timeout before inactive heads are shrieked."). - Default("2m").SetValue(&cfg.HeadRetentionTimeout) + Default("5m").SetValue(&cfg.HeadRetentionTimeout) // TODO: Remove in Prometheus 3.0. var b bool @@ -764,8 +764,9 @@ func main() { hManagerReadyNotifier := ready.NewNotifiableNotifier() hManager, err := pp_storage.NewManager( &pp_storage.Options{ - Seed: cfgFile.GlobalConfig.ExternalLabels.Hash(), - BlockDuration: time.Duration(cfg.tsdb.MinBlockDuration), + Seed: cfgFile.GlobalConfig.ExternalLabels.Hash(), + // BlockDuration: time.Duration(cfg.tsdb.MinBlockDuration), + BlockDuration: 7 * time.Minute, CommitInterval: time.Duration(cfg.WalCommitInterval), MaxRetentionPeriod: time.Duration(cfg.tsdb.RetentionDuration), HeadRetentionPeriod: time.Duration(cfg.HeadRetentionTimeout), @@ -1468,14 +1469,22 @@ func main() { ) } { // PP_CHANGES.md: rebuild on cpp start - // run receiver. + // run head manager. + cancel := make(chan struct{}) g.Add( func() error { - <-dbOpen + select { + case <-dbOpen: + // In case a shutdown is initiated before the dbOpen is released + case <-cancel: + return nil + } + return hManager.Run() }, func(err error) { - level.Info(logger).Log("msg", "Stopping head manager...") + level.Info(logger).Log("msg", "Stopping head manager...", "msg", err) + close(cancel) if err := hManager.Shutdown(context.Background()); err != nil { level.Error(logger).Log("msg", "Head manager shutdown failed", "err", err) } @@ -1495,6 +1504,7 @@ func main() { if errors.Is(err, querier.UnrecoverableError{}) { level.Error(logger).Log("msg", "Received unrecoverable error", "err", err) } + level.Info(logger).Log("msg", "Unrecoverable Error Handler stopped.") }, ) } // PP_CHANGES.md: rebuild on cpp end diff --git a/pp/bare_bones/concepts.h b/pp/bare_bones/concepts.h index 62e0053bfc..bb5d20be4f 100644 --- a/pp/bare_bones/concepts.h +++ b/pp/bare_bones/concepts.h @@ -14,6 +14,26 @@ concept has_allocated_memory = requires(const T& t) { { t.allocated_memory() } -> std::convertible_to; }; +template +concept has_earliest_timestamp = requires(const T& t) { + { t.earliest_timestamp() } -> std::convertible_to; +}; + +template +concept has_latest_timestamp = requires(const T& t) { + { t.latest_timestamp() } -> std::convertible_to; +}; + +template +concept has_series = requires(const T& t) { + { t.series() } -> std::convertible_to; +}; + +template +concept has_remainder_size = requires(const T& t) { + { t.remainder_size() } -> std::convertible_to; +}; + template concept dereferenceable_has_allocated_memory = requires(const T& t) { { t->allocated_memory() } -> std::convertible_to; diff --git a/pp/entrypoint/head_wal.cpp b/pp/entrypoint/head_wal.cpp index 74808059f4..11524871ba 100644 --- a/pp/entrypoint/head_wal.cpp +++ b/pp/entrypoint/head_wal.cpp @@ -67,12 +67,7 @@ extern "C" void prompp_head_wal_encoder_add_inner_series(void* args, void* res) }; struct Result { - int64_t earliest_timestamp; - int64_t latest_timestamp; - size_t allocated_memory; uint32_t samples; - uint32_t series; - uint32_t remainder_size; PromPP::Primitives::Go::Slice error; }; @@ -93,14 +88,9 @@ extern "C" void prompp_head_wal_encoder_finalize(void* args, void* res) { }; struct Result { - int64_t earliest_timestamp; - int64_t latest_timestamp; - size_t allocated_memory; - uint32_t samples; - uint32_t series; - uint32_t remainder_size; PromPP::Primitives::Go::Slice segment; PromPP::Primitives::Go::Slice error; + uint32_t samples; }; const auto in = static_cast(args); diff --git a/pp/entrypoint/head_wal.h b/pp/entrypoint/head_wal.h index d15999a7cc..d88ccde4bc 100644 --- a/pp/entrypoint/head_wal.h +++ b/pp/entrypoint/head_wal.h @@ -45,13 +45,8 @@ void prompp_head_wal_encoder_dtor(void* args); * encoder uintptr // pointer to constructed encoder; * } * @param res { - * earliestTimestamp int64 // minimal sample timestamp in segment - * latestTimestamp int64 // maximal sample timestamp in segment - * allocatedMemory uint64 // size of allocated memory for label sets; - * samples uint32 // number of samples in segment - * series uint32 // number of series in segment - * remainderSize uint32 // rest of internal buffers capacity * error []byte // error string if thrown + * samples uint32 // number of samples in segment * } */ void prompp_head_wal_encoder_add_inner_series(void* args, void* res); @@ -63,14 +58,9 @@ void prompp_head_wal_encoder_add_inner_series(void* args, void* res); * encoder uintptr // pointer to constructed encoder * } * @param res { - * earliestTimestamp int64 // minimal sample timestamp in segment - * latestTimestamp int64 // maximal sample timestamp in segment - * allocatedMemory uint64 // size of allocated memory for label sets; - * samples uint32 // number of samples in segment - * series uint32 // number of series in segment - * remainderSize uint32 // rest of internal buffers capacity * segment []byte // segment content * error []byte // error string if thrown + * samples uint32 // number of samples in segment * } */ void prompp_head_wal_encoder_finalize(void* args, void* res); diff --git a/pp/go/cppbridge/entrypoint.go b/pp/go/cppbridge/entrypoint.go index e69c0217ad..6bf981d913 100644 --- a/pp/go/cppbridge/entrypoint.go +++ b/pp/go/cppbridge/entrypoint.go @@ -3080,14 +3080,14 @@ func headWalEncoderCtor(shardID uint16, logShards uint8, lss uintptr) uintptr { return res.encoder } -func headWalEncoderAddInnerSeries(encoder uintptr, innerSeries []*InnerSeries) (stats WALEncoderStats, err error) { +func headWalEncoderAddInnerSeries(encoder uintptr, innerSeries []*InnerSeries) (samples uint32, err error) { args := struct { innerSeries []*InnerSeries encoder uintptr }{innerSeries, encoder} var res struct { - WALEncoderStats exception []byte + samples uint32 } start := time.Now().UnixNano() @@ -3100,18 +3100,18 @@ func headWalEncoderAddInnerSeries(encoder uintptr, innerSeries []*InnerSeries) ( headWalEncoderAddInnerSeriesSum.Add(float64(time.Now().UnixNano() - start)) headWalEncoderAddInnerSeriesCount.Inc() - return res.WALEncoderStats, handleException(res.exception) + return res.samples, handleException(res.exception) } // headWalEncoderFinalize - finalize the encoded data in the C++ encoder to Segment. -func headWalEncoderFinalize(encoder uintptr) (stats WALEncoderStats, segment []byte, err error) { +func headWalEncoderFinalize(encoder uintptr) (samples uint32, segment []byte, err error) { args := struct { encoder uintptr }{encoder} var res struct { - WALEncoderStats segment []byte exception []byte + samples uint32 } start := time.Now().UnixNano() @@ -3124,7 +3124,7 @@ func headWalEncoderFinalize(encoder uintptr) (stats WALEncoderStats, segment []b headWalEncoderFinalizeSum.Add(float64(time.Now().UnixNano() - start)) headWalEncoderFinalizeCount.Inc() - return res.WALEncoderStats, res.segment, handleException(res.exception) + return res.samples, res.segment, handleException(res.exception) } func headWalEncoderDtor(encoder uintptr) { diff --git a/pp/go/cppbridge/entrypoint.h b/pp/go/cppbridge/entrypoint.h index 188c5a8b10..3fd0ad8e6f 100755 --- a/pp/go/cppbridge/entrypoint.h +++ b/pp/go/cppbridge/entrypoint.h @@ -150,13 +150,8 @@ void prompp_head_wal_encoder_dtor(void* args); * encoder uintptr // pointer to constructed encoder; * } * @param res { - * earliestTimestamp int64 // minimal sample timestamp in segment - * latestTimestamp int64 // maximal sample timestamp in segment - * allocatedMemory uint64 // size of allocated memory for label sets; - * samples uint32 // number of samples in segment - * series uint32 // number of series in segment - * remainderSize uint32 // rest of internal buffers capacity * error []byte // error string if thrown + * samples uint32 // number of samples in segment * } */ void prompp_head_wal_encoder_add_inner_series(void* args, void* res); @@ -168,14 +163,9 @@ void prompp_head_wal_encoder_add_inner_series(void* args, void* res); * encoder uintptr // pointer to constructed encoder * } * @param res { - * earliestTimestamp int64 // minimal sample timestamp in segment - * latestTimestamp int64 // maximal sample timestamp in segment - * allocatedMemory uint64 // size of allocated memory for label sets; - * samples uint32 // number of samples in segment - * series uint32 // number of series in segment - * remainderSize uint32 // rest of internal buffers capacity * segment []byte // segment content * error []byte // error string if thrown + * samples uint32 // number of samples in segment * } */ void prompp_head_wal_encoder_finalize(void* args, void* res); diff --git a/pp/go/cppbridge/head_wal.go b/pp/go/cppbridge/head_wal.go index 039e95707b..0cb5a3ce7f 100644 --- a/pp/go/cppbridge/head_wal.go +++ b/pp/go/cppbridge/head_wal.go @@ -1,6 +1,59 @@ package cppbridge -import "runtime" +import ( + "hash/crc32" + "io" + "runtime" +) + +// +// HeadEncodedSegment +// + +// HeadEncodedSegment the encoded segment from the head wal. +type HeadEncodedSegment struct { + buf []byte + samples uint32 +} + +// NewHeadEncodedSegment init new [HeadEncodedSegment]. +func NewHeadEncodedSegment(b []byte, samples uint32) *HeadEncodedSegment { + s := &HeadEncodedSegment{ + buf: b, + samples: samples, + } + + runtime.SetFinalizer(s, func(s *HeadEncodedSegment) { + freeBytes(s.buf) + }) + + return s +} + +// Samples returns count of samples in segment. +func (s HeadEncodedSegment) Samples() uint32 { + return s.samples +} + +// Size returns len of bytes. +func (s *HeadEncodedSegment) Size() int64 { + return int64(len(s.buf)) +} + +// CRC32 the hash amount according to the data. +func (s *HeadEncodedSegment) CRC32() uint32 { + return crc32.ChecksumIEEE(s.buf) +} + +// WriteTo implements io.WriterTo inerface. +func (s *HeadEncodedSegment) WriteTo(w io.Writer) (int64, error) { + n, err := w.Write(s.buf) + return int64(n), err +} + +// +// HeadWalEncoder +// type HeadWalEncoder struct { lss *LabelSetStorage @@ -24,18 +77,22 @@ func (*HeadWalEncoder) Version() uint8 { return EncodersVersion() } -func (e *HeadWalEncoder) Encode(innerSeriesSlice []*InnerSeries) (WALEncoderStats, error) { - res, err := headWalEncoderAddInnerSeries(e.encoder, innerSeriesSlice) +func (e *HeadWalEncoder) Encode(innerSeriesSlice []*InnerSeries) (uint32, error) { + samples, err := headWalEncoderAddInnerSeries(e.encoder, innerSeriesSlice) runtime.KeepAlive(e) - return res, err + return samples, err } -func (e *HeadWalEncoder) Finalize() (*EncodedSegment, error) { - stats, segment, err := headWalEncoderFinalize(e.encoder) +func (e *HeadWalEncoder) Finalize() (*HeadEncodedSegment, error) { + samples, segment, err := headWalEncoderFinalize(e.encoder) runtime.KeepAlive(e) - return NewEncodedSegment(segment, stats), err + return NewHeadEncodedSegment(segment, samples), err } +// +// HeadWalDecoder +// + type HeadWalDecoder struct { lss *LabelSetStorage decoder uintptr diff --git a/pp/go/cppbridge/prometheus_relabeler_test.go b/pp/go/cppbridge/prometheus_relabeler_test.go index b268074bd2..9ca9122d13 100644 --- a/pp/go/cppbridge/prometheus_relabeler_test.go +++ b/pp/go/cppbridge/prometheus_relabeler_test.go @@ -1466,4 +1466,7 @@ func TestStateV2Suite(t *testing.T) { } func (s *StateV2Suite) TestHappyPath() { + state := cppbridge.NewStateV2() + + s.T().Log(state.IsTransition()) } diff --git a/pp/go/relabeler/head/wal.go b/pp/go/relabeler/head/wal.go index de982cf5b5..9f6499048a 100644 --- a/pp/go/relabeler/head/wal.go +++ b/pp/go/relabeler/head/wal.go @@ -53,7 +53,7 @@ func (w *ShardWal) Write(innerSeriesSlice []*cppbridge.InnerSeries) (bool, error return false, fmt.Errorf("writing in corrupted wal") } - stats, err := w.encoder.Encode(innerSeriesSlice) + samples, err := w.encoder.Encode(innerSeriesSlice) if err != nil { return false, fmt.Errorf("failed to encode inner series: %w", err) } @@ -63,7 +63,7 @@ func (w *ShardWal) Write(innerSeriesSlice []*cppbridge.InnerSeries) (bool, error } // memoize reaching of limits to deduplicate triggers - if !w.limitExhausted && stats.Samples() >= w.maxSegmentSize { + if !w.limitExhausted && samples >= w.maxSegmentSize { w.limitExhausted = true return true, nil } @@ -164,8 +164,8 @@ func ReadHeader(reader io.Reader) (fileFormatVersion uint8, encoderVersion uint8 type EncodedSegment interface { Size() int64 CRC32() uint32 + Samples() uint32 io.WriterTo - cppbridge.SegmentStats } func WriteSegment(writer io.Writer, segment EncodedSegment) (n int, err error) { diff --git a/pp/go/storage/block/block.go b/pp/go/storage/block/block.go index 3003692fd1..7987fbcf0d 100644 --- a/pp/go/storage/block/block.go +++ b/pp/go/storage/block/block.go @@ -157,3 +157,8 @@ func (iw *IndexWriter) WriteSeriesTo(id uint32, chunks []ChunkMetadata, w io.Wri return n, nil } + +// isEmpty returns true if [IndexWriter] contains no samples, an empty block. +func (iw *IndexWriter) isEmpty() bool { + return !iw.isPrefixWritten +} diff --git a/pp/go/storage/block/block_writer.go b/pp/go/storage/block/block_writer.go index 73bd9513aa..b89860f7b4 100644 --- a/pp/go/storage/block/block_writer.go +++ b/pp/go/storage/block/block_writer.go @@ -87,6 +87,11 @@ func newBlockWriter( return writer, err } +// isEmpty returns true if [IndexWriter] contains no samples, an empty block. +func (writer *blockWriter) isEmpty() bool { + return writer.indexWriter.isEmpty() +} + func (writer *blockWriter) createWriters(maxBlockChunkSegmentSize int64) error { chunkWriter, err := NewChunkWriter(writer.ChunkDir(), maxBlockChunkSegmentSize) if err != nil { @@ -189,6 +194,15 @@ func (bw *blockWriters) writeRestOfRecodedChunks() error { func (bw *blockWriters) writeIndexAndMoveTmpDirToDir() ([]WrittenBlock, error) { writtenBlocks := make([]WrittenBlock, 0, len(*bw)) for i := range *bw { + if (*bw)[i].isEmpty() { + _ = (*bw)[i].close() + if err := os.RemoveAll((*bw)[i].Dir); err != nil { + logger.Warnf("failed remove empty block: %s", (*bw)[i].Dir) + } + + continue + } + if err := (*bw)[i].writeIndex(); err != nil { return nil, err } diff --git a/pp/go/storage/builder.go b/pp/go/storage/builder.go index 04d8cd8ef4..37bff84189 100644 --- a/pp/go/storage/builder.go +++ b/pp/go/storage/builder.go @@ -134,7 +134,7 @@ func (b *Builder) createShardOnDisk( return nil, fmt.Errorf("failed to write header: %w", err) } - sw, err := writer.NewBuffered(shardID, shardFile, writer.WriteSegment[*cppbridge.EncodedSegment], swn) + sw, err := writer.NewBuffered(shardID, shardFile, writer.WriteSegment[*cppbridge.HeadEncodedSegment], swn) if err != nil { return nil, fmt.Errorf("failed to create buffered writer shard id %d: %w", shardID, err) } diff --git a/pp/go/storage/head/keeper/keeper.go b/pp/go/storage/head/keeper/keeper.go index 32c7af1570..5665777809 100644 --- a/pp/go/storage/head/keeper/keeper.go +++ b/pp/go/storage/head/keeper/keeper.go @@ -52,6 +52,10 @@ func (q *headSortedSlice[THead]) Pop() any { return item } +// +// Head +// + // Head the minimum required [Head] implementation for a [Keeper]. type Head[T any] interface { // ID returns id [Head]. @@ -64,16 +68,22 @@ type Head[T any] interface { *T } +// +// Keeper +// + // Keeper holds outdated heads until conversion. type Keeper[T any, THead Head[T]] struct { - heads headSortedSlice[THead] - lock sync.RWMutex + heads headSortedSlice[THead] + addTrigger func() + lock sync.RWMutex } // NewKeeper init new [Keeper]. -func NewKeeper[T any, THead Head[T]](queueSize int) *Keeper[T, THead] { +func NewKeeper[T any, THead Head[T]](queueSize int, addTrigger func()) *Keeper[T, THead] { return &Keeper[T, THead]{ - heads: make(headSortedSlice[THead], 0, max(queueSize, MinHeadConvertingQueueSize)), + heads: make(headSortedSlice[THead], 0, max(queueSize, MinHeadConvertingQueueSize)), + addTrigger: addTrigger, } } @@ -173,6 +183,7 @@ func (k *Keeper[T, THead]) Remove(headsForRemove []THead) { func (k *Keeper[T, THead]) addHead(head THead, createdAt time.Duration, policy addPolicy) error { if len(k.heads) < cap(k.heads) { heap.Push(&k.heads, sortableHead[THead]{head: head, createdAt: createdAt}) + k.addTrigger() return nil } @@ -181,6 +192,7 @@ func (k *Keeper[T, THead]) addHead(head THead, createdAt time.Duration, policy a k.heads[0].head = head k.heads[0].createdAt = createdAt heap.Fix(&k.heads, 0) + k.addTrigger() return nil } diff --git a/pp/go/storage/head/keeper/keeper_test.go b/pp/go/storage/head/keeper/keeper_test.go index 2bc1cb6b2a..6006c46327 100644 --- a/pp/go/storage/head/keeper/keeper_test.go +++ b/pp/go/storage/head/keeper/keeper_test.go @@ -35,7 +35,9 @@ func TestKeeperSuite(t *testing.T) { func (s *KeeperSuite) TestAdd() { // Arrange - s.keeper = NewKeeper[headForTest](2) + count := 0 + addTrigger := func() { count++ } + s.keeper = NewKeeper[headForTest](2, addTrigger) // Act _ = s.keeper.Add(newHeadForTest("d"), 4) @@ -47,12 +49,15 @@ func (s *KeeperSuite) TestAdd() { {head: newHeadForTest("c"), createdAt: 3}, {head: newHeadForTest("d"), createdAt: 4}, }, s.keeper.heads) + s.Equal(2, count) s.Equal(err, ErrorNoSlots) } func (s *KeeperSuite) TestAddWithReplaceNoReplace() { // Arrange - s.keeper = NewKeeper[headForTest](2) + count := 0 + addTrigger := func() { count++ } + s.keeper = NewKeeper[headForTest](2, addTrigger) // Act _ = s.keeper.Add(newHeadForTest("d"), 4) @@ -64,12 +69,15 @@ func (s *KeeperSuite) TestAddWithReplaceNoReplace() { {head: newHeadForTest("c"), createdAt: 3}, {head: newHeadForTest("d"), createdAt: 4}, }, s.keeper.heads) + s.Equal(2, count) s.Equal(err, ErrorNoSlots) } func (s *KeeperSuite) TestAddWithReplace() { // Arrange - s.keeper = NewKeeper[headForTest](2) + count := 0 + addTrigger := func() { count++ } + s.keeper = NewKeeper[headForTest](2, addTrigger) // Act _ = s.keeper.Add(newHeadForTest("d"), 4) @@ -81,6 +89,7 @@ func (s *KeeperSuite) TestAddWithReplace() { {head: newHeadForTest("b"), createdAt: 4}, {head: newHeadForTest("d"), createdAt: 4}, }, s.keeper.heads) + s.Equal(3, count) s.NoError(err) } @@ -88,7 +97,9 @@ func (s *KeeperSuite) TestRemove() { // Arrange const Slots = 5 - s.keeper = NewKeeper[headForTest](Slots) + count := 0 + addTrigger := func() { count++ } + s.keeper = NewKeeper[headForTest](Slots, addTrigger) _ = s.keeper.Add(newHeadForTest("a"), 1) _ = s.keeper.Add(newHeadForTest("b"), 2) _ = s.keeper.Add(newHeadForTest("c"), 3) @@ -103,5 +114,6 @@ func (s *KeeperSuite) TestRemove() { {head: newHeadForTest("b"), createdAt: 2}, {head: newHeadForTest("d"), createdAt: 4}, }, s.keeper.heads) + s.Equal(5, count) s.Equal(Slots, cap(s.keeper.heads)) } diff --git a/pp/go/storage/head/services/persistener.go b/pp/go/storage/head/services/persistener.go index 2cb957adfa..33ea8fcb22 100644 --- a/pp/go/storage/head/services/persistener.go +++ b/pp/go/storage/head/services/persistener.go @@ -100,6 +100,7 @@ func (p *Persistener[TTask, TShard, TGoShard, THeadBlockWriter, THead]) Persist( if record, err := p.catalog.Get(head.ID()); err != nil { logger.Errorf("[Persistener]: failed get head %s from catalog: %v", head.ID(), err) } else if record.Status() == catalog.StatusPersisted { + logger.Debugf("[Persistener]: persisted head %s is outdated", head.ID()) if p.persistedHeadIsOutdated(record.UpdatedAt()) { outdatedHeads = append(outdatedHeads, head) } @@ -109,6 +110,7 @@ func (p *Persistener[TTask, TShard, TGoShard, THeadBlockWriter, THead]) Persist( if p.HeadIsOutdated(head) { // the head is outdated and data on it is no longer required + logger.Debugf("[Persistener]: head %s is outdated", head.ID()) if _, err := p.catalog.SetStatus(head.ID(), catalog.StatusPersisted); err != nil { logger.Errorf("[Persistener]: set head status in catalog %s: %v", head.ID(), err) continue @@ -320,6 +322,10 @@ func (pg *PersistenerService[ continue } + if !pg.proxy.HasSlot() { + break + } + if !pg.loadAndAddHeadToKeeper(record) { break } diff --git a/pp/go/storage/head/services/persistener_test.go b/pp/go/storage/head/services/persistener_test.go index 90cc75e96b..0987c5f58d 100644 --- a/pp/go/storage/head/services/persistener_test.go +++ b/pp/go/storage/head/services/persistener_test.go @@ -51,7 +51,7 @@ func (s *GenericPersistenceSuite) SetupTest() { h := s.mustCreateHead() activeHeadContainer := container.NewWeighted(h) - hKeeper := keeper.NewKeeper[storage.HeadOnDisk](1) + hKeeper := keeper.NewKeeper[storage.HeadOnDisk](1, func() {}) s.proxy = proxy.NewProxy(activeHeadContainer, hKeeper, func(*storage.HeadOnDisk) error { return nil }) s.blockWriter = &mock.HeadBlockWriterMock[*storage.ShardOnDisk]{} s.writeNotifier = &mock.WriteNotifierMock{NotifyWrittenFunc: func() {}} @@ -104,10 +104,6 @@ type PersistenerSuite struct { ] } -// func (s *PersistenerSuite) SetupSuite() { -// s.GenericPersistenceSuite.SetupSuite() -// } - func (s *PersistenerSuite) SetupTest() { s.GenericPersistenceSuite.SetupTest() @@ -310,10 +306,6 @@ type PersistenerServiceSuite struct { ] } -// func (s *PersistenerServiceSuite) SetupSuite() { -// s.GenericPersistenceSuite.SetupSuite() -// } - func (s *PersistenerServiceSuite) SetupTest() { s.GenericPersistenceSuite.SetupTest() @@ -365,7 +357,7 @@ func (s *PersistenerServiceSuite) TestRemoveOutdatedHeadFromKeeper() { // Assert s.Empty(s.proxy.Heads()) - s.Equal(catalog.StatusRotated, record.Status()) + s.Equal(catalog.StatusPersisted, record.Status()) } func (s *PersistenerServiceSuite) TestLoadHeadsInKeeper() { diff --git a/pp/go/storage/head/shard/wal/wal.go b/pp/go/storage/head/shard/wal/wal.go index 5b8b2ed45e..ad0da4a1a7 100644 --- a/pp/go/storage/head/shard/wal/wal.go +++ b/pp/go/storage/head/shard/wal/wal.go @@ -10,7 +10,7 @@ import ( ) //go:generate -command moq go run github.com/matryer/moq --rm --skip-ensure --pkg wal_test --out -//go:generate moq wal_moq_test.go . SegmentWriter Encoder StatsSegment EncodedSegment +//go:generate moq wal_moq_test.go . SegmentWriter Encoder EncodedSegment const ( // FileFormatVersion wal file version. @@ -39,16 +39,11 @@ type SegmentWriter[TSegment EncodedSegment] interface { } // Encoder the minimum required Encoder implementation for a [Wal]. -type Encoder[TSegment EncodedSegment, TStats StatsSegment] interface { - Encode(innerSeriesSlice []*cppbridge.InnerSeries) (TStats, error) +type Encoder[TSegment EncodedSegment] interface { + Encode(innerSeriesSlice []*cppbridge.InnerSeries) (uint32, error) Finalize() (TSegment, error) } -// StatsSegment stats data for [Encoder]. -type StatsSegment interface { - Samples() uint32 -} - // EncodedSegment the minimum required Segment implementation for a [Wal]. type EncodedSegment interface { Size() int64 @@ -58,8 +53,8 @@ type EncodedSegment interface { } // Wal write-ahead log for [Shard]. -type Wal[TSegment EncodedSegment, TStats StatsSegment, TWriter SegmentWriter[TSegment]] struct { - encoder Encoder[TSegment, TStats] // *cppbridge.HeadWalEncoder +type Wal[TSegment EncodedSegment, TWriter SegmentWriter[TSegment]] struct { + encoder Encoder[TSegment] // *cppbridge.HeadWalEncoder segmentWriter TWriter locker sync.Mutex maxSegmentSize uint32 @@ -69,12 +64,12 @@ type Wal[TSegment EncodedSegment, TStats StatsSegment, TWriter SegmentWriter[TSe } // NewWal init new [Wal]. -func NewWal[TSegment EncodedSegment, TStats StatsSegment, TWriter SegmentWriter[TSegment]]( - encoder Encoder[TSegment, TStats], +func NewWal[TSegment EncodedSegment, TWriter SegmentWriter[TSegment]]( + encoder Encoder[TSegment], segmentWriter TWriter, maxSegmentSize uint32, -) *Wal[TSegment, TStats, TWriter] { - return &Wal[TSegment, TStats, TWriter]{ +) *Wal[TSegment, TWriter] { + return &Wal[TSegment, TWriter]{ encoder: encoder, segmentWriter: segmentWriter, locker: sync.Mutex{}, @@ -85,17 +80,16 @@ func NewWal[TSegment EncodedSegment, TStats StatsSegment, TWriter SegmentWriter[ // NewCorruptedWal init new corrupted [Wal]. func NewCorruptedWal[ TSegment EncodedSegment, - TStats StatsSegment, TWriter SegmentWriter[TSegment], -]() *Wal[TSegment, TStats, TWriter] { - return &Wal[TSegment, TStats, TWriter]{ +]() *Wal[TSegment, TWriter] { + return &Wal[TSegment, TWriter]{ locker: sync.Mutex{}, corrupted: true, } } // Close closes the wal segmentWriter. -func (w *Wal[TSegment, TStats, TWriter]) Close() error { +func (w *Wal[TSegment, TWriter]) Close() error { if w.corrupted { return nil } @@ -118,7 +112,7 @@ func (w *Wal[TSegment, TStats, TWriter]) Close() error { // Commit finalize segment from encoder and write to [SegmentWriter]. // It is necessary to lock the LSS for reading for the commit. -func (w *Wal[TSegment, TStats, TWriter]) Commit() error { +func (w *Wal[TSegment, TWriter]) Commit() error { if w.corrupted { return ErrWalIsCorrupted } @@ -140,7 +134,7 @@ func (w *Wal[TSegment, TStats, TWriter]) Commit() error { } // CurrentSize returns current wal size. -func (w *Wal[TSegment, TStats, TWriter]) CurrentSize() int64 { +func (w *Wal[TSegment, TWriter]) CurrentSize() int64 { if w.corrupted { return 0 } @@ -149,7 +143,7 @@ func (w *Wal[TSegment, TStats, TWriter]) CurrentSize() int64 { } // Flush wal [SegmentWriter], write all buffered data to storage. -func (w *Wal[TSegment, TStats, TWriter]) Flush() error { +func (w *Wal[TSegment, TWriter]) Flush() error { if w.corrupted { return ErrWalIsCorrupted } @@ -161,7 +155,7 @@ func (w *Wal[TSegment, TStats, TWriter]) Flush() error { } // Sync commits the current contents of the [SegmentWriter]. -func (w *Wal[TSegment, TStats, TWriter]) Sync() error { +func (w *Wal[TSegment, TWriter]) Sync() error { if w.corrupted { return ErrWalIsCorrupted } @@ -173,7 +167,7 @@ func (w *Wal[TSegment, TStats, TWriter]) Sync() error { } // Write the incoming inner series to wal encoder. -func (w *Wal[TSegment, TStats, TWriter]) Write(innerSeriesSlice []*cppbridge.InnerSeries) (bool, error) { +func (w *Wal[TSegment, TWriter]) Write(innerSeriesSlice []*cppbridge.InnerSeries) (bool, error) { if w.corrupted { return false, ErrWalIsCorrupted } @@ -181,7 +175,7 @@ func (w *Wal[TSegment, TStats, TWriter]) Write(innerSeriesSlice []*cppbridge.Inn w.locker.Lock() defer w.locker.Unlock() - stats, err := w.encoder.Encode(innerSeriesSlice) + samples, err := w.encoder.Encode(innerSeriesSlice) if err != nil { return false, fmt.Errorf("failed to encode inner series: %w", err) } @@ -191,7 +185,7 @@ func (w *Wal[TSegment, TStats, TWriter]) Write(innerSeriesSlice []*cppbridge.Inn } // memoize reaching of limits to deduplicate triggers - if !w.limitExhausted && stats.Samples() >= w.maxSegmentSize { + if !w.limitExhausted && samples >= w.maxSegmentSize { w.limitExhausted = true return true, nil } diff --git a/pp/go/storage/head/shard/wal/wal_moq_test.go b/pp/go/storage/head/shard/wal/wal_moq_test.go index 7012bb7246..c333066433 100644 --- a/pp/go/storage/head/shard/wal/wal_moq_test.go +++ b/pp/go/storage/head/shard/wal/wal_moq_test.go @@ -226,7 +226,7 @@ func (mock *SegmentWriterMock[TSegment]) WriteCalls() []struct { // // // make and configure a mocked wal.Encoder // mockedEncoder := &EncoderMock{ -// EncodeFunc: func(innerSeriesSlice []*cppbridge.InnerSeries) (TStats, error) { +// EncodeFunc: func(innerSeriesSlice []*cppbridge.InnerSeries) (uint32, error) { // panic("mock out the Encode method") // }, // FinalizeFunc: func() (TSegment, error) { @@ -238,9 +238,9 @@ func (mock *SegmentWriterMock[TSegment]) WriteCalls() []struct { // // and then make assertions. // // } -type EncoderMock[TSegment wal.EncodedSegment, TStats wal.StatsSegment] struct { +type EncoderMock[TSegment wal.EncodedSegment] struct { // EncodeFunc mocks the Encode method. - EncodeFunc func(innerSeriesSlice []*cppbridge.InnerSeries) (TStats, error) + EncodeFunc func(innerSeriesSlice []*cppbridge.InnerSeries) (uint32, error) // FinalizeFunc mocks the Finalize method. FinalizeFunc func() (TSegment, error) @@ -261,7 +261,7 @@ type EncoderMock[TSegment wal.EncodedSegment, TStats wal.StatsSegment] struct { } // Encode calls EncodeFunc. -func (mock *EncoderMock[TSegment, TStats]) Encode(innerSeriesSlice []*cppbridge.InnerSeries) (TStats, error) { +func (mock *EncoderMock[TSegment]) Encode(innerSeriesSlice []*cppbridge.InnerSeries) (uint32, error) { if mock.EncodeFunc == nil { panic("EncoderMock.EncodeFunc: method is nil but Encoder.Encode was just called") } @@ -280,7 +280,7 @@ func (mock *EncoderMock[TSegment, TStats]) Encode(innerSeriesSlice []*cppbridge. // Check the length with: // // len(mockedEncoder.EncodeCalls()) -func (mock *EncoderMock[TSegment, TStats]) EncodeCalls() []struct { +func (mock *EncoderMock[TSegment]) EncodeCalls() []struct { InnerSeriesSlice []*cppbridge.InnerSeries } { var calls []struct { @@ -293,7 +293,7 @@ func (mock *EncoderMock[TSegment, TStats]) EncodeCalls() []struct { } // Finalize calls FinalizeFunc. -func (mock *EncoderMock[TSegment, TStats]) Finalize() (TSegment, error) { +func (mock *EncoderMock[TSegment]) Finalize() (TSegment, error) { if mock.FinalizeFunc == nil { panic("EncoderMock.FinalizeFunc: method is nil but Encoder.Finalize was just called") } @@ -309,7 +309,7 @@ func (mock *EncoderMock[TSegment, TStats]) Finalize() (TSegment, error) { // Check the length with: // // len(mockedEncoder.FinalizeCalls()) -func (mock *EncoderMock[TSegment, TStats]) FinalizeCalls() []struct { +func (mock *EncoderMock[TSegment]) FinalizeCalls() []struct { } { var calls []struct { } @@ -319,61 +319,6 @@ func (mock *EncoderMock[TSegment, TStats]) FinalizeCalls() []struct { return calls } -// StatsSegmentMock is a mock implementation of wal.StatsSegment. -// -// func TestSomethingThatUsesStatsSegment(t *testing.T) { -// -// // make and configure a mocked wal.StatsSegment -// mockedStatsSegment := &StatsSegmentMock{ -// SamplesFunc: func() uint32 { -// panic("mock out the Samples method") -// }, -// } -// -// // use mockedStatsSegment in code that requires wal.StatsSegment -// // and then make assertions. -// -// } -type StatsSegmentMock struct { - // SamplesFunc mocks the Samples method. - SamplesFunc func() uint32 - - // calls tracks calls to the methods. - calls struct { - // Samples holds details about calls to the Samples method. - Samples []struct { - } - } - lockSamples sync.RWMutex -} - -// Samples calls SamplesFunc. -func (mock *StatsSegmentMock) Samples() uint32 { - if mock.SamplesFunc == nil { - panic("StatsSegmentMock.SamplesFunc: method is nil but StatsSegment.Samples was just called") - } - callInfo := struct { - }{} - mock.lockSamples.Lock() - mock.calls.Samples = append(mock.calls.Samples, callInfo) - mock.lockSamples.Unlock() - return mock.SamplesFunc() -} - -// SamplesCalls gets all the calls that were made to Samples. -// Check the length with: -// -// len(mockedStatsSegment.SamplesCalls()) -func (mock *StatsSegmentMock) SamplesCalls() []struct { -} { - var calls []struct { - } - mock.lockSamples.RLock() - calls = mock.calls.Samples - mock.lockSamples.RUnlock() - return calls -} - // EncodedSegmentMock is a mock implementation of wal.EncodedSegment. // // func TestSomethingThatUsesEncodedSegment(t *testing.T) { diff --git a/pp/go/storage/head/shard/wal/wal_test.go b/pp/go/storage/head/shard/wal/wal_test.go index 0f00fb2456..519d48e064 100644 --- a/pp/go/storage/head/shard/wal/wal_test.go +++ b/pp/go/storage/head/shard/wal/wal_test.go @@ -34,7 +34,7 @@ func TestXxx(t *testing.T) { _ = shardFile.Close() }() - sw, err := writer.NewBuffered(shardID, shardFile, writer.WriteSegment[*cppbridge.EncodedSegment], swn) + sw, err := writer.NewBuffered(shardID, shardFile, writer.WriteSegment[*cppbridge.HeadEncodedSegment], swn) require.NoError(t, err) shardWalEncoder := &cppbridge.HeadWalEncoder{} @@ -61,7 +61,7 @@ func TestWalSuite(t *testing.T) { func (s *WalSuite) TestCurrentSize() { expectedWalSize := int64(42) - enc := &EncoderMock[*EncodedSegmentMock, *StatsSegmentMock]{} + enc := &EncoderMock[*EncodedSegmentMock]{} segmentWriter := &SegmentWriterMock[*EncodedSegmentMock]{ CurrentSizeFunc: func() int64 { return expectedWalSize @@ -78,7 +78,7 @@ func (s *WalSuite) TestCurrentSize2() { maxSegmentSize := uint32(100) // enSegment := &EncodedSegmentMock{} // stats := &StatsSegmentMock{} - enc := &EncoderMock[*EncodedSegmentMock, *StatsSegmentMock]{} + enc := &EncoderMock[*EncodedSegmentMock]{} segmentWriter := &SegmentWriterMock[*EncodedSegmentMock]{} wl := wal.NewWal(enc, segmentWriter, maxSegmentSize) diff --git a/pp/go/storage/loader.go b/pp/go/storage/loader.go index 9dca1e2a98..253ed95738 100644 --- a/pp/go/storage/loader.go +++ b/pp/go/storage/loader.go @@ -214,9 +214,8 @@ func (l *ShardDataLoader) Load() error { lss: shard.NewLSS(), dataStorage: shard.NewDataStorage(), wal: wal.NewCorruptedWal[ - *cppbridge.EncodedSegment, - cppbridge.WALEncoderStats, - *writer.Buffered[*cppbridge.EncodedSegment], + *cppbridge.HeadEncodedSegment, + *writer.Buffered[*cppbridge.HeadEncodedSegment], ](), } @@ -303,7 +302,7 @@ func (l *ShardDataLoader) createShardWal(fileName string, walDecoder *cppbridge. sw, err := writer.NewBuffered( l.shardID, shardWalFile, - writer.WriteSegment[*cppbridge.EncodedSegment], + writer.WriteSegment[*cppbridge.HeadEncodedSegment], l.notifier, ) if err != nil { diff --git a/pp/go/storage/manager.go b/pp/go/storage/manager.go index 09d31f1c8a..41eddf082a 100644 --- a/pp/go/storage/manager.go +++ b/pp/go/storage/manager.go @@ -123,14 +123,14 @@ func (c *Config) SetNumberOfShards(numberOfShards uint16) bool { // Manager manages services for the work of the heads. type Manager struct { - g run.Group - closer *util.Closer - // proxy *proxy.Proxy[*HeadOnDisk] + g run.Group + closer *util.Closer proxy *Proxy cgogc *cppbridge.CGOGC cfg *Config rotatorMediator *mediator.Mediator mergerMediator *mediator.Mediator + isRunning bool } // NewManager init new [Manager]. @@ -180,14 +180,17 @@ func NewManager( return nil, errors.Join(fmt.Errorf("failed to set active status: %w", err), h.Close()) } - hKeeper := keeper.NewKeeper[HeadOnDisk](o.KeeperCapacity) + persistenerMediator := mediator.NewMediator( + mediator.NewConstantIntervalTimer(clock, defaultStartPersistnerInterval, DefaultPersistDuration), + ) + + hKeeper := keeper.NewKeeper[HeadOnDisk](o.KeeperCapacity, persistenerMediator.Trigger) m := &Manager{ g: run.Group{}, closer: util.NewCloser(), - // proxy: proxy.NewProxy(container.NewWeighted(h), hKeeper, services.CFSViaRange), - proxy: NewProxy(container.NewWeighted(h), hKeeper, services.CFSViaRange), - cgogc: cppbridge.NewCGOGC(r), - cfg: cfg, + proxy: NewProxy(container.NewWeighted(h), hKeeper, services.CFSViaRange), + cgogc: cppbridge.NewCGOGC(r), + cfg: cfg, rotatorMediator: mediator.NewMediator( mediator.NewRotateTimerWithSeed(clock, o.BlockDuration, o.Seed), ), @@ -196,7 +199,7 @@ func NewManager( ), } - m.initServices(o, hcatalog, builder, loader, triggerNotifier, readyNotifier, clock, r) + m.initServices(o, hcatalog, builder, loader, triggerNotifier, persistenerMediator, readyNotifier, clock, r) logger.Infof("[Head Manager] created") return m, nil @@ -251,6 +254,7 @@ func (m *Manager) initServices( builder *Builder, loader *Loader, triggerNotifier *ReloadBlocksTriggerNotifier, + persistenerMediator *mediator.Mediator, readyNotifier ready.Notifier, clock clockwork.Clock, r prometheus.Registerer, @@ -261,6 +265,7 @@ func (m *Manager) initServices( m.g.Add( func() error { readyNotifier.NotifyReady() + m.isRunning = true <-m.closer.Signal() return nil @@ -271,9 +276,6 @@ func (m *Manager) initServices( ) // Persistener - persistenerMediator := mediator.NewMediator( - mediator.NewConstantIntervalTimer(clock, defaultStartPersistnerInterval, DefaultPersistDuration), - ) m.g.Add( func() error { services.NewPersistenerService( @@ -379,6 +381,10 @@ func (m *Manager) initServices( } func (m *Manager) close() { + if !m.isRunning { + m.closer.Done() + } + select { case <-m.closer.Signal(): default: diff --git a/pp/go/storage/types.go b/pp/go/storage/types.go index 1f12b44065..79ac9eeb0f 100644 --- a/pp/go/storage/types.go +++ b/pp/go/storage/types.go @@ -11,9 +11,8 @@ import ( // WalOnDisk wal on disk. type WalOnDisk = wal.Wal[ - *cppbridge.EncodedSegment, - cppbridge.WALEncoderStats, - *writer.Buffered[*cppbridge.EncodedSegment], + *cppbridge.HeadEncodedSegment, + *writer.Buffered[*cppbridge.HeadEncodedSegment], ] // ShardOnDisk [shard.Shard]. diff --git a/pp/wal/encoder.h b/pp/wal/encoder.h index 20f6125eec..c601634610 100644 --- a/pp/wal/encoder.h +++ b/pp/wal/encoder.h @@ -17,13 +17,20 @@ class GenericEncoder { template void write_stats(Stats* stats) const { - size_t remaining_cap = std::numeric_limits::max(); - - stats->earliest_timestamp = encoder_.buffer().earliest_sample(); - stats->latest_timestamp = encoder_.buffer().latest_sample(); stats->samples = encoder_.buffer().samples_count(); - stats->series = encoder_.buffer().series_count(); - stats->remainder_size = std::min(remaining_cap, encoder_.remainder_size()); + + if constexpr (BareBones::concepts::has_earliest_timestamp) { + stats->earliest_timestamp = encoder_.buffer().earliest_sample(); + } + + if constexpr (BareBones::concepts::has_latest_timestamp) { + stats->latest_timestamp = encoder_.buffer().latest_sample(); + } + + if constexpr (BareBones::concepts::has_remainder_size) { + size_t remaining_cap = std::numeric_limits::max(); + stats->remainder_size = std::min(remaining_cap, encoder_.remainder_size()); + } if constexpr (BareBones::concepts::has_allocated_memory) { stats->allocated_memory = encoder_.allocated_memory(); From ca54456208f5731319dad5064f5fcb0f948698cc Mon Sep 17 00:00:00 2001 From: Alexandr Yudin Date: Tue, 30 Sep 2025 16:10:49 +0000 Subject: [PATCH 62/96] add notify to catalogGC --- cmd/prometheus/main.go | 18 +++++--- pp/go/storage/catalog/gc.go | 43 +++++++++++++------ pp/go/storage/head/keeper/keeper.go | 34 ++++++++++++--- pp/go/storage/head/keeper/keeper_test.go | 27 ++++++++++-- pp/go/storage/head/services/interface.go | 4 +- .../storage/head/services/mock/persistener.go | 42 +++++++++--------- pp/go/storage/head/services/persistener.go | 2 +- .../storage/head/services/persistener_test.go | 9 ++-- pp/go/storage/manager.go | 29 +++++++------ web/web_fuzzy_test.go | 3 +- 10 files changed, 139 insertions(+), 72 deletions(-) diff --git a/cmd/prometheus/main.go b/cmd/prometheus/main.go index 23f299002c..fdd138d692 100644 --- a/cmd/prometheus/main.go +++ b/cmd/prometheus/main.go @@ -723,7 +723,7 @@ func main() { pp_pkg_logger.InitLogHandler(log.With(logger, "component", "pp")) - reloadBlocksTriggerNotifier := pp_storage.NewReloadBlocksTriggerNotifier() + reloadBlocksTriggerNotifier := pp_storage.NewTriggerNotifier() cfg.tsdb.ReloadBlocksExternalTrigger = reloadBlocksTriggerNotifier dataDir, err := filepath.Abs(localStoragePath) @@ -761,12 +761,12 @@ func main() { os.Exit(1) } + removedHeadTriggerNotifier := pp_storage.NewTriggerNotifier() hManagerReadyNotifier := ready.NewNotifiableNotifier() hManager, err := pp_storage.NewManager( &pp_storage.Options{ - Seed: cfgFile.GlobalConfig.ExternalLabels.Hash(), - // BlockDuration: time.Duration(cfg.tsdb.MinBlockDuration), - BlockDuration: 7 * time.Minute, + Seed: cfgFile.GlobalConfig.ExternalLabels.Hash(), + BlockDuration: time.Duration(cfg.tsdb.MinBlockDuration), CommitInterval: time.Duration(cfg.WalCommitInterval), MaxRetentionPeriod: time.Duration(cfg.tsdb.RetentionDuration), HeadRetentionPeriod: time.Duration(cfg.HeadRetentionTimeout), @@ -778,6 +778,7 @@ func main() { clock, headCatalog, reloadBlocksTriggerNotifier, + removedHeadTriggerNotifier, hManagerReadyNotifier, prometheus.DefaultRegisterer, ) @@ -1167,7 +1168,14 @@ func main() { ).Add( remoteWriterReadyNotifier, ).Build() - opGC := catalog.NewGC(dataDir, headCatalog, clock, multiNotifiable, time.Duration(cfg.tsdb.RetentionDuration)) + opGC := catalog.NewGC( + dataDir, + headCatalog, + clock, + multiNotifiable, + removedHeadTriggerNotifier, + time.Duration(cfg.tsdb.RetentionDuration), + ) var g run.Group { diff --git a/pp/go/storage/catalog/gc.go b/pp/go/storage/catalog/gc.go index bf93394dcf..a947d237c1 100644 --- a/pp/go/storage/catalog/gc.go +++ b/pp/go/storage/catalog/gc.go @@ -34,19 +34,30 @@ type Notifiable interface { ReadyChan() <-chan struct{} } +// +// RemovedHeadNotifier +// + +// RemovedHeadNotifier notifies that the [Head] has been removed. +type RemovedHeadNotifier interface { + // Chan returns channel with notifications. + Chan() <-chan struct{} +} + // // GC // // GC garbage collector for old [Head]. type GC struct { - dataDir string - catalog HeadsCatalog - clock clockwork.Clock - readyNotifiable Notifiable - maxRetentionPeriod time.Duration - stop chan struct{} - stopped chan struct{} + dataDir string + catalog HeadsCatalog + clock clockwork.Clock + readyNotifiable Notifiable + removedHeadNotifier RemovedHeadNotifier + maxRetentionPeriod time.Duration + stop chan struct{} + stopped chan struct{} } // NewGC init new [GC]. @@ -55,16 +66,18 @@ func NewGC( catalog HeadsCatalog, clock clockwork.Clock, readyNotifiable Notifiable, + removedHeadNotifier RemovedHeadNotifier, maxRetentionPeriod time.Duration, ) *GC { return &GC{ - dataDir: dataDir, - catalog: catalog, - clock: clock, - readyNotifiable: readyNotifiable, - maxRetentionPeriod: maxRetentionPeriod, - stop: make(chan struct{}), - stopped: make(chan struct{}), + dataDir: dataDir, + catalog: catalog, + clock: clock, + readyNotifiable: readyNotifiable, + removedHeadNotifier: removedHeadNotifier, + maxRetentionPeriod: maxRetentionPeriod, + stop: make(chan struct{}), + stopped: make(chan struct{}), } } @@ -127,6 +140,8 @@ func (gc *GC) Run(ctx context.Context) error { return ctx.Err() case <-time.After(time.Minute): gc.Iterate() + case <-gc.removedHeadNotifier.Chan(): + gc.Iterate() case <-gc.stop: return errors.New("stopped") } diff --git a/pp/go/storage/head/keeper/keeper.go b/pp/go/storage/head/keeper/keeper.go index 5665777809..8932b0c8bc 100644 --- a/pp/go/storage/head/keeper/keeper.go +++ b/pp/go/storage/head/keeper/keeper.go @@ -22,6 +22,16 @@ const ( // ErrorNoSlots error when keeper has no slots. var ErrorNoSlots = errors.New("keeper has no slots") +// +// RemovedHeadNotifier +// + +// RemovedHeadNotifier sends a notify that the [Head] has been removed. +type RemovedHeadNotifier interface { + // Notify sends a notify that the [Head] has been removed. + Notify() +} + type sortableHead[THead any] struct { head THead createdAt time.Duration @@ -74,16 +84,22 @@ type Head[T any] interface { // Keeper holds outdated heads until conversion. type Keeper[T any, THead Head[T]] struct { - heads headSortedSlice[THead] - addTrigger func() - lock sync.RWMutex + heads headSortedSlice[THead] + addTrigger func() + removedHeadNotifier RemovedHeadNotifier + lock sync.RWMutex } // NewKeeper init new [Keeper]. -func NewKeeper[T any, THead Head[T]](queueSize int, addTrigger func()) *Keeper[T, THead] { +func NewKeeper[T any, THead Head[T]]( + keeperCapacity int, + addTrigger func(), + removedHeadNotifier RemovedHeadNotifier, +) *Keeper[T, THead] { return &Keeper[T, THead]{ - heads: make(headSortedSlice[THead], 0, max(queueSize, MinHeadConvertingQueueSize)), - addTrigger: addTrigger, + heads: make(headSortedSlice[THead], 0, max(keeperCapacity, MinHeadConvertingQueueSize)), + addTrigger: addTrigger, + removedHeadNotifier: removedHeadNotifier, } } @@ -172,12 +188,18 @@ func (k *Keeper[T, THead]) Remove(headsForRemove []THead) { k.setHeads(newHeads) k.lock.Unlock() + var shouldNotify bool for _, head := range headsMap { if head != nil { _ = head.Close() logger.Infof("[Keeper]: head %s persisted, closed and removed", head.ID()) + shouldNotify = true } } + + if shouldNotify { + k.removedHeadNotifier.Notify() + } } func (k *Keeper[T, THead]) addHead(head THead, createdAt time.Duration, policy addPolicy) error { diff --git a/pp/go/storage/head/keeper/keeper_test.go b/pp/go/storage/head/keeper/keeper_test.go index 6006c46327..8b0c672506 100644 --- a/pp/go/storage/head/keeper/keeper_test.go +++ b/pp/go/storage/head/keeper/keeper_test.go @@ -22,6 +22,20 @@ func (*headForTest) Close() error { return nil } +// +// testRemovedHeadNotifier +// + +// testRemovedHeadNotifier implementation [RemovedHeadNotifier]. +type testRemovedHeadNotifier struct { + count int +} + +// Notify implementation [RemovedHeadNotifier]. +func (n *testRemovedHeadNotifier) Notify() { + n.count++ +} + type sortedSlice = headSortedSlice[*headForTest] type KeeperSuite struct { @@ -37,7 +51,8 @@ func (s *KeeperSuite) TestAdd() { // Arrange count := 0 addTrigger := func() { count++ } - s.keeper = NewKeeper[headForTest](2, addTrigger) + removedHeadNotifier := &testRemovedHeadNotifier{} + s.keeper = NewKeeper[headForTest](2, addTrigger, removedHeadNotifier) // Act _ = s.keeper.Add(newHeadForTest("d"), 4) @@ -57,7 +72,8 @@ func (s *KeeperSuite) TestAddWithReplaceNoReplace() { // Arrange count := 0 addTrigger := func() { count++ } - s.keeper = NewKeeper[headForTest](2, addTrigger) + removedHeadNotifier := &testRemovedHeadNotifier{} + s.keeper = NewKeeper[headForTest](2, addTrigger, removedHeadNotifier) // Act _ = s.keeper.Add(newHeadForTest("d"), 4) @@ -77,7 +93,8 @@ func (s *KeeperSuite) TestAddWithReplace() { // Arrange count := 0 addTrigger := func() { count++ } - s.keeper = NewKeeper[headForTest](2, addTrigger) + removedHeadNotifier := &testRemovedHeadNotifier{} + s.keeper = NewKeeper[headForTest](2, addTrigger, removedHeadNotifier) // Act _ = s.keeper.Add(newHeadForTest("d"), 4) @@ -99,7 +116,8 @@ func (s *KeeperSuite) TestRemove() { count := 0 addTrigger := func() { count++ } - s.keeper = NewKeeper[headForTest](Slots, addTrigger) + removedHeadNotifier := &testRemovedHeadNotifier{} + s.keeper = NewKeeper[headForTest](Slots, addTrigger, removedHeadNotifier) _ = s.keeper.Add(newHeadForTest("a"), 1) _ = s.keeper.Add(newHeadForTest("b"), 2) _ = s.keeper.Add(newHeadForTest("c"), 3) @@ -116,4 +134,5 @@ func (s *KeeperSuite) TestRemove() { }, s.keeper.heads) s.Equal(5, count) s.Equal(Slots, cap(s.keeper.heads)) + s.Equal(1, removedHeadNotifier.count) } diff --git a/pp/go/storage/head/services/interface.go b/pp/go/storage/head/services/interface.go index 7c29f98442..46df84467b 100644 --- a/pp/go/storage/head/services/interface.go +++ b/pp/go/storage/head/services/interface.go @@ -226,8 +226,8 @@ type Task interface { // WriteNotifier sends a notify that the writing is completed. type WriteNotifier interface { - // NotifyWritten sends a notify that the writing is completed. - NotifyWritten() + // Notify sends a notify that the writing is completed. + Notify() } // diff --git a/pp/go/storage/head/services/mock/persistener.go b/pp/go/storage/head/services/mock/persistener.go index 219100712e..cf53f58912 100644 --- a/pp/go/storage/head/services/mock/persistener.go +++ b/pp/go/storage/head/services/mock/persistener.go @@ -77,8 +77,8 @@ func (mock *HeadBlockWriterMock[TShard]) WriteCalls() []struct { // // // make and configure a mocked services.WriteNotifier // mockedWriteNotifier := &WriteNotifierMock{ -// NotifyWrittenFunc: func() { -// panic("mock out the NotifyWritten method") +// NotifyFunc: func() { +// panic("mock out the Notify method") // }, // } // @@ -87,41 +87,41 @@ func (mock *HeadBlockWriterMock[TShard]) WriteCalls() []struct { // // } type WriteNotifierMock struct { - // NotifyWrittenFunc mocks the NotifyWritten method. - NotifyWrittenFunc func() + // NotifyFunc mocks the Notify method. + NotifyFunc func() // calls tracks calls to the methods. calls struct { - // NotifyWritten holds details about calls to the NotifyWritten method. - NotifyWritten []struct { + // Notify holds details about calls to the Notify method. + Notify []struct { } } - lockNotifyWritten sync.RWMutex + lockNotify sync.RWMutex } -// NotifyWritten calls NotifyWrittenFunc. -func (mock *WriteNotifierMock) NotifyWritten() { - if mock.NotifyWrittenFunc == nil { - panic("WriteNotifierMock.NotifyWrittenFunc: method is nil but WriteNotifier.NotifyWritten was just called") +// Notify calls NotifyFunc. +func (mock *WriteNotifierMock) Notify() { + if mock.NotifyFunc == nil { + panic("WriteNotifierMock.NotifyFunc: method is nil but WriteNotifier.Notify was just called") } callInfo := struct { }{} - mock.lockNotifyWritten.Lock() - mock.calls.NotifyWritten = append(mock.calls.NotifyWritten, callInfo) - mock.lockNotifyWritten.Unlock() - mock.NotifyWrittenFunc() + mock.lockNotify.Lock() + mock.calls.Notify = append(mock.calls.Notify, callInfo) + mock.lockNotify.Unlock() + mock.NotifyFunc() } -// NotifyWrittenCalls gets all the calls that were made to NotifyWritten. +// NotifyCalls gets all the calls that were made to Notify. // Check the length with: // -// len(mockedWriteNotifier.NotifyWrittenCalls()) -func (mock *WriteNotifierMock) NotifyWrittenCalls() []struct { +// len(mockedWriteNotifier.NotifyCalls()) +func (mock *WriteNotifierMock) NotifyCalls() []struct { } { var calls []struct { } - mock.lockNotifyWritten.RLock() - calls = mock.calls.NotifyWritten - mock.lockNotifyWritten.RUnlock() + mock.lockNotify.RLock() + calls = mock.calls.Notify + mock.lockNotify.RUnlock() return calls } diff --git a/pp/go/storage/head/services/persistener.go b/pp/go/storage/head/services/persistener.go index 33ea8fcb22..20e90819d1 100644 --- a/pp/go/storage/head/services/persistener.go +++ b/pp/go/storage/head/services/persistener.go @@ -142,7 +142,7 @@ func (p *Persistener[TTask, TShard, TGoShard, THeadBlockWriter, THead]) Persist( } if shouldNotify { - p.writeNotifier.NotifyWritten() + p.writeNotifier.Notify() } return outdatedHeads diff --git a/pp/go/storage/head/services/persistener_test.go b/pp/go/storage/head/services/persistener_test.go index 0987c5f58d..745d20b994 100644 --- a/pp/go/storage/head/services/persistener_test.go +++ b/pp/go/storage/head/services/persistener_test.go @@ -51,10 +51,11 @@ func (s *GenericPersistenceSuite) SetupTest() { h := s.mustCreateHead() activeHeadContainer := container.NewWeighted(h) - hKeeper := keeper.NewKeeper[storage.HeadOnDisk](1, func() {}) + removedHeadNotifier := &mock.WriteNotifierMock{NotifyFunc: func() {}} + hKeeper := keeper.NewKeeper[storage.HeadOnDisk](1, func() {}, removedHeadNotifier) s.proxy = proxy.NewProxy(activeHeadContainer, hKeeper, func(*storage.HeadOnDisk) error { return nil }) s.blockWriter = &mock.HeadBlockWriterMock[*storage.ShardOnDisk]{} - s.writeNotifier = &mock.WriteNotifierMock{NotifyWrittenFunc: func() {}} + s.writeNotifier = &mock.WriteNotifierMock{NotifyFunc: func() {}} } func (s *GenericPersistenceSuite) createDataDirectory() string { @@ -247,7 +248,7 @@ func (s *PersistenerSuite) TestPersistHeadSuccess() { // Assert s.Equal([]*storage.HeadOnDisk(nil), outdated) s.Len(s.blockWriter.WriteCalls(), 2) - s.Len(s.writeNotifier.NotifyWrittenCalls(), 1) + s.Len(s.writeNotifier.NotifyCalls(), 1) s.Require().NoError(err) s.Equal(catalog.StatusPersisted, record.Status()) } @@ -287,7 +288,7 @@ func (s *PersistenerSuite) TestPersistHeadErrorOnBlockWriterForSecondShard() { // Assert s.Equal([]*storage.HeadOnDisk(nil), outdated) s.Len(s.blockWriter.WriteCalls(), 2) - s.Empty(s.writeNotifier.NotifyWrittenCalls()) + s.Empty(s.writeNotifier.NotifyCalls()) s.Require().NoError(err) s.Equal(catalog.StatusNew, record.Status()) } diff --git a/pp/go/storage/manager.go b/pp/go/storage/manager.go index 41eddf082a..3fb7d7a903 100644 --- a/pp/go/storage/manager.go +++ b/pp/go/storage/manager.go @@ -140,7 +140,8 @@ func NewManager( o *Options, clock clockwork.Clock, hcatalog *catalog.Catalog, - triggerNotifier *ReloadBlocksTriggerNotifier, + reloadBlocksNotifier *TriggerNotifier, + removedHeadNotifier *TriggerNotifier, readyNotifier ready.Notifier, r prometheus.Registerer, ) (*Manager, error) { @@ -184,7 +185,7 @@ func NewManager( mediator.NewConstantIntervalTimer(clock, defaultStartPersistnerInterval, DefaultPersistDuration), ) - hKeeper := keeper.NewKeeper[HeadOnDisk](o.KeeperCapacity, persistenerMediator.Trigger) + hKeeper := keeper.NewKeeper[HeadOnDisk](o.KeeperCapacity, persistenerMediator.Trigger, removedHeadNotifier) m := &Manager{ g: run.Group{}, closer: util.NewCloser(), @@ -199,7 +200,7 @@ func NewManager( ), } - m.initServices(o, hcatalog, builder, loader, triggerNotifier, persistenerMediator, readyNotifier, clock, r) + m.initServices(o, hcatalog, builder, loader, reloadBlocksNotifier, persistenerMediator, readyNotifier, clock, r) logger.Infof("[Head Manager] created") return m, nil @@ -253,7 +254,7 @@ func (m *Manager) initServices( hcatalog *catalog.Catalog, builder *Builder, loader *Loader, - triggerNotifier *ReloadBlocksTriggerNotifier, + reloadBlocksTriggerNotifier *TriggerNotifier, persistenerMediator *mediator.Mediator, readyNotifier ready.Notifier, clock clockwork.Clock, @@ -288,7 +289,7 @@ func (m *Manager) initServices( o.BlockDuration, r, ), - triggerNotifier, + reloadBlocksTriggerNotifier, clock, persistenerMediator, o.MaxRetentionPeriod, @@ -424,26 +425,26 @@ func (hi *headInformer) SetRotatedStatus(headID string) error { } // -// ReloadBlocksTriggerNotifier +// TriggerNotifier // -// ReloadBlocksTriggerNotifier for notifications about the appearance of new blocks. -type ReloadBlocksTriggerNotifier struct { +// TriggerNotifier to receive notifications about new events. +type TriggerNotifier struct { c chan struct{} } -// NewReloadBlocksTriggerNotifier init new [ReloadBlocksTriggerNotifier]. -func NewReloadBlocksTriggerNotifier() *ReloadBlocksTriggerNotifier { - return &ReloadBlocksTriggerNotifier{c: make(chan struct{}, 1)} +// NewTriggerNotifier init new [TriggerNotifier]. +func NewTriggerNotifier() *TriggerNotifier { + return &TriggerNotifier{c: make(chan struct{}, 1)} } // Chan returns channel with notifications. -func (tn *ReloadBlocksTriggerNotifier) Chan() <-chan struct{} { +func (tn *TriggerNotifier) Chan() <-chan struct{} { return tn.c } -// NotifyWritten sends a notify that the writing is completed. -func (tn *ReloadBlocksTriggerNotifier) NotifyWritten() { +// Notify sends a notify that the writing is completed. +func (tn *TriggerNotifier) Notify() { select { case tn.c <- struct{}{}: default: diff --git a/web/web_fuzzy_test.go b/web/web_fuzzy_test.go index 2a2ac14da4..e007603714 100644 --- a/web/web_fuzzy_test.go +++ b/web/web_fuzzy_test.go @@ -163,7 +163,8 @@ func makeManager( }, clock, headCatalog, - pp_storage.NewReloadBlocksTriggerNotifier(), + pp_storage.NewTriggerNotifier(), + pp_storage.NewTriggerNotifier(), &mock.ReadyNotifierMock{NotifyReadyFunc: func() {}}, prometheus.DefaultRegisterer, ) From 003925180f84902f7d3e991417d0dd827de84e99 Mon Sep 17 00:00:00 2001 From: Alexandr Yudin Date: Tue, 30 Sep 2025 16:26:47 +0000 Subject: [PATCH 63/96] for save --- pp/go/storage/head/proxy/proxy.go | 265 +++++++++--------- .../storage/head/services/persistener_test.go | 9 +- pp/go/storage/types.go | 6 +- 3 files changed, 140 insertions(+), 140 deletions(-) diff --git a/pp/go/storage/head/proxy/proxy.go b/pp/go/storage/head/proxy/proxy.go index 9818e7b2b3..bcb290c7d6 100644 --- a/pp/go/storage/head/proxy/proxy.go +++ b/pp/go/storage/head/proxy/proxy.go @@ -1,134 +1,135 @@ package proxy -import ( - "context" - "errors" - "time" -) - -// Head the minimum required [Head] implementation for a proxy. -type Head interface { - // Close closes wals, query semaphore for the inability to get query and clear metrics. - Close() error -} - -// -// ActiveHeadContainer -// - -// ActiveHeadContainer container for active [Head], the minimum required [ActiveHeadContainer] implementation. -type ActiveHeadContainer[THead Head] interface { - // Close closes [ActiveHeadContainer] for the inability work with [Head]. - Close() error - - // Get the active [Head]. - Get() THead - - // Replace the active [Head] with a new [Head]. - Replace(ctx context.Context, newHead THead) error - - // With calls fn(h Head). - With(ctx context.Context, fn func(h THead) error) error -} - -// -// Keeper -// - -// Keeper holds outdated heads until conversion. -type Keeper[THead Head] interface { - // Add the [Head] to the [Keeper] if there is a free slot. - Add(head THead, createdAt time.Duration) error - - // AddWithReplace the [Head] to the [Keeper] with replace if the createdAt is earlier. - AddWithReplace(head THead, createdAt time.Duration) error - - // Close closes for the inability work with [Head]. - Close() error - - // HasSlot returns the tru if there is a slot in the [Keeper]. - HasSlot() bool - - // Heads returns a slice of the [Head]s stored in the [Keeper]. - Heads() []THead - - // Remove removes [Head]s from the [Keeper]. - Remove(headsForRemove []THead) -} - -// -// Proxy -// - -// Proxy it proxies requests to the active [Head] and the keeper of old [Head]s. -type Proxy[THead Head] struct { - activeHeadContainer ActiveHeadContainer[THead] - keeper Keeper[THead] - onClose func(h THead) error -} - -// NewProxy init new [Proxy]. -func NewProxy[THead Head]( - activeHeadContainer ActiveHeadContainer[THead], - keeper Keeper[THead], - onClose func(h THead) error, -) *Proxy[THead] { - return &Proxy[THead]{ - activeHeadContainer: activeHeadContainer, - keeper: keeper, - onClose: onClose, - } -} - -// Add the [Head] to the [Keeper] if there is a free slot. -func (p *Proxy[THead]) Add(head THead, createdAt time.Duration) error { - return p.keeper.Add(head, createdAt) -} - -// AddWithReplace the [Head] to the [Keeper] with replace if the createdAt is earlier. -func (p *Proxy[THead]) AddWithReplace(head THead, createdAt time.Duration) error { - return p.keeper.AddWithReplace(head, createdAt) -} - -// Close closes [ActiveHeadContainer] and [Keeper] for the inability work with [Head]. -func (p *Proxy[THead]) Close() error { - ahErr := p.activeHeadContainer.Close() - - h := p.activeHeadContainer.Get() - onCloseErr := p.onClose(h) - headCloseErr := h.Close() - - keeperErr := p.keeper.Close() - - return errors.Join(ahErr, onCloseErr, headCloseErr, keeperErr) -} - -// Get the active [Head]. -func (p *Proxy[THead]) Get() THead { - return p.activeHeadContainer.Get() -} - -// HasSlot returns the tru if there is a slot in the [Keeper]. -func (p *Proxy[THead]) HasSlot() bool { - return p.keeper.HasSlot() -} - -// Heads returns a slice of the [Head]s stored in the [Keeper]. -func (p *Proxy[THead]) Heads() []THead { - return p.keeper.Heads() -} - -// Remove removes [Head]s from the [Keeper]. -func (p *Proxy[THead]) Remove(headsForRemove []THead) { - p.keeper.Remove(headsForRemove) -} - -// Replace the active [Head] with a new [Head]. -func (p *Proxy[THead]) Replace(ctx context.Context, newHead THead) error { - return p.activeHeadContainer.Replace(ctx, newHead) -} - -// With calls fn(h Head) on active [Head]. -func (p *Proxy[THead]) With(ctx context.Context, fn func(h THead) error) error { - return p.activeHeadContainer.With(ctx, fn) -} +// TODO DELETE +// import ( +// "context" +// "errors" +// "time" +// ) + +// // Head the minimum required [Head] implementation for a proxy. +// type Head interface { +// // Close closes wals, query semaphore for the inability to get query and clear metrics. +// Close() error +// } + +// // +// // ActiveHeadContainer +// // + +// // ActiveHeadContainer container for active [Head], the minimum required [ActiveHeadContainer] implementation. +// type ActiveHeadContainer[THead Head] interface { +// // Close closes [ActiveHeadContainer] for the inability work with [Head]. +// Close() error + +// // Get the active [Head]. +// Get() THead + +// // Replace the active [Head] with a new [Head]. +// Replace(ctx context.Context, newHead THead) error + +// // With calls fn(h Head). +// With(ctx context.Context, fn func(h THead) error) error +// } + +// // +// // Keeper +// // + +// // Keeper holds outdated heads until conversion. +// type Keeper[THead Head] interface { +// // Add the [Head] to the [Keeper] if there is a free slot. +// Add(head THead, createdAt time.Duration) error + +// // AddWithReplace the [Head] to the [Keeper] with replace if the createdAt is earlier. +// AddWithReplace(head THead, createdAt time.Duration) error + +// // Close closes for the inability work with [Head]. +// Close() error + +// // HasSlot returns the tru if there is a slot in the [Keeper]. +// HasSlot() bool + +// // Heads returns a slice of the [Head]s stored in the [Keeper]. +// Heads() []THead + +// // Remove removes [Head]s from the [Keeper]. +// Remove(headsForRemove []THead) +// } + +// // +// // Proxy +// // + +// // Proxy it proxies requests to the active [Head] and the keeper of old [Head]s. +// type Proxy[THead Head] struct { +// activeHeadContainer ActiveHeadContainer[THead] +// keeper Keeper[THead] +// onClose func(h THead) error +// } + +// // NewProxy init new [Proxy]. +// func NewProxy[THead Head]( +// activeHeadContainer ActiveHeadContainer[THead], +// keeper Keeper[THead], +// onClose func(h THead) error, +// ) *Proxy[THead] { +// return &Proxy[THead]{ +// activeHeadContainer: activeHeadContainer, +// keeper: keeper, +// onClose: onClose, +// } +// } + +// // Add the [Head] to the [Keeper] if there is a free slot. +// func (p *Proxy[THead]) Add(head THead, createdAt time.Duration) error { +// return p.keeper.Add(head, createdAt) +// } + +// // AddWithReplace the [Head] to the [Keeper] with replace if the createdAt is earlier. +// func (p *Proxy[THead]) AddWithReplace(head THead, createdAt time.Duration) error { +// return p.keeper.AddWithReplace(head, createdAt) +// } + +// // Close closes [ActiveHeadContainer] and [Keeper] for the inability work with [Head]. +// func (p *Proxy[THead]) Close() error { +// ahErr := p.activeHeadContainer.Close() + +// h := p.activeHeadContainer.Get() +// onCloseErr := p.onClose(h) +// headCloseErr := h.Close() + +// keeperErr := p.keeper.Close() + +// return errors.Join(ahErr, onCloseErr, headCloseErr, keeperErr) +// } + +// // Get the active [Head]. +// func (p *Proxy[THead]) Get() THead { +// return p.activeHeadContainer.Get() +// } + +// // HasSlot returns the tru if there is a slot in the [Keeper]. +// func (p *Proxy[THead]) HasSlot() bool { +// return p.keeper.HasSlot() +// } + +// // Heads returns a slice of the [Head]s stored in the [Keeper]. +// func (p *Proxy[THead]) Heads() []THead { +// return p.keeper.Heads() +// } + +// // Remove removes [Head]s from the [Keeper]. +// func (p *Proxy[THead]) Remove(headsForRemove []THead) { +// p.keeper.Remove(headsForRemove) +// } + +// // Replace the active [Head] with a new [Head]. +// func (p *Proxy[THead]) Replace(ctx context.Context, newHead THead) error { +// return p.activeHeadContainer.Replace(ctx, newHead) +// } + +// // With calls fn(h Head) on active [Head]. +// func (p *Proxy[THead]) With(ctx context.Context, fn func(h THead) error) error { +// return p.activeHeadContainer.With(ctx, fn) +// } diff --git a/pp/go/storage/head/services/persistener_test.go b/pp/go/storage/head/services/persistener_test.go index 745d20b994..7e3c46fcfa 100644 --- a/pp/go/storage/head/services/persistener_test.go +++ b/pp/go/storage/head/services/persistener_test.go @@ -16,7 +16,6 @@ import ( "github.com/prometheus/prometheus/pp/go/storage/catalog" "github.com/prometheus/prometheus/pp/go/storage/head/container" "github.com/prometheus/prometheus/pp/go/storage/head/keeper" - "github.com/prometheus/prometheus/pp/go/storage/head/proxy" "github.com/prometheus/prometheus/pp/go/storage/head/services" "github.com/prometheus/prometheus/pp/go/storage/head/services/mock" "github.com/prometheus/prometheus/pp/go/storage/head/task" @@ -39,7 +38,7 @@ type GenericPersistenceSuite struct { dataDir string clock *clockwork.FakeClock catalog *catalog.Catalog - proxy *proxy.Proxy[*storage.HeadOnDisk] + proxy *storage.Proxy blockWriter *mock.HeadBlockWriterMock[*storage.ShardOnDisk] writeNotifier *mock.WriteNotifierMock } @@ -53,7 +52,7 @@ func (s *GenericPersistenceSuite) SetupTest() { activeHeadContainer := container.NewWeighted(h) removedHeadNotifier := &mock.WriteNotifierMock{NotifyFunc: func() {}} hKeeper := keeper.NewKeeper[storage.HeadOnDisk](1, func() {}, removedHeadNotifier) - s.proxy = proxy.NewProxy(activeHeadContainer, hKeeper, func(*storage.HeadOnDisk) error { return nil }) + s.proxy = storage.NewProxy(activeHeadContainer, hKeeper, func(*storage.HeadOnDisk) error { return nil }) s.blockWriter = &mock.HeadBlockWriterMock[*storage.ShardOnDisk]{} s.writeNotifier = &mock.WriteNotifierMock{NotifyFunc: func() {}} } @@ -302,7 +301,7 @@ type PersistenerServiceSuite struct { *storage.PerGoroutineShard, *mock.HeadBlockWriterMock[*storage.ShardOnDisk], *storage.HeadOnDisk, - *proxy.Proxy[*storage.HeadOnDisk], + *storage.Proxy, *storage.Loader, ] } @@ -317,7 +316,7 @@ func (s *PersistenerServiceSuite) SetupTest() { *storage.PerGoroutineShard, *mock.HeadBlockWriterMock[*storage.ShardOnDisk], *storage.HeadOnDisk, - *proxy.Proxy[*storage.HeadOnDisk], + *storage.Proxy, *storage.Loader, ]( s.proxy, diff --git a/pp/go/storage/types.go b/pp/go/storage/types.go index 79ac9eeb0f..5f7c18469a 100644 --- a/pp/go/storage/types.go +++ b/pp/go/storage/types.go @@ -3,7 +3,6 @@ package storage import ( "github.com/prometheus/prometheus/pp/go/cppbridge" "github.com/prometheus/prometheus/pp/go/storage/head/head" - "github.com/prometheus/prometheus/pp/go/storage/head/proxy" "github.com/prometheus/prometheus/pp/go/storage/head/shard" "github.com/prometheus/prometheus/pp/go/storage/head/shard/wal" "github.com/prometheus/prometheus/pp/go/storage/head/shard/wal/writer" @@ -24,5 +23,6 @@ type PerGoroutineShard = shard.PerGoroutineShard // HeadOnDisk [head.Head] with [ShardOnDisk]. type HeadOnDisk = head.Head[*ShardOnDisk, *PerGoroutineShard] -// ProxyHead [proxy.Proxy] for [HeadOnDisk]s. -type ProxyHead = proxy.Proxy[*HeadOnDisk] +// TODO DELETE +// // ProxyHead [proxy.Proxy] for [HeadOnDisk]s. +// type ProxyHead = proxy.Proxy[*HeadOnDisk] From 6ae6659642c276e47149d5fad2b678e76ddfe2b3 Mon Sep 17 00:00:00 2001 From: Vladimir Pustovalov Date: Tue, 30 Sep 2025 20:06:56 +0300 Subject: [PATCH 64/96] added unit test for skip empty block --- pp/go/storage/block/writer_test.go | 24 +++++++++++++++++++++++- 1 file changed, 23 insertions(+), 1 deletion(-) diff --git a/pp/go/storage/block/writer_test.go b/pp/go/storage/block/writer_test.go index cff4da6c95..f478748bea 100644 --- a/pp/go/storage/block/writer_test.go +++ b/pp/go/storage/block/writer_test.go @@ -27,6 +27,8 @@ const ( maxSegmentSize uint32 = 1024 unloadDataStorageInterval time.Duration = time.Second + + blockDuration = 2 * time.Hour ) type WriterSuite struct { @@ -46,7 +48,7 @@ func (s *WriterSuite) SetupTest() { s.blockWriter = block.NewWriter[*storage.ShardOnDisk]( s.dataDir, block.DefaultChunkSegmentSize, - 2*time.Hour, + blockDuration, prometheus.DefaultRegisterer, ) } @@ -276,3 +278,23 @@ func (s *WriterSuite) TestWriteWithDataUnloadingInBatches() { // Assert s.assertWrittenBlocks(blocks, err) } + +func (s *WriterSuite) TestSkipEmptyBlock() { + // Arrange + storagetest.MustAppendTimeSeries(&s.Suite, s.head, []storagetest.TimeSeries{ + { + Labels: labels.FromStrings("__name__", "value1"), + Samples: []cppbridge.Sample{ + {Timestamp: 0, Value: 0}, + {Timestamp: blockDuration.Milliseconds() * 2, Value: 1}, + }, + }, + }) + + // Act + blocks, err := s.blockWriter.Write(s.shard()) + + // Assert + s.Require().NoError(err) + s.Equal(2, len(blocks)) +} From d421a5a31c4fb12457acb165b6e39b5e4c96252f Mon Sep 17 00:00:00 2001 From: Alexander Yudin <57181751+u-veles-a@users.noreply.github.com> Date: Fri, 3 Oct 2025 12:21:22 +0300 Subject: [PATCH 65/96] Rebuild head container lss snapshot copy (#161) * add patch to btree * tracy headers update * lss copy benchmark * update benchmark * remove bool pointer comparator hack * count a total sum in benchmark * benchmark fix * fancy benchmark * snug reserve method fix * benchmark fix * relabler reserve method fix * entrypoint integration * fixes over tests * bitset copy fix + tests * MemoryControlBlockWithItemCount copy test * SliceControlBlock fix * DecodingTable reserve fix * tidy fix * tidy fix * review fixes * template name fix * remove entrypoint from benchmark * lss copy keeps label set order test * add go CopyAddedSeries * fix check new head * add trigger with reset timer * fix samples * fix test * add test state * fix linter --------- Co-authored-by: glebshigin Co-authored-by: Gleb Shigin --- cmd/prometheus/main.go | 48 +++--- pp/WORKSPACE | 8 +- pp/bare_bones/concepts.h | 25 +-- pp/bare_bones/memory.h | 40 +---- pp/bare_bones/snug_composite.h | 19 +-- pp/bare_bones/tests/bitset_tests.cpp | 102 ++++++++++++ pp/bare_bones/tests/memory_tests.cpp | 42 +++++ pp/entrypoint/head_wal.cpp | 2 +- pp/entrypoint/primitives_lss.cpp | 38 ++++- pp/entrypoint/primitives_lss.h | 35 ++++ pp/go/cppbridge/entrypoint.go | 42 +++++ pp/go/cppbridge/entrypoint.h | 35 ++++ pp/go/cppbridge/lss_snapshot.go | 9 ++ pp/go/cppbridge/primitives_lss.go | 28 ++++ pp/go/cppbridge/primitives_lss_test.go | 37 ++++- pp/go/cppbridge/prometheus_relabeler.go | 21 ++- pp/go/cppbridge/prometheus_relabeler_test.go | 28 +++- pp/go/storage/appender/appender.go | 11 +- pp/go/storage/head/head/head.go | 25 +++ pp/go/storage/head/keeper/keeper.go | 5 - pp/go/storage/head/keeper/keeper_test.go | 20 +-- .../storage/head/services/persistener_test.go | 2 +- pp/go/storage/head/services/rotator.go | 37 +++-- pp/go/storage/head/shard/lss.go | 9 +- pp/go/storage/head/shard/shard.go | 9 ++ pp/go/storage/loader_test.go | 3 + pp/go/storage/manager.go | 68 +++++--- pp/go/storage/mediator/mediator.go | 9 ++ pp/go/storage/mediator/mediator_test.go | 44 +++++ pp/go/storage/types.go | 4 - pp/performance_tests/benchmarks/BUILD | 11 ++ ...ueryable_encoding_bimap_copy_benchmark.cpp | 150 ++++++++++++++++++ pp/primitives/go_slice.h | 17 -- pp/primitives/snug_composites_filaments.h | 11 +- pp/prometheus/relabeler.h | 2 +- pp/series_index/queryable_encoding_bimap.h | 55 ++++--- .../tests/queryable_encoding_bimap_tests.cpp | 68 ++++++-- .../patches/parallel_hashmap/btree.h.patch | 65 ++++++++ pp/third_party/tracy.BUILD | 10 +- pp/wal/encoder.h | 12 +- 40 files changed, 976 insertions(+), 230 deletions(-) create mode 100644 pp/performance_tests/benchmarks/queryable_encoding_bimap_copy_benchmark.cpp create mode 100644 pp/third_party/patches/parallel_hashmap/btree.h.patch diff --git a/cmd/prometheus/main.go b/cmd/prometheus/main.go index fdd138d692..13fbd35ca3 100644 --- a/cmd/prometheus/main.go +++ b/cmd/prometheus/main.go @@ -57,20 +57,21 @@ import ( "k8s.io/klog" klogv2 "k8s.io/klog/v2" - pp_pkg_handler "github.com/prometheus/prometheus/pp-pkg/handler" - rwprocessor "github.com/prometheus/prometheus/pp-pkg/handler/processor" - pp_pkg_logger "github.com/prometheus/prometheus/pp-pkg/logger" // PP_CHANGES.md: rebuild on cpp - "github.com/prometheus/prometheus/pp-pkg/remote" // PP_CHANGES.md: rebuild on cpp - "github.com/prometheus/prometheus/pp-pkg/scrape" // PP_CHANGES.md: rebuild on cpp - pp_pkg_storage "github.com/prometheus/prometheus/pp-pkg/storage" // PP_CHANGES.md: rebuild on cpp - pp_pkg_tsdb "github.com/prometheus/prometheus/pp-pkg/tsdb" - - pp_storage "github.com/prometheus/prometheus/pp/go/storage" // PP_CHANGES.md: rebuild on cpp - "github.com/prometheus/prometheus/pp/go/storage/catalog" // PP_CHANGES.md: rebuild on cpp - "github.com/prometheus/prometheus/pp/go/storage/head/head" // PP_CHANGES.md: rebuild on cpp - "github.com/prometheus/prometheus/pp/go/storage/querier" - "github.com/prometheus/prometheus/pp/go/storage/ready" // PP_CHANGES.md: rebuild on cpp - "github.com/prometheus/prometheus/pp/go/storage/remotewriter" // PP_CHANGES.md: rebuild on cpp + pp_pkg_handler "github.com/prometheus/prometheus/pp-pkg/handler" // PP_CHANGES.md: rebuild on cpp + rwprocessor "github.com/prometheus/prometheus/pp-pkg/handler/processor" // PP_CHANGES.md: rebuild on cpp + pp_pkg_logger "github.com/prometheus/prometheus/pp-pkg/logger" // PP_CHANGES.md: rebuild on cpp + "github.com/prometheus/prometheus/pp-pkg/remote" // PP_CHANGES.md: rebuild on cpp + "github.com/prometheus/prometheus/pp-pkg/scrape" // PP_CHANGES.md: rebuild on cpp + pp_pkg_storage "github.com/prometheus/prometheus/pp-pkg/storage" // PP_CHANGES.md: rebuild on cpp + pp_pkg_tsdb "github.com/prometheus/prometheus/pp-pkg/tsdb" // PP_CHANGES.md: rebuild on cpp + + pp_storage "github.com/prometheus/prometheus/pp/go/storage" // PP_CHANGES.md: rebuild on cpp + "github.com/prometheus/prometheus/pp/go/storage/catalog" // PP_CHANGES.md: rebuild on cpp + "github.com/prometheus/prometheus/pp/go/storage/head/head" // PP_CHANGES.md: rebuild on cpp + "github.com/prometheus/prometheus/pp/go/storage/head/services" // PP_CHANGES.md: rebuild on cpp + "github.com/prometheus/prometheus/pp/go/storage/querier" // PP_CHANGES.md: rebuild on cpp + "github.com/prometheus/prometheus/pp/go/storage/ready" // PP_CHANGES.md: rebuild on cpp + "github.com/prometheus/prometheus/pp/go/storage/remotewriter" // PP_CHANGES.md: rebuild on cpp "github.com/prometheus/prometheus/config" "github.com/prometheus/prometheus/discovery" @@ -277,10 +278,10 @@ func (c *flagConfig) setFeatureListOptions(logger log.Logger) error { } func main() { - // if os.Getenv("DEBUG") != "" { - runtime.SetBlockProfileRate(20) - runtime.SetMutexProfileFraction(20) - // } + if os.Getenv("DEBUG") != "" { + runtime.SetBlockProfileRate(20) + runtime.SetMutexProfileFraction(20) + } var ( oldFlagRetentionDuration model.Duration @@ -742,11 +743,6 @@ func main() { level.Error(logger).Log("msg", "failed to create file log", "err", err) os.Exit(1) } - defer func() { - if err := fileLog.Close(); err != nil { - level.Error(logger).Log("msg", "failed to close file log", "err", err) - } - }() clock := clockwork.NewRealClock() headCatalog, err := catalog.New( @@ -1564,6 +1560,10 @@ func main() { } // PP_CHANGES.md: rebuild on cpp start the engine is really no longer in use before calling this to avoid + if err := fileLog.Close(); err != nil { + level.Error(logger).Log("msg", "failed to close file log", "err", err) + } + if err := queryEngine.Close(); err != nil { level.Warn(logger).Log("msg", "Closing query engine failed", "err", err) } @@ -2095,7 +2095,7 @@ func readPromPPFeatures(logger log.Logger) { fname, fvalue, _ := strings.Cut(feature, "=") switch strings.TrimSpace(fname) { case "head_copy_series_on_rotate": - pp_storage.CopySeriesOnRotate = true + services.CopySeriesOnRotate = true level.Info(logger).Log( "msg", "[FEATURE] Copying active series from current head to new head during rotation is enabled.", diff --git a/pp/WORKSPACE b/pp/WORKSPACE index b76ef933ff..6e6ee41b6c 100644 --- a/pp/WORKSPACE +++ b/pp/WORKSPACE @@ -1,8 +1,7 @@ workspace(name = "prompp") load("@bazel_tools//tools/build_defs/repo:git.bzl", "git_repository") -load("@bazel_tools//tools/build_defs/repo:http.bzl", "http_archive") -load("@bazel_tools//tools/build_defs/repo:http.bzl", "http_file") +load("@bazel_tools//tools/build_defs/repo:http.bzl", "http_archive", "http_file") http_archive( name = "bazel_skylib", @@ -41,7 +40,7 @@ git_repository( http_archive( name = "google_benchmark", patches = [ - "//third_party/patches/google_benchmark:0001-BUILD.bazel.patch" + "//third_party/patches/google_benchmark:0001-BUILD.bazel.patch", ], sha256 = "35a77f46cc782b16fac8d3b107fbfbb37dcd645f7c28eee19f3b8e0758b48994", strip_prefix = "benchmark-1.9.0/", @@ -82,6 +81,7 @@ http_archive( build_file = "//third_party:parallel_hashmap.BUILD", patches = [ "//third_party/patches/parallel_hashmap:0001-svacer_fixes.patch", + "//third_party/patches/parallel_hashmap:btree.h.patch", ], sha256 = "b61435437713e2d98ce2a5539a0bff7e6e9e6a6b9fe507dbf490a852b8c2904f", strip_prefix = "parallel-hashmap-1.35", @@ -195,8 +195,8 @@ http_archive( http_file( name = "fastfloat_header", downloaded_file_path = "fastfloat/fast_float.h", - url = "https://github.com/fastfloat/fast_float/releases/download/v8.0.0/fast_float.h", sha256 = "1335e82c61fda54476ecbd94b92356deebeb3f0122802c3f103ee528ac08624e", + url = "https://github.com/fastfloat/fast_float/releases/download/v8.0.0/fast_float.h", ) local_repository( diff --git a/pp/bare_bones/concepts.h b/pp/bare_bones/concepts.h index bb5d20be4f..605e0aac5d 100644 --- a/pp/bare_bones/concepts.h +++ b/pp/bare_bones/concepts.h @@ -15,23 +15,28 @@ concept has_allocated_memory = requires(const T& t) { }; template -concept has_earliest_timestamp = requires(const T& t) { - { t.earliest_timestamp() } -> std::convertible_to; +concept has_allocated_memory_field = requires(const T& t) { + { t.allocated_memory } -> std::convertible_to; }; template -concept has_latest_timestamp = requires(const T& t) { - { t.latest_timestamp() } -> std::convertible_to; +concept has_earliest_timestamp_field = requires(const T& t) { + { t.earliest_timestamp } -> std::convertible_to; }; template -concept has_series = requires(const T& t) { - { t.series() } -> std::convertible_to; +concept has_latest_timestamp_field = requires(const T& t) { + { t.latest_timestamp } -> std::convertible_to; }; template -concept has_remainder_size = requires(const T& t) { - { t.remainder_size() } -> std::convertible_to; +concept has_series_field = requires(const T& t) { + { t.series } -> std::convertible_to; +}; + +template +concept has_remainder_size_field = requires(const T& t) { + { t.remainder_size } -> std::convertible_to; }; template @@ -50,8 +55,8 @@ concept has_size = requires(const T& t) { }; template -concept has_reserve = requires(T& r) { - { r.reserve(size_t{}) }; +concept has_reserve = requires(T& r, uint32_t n) { + { r.reserve(n) } -> std::same_as; }; template diff --git a/pp/bare_bones/memory.h b/pp/bare_bones/memory.h index 4c55789f89..ca8642acc8 100644 --- a/pp/bare_bones/memory.h +++ b/pp/bare_bones/memory.h @@ -101,20 +101,6 @@ template struct MemoryControlBlock { using SizeType = uint32_t; - MemoryControlBlock() = default; - MemoryControlBlock(const MemoryControlBlock&) = delete; - MemoryControlBlock(MemoryControlBlock&& other) noexcept : data(std::exchange(other.data, nullptr)), data_size(std::exchange(other.data_size, 0)) {} - - MemoryControlBlock& operator=(const MemoryControlBlock&) = delete; - PROMPP_ALWAYS_INLINE MemoryControlBlock& operator=(MemoryControlBlock&& other) noexcept { - if (this != &other) [[likely]] { - data = std::exchange(other.data, nullptr); - data_size = std::exchange(other.data_size, 0); - } - - return *this; - } - T* data{}; SizeType data_size{}; }; @@ -123,22 +109,6 @@ template struct MemoryControlBlockWithItemCount { using SizeType = uint32_t; - MemoryControlBlockWithItemCount() = default; - MemoryControlBlockWithItemCount(const MemoryControlBlockWithItemCount&) = delete; - MemoryControlBlockWithItemCount(MemoryControlBlockWithItemCount&& other) noexcept - : data(std::exchange(other.data, nullptr)), data_size(std::exchange(other.data_size, 0)), items_count(std::exchange(other.items_count, 0)) {} - - MemoryControlBlockWithItemCount& operator=(const MemoryControlBlockWithItemCount&) = delete; - PROMPP_ALWAYS_INLINE MemoryControlBlockWithItemCount& operator=(MemoryControlBlockWithItemCount&& other) noexcept { - if (this != &other) [[likely]] { - data = std::exchange(other.data, nullptr); - data_size = std::exchange(other.data_size, 0); - items_count = std::exchange(other.items_count, 0); - } - - return *this; - } - T* data{}; SizeType data_size{}; SizeType items_count{}; @@ -152,7 +122,7 @@ class Memory : public GenericMemory, typename ControlBlo PROMPP_ALWAYS_INLINE Memory() noexcept = default; PROMPP_ALWAYS_INLINE Memory(const Memory& o) noexcept { copy(o); } - PROMPP_ALWAYS_INLINE Memory(Memory&& o) noexcept = default; + PROMPP_ALWAYS_INLINE Memory(Memory&& o) noexcept : control_block_(std::exchange(o.control_block_, {})) {}; PROMPP_ALWAYS_INLINE ~Memory() noexcept { std::free(control_block_.data); } PROMPP_ALWAYS_INLINE Memory& operator=(const Memory& o) noexcept { @@ -166,7 +136,7 @@ class Memory : public GenericMemory, typename ControlBlo PROMPP_ALWAYS_INLINE Memory& operator=(Memory&& o) noexcept { if (this != &o) [[likely]] { std::free(control_block_.data); - control_block_ = std::move(o.control_block_); + control_block_ = std::exchange(o.control_block_, {}); } return *this; @@ -202,7 +172,11 @@ class Memory : public GenericMemory, typename ControlBlo PROMPP_ALWAYS_INLINE void copy(const Memory& o) noexcept { static_assert(IsTriviallyCopyable::value, "it's not allowed to copy memory for non trivially copyable types"); - resize(o.control_block_.data_size); + T* data = control_block_.data; + control_block_ = o.control_block_; + control_block_.data = data; + + resize(control_block_.data_size); PRAGMA_DIAGNOSTIC(push) PRAGMA_DIAGNOSTIC(ignored DIAGNOSTIC_CLASS_MEMACCESS) diff --git a/pp/bare_bones/snug_composite.h b/pp/bare_bones/snug_composite.h index b2d2603ce7..1056b65155 100644 --- a/pp/bare_bones/snug_composite.h +++ b/pp/bare_bones/snug_composite.h @@ -110,14 +110,9 @@ class GenericDecodingTable { struct LessComparator { using is_transparent = void; - PROMPP_ALWAYS_INLINE explicit LessComparator(const GenericDecodingTable* decoding_table, bool* enabled) noexcept - : decoding_table_(decoding_table), enabled_(enabled) {} + PROMPP_ALWAYS_INLINE explicit LessComparator(const GenericDecodingTable* decoding_table) noexcept : decoding_table_(decoding_table) {} PROMPP_ALWAYS_INLINE bool operator()(const Proxy& a, const Proxy& b) const noexcept { - if (!*enabled_) { - return true; - } - return decoding_table_->items_[a].composite(decoding_table_->data_) < decoding_table_->items_[b].composite(decoding_table_->data_); } @@ -134,7 +129,6 @@ class GenericDecodingTable { private: const GenericDecodingTable* decoding_table_; - bool* enabled_; }; template @@ -263,7 +257,7 @@ class GenericDecodingTable { [[nodiscard]] PROMPP_ALWAYS_INLINE Hasher hasher() const noexcept { return Hasher(this); } [[nodiscard]] PROMPP_ALWAYS_INLINE EqualityComparator equality_comparator() const noexcept { return EqualityComparator(this); } - [[nodiscard]] PROMPP_ALWAYS_INLINE LessComparator less_comparator(bool* enabled) const noexcept { return LessComparator(this, enabled); } + [[nodiscard]] PROMPP_ALWAYS_INLINE LessComparator less_comparator() const noexcept { return LessComparator(this); } data_type data_; Vector> items_; @@ -293,7 +287,8 @@ class GenericDecodingTable { inline __attribute__((always_inline)) const auto& items() const noexcept { return items_; } - PROMPP_ALWAYS_INLINE void reserve(const GenericDecodingTable& other) { + template class> class FilamentOther, template class VectorOther> + PROMPP_ALWAYS_INLINE void reserve(const GenericDecodingTable& other) { items_.reserve(other.items_.size()); data_.reserve(other.data_); } @@ -791,7 +786,6 @@ class OrderedEncodingBimap : public GenericDecodingTable; Set set_; - bool set_soring_enabled_{true}; protected: PROMPP_ALWAYS_INLINE void after_items_load_impl(uint32_t first_loaded_id) noexcept { @@ -801,7 +795,7 @@ class OrderedEncodingBimap : public GenericDecodingTable; OrderedSet ordered_set_; - bool ordered_set_soring_enabled_{true}; using Set = phmap::flat_hash_set; Set set_; @@ -912,7 +905,7 @@ class EncodingBimapWithOrderedAccess : public GenericDecodingTable{1, 100, 1000})); +} + +TEST_F(BitsetConstructorsFixture, CopyAssignment) { + // Arrange + bs_.resize(1001); + + bs_.set(1); + bs_.set(100); + bs_.set(1000); + + // Act + BareBones::Bitset bs_copy = bs_; + + // Assert + EXPECT_TRUE(std::ranges::equal(bs_, bs_copy)); +} + +TEST_F(BitsetConstructorsFixture, CopyAssignmentNonEmpty) { + // Arrange + bs_.resize(1001); + + bs_.set(1); + bs_.set(100); + bs_.set(1000); + + BareBones::Bitset bs_copy; + bs_copy.resize(3); + bs_copy.set(0); + bs_copy.set(1); + bs_copy.set(2); + + // Act + bs_copy = bs_; + + // Assert + EXPECT_TRUE(std::ranges::equal(bs_, bs_copy)); +} + +TEST_F(BitsetConstructorsFixture, MoveAssignment) { + // Arrange + bs_.resize(1001); + + bs_.set(1); + bs_.set(100); + bs_.set(1000); + + // Act + BareBones::Bitset bs_move = std::move(bs_); + + // Assert + EXPECT_TRUE(std::ranges::equal(bs_move, std::initializer_list{1, 100, 1000})); +} + +TEST_F(BitsetConstructorsFixture, MoveAssignmentNonEmpty) { + // Arrange + bs_.resize(1001); + + bs_.set(1); + bs_.set(100); + bs_.set(1000); + + BareBones::Bitset bs_move; + bs_move.resize(3); + bs_move.set(0); + bs_move.set(1); + bs_move.set(2); + + // Act + bs_move = std::move(bs_); + + // Assert + EXPECT_TRUE(std::ranges::equal(bs_move, std::initializer_list{1, 100, 1000})); +} + } // namespace diff --git a/pp/bare_bones/tests/memory_tests.cpp b/pp/bare_bones/tests/memory_tests.cpp index 0c07a1b5e3..7a08654ebb 100644 --- a/pp/bare_bones/tests/memory_tests.cpp +++ b/pp/bare_bones/tests/memory_tests.cpp @@ -10,6 +10,7 @@ using BareBones::AllocationSizeCalculator; using BareBones::DefaultReallocator; using BareBones::Memory; using BareBones::MemoryControlBlock; +using BareBones::MemoryControlBlockWithItemCount; using BareBones::SharedMemory; using BareBones::SharedPtr; @@ -136,6 +137,47 @@ TEST_F(MemoryFixture, MoveOperator) { EXPECT_EQ(memory_size, memory2.size()); } +class MemoryWithItemCountFixture : public ::testing::Test { + protected: + Memory memory_; +}; + +TEST_F(MemoryWithItemCountFixture, CopyConstructor) { + // Arrange + memory_.resize_to_fit_at_least(1); + std::iota(memory_.begin(), memory_.end(), uint8_t{}); + memory_.control_block().items_count = 100; + + // Act + const auto memory2 = memory_; + + // Assert + EXPECT_NE(memory2, memory_); + ASSERT_EQ(memory_.size(), memory2.size()); + ASSERT_EQ(memory_.control_block().data_size, memory2.control_block().data_size); + ASSERT_EQ(memory_.control_block().items_count, memory2.control_block().items_count); + EXPECT_TRUE(std::ranges::equal(memory2, memory_)); +} + +TEST_F(MemoryWithItemCountFixture, CopyOperator) { + // Arrange + memory_.resize_to_fit_at_least(1); + std::iota(memory_.begin(), memory_.end(), uint8_t{}); + memory_.control_block().items_count = 100; + + // Act + decltype(memory_) memory2; + memory2.resize_to_fit_at_least(1); + memory2 = memory_; + + // Assert + EXPECT_NE(memory2, memory_); + ASSERT_EQ(memory_.size(), memory2.size()); + ASSERT_EQ(memory_.control_block().data_size, memory2.control_block().data_size); + ASSERT_EQ(memory_.control_block().items_count, memory2.control_block().items_count); + EXPECT_TRUE(std::ranges::equal(memory2, memory_)); +} + class SharedPtrFixture : public ::testing::Test { protected: template diff --git a/pp/entrypoint/head_wal.cpp b/pp/entrypoint/head_wal.cpp index 11524871ba..36b499e6a6 100644 --- a/pp/entrypoint/head_wal.cpp +++ b/pp/entrypoint/head_wal.cpp @@ -67,8 +67,8 @@ extern "C" void prompp_head_wal_encoder_add_inner_series(void* args, void* res) }; struct Result { - uint32_t samples; PromPP::Primitives::Go::Slice error; + uint32_t samples; }; const auto in = static_cast(args); diff --git a/pp/entrypoint/primitives_lss.cpp b/pp/entrypoint/primitives_lss.cpp index b09f23d11e..f6e62d22b8 100644 --- a/pp/entrypoint/primitives_lss.cpp +++ b/pp/entrypoint/primitives_lss.cpp @@ -251,7 +251,41 @@ extern "C" void prompp_create_readonly_lss(void* args, void* res) { } extern "C" void prompp_primitives_lss_copy_added_series(uint64_t source_lss, uint64_t destination_lss) { - series_index::QueryableEncodingBimapCopier copier(std::get(*std::bit_cast(source_lss)), - std::get(*std::bit_cast(destination_lss))); + auto& src = std::get(*std::bit_cast(source_lss)); + auto& dst = std::get(*std::bit_cast(destination_lss)); + src.build_deferred_indexes(); + + series_index::QueryableEncodingBimapCopier copier(src, src.sorting_index(), src.added_series(), dst); + copier.copy_added_series_and_build_indexes(); +} + +using BitsetPtr = std::unique_ptr; + +extern "C" void prompp_primitives_lss_bitset_series(void* args, void* res) { + struct Arguments { + LssVariantPtr lss; + }; + struct Result { + BitsetPtr bitset; + }; + + const auto& src = std::get(*static_cast(args)->lss); + new (res) Result{.bitset = std::make_unique(src.added_series())}; +} + +extern "C" void prompp_primitives_lss_bitset_dtor(void* args) { + struct Arguments { + BitsetPtr bitset; + }; + + static_cast(args)->~Arguments(); +} + +extern "C" void prompp_primitives_readonly_lss_copy_added_series(uint64_t source_lss, uint64_t source_bitset, uint64_t destination_lss) { + const auto& src = std::get(*std::bit_cast(source_lss)); + const auto& src_bitset = *std::bit_cast(source_bitset); + auto& dst = std::get(*std::bit_cast(destination_lss)); + + series_index::QueryableEncodingBimapCopier copier(src, src.sorting_index(), src_bitset, dst); copier.copy_added_series_and_build_indexes(); } diff --git a/pp/entrypoint/primitives_lss.h b/pp/entrypoint/primitives_lss.h index 00852f975f..17d51954f5 100644 --- a/pp/entrypoint/primitives_lss.h +++ b/pp/entrypoint/primitives_lss.h @@ -194,6 +194,41 @@ void prompp_create_readonly_lss(void* args, void* res); */ void prompp_primitives_lss_copy_added_series(uint64_t source_lss, uint64_t destination_lss); +/** + * @brief returns a copy of the bitset of added series from the lss. + * + * @param args { + * lss uintptr // pointer to constructed queryable lss; + * } + * + * @param res { + * bitset uintptr // bitset of added series; + * } + */ +void prompp_primitives_lss_bitset_series(void* args, void* res); + +/** + * @brief destroy bitset of added series. + * + * @param args { + * bitset uintptr // bitset of added series; + * } + * + */ +void prompp_primitives_lss_bitset_dtor(void* args); + +/** + * @brief Copy the label sets from the source lss to the destination lss that were added source lss. + * + * @param source_lss pointer to source label sets; + * @param source_bitset pointer to source bitset; + * @param destination_lss pointer to destination label sets; + * + * @attention This binding used as a CGO call!!! + * + */ +void prompp_primitives_readonly_lss_copy_added_series(uint64_t source_lss, uint64_t source_bitset, uint64_t destination_lss); + #ifdef __cplusplus } // extern "C" #endif diff --git a/pp/go/cppbridge/entrypoint.go b/pp/go/cppbridge/entrypoint.go index 6bf981d913..216c26ff99 100644 --- a/pp/go/cppbridge/entrypoint.go +++ b/pp/go/cppbridge/entrypoint.go @@ -1527,6 +1527,48 @@ func primitivesLSSCreateReadonlyLss(lss uintptr) uintptr { return res.lss } +// primitivesLSSBitsetSeries returns a copy of the bitset of added series from the lss. +func primitivesLSSBitsetSeries(lss uintptr) uintptr { + args := struct { + lss uintptr + }{lss} + var res struct { + bitset uintptr + } + + testGC() + fastcgo.UnsafeCall2( + C.prompp_primitives_lss_bitset_series, + uintptr(unsafe.Pointer(&args)), + uintptr(unsafe.Pointer(&res)), + ) + + return res.bitset +} + +// primitivesLSSBitsetDtor destroy bitset of added series. +func primitivesLSSBitsetDtor(bitset uintptr) { + args := struct { + bitset uintptr + }{bitset} + + testGC() + fastcgo.UnsafeCall1( + C.prompp_primitives_lss_bitset_dtor, + uintptr(unsafe.Pointer(&args)), + ) +} + +// primitivesReadonlyLSSCopyAddedSeries copy the label sets from the source lss to the destination lss +// that were added source lss. +func primitivesReadonlyLSSCopyAddedSeries(source, sourceBitset, destination uintptr) { + C.prompp_primitives_readonly_lss_copy_added_series( + C.uint64_t(source), + C.uint64_t(sourceBitset), + C.uint64_t(destination), + ) +} + func primitivesLSSCopyAddedSeries(source, destination uintptr) { C.prompp_primitives_lss_copy_added_series(C.uint64_t(source), C.uint64_t(destination)) } diff --git a/pp/go/cppbridge/entrypoint.h b/pp/go/cppbridge/entrypoint.h index 3fd0ad8e6f..d699fd2a53 100755 --- a/pp/go/cppbridge/entrypoint.h +++ b/pp/go/cppbridge/entrypoint.h @@ -665,6 +665,41 @@ void prompp_create_readonly_lss(void* args, void* res); */ void prompp_primitives_lss_copy_added_series(uint64_t source_lss, uint64_t destination_lss); +/** + * @brief returns a copy of the bitset of added series from the lss. + * + * @param args { + * lss uintptr // pointer to constructed queryable lss; + * } + * + * @param res { + * bitset uintptr // bitset of added series; + * } + */ +void prompp_primitives_lss_bitset_series(void* args, void* res); + +/** + * @brief destroy bitset of added series. + * + * @param args { + * bitset uintptr // bitset of added series; + * } + * + */ +void prompp_primitives_lss_bitset_dtor(void* args); + +/** + * @brief Copy the label sets from the source lss to the destination lss that were added source lss. + * + * @param source_lss pointer to source label sets; + * @param source_bitset pointer to source bitset; + * @param destination_lss pointer to destination label sets; + * + * @attention This binding used as a CGO call!!! + * + */ +void prompp_primitives_readonly_lss_copy_added_series(uint64_t source_lss, uint64_t source_bitset, uint64_t destination_lss); + #ifdef __cplusplus } // extern "C" #endif diff --git a/pp/go/cppbridge/lss_snapshot.go b/pp/go/cppbridge/lss_snapshot.go index 9d7535accb..8b95d1aca0 100644 --- a/pp/go/cppbridge/lss_snapshot.go +++ b/pp/go/cppbridge/lss_snapshot.go @@ -76,3 +76,12 @@ func (lss *LabelSetSnapshot) Query(selector uintptr) *LSSQueryResult { runtime.KeepAlive(lss) return result } + +// CopyAddedSeries copy the label sets from the source lss to the destination lss +// that were added source lss. +func (lss *LabelSetSnapshot) CopyAddedSeries(bitsetSeries *BitsetSeries, destination *LabelSetStorage) { + primitivesReadonlyLSSCopyAddedSeries(lss.pointer, bitsetSeries.pointer, destination.pointer) + runtime.KeepAlive(lss) + runtime.KeepAlive(bitsetSeries) + runtime.KeepAlive(destination) +} diff --git a/pp/go/cppbridge/primitives_lss.go b/pp/go/cppbridge/primitives_lss.go index 1a5d6493a0..802dab6132 100644 --- a/pp/go/cppbridge/primitives_lss.go +++ b/pp/go/cppbridge/primitives_lss.go @@ -71,6 +71,14 @@ func (lss *LabelSetStorage) AllocatedMemory() uint64 { return res } +// BitsetSeries returns a copy of the bitset of added series from the lss. Read operation. +func (lss *LabelSetStorage) BitsetSeries() *BitsetSeries { + bsPointer := primitivesLSSBitsetSeries(lss.pointer) + runtime.KeepAlive(lss) + + return newBitsetSeriesFromPointer(bsPointer) +} + // FindOrEmplace find in lss LabelSet or emplace and return ls id. func (lss *LabelSetStorage) FindOrEmplace(labelSet model.LabelSet) FindOrEmplaceResult { res := primitivesLSSFindOrEmplace(lss.pointer, labelSet) @@ -291,3 +299,23 @@ type CppLabelSetBuilder struct { SortedAdd []Label SortedDel []string } + +// +// BitsetSeries +// + +// BitsetSeries copies of the bitset of added series from the lss. +type BitsetSeries struct { + pointer uintptr + gcDestroyDetector *uint64 +} + +// newBitsetSeriesFromPointer init new [BitsetSeries]. +func newBitsetSeriesFromPointer(bitsetSeriesPointer uintptr) *BitsetSeries { + bitsetSeries := &BitsetSeries{pointer: bitsetSeriesPointer, gcDestroyDetector: &gcDestroyDetector} + runtime.SetFinalizer(bitsetSeries, func(bs *BitsetSeries) { + primitivesLSSBitsetDtor(bs.pointer) + }) + + return bitsetSeries +} diff --git a/pp/go/cppbridge/primitives_lss_test.go b/pp/go/cppbridge/primitives_lss_test.go index 96d9736389..07918d903f 100644 --- a/pp/go/cppbridge/primitives_lss_test.go +++ b/pp/go/cppbridge/primitives_lss_test.go @@ -3,6 +3,7 @@ package cppbridge_test import ( "context" "runtime" + "sort" "testing" "github.com/prometheus/prometheus/pp/go/model" @@ -413,8 +414,15 @@ func (s *QueryableLSSSuite) TestCopyAddedSeries() { s.lss.CopyAddedSeries(lssCopy) lssCopy.CopyAddedSeries(lssCopyOfCopy) + // lssCopy will contain lexicographically sorted labels with new IDs. + expectedLabelSets := make([]model.LabelSet, len(s.labelSets)) + copy(expectedLabelSets, s.labelSets) + sort.Slice(expectedLabelSets, func(i, j int) bool { + return expectedLabelSets[i].String() < expectedLabelSets[j].String() + }) + // Assert - s.Equal(labelSetToCppBridgeLabels(s.labelSets), lssCopy.GetLabelSets(s.labelSetIDs).LabelsSets()) + s.Equal(labelSetToCppBridgeLabels(expectedLabelSets), lssCopy.GetLabelSets(s.labelSetIDs).LabelsSets()) s.Equal(emptyLabelsSets, lssCopyOfCopy.GetLabelSets(s.labelSetIDs).LabelsSets()) } @@ -478,3 +486,30 @@ func (s *QueryableLSSSuite) TestFindOrEmplaceBuilderWithoutReadonlyLss() { // Assert s.Equal(uint32(expectedLsId), existingLsId) } + +func (s *QueryableLSSSuite) TestCopyAddedSeriesFromSnapshot() { + // Arrange + emptyLabelsSets := make([]cppbridge.Labels, len(s.labelSetIDs)) + lssCopy := cppbridge.NewQueryableLssStorage() + lssCopyOfCopy := cppbridge.NewQueryableLssStorage() + + // Act + snapshot := s.lss.CreateLabelSetSnapshot() + bitsetSeries := s.lss.BitsetSeries() + snapshot.CopyAddedSeries(bitsetSeries, lssCopy) + + snapshotCopy := lssCopy.CreateLabelSetSnapshot() + bitsetSeriesCopy := lssCopy.BitsetSeries() + snapshotCopy.CopyAddedSeries(bitsetSeriesCopy, lssCopyOfCopy) + + // lssCopy will contain lexicographically sorted labels with new IDs. + expectedLabelSets := make([]model.LabelSet, len(s.labelSets)) + copy(expectedLabelSets, s.labelSets) + sort.Slice(expectedLabelSets, func(i, j int) bool { + return expectedLabelSets[i].String() < expectedLabelSets[j].String() + }) + + // Assert + s.Equal(labelSetToCppBridgeLabels(expectedLabelSets), lssCopy.GetLabelSets(s.labelSetIDs).LabelsSets()) + s.Equal(emptyLabelsSets, lssCopyOfCopy.GetLabelSets(s.labelSetIDs).LabelsSets()) +} diff --git a/pp/go/cppbridge/prometheus_relabeler.go b/pp/go/cppbridge/prometheus_relabeler.go index da4ae09eaa..c5448e6601 100644 --- a/pp/go/cppbridge/prometheus_relabeler.go +++ b/pp/go/cppbridge/prometheus_relabeler.go @@ -1562,6 +1562,10 @@ func NewStateV2WithoutLock() *StateV2 { // CacheByShard return *Cache for shard. func (s *StateV2) CacheByShard(shardID uint16) *Cache { + if s.IsTransition() { + panic("state is transition") + } + if int(shardID) >= len(s.caches) { panic(fmt.Sprintf( "shardID(%d) out of range in caches(%d)", @@ -1582,13 +1586,12 @@ func (s *StateV2) DefTimestamp() int64 { return s.defTimestamp } -// DisableTrackStaleness disable track stalenans. -func (s *StateV2) DisableTrackStaleness() { - s.trackStaleness = false -} - // EnableTrackStaleness enable track stalenans. func (s *StateV2) EnableTrackStaleness() { + if s.IsTransition() { + panic("state is transition") + } + s.trackStaleness = true } @@ -1657,6 +1660,10 @@ func (s *StateV2) SetStatelessRelabeler(statelessRelabeler *StatelessRelabeler) // StaleNansStateByShard return SourceStaleNansState for shard. func (s *StateV2) StaleNansStateByShard(shardID uint16) *StaleNansState { + if s.IsTransition() { + panic("state is transition") + } + if int(shardID) >= len(s.staleNansStates) { panic(fmt.Sprintf( "shardID(%d) out of range in staleNansStates(%d)", @@ -1674,6 +1681,10 @@ func (s *StateV2) StatelessRelabeler() *StatelessRelabeler { panic("state is transition") } + if s.statelessRelabeler == nil { + panic("statelessRelabeler is nil") + } + return s.statelessRelabeler } diff --git a/pp/go/cppbridge/prometheus_relabeler_test.go b/pp/go/cppbridge/prometheus_relabeler_test.go index 9ca9122d13..c83989f2cb 100644 --- a/pp/go/cppbridge/prometheus_relabeler_test.go +++ b/pp/go/cppbridge/prometheus_relabeler_test.go @@ -1465,8 +1465,32 @@ func TestStateV2Suite(t *testing.T) { suite.Run(t, new(StateV2Suite)) } -func (s *StateV2Suite) TestHappyPath() { +func (s *StateV2Suite) TestInitState() { state := cppbridge.NewStateV2() - s.T().Log(state.IsTransition()) + s.Panics(func() { state.CacheByShard(0) }) + s.Equal(time.Now().UnixMilli(), state.DefTimestamp()) + s.False(state.IsTransition()) + s.Equal(cppbridge.RelabelerOptions{}, state.RelabelerOptions()) + s.Panics(func() { state.StaleNansStateByShard(0) }) + s.False(state.TrackStaleness()) +} + +func (s *StateV2Suite) TestStateReconfigure() { + state := cppbridge.NewStateV2() + state.Reconfigure(0, 1) + + s.NotNil(state.CacheByShard(0)) + s.False(state.TrackStaleness()) + s.Panics(func() { state.StaleNansStateByShard(0) }) +} + +func (s *StateV2Suite) TestStateReconfigureTrackStaleness() { + state := cppbridge.NewStateV2() + state.EnableTrackStaleness() + state.Reconfigure(0, 1) + + s.NotNil(state.CacheByShard(0)) + s.True(state.TrackStaleness()) + s.NotNil(state.StaleNansStateByShard(0)) } diff --git a/pp/go/storage/appender/appender.go b/pp/go/storage/appender/appender.go index ab44a538e2..feb6a31339 100644 --- a/pp/go/storage/appender/appender.go +++ b/pp/go/storage/appender/appender.go @@ -195,8 +195,8 @@ func (a Appender[TTask, TShard, THead]) inputRelabelingStage( relabeler = shard.Relabeler() shardID = shard.ShardID() ok bool - innerSeries = shardedInnerSeries.DataBySourceShard(shardID) shardedData = incomingData.ShardedData() + innerSeries = shardedInnerSeries.DataBySourceShard(shardID) ) if err := shard.LSSWithRLock(func(target, input *cppbridge.LabelSetStorage) (rErr error) { @@ -223,7 +223,6 @@ func (a Appender[TTask, TShard, THead]) inputRelabelingStage( var ( hasReallocations bool rstats = cppbridge.RelabelerStats{} - relabeledSeries = shardedRelabeledSeries.DataByShard(shardID) ) err := shard.LSSWithLock(func(target, input *cppbridge.LabelSetStorage) (rErr error) { rstats, hasReallocations, rErr = relabeler.Relabeling( @@ -233,7 +232,7 @@ func (a Appender[TTask, TShard, THead]) inputRelabelingStage( state, shardedData, innerSeries, - relabeledSeries, + shardedRelabeledSeries.DataByShard(shardID), ) if hasReallocations { @@ -282,15 +281,13 @@ func (a Appender[TTask, TShard, THead]) appendRelabelerSeriesStage( return nil } - innerSeries := shardedInnerSeries.DataByShard(shardID) - stateUpdates := shardedStateUpdates.DataByShard(shardID) return shard.LSSWithLock(func(target, _ *cppbridge.LabelSetStorage) error { hasReallocations, err := shard.Relabeler().AppendRelabelerSeries( ctx, target, - innerSeries, + shardedInnerSeries.DataByShard(shardID), relabeledSeries, - stateUpdates, + shardedStateUpdates.DataByShard(shardID), ) if err != nil { return fmt.Errorf("shard %d: %w", shardID, err) diff --git a/pp/go/storage/head/head/head.go b/pp/go/storage/head/head/head.go index 8b89ce976d..aeed2ee3d6 100644 --- a/pp/go/storage/head/head/head.go +++ b/pp/go/storage/head/head/head.go @@ -291,3 +291,28 @@ func (h *Head[TShard, TGorutineShard]) shardLoop( func calculateHeadConcurrency(numberOfShards int) int64 { return int64(defaultNumberOfWorkers+ExtraWorkers) * int64(numberOfShards) } + +// +// CopyAddedSeries +// + +// CopyAddedSeries copy the label sets from the source lss to the destination lss that were added source lss. +func CopyAddedSeries[TShard Shard, TGorutineShard Shard]( + shardCopier func(source, destination TShard), +) func(source, destination *Head[TShard, TGorutineShard]) { + return func(source, destination *Head[TShard, TGorutineShard]) { + if source.NumberOfShards() != destination.NumberOfShards() { + logger.Warnf( + "source[%d] and destination[%d] number of shards must be the same", + source.NumberOfShards(), + destination.NumberOfShards(), + ) + + return + } + + for shardID := range source.NumberOfShards() { + shardCopier(source.shards[shardID], destination.shards[shardID]) + } + } +} diff --git a/pp/go/storage/head/keeper/keeper.go b/pp/go/storage/head/keeper/keeper.go index 8932b0c8bc..202eb4aff0 100644 --- a/pp/go/storage/head/keeper/keeper.go +++ b/pp/go/storage/head/keeper/keeper.go @@ -85,7 +85,6 @@ type Head[T any] interface { // Keeper holds outdated heads until conversion. type Keeper[T any, THead Head[T]] struct { heads headSortedSlice[THead] - addTrigger func() removedHeadNotifier RemovedHeadNotifier lock sync.RWMutex } @@ -93,12 +92,10 @@ type Keeper[T any, THead Head[T]] struct { // NewKeeper init new [Keeper]. func NewKeeper[T any, THead Head[T]]( keeperCapacity int, - addTrigger func(), removedHeadNotifier RemovedHeadNotifier, ) *Keeper[T, THead] { return &Keeper[T, THead]{ heads: make(headSortedSlice[THead], 0, max(keeperCapacity, MinHeadConvertingQueueSize)), - addTrigger: addTrigger, removedHeadNotifier: removedHeadNotifier, } } @@ -205,7 +202,6 @@ func (k *Keeper[T, THead]) Remove(headsForRemove []THead) { func (k *Keeper[T, THead]) addHead(head THead, createdAt time.Duration, policy addPolicy) error { if len(k.heads) < cap(k.heads) { heap.Push(&k.heads, sortableHead[THead]{head: head, createdAt: createdAt}) - k.addTrigger() return nil } @@ -214,7 +210,6 @@ func (k *Keeper[T, THead]) addHead(head THead, createdAt time.Duration, policy a k.heads[0].head = head k.heads[0].createdAt = createdAt heap.Fix(&k.heads, 0) - k.addTrigger() return nil } diff --git a/pp/go/storage/head/keeper/keeper_test.go b/pp/go/storage/head/keeper/keeper_test.go index 8b0c672506..47a800bfcb 100644 --- a/pp/go/storage/head/keeper/keeper_test.go +++ b/pp/go/storage/head/keeper/keeper_test.go @@ -49,10 +49,8 @@ func TestKeeperSuite(t *testing.T) { func (s *KeeperSuite) TestAdd() { // Arrange - count := 0 - addTrigger := func() { count++ } removedHeadNotifier := &testRemovedHeadNotifier{} - s.keeper = NewKeeper[headForTest](2, addTrigger, removedHeadNotifier) + s.keeper = NewKeeper[headForTest](2, removedHeadNotifier) // Act _ = s.keeper.Add(newHeadForTest("d"), 4) @@ -64,16 +62,13 @@ func (s *KeeperSuite) TestAdd() { {head: newHeadForTest("c"), createdAt: 3}, {head: newHeadForTest("d"), createdAt: 4}, }, s.keeper.heads) - s.Equal(2, count) s.Equal(err, ErrorNoSlots) } func (s *KeeperSuite) TestAddWithReplaceNoReplace() { // Arrange - count := 0 - addTrigger := func() { count++ } removedHeadNotifier := &testRemovedHeadNotifier{} - s.keeper = NewKeeper[headForTest](2, addTrigger, removedHeadNotifier) + s.keeper = NewKeeper[headForTest](2, removedHeadNotifier) // Act _ = s.keeper.Add(newHeadForTest("d"), 4) @@ -85,16 +80,13 @@ func (s *KeeperSuite) TestAddWithReplaceNoReplace() { {head: newHeadForTest("c"), createdAt: 3}, {head: newHeadForTest("d"), createdAt: 4}, }, s.keeper.heads) - s.Equal(2, count) s.Equal(err, ErrorNoSlots) } func (s *KeeperSuite) TestAddWithReplace() { // Arrange - count := 0 - addTrigger := func() { count++ } removedHeadNotifier := &testRemovedHeadNotifier{} - s.keeper = NewKeeper[headForTest](2, addTrigger, removedHeadNotifier) + s.keeper = NewKeeper[headForTest](2, removedHeadNotifier) // Act _ = s.keeper.Add(newHeadForTest("d"), 4) @@ -106,7 +98,6 @@ func (s *KeeperSuite) TestAddWithReplace() { {head: newHeadForTest("b"), createdAt: 4}, {head: newHeadForTest("d"), createdAt: 4}, }, s.keeper.heads) - s.Equal(3, count) s.NoError(err) } @@ -114,10 +105,8 @@ func (s *KeeperSuite) TestRemove() { // Arrange const Slots = 5 - count := 0 - addTrigger := func() { count++ } removedHeadNotifier := &testRemovedHeadNotifier{} - s.keeper = NewKeeper[headForTest](Slots, addTrigger, removedHeadNotifier) + s.keeper = NewKeeper[headForTest](Slots, removedHeadNotifier) _ = s.keeper.Add(newHeadForTest("a"), 1) _ = s.keeper.Add(newHeadForTest("b"), 2) _ = s.keeper.Add(newHeadForTest("c"), 3) @@ -132,7 +121,6 @@ func (s *KeeperSuite) TestRemove() { {head: newHeadForTest("b"), createdAt: 2}, {head: newHeadForTest("d"), createdAt: 4}, }, s.keeper.heads) - s.Equal(5, count) s.Equal(Slots, cap(s.keeper.heads)) s.Equal(1, removedHeadNotifier.count) } diff --git a/pp/go/storage/head/services/persistener_test.go b/pp/go/storage/head/services/persistener_test.go index 7e3c46fcfa..02a58a36f1 100644 --- a/pp/go/storage/head/services/persistener_test.go +++ b/pp/go/storage/head/services/persistener_test.go @@ -51,7 +51,7 @@ func (s *GenericPersistenceSuite) SetupTest() { h := s.mustCreateHead() activeHeadContainer := container.NewWeighted(h) removedHeadNotifier := &mock.WriteNotifierMock{NotifyFunc: func() {}} - hKeeper := keeper.NewKeeper[storage.HeadOnDisk](1, func() {}, removedHeadNotifier) + hKeeper := keeper.NewKeeper[storage.HeadOnDisk](1, removedHeadNotifier) s.proxy = storage.NewProxy(activeHeadContainer, hKeeper, func(*storage.HeadOnDisk) error { return nil }) s.blockWriter = &mock.HeadBlockWriterMock[*storage.ShardOnDisk]{} s.writeNotifier = &mock.WriteNotifierMock{NotifyFunc: func() {}} diff --git a/pp/go/storage/head/services/rotator.go b/pp/go/storage/head/services/rotator.go index 1ded614c04..88d5b7e214 100644 --- a/pp/go/storage/head/services/rotator.go +++ b/pp/go/storage/head/services/rotator.go @@ -11,6 +11,9 @@ import ( "github.com/prometheus/prometheus/pp/go/util" ) +// CopySeriesOnRotate copy active series from the current head to the new head during rotation. +var CopySeriesOnRotate = false + // // RotatorConfig // @@ -32,11 +35,13 @@ type Rotator[ TShard, TGoShard Shard, THead Head[TTask, TShard, TGoShard], ] struct { - proxyHead ProxyHead[TTask, TShard, TGoShard, THead] - headBuilder HeadBuilder[TTask, TShard, TGoShard, THead] - m Mediator - cfg RotatorConfig - headInformer HeadInformer + proxyHead ProxyHead[TTask, TShard, TGoShard, THead] + headBuilder HeadBuilder[TTask, TShard, TGoShard, THead] + m Mediator + cfg RotatorConfig + headInformer HeadInformer + headAddedSeriesCopier func(source, destination THead) + rotatedTrigger func() // stat rotateCounter prometheus.Counter @@ -56,15 +61,19 @@ func NewRotator[ m Mediator, cfg RotatorConfig, headInformer HeadInformer, + headAddedSeriesCopier func(source, destination THead), + rotatedTrigger func(), registerer prometheus.Registerer, ) *Rotator[TTask, TShard, TGoShard, THead] { factory := util.NewUnconflictRegisterer(registerer) return &Rotator[TTask, TShard, TGoShard, THead]{ - proxyHead: proxyHead, - headBuilder: headBuilder, - m: m, - cfg: cfg, - headInformer: headInformer, + proxyHead: proxyHead, + headBuilder: headBuilder, + m: m, + cfg: cfg, + headInformer: headInformer, + headAddedSeriesCopier: headAddedSeriesCopier, + rotatedTrigger: rotatedTrigger, rotateCounter: factory.NewCounter( prometheus.CounterOpts{ Name: "prompp_rotator_rotate_count", @@ -113,6 +122,8 @@ func (s *Rotator[TTask, TShard, TGoShard, THead]) Execute(ctx context.Context) e } // rotate it creates a new [Head] and makes it active, and sends the old [Head] to the [Keeper]. +// +//revive:disable-next-line:cyclomatic // long but readable. func (s *Rotator[TTask, TShard, TGoShard, THead]) rotate( ctx context.Context, numberOfShards uint16, @@ -124,8 +135,9 @@ func (s *Rotator[TTask, TShard, TGoShard, THead]) rotate( return fmt.Errorf("failed to build a new head: %w", err) } - // TODO CopySeriesFrom only old nunber of shards == new - // newHead.CopySeriesFrom(oldHead) + if CopySeriesOnRotate && oldHead.NumberOfShards() == newHead.NumberOfShards() { + s.headAddedSeriesCopier(oldHead, newHead) + } if err = s.proxyHead.AddWithReplace(oldHead, s.headInformer.CreatedAt(oldHead.ID())); err != nil { return fmt.Errorf("failed add to keeper old head: %w", err) @@ -160,6 +172,7 @@ func (s *Rotator[TTask, TShard, TGoShard, THead]) rotate( oldHead.SetReadOnly() s.events.With(prometheus.Labels{"type": "rotated"}).Inc() s.rotationDuration.Set(float64(time.Since(start).Nanoseconds())) + s.rotatedTrigger() return nil } diff --git a/pp/go/storage/head/shard/lss.go b/pp/go/storage/head/shard/lss.go index de0e6dda42..6a30a0d165 100644 --- a/pp/go/storage/head/shard/lss.go +++ b/pp/go/storage/head/shard/lss.go @@ -35,11 +35,14 @@ func (l *LSS) AllocatedMemory() uint64 { return am } -// CopyAddedSeries copy label sets which were added via FindOrEmplace to destination. -func (l *LSS) CopyAddedSeries(destination *LSS) { +// CopyAddedSeriesTo copy the label sets from the source lss to the destination lss that were added source lss. +func (l *LSS) CopyAddedSeriesTo(destination *LSS) { l.locker.RLock() - l.target.CopyAddedSeries(destination.target) + snapshot := l.getSnapshot() + bitsetSeries := l.target.BitsetSeries() l.locker.RUnlock() + + snapshot.CopyAddedSeries(bitsetSeries, destination.target) } // Input returns input lss. diff --git a/pp/go/storage/head/shard/shard.go b/pp/go/storage/head/shard/shard.go index f3a47953b7..851b21de2d 100644 --- a/pp/go/storage/head/shard/shard.go +++ b/pp/go/storage/head/shard/shard.go @@ -252,3 +252,12 @@ func NewPerGoroutineShard[TWal Wal](s *Shard, numberOfShards uint16) *PerGorouti func (s *PerGoroutineShard) Relabeler() *cppbridge.PerGoroutineRelabeler { return s.relabeler } + +// +// CopyAddedSeries +// + +// CopyAddedSeries copy the label sets from the source lss to the destination lss that were added source lss. +func CopyAddedSeries(source, destination *Shard) { + source.lss.CopyAddedSeriesTo(destination.lss) +} diff --git a/pp/go/storage/loader_test.go b/pp/go/storage/loader_test.go index bdf963a5a8..13ca3c176a 100644 --- a/pp/go/storage/loader_test.go +++ b/pp/go/storage/loader_test.go @@ -101,6 +101,9 @@ func (s *HeadLoadSuite) createHead(unloadDataStorageInterval time.Duration) (*st func (s *HeadLoadSuite) mustCreateHead(unloadDataStorageInterval time.Duration) *storage.HeadOnDisk { h, err := s.createHead(unloadDataStorageInterval) s.Require().NoError(err) + + s.catalog.SetStatus(h.ID(), catalog.StatusActive) + return h } diff --git a/pp/go/storage/manager.go b/pp/go/storage/manager.go index 3fb7d7a903..5090c00557 100644 --- a/pp/go/storage/manager.go +++ b/pp/go/storage/manager.go @@ -21,8 +21,10 @@ import ( "github.com/prometheus/prometheus/pp/go/storage/block" "github.com/prometheus/prometheus/pp/go/storage/catalog" "github.com/prometheus/prometheus/pp/go/storage/head/container" + "github.com/prometheus/prometheus/pp/go/storage/head/head" "github.com/prometheus/prometheus/pp/go/storage/head/keeper" "github.com/prometheus/prometheus/pp/go/storage/head/services" + "github.com/prometheus/prometheus/pp/go/storage/head/shard" "github.com/prometheus/prometheus/pp/go/storage/mediator" "github.com/prometheus/prometheus/pp/go/storage/querier" "github.com/prometheus/prometheus/pp/go/storage/ready" @@ -39,8 +41,11 @@ const ( // DefaultMetricWriteInterval default metric scrape interval. DefaultMetricWriteInterval = 15 * time.Second + // defaultStartMetricWriteInterval the default interval for start [MetricsUpdater] timer. + defaultStartMetricWriteInterval = 5 * time.Second + // DefaultPersistDuration the default interval for persisting [Head]. - DefaultPersistDuration = 5 * time.Minute + DefaultPersistDuration = 2 * time.Minute // DefaultUnloadDataStorageInterval the default interval for unloading [DataStorage]. DefaultUnloadDataStorageInterval = 5 * time.Minute @@ -50,9 +55,6 @@ const ( ) var ( - // CopySeriesOnRotate copy active series from the current head to the new head during rotation. - CopySeriesOnRotate = false - // UnloadDataStorage flags for unloading [DataStorage]. UnloadDataStorage = false @@ -181,11 +183,11 @@ func NewManager( return nil, errors.Join(fmt.Errorf("failed to set active status: %w", err), h.Close()) } - persistenerMediator := mediator.NewMediator( - mediator.NewConstantIntervalTimer(clock, defaultStartPersistnerInterval, DefaultPersistDuration), + hKeeper := keeper.NewKeeper[HeadOnDisk]( + o.KeeperCapacity, + removedHeadNotifier, ) - hKeeper := keeper.NewKeeper[HeadOnDisk](o.KeeperCapacity, persistenerMediator.Trigger, removedHeadNotifier) m := &Manager{ g: run.Group{}, closer: util.NewCloser(), @@ -200,7 +202,7 @@ func NewManager( ), } - m.initServices(o, hcatalog, builder, loader, reloadBlocksNotifier, persistenerMediator, readyNotifier, clock, r) + m.initServices(o, hcatalog, builder, loader, reloadBlocksNotifier, readyNotifier, clock, r) logger.Infof("[Head Manager] created") return m, nil @@ -255,7 +257,6 @@ func (m *Manager) initServices( builder *Builder, loader *Loader, reloadBlocksTriggerNotifier *TriggerNotifier, - persistenerMediator *mediator.Mediator, readyNotifier ready.Notifier, clock clockwork.Clock, r prometheus.Registerer, @@ -277,6 +278,9 @@ func (m *Manager) initServices( ) // Persistener + persistenerMediator := mediator.NewMediator( + mediator.NewConstantIntervalTimer(clock, defaultStartPersistnerInterval, DefaultPersistDuration), + ) m.g.Add( func() error { services.NewPersistenerService( @@ -314,6 +318,8 @@ func (m *Manager) initServices( m.rotatorMediator, m.cfg, &headInformer{catalog: hcatalog}, + head.CopyAddedSeries[*ShardOnDisk, *PerGoroutineShard](shard.CopyAddedSeries), + persistenerMediator.TriggerWithResetTimer, r, ).Execute(rotatorCtx) }, @@ -323,16 +329,6 @@ func (m *Manager) initServices( }, ) - // checks if the head is new - isNewHead := func(headID string) bool { - rec, err := hcatalog.Get(headID) - if err != nil { - return true - } - - return clock.Now().Add(-DefaultMergeDuration).UnixMilli() < rec.CreatedAt() - } - // Committer committerMediator := mediator.NewMediator( mediator.NewConstantIntervalTimer(clock, o.CommitInterval, o.CommitInterval), @@ -340,7 +336,11 @@ func (m *Manager) initServices( committerCtx, committerCancel := context.WithCancel(baseCtx) m.g.Add( func() error { - return services.NewCommitter(m.proxy, committerMediator, isNewHead).Execute(committerCtx) + return services.NewCommitter( + m.proxy, + committerMediator, + isNewHead(clock, hcatalog, o.CommitInterval), + ).Execute(committerCtx) }, func(error) { committerMediator.Close() @@ -352,7 +352,11 @@ func (m *Manager) initServices( mergerCtx, mergerCancel := context.WithCancel(baseCtx) m.g.Add( func() error { - return services.NewMerger(m.proxy, m.mergerMediator, isNewHead).Execute(mergerCtx) + return services.NewMerger( + m.proxy, + m.mergerMediator, + isNewHead(clock, hcatalog, DefaultMergeDuration), + ).Execute(mergerCtx) }, func(error) { m.mergerMediator.Close() @@ -362,7 +366,7 @@ func (m *Manager) initServices( // MetricsUpdater metricsUpdaterMediator := mediator.NewMediator( - mediator.NewConstantIntervalTimer(clock, DefaultMetricWriteInterval, DefaultMetricWriteInterval), + mediator.NewConstantIntervalTimer(clock, defaultStartMetricWriteInterval, DefaultMetricWriteInterval), ) metricsUpdaterCtx, metricsUpdaterCancel := context.WithCancel(baseCtx) m.g.Add( @@ -424,6 +428,22 @@ func (hi *headInformer) SetRotatedStatus(headID string) error { return err } +// +// isNewHead +// + +// isNewHead builds a checker that checks if the head is new. +func isNewHead(clock clockwork.Clock, hcatalog *catalog.Catalog, interval time.Duration) func(headID string) bool { + return func(headID string) bool { + rec, err := hcatalog.Get(headID) + if err != nil { + return true + } + + return clock.Now().Add(-interval).UnixMilli() < rec.CreatedAt() + } +} + // // TriggerNotifier // @@ -455,6 +475,10 @@ func (tn *TriggerNotifier) Notify() { // uploadOrBuildHead // +// uploadOrBuildHead uploads or builds a new head. +// +//revive:disable-next-line:function-length // long but readable. +//revive:disable-next-line:cyclomatic // long but readable. func uploadOrBuildHead( clock clockwork.Clock, hcatalog *catalog.Catalog, diff --git a/pp/go/storage/mediator/mediator.go b/pp/go/storage/mediator/mediator.go index a200fc162f..b3bcfef5b8 100644 --- a/pp/go/storage/mediator/mediator.go +++ b/pp/go/storage/mediator/mediator.go @@ -75,6 +75,15 @@ func (m *Mediator) Trigger() { } } +// TriggerWithResetTimer send notify to channel and reset timer. +func (m *Mediator) TriggerWithResetTimer() { + select { + case m.c <- struct{}{}: + m.timer.Reset() + default: + } +} + // loop by timer. func (m *Mediator) loop() { defer m.closer.Done() diff --git a/pp/go/storage/mediator/mediator_test.go b/pp/go/storage/mediator/mediator_test.go index b819b5dbe6..a0f63084f6 100644 --- a/pp/go/storage/mediator/mediator_test.go +++ b/pp/go/storage/mediator/mediator_test.go @@ -147,6 +147,50 @@ func (s *MediatorSuite) TestTrigger() { cancel() s.Equal(1, counter) + s.Empty(timer.ResetCalls()) +} + +func (s *MediatorSuite) TestTriggerWithResetTimer() { + chTimer := make(chan time.Time, 1) + + timer := &TimerMock{ + ChanFunc: func() <-chan time.Time { + return chTimer + }, + ResetFunc: func() {}, + StopFunc: func() {}, + } + + m := mediator.NewMediator(timer) + defer m.Close() + + ctx, cancel := context.WithTimeout(context.Background(), 5*time.Second) + counter := 0 + done := make(chan struct{}) + start := sync.WaitGroup{} + start.Add(1) + go func() { + start.Done() + select { + case <-m.C(): + counter++ + close(done) + case <-ctx.Done(): + } + }() + + start.Wait() + s.T().Log("trigger with reset timer") + m.TriggerWithResetTimer() + + select { + case <-done: + case <-ctx.Done(): + } + cancel() + + s.Equal(1, counter) + s.Len(timer.ResetCalls(), 1) } func (s *MediatorSuite) TestTriggerSkip() { diff --git a/pp/go/storage/types.go b/pp/go/storage/types.go index 5f7c18469a..67328814b4 100644 --- a/pp/go/storage/types.go +++ b/pp/go/storage/types.go @@ -22,7 +22,3 @@ type PerGoroutineShard = shard.PerGoroutineShard // HeadOnDisk [head.Head] with [ShardOnDisk]. type HeadOnDisk = head.Head[*ShardOnDisk, *PerGoroutineShard] - -// TODO DELETE -// // ProxyHead [proxy.Proxy] for [HeadOnDisk]s. -// type ProxyHead = proxy.Proxy[*HeadOnDisk] diff --git a/pp/performance_tests/benchmarks/BUILD b/pp/performance_tests/benchmarks/BUILD index e9e4aea3e3..4e18945e48 100644 --- a/pp/performance_tests/benchmarks/BUILD +++ b/pp/performance_tests/benchmarks/BUILD @@ -9,3 +9,14 @@ cc_binary( "@google_benchmark//:benchmark_main", ], ) + +cc_binary( + name = "copy_lss", + srcs = ["queryable_encoding_bimap_copy_benchmark.cpp"], + malloc = "@jemalloc", + deps = [ + "//:performance_tests_headers", + "@google_benchmark//:benchmark_main", + "//:entrypoint", + ], +) diff --git a/pp/performance_tests/benchmarks/queryable_encoding_bimap_copy_benchmark.cpp b/pp/performance_tests/benchmarks/queryable_encoding_bimap_copy_benchmark.cpp new file mode 100644 index 0000000000..ddd962f363 --- /dev/null +++ b/pp/performance_tests/benchmarks/queryable_encoding_bimap_copy_benchmark.cpp @@ -0,0 +1,150 @@ +// ./bazel-bin/performance_tests/benchmarks/copy_lss --benchmark_context=wal_file="performance_tests/test_data/new/lss_real" --benchmark_counters_tabular=true +// --benchmark_repetitions=25 +#include + +#include +#include +#include +#include +#include +#include + +#include "primitives/snug_composites.h" +#include "series_index/queryable_encoding_bimap.h" +#include "series_index/trie/cedarpp_tree.h" + +namespace { +using Lss = + series_index::QueryableEncodingBimap; + +template +using LssCopier = series_index::QueryableEncodingBimapCopier; + +std::string GetWalFileName() { + if (auto& context = benchmark::internal::GetGlobalContext(); context != nullptr) { + return context->operator[]("wal_file"); + } + return {}; +} + +void mark_all_series_as_added(const std::shared_ptr& lss) { + for (auto label_set : *lss) { + lss->find_or_emplace(label_set); + } +} + +std::shared_ptr LoadLssFromFile() { + auto file_name = GetWalFileName(); + auto lss = std::make_shared(); + + std::ifstream istrm(file_name, std::ios::binary); + istrm >> *lss; + mark_all_series_as_added(lss); + return lss; +} + +std::vector copy_added_series_times; +std::vector copy_ls_id_set_times; +std::vector build_trie_index_times; +std::vector build_ls_id_hashset_times; +std::vector build_reverse_index_times; + +void BM_CopyAllStepsWithTiming(benchmark::State& state) { + using std::chrono::nanoseconds; + using std::chrono::steady_clock; + + static auto lss = LoadLssFromFile(); + lss->build_deferred_indexes(); + + for ([[maybe_unused]] auto _ : state) { + Lss lss_copy; + LssCopier copier(*lss, lss->sorting_index(), lss->added_series(), lss_copy); + + { + auto start = steady_clock::now(); + copier.copy_added_series(); + auto end = steady_clock::now(); + copy_added_series_times.push_back(duration_cast(end - start).count()); + } + + { + auto start = steady_clock::now(); + copier.copy_ls_id_set(); + auto end = steady_clock::now(); + copy_ls_id_set_times.push_back(duration_cast(end - start).count()); + } + + { + auto start = steady_clock::now(); + copier.build_trie_index(); + auto end = steady_clock::now(); + build_trie_index_times.push_back(duration_cast(end - start).count()); + } + + { + auto start = steady_clock::now(); + copier.build_ls_id_hashset(); + auto end = steady_clock::now(); + build_ls_id_hashset_times.push_back(duration_cast(end - start).count()); + } + + { + auto start = steady_clock::now(); + copier.build_reverse_index(); + auto end = steady_clock::now(); + build_reverse_index_times.push_back(duration_cast(end - start).count()); + } + } + + benchmark::DoNotOptimize(copy_added_series_times); + benchmark::DoNotOptimize(copy_ls_id_set_times); + benchmark::DoNotOptimize(build_trie_index_times); + benchmark::DoNotOptimize(build_ls_id_hashset_times); + benchmark::DoNotOptimize(build_reverse_index_times); +} + +BENCHMARK(BM_CopyAllStepsWithTiming); + +uint64_t Min(const std::vector& v) { + return v.empty() ? 0uz : static_cast(*std::ranges::min_element(v)); +} + +void PrintMinStats() { + std::cout << "\n=== Min method timings (ns) ===\n"; + + constexpr int words_width = 20; + constexpr int numbers_width = 10; + + std::cout << std::left << std::setw(words_width) << "copy_added_series" + << ": " << std::right << std::setw(numbers_width) << Min(copy_added_series_times) << '\n'; + + std::cout << std::left << std::setw(words_width) << "copy_ls_id_set" + << ": " << std::right << std::setw(numbers_width) << Min(copy_ls_id_set_times) << '\n'; + + std::cout << std::left << std::setw(words_width) << "build_trie_index" + << ": " << std::right << std::setw(numbers_width) << Min(build_trie_index_times) << '\n'; + + std::cout << std::left << std::setw(words_width) << "build_ls_id_hashset" + << ": " << std::right << std::setw(numbers_width) << Min(build_ls_id_hashset_times) << '\n'; + + std::cout << std::left << std::setw(words_width) << "build_reverse_index" + << ": " << std::right << std::setw(numbers_width) << Min(build_reverse_index_times) << '\n'; + + std::cout << "-------------------------------\n"; + + std::cout << std::left << std::setw(words_width) << "total sum" + << ": " << std::right << std::setw(numbers_width) + << (Min(copy_added_series_times) + Min(copy_ls_id_set_times) + Min(build_trie_index_times) + Min(build_ls_id_hashset_times) + + Min(build_reverse_index_times)) + << '\n'; + + std::cout << "===============================\n"; +} + +} // namespace + +int main(int argc, char** argv) { + ::benchmark::Initialize(&argc, argv); + ::benchmark::RunSpecifiedBenchmarks(); + PrintMinStats(); +} diff --git a/pp/primitives/go_slice.h b/pp/primitives/go_slice.h index 4a8ce297df..e32e113b4c 100644 --- a/pp/primitives/go_slice.h +++ b/pp/primitives/go_slice.h @@ -125,23 +125,6 @@ template struct SliceControlBlock { using SizeType = size_t; - SliceControlBlock() = default; - SliceControlBlock(const SliceControlBlock&) = delete; - - SliceControlBlock(SliceControlBlock&& other) noexcept - : data(std::exchange(other.data, nullptr)), items_count(std::exchange(other.items_count, 0)), data_size(std::exchange(other.data_size, 0)) {} - - SliceControlBlock& operator=(const SliceControlBlock&) = delete; - PROMPP_ALWAYS_INLINE SliceControlBlock& operator=(SliceControlBlock&& other) noexcept { - if (this != &other) [[likely]] { - data = std::exchange(other.data, nullptr); - data_size = std::exchange(other.data_size, 0); - items_count = std::exchange(other.items_count, 0); - } - - return *this; - } - T* data{}; union { diff --git a/pp/primitives/snug_composites_filaments.h b/pp/primitives/snug_composites_filaments.h index c34105f35e..b2f7421f63 100644 --- a/pp/primitives/snug_composites_filaments.h +++ b/pp/primitives/snug_composites_filaments.h @@ -144,7 +144,10 @@ class Symbol { return max_ui32 - this->size(); } - PROMPP_ALWAYS_INLINE void reserve(const data_type& other) { Vector::reserve(other.size()); } + template + PROMPP_ALWAYS_INLINE void reserve(const DataType& other) { + Vector::reserve(other.size()); + } }; using composite_type = std::string_view; @@ -369,7 +372,8 @@ class LabelNameSet { return symbols_table.allocated_memory() + BareBones::mem::allocated_memory(symbols_ids_sequences); } - PROMPP_ALWAYS_INLINE void reserve(const data_type& other) noexcept { + template + PROMPP_ALWAYS_INLINE void reserve(const DataType& other) noexcept { symbols_table.reserve(other.symbols_table); symbols_ids_sequences.reserve(other.symbols_ids_sequences.size()); } @@ -691,7 +695,8 @@ class LabelSet { symbols_tables.resize(symbols_tables_checkpoints.size()); } - void reserve(const data_type& other) { + template + void reserve(const DataType& other) { symbols_ids_sequences.reserve(other.symbols_ids_sequences.size()); symbols_tables.reserve(other.symbols_tables.size()); label_name_sets_table.reserve(other.label_name_sets_table); diff --git a/pp/prometheus/relabeler.h b/pp/prometheus/relabeler.h index 09feecf9a1..97ed91c1ce 100644 --- a/pp/prometheus/relabeler.h +++ b/pp/prometheus/relabeler.h @@ -811,7 +811,7 @@ class PerShardRelabeler { relabeler_state_update->reserve(relabeler_state_update->size() + relabeled_series->size()); inner_series->reserve(inner_series->size() + relabeled_series->size()); if constexpr (BareBones::concepts::has_reserve) { - lss.reserve(lss.size() + relabeled_series->size()); + lss.reserve(static_cast(lss.size() + relabeled_series->size())); } for (const auto& relabeled_serie : relabeled_series->data()) { diff --git a/pp/series_index/queryable_encoding_bimap.h b/pp/series_index/queryable_encoding_bimap.h index bb9e147fd6..06c06a2733 100644 --- a/pp/series_index/queryable_encoding_bimap.h +++ b/pp/series_index/queryable_encoding_bimap.h @@ -8,6 +8,7 @@ #include "queried_series.h" #include "reverse_index.h" #include "sorting_index.h" +#include "trie/cedarpp_tree.h" #include "trie_index.h" namespace series_index { @@ -24,8 +25,6 @@ class QueryableEncodingBimap final : public BareBones::SnugComposite::GenericDec using TrieIndex = series_index::TrieIndex; using TrieIndexIterator = typename TrieIndex::Iterator; - using Base::reserve; - friend class BareBones::SnugComposite::GenericDecodingTable; [[nodiscard]] PROMPP_ALWAYS_INLINE const TrieIndex& trie_index() const noexcept { return trie_index_; } @@ -41,6 +40,8 @@ class QueryableEncodingBimap final : public BareBones::SnugComposite::GenericDec sorting_index_.allocated_memory() + Base::allocated_memory(); } + [[nodiscard]] PROMPP_ALWAYS_INLINE const auto& added_series() const noexcept { return added_series_; } + template PROMPP_ALWAYS_INLINE uint32_t find_or_emplace(const LabelSet& label_set) noexcept { return find_or_emplace(label_set, Base::hasher()(label_set)); @@ -74,6 +75,7 @@ class QueryableEncodingBimap final : public BareBones::SnugComposite::GenericDec return {}; } + using Base::reserve; PROMPP_ALWAYS_INLINE void reserve(uint32_t count) { Base::items_.reserve(count); ls_id_hash_set_.reserve(count); @@ -83,15 +85,14 @@ class QueryableEncodingBimap final : public BareBones::SnugComposite::GenericDec private: using LabelSet = typename Base::value_type; - template + template friend class QueryableEncodingBimapCopier; TrieIndex trie_index_; SeriesReverseIndex reverse_index_; size_t ls_id_set_allocated_memory_{}; - bool ls_id_comparator_enabled_{true}; - LsIdSet ls_id_set_{{}, Base::less_comparator(&ls_id_comparator_enabled_), BareBones::Allocator{ls_id_set_allocated_memory_}}; + LsIdSet ls_id_set_{{}, Base::less_comparator(), BareBones::Allocator{ls_id_set_allocated_memory_}}; size_t ls_id_hash_set_allocated_memory_{}; HashSet ls_id_hash_set_{0, Base::hasher(), Base::equality_comparator(), BareBones::Allocator{ls_id_hash_set_allocated_memory_}}; @@ -139,10 +140,10 @@ class QueryableEncodingBimap final : public BareBones::SnugComposite::GenericDec PROMPP_ALWAYS_INLINE static size_t phmap_hash(size_t hash) noexcept { return phmap::phmap_mix()(hash); } }; -template +template class QueryableEncodingBimapCopier { public: - static constexpr auto kInvalidIdFillByte = static_cast(QueryableEncodingBimap::kInvalidId); + static constexpr auto kInvalidIdFillByte = static_cast(DecodingTable::kInvalidId); template PROMPP_ALWAYS_INLINE static void resize_and_fill_ids_list(IdsList& list, uint32_t size) { @@ -163,7 +164,11 @@ class QueryableEncodingBimapCopier { values.resize(names_count); for (auto [value_cache, symbol_table] : std::ranges::views::zip(values, symbols_tables)) { - resize_and_fill_ids_list(value_cache, symbol_table->size()); + if constexpr (BareBones::concepts::is_dereferenceable) { + resize_and_fill_ids_list(value_cache, symbol_table->size()); + } else { + resize_and_fill_ids_list(value_cache, symbol_table.size()); + } } } @@ -172,36 +177,35 @@ class QueryableEncodingBimapCopier { BareBones::Vector values; }; - QueryableEncodingBimapCopier(const QueryableEncodingBimap& source, QueryableEncodingBimap& destination) : source_(source), destination_(destination) { + QueryableEncodingBimapCopier(const DecodingTable& source, + const SortingIndex& sorting_index, + const SeriesIds& ls_id_range, + QueryableEncodingBimap& destination) + : source_(source), sorting_index_(sorting_index), destination_(destination), ls_id_range_(ls_id_range) { assert(destination.size() == 0); } void copy_added_series() { - resize_and_fill_ids_list(ids_map_, source_.items_.size()); + BareBones::Vector ids; + ids.reserve(source_.size()); + std::ranges::copy(ls_id_range_.begin(), ls_id_range_.end(), std::back_inserter(ids)); + + sorting_index_.sort(ids); Cache cache; - cache.reserve(source_.data_.label_name_sets_table.size(), source_.data_.label_name_sets_table.data().symbols_table.size(), source_.data_.symbols_tables); + cache.reserve(source_.data().label_name_sets_table.size(), source_.data().label_name_sets_table.data().symbols_table.size(), source_.data().symbols_tables); destination_.reserve(source_); - for (const auto ls_id : source_.added_series_) { - ids_map_[ls_id] = destination_.next_item_index(); + for (const auto ls_id : ids) { destination_.items_.emplace_back(destination_.data_, source_[ls_id], cache); } } void copy_ls_id_set() { - destination_.ls_id_comparator_enabled_ = false; - - for (auto ls_id : source_.ls_id_set_) { - if (const auto new_ls_id = ids_map_[ls_id]; new_ls_id != QueryableEncodingBimap::kInvalidId) { - destination_.ls_id_set_.emplace_hint(destination_.ls_id_set_.end(), new_ls_id); - } + for (uint32_t ls_id_new = 0; ls_id_new < destination_.size(); ++ls_id_new) { + destination_.ls_id_set_.emplace_hint_cmp(destination_.ls_id_set_.end(), [&](auto, auto) { return true; }, ls_id_new); } - - destination_.ls_id_comparator_enabled_ = true; - - ids_map_ = BareBones::Vector{}; } void build_reverse_index() { @@ -246,9 +250,10 @@ class QueryableEncodingBimapCopier { } private: - const QueryableEncodingBimap& source_; + const DecodingTable& source_; + const SortingIndex& sorting_index_; QueryableEncodingBimap& destination_; - BareBones::Vector ids_map_; + const SeriesIds& ls_id_range_; }; } // namespace series_index diff --git a/pp/series_index/tests/queryable_encoding_bimap_tests.cpp b/pp/series_index/tests/queryable_encoding_bimap_tests.cpp index 19e264828a..2b723427fe 100644 --- a/pp/series_index/tests/queryable_encoding_bimap_tests.cpp +++ b/pp/series_index/tests/queryable_encoding_bimap_tests.cpp @@ -15,6 +15,9 @@ using series_index::SeriesReverseIndex; using series_index::trie::CedarMatchesList; using series_index::trie::CedarTrie; +template +using Copier = QueryableEncodingBimapCopier; + class QueryableEncodingBimapFixture : public testing::Test { protected: using Lss = QueryableEncodingBimap; @@ -116,15 +119,12 @@ TEST_F(QueryableEncodingBimapFixture, EmplaceDuplicatedLabelSet) { EXPECT_NE(ls_id1, ls_id2); } -class QueryableEncodingBimapCopierFixture : public QueryableEncodingBimapFixture { - protected: - using Copier = QueryableEncodingBimapCopier; -}; +class QueryableEncodingBimapCopierFixture : public QueryableEncodingBimapFixture {}; TEST_F(QueryableEncodingBimapCopierFixture, EmptyLss) { // Arrange Lss lss_copy; - Copier copier(lss_, lss_copy); + Copier copier(lss_, lss_.sorting_index(), lss_.added_series(), lss_copy); // Act copier.copy_added_series_and_build_indexes(); @@ -135,34 +135,67 @@ TEST_F(QueryableEncodingBimapCopierFixture, EmptyLss) { TEST_F(QueryableEncodingBimapCopierFixture, NonEmptyLss) { // Arrange - Lss lss_copy; - Copier copier(lss_, lss_copy); - const auto label_set = LabelViewSet{{"job", "cron"}, {"key", ""}, {"process", "php1"}}; const auto label_set2 = LabelViewSet{{"job", "cron"}, {"key", ""}, {"process", "php"}}; lss_.find_or_emplace(label_set); lss_.find_or_emplace(label_set2); + lss_.build_deferred_indexes(); + + Lss lss_copy; + Copier copier(lss_, lss_.sorting_index(), lss_.added_series(), lss_copy); + // Act copier.copy_added_series_and_build_indexes(); // Assert EXPECT_EQ(2U, lss_copy.size()); - EXPECT_EQ(0U, lss_copy.find(label_set)); - EXPECT_EQ(1U, lss_copy.find(label_set2)); + + EXPECT_TRUE(lss_copy.find(label_set).has_value()); + EXPECT_TRUE(lss_copy.find(label_set2).has_value()); + EXPECT_EQ(2U, lss_copy.reverse_index().names_count()); - EXPECT_THAT(lss_copy.ls_id_set(), testing::ElementsAre(1, 0)); + + EXPECT_EQ(2U, lss_copy.ls_id_set().size()); EXPECT_FALSE(lss_copy.ls_id_set().empty()); + EXPECT_TRUE(lss_copy.trie_index().names_trie().lookup("job")); } +TEST_F(QueryableEncodingBimapCopierFixture, NonEmptyLssKeepOrder) { + // Arrange + const auto label_set1 = LabelViewSet{{"job", "cron"}, {"key", "1"}, {"process", "php"}}; + const auto label_set2 = LabelViewSet{{"job", "cron"}, {"key", "2"}, {"process", "php"}}; + const auto label_set3 = LabelViewSet{{"job", "cron"}, {"key", "3"}, {"process", "php"}}; + const auto label_set4 = LabelViewSet{{"job", "cron"}, {"key", "4"}, {"process", "php"}}; + const auto label_set5 = LabelViewSet{{"job", "cron"}, {"key", "5"}, {"process", "php"}}; + + lss_.find_or_emplace(label_set4); + lss_.find_or_emplace(label_set1); + lss_.find_or_emplace(label_set3); + lss_.find_or_emplace(label_set5); + lss_.find_or_emplace(label_set2); + + lss_.build_deferred_indexes(); + + Lss lss_copy; + Copier copier(lss_, lss_.sorting_index(), lss_.added_series(), lss_copy); + + // Act + copier.copy_added_series_and_build_indexes(); + + // Assert + EXPECT_TRUE(std::ranges::is_sorted(lss_copy.ls_id_set(), + [&](const auto idl, const auto idr) { return std::ranges::lexicographical_compare(lss_copy[idl], lss_copy[idr]); })); +} + TEST_F(QueryableEncodingBimapCopierFixture, CopyOfCopy) { // Arrange Lss lss_copy; Lss lss_copy_of_copy; - Copier copier(lss_, lss_copy); - Copier copier2(lss_copy, lss_copy_of_copy); + Copier copier(lss_, lss_.sorting_index(), lss_.added_series(), lss_copy); + Copier copier2(lss_copy, lss_copy.sorting_index(), lss_copy.added_series(), lss_copy_of_copy); const auto label_set = LabelViewSet{{"job", "cron"}, {"key", ""}, {"process", "php"}}; const auto label_set2 = LabelViewSet{{"job", "cron"}, {"key", ""}, {"process", "php1"}}; @@ -172,18 +205,25 @@ TEST_F(QueryableEncodingBimapCopierFixture, CopyOfCopy) { lss_.find_or_emplace(label_set2); lss_.find_or_emplace(label_set3); + lss_.build_deferred_indexes(); + // Act copier.copy_added_series_and_build_indexes(); + lss_copy.build_deferred_indexes(); copier2.copy_added_series_and_build_indexes(); lss_copy_of_copy.find_or_emplace(label_set3); // Assert EXPECT_EQ(1U, lss_copy_of_copy.size()); + EXPECT_FALSE(lss_copy_of_copy.find(label_set)); EXPECT_FALSE(lss_copy_of_copy.find(label_set2)); EXPECT_TRUE(lss_copy_of_copy.find(label_set3)); + EXPECT_EQ(1U, lss_copy_of_copy.reverse_index().names_count()); - EXPECT_THAT(lss_copy_of_copy.ls_id_set(), testing::ElementsAre(0)); + + EXPECT_EQ(1U, lss_copy_of_copy.ls_id_set().size()); + EXPECT_FALSE(lss_copy_of_copy.trie_index().names_trie().lookup("job")); EXPECT_TRUE(lss_copy_of_copy.trie_index().names_trie().lookup("server")); } diff --git a/pp/third_party/patches/parallel_hashmap/btree.h.patch b/pp/third_party/patches/parallel_hashmap/btree.h.patch new file mode 100644 index 0000000000..00f82b2d5b --- /dev/null +++ b/pp/third_party/patches/parallel_hashmap/btree.h.patch @@ -0,0 +1,65 @@ +--- parallel_hashmap/btree.h ++++ parallel_hashmap/btree.h +@@ -1781,6 +1781,12 @@ namespace priv { + const key_type &key, + Args &&... args); + ++ template ++ std::pair insert_hint_unique_cmp(iterator position, ++ const key_type &key, ++ C && comp, ++ Args &&... args); ++ + // Insert a range of values into the btree. + template + void insert_iterator_unique(InputIterator b, InputIterator e); +@@ -2577,6 +2583,33 @@ namespace priv { + return insert_unique(key, std::forward(args)...); + } + ++ template ++ template ++ inline auto btree

::insert_hint_unique_cmp(iterator position, const key_type &key, ++ C && comp, Args &&... args) ++ -> std::pair { ++ auto comparator = std::forward(comp); ++ if (!empty()) { ++ if (position == end() || comparator(key, position.key())) { ++ iterator prev = position; ++ if (position == begin() || comparator((--prev).key(), key)) { ++ // prev.key() < key < position.key() ++ return {internal_emplace(position, std::forward(args)...), true}; ++ } ++ } else if (comparator(position.key(), key)) { ++ ++position; ++ if (position == end() || comparator(key, position.key())) { ++ // {original `position`}.key() < key < {current `position`}.key() ++ return {internal_emplace(position, std::forward(args)...), true}; ++ } ++ } else { ++ // position.key() == key ++ return {position, false}; ++ } ++ } ++ return insert_unique(key, std::forward(args)...); ++ } ++ + template + template + void btree

::insert_iterator_unique(InputIterator b, InputIterator e) { +@@ -3510,6 +3543,15 @@ namespace priv { + .first; + } + ++ template ++ iterator emplace_hint_cmp(const_iterator hint, C && comp, Args &&... args) { ++ init_type v(std::forward(args)...); ++ return this->tree_ ++ .insert_hint_unique_cmp(iterator(hint), params_type::key(v), ++ std::forward(comp), std::move(v)) ++ .first; ++ } ++ + template + void insert(InputIterator b, InputIterator e) { + this->tree_.insert_iterator_unique(b, e); diff --git a/pp/third_party/tracy.BUILD b/pp/third_party/tracy.BUILD index e9a5a1bab2..4704f7a031 100644 --- a/pp/third_party/tracy.BUILD +++ b/pp/third_party/tracy.BUILD @@ -25,11 +25,19 @@ cmake( visibility = ["//visibility:private"], ) +cc_library( + name = "tracy_headers", + hdrs = glob(["public/**/*.h", "public/**/*.hpp"]), + strip_include_prefix = "public", + visibility = ["//visibility:public"], +) + cc_library( name = "tracy", hdrs = glob(["public/**/*.h", "public/**/*.hpp"]), deps = [ - ":tracy_client" + ":tracy_headers", + ":tracy_client", ], strip_include_prefix = "public", visibility = ["//visibility:public"], diff --git a/pp/wal/encoder.h b/pp/wal/encoder.h index c601634610..50f7f09d87 100644 --- a/pp/wal/encoder.h +++ b/pp/wal/encoder.h @@ -19,20 +19,24 @@ class GenericEncoder { void write_stats(Stats* stats) const { stats->samples = encoder_.buffer().samples_count(); - if constexpr (BareBones::concepts::has_earliest_timestamp) { + if constexpr (BareBones::concepts::has_earliest_timestamp_field) { stats->earliest_timestamp = encoder_.buffer().earliest_sample(); } - if constexpr (BareBones::concepts::has_latest_timestamp) { + if constexpr (BareBones::concepts::has_latest_timestamp_field) { stats->latest_timestamp = encoder_.buffer().latest_sample(); } - if constexpr (BareBones::concepts::has_remainder_size) { + if constexpr (BareBones::concepts::has_series_field) { + stats->series = encoder_.buffer().series_count(); + } + + if constexpr (BareBones::concepts::has_remainder_size_field) { size_t remaining_cap = std::numeric_limits::max(); stats->remainder_size = std::min(remaining_cap, encoder_.remainder_size()); } - if constexpr (BareBones::concepts::has_allocated_memory) { + if constexpr (BareBones::concepts::has_allocated_memory_field) { stats->allocated_memory = encoder_.allocated_memory(); } } From 0f13ae0695d16f826b711d5648a04611a5a9d6f5 Mon Sep 17 00:00:00 2001 From: Alexandr Yudin Date: Fri, 3 Oct 2025 11:02:58 +0000 Subject: [PATCH 66/96] add test --- pp-pkg/scrape/scrape.go | 6 ++- pp/go/cppbridge/prometheus_relabeler_test.go | 41 ++++++++++++++++++++ 2 files changed, 46 insertions(+), 1 deletion(-) diff --git a/pp-pkg/scrape/scrape.go b/pp-pkg/scrape/scrape.go index 72c5a615ac..c2a9b02b2c 100644 --- a/pp-pkg/scrape/scrape.go +++ b/pp-pkg/scrape/scrape.go @@ -1164,7 +1164,11 @@ func (sl *scrapeLoop) getCache() *scrapeCache { return sl.cache } -// TODO Delete. +// append via prometheus parser. +// +//revive:disable-next-line:cognitive-complexity // legacy code +//revive:disable-next-line:function-length // legacy code +//revive:disable-next-line:cyclomatic // legacy code func (sl *scrapeLoop) append( b []byte, contentType string, diff --git a/pp/go/cppbridge/prometheus_relabeler_test.go b/pp/go/cppbridge/prometheus_relabeler_test.go index c83989f2cb..85fa09a083 100644 --- a/pp/go/cppbridge/prometheus_relabeler_test.go +++ b/pp/go/cppbridge/prometheus_relabeler_test.go @@ -1485,6 +1485,35 @@ func (s *StateV2Suite) TestStateReconfigure() { s.Panics(func() { state.StaleNansStateByShard(0) }) } +func (s *StateV2Suite) TestStateReconfigureWithoutReconfigure() { + state := cppbridge.NewStateV2() + state.Reconfigure(0, 1) + + cache1 := state.CacheByShard(0) + s.NotNil(cache1) + + state.Reconfigure(0, 1) + cache2 := state.CacheByShard(0) + s.NotNil(cache2) + s.Equal(cache1, cache2) +} + +func (s *StateV2Suite) TestStateReconfigureNumberOfShards() { + state := cppbridge.NewStateV2() + state.Reconfigure(0, 2) + + cache0 := state.CacheByShard(0) + s.NotNil(cache0) + cache1 := state.CacheByShard(1) + s.NotNil(cache1) + + state.Reconfigure(1, 1) + newCache0 := state.CacheByShard(0) + s.NotNil(newCache0) + s.NotEqual(cache0, newCache0) + s.Panics(func() { state.CacheByShard(1) }) +} + func (s *StateV2Suite) TestStateReconfigureTrackStaleness() { state := cppbridge.NewStateV2() state.EnableTrackStaleness() @@ -1494,3 +1523,15 @@ func (s *StateV2Suite) TestStateReconfigureTrackStaleness() { s.True(state.TrackStaleness()) s.NotNil(state.StaleNansStateByShard(0)) } + +func (s *StateV2Suite) TestStatelessRelabeler() { + state := cppbridge.NewStateV2() + + s.Panics(func() { state.StatelessRelabeler() }) + + statelessRelabeler, err := cppbridge.NewStatelessRelabeler([]*cppbridge.RelabelConfig{}) + s.Require().NoError(err) + + state.SetStatelessRelabeler(statelessRelabeler) + s.NotNil(state.StatelessRelabeler()) +} From 9c4a4bb44b8180c14508cfeeaeee27ff1029b240 Mon Sep 17 00:00:00 2001 From: Alexandr Yudin Date: Mon, 6 Oct 2025 09:13:00 +0000 Subject: [PATCH 67/96] fix test --- pp/go/cppbridge/prometheus_relabeler_test.go | 1 + 1 file changed, 1 insertion(+) diff --git a/pp/go/cppbridge/prometheus_relabeler_test.go b/pp/go/cppbridge/prometheus_relabeler_test.go index 85fa09a083..0a80e4f19a 100644 --- a/pp/go/cppbridge/prometheus_relabeler_test.go +++ b/pp/go/cppbridge/prometheus_relabeler_test.go @@ -1500,6 +1500,7 @@ func (s *StateV2Suite) TestStateReconfigureWithoutReconfigure() { func (s *StateV2Suite) TestStateReconfigureNumberOfShards() { state := cppbridge.NewStateV2() + state.EnableTrackStaleness() state.Reconfigure(0, 2) cache0 := state.CacheByShard(0) From 364509669d0c985a3898efb102e0cb2882dc95c2 Mon Sep 17 00:00:00 2001 From: Alexandr Yudin Date: Mon, 6 Oct 2025 10:46:33 +0000 Subject: [PATCH 68/96] fix test --- pp/go/storage/mediator/mediator_test.go | 16 ++++------------ 1 file changed, 4 insertions(+), 12 deletions(-) diff --git a/pp/go/storage/mediator/mediator_test.go b/pp/go/storage/mediator/mediator_test.go index a0f63084f6..3a0895ebd7 100644 --- a/pp/go/storage/mediator/mediator_test.go +++ b/pp/go/storage/mediator/mediator_test.go @@ -32,30 +32,22 @@ func (s *MediatorSuite) TestC() { m := mediator.NewMediator(timer) defer m.Close() - ctx, cancel := context.WithTimeout(context.Background(), 5*time.Second) counter := 0 done := make(chan struct{}) start := sync.WaitGroup{} start.Add(1) go func() { start.Done() - select { - case <-m.C(): - counter++ - close(done) - case <-ctx.Done(): - } + <-m.C() + counter++ + close(done) }() start.Wait() s.T().Log("timer tick") chTimer <- time.Time{} - select { - case <-done: - case <-ctx.Done(): - } - cancel() + <-done s.Equal(1, counter) } From d64b42ff0ec396eebae1ffa625e7a2be280c1b24 Mon Sep 17 00:00:00 2001 From: Alexandr Yudin Date: Mon, 6 Oct 2025 13:33:08 +0000 Subject: [PATCH 69/96] add test state --- pp/go/cppbridge/prometheus_relabeler_test.go | 94 ++++++++++++++++++-- 1 file changed, 88 insertions(+), 6 deletions(-) diff --git a/pp/go/cppbridge/prometheus_relabeler_test.go b/pp/go/cppbridge/prometheus_relabeler_test.go index 0a80e4f19a..3ab1c898e5 100644 --- a/pp/go/cppbridge/prometheus_relabeler_test.go +++ b/pp/go/cppbridge/prometheus_relabeler_test.go @@ -1466,10 +1466,18 @@ func TestStateV2Suite(t *testing.T) { } func (s *StateV2Suite) TestInitState() { - state := cppbridge.NewStateV2() + s.testInitState(cppbridge.NewStateV2()) + s.testInitState(cppbridge.NewStateV2WithoutLock()) +} +func (s *StateV2Suite) testInitState(state *cppbridge.StateV2) { s.Panics(func() { state.CacheByShard(0) }) s.Equal(time.Now().UnixMilli(), state.DefTimestamp()) + + newDeftime := time.Now().Add(5 * time.Minute).UnixMilli() + state.SetDefTimestamp(newDeftime) + s.Equal(newDeftime, state.DefTimestamp()) + s.False(state.IsTransition()) s.Equal(cppbridge.RelabelerOptions{}, state.RelabelerOptions()) s.Panics(func() { state.StaleNansStateByShard(0) }) @@ -1477,7 +1485,11 @@ func (s *StateV2Suite) TestInitState() { } func (s *StateV2Suite) TestStateReconfigure() { - state := cppbridge.NewStateV2() + s.testStateReconfigure(cppbridge.NewStateV2()) + s.testStateReconfigure(cppbridge.NewStateV2WithoutLock()) +} + +func (s *StateV2Suite) testStateReconfigure(state *cppbridge.StateV2) { state.Reconfigure(0, 1) s.NotNil(state.CacheByShard(0)) @@ -1486,7 +1498,11 @@ func (s *StateV2Suite) TestStateReconfigure() { } func (s *StateV2Suite) TestStateReconfigureWithoutReconfigure() { - state := cppbridge.NewStateV2() + s.testStateReconfigureWithoutReconfigure(cppbridge.NewStateV2()) + s.testStateReconfigureWithoutReconfigure(cppbridge.NewStateV2WithoutLock()) +} + +func (s *StateV2Suite) testStateReconfigureWithoutReconfigure(state *cppbridge.StateV2) { state.Reconfigure(0, 1) cache1 := state.CacheByShard(0) @@ -1499,7 +1515,11 @@ func (s *StateV2Suite) TestStateReconfigureWithoutReconfigure() { } func (s *StateV2Suite) TestStateReconfigureNumberOfShards() { - state := cppbridge.NewStateV2() + s.testStateReconfigureNumberOfShards(cppbridge.NewStateV2()) + s.testStateReconfigureNumberOfShards(cppbridge.NewStateV2WithoutLock()) +} + +func (s *StateV2Suite) testStateReconfigureNumberOfShards(state *cppbridge.StateV2) { state.EnableTrackStaleness() state.Reconfigure(0, 2) @@ -1516,7 +1536,11 @@ func (s *StateV2Suite) TestStateReconfigureNumberOfShards() { } func (s *StateV2Suite) TestStateReconfigureTrackStaleness() { - state := cppbridge.NewStateV2() + s.testStateReconfigureTrackStaleness(cppbridge.NewStateV2()) + s.testStateReconfigureTrackStaleness(cppbridge.NewStateV2WithoutLock()) +} + +func (s *StateV2Suite) testStateReconfigureTrackStaleness(state *cppbridge.StateV2) { state.EnableTrackStaleness() state.Reconfigure(0, 1) @@ -1526,8 +1550,11 @@ func (s *StateV2Suite) TestStateReconfigureTrackStaleness() { } func (s *StateV2Suite) TestStatelessRelabeler() { - state := cppbridge.NewStateV2() + s.testStatelessRelabeler(cppbridge.NewStateV2()) + s.testStatelessRelabeler(cppbridge.NewStateV2WithoutLock()) +} +func (s *StateV2Suite) testStatelessRelabeler(state *cppbridge.StateV2) { s.Panics(func() { state.StatelessRelabeler() }) statelessRelabeler, err := cppbridge.NewStatelessRelabeler([]*cppbridge.RelabelConfig{}) @@ -1536,3 +1563,58 @@ func (s *StateV2Suite) TestStatelessRelabeler() { state.SetStatelessRelabeler(statelessRelabeler) s.NotNil(state.StatelessRelabeler()) } + +func (s *StateV2Suite) TestInitTransitionStateV2() { + s.testInitTransitionState(cppbridge.NewTransitionStateV2()) + s.testInitTransitionState(cppbridge.NewTransitionStateV2WithoutLock()) +} + +func (s *StateV2Suite) testInitTransitionState(state *cppbridge.StateV2) { + s.True(state.IsTransition()) + s.Equal(cppbridge.RelabelerOptions{}, state.RelabelerOptions()) + s.Panics(func() { state.CacheByShard(0) }) + s.Panics(func() { state.StaleNansStateByShard(0) }) + s.False(state.TrackStaleness()) +} + +func (s *StateV2Suite) TestStateTransitionReconfigure() { + s.testStateTransitionReconfigure(cppbridge.NewTransitionStateV2()) + s.testStateTransitionReconfigure(cppbridge.NewTransitionStateV2WithoutLock()) +} + +func (s *StateV2Suite) testStateTransitionReconfigure(state *cppbridge.StateV2) { + state.Reconfigure(0, 1) + + s.False(state.TrackStaleness()) + s.Panics(func() { state.CacheByShard(0) }) + s.Panics(func() { state.StaleNansStateByShard(0) }) +} + +func (s *StateV2Suite) TestStateTransitionReconfigureTrackStaleness() { + s.testStateTransitionReconfigureTrackStaleness(cppbridge.NewTransitionStateV2()) + s.testStateTransitionReconfigureTrackStaleness(cppbridge.NewTransitionStateV2WithoutLock()) +} + +func (s *StateV2Suite) testStateTransitionReconfigureTrackStaleness(state *cppbridge.StateV2) { + s.Panics(func() { state.EnableTrackStaleness() }) + state.Reconfigure(0, 1) + + s.False(state.TrackStaleness()) + s.Panics(func() { state.CacheByShard(0) }) + s.Panics(func() { state.StaleNansStateByShard(0) }) +} + +func (s *StateV2Suite) TestStatelessRelabelerTransition() { + s.testStatelessRelabelerTransition(cppbridge.NewTransitionStateV2()) + s.testStatelessRelabelerTransition(cppbridge.NewTransitionStateV2WithoutLock()) +} + +func (s *StateV2Suite) testStatelessRelabelerTransition(state *cppbridge.StateV2) { + s.Panics(func() { state.StatelessRelabeler() }) + + statelessRelabeler, err := cppbridge.NewStatelessRelabeler([]*cppbridge.RelabelConfig{}) + s.Require().NoError(err) + + s.Panics(func() { state.SetStatelessRelabeler(statelessRelabeler) }) + s.Panics(func() { state.StatelessRelabeler() }) +} From c70b0e033e92faf72b4a61afa5ba4b037f669ac6 Mon Sep 17 00:00:00 2001 From: Alexandr Yudin Date: Mon, 6 Oct 2025 14:13:01 +0000 Subject: [PATCH 70/96] add test head wal --- pp/go/cppbridge/head_wal.go | 13 +++++++ pp/go/cppbridge/head_wal_test.go | 63 +++++++++++++++++++++++++++++--- 2 files changed, 71 insertions(+), 5 deletions(-) diff --git a/pp/go/cppbridge/head_wal.go b/pp/go/cppbridge/head_wal.go index 0cb5a3ce7f..e5d40fe823 100644 --- a/pp/go/cppbridge/head_wal.go +++ b/pp/go/cppbridge/head_wal.go @@ -55,11 +55,13 @@ func (s *HeadEncodedSegment) WriteTo(w io.Writer) (int64, error) { // HeadWalEncoder // +// HeadWalEncoder the encoder for the head wal. type HeadWalEncoder struct { lss *LabelSetStorage encoder uintptr } +// NewHeadWalEncoder initializes a new [HeadWalEncoder]. func NewHeadWalEncoder(shardID uint16, logShards uint8, lss *LabelSetStorage) *HeadWalEncoder { e := &HeadWalEncoder{ lss: lss, @@ -73,16 +75,19 @@ func NewHeadWalEncoder(shardID uint16, logShards uint8, lss *LabelSetStorage) *H return e } +// Version returns current encoder version. func (*HeadWalEncoder) Version() uint8 { return EncodersVersion() } +// Encode encodes inner series into a segment. func (e *HeadWalEncoder) Encode(innerSeriesSlice []*InnerSeries) (uint32, error) { samples, err := headWalEncoderAddInnerSeries(e.encoder, innerSeriesSlice) runtime.KeepAlive(e) return samples, err } +// Finalize finalizes the encoder and returns the encoded segment. func (e *HeadWalEncoder) Finalize() (*HeadEncodedSegment, error) { samples, segment, err := headWalEncoderFinalize(e.encoder) runtime.KeepAlive(e) @@ -93,11 +98,13 @@ func (e *HeadWalEncoder) Finalize() (*HeadEncodedSegment, error) { // HeadWalDecoder // +// HeadWalDecoder the decoder for the head wal. type HeadWalDecoder struct { lss *LabelSetStorage decoder uintptr } +// NewHeadWalDecoder initializes a new [HeadWalDecoder]. func NewHeadWalDecoder(lss *LabelSetStorage, encoderVersion uint8) *HeadWalDecoder { d := &HeadWalDecoder{ lss: lss, @@ -111,12 +118,17 @@ func NewHeadWalDecoder(lss *LabelSetStorage, encoderVersion uint8) *HeadWalDecod return d } +// Decode decodes a segment into an inner series. func (d *HeadWalDecoder) Decode(segment []byte, innerSeries *InnerSeries) error { err := headWalDecoderDecode(d.decoder, segment, innerSeries) runtime.KeepAlive(d) return err } +// DecodeToDataStorage decodes a segment into a data storage. +// +//revive:disable-next-line:confusing-results // returns createTimestamp, encodeTimestamp, error. +//nolint:gocritic // unnamedResult // returns createTimestamp, encodeTimestamp, error. func (d *HeadWalDecoder) DecodeToDataStorage(segment []byte, headEncoder *HeadEncoder) (int64, int64, error) { createTimestamp, encodeTimestamp, err := headWalDecoderDecodeToDataStorage(d.decoder, segment, headEncoder.encoder) runtime.KeepAlive(d) @@ -124,6 +136,7 @@ func (d *HeadWalDecoder) DecodeToDataStorage(segment []byte, headEncoder *HeadEn return createTimestamp, encodeTimestamp, err } +// CreateEncoder creates a new [HeadWalEncoder] from the decoder. func (d *HeadWalDecoder) CreateEncoder() *HeadWalEncoder { e := &HeadWalEncoder{ lss: d.lss, diff --git a/pp/go/cppbridge/head_wal_test.go b/pp/go/cppbridge/head_wal_test.go index e914fe4814..6698ac733d 100644 --- a/pp/go/cppbridge/head_wal_test.go +++ b/pp/go/cppbridge/head_wal_test.go @@ -4,16 +4,69 @@ import ( "testing" "github.com/gobuffalo/packr/v2/file/resolver/encoding/hex" - "github.com/prometheus/prometheus/pp/go/cppbridge" "github.com/stretchr/testify/require" + "github.com/stretchr/testify/suite" + + "github.com/prometheus/prometheus/pp/go/cppbridge" ) -func TestHeadWalEncoder_Finalize(t *testing.T) { +const hexSegment = "04224D184040C025030000B1D62B2CEB28C11C18DFD92E08008601020000000001000100021200170D1200071700042900150A1200140A2000300000001100FD0A03000000040000000500000006000000070000000800000009320013080400F316190000002100000014000000350000001E000000530000000E0000006100000013000000744E00937A0000001700000091180055A4000000165A00F5AFBA0000002C2975312C3F7C6A3C47343C55453531246E5A673044594B562A3645216E6F3F4D4A6F6D7154273D696C6C5F465765725750304672722F246D3F2E216D4B64313678506C30356C6A4555403F5961284768315F7C35596D713F4D4C293B713D684B786368562F393B593C382E492F597200744A2B244F233B375A32362F703D5F30596E3E3D54235E4A7946675F6267704C5445492E5F66576E704D413C6D28465227784F5E495E64687937776A7462296E52282C43463B4762734A0114016601110D0E00000600E92E3852473464414D6547436F67012500111D0E00000600F90F2262376555745376402A3B4D542A3E6E33243F34242C48716D39792C2302350011170E00000600F90977456B216B512A6C6643244E4E67252C512D3362794B3A032F00110E0E00000600F900474D2D55766D53614B764D6C626D042600111F0E00000600F9116B002A41224D703A3F656A3C4D252B3D6C46246D554E242A623972644C742F05370011180E00000600F90A3C4B356F7724234E7233464B2D38686D5C504763636D495306300011190E00000600F90B65282D795533525F636B6E45693C23702E4749776723503C40073100111B0E00000600F90D764900705C6C32584E4F30725978686975624066253024624C6B21083300111A0E00000600F90C504A7A6377575549593F6D2262672D705E5C21432F2E325F492409320011110E00000600F10464544E562A642D4835534727507500396401019F02D0040901FFFFFFFF683D16B182012D00141C5C03B27D000001773742790135021100F03DAF1C5A643BDFC7FF1C96FFCFE8F5A65B4E3C827DD5EB4DB79C7804BBEE57252B06C1938D78BF329D216BC9D4C8F743B0ACE2AEC78B9C19FAB779E7FBA5CD20B899C395ED280C0901677407941" + +// +// HeadWalSuite +// + +type HeadWalSuite struct { + suite.Suite +} + +func TestHeadWalSuite(t *testing.T) { + suite.Run(t, new(HeadWalSuite)) +} + +func (s *HeadWalSuite) TestHeadWalEncoder_Encode() { + encoder := cppbridge.NewHeadWalEncoder(0, 1, cppbridge.NewQueryableLssStorage()) + + samples, err := encoder.Encode([]*cppbridge.InnerSeries{}) + s.Require().NoError(err) + + s.Empty(samples) +} + +func (s *HeadWalSuite) TestHeadWalEncoder_Finalize() { + encoder := cppbridge.NewHeadWalEncoder(0, 1, cppbridge.NewQueryableLssStorage()) + + segmentData, err := encoder.Finalize() + s.Require().NoError(err) + + s.NotNil(segmentData) + s.Empty(segmentData.Samples()) +} + +func (s *HeadWalSuite) TestHeadWalEncoder_EncodeAndFinalize() { + const kTestBufferVersion = 3 + + segment, _ := hex.DecodeString(hexSegment) lss := cppbridge.NewQueryableLssStorage() + decoder := cppbridge.NewHeadWalDecoder(lss, kTestBufferVersion) encoder := cppbridge.NewHeadWalEncoder(0, 1, lss) + innerSeries := cppbridge.NewInnerSeries() + err := decoder.Decode(segment, innerSeries) + s.Require().NoError(err) + s.NotNil(innerSeries) + + expectedSamples := innerSeries.Size() + samples, err := encoder.Encode([]*cppbridge.InnerSeries{innerSeries}) + s.Require().NoError(err) + + s.NotNil(samples) + s.Equal(expectedSamples, uint64(samples)) + segmentData, err := encoder.Finalize() - require.NoError(t, err) - _ = segmentData + s.Require().NoError(err) + + s.NotNil(segmentData) + s.Equal(expectedSamples, uint64(segmentData.Samples())) } func TestHeadWalDecoder_DecodeToDataStorage(t *testing.T) { @@ -23,7 +76,7 @@ func TestHeadWalDecoder_DecodeToDataStorage(t *testing.T) { dataStorage := cppbridge.NewHeadDataStorage() encoder := cppbridge.NewHeadEncoderWithDataStorage(dataStorage) decoder := cppbridge.NewHeadWalDecoder(cppbridge.NewQueryableLssStorage(), kTestBufferVersion) - segment, _ := hex.DecodeString("04224D184040C025030000B1D62B2CEB28C11C18DFD92E08008601020000000001000100021200170D1200071700042900150A1200140A2000300000001100FD0A03000000040000000500000006000000070000000800000009320013080400F316190000002100000014000000350000001E000000530000000E0000006100000013000000744E00937A0000001700000091180055A4000000165A00F5AFBA0000002C2975312C3F7C6A3C47343C55453531246E5A673044594B562A3645216E6F3F4D4A6F6D7154273D696C6C5F465765725750304672722F246D3F2E216D4B64313678506C30356C6A4555403F5961284768315F7C35596D713F4D4C293B713D684B786368562F393B593C382E492F597200744A2B244F233B375A32362F703D5F30596E3E3D54235E4A7946675F6267704C5445492E5F66576E704D413C6D28465227784F5E495E64687937776A7462296E52282C43463B4762734A0114016601110D0E00000600E92E3852473464414D6547436F67012500111D0E00000600F90F2262376555745376402A3B4D542A3E6E33243F34242C48716D39792C2302350011170E00000600F90977456B216B512A6C6643244E4E67252C512D3362794B3A032F00110E0E00000600F900474D2D55766D53614B764D6C626D042600111F0E00000600F9116B002A41224D703A3F656A3C4D252B3D6C46246D554E242A623972644C742F05370011180E00000600F90A3C4B356F7724234E7233464B2D38686D5C504763636D495306300011190E00000600F90B65282D795533525F636B6E45693C23702E4749776723503C40073100111B0E00000600F90D764900705C6C32584E4F30725978686975624066253024624C6B21083300111A0E00000600F90C504A7A6377575549593F6D2262672D705E5C21432F2E325F492409320011110E00000600F10464544E562A642D4835534727507500396401019F02D0040901FFFFFFFF683D16B182012D00141C5C03B27D000001773742790135021100F03DAF1C5A643BDFC7FF1C96FFCFE8F5A65B4E3C827DD5EB4DB79C7804BBEE57252B06C1938D78BF329D216BC9D4C8F743B0ACE2AEC78B9C19FAB779E7FBA5CD20B899C395ED280C0901677407941") + segment, _ := hex.DecodeString(hexSegment) // Act _, _, err := decoder.DecodeToDataStorage(segment, encoder) From cfb0f6e82aa785f282d60567bd83b043acb0d2ef Mon Sep 17 00:00:00 2001 From: Alexandr Yudin Date: Mon, 6 Oct 2025 16:29:47 +0000 Subject: [PATCH 71/96] add test PerGoroutineRelabeler --- pp/go/cppbridge/prometheus_relabeler.go | 10 +- pp/go/cppbridge/prometheus_relabeler_test.go | 845 +++++++++++++------ 2 files changed, 593 insertions(+), 262 deletions(-) diff --git a/pp/go/cppbridge/prometheus_relabeler.go b/pp/go/cppbridge/prometheus_relabeler.go index c5448e6601..53a15806fd 100644 --- a/pp/go/cppbridge/prometheus_relabeler.go +++ b/pp/go/cppbridge/prometheus_relabeler.go @@ -1563,7 +1563,7 @@ func NewStateV2WithoutLock() *StateV2 { // CacheByShard return *Cache for shard. func (s *StateV2) CacheByShard(shardID uint16) *Cache { if s.IsTransition() { - panic("state is transition") + panic("CacheByShard: state is transition") } if int(shardID) >= len(s.caches) { @@ -1589,7 +1589,7 @@ func (s *StateV2) DefTimestamp() int64 { // EnableTrackStaleness enable track stalenans. func (s *StateV2) EnableTrackStaleness() { if s.IsTransition() { - panic("state is transition") + panic("EnableTrackStaleness: state is transition") } s.trackStaleness = true @@ -1652,7 +1652,7 @@ func (s *StateV2) SetRelabelerOptions(options *RelabelerOptions) { // SetStatelessRelabeler sets [StatelessRelabeler] for [PerGoroutineRelabeler]. func (s *StateV2) SetStatelessRelabeler(statelessRelabeler *StatelessRelabeler) { if s.IsTransition() { - panic("state is transition") + panic("SetStatelessRelabeler: state is transition") } s.statelessRelabeler = statelessRelabeler @@ -1661,7 +1661,7 @@ func (s *StateV2) SetStatelessRelabeler(statelessRelabeler *StatelessRelabeler) // StaleNansStateByShard return SourceStaleNansState for shard. func (s *StateV2) StaleNansStateByShard(shardID uint16) *StaleNansState { if s.IsTransition() { - panic("state is transition") + panic("StaleNansStateByShard: state is transition") } if int(shardID) >= len(s.staleNansStates) { @@ -1678,7 +1678,7 @@ func (s *StateV2) StaleNansStateByShard(shardID uint16) *StaleNansState { // StatelessRelabeler returns [StatelessRelabeler] for [PerGoroutineRelabeler]. func (s *StateV2) StatelessRelabeler() *StatelessRelabeler { if s.IsTransition() { - panic("state is transition") + panic("StatelessRelabeler: state is transition") } if s.statelessRelabeler == nil { diff --git a/pp/go/cppbridge/prometheus_relabeler_test.go b/pp/go/cppbridge/prometheus_relabeler_test.go index 3ab1c898e5..1f31dadf90 100644 --- a/pp/go/cppbridge/prometheus_relabeler_test.go +++ b/pp/go/cppbridge/prometheus_relabeler_test.go @@ -601,8 +601,46 @@ func (s *RelabelerSuite) TestToHash_EmptyConfig() { s.Require().Equal(xxhash.Sum64String("0"+a.String()), cppbridge.ToHash(rCfgs)) } -func (s *RelabelerSuite) TestPerGoroutineRelabeler() { - wr := prompb.WriteRequest{ +// +// PerGoroutineRelabelerSuite +// + +type PerGoroutineRelabelerSuite struct { + suite.Suite + baseCtx context.Context + options cppbridge.RelabelerOptions + hlimits cppbridge.WALHashdexLimits + rCfgs []*cppbridge.RelabelConfig + inputLss *cppbridge.LabelSetStorage + targetLss *cppbridge.LabelSetStorage + numberOfShards uint16 +} + +func TestPerGoroutineRelabelerSuite(t *testing.T) { + suite.Run(t, new(PerGoroutineRelabelerSuite)) +} + +func (s *PerGoroutineRelabelerSuite) SetupSuite() { + s.baseCtx = context.Background() + s.hlimits = cppbridge.DefaultWALHashdexLimits() + s.rCfgs = []*cppbridge.RelabelConfig{ + { + SourceLabels: []string{"job"}, + Regex: "abc", + Action: cppbridge.Keep, + }, + } + s.numberOfShards = 1 +} + +func (s *PerGoroutineRelabelerSuite) SetupTest() { + s.options = cppbridge.RelabelerOptions{} + s.inputLss = cppbridge.NewLssStorage() + s.targetLss = cppbridge.NewQueryableLssStorage() +} + +func (s *PerGoroutineRelabelerSuite) TestRelabeling() { + h, err := s.makeSnappyProtobufHashdex(&prompb.WriteRequest{ Timeseries: []prompb.TimeSeries{ { Labels: []prompb.Label{ @@ -624,43 +662,25 @@ func (s *RelabelerSuite) TestPerGoroutineRelabeler() { }, }, }, - } - data, err := wr.Marshal() - s.Require().NoError(err) - - rCfgs := []*cppbridge.RelabelConfig{ - { - SourceLabels: []string{"job"}, - Regex: "abc", - Action: cppbridge.Keep, - }, - } - - inputLss := cppbridge.NewLssStorage() - targetLss := cppbridge.NewQueryableLssStorage() - - var numberOfShards uint16 = 1 - - hlimits := cppbridge.DefaultWALHashdexLimits() - h, err := cppbridge.NewWALSnappyProtobufHashdex(snappy.Encode(nil, data), hlimits) + }) s.Require().NoError(err) - shardsInnerSeries := cppbridge.NewShardsInnerSeries(numberOfShards) - shardsRelabeledSeries := cppbridge.NewShardsRelabeledSeries(numberOfShards) + shardsInnerSeries := cppbridge.NewShardsInnerSeries(s.numberOfShards) + shardsRelabeledSeries := cppbridge.NewShardsRelabeledSeries(s.numberOfShards) - statelessRelabeler, err := cppbridge.NewStatelessRelabeler(rCfgs) + statelessRelabeler, err := cppbridge.NewStatelessRelabeler(s.rCfgs) s.Require().NoError(err) state := cppbridge.NewStateV2WithoutLock() state.SetRelabelerOptions(&s.options) state.SetStatelessRelabeler(statelessRelabeler) - state.Reconfigure(0, numberOfShards) + state.Reconfigure(0, s.numberOfShards) - pgr := cppbridge.NewPerGoroutineRelabeler(numberOfShards, 0) + pgr := cppbridge.NewPerGoroutineRelabeler(s.numberOfShards, 0) stats, hasReallocations, err := pgr.Relabeling( s.baseCtx, - inputLss, - targetLss, + s.inputLss, + s.targetLss, state, h, shardsInnerSeries, @@ -671,8 +691,8 @@ func (s *RelabelerSuite) TestPerGoroutineRelabeler() { s.True(hasReallocations) } -func (s *RelabelerSuite) TestPerGoroutineRelabelerDrop() { - wr := prompb.WriteRequest{ +func (s *PerGoroutineRelabelerSuite) TestRelabelingDrop() { + h, err := s.makeSnappyProtobufHashdex(&prompb.WriteRequest{ Timeseries: []prompb.TimeSeries{ { Labels: []prompb.Label{ @@ -692,43 +712,25 @@ func (s *RelabelerSuite) TestPerGoroutineRelabelerDrop() { }, }, }, - } - data, err := wr.Marshal() - s.Require().NoError(err) - - rCfgs := []*cppbridge.RelabelConfig{ - { - SourceLabels: []string{"job"}, - Regex: "abc", - Action: cppbridge.Keep, - }, - } - - inputLss := cppbridge.NewLssStorage() - targetLss := cppbridge.NewQueryableLssStorage() - - var numberOfShards uint16 = 1 - - hlimits := cppbridge.DefaultWALHashdexLimits() - h, err := cppbridge.NewWALSnappyProtobufHashdex(snappy.Encode(nil, data), hlimits) + }) s.Require().NoError(err) - shardsInnerSeries := cppbridge.NewShardsInnerSeries(numberOfShards) - shardsRelabeledSeries := cppbridge.NewShardsRelabeledSeries(numberOfShards) + shardsInnerSeries := cppbridge.NewShardsInnerSeries(s.numberOfShards) + shardsRelabeledSeries := cppbridge.NewShardsRelabeledSeries(s.numberOfShards) - statelessRelabeler, err := cppbridge.NewStatelessRelabeler(rCfgs) + statelessRelabeler, err := cppbridge.NewStatelessRelabeler(s.rCfgs) s.Require().NoError(err) state := cppbridge.NewStateV2WithoutLock() state.SetRelabelerOptions(&s.options) state.SetStatelessRelabeler(statelessRelabeler) - state.Reconfigure(0, numberOfShards) + state.Reconfigure(0, s.numberOfShards) - pgr := cppbridge.NewPerGoroutineRelabeler(numberOfShards, 0) + pgr := cppbridge.NewPerGoroutineRelabeler(s.numberOfShards, 0) stats, hasReallocations, err := pgr.Relabeling( s.baseCtx, - inputLss, - targetLss, + s.inputLss, + s.targetLss, state, h, shardsInnerSeries, @@ -739,8 +741,8 @@ func (s *RelabelerSuite) TestPerGoroutineRelabelerDrop() { s.True(hasReallocations) } -func (s *RelabelerSuite) TestPerGoroutineRelabelerFromCacheTrue() { - wr := prompb.WriteRequest{ +func (s *PerGoroutineRelabelerSuite) TestRelabelingFromCacheTrue() { + h, err := s.makeSnappyProtobufHashdex(&prompb.WriteRequest{ Timeseries: []prompb.TimeSeries{ { Labels: []prompb.Label{ @@ -753,43 +755,25 @@ func (s *RelabelerSuite) TestPerGoroutineRelabelerFromCacheTrue() { }, }, }, - } - data, err := wr.Marshal() - s.Require().NoError(err) - - rCfgs := []*cppbridge.RelabelConfig{ - { - SourceLabels: []string{"job"}, - Regex: "abc", - Action: cppbridge.Keep, - }, - } - - inputLss := cppbridge.NewLssStorage() - targetLss := cppbridge.NewQueryableLssStorage() - - var numberOfShards uint16 = 1 - - hlimits := cppbridge.DefaultWALHashdexLimits() - h, err := cppbridge.NewWALSnappyProtobufHashdex(snappy.Encode(nil, data), hlimits) + }) s.Require().NoError(err) - shardsInnerSeries := cppbridge.NewShardsInnerSeries(numberOfShards) - shardsRelabeledSeries := cppbridge.NewShardsRelabeledSeries(numberOfShards) + shardsInnerSeries := cppbridge.NewShardsInnerSeries(s.numberOfShards) + shardsRelabeledSeries := cppbridge.NewShardsRelabeledSeries(s.numberOfShards) - statelessRelabeler, err := cppbridge.NewStatelessRelabeler(rCfgs) + statelessRelabeler, err := cppbridge.NewStatelessRelabeler(s.rCfgs) s.Require().NoError(err) state := cppbridge.NewStateV2WithoutLock() state.SetStatelessRelabeler(statelessRelabeler) state.SetRelabelerOptions(&s.options) - state.Reconfigure(0, numberOfShards) + state.Reconfigure(0, s.numberOfShards) - pgr := cppbridge.NewPerGoroutineRelabeler(numberOfShards, 0) + pgr := cppbridge.NewPerGoroutineRelabeler(s.numberOfShards, 0) stats, hasReallocations, err := pgr.Relabeling( s.baseCtx, - inputLss, - targetLss, + s.inputLss, + s.targetLss, state, h, shardsInnerSeries, @@ -801,8 +785,8 @@ func (s *RelabelerSuite) TestPerGoroutineRelabelerFromCacheTrue() { stats, ok, err := pgr.RelabelingFromCache( s.baseCtx, - inputLss, - targetLss, + s.inputLss, + s.targetLss, state, h, shardsInnerSeries, @@ -812,8 +796,8 @@ func (s *RelabelerSuite) TestPerGoroutineRelabelerFromCacheTrue() { s.True(ok) } -func (s *RelabelerSuite) TestPerGoroutineRelabelerFromCacheFalse() { - wr := prompb.WriteRequest{ +func (s *PerGoroutineRelabelerSuite) TestRelabelingFromCacheFalse() { + h, err := s.makeSnappyProtobufHashdex(&prompb.WriteRequest{ Timeseries: []prompb.TimeSeries{ { Labels: []prompb.Label{ @@ -835,29 +819,19 @@ func (s *RelabelerSuite) TestPerGoroutineRelabelerFromCacheFalse() { }, }, }, - } - data, err := wr.Marshal() - s.Require().NoError(err) - - inputLss := cppbridge.NewLssStorage() - targetLss := cppbridge.NewQueryableLssStorage() - - var numberOfShards uint16 = 1 - - hlimits := cppbridge.DefaultWALHashdexLimits() - h, err := cppbridge.NewWALSnappyProtobufHashdex(snappy.Encode(nil, data), hlimits) + }) s.Require().NoError(err) - shardsInnerSeries := cppbridge.NewShardsInnerSeries(numberOfShards) + shardsInnerSeries := cppbridge.NewShardsInnerSeries(s.numberOfShards) state := cppbridge.NewStateV2WithoutLock() state.SetRelabelerOptions(&s.options) - state.Reconfigure(0, numberOfShards) + state.Reconfigure(0, s.numberOfShards) - pgr := cppbridge.NewPerGoroutineRelabeler(numberOfShards, 0) + pgr := cppbridge.NewPerGoroutineRelabeler(s.numberOfShards, 0) stats, ok, err := pgr.RelabelingFromCache( s.baseCtx, - inputLss, - targetLss, + s.inputLss, + s.targetLss, state, h, shardsInnerSeries, @@ -867,9 +841,9 @@ func (s *RelabelerSuite) TestPerGoroutineRelabelerFromCacheFalse() { s.False(ok) } -func (s *RelabelerSuite) TestPerGoroutineRelabelerFromCachePartially() { +func (s *PerGoroutineRelabelerSuite) TestRelabelingFromCachePartially() { ts := time.Now().UnixMilli() - wr1 := prompb.WriteRequest{ + h1, err := s.makeSnappyProtobufHashdex(&prompb.WriteRequest{ Timeseries: []prompb.TimeSeries{ { Labels: []prompb.Label{ @@ -948,15 +922,11 @@ func (s *RelabelerSuite) TestPerGoroutineRelabelerFromCachePartially() { }, }, }, - } - data1, err := wr1.Marshal() - s.Require().NoError(err) - hlimits := cppbridge.DefaultWALHashdexLimits() - h1, err := cppbridge.NewWALSnappyProtobufHashdex(snappy.Encode(nil, data1), hlimits) + }) s.Require().NoError(err) ts += 6000 - wr2 := prompb.WriteRequest{ + h2, err := s.makeSnappyProtobufHashdex(&prompb.WriteRequest{ Timeseries: []prompb.TimeSeries{ { Labels: []prompb.Label{ @@ -1009,41 +979,25 @@ func (s *RelabelerSuite) TestPerGoroutineRelabelerFromCachePartially() { }, }, }, - } - data2, err := wr2.Marshal() - s.Require().NoError(err) - h2, err := cppbridge.NewWALSnappyProtobufHashdex(snappy.Encode(nil, data2), hlimits) + }) s.Require().NoError(err) - rCfgs := []*cppbridge.RelabelConfig{ - { - SourceLabels: []string{"job"}, - Regex: "abc", - Action: cppbridge.Keep, - }, - } - - inputLss := cppbridge.NewLssStorage() - targetLss := cppbridge.NewQueryableLssStorage() - - var numberOfShards uint16 = 1 + shardsInnerSeries := cppbridge.NewShardsInnerSeries(s.numberOfShards) + shardsRelabeledSeries := cppbridge.NewShardsRelabeledSeries(s.numberOfShards) - shardsInnerSeries := cppbridge.NewShardsInnerSeries(numberOfShards) - shardsRelabeledSeries := cppbridge.NewShardsRelabeledSeries(numberOfShards) - - statelessRelabeler, err := cppbridge.NewStatelessRelabeler(rCfgs) + statelessRelabeler, err := cppbridge.NewStatelessRelabeler(s.rCfgs) s.Require().NoError(err) state := cppbridge.NewStateV2WithoutLock() state.SetRelabelerOptions(&s.options) state.SetStatelessRelabeler(statelessRelabeler) - state.Reconfigure(0, numberOfShards) + state.Reconfigure(0, s.numberOfShards) - pgr := cppbridge.NewPerGoroutineRelabeler(numberOfShards, 0) + pgr := cppbridge.NewPerGoroutineRelabeler(s.numberOfShards, 0) stats, hasReallocations, err := pgr.Relabeling( s.baseCtx, - inputLss, - targetLss, + s.inputLss, + s.targetLss, state, h1, shardsInnerSeries, @@ -1053,11 +1007,11 @@ func (s *RelabelerSuite) TestPerGoroutineRelabelerFromCachePartially() { s.Equal(cppbridge.RelabelerStats{4, 4, 4}, stats) s.True(hasReallocations) - shardsInnerSeries = cppbridge.NewShardsInnerSeries(numberOfShards) + shardsInnerSeries = cppbridge.NewShardsInnerSeries(s.numberOfShards) stats, ok, err := pgr.RelabelingFromCache( s.baseCtx, - inputLss, - targetLss, + s.inputLss, + s.targetLss, state, h2, shardsInnerSeries, @@ -1069,8 +1023,8 @@ func (s *RelabelerSuite) TestPerGoroutineRelabelerFromCachePartially() { stats, _, err = pgr.Relabeling( s.baseCtx, - inputLss, - targetLss, + s.inputLss, + s.targetLss, state, h2, shardsInnerSeries, @@ -1081,8 +1035,8 @@ func (s *RelabelerSuite) TestPerGoroutineRelabelerFromCachePartially() { s.Equal(uint64(5), shardsInnerSeries[0].Size()) } -func (s *RelabelerSuite) TestPerGoroutineRelabelerTransition() { - wr := prompb.WriteRequest{ +func (s *PerGoroutineRelabelerSuite) TestRelabelingTransition() { + h, err := s.makeSnappyProtobufHashdex(&prompb.WriteRequest{ Timeseries: []prompb.TimeSeries{ { Labels: []prompb.Label{ @@ -1104,30 +1058,20 @@ func (s *RelabelerSuite) TestPerGoroutineRelabelerTransition() { }, }, }, - } - data, err := wr.Marshal() - s.Require().NoError(err) - - inputLss := cppbridge.NewLssStorage() - targetLss := cppbridge.NewQueryableLssStorage() - - var numberOfShards uint16 = 1 - - hlimits := cppbridge.DefaultWALHashdexLimits() - h, err := cppbridge.NewWALSnappyProtobufHashdex(snappy.Encode(nil, data), hlimits) + }) s.Require().NoError(err) - shardsInnerSeries := cppbridge.NewShardsInnerSeries(numberOfShards) - shardsRelabeledSeries := cppbridge.NewShardsRelabeledSeries(numberOfShards) + shardsInnerSeries := cppbridge.NewShardsInnerSeries(s.numberOfShards) + shardsRelabeledSeries := cppbridge.NewShardsRelabeledSeries(s.numberOfShards) state := cppbridge.NewTransitionStateV2() - state.Reconfigure(0, numberOfShards) + state.Reconfigure(0, s.numberOfShards) - pgr := cppbridge.NewPerGoroutineRelabeler(numberOfShards, 0) + pgr := cppbridge.NewPerGoroutineRelabeler(s.numberOfShards, 0) stats, hasReallocations, err := pgr.Relabeling( s.baseCtx, - inputLss, - targetLss, + s.inputLss, + s.targetLss, state, h, shardsInnerSeries, @@ -1138,8 +1082,8 @@ func (s *RelabelerSuite) TestPerGoroutineRelabelerTransition() { s.True(hasReallocations) } -func (s *RelabelerSuite) TestPerGoroutineRelabelerFromCacheTrueTransition() { - wr := prompb.WriteRequest{ +func (s *PerGoroutineRelabelerSuite) TestRelabelingFromCacheTrueTransition() { + h, err := s.makeSnappyProtobufHashdex(&prompb.WriteRequest{ Timeseries: []prompb.TimeSeries{ { Labels: []prompb.Label{ @@ -1152,30 +1096,20 @@ func (s *RelabelerSuite) TestPerGoroutineRelabelerFromCacheTrueTransition() { }, }, }, - } - data, err := wr.Marshal() - s.Require().NoError(err) - - inputLss := cppbridge.NewLssStorage() - targetLss := cppbridge.NewQueryableLssStorage() - - var numberOfShards uint16 = 1 - - hlimits := cppbridge.DefaultWALHashdexLimits() - h, err := cppbridge.NewWALSnappyProtobufHashdex(snappy.Encode(nil, data), hlimits) + }) s.Require().NoError(err) - shardsInnerSeries := cppbridge.NewShardsInnerSeries(numberOfShards) - shardsRelabeledSeries := cppbridge.NewShardsRelabeledSeries(numberOfShards) + shardsInnerSeries := cppbridge.NewShardsInnerSeries(s.numberOfShards) + shardsRelabeledSeries := cppbridge.NewShardsRelabeledSeries(s.numberOfShards) state := cppbridge.NewTransitionStateV2() - state.Reconfigure(0, numberOfShards) + state.Reconfigure(0, s.numberOfShards) - pgr := cppbridge.NewPerGoroutineRelabeler(numberOfShards, 0) + pgr := cppbridge.NewPerGoroutineRelabeler(s.numberOfShards, 0) stats, hasReallocations, err := pgr.Relabeling( s.baseCtx, - inputLss, - targetLss, + s.inputLss, + s.targetLss, state, h, shardsInnerSeries, @@ -1187,8 +1121,8 @@ func (s *RelabelerSuite) TestPerGoroutineRelabelerFromCacheTrueTransition() { stats, ok, err := pgr.RelabelingFromCache( s.baseCtx, - inputLss, - targetLss, + s.inputLss, + s.targetLss, state, h, shardsInnerSeries, @@ -1198,8 +1132,8 @@ func (s *RelabelerSuite) TestPerGoroutineRelabelerFromCacheTrueTransition() { s.True(ok) } -func (s *RelabelerSuite) TestPerGoroutineRelabelerFromCacheFalseTransition() { - wr := prompb.WriteRequest{ +func (s *PerGoroutineRelabelerSuite) TestRelabelingFromCacheFalseTransition() { + h, err := s.makeSnappyProtobufHashdex(&prompb.WriteRequest{ Timeseries: []prompb.TimeSeries{ { Labels: []prompb.Label{ @@ -1221,28 +1155,18 @@ func (s *RelabelerSuite) TestPerGoroutineRelabelerFromCacheFalseTransition() { }, }, }, - } - data, err := wr.Marshal() + }) s.Require().NoError(err) - inputLss := cppbridge.NewLssStorage() - targetLss := cppbridge.NewQueryableLssStorage() - - var numberOfShards uint16 = 1 - - hlimits := cppbridge.DefaultWALHashdexLimits() - h, err := cppbridge.NewWALSnappyProtobufHashdex(snappy.Encode(nil, data), hlimits) - s.Require().NoError(err) - - shardsInnerSeries := cppbridge.NewShardsInnerSeries(numberOfShards) + shardsInnerSeries := cppbridge.NewShardsInnerSeries(s.numberOfShards) state := cppbridge.NewTransitionStateV2() - state.Reconfigure(0, numberOfShards) + state.Reconfigure(0, s.numberOfShards) - pgr := cppbridge.NewPerGoroutineRelabeler(numberOfShards, 0) + pgr := cppbridge.NewPerGoroutineRelabeler(s.numberOfShards, 0) stats, ok, err := pgr.RelabelingFromCache( s.baseCtx, - inputLss, - targetLss, + s.inputLss, + s.targetLss, state, h, shardsInnerSeries, @@ -1252,9 +1176,9 @@ func (s *RelabelerSuite) TestPerGoroutineRelabelerFromCacheFalseTransition() { s.False(ok) } -func (s *RelabelerSuite) TestPerGoroutineRelabelerFromCachePartiallyTransition() { +func (s *PerGoroutineRelabelerSuite) TestRelabelingFromCachePartiallyTransition() { ts := time.Now().UnixMilli() - wr1 := prompb.WriteRequest{ + h1, err := s.makeSnappyProtobufHashdex(&prompb.WriteRequest{ Timeseries: []prompb.TimeSeries{ { Labels: []prompb.Label{ @@ -1333,15 +1257,11 @@ func (s *RelabelerSuite) TestPerGoroutineRelabelerFromCachePartiallyTransition() }, }, }, - } - data1, err := wr1.Marshal() - s.Require().NoError(err) - hlimits := cppbridge.DefaultWALHashdexLimits() - h1, err := cppbridge.NewWALSnappyProtobufHashdex(snappy.Encode(nil, data1), hlimits) + }) s.Require().NoError(err) ts += 6000 - wr2 := prompb.WriteRequest{ + h2, err := s.makeSnappyProtobufHashdex(&prompb.WriteRequest{ Timeseries: []prompb.TimeSeries{ { Labels: []prompb.Label{ @@ -1394,28 +1314,20 @@ func (s *RelabelerSuite) TestPerGoroutineRelabelerFromCachePartiallyTransition() }, }, }, - } - data2, err := wr2.Marshal() - s.Require().NoError(err) - h2, err := cppbridge.NewWALSnappyProtobufHashdex(snappy.Encode(nil, data2), hlimits) + }) s.Require().NoError(err) - inputLss := cppbridge.NewLssStorage() - targetLss := cppbridge.NewQueryableLssStorage() - - var numberOfShards uint16 = 1 - - shardsInnerSeries := cppbridge.NewShardsInnerSeries(numberOfShards) - shardsRelabeledSeries := cppbridge.NewShardsRelabeledSeries(numberOfShards) + shardsInnerSeries := cppbridge.NewShardsInnerSeries(s.numberOfShards) + shardsRelabeledSeries := cppbridge.NewShardsRelabeledSeries(s.numberOfShards) state := cppbridge.NewTransitionStateV2() - state.Reconfigure(0, numberOfShards) + state.Reconfigure(0, s.numberOfShards) - pgr := cppbridge.NewPerGoroutineRelabeler(numberOfShards, 0) + pgr := cppbridge.NewPerGoroutineRelabeler(s.numberOfShards, 0) stats, hasReallocations, err := pgr.Relabeling( s.baseCtx, - inputLss, - targetLss, + s.inputLss, + s.targetLss, state, h1, shardsInnerSeries, @@ -1425,11 +1337,11 @@ func (s *RelabelerSuite) TestPerGoroutineRelabelerFromCachePartiallyTransition() s.Equal(cppbridge.RelabelerStats{8, 8, 0}, stats) s.True(hasReallocations) - shardsInnerSeries = cppbridge.NewShardsInnerSeries(numberOfShards) + shardsInnerSeries = cppbridge.NewShardsInnerSeries(s.numberOfShards) stats, ok, err := pgr.RelabelingFromCache( s.baseCtx, - inputLss, - targetLss, + s.inputLss, + s.targetLss, state, h2, shardsInnerSeries, @@ -1441,8 +1353,8 @@ func (s *RelabelerSuite) TestPerGoroutineRelabelerFromCachePartiallyTransition() stats, _, err = pgr.Relabeling( s.baseCtx, - inputLss, - targetLss, + s.inputLss, + s.targetLss, state, h2, shardsInnerSeries, @@ -1453,6 +1365,425 @@ func (s *RelabelerSuite) TestPerGoroutineRelabelerFromCachePartiallyTransition() s.Equal(uint64(5), shardsInnerSeries[0].Size()) } +func (s *PerGoroutineRelabelerSuite) TestRelabelingWithStalenans() { + h, err := s.makeSnappyProtobufHashdex(&prompb.WriteRequest{ + Timeseries: []prompb.TimeSeries{ + { + Labels: []prompb.Label{ + {Name: "__name__", Value: "value"}, + {Name: "job", Value: "abc"}, + {Name: "instance", Value: "value1"}, + }, + Samples: []prompb.Sample{ + {Value: 0.1, Timestamp: time.Now().UnixMilli()}, + }, + }, + { + Labels: []prompb.Label{ + {Name: "__name__", Value: "value"}, + {Name: "instance", Value: "value1"}, + }, + Samples: []prompb.Sample{ + {Value: 0.1, Timestamp: time.Now().UnixMilli()}, + }, + }, + }, + }) + s.Require().NoError(err) + + shardsInnerSeries := cppbridge.NewShardsInnerSeries(s.numberOfShards) + shardsRelabeledSeries := cppbridge.NewShardsRelabeledSeries(s.numberOfShards) + + statelessRelabeler, err := cppbridge.NewStatelessRelabeler(s.rCfgs) + s.Require().NoError(err) + + state := cppbridge.NewStateV2WithoutLock() + state.SetRelabelerOptions(&s.options) + state.SetStatelessRelabeler(statelessRelabeler) + state.EnableTrackStaleness() + state.Reconfigure(0, s.numberOfShards) + + pgr := cppbridge.NewPerGoroutineRelabeler(s.numberOfShards, 0) + stats, hasReallocations, err := pgr.Relabeling( + s.baseCtx, + s.inputLss, + s.targetLss, + state, + h, + shardsInnerSeries, + shardsRelabeledSeries, + ) + s.Require().NoError(err) + s.Equal(cppbridge.RelabelerStats{1, 1, 1}, stats) + s.True(hasReallocations) + s.Equal(uint64(1), shardsInnerSeries[0].Size()) + + h, err = s.makeSnappyProtobufHashdex(&prompb.WriteRequest{}) + s.Require().NoError(err) + + shardsInnerSeries = cppbridge.NewShardsInnerSeries(s.numberOfShards) + shardsRelabeledSeries = cppbridge.NewShardsRelabeledSeries(s.numberOfShards) + stats, hasReallocations, err = pgr.Relabeling( + s.baseCtx, + s.inputLss, + s.targetLss, + state, + h, + shardsInnerSeries, + shardsRelabeledSeries, + ) + s.Require().NoError(err) + s.Equal(cppbridge.RelabelerStats{0, 0, 0}, stats) + s.False(hasReallocations) + s.Equal(uint64(1), shardsInnerSeries[0].Size()) +} + +func (s *PerGoroutineRelabelerSuite) TestRelabelingWithStalenansFromCacheTrue() { + h, err := s.makeSnappyProtobufHashdex(&prompb.WriteRequest{ + Timeseries: []prompb.TimeSeries{ + { + Labels: []prompb.Label{ + {Name: "__name__", Value: "value"}, + {Name: "job", Value: "abc"}, + {Name: "instance", Value: "value1"}, + }, + Samples: []prompb.Sample{ + {Value: 0.1, Timestamp: time.Now().UnixMilli()}, + }, + }, + }, + }) + s.Require().NoError(err) + + shardsInnerSeries := cppbridge.NewShardsInnerSeries(s.numberOfShards) + shardsRelabeledSeries := cppbridge.NewShardsRelabeledSeries(s.numberOfShards) + + statelessRelabeler, err := cppbridge.NewStatelessRelabeler(s.rCfgs) + s.Require().NoError(err) + + state := cppbridge.NewStateV2WithoutLock() + state.SetStatelessRelabeler(statelessRelabeler) + state.SetRelabelerOptions(&s.options) + state.EnableTrackStaleness() + state.Reconfigure(0, s.numberOfShards) + + pgr := cppbridge.NewPerGoroutineRelabeler(s.numberOfShards, 0) + stats, hasReallocations, err := pgr.Relabeling( + s.baseCtx, + s.inputLss, + s.targetLss, + state, + h, + shardsInnerSeries, + shardsRelabeledSeries, + ) + s.Require().NoError(err) + s.Equal(cppbridge.RelabelerStats{1, 1, 0}, stats) + s.True(hasReallocations) + + stats, ok, err := pgr.RelabelingFromCache( + s.baseCtx, + s.inputLss, + s.targetLss, + state, + h, + shardsInnerSeries, + ) + s.Require().NoError(err) + s.Equal(cppbridge.RelabelerStats{1, 0, 0}, stats) + s.True(ok) + + h, err = s.makeSnappyProtobufHashdex(&prompb.WriteRequest{}) + s.Require().NoError(err) + + shardsInnerSeries = cppbridge.NewShardsInnerSeries(s.numberOfShards) + shardsRelabeledSeries = cppbridge.NewShardsRelabeledSeries(s.numberOfShards) + stats, hasReallocations, err = pgr.Relabeling( + s.baseCtx, + s.inputLss, + s.targetLss, + state, + h, + shardsInnerSeries, + shardsRelabeledSeries, + ) + s.Require().NoError(err) + s.Equal(cppbridge.RelabelerStats{0, 0, 0}, stats) + s.False(hasReallocations) + s.Equal(uint64(1), shardsInnerSeries[0].Size()) +} + +func (s *PerGoroutineRelabelerSuite) TestRelabelingWithStalenansFromCacheFalse() { + h, err := s.makeSnappyProtobufHashdex(&prompb.WriteRequest{ + Timeseries: []prompb.TimeSeries{ + { + Labels: []prompb.Label{ + {Name: "__name__", Value: "value"}, + {Name: "job", Value: "abc"}, + {Name: "instance", Value: "value1"}, + }, + Samples: []prompb.Sample{ + {Value: 0.1, Timestamp: time.Now().UnixMilli()}, + }, + }, + { + Labels: []prompb.Label{ + {Name: "__name__", Value: "value"}, + {Name: "instance", Value: "value1"}, + }, + Samples: []prompb.Sample{ + {Value: 0.1, Timestamp: time.Now().UnixMilli()}, + }, + }, + }, + }) + s.Require().NoError(err) + + shardsInnerSeries := cppbridge.NewShardsInnerSeries(s.numberOfShards) + state := cppbridge.NewStateV2WithoutLock() + state.SetRelabelerOptions(&s.options) + state.EnableTrackStaleness() + state.Reconfigure(0, s.numberOfShards) + + pgr := cppbridge.NewPerGoroutineRelabeler(s.numberOfShards, 0) + stats, ok, err := pgr.RelabelingFromCache( + s.baseCtx, + s.inputLss, + s.targetLss, + state, + h, + shardsInnerSeries, + ) + s.Require().NoError(err) + s.Equal(cppbridge.RelabelerStats{0, 0, 0}, stats) + s.False(ok) +} + +func (s *PerGoroutineRelabelerSuite) TestRelabelingWithStalenansFromCachePartially() { + ts := time.Now().UnixMilli() + h1, err := s.makeSnappyProtobufHashdex(&prompb.WriteRequest{ + Timeseries: []prompb.TimeSeries{ + { + Labels: []prompb.Label{ + {Name: "__name__", Value: "value0"}, + {Name: "job", Value: "abc"}, + {Name: "instance", Value: "value0"}, + }, + Samples: []prompb.Sample{ + {Value: 0.1, Timestamp: ts}, + }, + }, + { + Labels: []prompb.Label{ + {Name: "__name__", Value: "value1"}, + {Name: "job", Value: "abc"}, + {Name: "instance", Value: "value1"}, + }, + Samples: []prompb.Sample{ + {Value: 0.1, Timestamp: ts}, + }, + }, + { + Labels: []prompb.Label{ + {Name: "__name__", Value: "value2"}, + {Name: "job", Value: "abc"}, + {Name: "instance", Value: "value2"}, + }, + Samples: []prompb.Sample{ + {Value: 0.1, Timestamp: ts}, + }, + }, + { + Labels: []prompb.Label{ + {Name: "__name__", Value: "value3"}, + {Name: "job", Value: "abc"}, + {Name: "instance", Value: "value3"}, + }, + Samples: []prompb.Sample{ + {Value: 0.1, Timestamp: ts}, + }, + }, + { + Labels: []prompb.Label{ + {Name: "__name__", Value: "value3"}, + {Name: "instance", Value: "value3"}, + }, + Samples: []prompb.Sample{ + {Value: 0.1, Timestamp: ts}, + }, + }, + { + Labels: []prompb.Label{ + {Name: "__name__", Value: "value4"}, + {Name: "instance", Value: "value4"}, + }, + Samples: []prompb.Sample{ + {Value: 0.1, Timestamp: ts}, + }, + }, + { + Labels: []prompb.Label{ + {Name: "__name__", Value: "value5"}, + {Name: "instance", Value: "value5"}, + }, + Samples: []prompb.Sample{ + {Value: 0.1, Timestamp: ts}, + }, + }, + { + Labels: []prompb.Label{ + {Name: "__name__", Value: "value6"}, + {Name: "instance", Value: "value6"}, + }, + Samples: []prompb.Sample{ + {Value: 0.1, Timestamp: ts}, + }, + }, + }, + }) + s.Require().NoError(err) + + ts += 6000 + h2, err := s.makeSnappyProtobufHashdex(&prompb.WriteRequest{ + Timeseries: []prompb.TimeSeries{ + { + Labels: []prompb.Label{ + {Name: "__name__", Value: "value0"}, + {Name: "job", Value: "abc"}, + {Name: "instance", Value: "value0"}, + }, + Samples: []prompb.Sample{ + {Value: 0.1, Timestamp: ts}, + }, + }, + { + Labels: []prompb.Label{ + {Name: "__name__", Value: "value1"}, + {Name: "job", Value: "abc"}, + {Name: "instance", Value: "value1"}, + }, + Samples: []prompb.Sample{ + {Value: 0.1, Timestamp: ts}, + }, + }, + { + Labels: []prompb.Label{ + {Name: "__name__", Value: "value2"}, + {Name: "job", Value: "abc"}, + {Name: "instance", Value: "value2"}, + }, + Samples: []prompb.Sample{ + {Value: 0.1, Timestamp: ts}, + }, + }, + { + Labels: []prompb.Label{ + {Name: "__name__", Value: "value3"}, + {Name: "job", Value: "abc"}, + {Name: "instance", Value: "value3"}, + }, + Samples: []prompb.Sample{ + {Value: 0.1, Timestamp: ts}, + }, + }, + { + Labels: []prompb.Label{ + {Name: "__name__", Value: "value4"}, + {Name: "job", Value: "abc"}, + {Name: "instance", Value: "value4"}, + }, + Samples: []prompb.Sample{ + {Value: 0.1, Timestamp: ts}, + }, + }, + }, + }) + s.Require().NoError(err) + + shardsInnerSeries := cppbridge.NewShardsInnerSeries(s.numberOfShards) + shardsRelabeledSeries := cppbridge.NewShardsRelabeledSeries(s.numberOfShards) + + statelessRelabeler, err := cppbridge.NewStatelessRelabeler(s.rCfgs) + s.Require().NoError(err) + + state := cppbridge.NewStateV2WithoutLock() + state.SetRelabelerOptions(&s.options) + state.SetStatelessRelabeler(statelessRelabeler) + state.EnableTrackStaleness() + state.Reconfigure(0, s.numberOfShards) + + pgr := cppbridge.NewPerGoroutineRelabeler(s.numberOfShards, 0) + stats, hasReallocations, err := pgr.Relabeling( + s.baseCtx, + s.inputLss, + s.targetLss, + state, + h1, + shardsInnerSeries, + shardsRelabeledSeries, + ) + s.Require().NoError(err) + s.Equal(cppbridge.RelabelerStats{4, 4, 4}, stats) + s.True(hasReallocations) + + shardsInnerSeries = cppbridge.NewShardsInnerSeries(s.numberOfShards) + stats, ok, err := pgr.RelabelingFromCache( + s.baseCtx, + s.inputLss, + s.targetLss, + state, + h2, + shardsInnerSeries, + ) + s.Require().NoError(err) + s.Equal(cppbridge.RelabelerStats{4, 0, 0}, stats) + s.False(ok) + s.Equal(uint64(4), shardsInnerSeries[0].Size()) + + stats, _, err = pgr.Relabeling( + s.baseCtx, + s.inputLss, + s.targetLss, + state, + h2, + shardsInnerSeries, + shardsRelabeledSeries, + ) + s.Require().NoError(err) + s.Equal(cppbridge.RelabelerStats{1, 1, 0}, stats) + s.Equal(uint64(5), shardsInnerSeries[0].Size()) + + h, err := s.makeSnappyProtobufHashdex(&prompb.WriteRequest{}) + s.Require().NoError(err) + + shardsInnerSeries = cppbridge.NewShardsInnerSeries(s.numberOfShards) + shardsRelabeledSeries = cppbridge.NewShardsRelabeledSeries(s.numberOfShards) + stats, hasReallocations, err = pgr.Relabeling( + s.baseCtx, + s.inputLss, + s.targetLss, + state, + h, + shardsInnerSeries, + shardsRelabeledSeries, + ) + s.Require().NoError(err) + s.Equal(cppbridge.RelabelerStats{0, 0, 0}, stats) + s.False(hasReallocations) + s.Equal(uint64(5), shardsInnerSeries[0].Size()) +} + +func (s *PerGoroutineRelabelerSuite) makeSnappyProtobufHashdex( + wr *prompb.WriteRequest, +) (cppbridge.ShardedData, error) { + data, err := wr.Marshal() + if err != nil { + return nil, err + } + + return cppbridge.NewWALSnappyProtobufHashdex(snappy.Encode(nil, data), s.hlimits) +} + // // StateV2Suite // @@ -1466,11 +1797,11 @@ func TestStateV2Suite(t *testing.T) { } func (s *StateV2Suite) TestInitState() { - s.testInitState(cppbridge.NewStateV2()) - s.testInitState(cppbridge.NewStateV2WithoutLock()) + s.initState(cppbridge.NewStateV2()) + s.initState(cppbridge.NewStateV2WithoutLock()) } -func (s *StateV2Suite) testInitState(state *cppbridge.StateV2) { +func (s *StateV2Suite) initState(state *cppbridge.StateV2) { s.Panics(func() { state.CacheByShard(0) }) s.Equal(time.Now().UnixMilli(), state.DefTimestamp()) @@ -1485,11 +1816,11 @@ func (s *StateV2Suite) testInitState(state *cppbridge.StateV2) { } func (s *StateV2Suite) TestStateReconfigure() { - s.testStateReconfigure(cppbridge.NewStateV2()) - s.testStateReconfigure(cppbridge.NewStateV2WithoutLock()) + s.stateReconfigure(cppbridge.NewStateV2()) + s.stateReconfigure(cppbridge.NewStateV2WithoutLock()) } -func (s *StateV2Suite) testStateReconfigure(state *cppbridge.StateV2) { +func (s *StateV2Suite) stateReconfigure(state *cppbridge.StateV2) { state.Reconfigure(0, 1) s.NotNil(state.CacheByShard(0)) @@ -1498,11 +1829,11 @@ func (s *StateV2Suite) testStateReconfigure(state *cppbridge.StateV2) { } func (s *StateV2Suite) TestStateReconfigureWithoutReconfigure() { - s.testStateReconfigureWithoutReconfigure(cppbridge.NewStateV2()) - s.testStateReconfigureWithoutReconfigure(cppbridge.NewStateV2WithoutLock()) + s.stateReconfigureWithoutReconfigure(cppbridge.NewStateV2()) + s.stateReconfigureWithoutReconfigure(cppbridge.NewStateV2WithoutLock()) } -func (s *StateV2Suite) testStateReconfigureWithoutReconfigure(state *cppbridge.StateV2) { +func (s *StateV2Suite) stateReconfigureWithoutReconfigure(state *cppbridge.StateV2) { state.Reconfigure(0, 1) cache1 := state.CacheByShard(0) @@ -1515,11 +1846,11 @@ func (s *StateV2Suite) testStateReconfigureWithoutReconfigure(state *cppbridge.S } func (s *StateV2Suite) TestStateReconfigureNumberOfShards() { - s.testStateReconfigureNumberOfShards(cppbridge.NewStateV2()) - s.testStateReconfigureNumberOfShards(cppbridge.NewStateV2WithoutLock()) + s.stateReconfigureNumberOfShards(cppbridge.NewStateV2()) + s.stateReconfigureNumberOfShards(cppbridge.NewStateV2WithoutLock()) } -func (s *StateV2Suite) testStateReconfigureNumberOfShards(state *cppbridge.StateV2) { +func (s *StateV2Suite) stateReconfigureNumberOfShards(state *cppbridge.StateV2) { state.EnableTrackStaleness() state.Reconfigure(0, 2) @@ -1536,11 +1867,11 @@ func (s *StateV2Suite) testStateReconfigureNumberOfShards(state *cppbridge.State } func (s *StateV2Suite) TestStateReconfigureTrackStaleness() { - s.testStateReconfigureTrackStaleness(cppbridge.NewStateV2()) - s.testStateReconfigureTrackStaleness(cppbridge.NewStateV2WithoutLock()) + s.stateReconfigureTrackStaleness(cppbridge.NewStateV2()) + s.stateReconfigureTrackStaleness(cppbridge.NewStateV2WithoutLock()) } -func (s *StateV2Suite) testStateReconfigureTrackStaleness(state *cppbridge.StateV2) { +func (s *StateV2Suite) stateReconfigureTrackStaleness(state *cppbridge.StateV2) { state.EnableTrackStaleness() state.Reconfigure(0, 1) @@ -1550,11 +1881,11 @@ func (s *StateV2Suite) testStateReconfigureTrackStaleness(state *cppbridge.State } func (s *StateV2Suite) TestStatelessRelabeler() { - s.testStatelessRelabeler(cppbridge.NewStateV2()) - s.testStatelessRelabeler(cppbridge.NewStateV2WithoutLock()) + s.statelessRelabeler(cppbridge.NewStateV2()) + s.statelessRelabeler(cppbridge.NewStateV2WithoutLock()) } -func (s *StateV2Suite) testStatelessRelabeler(state *cppbridge.StateV2) { +func (s *StateV2Suite) statelessRelabeler(state *cppbridge.StateV2) { s.Panics(func() { state.StatelessRelabeler() }) statelessRelabeler, err := cppbridge.NewStatelessRelabeler([]*cppbridge.RelabelConfig{}) @@ -1565,11 +1896,11 @@ func (s *StateV2Suite) testStatelessRelabeler(state *cppbridge.StateV2) { } func (s *StateV2Suite) TestInitTransitionStateV2() { - s.testInitTransitionState(cppbridge.NewTransitionStateV2()) - s.testInitTransitionState(cppbridge.NewTransitionStateV2WithoutLock()) + s.initTransitionState(cppbridge.NewTransitionStateV2()) + s.initTransitionState(cppbridge.NewTransitionStateV2WithoutLock()) } -func (s *StateV2Suite) testInitTransitionState(state *cppbridge.StateV2) { +func (s *StateV2Suite) initTransitionState(state *cppbridge.StateV2) { s.True(state.IsTransition()) s.Equal(cppbridge.RelabelerOptions{}, state.RelabelerOptions()) s.Panics(func() { state.CacheByShard(0) }) @@ -1578,11 +1909,11 @@ func (s *StateV2Suite) testInitTransitionState(state *cppbridge.StateV2) { } func (s *StateV2Suite) TestStateTransitionReconfigure() { - s.testStateTransitionReconfigure(cppbridge.NewTransitionStateV2()) - s.testStateTransitionReconfigure(cppbridge.NewTransitionStateV2WithoutLock()) + s.stateTransitionReconfigure(cppbridge.NewTransitionStateV2()) + s.stateTransitionReconfigure(cppbridge.NewTransitionStateV2WithoutLock()) } -func (s *StateV2Suite) testStateTransitionReconfigure(state *cppbridge.StateV2) { +func (s *StateV2Suite) stateTransitionReconfigure(state *cppbridge.StateV2) { state.Reconfigure(0, 1) s.False(state.TrackStaleness()) @@ -1591,11 +1922,11 @@ func (s *StateV2Suite) testStateTransitionReconfigure(state *cppbridge.StateV2) } func (s *StateV2Suite) TestStateTransitionReconfigureTrackStaleness() { - s.testStateTransitionReconfigureTrackStaleness(cppbridge.NewTransitionStateV2()) - s.testStateTransitionReconfigureTrackStaleness(cppbridge.NewTransitionStateV2WithoutLock()) + s.stateTransitionReconfigureTrackStaleness(cppbridge.NewTransitionStateV2()) + s.stateTransitionReconfigureTrackStaleness(cppbridge.NewTransitionStateV2WithoutLock()) } -func (s *StateV2Suite) testStateTransitionReconfigureTrackStaleness(state *cppbridge.StateV2) { +func (s *StateV2Suite) stateTransitionReconfigureTrackStaleness(state *cppbridge.StateV2) { s.Panics(func() { state.EnableTrackStaleness() }) state.Reconfigure(0, 1) @@ -1605,11 +1936,11 @@ func (s *StateV2Suite) testStateTransitionReconfigureTrackStaleness(state *cppbr } func (s *StateV2Suite) TestStatelessRelabelerTransition() { - s.testStatelessRelabelerTransition(cppbridge.NewTransitionStateV2()) - s.testStatelessRelabelerTransition(cppbridge.NewTransitionStateV2WithoutLock()) + s.statelessRelabelerTransition(cppbridge.NewTransitionStateV2()) + s.statelessRelabelerTransition(cppbridge.NewTransitionStateV2WithoutLock()) } -func (s *StateV2Suite) testStatelessRelabelerTransition(state *cppbridge.StateV2) { +func (s *StateV2Suite) statelessRelabelerTransition(state *cppbridge.StateV2) { s.Panics(func() { state.StatelessRelabeler() }) statelessRelabeler, err := cppbridge.NewStatelessRelabeler([]*cppbridge.RelabelConfig{}) From 8cdd96a1600890bc7afe25dd36833493fb9d39b3 Mon Sep 17 00:00:00 2001 From: Alexandr Yudin Date: Tue, 7 Oct 2025 11:43:53 +0000 Subject: [PATCH 72/96] add twst wal --- pp/go/storage/appender/appender.go | 10 +- pp/go/storage/head/shard/wal/wal.go | 12 +- .../storage/head/shard/wal/wal_reader_test.go | 79 +++++ pp/go/storage/head/shard/wal/wal_test.go | 276 +++++++++++++++--- 4 files changed, 325 insertions(+), 52 deletions(-) diff --git a/pp/go/storage/appender/appender.go b/pp/go/storage/appender/appender.go index feb6a31339..f4213f983c 100644 --- a/pp/go/storage/appender/appender.go +++ b/pp/go/storage/appender/appender.go @@ -180,7 +180,7 @@ func (a Appender[TTask, TShard, THead]) Append( // inputRelabelingStage first stage - relabeling. // //revive:disable-next-line:function-length long but this is first stage. -func (a Appender[TTask, TShard, THead]) inputRelabelingStage( +func (a *Appender[TTask, TShard, THead]) inputRelabelingStage( ctx context.Context, state *cppbridge.StateV2, incomingData *DestructibleIncomingData, @@ -265,7 +265,7 @@ func (a Appender[TTask, TShard, THead]) inputRelabelingStage( } // appendRelabelerSeriesStage second stage - append to lss relabeling ls. -func (a Appender[TTask, TShard, THead]) appendRelabelerSeriesStage( +func (a *Appender[TTask, TShard, THead]) appendRelabelerSeriesStage( ctx context.Context, shardedInnerSeries *ShardedInnerSeries, shardedRelabeledSeries *ShardedRelabeledSeries, @@ -307,7 +307,7 @@ func (a Appender[TTask, TShard, THead]) appendRelabelerSeriesStage( } // updateRelabelerStateStage third stage - update state cache. -func (a Appender[TTask, TShard, THead]) updateRelabelerStateStage( +func (a *Appender[TTask, TShard, THead]) updateRelabelerStateStage( ctx context.Context, state *cppbridge.StateV2, shardedStateUpdates *ShardedStateUpdates, @@ -328,7 +328,7 @@ func (a Appender[TTask, TShard, THead]) updateRelabelerStateStage( } // appendInnerSeriesAndWriteToWal append [cppbridge.InnerSeries] to [Shard]'s to [DataStorage] and write to [Wal]. -func (a Appender[TTask, TShard, THead]) appendInnerSeriesAndWriteToWal( +func (a *Appender[TTask, TShard, THead]) appendInnerSeriesAndWriteToWal( shardedInnerSeries *ShardedInnerSeries, ) (uint32, error) { tw := task.NewTaskWaiter[TTask](2) //revive:disable-line:add-constant // 2 task for wait @@ -367,7 +367,7 @@ func (a Appender[TTask, TShard, THead]) appendInnerSeriesAndWriteToWal( return atomicLimitExhausted, tw.Wait() } -func (a Appender[TTask, TShard, THead]) resolveState(state *cppbridge.StateV2) error { +func (a *Appender[TTask, TShard, THead]) resolveState(state *cppbridge.StateV2) error { if state == nil { return errNilState } diff --git a/pp/go/storage/head/shard/wal/wal.go b/pp/go/storage/head/shard/wal/wal.go index ad0da4a1a7..56c526078b 100644 --- a/pp/go/storage/head/shard/wal/wal.go +++ b/pp/go/storage/head/shard/wal/wal.go @@ -40,15 +40,25 @@ type SegmentWriter[TSegment EncodedSegment] interface { // Encoder the minimum required Encoder implementation for a [Wal]. type Encoder[TSegment EncodedSegment] interface { + // Encode encodes the inner series into a segment. Encode(innerSeriesSlice []*cppbridge.InnerSeries) (uint32, error) + + // Finalize finalizes the encoder and returns the encoded segment. Finalize() (TSegment, error) } // EncodedSegment the minimum required Segment implementation for a [Wal]. type EncodedSegment interface { + // Size returns the size of the segment. Size() int64 + + // CRC32 returns the CRC32 of the segment. CRC32() uint32 + + // Samples returns the number of samples in the segment. Samples() uint32 + + // WriteTo implements [io.WriterTo] interface. io.WriterTo } @@ -145,7 +155,7 @@ func (w *Wal[TSegment, TWriter]) CurrentSize() int64 { // Flush wal [SegmentWriter], write all buffered data to storage. func (w *Wal[TSegment, TWriter]) Flush() error { if w.corrupted { - return ErrWalIsCorrupted + return nil } w.locker.Lock() diff --git a/pp/go/storage/head/shard/wal/wal_reader_test.go b/pp/go/storage/head/shard/wal/wal_reader_test.go index c246eaac70..25a9588f8a 100644 --- a/pp/go/storage/head/shard/wal/wal_reader_test.go +++ b/pp/go/storage/head/shard/wal/wal_reader_test.go @@ -32,10 +32,12 @@ func (s *SegmentWalReaderSuite) TestHappyPath() { err = swr.ForEachSegment(func(rsm *testSegment) error { actual = append(actual, rsm.Bytes()...) + // protect from infinite loop limiter++ if limiter == 1000 { return errors.New("limiter") } + return nil }) s.Require().NoError(err) @@ -43,6 +45,56 @@ func (s *SegmentWalReaderSuite) TestHappyPath() { s.Equal(data, actual) } +func (s *SegmentWalReaderSuite) TestForEachSegmentError() { + buf := &bytes.Buffer{} + data := []byte(faker.Paragraph()) + data = data[:(len(data)/10)*10] + _, err := buf.Write(data) + s.Require().NoError(err) + + swr := wal.NewSegmentWalReader(buf, newTestSegment) + limiter := 0 + actual := make([]byte, 0, len(data)) + expectedError := errors.New("test error") + err = swr.ForEachSegment(func(rsm *testSegment) error { + actual = append(actual, rsm.Bytes()...) + + // protect from infinite loop + limiter++ + if limiter == 1 { + return expectedError + } + + return nil + }) + s.Require().ErrorIs(err, expectedError) +} + +func (s *SegmentWalReaderSuite) TestForEachSegmentReadError() { + buf := &bytes.Buffer{} + data := []byte(faker.Paragraph()) + data = data[:(len(data)/10)*10] + _, err := buf.Write(data) + s.Require().NoError(err) + + expectedError := errors.New("test error") + swr := wal.NewSegmentWalReader(buf, newTestSegmentWithError(expectedError)) + limiter := 0 + actual := make([]byte, 0, len(data)) + err = swr.ForEachSegment(func(rsm *testSegment) error { + actual = append(actual, rsm.Bytes()...) + + // protect from infinite loop + limiter++ + if limiter == 1 { + return errors.New("another error") + } + + return nil + }) + s.Require().ErrorIs(err, expectedError) +} + // // testSegment // @@ -74,6 +126,33 @@ func newTestSegment() *testSegment { return s } +// newTestSegmentWithError init new [testSegment] with error. +func newTestSegmentWithError(err error) func() *testSegment { + return func() *testSegment { + s := &testSegment{ + buf: make([]byte, 10), + } + + s.ReadSegmentMock = &ReadSegmentMock{ + ReadFromFunc: func(r io.Reader) (int64, error) { + n, errRead := io.ReadFull(r, s.buf) + if errRead != nil { + return int64(n), errRead + } + + return int64(n), err + }, + ResetFunc: func() { + for i := range s.buf { + s.buf[i] = 0 + } + }, + } + + return s + } +} + // Bytes returns data. func (s *testSegment) Bytes() []byte { return s.buf diff --git a/pp/go/storage/head/shard/wal/wal_test.go b/pp/go/storage/head/shard/wal/wal_test.go index 519d48e064..36fb95d286 100644 --- a/pp/go/storage/head/shard/wal/wal_test.go +++ b/pp/go/storage/head/shard/wal/wal_test.go @@ -1,87 +1,271 @@ package wal_test import ( - "fmt" - "os" - "path/filepath" + "errors" "testing" - "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" "github.com/prometheus/prometheus/pp/go/cppbridge" "github.com/prometheus/prometheus/pp/go/storage/head/shard/wal" - "github.com/prometheus/prometheus/pp/go/storage/head/shard/wal/writer" ) -func TestXxx(t *testing.T) { - shardID := uint16(0) - tmpDir, err := os.MkdirTemp("", "shard") - require.NoError(t, err) - defer func() { - _ = os.RemoveAll(tmpDir) - }() +type WalSuite struct { + suite.Suite +} + +func TestWalSuite(t *testing.T) { + suite.Run(t, new(WalSuite)) +} + +func (s *WalSuite) TestCurrentSize() { + expectedWalSize := int64(42) + enc := &EncoderMock[*EncodedSegmentMock]{} + segmentWriter := &SegmentWriterMock[*EncodedSegmentMock]{ + CurrentSizeFunc: func() int64 { return expectedWalSize }, + CloseFunc: func() error { return nil }, + } + maxSegmentSize := uint32(100) + + wl := wal.NewWal(enc, segmentWriter, maxSegmentSize) - shardFile, err := os.Create(filepath.Join(filepath.Clean(tmpDir), fmt.Sprintf("shard_%d.wal", shardID))) - require.NoError(t, err) + s.Equal(expectedWalSize, wl.CurrentSize()) - swn := &segmentWriteNotifier{} + s.Require().NoError(wl.Close()) + s.Len(segmentWriter.CloseCalls(), 1) +} - defer func() { - if err == nil { - return - } - _ = shardFile.Close() - }() +func (s *WalSuite) TestClose() { + enc := &EncoderMock[*EncodedSegmentMock]{} + segmentWriter := &SegmentWriterMock[*EncodedSegmentMock]{ + CloseFunc: func() error { return nil }, + } + maxSegmentSize := uint32(100) - sw, err := writer.NewBuffered(shardID, shardFile, writer.WriteSegment[*cppbridge.HeadEncodedSegment], swn) - require.NoError(t, err) + wl := wal.NewWal(enc, segmentWriter, maxSegmentSize) - shardWalEncoder := &cppbridge.HeadWalEncoder{} + s.Require().NoError(wl.Close()) + s.Len(segmentWriter.CloseCalls(), 1) - wl := wal.NewWal(shardWalEncoder, sw, 10) - _ = wl + s.Require().NoError(wl.Close()) + s.Len(segmentWriter.CloseCalls(), 1) } -// segmentWriteNotifier test implementation [writer.SegmentIsWrittenNotifier]. -type segmentWriteNotifier struct{} +func (s *WalSuite) TestCloseError() { + expectedError := errors.New("test error") + enc := &EncoderMock[*EncodedSegmentMock]{} + segmentWriter := &SegmentWriterMock[*EncodedSegmentMock]{ + CloseFunc: func() error { return expectedError }, + } + maxSegmentSize := uint32(100) + + wl := wal.NewWal(enc, segmentWriter, maxSegmentSize) -// NotifySegmentIsWritten test implementation [writer.SegmentIsWrittenNotifier]. -func (*segmentWriteNotifier) NotifySegmentIsWritten(shardID uint16) { - _ = shardID + s.Require().ErrorIs(wl.Close(), expectedError) + s.Len(segmentWriter.CloseCalls(), 1) } -type WalSuite struct { - suite.Suite +func (s *WalSuite) TestCommit() { + enc := &EncoderMock[*EncodedSegmentMock]{ + FinalizeFunc: func() (*EncodedSegmentMock, error) { return &EncodedSegmentMock{}, nil }, + } + segmentWriter := &SegmentWriterMock[*EncodedSegmentMock]{ + WriteFunc: func(*EncodedSegmentMock) error { return nil }, + } + maxSegmentSize := uint32(100) + + wl := wal.NewWal(enc, segmentWriter, maxSegmentSize) + + s.Require().NoError(wl.Commit()) + s.Len(enc.FinalizeCalls(), 1) + s.Len(segmentWriter.WriteCalls(), 1) } -func TestWalSuite(t *testing.T) { - suite.Run(t, new(WalSuite)) +func (s *WalSuite) TestCommitEncodeError() { + expectedError := errors.New("test error") + enc := &EncoderMock[*EncodedSegmentMock]{ + FinalizeFunc: func() (*EncodedSegmentMock, error) { return &EncodedSegmentMock{}, expectedError }, + } + segmentWriter := &SegmentWriterMock[*EncodedSegmentMock]{ + WriteFunc: func(*EncodedSegmentMock) error { return nil }, + } + maxSegmentSize := uint32(100) + + wl := wal.NewWal(enc, segmentWriter, maxSegmentSize) + + s.Require().ErrorIs(wl.Commit(), expectedError) + s.Len(enc.FinalizeCalls(), 1) + s.Empty(segmentWriter.WriteCalls()) } -func (s *WalSuite) TestCurrentSize() { - expectedWalSize := int64(42) +func (s *WalSuite) TestCommitWriteError() { + expectedError := errors.New("test error") + enc := &EncoderMock[*EncodedSegmentMock]{ + FinalizeFunc: func() (*EncodedSegmentMock, error) { return &EncodedSegmentMock{}, nil }, + } + segmentWriter := &SegmentWriterMock[*EncodedSegmentMock]{ + WriteFunc: func(*EncodedSegmentMock) error { return expectedError }, + } + maxSegmentSize := uint32(100) + + wl := wal.NewWal(enc, segmentWriter, maxSegmentSize) + + s.Require().ErrorIs(wl.Commit(), expectedError) + s.Len(enc.FinalizeCalls(), 1) + s.Len(segmentWriter.WriteCalls(), 1) +} + +func (s *WalSuite) TestFlush() { enc := &EncoderMock[*EncodedSegmentMock]{} segmentWriter := &SegmentWriterMock[*EncodedSegmentMock]{ - CurrentSizeFunc: func() int64 { - return expectedWalSize - }, + FlushFunc: func() error { return nil }, } maxSegmentSize := uint32(100) wl := wal.NewWal(enc, segmentWriter, maxSegmentSize) - s.Equal(expectedWalSize, wl.CurrentSize()) + s.Require().NoError(wl.Flush()) + s.Len(segmentWriter.FlushCalls(), 1) } -func (s *WalSuite) TestCurrentSize2() { +func (s *WalSuite) TestFlushError() { + expectedError := errors.New("test error") + enc := &EncoderMock[*EncodedSegmentMock]{} + segmentWriter := &SegmentWriterMock[*EncodedSegmentMock]{ + FlushFunc: func() error { return expectedError }, + } maxSegmentSize := uint32(100) - // enSegment := &EncodedSegmentMock{} - // stats := &StatsSegmentMock{} + + wl := wal.NewWal(enc, segmentWriter, maxSegmentSize) + + s.Require().ErrorIs(wl.Flush(), expectedError) + s.Len(segmentWriter.FlushCalls(), 1) +} + +func (s *WalSuite) TestSync() { + enc := &EncoderMock[*EncodedSegmentMock]{} + segmentWriter := &SegmentWriterMock[*EncodedSegmentMock]{ + SyncFunc: func() error { return nil }, + } + maxSegmentSize := uint32(100) + + wl := wal.NewWal(enc, segmentWriter, maxSegmentSize) + + s.Require().NoError(wl.Sync()) + s.Len(segmentWriter.SyncCalls(), 1) +} + +func (s *WalSuite) TestSyncError() { + expectedError := errors.New("test error") enc := &EncoderMock[*EncodedSegmentMock]{} - segmentWriter := &SegmentWriterMock[*EncodedSegmentMock]{} + segmentWriter := &SegmentWriterMock[*EncodedSegmentMock]{ + SyncFunc: func() error { return expectedError }, + } + maxSegmentSize := uint32(100) + + wl := wal.NewWal(enc, segmentWriter, maxSegmentSize) + + s.Require().ErrorIs(wl.Sync(), expectedError) + s.Len(segmentWriter.SyncCalls(), 1) +} + +func (s *WalSuite) TestWrite() { + enc := &EncoderMock[*EncodedSegmentMock]{ + EncodeFunc: func([]*cppbridge.InnerSeries) (uint32, error) { return 100, nil }, + } + segmentWriter := &SegmentWriterMock[*EncodedSegmentMock]{ + CloseFunc: func() error { return nil }, + } + + maxSegmentSize := uint32(0) + wl := wal.NewWal(enc, segmentWriter, maxSegmentSize) + + limitExhausted, err := wl.Write([]*cppbridge.InnerSeries{}) + s.Require().NoError(err) + s.Len(enc.EncodeCalls(), 1) + s.False(limitExhausted) + + s.Require().NoError(wl.Close()) + s.Len(segmentWriter.CloseCalls(), 1) +} + +func (s *WalSuite) TestWriteLimitExhausted() { + maxSegmentSize := uint32(100) + enc := &EncoderMock[*EncodedSegmentMock]{ + EncodeFunc: func([]*cppbridge.InnerSeries) (uint32, error) { return maxSegmentSize, nil }, + } + segmentWriter := &SegmentWriterMock[*EncodedSegmentMock]{ + CloseFunc: func() error { return nil }, + } + + wl := wal.NewWal(enc, segmentWriter, maxSegmentSize) + + limitExhausted, err := wl.Write([]*cppbridge.InnerSeries{}) + s.Require().NoError(err) + s.Len(enc.EncodeCalls(), 1) + s.True(limitExhausted) + + s.Require().NoError(wl.Close()) + s.Len(segmentWriter.CloseCalls(), 1) +} + +func (s *WalSuite) TestWriteLimitNotExhausted() { + maxSegmentSize := uint32(100) + enc := &EncoderMock[*EncodedSegmentMock]{ + EncodeFunc: func([]*cppbridge.InnerSeries) (uint32, error) { return maxSegmentSize / 2, nil }, + } + segmentWriter := &SegmentWriterMock[*EncodedSegmentMock]{ + CloseFunc: func() error { return nil }, + } wl := wal.NewWal(enc, segmentWriter, maxSegmentSize) - s.T().Log(wl) + limitExhausted, err := wl.Write([]*cppbridge.InnerSeries{}) + s.Require().NoError(err) + s.Len(enc.EncodeCalls(), 1) + s.False(limitExhausted) + + s.Require().NoError(wl.Close()) + s.Len(segmentWriter.CloseCalls(), 1) +} + +func (s *WalSuite) TestWriteError() { + maxSegmentSize := uint32(100) + expectedError := errors.New("test error") + enc := &EncoderMock[*EncodedSegmentMock]{ + EncodeFunc: func([]*cppbridge.InnerSeries) (uint32, error) { return maxSegmentSize / 2, expectedError }, + } + segmentWriter := &SegmentWriterMock[*EncodedSegmentMock]{ + CloseFunc: func() error { return nil }, + } + + wl := wal.NewWal(enc, segmentWriter, maxSegmentSize) + + limitExhausted, err := wl.Write([]*cppbridge.InnerSeries{}) + s.Require().ErrorIs(err, expectedError) + s.Len(enc.EncodeCalls(), 1) + s.False(limitExhausted) + + s.Require().NoError(wl.Close()) + s.Len(segmentWriter.CloseCalls(), 1) +} + +func (s *WalSuite) TestCorrupted() { + wl := wal.NewCorruptedWal[*EncodedSegmentMock, *SegmentWriterMock[*EncodedSegmentMock]]() + s.Equal(int64(0), wl.CurrentSize()) + + limitExhausted, err := wl.Write([]*cppbridge.InnerSeries{}) + s.Require().ErrorIs(err, wal.ErrWalIsCorrupted) + s.False(limitExhausted) + + err = wl.Commit() + s.Require().ErrorIs(err, wal.ErrWalIsCorrupted) + + err = wl.Flush() + s.Require().NoError(err) + + err = wl.Sync() + s.Require().ErrorIs(err, wal.ErrWalIsCorrupted) + + s.Require().NoError(wl.Close()) } From cbce844693df23cd3b04a14b2823287049567f8f Mon Sep 17 00:00:00 2001 From: Alexandr Yudin Date: Tue, 7 Oct 2025 13:18:51 +0000 Subject: [PATCH 73/96] add descriptions --- pp/go/storage/block/block.go | 15 +- pp/go/storage/block/block_writer.go | 11 ++ pp/go/storage/block/chunk_writer.go | 2 +- pp/go/storage/block/writer.go | 6 + pp/go/storage/block/writer_test.go | 5 +- pp/go/storage/head/proxy/proxy.go | 135 ---------------- pp/go/storage/head/shard/data_storage.go | 3 + .../shard/load_and_query_series_data_task.go | 3 + .../head/shard/unloaded_data_storage.go | 13 ++ pp/go/storage/head/task/waiter_test.go | 2 +- pp/go/storage/remotewriter/datasource.go | 30 +++- pp/go/storage/remotewriter/destination.go | 149 +++++++++++------- pp/go/storage/remotewriter/iterator.go | 50 ++++-- pp/go/storage/remotewriter/remotewriter.go | 2 +- .../storage/remotewriter/remotewriter_test.go | 2 +- pp/go/storage/remotewriter/writeloop.go | 60 +++++-- 16 files changed, 264 insertions(+), 224 deletions(-) delete mode 100644 pp/go/storage/head/proxy/proxy.go diff --git a/pp/go/storage/block/block.go b/pp/go/storage/block/block.go index 7987fbcf0d..3b1bb834cf 100644 --- a/pp/go/storage/block/block.go +++ b/pp/go/storage/block/block.go @@ -10,34 +10,42 @@ import ( "github.com/prometheus/prometheus/tsdb/chunkenc" ) +// Chunk represents a recoded chunk. type Chunk struct { rc *cppbridge.RecodedChunk } +// MinT returns the minimum timestamp of the chunk. func (c *Chunk) MinT() int64 { return c.rc.MinT } +// MaxT returns the maximum timestamp of the chunk. func (c *Chunk) MaxT() int64 { return c.rc.MaxT } +// SeriesID returns the series ID of the chunk. func (c *Chunk) SeriesID() uint32 { return c.rc.SeriesId } +// Encoding returns is the identifier of the encoding of the chunk. func (*Chunk) Encoding() chunkenc.Encoding { return chunkenc.EncXOR } +// SampleCount returns the number of samples in the chunk. func (c *Chunk) SampleCount() uint8 { return c.rc.SamplesCount } +// Bytes returns the bytes blob of the chunk data. func (c *Chunk) Bytes() []byte { return c.rc.ChunkData } +// ChunkIterator represents a chunk iterator, it is used to iterate over the chunks. type ChunkIterator struct { r *cppbridge.ChunkRecoder rc *cppbridge.RecodedChunk @@ -55,6 +63,7 @@ func NewChunkIterator( } } +// Next advances the iterator by one, if possible. func (i *ChunkIterator) Next() bool { if i.rc != nil && !i.rc.HasMoreData { return false @@ -65,11 +74,13 @@ func (i *ChunkIterator) Next() bool { return rc.SeriesId != math.MaxUint32 } +// NextBatch advances the iterator by one batch, if if there is more data. func (i *ChunkIterator) NextBatch() bool { i.rc.HasMoreData = i.r.NextBatch() return i.rc.HasMoreData } +// At returns the current chunk. func (i *ChunkIterator) At() Chunk { return Chunk{rc: i.rc} } @@ -78,6 +89,7 @@ func (i *ChunkIterator) At() Chunk { // IndexWriter // +// IndexWriter represents a index writer, it is used to write the index. type IndexWriter struct { cppIndexWriter *cppbridge.IndexWriter isPrefixWritten bool @@ -88,6 +100,7 @@ func NewIndexWriter(lss *cppbridge.LabelSetStorage) IndexWriter { return IndexWriter{cppIndexWriter: cppbridge.NewIndexWriter(lss)} } +// WriteRestTo writes the rest of the index to the writer. func (iw *IndexWriter) WriteRestTo(w io.Writer) (n int64, err error) { bytesWritten, err := w.Write(iw.cppIndexWriter.WriteLabelIndices()) n += int64(bytesWritten) @@ -128,7 +141,7 @@ func (iw *IndexWriter) WriteRestTo(w io.Writer) (n int64, err error) { return n, nil } -// WriteSeriesTo write series(id and chunks) to [io.Writer]. +// WriteSeriesTo writes series(id and chunks) to [io.Writer]. func (iw *IndexWriter) WriteSeriesTo(id uint32, chunks []ChunkMetadata, w io.Writer) (n int64, err error) { if !iw.isPrefixWritten { var bytesWritten int diff --git a/pp/go/storage/block/block_writer.go b/pp/go/storage/block/block_writer.go index b89860f7b4..da8e90f81e 100644 --- a/pp/go/storage/block/block_writer.go +++ b/pp/go/storage/block/block_writer.go @@ -26,19 +26,23 @@ const ( metaVersion1 = 1 ) +// WrittenBlock represents a written block. type WrittenBlock struct { Dir string Meta tsdb.BlockMeta } +// ChunkDir returns the chunk directory. func (block *WrittenBlock) ChunkDir() string { return filepath.Join(block.Dir, "chunks") } +// IndexFilename returns the index filename. func (block *WrittenBlock) IndexFilename() string { return filepath.Join(block.Dir, indexFilename) } +// MetaFilename returns the meta filename. func (block *WrittenBlock) MetaFilename() string { return filepath.Join(block.Dir, metaFilename) } @@ -161,16 +165,21 @@ func (writer *blockWriter) moveTmpDirToDir() error { type blockWriters []blockWriter +// append appends a writer to the block writers. + +//nolint:gocritic // hugeParam // we accumulate the writers func (bw *blockWriters) append(writer blockWriter) { *bw = append(*bw, writer) } +// close closes the block writers. func (bw *blockWriters) close() { for i := range *bw { _ = (*bw)[i].close() } } +// recodeAndWriteChunksBatch recodes and writes the chunks batch. func (bw *blockWriters) recodeAndWriteChunksBatch() error { for i := range *bw { if err := (*bw)[i].recodeAndWriteChunksBatch(); err != nil { @@ -181,6 +190,7 @@ func (bw *blockWriters) recodeAndWriteChunksBatch() error { return nil } +// writeRestOfRecodedChunks writes the rest of the recoded chunks. func (bw *blockWriters) writeRestOfRecodedChunks() error { for i := range *bw { if err := (*bw)[i].writeRestOfRecodedChunks(); err != nil { @@ -191,6 +201,7 @@ func (bw *blockWriters) writeRestOfRecodedChunks() error { return nil } +// writeIndexAndMoveTmpDirToDir writes the index and moves the temporary directory to the directory. func (bw *blockWriters) writeIndexAndMoveTmpDirToDir() ([]WrittenBlock, error) { writtenBlocks := make([]WrittenBlock, 0, len(*bw)) for i := range *bw { diff --git a/pp/go/storage/block/chunk_writer.go b/pp/go/storage/block/chunk_writer.go index d1a531c884..7d8648f9e9 100644 --- a/pp/go/storage/block/chunk_writer.go +++ b/pp/go/storage/block/chunk_writer.go @@ -71,7 +71,7 @@ func (w *ChunkWriter) Close() (err error) { return w.dirFile.Close() } -// Write encoding and write to buffer chunk. +// Write encoding and write to buffer chunk. func (w *ChunkWriter) Write(chunk Chunk) (meta ChunkMetadata, err error) { // calculate chunk size chunkSize := int64(chunks.MaxChunkLengthFieldSize) diff --git a/pp/go/storage/block/writer.go b/pp/go/storage/block/writer.go index 923cea5339..3d2b154af7 100644 --- a/pp/go/storage/block/writer.go +++ b/pp/go/storage/block/writer.go @@ -16,6 +16,7 @@ const ( DefaultBlockDuration = 2 * time.Hour ) +// LsIdBatchSize is the batch size for label set ID. var LsIdBatchSize uint32 = 100000 // Shard the minimum required head [Shard] implementation. @@ -27,6 +28,7 @@ type Shard interface { UnloadedDataStorage() *shard.UnloadedDataStorage } +// Writer represents a block writer. It is used to write blocks to disk from a shard. type Writer[TShard Shard] struct { dataDir string maxBlockChunkSegmentSize int64 @@ -34,6 +36,7 @@ type Writer[TShard Shard] struct { blockWriteDuration *prometheus.GaugeVec } +// NewWriter creates a new [Writer]. func NewWriter[TShard Shard]( dataDir string, maxBlockChunkSegmentSize int64, @@ -52,6 +55,7 @@ func NewWriter[TShard Shard]( } } +// Write writes blocks to disk from a shard. func (w *Writer[TShard]) Write(sd TShard) (writtenBlocks []WrittenBlock, err error) { _ = sd.LSS().WithRLock(func(_, _ *cppbridge.LabelSetStorage) error { var writers blockWriters @@ -76,6 +80,7 @@ func (w *Writer[TShard]) Write(sd TShard) (writtenBlocks []WrittenBlock, err err return writtenBlocks, err } +// createWriters creates writers for the shard. func (w *Writer[TShard]) createWriters(sd TShard) (blockWriters, error) { var writers blockWriters @@ -114,6 +119,7 @@ func (w *Writer[TShard]) createWriters(sd TShard) (blockWriters, error) { return writers, nil } +// recodeAndWriteChunks recodes and writes chunks for the shard. func (*Writer[TShard]) recodeAndWriteChunks(sd TShard, writers blockWriters) error { var loader *cppbridge.UnloadedDataRevertableLoader _ = sd.DataStorage().WithRLock(func(*cppbridge.HeadDataStorage) error { diff --git a/pp/go/storage/block/writer_test.go b/pp/go/storage/block/writer_test.go index f478748bea..b99bb59e79 100644 --- a/pp/go/storage/block/writer_test.go +++ b/pp/go/storage/block/writer_test.go @@ -10,6 +10,8 @@ import ( "github.com/jonboulle/clockwork" "github.com/oklog/ulid" "github.com/prometheus/client_golang/prometheus" + "github.com/stretchr/testify/suite" + "github.com/prometheus/prometheus/model/labels" "github.com/prometheus/prometheus/pp/go/cppbridge" "github.com/prometheus/prometheus/pp/go/storage" @@ -18,7 +20,6 @@ import ( "github.com/prometheus/prometheus/pp/go/storage/head/services" "github.com/prometheus/prometheus/pp/go/storage/storagetest" "github.com/prometheus/prometheus/tsdb" - "github.com/stretchr/testify/suite" ) const ( @@ -296,5 +297,5 @@ func (s *WriterSuite) TestSkipEmptyBlock() { // Assert s.Require().NoError(err) - s.Equal(2, len(blocks)) + s.Len(blocks, 2) } diff --git a/pp/go/storage/head/proxy/proxy.go b/pp/go/storage/head/proxy/proxy.go deleted file mode 100644 index bcb290c7d6..0000000000 --- a/pp/go/storage/head/proxy/proxy.go +++ /dev/null @@ -1,135 +0,0 @@ -package proxy - -// TODO DELETE -// import ( -// "context" -// "errors" -// "time" -// ) - -// // Head the minimum required [Head] implementation for a proxy. -// type Head interface { -// // Close closes wals, query semaphore for the inability to get query and clear metrics. -// Close() error -// } - -// // -// // ActiveHeadContainer -// // - -// // ActiveHeadContainer container for active [Head], the minimum required [ActiveHeadContainer] implementation. -// type ActiveHeadContainer[THead Head] interface { -// // Close closes [ActiveHeadContainer] for the inability work with [Head]. -// Close() error - -// // Get the active [Head]. -// Get() THead - -// // Replace the active [Head] with a new [Head]. -// Replace(ctx context.Context, newHead THead) error - -// // With calls fn(h Head). -// With(ctx context.Context, fn func(h THead) error) error -// } - -// // -// // Keeper -// // - -// // Keeper holds outdated heads until conversion. -// type Keeper[THead Head] interface { -// // Add the [Head] to the [Keeper] if there is a free slot. -// Add(head THead, createdAt time.Duration) error - -// // AddWithReplace the [Head] to the [Keeper] with replace if the createdAt is earlier. -// AddWithReplace(head THead, createdAt time.Duration) error - -// // Close closes for the inability work with [Head]. -// Close() error - -// // HasSlot returns the tru if there is a slot in the [Keeper]. -// HasSlot() bool - -// // Heads returns a slice of the [Head]s stored in the [Keeper]. -// Heads() []THead - -// // Remove removes [Head]s from the [Keeper]. -// Remove(headsForRemove []THead) -// } - -// // -// // Proxy -// // - -// // Proxy it proxies requests to the active [Head] and the keeper of old [Head]s. -// type Proxy[THead Head] struct { -// activeHeadContainer ActiveHeadContainer[THead] -// keeper Keeper[THead] -// onClose func(h THead) error -// } - -// // NewProxy init new [Proxy]. -// func NewProxy[THead Head]( -// activeHeadContainer ActiveHeadContainer[THead], -// keeper Keeper[THead], -// onClose func(h THead) error, -// ) *Proxy[THead] { -// return &Proxy[THead]{ -// activeHeadContainer: activeHeadContainer, -// keeper: keeper, -// onClose: onClose, -// } -// } - -// // Add the [Head] to the [Keeper] if there is a free slot. -// func (p *Proxy[THead]) Add(head THead, createdAt time.Duration) error { -// return p.keeper.Add(head, createdAt) -// } - -// // AddWithReplace the [Head] to the [Keeper] with replace if the createdAt is earlier. -// func (p *Proxy[THead]) AddWithReplace(head THead, createdAt time.Duration) error { -// return p.keeper.AddWithReplace(head, createdAt) -// } - -// // Close closes [ActiveHeadContainer] and [Keeper] for the inability work with [Head]. -// func (p *Proxy[THead]) Close() error { -// ahErr := p.activeHeadContainer.Close() - -// h := p.activeHeadContainer.Get() -// onCloseErr := p.onClose(h) -// headCloseErr := h.Close() - -// keeperErr := p.keeper.Close() - -// return errors.Join(ahErr, onCloseErr, headCloseErr, keeperErr) -// } - -// // Get the active [Head]. -// func (p *Proxy[THead]) Get() THead { -// return p.activeHeadContainer.Get() -// } - -// // HasSlot returns the tru if there is a slot in the [Keeper]. -// func (p *Proxy[THead]) HasSlot() bool { -// return p.keeper.HasSlot() -// } - -// // Heads returns a slice of the [Head]s stored in the [Keeper]. -// func (p *Proxy[THead]) Heads() []THead { -// return p.keeper.Heads() -// } - -// // Remove removes [Head]s from the [Keeper]. -// func (p *Proxy[THead]) Remove(headsForRemove []THead) { -// p.keeper.Remove(headsForRemove) -// } - -// // Replace the active [Head] with a new [Head]. -// func (p *Proxy[THead]) Replace(ctx context.Context, newHead THead) error { -// return p.activeHeadContainer.Replace(ctx, newHead) -// } - -// // With calls fn(h Head) on active [Head]. -// func (p *Proxy[THead]) With(ctx context.Context, fn func(h THead) error) error { -// return p.activeHeadContainer.With(ctx, fn) -// } diff --git a/pp/go/storage/head/shard/data_storage.go b/pp/go/storage/head/shard/data_storage.go index 8d499b6d31..3dafc62979 100644 --- a/pp/go/storage/head/shard/data_storage.go +++ b/pp/go/storage/head/shard/data_storage.go @@ -78,6 +78,7 @@ func (ds *DataStorage) Query( return serializedChunks, res } +// QueryFinal finishes all the queries after data load. func (ds *DataStorage) QueryFinal(queriers []uintptr) { ds.locker.RLock() ds.dataStorage.QueryFinal(queriers) @@ -141,10 +142,12 @@ func (ds *DataStorage) CreateRevertableLoader( return ds.dataStorage.CreateRevertableLoader(lss, lsIdBatchSize) } +// GetQueriedSeriesBitset gets the queried series bitset memory. func (ds *DataStorage) GetQueriedSeriesBitset() []byte { return ds.dataStorage.GetQueriedSeriesBitset() } +// SetQueriedSeriesBitset sets the queried series bitset. func (ds *DataStorage) SetQueriedSeriesBitset(bitset []byte) bool { return ds.dataStorage.SetQueriedSeriesBitset(bitset) } diff --git a/pp/go/storage/head/shard/load_and_query_series_data_task.go b/pp/go/storage/head/shard/load_and_query_series_data_task.go index 2ab38b9871..8d5562c6aa 100644 --- a/pp/go/storage/head/shard/load_and_query_series_data_task.go +++ b/pp/go/storage/head/shard/load_and_query_series_data_task.go @@ -9,12 +9,14 @@ type Task interface { Wait() error } +// LoadAndQuerySeriesDataTask represents a task to load and query series data. type LoadAndQuerySeriesDataTask struct { queriers []uintptr task Task lock sync.Mutex } +// Add adds a querier to the task, if exists no task, it creates and enqueues a task. func (t *LoadAndQuerySeriesDataTask) Add(querier uintptr, createAndEnqueueTask func() Task) Task { t.lock.Lock() defer t.lock.Unlock() @@ -26,6 +28,7 @@ func (t *LoadAndQuerySeriesDataTask) Add(querier uintptr, createAndEnqueueTask f return t.task } +// Release executes and releases the queriers. func (t *LoadAndQuerySeriesDataTask) Release(callback func([]uintptr)) { t.lock.Lock() callback(t.queriers) diff --git a/pp/go/storage/head/shard/unloaded_data_storage.go b/pp/go/storage/head/shard/unloaded_data_storage.go index 1b343f9fce..04bbd1f477 100644 --- a/pp/go/storage/head/shard/unloaded_data_storage.go +++ b/pp/go/storage/head/shard/unloaded_data_storage.go @@ -49,18 +49,21 @@ func (h UnloadedDataSnapshotHeader) IsValid(snapshot []byte) bool { return h.Crc32 == crc32.ChecksumIEEE(snapshot) } +// UnloadedDataStorage represents a unloaded data storage, unloads snapshots to the storage from [DataStorage]. type UnloadedDataStorage struct { storage StorageFile snapshots []UnloadedDataSnapshotHeader maxSnapshotSize uint32 } +// NewUnloadedDataStorage creates a new [UnloadedDataStorage]. func NewUnloadedDataStorage(storage StorageFile) *UnloadedDataStorage { return &UnloadedDataStorage{ storage: storage, } } +// WriteSnapshot writes a snapshot to the storage. func (s *UnloadedDataStorage) WriteSnapshot(snapshot []byte) (UnloadedDataSnapshotHeader, error) { if len(snapshot) == 0 { return UnloadedDataSnapshotHeader{}, nil @@ -83,16 +86,19 @@ func (s *UnloadedDataStorage) WriteSnapshot(snapshot []byte) (UnloadedDataSnapsh return NewUnloadedDataSnapshotHeader(snapshot), err } +// WriteIndex writes an index to the storage. func (s *UnloadedDataStorage) WriteIndex(header UnloadedDataSnapshotHeader) { s.snapshots = append(s.snapshots, header) s.maxSnapshotSize = max(header.SnapshotSize, s.maxSnapshotSize) } +// WriteFormatVersion writes the format version to the storage. func (s *UnloadedDataStorage) WriteFormatVersion() error { _, err := s.storage.Write([]byte{UnloadedDataStorageVersion}) return err } +// ForEachSnapshot iterates over the snapshots and calls the callback function. func (s *UnloadedDataStorage) ForEachSnapshot(f func(snapshot []byte, isLast bool)) error { if len(s.snapshots) == 0 { return nil @@ -122,6 +128,7 @@ func (s *UnloadedDataStorage) ForEachSnapshot(f func(snapshot []byte, isLast boo return nil } +// validateFormatVersion validates the format version. func (s *UnloadedDataStorage) validateFormatVersion() (offset int64, err error) { version := []byte{0} if _, err = s.storage.ReadAt(version, 0); err != nil { @@ -135,6 +142,7 @@ func (s *UnloadedDataStorage) validateFormatVersion() (offset int64, err error) return int64(len(version)), nil } +// Close closes the storage. func (s *UnloadedDataStorage) Close() (err error) { if s.storage != nil { err = s.storage.Close() @@ -144,15 +152,19 @@ func (s *UnloadedDataStorage) Close() (err error) { return err } +// IsEmpty checks if the storage is empty. func (s *UnloadedDataStorage) IsEmpty() bool { return len(s.snapshots) == 0 } +// QueriedSeriesStorage represents a queried series storage, +// it contains two file stores that it swaps them as needed. type QueriedSeriesStorage struct { storages [2]StorageFile validStorage StorageFile } +// NewQueriedSeriesStorage creates a new [QueriedSeriesStorage]. func NewQueriedSeriesStorage(storage1, storage2 StorageFile) *QueriedSeriesStorage { return &QueriedSeriesStorage{ storages: [2]StorageFile{storage1, storage2}, //revive:disable-line:add-constant // 2 working files @@ -287,6 +299,7 @@ func (s *QueriedSeriesStorage) readStorageHeaders() (result []storageHeaderReade return result, maxSize } +// Close closes the storage. func (s *QueriedSeriesStorage) Close() error { return errors.Join(s.storages[0].Close(), s.storages[1].Close()) } diff --git a/pp/go/storage/head/task/waiter_test.go b/pp/go/storage/head/task/waiter_test.go index 77be3acc65..d352f6377e 100644 --- a/pp/go/storage/head/task/waiter_test.go +++ b/pp/go/storage/head/task/waiter_test.go @@ -7,7 +7,7 @@ import ( "github.com/stretchr/testify/require" ) -func TestXxx(t *testing.T) { +func TestTaskWaiter(t *testing.T) { tw := task.NewTaskWaiter[*task.Generic[*testShard]](5) err := tw.Wait() require.NoError(t, err) diff --git a/pp/go/storage/remotewriter/datasource.go b/pp/go/storage/remotewriter/datasource.go index db9c04ea29..2f71e3b3c9 100644 --- a/pp/go/storage/remotewriter/datasource.go +++ b/pp/go/storage/remotewriter/datasource.go @@ -104,6 +104,8 @@ type shard struct { } // newShard init new [shard]. +// +//revive:disable-next-line:flag-parameter this is a flag, but it's more convenient this way func newShard( headID string, shardID uint16, @@ -131,13 +133,18 @@ func newShard( decoderStateFileFlags := os.O_CREATE | os.O_RDWR if resetDecoderState { - decoderStateFileFlags = decoderStateFileFlags | os.O_TRUNC + decoderStateFileFlags |= os.O_TRUNC } - decoderStateFile, err := os.OpenFile(decoderStateFileName, decoderStateFileFlags, 0o600) + decoderStateFile, err := os.OpenFile( // #nosec G304 // it's meant to be that way + decoderStateFileName, + decoderStateFileFlags, + 0o600, //revive:disable-line:add-constant // file permissions simple readable as octa-number + ) if err != nil { - return nil, errors.Join(fmt.Errorf("failed to open cache file: %w", err), wr.Close()) + return nil, errors.Join(fmt.Errorf("failed to open decoder state file: %w", err), wr.Close()) } + // create new shard s := &shard{ headID: headID, shardID: shardID, @@ -212,7 +219,9 @@ func (s *shard) SetCorrupted() { // SegmentReadyChecker // +// SegmentReadyChecker is a segment ready checker. type SegmentReadyChecker interface { + // SegmentIsReady checks if the segment is ready. SegmentIsReady(segmentID uint32) (ready bool, outOfRange bool) } @@ -272,9 +281,10 @@ type dataSource struct { segmentSize prometheus.Histogram } +// newDataSource creates a new [dataSource]. func newDataSource(dataDir string, numberOfShards uint16, - config DestinationConfig, + config DestinationConfig, //nolint:gocritic // hugeParam // config discardCache bool, segmentReadyChecker SegmentReadyChecker, corruptMarker CorruptMarker, @@ -442,7 +452,10 @@ func (ds *dataSource) Read(ctx context.Context, segmentID uint32, minTimestamp i readShardResults := make([]readShardResult, len(ds.shards)) for shardID := 0; shardID < len(ds.shards); shardID++ { if ds.shards[shardID].corrupted { - readShardResults[shardID] = readShardResult{segment: nil, err: NewShardError(shardID, false, ErrShardIsCorrupted)} + readShardResults[shardID] = readShardResult{ + segment: nil, + err: NewShardError(shardID, false, ErrShardIsCorrupted), + } continue } wg.Add(1) @@ -512,6 +525,7 @@ func (ds *dataSource) LSSes() []*cppbridge.LabelSetStorage { return ds.lssSlice } +// WriteCaches writes caches to the buffer and sends the signal to write the caches. func (ds *dataSource) WriteCaches() { ds.cacheMtx.Lock() for shardID, sc := range ds.caches { @@ -533,6 +547,7 @@ func (ds *dataSource) WriteCaches() { } } +// cacheWriteLoop loop that writes caches to the buffer and sends the signal to write the caches. func (ds *dataSource) cacheWriteLoop() { defer close(ds.cacheWriteLoopClosed) var closed bool @@ -544,7 +559,7 @@ func (ds *dataSource) cacheWriteLoop() { writeResultc = make(chan struct{}) go func() { defer close(writeResultc) - ds.writeCaches() + ds.writeBufferedCaches() }() writeRequested = false } @@ -565,7 +580,8 @@ func (ds *dataSource) cacheWriteLoop() { } } -func (ds *dataSource) writeCaches() { +// writeBufferedCaches writes cached data to the disk and resets the cache. +func (ds *dataSource) writeBufferedCaches() { ds.cacheMtx.Lock() caches := make([]*shardCache, 0, len(ds.caches)) for _, sc := range ds.caches { diff --git a/pp/go/storage/remotewriter/destination.go b/pp/go/storage/remotewriter/destination.go index 5fd36fbcf1..bda340459f 100644 --- a/pp/go/storage/remotewriter/destination.go +++ b/pp/go/storage/remotewriter/destination.go @@ -27,19 +27,24 @@ const ( DefaultSampleAgeLimit = model.Duration(time.Hour * 24 * 30) ) +// DestinationConfig is a remote write destination config. type DestinationConfig struct { config.RemoteWriteConfig ExternalLabels labels.Labels `yaml:"external_labels"` ReadTimeout time.Duration } -func (c DestinationConfig) EqualTo(other DestinationConfig) bool { +// EqualTo checks if the config is equal to the other config. +// +//nolint:gocritic // hugeParam // equal configs +func (c *DestinationConfig) EqualTo(other DestinationConfig) bool { return c.ExternalLabels.Hash() == other.ExternalLabels.Hash() && c.ReadTimeout == other.ReadTimeout && remoteWriteConfigsAreEqual(c.RemoteWriteConfig, other.RemoteWriteConfig) } -func (c DestinationConfig) CRC32() (uint32, error) { +// CRC32 returns the CRC32 hash of the config. +func (c *DestinationConfig) CRC32() (uint32, error) { data, err := yaml.Marshal(c) if err != nil { return 0, err @@ -48,19 +53,28 @@ func (c DestinationConfig) CRC32() (uint32, error) { return crc32.ChecksumIEEE(data), nil } +// Destination is a remote write destination. type Destination struct { config DestinationConfig metrics *DestinationMetrics } +// Config returns current config. func (d *Destination) Config() DestinationConfig { return d.config } -func (d *Destination) ResetConfig(config DestinationConfig) { - d.config = config +// ResetConfig resets current config to the new one. +// +//nolint:gocritic // hugeParam // resets config +func (d *Destination) ResetConfig(cfg DestinationConfig) { + d.config = cfg } +// NewDestination creates a new [Destination]. +// +//revive:disable-next-line:function-length // this is a constructor +//nolint:gocritic // hugeParam // this is a constructor func NewDestination(cfg DestinationConfig) *Destination { constLabels := prometheus.Labels{ remoteName: cfg.Name, @@ -106,86 +120,98 @@ func NewDestination(cfg DestinationConfig) *Destination { ConstLabels: constLabels, }), failedExemplarsTotal: prometheus.NewCounter(prometheus.CounterOpts{ - Namespace: namespace, - Subsystem: subsystem, - Name: "exemplars_failed_total", + Namespace: namespace, + Subsystem: subsystem, + Name: "exemplars_failed_total", + //revive:disable-next-line:line-length-limit // this is a description of the metric Help: "Total number of exemplars which failed on send to remote storage, non-recoverable errors.", ConstLabels: constLabels, }), failedHistogramsTotal: prometheus.NewCounter(prometheus.CounterOpts{ - Namespace: namespace, - Subsystem: subsystem, - Name: "histograms_failed_total", + Namespace: namespace, + Subsystem: subsystem, + Name: "histograms_failed_total", + //revive:disable-next-line:line-length-limit // this is a description of the metric Help: "Total number of histograms which failed on send to remote storage, non-recoverable errors.", ConstLabels: constLabels, }), failedMetadataTotal: prometheus.NewCounter(prometheus.CounterOpts{ - Namespace: namespace, - Subsystem: subsystem, - Name: "metadata_failed_total", + Namespace: namespace, + Subsystem: subsystem, + Name: "metadata_failed_total", + //revive:disable-next-line:line-length-limit // this is a description of the metric Help: "Total number of metadata entries which failed on send to remote storage, non-recoverable errors.", ConstLabels: constLabels, }), retriedSamplesTotal: prometheus.NewCounter(prometheus.CounterOpts{ - Namespace: namespace, - Subsystem: subsystem, - Name: "samples_retried_total", + Namespace: namespace, + Subsystem: subsystem, + Name: "samples_retried_total", + //revive:disable-next-line:line-length-limit // this is a description of the metric Help: "Total number of samples which failed on send to remote storage but were retried because the send error was recoverable.", ConstLabels: constLabels, }), retriedExemplarsTotal: prometheus.NewCounter(prometheus.CounterOpts{ - Namespace: namespace, - Subsystem: subsystem, - Name: "exemplars_retried_total", + Namespace: namespace, + Subsystem: subsystem, + Name: "exemplars_retried_total", + //revive:disable-next-line:line-length-limit // this is a description of the metric Help: "Total number of exemplars which failed on send to remote storage but were retried because the send error was recoverable.", ConstLabels: constLabels, }), retriedHistogramsTotal: prometheus.NewCounter(prometheus.CounterOpts{ - Namespace: namespace, - Subsystem: subsystem, - Name: "histograms_retried_total", + Namespace: namespace, + Subsystem: subsystem, + Name: "histograms_retried_total", + //revive:disable-next-line:line-length-limit // this is a description of the metric Help: "Total number of histograms which failed on send to remote storage but were retried because the send error was recoverable.", ConstLabels: constLabels, }), retriedMetadataTotal: prometheus.NewCounter(prometheus.CounterOpts{ - Namespace: namespace, - Subsystem: subsystem, - Name: "metadata_retried_total", + Namespace: namespace, + Subsystem: subsystem, + Name: "metadata_retried_total", + //revive:disable-next-line:line-length-limit // this is a description of the metric Help: "Total number of metadata entries which failed on send to remote storage but were retried because the send error was recoverable.", ConstLabels: constLabels, }), droppedSamplesTotal: prometheus.NewCounterVec(prometheus.CounterOpts{ - Namespace: namespace, - Subsystem: subsystem, - Name: "samples_dropped_total", + Namespace: namespace, + Subsystem: subsystem, + Name: "samples_dropped_total", + //revive:disable-next-line:line-length-limit // this is a description of the metric Help: "Total number of samples which were dropped after being read from the WAL before being sent via remote write, either via relabelling, due to being too old or unintentionally because of an unknown reference ID.", ConstLabels: constLabels, }, []string{"reason"}), addSeriesTotal: prometheus.NewCounter(prometheus.CounterOpts{ - Namespace: namespace, - Subsystem: subsystem, - Name: "series_added_total", + Namespace: namespace, + Subsystem: subsystem, + Name: "series_added_total", + //revive:disable-next-line:line-length-limit // this is a description of the metric Help: "Total number of series which were add after being read from the WAL before being sent via remote write, either via relabelling.", ConstLabels: constLabels, }), droppedSeriesTotal: prometheus.NewCounter(prometheus.CounterOpts{ - Namespace: namespace, - Subsystem: subsystem, - Name: "series_dropped_total", + Namespace: namespace, + Subsystem: subsystem, + Name: "series_dropped_total", + //revive:disable-next-line:line-length-limit // this is a description of the metric Help: "Total number of series which were dropped after being read from the WAL before being sent via remote write, either via relabelling.", ConstLabels: constLabels, }), droppedExemplarsTotal: prometheus.NewCounterVec(prometheus.CounterOpts{ - Namespace: namespace, - Subsystem: subsystem, - Name: "exemplars_dropped_total", + Namespace: namespace, + Subsystem: subsystem, + Name: "exemplars_dropped_total", + //revive:disable-next-line:line-length-limit // this is a description of the metric Help: "Total number of exemplars which were dropped after being read from the WAL before being sent via remote write, either via relabelling, due to being too old or unintentionally because of an unknown reference ID.", ConstLabels: constLabels, }, []string{"reason"}), droppedHistogramsTotal: prometheus.NewCounterVec(prometheus.CounterOpts{ - Namespace: namespace, - Subsystem: subsystem, - Name: "histograms_dropped_total", + Namespace: namespace, + Subsystem: subsystem, + Name: "histograms_dropped_total", + //revive:disable-next-line:line-length-limit // this is a description of the metric Help: "Total number of histograms which were dropped after being read from the WAL before being sent via remote write, either via relabelling, due to being too old or unintentionally because of an unknown reference ID.", ConstLabels: constLabels, }, []string{"reason"}), @@ -209,9 +235,10 @@ func NewDestination(cfg DestinationConfig) *Destination { }), highestSentTimestamp: &maxTimestamp{ Gauge: prometheus.NewGauge(prometheus.GaugeOpts{ - Namespace: namespace, - Subsystem: subsystem, - Name: "queue_highest_sent_timestamp_seconds", + Namespace: namespace, + Subsystem: subsystem, + Name: "queue_highest_sent_timestamp_seconds", + //revive:disable-next-line:line-length-limit // this is a description of the metric Help: "Timestamp from a WAL sample, the highest timestamp successfully sent by this queue, in seconds since epoch.", ConstLabels: constLabels, }), @@ -266,9 +293,10 @@ func NewDestination(cfg DestinationConfig) *Destination { ConstLabels: constLabels, }), desiredNumShards: prometheus.NewGauge(prometheus.GaugeOpts{ - Namespace: namespace, - Subsystem: subsystem, - Name: "shards_desired", + Namespace: namespace, + Subsystem: subsystem, + Name: "shards_desired", + //revive:disable-next-line:line-length-limit // this is a description of the metric Help: "The number of shards that the queues shard calculation wants to run based on the rate of samples in vs. samples out.", ConstLabels: constLabels, }), @@ -280,9 +308,10 @@ func NewDestination(cfg DestinationConfig) *Destination { ConstLabels: constLabels, }), sentBytesTotal: prometheus.NewCounter(prometheus.CounterOpts{ - Namespace: namespace, - Subsystem: subsystem, - Name: "bytes_total", + Namespace: namespace, + Subsystem: subsystem, + Name: "bytes_total", + //revive:disable-next-line:line-length-limit // this is a description of the metric Help: "The total number of bytes of data (not metadata) sent by the queue after compression. Note that when exemplars over remote write is enabled the exemplars included in a remote write request count towards this metric.", ConstLabels: constLabels, }), @@ -294,9 +323,10 @@ func NewDestination(cfg DestinationConfig) *Destination { ConstLabels: constLabels, }), maxSamplesPerSend: prometheus.NewGauge(prometheus.GaugeOpts{ - Namespace: namespace, - Subsystem: subsystem, - Name: "max_samples_per_send", + Namespace: namespace, + Subsystem: subsystem, + Name: "max_samples_per_send", + //revive:disable-next-line:line-length-limit // this is a description of the metric Help: "The maximum number of samples to be sent, in a single request, to the remote storage. Note that, when sending of exemplars over remote write is enabled, exemplars count towards this limt.", ConstLabels: constLabels, }), @@ -316,13 +346,17 @@ func NewDestination(cfg DestinationConfig) *Destination { Name: "segment_size_bytes", Help: "Size of segment.", ConstLabels: constLabels, - Buckets: []float64{1 << 10, 1 << 11, 1 << 12, 1 << 13, 1 << 14, 1 << 15, 1 << 16, 1 << 17, 1 << 18, 1 << 19, 1 << 20}, + Buckets: []float64{ + 1 << 10, 1 << 11, 1 << 12, 1 << 13, 1 << 14, 1 << 15, + 1 << 16, 1 << 17, 1 << 18, 1 << 19, 1 << 20, + }, }, ), }, } } +// RegisterMetrics registers the metrics for the [Destination]. func (d *Destination) RegisterMetrics(registerer prometheus.Registerer) { registerer.MustRegister(d.metrics.samplesTotal) registerer.MustRegister(d.metrics.exemplarsTotal) @@ -360,6 +394,7 @@ func (d *Destination) RegisterMetrics(registerer prometheus.Registerer) { registerer.MustRegister(d.metrics.segmentSizeInBytes) } +// UnregisterMetrics unregisters the metrics for the [Destination]. func (d *Destination) UnregisterMetrics(registerer prometheus.Registerer) { registerer.Unregister(d.metrics.samplesTotal) registerer.Unregister(d.metrics.exemplarsTotal) @@ -397,18 +432,23 @@ func (d *Destination) UnregisterMetrics(registerer prometheus.Registerer) { registerer.Unregister(d.metrics.segmentSizeInBytes) } +// remoteWriteConfigsAreEqual compares two remote write configs. +// +//nolint:gocritic // this is a compares configs func remoteWriteConfigsAreEqual(lrwc, rwrc config.RemoteWriteConfig) bool { ldata, _ := yaml.Marshal(lrwc) rdata, _ := yaml.Marshal(rwrc) return bytes.Equal(ldata, rdata) } +// maxTimestamp is a metric for the highest sent timestamp. type maxTimestamp struct { mtx sync.Mutex value float64 prometheus.Gauge } +// Set sets the value of the metric. func (m *maxTimestamp) Set(value float64) { m.mtx.Lock() defer m.mtx.Unlock() @@ -418,18 +458,21 @@ func (m *maxTimestamp) Set(value float64) { } } +// Get gets the value of the metric. func (m *maxTimestamp) Get() float64 { m.mtx.Lock() defer m.mtx.Unlock() return m.value } +// Collect collects the metric. func (m *maxTimestamp) Collect(c chan<- prometheus.Metric) { if m.Get() > 0 { m.Gauge.Collect(c) } } +// DestinationMetrics is container for the metrics of the [Destination]. type DestinationMetrics struct { samplesTotal prometheus.Counter exemplarsTotal prometheus.Counter diff --git a/pp/go/storage/remotewriter/iterator.go b/pp/go/storage/remotewriter/iterator.go index 368340799d..f16b2c6479 100644 --- a/pp/go/storage/remotewriter/iterator.go +++ b/pp/go/storage/remotewriter/iterator.go @@ -18,6 +18,11 @@ import ( "github.com/prometheus/prometheus/pp/go/logger" ) +// +// DataSource +// + +// DataSource is a implementation of data source. type DataSource interface { Read(ctx context.Context, targetSegmentID uint32, minTimestamp int64) ([]*DecodedSegment, error) LSSes() []*cppbridge.LabelSetStorage @@ -25,11 +30,21 @@ type DataSource interface { Close() error } +// +// TargetSegmentIDSetCloser +// + +// TargetSegmentIDSetCloser is a implementation of target segment id set closer. type TargetSegmentIDSetCloser interface { SetTargetSegmentID(segmentID uint32) error Close() error } +// +// ProtobufWriter +// + +// ProtobufWriter is a implementation of protobuf writer. type ProtobufWriter interface { Write(ctx context.Context, data *cppbridge.SnappyProtobufEncodedData) error } @@ -40,17 +55,19 @@ type sharder struct { numberOfShards int } -func newSharder(min, max int) (*sharder, error) { - if min > max || min <= 0 { - return nil, fmt.Errorf("failed to create sharder, min: %d, max: %d", min, max) +// newSharder creates a new [sharder]. +func newSharder(minShards, maxShards int) (*sharder, error) { + if minShards > maxShards || minShards <= 0 { + return nil, fmt.Errorf("failed to create sharder, min: %d, max: %d", minShards, maxShards) } return &sharder{ - min: min, - max: max, - numberOfShards: min, + min: minShards, + max: maxShards, + numberOfShards: minShards, }, nil } +// Apply applies the value for the number of shards to the sharder. func (s *sharder) Apply(value float64) { newValue := int(math.Ceil(value)) if newValue < s.min { @@ -74,17 +91,18 @@ func (s *sharder) BestNumberOfShards(value float64) int { return newValue } +// NumberOfShards returns the number of shards. func (s *sharder) NumberOfShards() int { return s.numberOfShards } +// Iterator is a iterator for sending data to the remote storage. type Iterator struct { clock clockwork.Clock queueConfig config.QueueConfig dataSource DataSource protobufWriter ProtobufWriter targetSegmentIDSetCloser TargetSegmentIDSetCloser - segmentReadyChecker SegmentReadyChecker metrics *DestinationMetrics targetSegmentID uint32 targetSegmentIsPartiallyRead bool @@ -95,6 +113,7 @@ type Iterator struct { endOfBlockReached bool } +// MessageShard is a shard of a message for sending to the remote storage. type MessageShard struct { Protobuf *cppbridge.SnappyProtobufEncodedData Size uint64 @@ -104,24 +123,29 @@ type MessageShard struct { PostProcessed bool } +// Message is a message for sending to the remote storage. type Message struct { MaxTimestamp int64 Shards []*MessageShard } +// HasDataToDeliver checks if the message has data to deliver. func (m *Message) HasDataToDeliver() bool { for _, shrd := range m.Shards { if !shrd.Delivered { return true } } + return false } +// IsObsoleted checks if the message is obsoleted. func (m *Message) IsObsoleted(minTimestamp int64) bool { return m.MaxTimestamp < minTimestamp } +// newIterator creates a new [Iterator]. func newIterator( clock clockwork.Clock, queueConfig config.QueueConfig, @@ -150,6 +174,7 @@ func newIterator( }, nil } +// wrapError wraps the error. func (i *Iterator) wrapError(err error) error { if err != nil { return err @@ -162,6 +187,11 @@ func (i *Iterator) wrapError(err error) error { return nil } +// Next reads data from the data source and writes it to the protobuf writer. +// +//revive:disable-next-line:function-length // long but readable +//revive:disable-next-line:cyclomatic // long but readable +//revive:disable-next-line:cognitive-complexity // long but readable func (i *Iterator) Next(ctx context.Context) error { if i.endOfBlockReached { return i.wrapError(nil) @@ -248,7 +278,7 @@ readLoop: i.writeCaches() - msg, err := i.encode(b.Data(), uint16(bestNumberOfShards)) + msg, err := i.encode(b.Data(), uint16(bestNumberOfShards)) // #nosec G115 // no overflow if err != nil { return i.wrapError(err) } @@ -413,6 +443,7 @@ func (i *Iterator) minTimestamp() int64 { return i.clock.Now().Add(-sampleAgeLimit).UnixMilli() } +// Close closes the iterator. func (i *Iterator) Close() error { return errors.Join(i.dataSource.Close(), i.targetSegmentIDSetCloser.Close()) } @@ -428,7 +459,8 @@ type batch struct { maxNumberOfSamplesPerShard int } -func newBatch(numberOfShards int, maxNumberOfSamplesPerShard int) *batch { +// newBatch creates a new [batch]. +func newBatch(numberOfShards, maxNumberOfSamplesPerShard int) *batch { return &batch{ numberOfShards: numberOfShards, maxNumberOfSamplesPerShard: maxNumberOfSamplesPerShard, diff --git a/pp/go/storage/remotewriter/remotewriter.go b/pp/go/storage/remotewriter/remotewriter.go index a05805fab7..1d88242db4 100644 --- a/pp/go/storage/remotewriter/remotewriter.go +++ b/pp/go/storage/remotewriter/remotewriter.go @@ -102,7 +102,7 @@ func (rw *RemoteWriter) Run(ctx context.Context) error { } } - for _, config := range configs { + for _, config := range configs { //nolint:gocritic // hugeParam // constructor destination, ok := destinations[config.Name] if !ok { destination = NewDestination(config) diff --git a/pp/go/storage/remotewriter/remotewriter_test.go b/pp/go/storage/remotewriter/remotewriter_test.go index 24d67b2519..cbcdeb1323 100644 --- a/pp/go/storage/remotewriter/remotewriter_test.go +++ b/pp/go/storage/remotewriter/remotewriter_test.go @@ -9,7 +9,7 @@ import ( "github.com/prometheus/prometheus/pp/go/storage/ready" ) -func TestRemoteWriter_Run(t *testing.T) { +func TestRemoteWriter_Run(_ *testing.T) { rw := New("", nil, clockwork.NewFakeClock(), ready.NoOpNotifier{}, prometheus.DefaultRegisterer) _ = rw } diff --git a/pp/go/storage/remotewriter/writeloop.go b/pp/go/storage/remotewriter/writeloop.go index d639d43463..84d68c4eb7 100644 --- a/pp/go/storage/remotewriter/writeloop.go +++ b/pp/go/storage/remotewriter/writeloop.go @@ -36,6 +36,11 @@ func newWriteLoop(dataDir string, destination *Destination, hcatalog Catalog, cl } } +// run sending data via RemoteWriter. +// +//revive:disable-next-line:cyclomatic // but readable +//revive:disable-next-line:function-length // long but readable +//revive:disable-next-line:cognitive-complexity // long but readable func (wl *writeLoop) run(ctx context.Context) { var delay time.Duration var err error @@ -44,8 +49,9 @@ func (wl *writeLoop) run(ctx context.Context) { rw := &readyProtobufWriter{} - wl.destination.metrics.maxNumShards.Set(float64(wl.destination.Config().QueueConfig.MaxShards)) - wl.destination.metrics.minNumShards.Set(float64(wl.destination.Config().QueueConfig.MinShards)) + dcfg := wl.destination.Config() + wl.destination.metrics.maxNumShards.Set(float64(dcfg.QueueConfig.MaxShards)) + wl.destination.metrics.minNumShards.Set(float64(dcfg.QueueConfig.MinShards)) defer func() { if i != nil { @@ -114,6 +120,9 @@ func (wl *writeLoop) run(ctx context.Context) { } } +// createClient creates a new [remote.WriteClient]. +// +//nolint:gocritic // hugeParam // this is a constructor for new client func createClient(config DestinationConfig) (client remote.WriteClient, err error) { clientConfig := remote.ClientConfig{ URL: config.URL, @@ -133,7 +142,8 @@ func createClient(config DestinationConfig) (client remote.WriteClient, err erro return client, nil } -func (wl *writeLoop) write(ctx context.Context, iterator *Iterator) error { +// write writes data from iterator to the remote write storage. +func (*writeLoop) write(ctx context.Context, iterator *Iterator) error { for { select { case <-ctx.Done(): @@ -150,27 +160,35 @@ func (wl *writeLoop) write(ctx context.Context, iterator *Iterator) error { } } +// nextIterator returns next iterator. +// +//revive:disable-next-line:cyclomatic // this is a constructor for new iterator +//revive:disable-next-line:function-length // this is a constructor for new iterator func (wl *writeLoop) nextIterator(ctx context.Context, protobufWriter ProtobufWriter) (*Iterator, error) { var nextHeadRecord *catalog.Record var err error var cleanStart bool + dcfg := wl.destination.Config() if wl.currentHeadID != nil { nextHeadRecord, err = nextHead(ctx, wl.dataDir, wl.catalog, *wl.currentHeadID) } else { var headFound bool - nextHeadRecord, headFound, err = scanForNextHead(ctx, wl.dataDir, wl.catalog, wl.destination.Config().Name) + nextHeadRecord, headFound, err = scanForNextHead(ctx, wl.dataDir, wl.catalog, dcfg.Name) cleanStart = !headFound } if err != nil { return nil, fmt.Errorf("find next head: %w", err) } headDir := filepath.Join(wl.dataDir, nextHeadRecord.Dir()) - crw, err := NewCursorReadWriter(filepath.Join(headDir, fmt.Sprintf("%s.cursor", wl.destination.Config().Name)), nextHeadRecord.NumberOfShards()) + crw, err := NewCursorReadWriter( + filepath.Join(headDir, fmt.Sprintf("%s.cursor", dcfg.Name)), + nextHeadRecord.NumberOfShards(), + ) if err != nil { return nil, fmt.Errorf("create cursor: %w", err) } - crc32, err := wl.destination.Config().CRC32() + crc32, err := dcfg.CRC32() if err != nil { return nil, errors.Join(fmt.Errorf("calculate crc32: %w", err), crw.Close()) } @@ -186,7 +204,7 @@ func (wl *writeLoop) nextIterator(ctx context.Context, protobufWriter ProtobufWr ds, err := newDataSource( headDir, nextHeadRecord.NumberOfShards(), - wl.destination.Config(), + dcfg, discardCache, newSegmentReadyChecker(nextHeadRecord), wl.makeCorruptMarker(), @@ -214,11 +232,11 @@ func (wl *writeLoop) nextIterator(ctx context.Context, protobufWriter ProtobufWr i, err := newIterator( wl.clock, - wl.destination.Config().QueueConfig, + dcfg.QueueConfig, ds, crw, targetSegmentID, - wl.destination.Config().ReadTimeout, + dcfg.ReadTimeout, protobufWriter, wl.destination.metrics, ) @@ -247,6 +265,10 @@ func (wl *writeLoop) makeCorruptMarker() CorruptMarker { }) } +// nextHead returns next head record from catalog. +// +//nolint:gocritic // hugeParam // this is a extractor +//revive:disable-next-line:cyclomatic // this is a extractor func nextHead(ctx context.Context, dataDir string, headCatalog Catalog, headID string) (*catalog.Record, error) { if err := contextErr(ctx); err != nil { return nil, err @@ -298,8 +320,9 @@ func nextHead(ctx context.Context, dataDir string, headCatalog Catalog, headID s return nil, fmt.Errorf("nextHead: no new heads: appropriate head not found") } +// validateHead validates head directory. func validateHead(headDir string) error { - dir, err := os.Open(headDir) + dir, err := os.Open(headDir) // #nosec G304 // it's meant to be that way if err != nil { return err } @@ -307,7 +330,13 @@ func validateHead(headDir string) error { return dir.Close() } -func scanForNextHead(ctx context.Context, dataDir string, headCatalog Catalog, destinationName string) (*catalog.Record, bool, error) { +// scanForNextHead scans catalog for next head record. +func scanForNextHead( + ctx context.Context, + dataDir string, + headCatalog Catalog, + destinationName string, +) (*catalog.Record, bool, error) { if err := contextErr(ctx); err != nil { return nil, false, err } @@ -344,8 +373,9 @@ func scanForNextHead(ctx context.Context, dataDir string, headCatalog Catalog, d return headRecords[0], false, nil } -func scanHeadForDestination(dirPath string, destinationName string) (bool, error) { - dir, err := os.Open(dirPath) +// scanHeadForDestination scans head directory for [Destination]. +func scanHeadForDestination(dirPath, destinationName string) (bool, error) { + dir, err := os.Open(dirPath) // #nosec G304 // it's meant to be that way if err != nil { return false, fmt.Errorf("open head dir: %w", err) } @@ -365,6 +395,7 @@ func scanHeadForDestination(dirPath string, destinationName string) (bool, error return false, nil } +// contextErr returns error if context is done. func contextErr(ctx context.Context) error { select { case <-ctx.Done(): @@ -374,14 +405,17 @@ func contextErr(ctx context.Context) error { } } +// readyProtobufWriter is a writer for ready protobuf. type readyProtobufWriter struct { protobufWriter ProtobufWriter } +// SetProtobufWriter sets protobuf writer. func (rpw *readyProtobufWriter) SetProtobufWriter(protobufWriter ProtobufWriter) { rpw.protobufWriter = protobufWriter } +// Write writes protobuf to the remote write storage. func (rpw *readyProtobufWriter) Write(ctx context.Context, protobuf *cppbridge.SnappyProtobufEncodedData) error { return rpw.protobufWriter.Write(ctx, protobuf) } From 90eba408bc8b556e95c3ee9f772ad3a6cd47894b Mon Sep 17 00:00:00 2001 From: Alexandr Yudin Date: Tue, 7 Oct 2025 13:57:13 +0000 Subject: [PATCH 74/96] add test querier --- .../storage/querier/load_and_query_waiter.go | 4 + pp/go/storage/querier/querier_test.go | 203 +++++++++++++++++- pp/go/storage/remotewriter/destination.go | 6 + 3 files changed, 212 insertions(+), 1 deletion(-) diff --git a/pp/go/storage/querier/load_and_query_waiter.go b/pp/go/storage/querier/load_and_query_waiter.go index 5c5bb8a289..a11c54a366 100644 --- a/pp/go/storage/querier/load_and_query_waiter.go +++ b/pp/go/storage/querier/load_and_query_waiter.go @@ -10,6 +10,7 @@ const ( dsLoadAndQuerySeriesData = "data_storage_load_and_query_series_data" ) +// LoadAndQueryWaiter is a waiter for the load and query series data task. type LoadAndQueryWaiter[ TTask Task, TDataStorage DataStorage, @@ -21,6 +22,7 @@ type LoadAndQueryWaiter[ head THead } +// NewLoadAndQueryWaiter creates a new [LoadAndQueryWaiter]. func NewLoadAndQueryWaiter[ TTask Task, TDataStorage DataStorage, @@ -33,6 +35,7 @@ func NewLoadAndQueryWaiter[ } } +// Add adds a querier to the load and query series data task. func (l *LoadAndQueryWaiter[TTask, TDataStorage, TLSS, TShard, THead]) Add(s TShard, querier uintptr) { l.waiter.Add(s.LoadAndQuerySeriesDataTask().Add(querier, func() shard.Task { t := l.head.CreateTask(dsLoadAndQuerySeriesData, func(s TShard) error { @@ -43,6 +46,7 @@ func (l *LoadAndQueryWaiter[TTask, TDataStorage, TLSS, TShard, THead]) Add(s TSh })) } +// Wait waits for the load and query series data task to complete. func (l *LoadAndQueryWaiter[TTask, TDataStorage, TLSS, TShard, THead]) Wait() error { return l.waiter.Wait() } diff --git a/pp/go/storage/querier/querier_test.go b/pp/go/storage/querier/querier_test.go index 4264873e4d..f263405bc3 100644 --- a/pp/go/storage/querier/querier_test.go +++ b/pp/go/storage/querier/querier_test.go @@ -9,7 +9,11 @@ import ( "github.com/jonboulle/clockwork" "github.com/prometheus/client_golang/prometheus" + "github.com/stretchr/testify/suite" + "github.com/prometheus/prometheus/model/labels" + prom_storage "github.com/prometheus/prometheus/storage" + "github.com/prometheus/prometheus/pp/go/cppbridge" "github.com/prometheus/prometheus/pp/go/storage" "github.com/prometheus/prometheus/pp/go/storage/catalog" @@ -18,7 +22,6 @@ import ( "github.com/prometheus/prometheus/pp/go/storage/head/task" "github.com/prometheus/prometheus/pp/go/storage/querier" "github.com/prometheus/prometheus/pp/go/storage/storagetest" - "github.com/stretchr/testify/suite" ) const ( @@ -311,3 +314,201 @@ func (s *QuerierSuite) TestInstantQueryWithDataStorageLoading() { }, }, storagetest.TimeSeriesFromSeriesSet(seriesSet)) } + +func (s *QuerierSuite) TestLabelNames() { + // Arrange + timeSeries := []storagetest.TimeSeries{ + { + Labels: labels.FromStrings("__name__", "metric0", "job0", "test0"), + Samples: []cppbridge.Sample{ + {Timestamp: 0, Value: 1}, + }, + }, + { + Labels: labels.FromStrings("__name__", "metric1", "job1", "test1"), + Samples: []cppbridge.Sample{ + {Timestamp: 0, Value: 10}, + }, + }, + } + s.appendTimeSeries(timeSeries) + + q := querier.NewQuerier(s.head, querier.NewNoOpShardedDeduplicator, 0, 2, nil, nil) + defer func() { _ = q.Close() }() + matcher, err := labels.NewMatcher(labels.MatchEqual, "__name__", "metric0") + s.Require().NoError(err) + hints := &prom_storage.LabelHints{Limit: 10} + + // Act + names, anns, err := q.LabelNames(s.context, hints, matcher) + s.Require().NoError(err) + + // Assert + s.Equal([]string{"__name__", "job0"}, names) + s.Len(anns.AsErrors(), 1) +} + +func (s *QuerierSuite) TestLabelNamesWithLimit() { + // Arrange + timeSeries := []storagetest.TimeSeries{ + { + Labels: labels.FromStrings("__name__", "metric0", "job0", "test0"), + Samples: []cppbridge.Sample{ + {Timestamp: 0, Value: 1}, + }, + }, + { + Labels: labels.FromStrings("__name__", "metric1", "job1", "test1"), + Samples: []cppbridge.Sample{ + {Timestamp: 0, Value: 10}, + }, + }, + } + s.appendTimeSeries(timeSeries) + + q := querier.NewQuerier(s.head, querier.NewNoOpShardedDeduplicator, 0, 2, nil, nil) + defer func() { _ = q.Close() }() + matcher, err := labels.NewMatcher(labels.MatchEqual, "__name__", "metric0") + s.Require().NoError(err) + hints := &prom_storage.LabelHints{Limit: 1} + + // Act + names, anns, err := q.LabelNames(s.context, hints, matcher) + s.Require().NoError(err) + + // Assert + s.Equal([]string{"__name__"}, names) + s.Len(anns.AsErrors(), 1) +} + +func (s *QuerierSuite) TestLabelNamesNoMatches() { + // Arrange + timeSeries := []storagetest.TimeSeries{ + { + Labels: labels.FromStrings("__name__", "metric0", "job0", "test0"), + Samples: []cppbridge.Sample{ + {Timestamp: 0, Value: 1}, + }, + }, + { + Labels: labels.FromStrings("__name__", "metric1", "job1", "test1"), + Samples: []cppbridge.Sample{ + {Timestamp: 0, Value: 10}, + }, + }, + } + s.appendTimeSeries(timeSeries) + + q := querier.NewQuerier(s.head, querier.NewNoOpShardedDeduplicator, 0, 2, nil, nil) + defer func() { _ = q.Close() }() + matcher, err := labels.NewMatcher(labels.MatchEqual, "__name__", "metric3") + s.Require().NoError(err) + hints := &prom_storage.LabelHints{Limit: 1} + + // Act + names, anns, err := q.LabelNames(s.context, hints, matcher) + s.Require().NoError(err) + + // Assert + s.Equal([]string{}, names) + s.Len(anns.AsErrors(), 1) +} + +func (s *QuerierSuite) TestLabelValues() { + // Arrange + timeSeries := []storagetest.TimeSeries{ + { + Labels: labels.FromStrings("__name__", "metric0", "job0", "test0"), + Samples: []cppbridge.Sample{ + {Timestamp: 0, Value: 1}, + }, + }, + { + Labels: labels.FromStrings("__name__", "metric1", "job1", "test1"), + Samples: []cppbridge.Sample{ + {Timestamp: 0, Value: 10}, + }, + }, + } + s.appendTimeSeries(timeSeries) + + q := querier.NewQuerier(s.head, querier.NewNoOpShardedDeduplicator, 0, 2, nil, nil) + defer func() { _ = q.Close() }() + matcher, err := labels.NewMatcher(labels.MatchRegexp, "__name__", "metric.*") + s.Require().NoError(err) + hints := &prom_storage.LabelHints{Limit: 10} + + // Act + names, anns, err := q.LabelValues(s.context, "__name__", hints, matcher) + s.Require().NoError(err) + + // Assert + s.Equal([]string{"metric0", "metric1"}, names) + s.Len(anns.AsErrors(), 1) +} + +func (s *QuerierSuite) TestLabelValuesNoMatches() { + // Arrange + timeSeries := []storagetest.TimeSeries{ + { + Labels: labels.FromStrings("__name__", "metric0", "job0", "test0"), + Samples: []cppbridge.Sample{ + {Timestamp: 0, Value: 1}, + }, + }, + { + Labels: labels.FromStrings("__name__", "metric1", "job1", "test1"), + Samples: []cppbridge.Sample{ + {Timestamp: 0, Value: 10}, + }, + }, + } + s.appendTimeSeries(timeSeries) + + q := querier.NewQuerier(s.head, querier.NewNoOpShardedDeduplicator, 0, 2, nil, nil) + defer func() { _ = q.Close() }() + matcher, err := labels.NewMatcher(labels.MatchEqual, "__name__", "metric2") + s.Require().NoError(err) + hints := &prom_storage.LabelHints{Limit: 10} + + // Act + names, anns, err := q.LabelValues(s.context, "__name__", hints, matcher) + s.Require().NoError(err) + + // Assert + s.Equal([]string{}, names) + s.Len(anns.AsErrors(), 1) +} + +func (s *QuerierSuite) TestLabelValuesNoMatchesOnName() { + // Arrange + timeSeries := []storagetest.TimeSeries{ + { + Labels: labels.FromStrings("__name__", "metric0", "job0", "test0"), + Samples: []cppbridge.Sample{ + {Timestamp: 0, Value: 1}, + }, + }, + { + Labels: labels.FromStrings("__name__", "metric1", "job1", "test1"), + Samples: []cppbridge.Sample{ + {Timestamp: 0, Value: 10}, + }, + }, + } + s.appendTimeSeries(timeSeries) + + q := querier.NewQuerier(s.head, querier.NewNoOpShardedDeduplicator, 0, 2, nil, nil) + defer func() { _ = q.Close() }() + matcher, err := labels.NewMatcher(labels.MatchRegexp, "__name__", "metric.*") + s.Require().NoError(err) + hints := &prom_storage.LabelHints{Limit: 10} + + // Act + names, anns, err := q.LabelValues(s.context, "instance", hints, matcher) + s.Require().NoError(err) + + // Assert + s.Equal([]string{}, names) + s.Len(anns.AsErrors(), 1) +} diff --git a/pp/go/storage/remotewriter/destination.go b/pp/go/storage/remotewriter/destination.go index bda340459f..826a732c9b 100644 --- a/pp/go/storage/remotewriter/destination.go +++ b/pp/go/storage/remotewriter/destination.go @@ -357,6 +357,9 @@ func NewDestination(cfg DestinationConfig) *Destination { } // RegisterMetrics registers the metrics for the [Destination]. +// +//nolint:dupl // it's not duplicate, it's different function +//revive:disable-next-line:function-length // register metrics func (d *Destination) RegisterMetrics(registerer prometheus.Registerer) { registerer.MustRegister(d.metrics.samplesTotal) registerer.MustRegister(d.metrics.exemplarsTotal) @@ -395,6 +398,9 @@ func (d *Destination) RegisterMetrics(registerer prometheus.Registerer) { } // UnregisterMetrics unregisters the metrics for the [Destination]. +// +//nolint:dupl // it's not duplicate, it's different function +//revive:disable-next-line:function-length // register metrics func (d *Destination) UnregisterMetrics(registerer prometheus.Registerer) { registerer.Unregister(d.metrics.samplesTotal) registerer.Unregister(d.metrics.exemplarsTotal) From aed6be65692c99be7a60726a18aa1d46a3fd699c Mon Sep 17 00:00:00 2001 From: Alexandr Yudin Date: Tue, 7 Oct 2025 15:06:56 +0000 Subject: [PATCH 75/96] for save --- pp/go/storage/head/head/head_test.go | 14 ++++++++++++++ 1 file changed, 14 insertions(+) diff --git a/pp/go/storage/head/head/head_test.go b/pp/go/storage/head/head/head_test.go index f3d718c6f3..e80f4997eb 100644 --- a/pp/go/storage/head/head/head_test.go +++ b/pp/go/storage/head/head/head_test.go @@ -3,6 +3,8 @@ package head_test import ( "testing" + "github.com/stretchr/testify/suite" + "github.com/prometheus/prometheus/pp/go/cppbridge" "github.com/prometheus/prometheus/pp/go/storage/head/head" "github.com/prometheus/prometheus/pp/go/storage/head/shard" @@ -60,3 +62,15 @@ func (*testWal) Sync() error { func (*testWal) Write(_ []*cppbridge.InnerSeries) (bool, error) { return false, nil } + +type HeadSuite struct { + suite.Suite +} + +func TestHeadSuite(t *testing.T) { + suite.Run(t, new(HeadSuite)) +} + +func (s *HeadSuite) TestHappyPath() { + // +} From fce63952f0f3adf3412cd99695b8c28e361c9c55 Mon Sep 17 00:00:00 2001 From: Alexandr Yudin Date: Wed, 8 Oct 2025 09:11:39 +0000 Subject: [PATCH 76/96] add test head --- pp/go/storage/head/head/head.go | 4 + pp/go/storage/head/head/head_moq_test.go | 100 +++++++ pp/go/storage/head/head/head_test.go | 352 ++++++++++++++++++++--- 3 files changed, 416 insertions(+), 40 deletions(-) create mode 100644 pp/go/storage/head/head/head_moq_test.go diff --git a/pp/go/storage/head/head/head.go b/pp/go/storage/head/head/head.go index aeed2ee3d6..8014f05bf2 100644 --- a/pp/go/storage/head/head/head.go +++ b/pp/go/storage/head/head/head.go @@ -16,6 +16,8 @@ import ( "github.com/prometheus/prometheus/pp/go/util/locker" ) +//go:generate -command moq go run github.com/matryer/moq --rm --skip-ensure --pkg head_test --out + // ExtraWorkers number of extra workers for operation on shards. var ExtraWorkers = 0 @@ -27,6 +29,8 @@ const defaultNumberOfWorkers = 2 // // Shard the minimum required head Shard implementation. +// +//go:generate moq head_moq_test.go . Shard type Shard interface { // ShardID returns the shard ID. ShardID() uint16 diff --git a/pp/go/storage/head/head/head_moq_test.go b/pp/go/storage/head/head/head_moq_test.go new file mode 100644 index 0000000000..bf5b0e7321 --- /dev/null +++ b/pp/go/storage/head/head/head_moq_test.go @@ -0,0 +1,100 @@ +// Code generated by moq; DO NOT EDIT. +// github.com/matryer/moq + +package head_test + +import ( + "sync" +) + +// ShardMock is a mock implementation of head.Shard. +// +// func TestSomethingThatUsesShard(t *testing.T) { +// +// // make and configure a mocked head.Shard +// mockedShard := &ShardMock{ +// CloseFunc: func() error { +// panic("mock out the Close method") +// }, +// ShardIDFunc: func() uint16 { +// panic("mock out the ShardID method") +// }, +// } +// +// // use mockedShard in code that requires head.Shard +// // and then make assertions. +// +// } +type ShardMock struct { + // CloseFunc mocks the Close method. + CloseFunc func() error + + // ShardIDFunc mocks the ShardID method. + ShardIDFunc func() uint16 + + // calls tracks calls to the methods. + calls struct { + // Close holds details about calls to the Close method. + Close []struct { + } + // ShardID holds details about calls to the ShardID method. + ShardID []struct { + } + } + lockClose sync.RWMutex + lockShardID sync.RWMutex +} + +// Close calls CloseFunc. +func (mock *ShardMock) Close() error { + if mock.CloseFunc == nil { + panic("ShardMock.CloseFunc: method is nil but Shard.Close was just called") + } + callInfo := struct { + }{} + mock.lockClose.Lock() + mock.calls.Close = append(mock.calls.Close, callInfo) + mock.lockClose.Unlock() + return mock.CloseFunc() +} + +// CloseCalls gets all the calls that were made to Close. +// Check the length with: +// +// len(mockedShard.CloseCalls()) +func (mock *ShardMock) CloseCalls() []struct { +} { + var calls []struct { + } + mock.lockClose.RLock() + calls = mock.calls.Close + mock.lockClose.RUnlock() + return calls +} + +// ShardID calls ShardIDFunc. +func (mock *ShardMock) ShardID() uint16 { + if mock.ShardIDFunc == nil { + panic("ShardMock.ShardIDFunc: method is nil but Shard.ShardID was just called") + } + callInfo := struct { + }{} + mock.lockShardID.Lock() + mock.calls.ShardID = append(mock.calls.ShardID, callInfo) + mock.lockShardID.Unlock() + return mock.ShardIDFunc() +} + +// ShardIDCalls gets all the calls that were made to ShardID. +// Check the length with: +// +// len(mockedShard.ShardIDCalls()) +func (mock *ShardMock) ShardIDCalls() []struct { +} { + var calls []struct { + } + mock.lockShardID.RLock() + calls = mock.calls.ShardID + mock.lockShardID.RUnlock() + return calls +} diff --git a/pp/go/storage/head/head/head_test.go b/pp/go/storage/head/head/head_test.go index e80f4997eb..91713b1b01 100644 --- a/pp/go/storage/head/head/head_test.go +++ b/pp/go/storage/head/head/head_test.go @@ -1,76 +1,348 @@ package head_test import ( + "sync" + "sync/atomic" "testing" "github.com/stretchr/testify/suite" - "github.com/prometheus/prometheus/pp/go/cppbridge" "github.com/prometheus/prometheus/pp/go/storage/head/head" - "github.com/prometheus/prometheus/pp/go/storage/head/shard" ) -func TestXxx(t *testing.T) { - lss := &shard.LSS{} - ds := shard.NewDataStorage() - wl := &testWal{} - sd := shard.NewShard(lss, ds, nil, nil, wl, 0) - id := "test-head-id" - generation := uint64(0) +type HeadSuite struct { + suite.Suite + + id string + generation uint64 +} +func TestHeadSuite(t *testing.T) { + suite.Run(t, new(HeadSuite)) +} + +func (s *HeadSuite) SetupSuite() { + s.id = "test-head-id" + s.generation = uint64(42) +} + +func (s *HeadSuite) TestClose() { + sd := &ShardMock{ + ShardIDFunc: func() uint16 { return 0 }, + CloseFunc: func() error { return nil }, + } + + closeCount := 0 h := head.NewHead( - id, - []*shard.Shard{sd}, - shard.NewPerGoroutineShard[*testWal], + s.id, + []*ShardMock{sd}, + newPerGoroutineShardMockfunc, + func() { closeCount++ }, + s.generation, nil, - generation, + ) + + s.T().Log("first close head", h.String()) + err := h.Close() + s.Require().NoError(err) + + s.Len(sd.CloseCalls(), 1) + s.Equal(1, closeCount) + + s.T().Log("second close head", h.String()) + err = h.Close() + s.Require().NoError(err) + + s.Len(sd.CloseCalls(), 1) + s.Equal(1, closeCount) +} + +func (s *HeadSuite) TestConcurrency() { + sd := &ShardMock{ + ShardIDFunc: func() uint16 { return 0 }, + CloseFunc: func() error { return nil }, + } + + h := head.NewHead( + s.id, + []*ShardMock{sd, sd}, + newPerGoroutineShardMockfunc, + nil, + s.generation, nil, ) + defer h.Close() - t.Log(h) + s.Equal(int64(4), h.Concurrency()) } -// testWal test implementation wal. -type testWal struct{} +func (s *HeadSuite) TestEnqueue() { + sd0 := &ShardMock{ + ShardIDFunc: func() uint16 { return 0 }, + CloseFunc: func() error { return nil }, + } + + sd1 := &ShardMock{ + ShardIDFunc: func() uint16 { return 1 }, + CloseFunc: func() error { return nil }, + } + + h := head.NewHead( + s.id, + []*ShardMock{sd0, sd1}, + newPerGoroutineShardMockfunc, + nil, + s.generation, + nil, + ) + defer h.Close() + + shardsExecuted := uint32(0) + t := h.CreateTask("test-task", func(shard *perGoroutineShardMock) error { + atomic.AddUint32(&shardsExecuted, uint32(shard.ShardID()+1)) + return nil + }) + + h.Enqueue(t) -// Close test implementation wal. -func (*testWal) Close() error { - return nil + err := t.Wait() + s.Require().NoError(err) + + s.Equal(uint32(3), shardsExecuted) } -// Commit test implementation wal. -func (*testWal) Commit() error { - return nil +func (s *HeadSuite) TestEnqueueOnShard() { + sd0 := &ShardMock{ + ShardIDFunc: func() uint16 { return 0 }, + CloseFunc: func() error { return nil }, + } + + sd1 := &ShardMock{ + ShardIDFunc: func() uint16 { return 1 }, + CloseFunc: func() error { return nil }, + } + + h := head.NewHead( + s.id, + []*ShardMock{sd0, sd1}, + newPerGoroutineShardMockfunc, + nil, + s.generation, + nil, + ) + defer h.Close() + + shardActual := uint16(1<<16 - 1) + expectedShard := uint16(1) + t := h.CreateTask("test-task", func(shard *perGoroutineShardMock) error { + shardActual = shard.ShardID() + return nil + }) + + h.EnqueueOnShard(t, expectedShard) + + err := t.Wait() + s.Require().NoError(err) + + s.Equal(expectedShard, shardActual) + + expectedShard = uint16(0) + t = h.CreateTask("test-task", func(shard *perGoroutineShardMock) error { + shardActual = shard.ShardID() + return nil + }) + + h.EnqueueOnShard(t, expectedShard) + + err = t.Wait() + s.Require().NoError(err) + + s.Equal(expectedShard, shardActual) } -// CurrentSize test implementation wal. -func (*testWal) CurrentSize() int64 { - return 0 +func (s *HeadSuite) TestGeneration() { + sd := &ShardMock{ + ShardIDFunc: func() uint16 { return 0 }, + CloseFunc: func() error { return nil }, + } + + h := head.NewHead( + s.id, + []*ShardMock{sd, sd}, + newPerGoroutineShardMockfunc, + nil, + s.generation, + nil, + ) + defer h.Close() + + s.Equal(s.generation, h.Generation()) } -// Flush test implementation wal. -func (*testWal) Flush() error { - return nil +func (s *HeadSuite) TestID() { + sd := &ShardMock{ + ShardIDFunc: func() uint16 { return 0 }, + CloseFunc: func() error { return nil }, + } + + h := head.NewHead( + s.id, + []*ShardMock{sd, sd}, + newPerGoroutineShardMockfunc, + nil, + s.generation, + nil, + ) + defer h.Close() + + s.Equal(s.id, h.ID()) } -// Sync test implementation wal. -func (*testWal) Sync() error { - return nil +func (s *HeadSuite) TestIsReadOnly() { + sd := &ShardMock{ + ShardIDFunc: func() uint16 { return 0 }, + CloseFunc: func() error { return nil }, + } + + h := head.NewHead( + s.id, + []*ShardMock{sd, sd}, + newPerGoroutineShardMockfunc, + nil, + s.generation, + nil, + ) + defer h.Close() + + s.False(h.IsReadOnly()) + + h.SetReadOnly() + s.True(h.IsReadOnly()) } -// Write test implementation wal. -func (*testWal) Write(_ []*cppbridge.InnerSeries) (bool, error) { - return false, nil +func (s *HeadSuite) TestNumberOfShards() { + sd := &ShardMock{ + ShardIDFunc: func() uint16 { return 0 }, + CloseFunc: func() error { return nil }, + } + + h := head.NewHead( + s.id, + []*ShardMock{sd, sd, sd}, + newPerGoroutineShardMockfunc, + nil, + s.generation, + nil, + ) + defer h.Close() + + s.Equal(uint16(3), h.NumberOfShards()) } -type HeadSuite struct { - suite.Suite +func (s *HeadSuite) TestRangeQueueSize() { + sd0 := &ShardMock{ + ShardIDFunc: func() uint16 { return 0 }, + CloseFunc: func() error { return nil }, + } + + sd1 := &ShardMock{ + ShardIDFunc: func() uint16 { return 1 }, + CloseFunc: func() error { return nil }, + } + + h := head.NewHead( + s.id, + []*ShardMock{sd0, sd1}, + newPerGoroutineShardMockfunc, + nil, + s.generation, + nil, + ) + defer h.Close() + + execute := sync.WaitGroup{} + execute.Add(4) + + done := make(chan struct{}) + t1 := h.CreateTask("test-task", func(_ *perGoroutineShardMock) error { + execute.Done() + <-done + return nil + }) + h.Enqueue(t1) + + t2 := h.CreateTask("test-task", func(_ *perGoroutineShardMock) error { + execute.Done() + <-done + return nil + }) + h.Enqueue(t2) + + execute.Wait() + + t3 := h.CreateTask("test-task", func(_ *perGoroutineShardMock) error { + <-done + return nil + }) + h.Enqueue(t3) + + expectedShardID := 0 + for shardID, size := range h.RangeQueueSize() { + s.Equal(expectedShardID, shardID) + s.Equal(1, size) + expectedShardID++ + } + + close(done) + + err := t1.Wait() + s.Require().NoError(err) + + err = t2.Wait() + s.Require().NoError(err) + + err = t2.Wait() + s.Require().NoError(err) } -func TestHeadSuite(t *testing.T) { - suite.Run(t, new(HeadSuite)) +func (s *HeadSuite) TestRangeShards() { + sd0 := &ShardMock{ + ShardIDFunc: func() uint16 { return 0 }, + CloseFunc: func() error { return nil }, + } + + sd1 := &ShardMock{ + ShardIDFunc: func() uint16 { return 1 }, + CloseFunc: func() error { return nil }, + } + + h := head.NewHead( + s.id, + []*ShardMock{sd0, sd1}, + newPerGoroutineShardMockfunc, + nil, + s.generation, + nil, + ) + defer h.Close() + + expectedShardID := uint16(0) + for shard := range h.RangeShards() { + s.Equal(expectedShardID, shard.ShardID()) + expectedShardID++ + } +} + +// +// perGoroutineShardMock +// + +// perGoroutineShardMock mock for [PerGoroutineShard]. +type perGoroutineShardMock struct { + *ShardMock } -func (s *HeadSuite) TestHappyPath() { - // +// newPerGoroutineShardMockfunc constructor for [PerGoroutineShard]. +func newPerGoroutineShardMockfunc(sd *ShardMock, _ uint16) *perGoroutineShardMock { + return &perGoroutineShardMock{ShardMock: sd} } From bc3bf1d81921b369db0f92b4e64836c5ad770f27 Mon Sep 17 00:00:00 2001 From: Alexandr Yudin Date: Wed, 8 Oct 2025 11:28:38 +0000 Subject: [PATCH 77/96] for save --- pp/go/storage/head/head/head.go | 3 +- pp/go/storage/head/services/commiter_test.go | 119 +++++++++++++++++++ pp/go/storage/head/services/interface.go | 1 + pp/go/storage/head/services/mock/mediator.go | 63 ++++++++++ 4 files changed, 184 insertions(+), 2 deletions(-) create mode 100644 pp/go/storage/head/services/commiter_test.go create mode 100644 pp/go/storage/head/services/mock/mediator.go diff --git a/pp/go/storage/head/head/head.go b/pp/go/storage/head/head/head.go index 8014f05bf2..a10e84e08f 100644 --- a/pp/go/storage/head/head/head.go +++ b/pp/go/storage/head/head/head.go @@ -17,6 +17,7 @@ import ( ) //go:generate -command moq go run github.com/matryer/moq --rm --skip-ensure --pkg head_test --out +//go:generate moq head_moq_test.go . Shard // ExtraWorkers number of extra workers for operation on shards. var ExtraWorkers = 0 @@ -29,8 +30,6 @@ const defaultNumberOfWorkers = 2 // // Shard the minimum required head Shard implementation. -// -//go:generate moq head_moq_test.go . Shard type Shard interface { // ShardID returns the shard ID. ShardID() uint16 diff --git a/pp/go/storage/head/services/commiter_test.go b/pp/go/storage/head/services/commiter_test.go new file mode 100644 index 0000000000..65bb5eb264 --- /dev/null +++ b/pp/go/storage/head/services/commiter_test.go @@ -0,0 +1,119 @@ +package services_test + +import ( + "context" + "os" + "path/filepath" + "testing" + + "github.com/jonboulle/clockwork" + "github.com/stretchr/testify/suite" + + "github.com/prometheus/prometheus/pp/go/storage" + "github.com/prometheus/prometheus/pp/go/storage/catalog" + "github.com/prometheus/prometheus/pp/go/storage/head/container" + "github.com/prometheus/prometheus/pp/go/storage/head/services" + "github.com/prometheus/prometheus/pp/go/storage/head/services/mock" +) + +type CommitterSuite struct { + suite.Suite + + baseCtx context.Context + dataDir string + log *catalog.FileLog + catalog *catalog.Catalog + activeHeadContainer *container.Weighted[storage.HeadOnDisk, *storage.HeadOnDisk] +} + +func TestCommitterSuite(t *testing.T) { + suite.Run(t, new(CommitterSuite)) +} + +func (s *CommitterSuite) SetupSuite() { + s.baseCtx = context.Background() +} + +func (s *CommitterSuite) SetupTest() { + s.createDataDirectory() + s.createCatalog() + s.activeHeadContainer = container.NewWeighted(s.createHead()) +} + +func (s *CommitterSuite) TearDownTest() { + s.dataDir = "" + + if s.log != nil { + s.NoError(s.log.Close()) + s.log = nil + } + + if s.catalog != nil { + s.catalog = nil + } + + if s.activeHeadContainer != nil { + s.NoError(s.activeHeadContainer.Close()) + s.activeHeadContainer = nil + } +} + +func (s *CommitterSuite) createDataDirectory() { + dataDir := filepath.Join(s.T().TempDir(), "data") + s.Require().NoError(os.MkdirAll(dataDir, os.ModeDir)) + s.dataDir = dataDir +} + +func (s *CommitterSuite) createCatalog() { + l, err := catalog.NewFileLogV2(filepath.Join(s.dataDir, "head.log")) + s.Require().NoError(err) + clock := clockwork.NewRealClock() + + s.catalog, err = catalog.New( + clock, + l, + &catalog.DefaultIDGenerator{}, + catalog.DefaultMaxLogFileSize, + nil, + ) + s.Require().NoError(err) +} + +func (s *CommitterSuite) createHead() *storage.HeadOnDisk { + h, err := storage.NewBuilder( + s.catalog, + s.dataDir, + maxSegmentSize, + nil, + unloadDataStorageInterval, + ).Build(0, shardsCount) + s.Require().NoError(err) + + return h +} + +func (s *CommitterSuite) TestCommitter() { + trigger := make(chan struct{}, 1) + start := make(chan struct{}) + mediator := &mock.MediatorMock{ + CFunc: func() <-chan struct{} { + close(start) + return trigger + }, + } + isNewHead := func(string) bool { + return false + } + committer := services.NewCommitter(s.activeHeadContainer, mediator, isNewHead) + + done := make(chan struct{}) + go func() { + s.NoError(committer.Execute(s.baseCtx)) + close(done) + }() + + <-start + trigger <- struct{}{} + close(trigger) + <-done +} diff --git a/pp/go/storage/head/services/interface.go b/pp/go/storage/head/services/interface.go index 46df84467b..48339fa056 100644 --- a/pp/go/storage/head/services/interface.go +++ b/pp/go/storage/head/services/interface.go @@ -11,6 +11,7 @@ import ( //go:generate -command moq go run github.com/matryer/moq --rm --skip-ensure --pkg mock --out //go:generate moq mock/persistener.go . HeadBlockWriter WriteNotifier +//go:generate moq mock/mediator.go . Mediator // // ActiveHeadContainer diff --git a/pp/go/storage/head/services/mock/mediator.go b/pp/go/storage/head/services/mock/mediator.go new file mode 100644 index 0000000000..ed0357c916 --- /dev/null +++ b/pp/go/storage/head/services/mock/mediator.go @@ -0,0 +1,63 @@ +// Code generated by moq; DO NOT EDIT. +// github.com/matryer/moq + +package mock + +import ( + "sync" +) + +// MediatorMock is a mock implementation of services.Mediator. +// +// func TestSomethingThatUsesMediator(t *testing.T) { +// +// // make and configure a mocked services.Mediator +// mockedMediator := &MediatorMock{ +// CFunc: func() <-chan struct{} { +// panic("mock out the C method") +// }, +// } +// +// // use mockedMediator in code that requires services.Mediator +// // and then make assertions. +// +// } +type MediatorMock struct { + // CFunc mocks the C method. + CFunc func() <-chan struct{} + + // calls tracks calls to the methods. + calls struct { + // C holds details about calls to the C method. + C []struct { + } + } + lockC sync.RWMutex +} + +// C calls CFunc. +func (mock *MediatorMock) C() <-chan struct{} { + if mock.CFunc == nil { + panic("MediatorMock.CFunc: method is nil but Mediator.C was just called") + } + callInfo := struct { + }{} + mock.lockC.Lock() + mock.calls.C = append(mock.calls.C, callInfo) + mock.lockC.Unlock() + return mock.CFunc() +} + +// CCalls gets all the calls that were made to C. +// Check the length with: +// +// len(mockedMediator.CCalls()) +func (mock *MediatorMock) CCalls() []struct { +} { + var calls []struct { + } + mock.lockC.RLock() + calls = mock.calls.C + mock.lockC.RUnlock() + return calls +} From d745cf7beb9f9ec103767d90db0d7a6e7fc3bba6 Mon Sep 17 00:00:00 2001 From: Alexandr Yudin Date: Wed, 8 Oct 2025 12:36:05 +0000 Subject: [PATCH 78/96] fix test mediator --- pp/go/storage/mediator/mediator_test.go | 23 +++++++++++++++-------- 1 file changed, 15 insertions(+), 8 deletions(-) diff --git a/pp/go/storage/mediator/mediator_test.go b/pp/go/storage/mediator/mediator_test.go index 3a0895ebd7..1595c9a6a6 100644 --- a/pp/go/storage/mediator/mediator_test.go +++ b/pp/go/storage/mediator/mediator_test.go @@ -30,26 +30,33 @@ func (s *MediatorSuite) TestC() { } m := mediator.NewMediator(timer) - defer m.Close() counter := 0 done := make(chan struct{}) start := sync.WaitGroup{} start.Add(1) - go func() { + + s.T().Run("service_run", func(t *testing.T) { + t.Parallel() start.Done() <-m.C() counter++ close(done) - }() + }) - start.Wait() - s.T().Log("timer tick") - chTimer <- time.Time{} + s.T().Run("timer_tick", func(t *testing.T) { + t.Parallel() + start.Wait() - <-done + s.T().Log("timer tick") + chTimer <- time.Time{} + chTimer <- time.Time{} - s.Equal(1, counter) + <-done + + s.Equal(1, counter) + m.Close() + }) } func (s *MediatorSuite) TestClose() { From 144e5b1f61bd1b46286e75efe291b1ce2f9af8cc Mon Sep 17 00:00:00 2001 From: Alexandr Yudin Date: Wed, 8 Oct 2025 13:27:18 +0000 Subject: [PATCH 79/96] fix types --- cmd/prompptool/walpp.go | 3 +- pp-pkg/storage/adapter.go | 8 +- pp/go/storage/block/writer_test.go | 15 ++-- pp/go/storage/builder.go | 12 +-- pp/go/storage/head/services/commiter_test.go | 74 +++++++++++++++- .../storage/head/services/persistener_test.go | 87 ++++++++++--------- pp/go/storage/loader.go | 14 +-- pp/go/storage/loader_test.go | 13 +-- pp/go/storage/manager.go | 8 +- pp/go/storage/proxy.go | 26 +++--- pp/go/storage/storagetest/fixtures.go | 2 +- pp/go/storage/types.go | 17 +--- 12 files changed, 172 insertions(+), 107 deletions(-) diff --git a/cmd/prompptool/walpp.go b/cmd/prompptool/walpp.go index df91dd8a3c..afdc70e9e1 100644 --- a/cmd/prompptool/walpp.go +++ b/cmd/prompptool/walpp.go @@ -17,6 +17,7 @@ import ( "github.com/prometheus/prometheus/pp/go/storage" "github.com/prometheus/prometheus/pp/go/storage/block" "github.com/prometheus/prometheus/pp/go/storage/catalog" + "github.com/prometheus/prometheus/pp/go/storage/head/shard" ) type cmdWALPPToBlock struct { @@ -79,7 +80,7 @@ func (cmd *cmdWALPPToBlock) Do( level.Debug(logger).Log("msg", "catalog records", "len", len(headRecords)) - bw := block.NewWriter[*storage.ShardOnDisk]( + bw := block.NewWriter[*shard.Shard]( workingDir, block.DefaultChunkSegmentSize, time.Duration(cmd.blockDuration), diff --git a/pp-pkg/storage/adapter.go b/pp-pkg/storage/adapter.go index 94d6a3beaa..261a0430a9 100644 --- a/pp-pkg/storage/adapter.go +++ b/pp-pkg/storage/adapter.go @@ -88,7 +88,7 @@ func (ar *Adapter) AppendHashdex( ar.appendDuration.Observe(float64(time.Since(start).Microseconds())) }() - return ar.proxy.With(ctx, func(h *pp_storage.HeadOnDisk) error { + return ar.proxy.With(ctx, func(h *pp_storage.Head) error { _, _, err := appender.New(h, services.CFViaRange).Append( ctx, &appender.IncomingData{Hashdex: hashdex}, @@ -112,7 +112,7 @@ func (ar *Adapter) AppendScraperHashdex( ar.appendDuration.Observe(float64(time.Since(start).Microseconds())) }() - _ = ar.proxy.With(ctx, func(h *pp_storage.HeadOnDisk) error { + _ = ar.proxy.With(ctx, func(h *pp_storage.Head) error { _, stats, err = appender.New(h, services.CFViaRange).Append( ctx, &appender.IncomingData{Hashdex: hashdex}, @@ -148,7 +148,7 @@ func (ar *Adapter) AppendSnappyProtobuf( ar.appendDuration.Observe(float64(time.Since(start).Microseconds())) }() - return ar.proxy.With(ctx, func(h *pp_storage.HeadOnDisk) error { + return ar.proxy.With(ctx, func(h *pp_storage.Head) error { _, _, err := appender.New(h, services.CFViaRange).Append( ctx, &appender.IncomingData{Hashdex: hx}, @@ -183,7 +183,7 @@ func (ar *Adapter) AppendTimeSeries( ar.appendDuration.Observe(float64(time.Since(start).Microseconds())) }() - _ = ar.proxy.With(ctx, func(h *pp_storage.HeadOnDisk) error { + _ = ar.proxy.With(ctx, func(h *pp_storage.Head) error { _, stats, err = appender.New(h, services.CFViaRange).Append( ctx, &appender.IncomingData{Hashdex: hx, Data: data}, diff --git a/pp/go/storage/block/writer_test.go b/pp/go/storage/block/writer_test.go index b99bb59e79..ce1a070921 100644 --- a/pp/go/storage/block/writer_test.go +++ b/pp/go/storage/block/writer_test.go @@ -18,6 +18,7 @@ import ( "github.com/prometheus/prometheus/pp/go/storage/block" "github.com/prometheus/prometheus/pp/go/storage/catalog" "github.com/prometheus/prometheus/pp/go/storage/head/services" + "github.com/prometheus/prometheus/pp/go/storage/head/shard" "github.com/prometheus/prometheus/pp/go/storage/storagetest" "github.com/prometheus/prometheus/tsdb" ) @@ -35,8 +36,8 @@ const ( type WriterSuite struct { suite.Suite dataDir string - head *storage.HeadOnDisk - blockWriter *block.Writer[*storage.ShardOnDisk] + head *storage.Head + blockWriter *block.Writer[*shard.Shard] } func TestWriterSuite(t *testing.T) { @@ -46,7 +47,7 @@ func TestWriterSuite(t *testing.T) { func (s *WriterSuite) SetupTest() { s.dataDir = s.createDataDirectory() s.head = s.mustCreateHead() - s.blockWriter = block.NewWriter[*storage.ShardOnDisk]( + s.blockWriter = block.NewWriter[*shard.Shard]( s.dataDir, block.DefaultChunkSegmentSize, blockDuration, @@ -76,7 +77,7 @@ func (s *WriterSuite) mustCreateCatalog() *catalog.Catalog { return c } -func (s *WriterSuite) mustCreateHead() *storage.HeadOnDisk { +func (s *WriterSuite) mustCreateHead() *storage.Head { h, err := storage.NewBuilder( s.mustCreateCatalog(), s.dataDir, @@ -100,9 +101,9 @@ func (s *WriterSuite) mustReadBlockMeta(filename string) tsdb.BlockMeta { return meta } -func (s *WriterSuite) shard() *storage.ShardOnDisk { - for shard := range s.head.RangeShards() { - return shard +func (s *WriterSuite) shard() *shard.Shard { + for sd := range s.head.RangeShards() { + return sd } return nil diff --git a/pp/go/storage/builder.go b/pp/go/storage/builder.go index 37bff84189..13b84042e1 100644 --- a/pp/go/storage/builder.go +++ b/pp/go/storage/builder.go @@ -22,7 +22,7 @@ import ( // Builder // -// Builder building new [HeadOnDisk] with parameters. +// Builder building new [Head] with parameters. type Builder struct { catalog *catalog.Catalog dataDir string @@ -58,8 +58,8 @@ func NewBuilder( } } -// Build new [HeadOnDisk] - [head.Head] with [shard.Shard] with [wal.Wal] which is written to disk. -func (b *Builder) Build(generation uint64, numberOfShards uint16) (*HeadOnDisk, error) { +// Build new [Head] - [head.Head] with [shard.Shard] with [wal.Wal] which is written to disk. +func (b *Builder) Build(generation uint64, numberOfShards uint16) (*Head, error) { headRecord, err := b.catalog.Create(numberOfShards) if err != nil { return nil, err @@ -76,7 +76,7 @@ func (b *Builder) Build(generation uint64, numberOfShards uint16) (*HeadOnDisk, } }() - shards := make([]*ShardOnDisk, numberOfShards) + shards := make([]*shard.Shard, numberOfShards) swn := writer.NewSegmentWriteNotifier(numberOfShards, headRecord.SetLastAppendedSegmentID) for shardID := range numberOfShards { s, err := b.createShardOnDisk(headDir, swn, shardID) @@ -92,7 +92,7 @@ func (b *Builder) Build(generation uint64, numberOfShards uint16) (*HeadOnDisk, return head.NewHead( headRecord.ID(), shards, - shard.NewPerGoroutineShard[*WalOnDisk], + shard.NewPerGoroutineShard[*Wal], headRecord.Acquire(), generation, b.registerer, @@ -106,7 +106,7 @@ func (b *Builder) createShardOnDisk( headDir string, swn *writer.SegmentWriteNotifier, shardID uint16, -) (*ShardOnDisk, error) { +) (*shard.Shard, error) { headDir = filepath.Clean(headDir) shardFile, err := os.OpenFile( //nolint:gosec // need this permissions GetShardWalFilename(headDir, shardID), diff --git a/pp/go/storage/head/services/commiter_test.go b/pp/go/storage/head/services/commiter_test.go index 65bb5eb264..0611014040 100644 --- a/pp/go/storage/head/services/commiter_test.go +++ b/pp/go/storage/head/services/commiter_test.go @@ -23,7 +23,7 @@ type CommitterSuite struct { dataDir string log *catalog.FileLog catalog *catalog.Catalog - activeHeadContainer *container.Weighted[storage.HeadOnDisk, *storage.HeadOnDisk] + activeHeadContainer *container.Weighted[storage.Head, *storage.Head] } func TestCommitterSuite(t *testing.T) { @@ -79,7 +79,7 @@ func (s *CommitterSuite) createCatalog() { s.Require().NoError(err) } -func (s *CommitterSuite) createHead() *storage.HeadOnDisk { +func (s *CommitterSuite) createHead() *storage.Head { h, err := storage.NewBuilder( s.catalog, s.dataDir, @@ -89,9 +89,79 @@ func (s *CommitterSuite) createHead() *storage.HeadOnDisk { ).Build(0, shardsCount) s.Require().NoError(err) + // swn := writer.NewSegmentWriteNotifier(shardsCount, func(uint32) {}) + // for shardID := range shardsCount { + // s, err := b.createShardOnDisk(headDir, swn, shardID) + // if err != nil { + // return nil, err + // } + + // shards[shardID] = s + // } + return h } +// func (s *CommitterSuite) createShardOnMemory( +// headDir string, +// swn *writer.SegmentWriteNotifier, +// shardID uint16, +// ) (*shard.Shard, error) { +// headDir = filepath.Clean(headDir) +// shardFile, err := os.OpenFile( //nolint:gosec // need this permissions +// GetShardWalFilename(headDir, shardID), +// os.O_WRONLY|os.O_CREATE|os.O_APPEND, +// 0o666, //revive:disable-line:add-constant // file permissions simple readable as octa-number +// ) +// if err != nil { +// return nil, fmt.Errorf("failed to create shard wal file id %d: %w", shardID, err) +// } + +// defer func() { +// if err == nil { +// return +// } + +// _ = shardFile.Close() +// }() + +// lss := shard.NewLSS() +// // logShards is 0 for single encoder +// shardWalEncoder := cppbridge.NewHeadWalEncoder(shardID, 0, lss.Target()) + +// _, err = writer.WriteHeader(shardFile, wal.FileFormatVersion, shardWalEncoder.Version()) +// if err != nil { +// return nil, fmt.Errorf("failed to write header: %w", err) +// } + +// sw, err := writer.NewBuffered(shardID, shardFile, writer.WriteSegment[*cppbridge.HeadEncodedSegment], swn) +// if err != nil { +// return nil, fmt.Errorf("failed to create buffered writer shard id %d: %w", shardID, err) +// } + +// var unloadedDataStorage *shard.UnloadedDataStorage +// var queriedSeriesStorage *shard.QueriedSeriesStorage +// if b.unloadDataStorageInterval != 0 { +// unloadedDataStorage = shard.NewUnloadedDataStorage( +// shard.NewFileStorage(GetUnloadedDataStorageFilename(headDir, shardID)), +// ) + +// queriedSeriesStorage = shard.NewQueriedSeriesStorage( +// shard.NewFileStorage(GetQueriedSeriesStorageFilename(headDir, shardID, 0)), +// shard.NewFileStorage(GetQueriedSeriesStorageFilename(headDir, shardID, 1)), +// ) +// } + +// return shard.NewShard( +// lss, +// shard.NewDataStorage(), +// unloadedDataStorage, +// queriedSeriesStorage, +// wal.NewWal(shardWalEncoder, sw, b.maxSegmentSize), +// shardID, +// ), nil +// } + func (s *CommitterSuite) TestCommitter() { trigger := make(chan struct{}, 1) start := make(chan struct{}) diff --git a/pp/go/storage/head/services/persistener_test.go b/pp/go/storage/head/services/persistener_test.go index 02a58a36f1..f988ecf441 100644 --- a/pp/go/storage/head/services/persistener_test.go +++ b/pp/go/storage/head/services/persistener_test.go @@ -18,6 +18,7 @@ import ( "github.com/prometheus/prometheus/pp/go/storage/head/keeper" "github.com/prometheus/prometheus/pp/go/storage/head/services" "github.com/prometheus/prometheus/pp/go/storage/head/services/mock" + "github.com/prometheus/prometheus/pp/go/storage/head/shard" "github.com/prometheus/prometheus/pp/go/storage/head/task" "github.com/prometheus/prometheus/pp/go/storage/storagetest" "github.com/stretchr/testify/suite" @@ -39,7 +40,7 @@ type GenericPersistenceSuite struct { clock *clockwork.FakeClock catalog *catalog.Catalog proxy *storage.Proxy - blockWriter *mock.HeadBlockWriterMock[*storage.ShardOnDisk] + blockWriter *mock.HeadBlockWriterMock[*shard.Shard] writeNotifier *mock.WriteNotifierMock } @@ -51,9 +52,9 @@ func (s *GenericPersistenceSuite) SetupTest() { h := s.mustCreateHead() activeHeadContainer := container.NewWeighted(h) removedHeadNotifier := &mock.WriteNotifierMock{NotifyFunc: func() {}} - hKeeper := keeper.NewKeeper[storage.HeadOnDisk](1, removedHeadNotifier) - s.proxy = storage.NewProxy(activeHeadContainer, hKeeper, func(*storage.HeadOnDisk) error { return nil }) - s.blockWriter = &mock.HeadBlockWriterMock[*storage.ShardOnDisk]{} + hKeeper := keeper.NewKeeper[storage.Head](1, removedHeadNotifier) + s.proxy = storage.NewProxy(activeHeadContainer, hKeeper, func(*storage.Head) error { return nil }) + s.blockWriter = &mock.HeadBlockWriterMock[*shard.Shard]{} s.writeNotifier = &mock.WriteNotifierMock{NotifyFunc: func() {}} } @@ -63,7 +64,7 @@ func (s *GenericPersistenceSuite) createDataDirectory() string { return dataDir } -func (s *GenericPersistenceSuite) createHead() (*storage.HeadOnDisk, error) { +func (s *GenericPersistenceSuite) createHead() (*storage.Head, error) { return storage.NewBuilder( s.catalog, s.dataDir, @@ -73,7 +74,7 @@ func (s *GenericPersistenceSuite) createHead() (*storage.HeadOnDisk, error) { ).Build(0, shardsCount) } -func (s *GenericPersistenceSuite) mustCreateHead() *storage.HeadOnDisk { +func (s *GenericPersistenceSuite) mustCreateHead() *storage.Head { h, err := s.createHead() s.Require().NoError(err) return h @@ -96,11 +97,11 @@ func (s *GenericPersistenceSuite) createCatalog() { type PersistenerSuite struct { GenericPersistenceSuite persistener *services.Persistener[ - *task.Generic[*storage.PerGoroutineShard], - *storage.ShardOnDisk, - *storage.PerGoroutineShard, - *mock.HeadBlockWriterMock[*storage.ShardOnDisk], - *storage.HeadOnDisk, + *task.Generic[*shard.PerGoroutineShard], + *shard.Shard, + *shard.PerGoroutineShard, + *mock.HeadBlockWriterMock[*shard.Shard], + *storage.Head, ] } @@ -108,11 +109,11 @@ func (s *PersistenerSuite) SetupTest() { s.GenericPersistenceSuite.SetupTest() s.persistener = services.NewPersistener[ - *task.Generic[*storage.PerGoroutineShard], - *storage.ShardOnDisk, - *storage.PerGoroutineShard, - *mock.HeadBlockWriterMock[*storage.ShardOnDisk], - *storage.HeadOnDisk, + *task.Generic[*shard.PerGoroutineShard], + *shard.Shard, + *shard.PerGoroutineShard, + *mock.HeadBlockWriterMock[*shard.Shard], + *storage.Head, ](s.catalog, s.blockWriter, s.writeNotifier, s.clock, tsdbRetentionPeriod, retentionPeriod, nil) } @@ -127,19 +128,19 @@ func (s *PersistenerSuite) TestNoHeads() { outdated := s.persistener.Persist(nil) // Assert - s.Equal([]*storage.HeadOnDisk(nil), outdated) + s.Equal([]*storage.Head(nil), outdated) s.Empty(s.blockWriter.WriteCalls()) } func (s *PersistenerSuite) TestNoPersistWritableHead() { // Arrange - heads := []*storage.HeadOnDisk{s.mustCreateHead()} + heads := []*storage.Head{s.mustCreateHead()} // Act outdated := s.persistener.Persist(heads) // Assert - s.Equal([]*storage.HeadOnDisk(nil), outdated) + s.Equal([]*storage.Head(nil), outdated) s.Empty(s.blockWriter.WriteCalls()) } @@ -161,10 +162,10 @@ func (s *PersistenerSuite) TestNoPersistPersistedHead() { s.clock.Advance(retentionPeriod - 1) // Act - outdated := s.persistener.Persist([]*storage.HeadOnDisk{head}) + outdated := s.persistener.Persist([]*storage.Head{head}) // Assert - s.Equal([]*storage.HeadOnDisk(nil), outdated) + s.Equal([]*storage.Head(nil), outdated) s.Empty(s.blockWriter.WriteCalls()) } @@ -186,10 +187,10 @@ func (s *PersistenerSuite) TestOutdatedPersistedHead() { s.clock.Advance(retentionPeriod) // Act - outdated := s.persistener.Persist([]*storage.HeadOnDisk{head}) + outdated := s.persistener.Persist([]*storage.Head{head}) // Assert - s.Equal([]*storage.HeadOnDisk{head}, outdated) + s.Equal([]*storage.Head{head}, outdated) s.Empty(s.blockWriter.WriteCalls()) } @@ -209,23 +210,23 @@ func (s *PersistenerSuite) TestOutdatedHead() { head.SetReadOnly() // Act - outdated := s.persistener.Persist([]*storage.HeadOnDisk{head}) + outdated := s.persistener.Persist([]*storage.Head{head}) // Assert - s.Equal([]*storage.HeadOnDisk{head}, outdated) + s.Equal([]*storage.Head{head}, outdated) s.Empty(s.blockWriter.WriteCalls()) } func (s *PersistenerSuite) TestPersistHeadSuccess() { // Arrange s.clock.Advance(tsdbRetentionPeriod) - blockWriter := block.NewWriter[*storage.ShardOnDisk]( + blockWriter := block.NewWriter[*shard.Shard]( s.dataDir, block.DefaultChunkSegmentSize, 2*time.Hour, prometheus.DefaultRegisterer, ) - s.blockWriter.WriteFunc = func(shard *storage.ShardOnDisk) ([]block.WrittenBlock, error) { + s.blockWriter.WriteFunc = func(shard *shard.Shard) ([]block.WrittenBlock, error) { return blockWriter.Write(shard) } @@ -241,11 +242,11 @@ func (s *PersistenerSuite) TestPersistHeadSuccess() { head.SetReadOnly() // Act - outdated := s.persistener.Persist([]*storage.HeadOnDisk{head}) + outdated := s.persistener.Persist([]*storage.Head{head}) record, err := s.catalog.Get(head.ID()) // Assert - s.Equal([]*storage.HeadOnDisk(nil), outdated) + s.Equal([]*storage.Head(nil), outdated) s.Len(s.blockWriter.WriteCalls(), 2) s.Len(s.writeNotifier.NotifyCalls(), 1) s.Require().NoError(err) @@ -255,13 +256,13 @@ func (s *PersistenerSuite) TestPersistHeadSuccess() { func (s *PersistenerSuite) TestPersistHeadErrorOnBlockWriterForSecondShard() { // Arrange s.clock.Advance(tsdbRetentionPeriod) - blockWriter := block.NewWriter[*storage.ShardOnDisk]( + blockWriter := block.NewWriter[*shard.Shard]( s.dataDir, block.DefaultChunkSegmentSize, 2*time.Hour, prometheus.DefaultRegisterer, ) - s.blockWriter.WriteFunc = func(shard *storage.ShardOnDisk) ([]block.WrittenBlock, error) { + s.blockWriter.WriteFunc = func(shard *shard.Shard) ([]block.WrittenBlock, error) { if len(s.blockWriter.WriteCalls()) == 2 { return nil, errors.New("some error") } @@ -281,11 +282,11 @@ func (s *PersistenerSuite) TestPersistHeadErrorOnBlockWriterForSecondShard() { head.SetReadOnly() // Act - outdated := s.persistener.Persist([]*storage.HeadOnDisk{head}) + outdated := s.persistener.Persist([]*storage.Head{head}) record, err := s.catalog.Get(head.ID()) // Assert - s.Equal([]*storage.HeadOnDisk(nil), outdated) + s.Equal([]*storage.Head(nil), outdated) s.Len(s.blockWriter.WriteCalls(), 2) s.Empty(s.writeNotifier.NotifyCalls()) s.Require().NoError(err) @@ -296,11 +297,11 @@ type PersistenerServiceSuite struct { GenericPersistenceSuite loader *storage.Loader service *services.PersistenerService[ - *task.Generic[*storage.PerGoroutineShard], - *storage.ShardOnDisk, - *storage.PerGoroutineShard, - *mock.HeadBlockWriterMock[*storage.ShardOnDisk], - *storage.HeadOnDisk, + *task.Generic[*shard.PerGoroutineShard], + *shard.Shard, + *shard.PerGoroutineShard, + *mock.HeadBlockWriterMock[*shard.Shard], + *storage.Head, *storage.Proxy, *storage.Loader, ] @@ -311,11 +312,11 @@ func (s *PersistenerServiceSuite) SetupTest() { s.loader = storage.NewLoader(s.dataDir, maxSegmentSize, prometheus.DefaultRegisterer, unloadDataStorageInterval) s.service = services.NewPersistenerService[ - *task.Generic[*storage.PerGoroutineShard], - *storage.ShardOnDisk, - *storage.PerGoroutineShard, - *mock.HeadBlockWriterMock[*storage.ShardOnDisk], - *storage.HeadOnDisk, + *task.Generic[*shard.PerGoroutineShard], + *shard.Shard, + *shard.PerGoroutineShard, + *mock.HeadBlockWriterMock[*shard.Shard], + *storage.Head, *storage.Proxy, *storage.Loader, ]( diff --git a/pp/go/storage/loader.go b/pp/go/storage/loader.go index 253ed95738..c6031b4448 100644 --- a/pp/go/storage/loader.go +++ b/pp/go/storage/loader.go @@ -24,7 +24,7 @@ import ( "github.com/prometheus/prometheus/pp/go/util/optional" ) -// Loader loads [HeadOnDisk] or [ShardOnDisk] from [WalOnDisk]. +// Loader loads [Head] or [shard.Shard] from [Wal]. type Loader struct { dataDir string maxSegmentSize uint32 @@ -47,7 +47,7 @@ func NewLoader( } } -// Load [HeadOnDisk] from [WalOnDisk] by head ID. +// Load [Head] from [Wal] by head ID. // //revive:disable-next-line:cognitive-complexity // function is not complicated //revive:disable-next-line:function-length // long but readable. @@ -55,7 +55,7 @@ func NewLoader( func (l *Loader) Load( headRecord *catalog.Record, generation uint64, -) (_ *HeadOnDisk, corrupted bool) { +) (_ *Head, corrupted bool) { headID := headRecord.ID() headDir := filepath.Join(l.dataDir, headID) numberOfShards := headRecord.NumberOfShards() @@ -78,7 +78,7 @@ func (l *Loader) Load( } wg.Wait() - shards := make([]*ShardOnDisk, numberOfShards) + shards := make([]*shard.Shard, numberOfShards) numberOfSegmentsRead := optional.Optional[uint32]{} for shardID, res := range shardLoadResults { shards[shardID] = res.shard @@ -126,7 +126,7 @@ func (l *Loader) Load( h := head.NewHead( headID, shards, - shard.NewPerGoroutineShard[*WalOnDisk], + shard.NewPerGoroutineShard[*Wal], headRecord.Acquire(), generation, l.registerer, @@ -166,7 +166,7 @@ func (*Loader) loadShard( // ShardLoadResult the result of loading a shard from a wal file. type ShardLoadResult struct { - shard *ShardOnDisk + shard *shard.Shard numberOfSegments uint32 corrupted bool } @@ -175,7 +175,7 @@ type ShardLoadResult struct { type ShardData struct { lss *shard.LSS dataStorage *shard.DataStorage - wal *WalOnDisk + wal *Wal unloadedDataStorage *shard.UnloadedDataStorage queriedSeriesStorage *shard.QueriedSeriesStorage numberOfSegments uint32 diff --git a/pp/go/storage/loader_test.go b/pp/go/storage/loader_test.go index 13ca3c176a..244f7689af 100644 --- a/pp/go/storage/loader_test.go +++ b/pp/go/storage/loader_test.go @@ -14,6 +14,7 @@ import ( "github.com/prometheus/prometheus/pp/go/storage" "github.com/prometheus/prometheus/pp/go/storage/catalog" "github.com/prometheus/prometheus/pp/go/storage/head/services" + "github.com/prometheus/prometheus/pp/go/storage/head/shard" "github.com/prometheus/prometheus/pp/go/storage/storagetest" "github.com/stretchr/testify/suite" ) @@ -88,7 +89,7 @@ func (s *HeadLoadSuite) headDir() string { return filepath.Join(s.dataDir, s.headIdGenerator.last()) } -func (s *HeadLoadSuite) createHead(unloadDataStorageInterval time.Duration) (*storage.HeadOnDisk, error) { +func (s *HeadLoadSuite) createHead(unloadDataStorageInterval time.Duration) (*storage.Head, error) { return storage.NewBuilder( s.catalog, s.dataDir, @@ -98,7 +99,7 @@ func (s *HeadLoadSuite) createHead(unloadDataStorageInterval time.Duration) (*st ).Build(0, numberOfShards) } -func (s *HeadLoadSuite) mustCreateHead(unloadDataStorageInterval time.Duration) *storage.HeadOnDisk { +func (s *HeadLoadSuite) mustCreateHead(unloadDataStorageInterval time.Duration) *storage.Head { h, err := s.createHead(unloadDataStorageInterval) s.Require().NoError(err) @@ -107,14 +108,14 @@ func (s *HeadLoadSuite) mustCreateHead(unloadDataStorageInterval time.Duration) return h } -func (s *HeadLoadSuite) loadHead(unloadDataStorageInterval time.Duration) (*storage.HeadOnDisk, bool) { +func (s *HeadLoadSuite) loadHead(unloadDataStorageInterval time.Duration) (*storage.Head, bool) { record, err := s.catalog.Get(s.headIdGenerator.last()) s.Require().NoError(err) return storage.NewLoader(s.dataDir, maxSegmentSize, prometheus.DefaultRegisterer, unloadDataStorageInterval).Load(record, 0) } -func (s *HeadLoadSuite) mustLoadHead(unloadDataStorageInterval time.Duration) *storage.HeadOnDisk { +func (s *HeadLoadSuite) mustLoadHead(unloadDataStorageInterval time.Duration) *storage.Head { loadedHead, corrupted := s.loadHead(unloadDataStorageInterval) s.False(corrupted) @@ -126,11 +127,11 @@ func (s *HeadLoadSuite) lockFileForCreation(fileName string) { s.Require().NoError(os.Mkdir(fileName, os.ModeDir)) } -func (s *HeadLoadSuite) appendTimeSeries(head *storage.HeadOnDisk, timeSeries []storagetest.TimeSeries) { +func (s *HeadLoadSuite) appendTimeSeries(head *storage.Head, timeSeries []storagetest.TimeSeries) { storagetest.MustAppendTimeSeries(&s.Suite, head, timeSeries) } -func (*HeadLoadSuite) shards(head *storage.HeadOnDisk) (result []*storage.ShardOnDisk) { +func (*HeadLoadSuite) shards(head *storage.Head) (result []*shard.Shard) { for shard := range head.RangeShards() { result = append(result, shard) } diff --git a/pp/go/storage/manager.go b/pp/go/storage/manager.go index 5090c00557..956dde3cbb 100644 --- a/pp/go/storage/manager.go +++ b/pp/go/storage/manager.go @@ -183,7 +183,7 @@ func NewManager( return nil, errors.Join(fmt.Errorf("failed to set active status: %w", err), h.Close()) } - hKeeper := keeper.NewKeeper[HeadOnDisk]( + hKeeper := keeper.NewKeeper[Head]( o.KeeperCapacity, removedHeadNotifier, ) @@ -287,7 +287,7 @@ func (m *Manager) initServices( m.proxy, loader, hcatalog, - block.NewWriter[*ShardOnDisk]( + block.NewWriter[*shard.Shard]( o.DataDir, block.DefaultChunkSegmentSize, o.BlockDuration, @@ -318,7 +318,7 @@ func (m *Manager) initServices( m.rotatorMediator, m.cfg, &headInformer{catalog: hcatalog}, - head.CopyAddedSeries[*ShardOnDisk, *PerGoroutineShard](shard.CopyAddedSeries), + head.CopyAddedSeries[*shard.Shard, *shard.PerGoroutineShard](shard.CopyAddedSeries), persistenerMediator.TriggerWithResetTimer, r, ).Execute(rotatorCtx) @@ -486,7 +486,7 @@ func uploadOrBuildHead( loader *Loader, blockDuration time.Duration, numberOfShards uint16, -) (*HeadOnDisk, error) { +) (*Head, error) { headRecords := hcatalog.List( func(record *catalog.Record) bool { statusIsAppropriate := record.Status() == catalog.StatusNew || diff --git a/pp/go/storage/proxy.go b/pp/go/storage/proxy.go index 6e8a23f063..5e265b4fc0 100644 --- a/pp/go/storage/proxy.go +++ b/pp/go/storage/proxy.go @@ -15,16 +15,16 @@ import ( // Proxy it proxies requests to the active [Head] and the keeper of old [Head]s. type Proxy struct { - activeHeadContainer *container.Weighted[HeadOnDisk, *HeadOnDisk] - keeper *keeper.Keeper[HeadOnDisk, *HeadOnDisk] - onClose func(h *HeadOnDisk) error + activeHeadContainer *container.Weighted[Head, *Head] + keeper *keeper.Keeper[Head, *Head] + onClose func(h *Head) error } // NewProxy init new [Proxy]. func NewProxy( - activeHeadContainer *container.Weighted[HeadOnDisk, *HeadOnDisk], - hKeeper *keeper.Keeper[HeadOnDisk, *HeadOnDisk], - onClose func(h *HeadOnDisk) error, + activeHeadContainer *container.Weighted[Head, *Head], + hKeeper *keeper.Keeper[Head, *Head], + onClose func(h *Head) error, ) *Proxy { return &Proxy{ activeHeadContainer: activeHeadContainer, @@ -34,12 +34,12 @@ func NewProxy( } // Add the [Head] to the [Keeper] if there is a free slot. -func (p *Proxy) Add(head *HeadOnDisk, createdAt time.Duration) error { +func (p *Proxy) Add(head *Head, createdAt time.Duration) error { return p.keeper.Add(head, createdAt) } // AddWithReplace the [Head] to the [Keeper] with replace if the createdAt is earlier. -func (p *Proxy) AddWithReplace(head *HeadOnDisk, createdAt time.Duration) error { +func (p *Proxy) AddWithReplace(head *Head, createdAt time.Duration) error { return p.keeper.AddWithReplace(head, createdAt) } @@ -57,7 +57,7 @@ func (p *Proxy) Close() error { } // Get the active [Head]. -func (p *Proxy) Get() *HeadOnDisk { +func (p *Proxy) Get() *Head { return p.activeHeadContainer.Get() } @@ -67,21 +67,21 @@ func (p *Proxy) HasSlot() bool { } // Heads returns a slice of the [Head]s stored in the [Keeper]. -func (p *Proxy) Heads() []*HeadOnDisk { +func (p *Proxy) Heads() []*Head { return p.keeper.Heads() } // Remove removes [Head]s from the [Keeper]. -func (p *Proxy) Remove(headsForRemove []*HeadOnDisk) { +func (p *Proxy) Remove(headsForRemove []*Head) { p.keeper.Remove(headsForRemove) } // Replace the active [Head] with a new [Head]. -func (p *Proxy) Replace(ctx context.Context, newHead *HeadOnDisk) error { +func (p *Proxy) Replace(ctx context.Context, newHead *Head) error { return p.activeHeadContainer.Replace(ctx, newHead) } // With calls fn(h Head) on active [Head]. -func (p *Proxy) With(ctx context.Context, fn func(h *HeadOnDisk) error) error { +func (p *Proxy) With(ctx context.Context, fn func(h *Head) error) error { return p.activeHeadContainer.With(ctx, fn) } diff --git a/pp/go/storage/storagetest/fixtures.go b/pp/go/storage/storagetest/fixtures.go index db0fc2f7cf..a91f319cee 100644 --- a/pp/go/storage/storagetest/fixtures.go +++ b/pp/go/storage/storagetest/fixtures.go @@ -58,7 +58,7 @@ func (tsd *timeSeriesDataSlice) Destroy() { } // MustAppendTimeSeries add time series to head. -func MustAppendTimeSeries(s *suite.Suite, head *storage.HeadOnDisk, timeSeries []TimeSeries) { +func MustAppendTimeSeries(s *suite.Suite, head *storage.Head, timeSeries []TimeSeries) { headAppender := appender.New(head, services.CFViaRange) statelessRelabeler, err := cppbridge.NewStatelessRelabeler([]*cppbridge.RelabelConfig{}) diff --git a/pp/go/storage/types.go b/pp/go/storage/types.go index 67328814b4..b22ed24652 100644 --- a/pp/go/storage/types.go +++ b/pp/go/storage/types.go @@ -8,17 +8,8 @@ import ( "github.com/prometheus/prometheus/pp/go/storage/head/shard/wal/writer" ) -// WalOnDisk wal on disk. -type WalOnDisk = wal.Wal[ - *cppbridge.HeadEncodedSegment, - *writer.Buffered[*cppbridge.HeadEncodedSegment], -] +// Wal alias for [wal.Wal] based on [cppbridge.HeadEncodedSegment] and [writer.Buffered]. +type Wal = wal.Wal[*cppbridge.HeadEncodedSegment, *writer.Buffered[*cppbridge.HeadEncodedSegment]] -// ShardOnDisk [shard.Shard]. -type ShardOnDisk = shard.Shard - -// PerGoroutineShard [shard.PerGoroutineShard]. -type PerGoroutineShard = shard.PerGoroutineShard - -// HeadOnDisk [head.Head] with [ShardOnDisk]. -type HeadOnDisk = head.Head[*ShardOnDisk, *PerGoroutineShard] +// Head alias for [head.Head] with [shard.Shard] and [shard.PerGoroutineShard]. +type Head = head.Head[*shard.Shard, *shard.PerGoroutineShard] From a40214a10d8c852e764b1d4306de731e4cdfada5 Mon Sep 17 00:00:00 2001 From: Alexandr Yudin Date: Wed, 8 Oct 2025 15:50:05 +0000 Subject: [PATCH 80/96] add commiter test --- pp/go/storage/head/services/commiter_test.go | 230 ++++++++---------- pp/go/storage/head/services/merger_test.go | 111 +++++++++ pp/go/storage/head/services/mock/mock.go | 12 + .../head/services/mock/segment_writer.go | 219 +++++++++++++++++ 4 files changed, 443 insertions(+), 129 deletions(-) create mode 100644 pp/go/storage/head/services/merger_test.go create mode 100644 pp/go/storage/head/services/mock/mock.go create mode 100644 pp/go/storage/head/services/mock/segment_writer.go diff --git a/pp/go/storage/head/services/commiter_test.go b/pp/go/storage/head/services/commiter_test.go index 0611014040..68b0527b90 100644 --- a/pp/go/storage/head/services/commiter_test.go +++ b/pp/go/storage/head/services/commiter_test.go @@ -2,27 +2,25 @@ package services_test import ( "context" - "os" - "path/filepath" "testing" - "github.com/jonboulle/clockwork" "github.com/stretchr/testify/suite" + "github.com/prometheus/prometheus/pp/go/cppbridge" "github.com/prometheus/prometheus/pp/go/storage" - "github.com/prometheus/prometheus/pp/go/storage/catalog" "github.com/prometheus/prometheus/pp/go/storage/head/container" + "github.com/prometheus/prometheus/pp/go/storage/head/head" "github.com/prometheus/prometheus/pp/go/storage/head/services" "github.com/prometheus/prometheus/pp/go/storage/head/services/mock" + "github.com/prometheus/prometheus/pp/go/storage/head/shard" + "github.com/prometheus/prometheus/pp/go/storage/head/shard/wal" ) type CommitterSuite struct { suite.Suite baseCtx context.Context - dataDir string - log *catalog.FileLog - catalog *catalog.Catalog + segmentWriter *mock.SegmentWriterMock activeHeadContainer *container.Weighted[storage.Head, *storage.Head] } @@ -35,134 +33,97 @@ func (s *CommitterSuite) SetupSuite() { } func (s *CommitterSuite) SetupTest() { - s.createDataDirectory() - s.createCatalog() s.activeHeadContainer = container.NewWeighted(s.createHead()) } -func (s *CommitterSuite) TearDownTest() { - s.dataDir = "" - - if s.log != nil { - s.NoError(s.log.Close()) - s.log = nil - } - - if s.catalog != nil { - s.catalog = nil +func (s *CommitterSuite) createHead() *storage.Head { + shards := make([]*shard.Shard, shardsCount) + for shardID := range shardsCount { + shards[shardID] = s.createShardOnMemory(maxSegmentSize, uint16(shardID)) } - if s.activeHeadContainer != nil { - s.NoError(s.activeHeadContainer.Close()) - s.activeHeadContainer = nil - } + return head.NewHead( + "test-head-id", + shards, + shard.NewPerGoroutineShard[*storage.Wal], + nil, + 0, + nil, + ) } -func (s *CommitterSuite) createDataDirectory() { - dataDir := filepath.Join(s.T().TempDir(), "data") - s.Require().NoError(os.MkdirAll(dataDir, os.ModeDir)) - s.dataDir = dataDir -} +func (s *CommitterSuite) createShardOnMemory(maxSegmentSize uint32, shardID uint16) *shard.Shard { + s.segmentWriter = &mock.SegmentWriterMock{ + WriteFunc: func(*cppbridge.HeadEncodedSegment) error { return nil }, + FlushFunc: func() error { return nil }, + SyncFunc: func() error { return nil }, + CloseFunc: func() error { return nil }, + CurrentSizeFunc: func() int64 { return 0 }, + } -func (s *CommitterSuite) createCatalog() { - l, err := catalog.NewFileLogV2(filepath.Join(s.dataDir, "head.log")) - s.Require().NoError(err) - clock := clockwork.NewRealClock() + lss := shard.NewLSS() + // logShards is 0 for single encoder + shardWalEncoder := cppbridge.NewHeadWalEncoder(shardID, 0, lss.Target()) - s.catalog, err = catalog.New( - clock, - l, - &catalog.DefaultIDGenerator{}, - catalog.DefaultMaxLogFileSize, + return shard.NewShard( + lss, + shard.NewDataStorage(), nil, + nil, + wal.NewWal(shardWalEncoder, s.segmentWriter, maxSegmentSize), + shardID, ) - s.Require().NoError(err) } -func (s *CommitterSuite) createHead() *storage.Head { - h, err := storage.NewBuilder( - s.catalog, - s.dataDir, - maxSegmentSize, - nil, - unloadDataStorageInterval, - ).Build(0, shardsCount) - s.Require().NoError(err) - - // swn := writer.NewSegmentWriteNotifier(shardsCount, func(uint32) {}) - // for shardID := range shardsCount { - // s, err := b.createShardOnDisk(headDir, swn, shardID) - // if err != nil { - // return nil, err - // } +func (s *CommitterSuite) TestHappyPath() { + trigger := make(chan struct{}, 1) + start := make(chan struct{}) + mediator := &mock.MediatorMock{ + CFunc: func() <-chan struct{} { + close(start) + return trigger + }, + } + isNewHead := func(string) bool { return false } + committer := services.NewCommitter(s.activeHeadContainer, mediator, isNewHead) + done := make(chan struct{}) - // shards[shardID] = s - // } + s.T().Run("execute", func(t *testing.T) { + t.Parallel() - return h + err := committer.Execute(s.baseCtx) + close(done) + s.NoError(err) + }) + + s.T().Run("tick", func(t *testing.T) { + t.Parallel() + + <-start + trigger <- struct{}{} + trigger <- struct{}{} + close(trigger) + <-done + + s.Require().NoError(s.activeHeadContainer.Close()) + + if !s.Len(s.segmentWriter.WriteCalls(), 2) { + return + } + if !s.Len(s.segmentWriter.FlushCalls(), 2) { + return + } + if !s.Len(s.segmentWriter.SyncCalls(), 2) { + return + } + + for _, call := range s.segmentWriter.WriteCalls() { + s.Equal(uint32(0), call.Segment.Samples()) + } + }) } -// func (s *CommitterSuite) createShardOnMemory( -// headDir string, -// swn *writer.SegmentWriteNotifier, -// shardID uint16, -// ) (*shard.Shard, error) { -// headDir = filepath.Clean(headDir) -// shardFile, err := os.OpenFile( //nolint:gosec // need this permissions -// GetShardWalFilename(headDir, shardID), -// os.O_WRONLY|os.O_CREATE|os.O_APPEND, -// 0o666, //revive:disable-line:add-constant // file permissions simple readable as octa-number -// ) -// if err != nil { -// return nil, fmt.Errorf("failed to create shard wal file id %d: %w", shardID, err) -// } - -// defer func() { -// if err == nil { -// return -// } - -// _ = shardFile.Close() -// }() - -// lss := shard.NewLSS() -// // logShards is 0 for single encoder -// shardWalEncoder := cppbridge.NewHeadWalEncoder(shardID, 0, lss.Target()) - -// _, err = writer.WriteHeader(shardFile, wal.FileFormatVersion, shardWalEncoder.Version()) -// if err != nil { -// return nil, fmt.Errorf("failed to write header: %w", err) -// } - -// sw, err := writer.NewBuffered(shardID, shardFile, writer.WriteSegment[*cppbridge.HeadEncodedSegment], swn) -// if err != nil { -// return nil, fmt.Errorf("failed to create buffered writer shard id %d: %w", shardID, err) -// } - -// var unloadedDataStorage *shard.UnloadedDataStorage -// var queriedSeriesStorage *shard.QueriedSeriesStorage -// if b.unloadDataStorageInterval != 0 { -// unloadedDataStorage = shard.NewUnloadedDataStorage( -// shard.NewFileStorage(GetUnloadedDataStorageFilename(headDir, shardID)), -// ) - -// queriedSeriesStorage = shard.NewQueriedSeriesStorage( -// shard.NewFileStorage(GetQueriedSeriesStorageFilename(headDir, shardID, 0)), -// shard.NewFileStorage(GetQueriedSeriesStorageFilename(headDir, shardID, 1)), -// ) -// } - -// return shard.NewShard( -// lss, -// shard.NewDataStorage(), -// unloadedDataStorage, -// queriedSeriesStorage, -// wal.NewWal(shardWalEncoder, sw, b.maxSegmentSize), -// shardID, -// ), nil -// } - -func (s *CommitterSuite) TestCommitter() { +func (s *CommitterSuite) TestSkipNewHead() { trigger := make(chan struct{}, 1) start := make(chan struct{}) mediator := &mock.MediatorMock{ @@ -171,19 +132,30 @@ func (s *CommitterSuite) TestCommitter() { return trigger }, } - isNewHead := func(string) bool { - return false - } + isNewHead := func(string) bool { return true } committer := services.NewCommitter(s.activeHeadContainer, mediator, isNewHead) - done := make(chan struct{}) - go func() { - s.NoError(committer.Execute(s.baseCtx)) + + s.T().Run("execute", func(t *testing.T) { + t.Parallel() + + err := committer.Execute(s.baseCtx) close(done) - }() + s.Require().NoError(err) + }) + + s.T().Run("tick", func(t *testing.T) { + t.Parallel() + + <-start + trigger <- struct{}{} + close(trigger) + <-done + + s.Empty(s.segmentWriter.WriteCalls()) + s.Empty(s.segmentWriter.FlushCalls()) + s.Empty(s.segmentWriter.SyncCalls()) - <-start - trigger <- struct{}{} - close(trigger) - <-done + s.Require().NoError(s.activeHeadContainer.Close()) + }) } diff --git a/pp/go/storage/head/services/merger_test.go b/pp/go/storage/head/services/merger_test.go new file mode 100644 index 0000000000..bcd5eb7e1c --- /dev/null +++ b/pp/go/storage/head/services/merger_test.go @@ -0,0 +1,111 @@ +package services_test + +import ( + "context" + "testing" + + "github.com/prometheus/prometheus/pp/go/cppbridge" + "github.com/prometheus/prometheus/pp/go/storage" + "github.com/prometheus/prometheus/pp/go/storage/head/container" + "github.com/prometheus/prometheus/pp/go/storage/head/head" + "github.com/prometheus/prometheus/pp/go/storage/head/services" + "github.com/prometheus/prometheus/pp/go/storage/head/services/mock" + "github.com/prometheus/prometheus/pp/go/storage/head/shard" + "github.com/prometheus/prometheus/pp/go/storage/head/shard/wal" + "github.com/stretchr/testify/suite" +) + +type MergerSuite struct { + suite.Suite + + baseCtx context.Context + segmentWriter *mock.SegmentWriterMock + activeHeadContainer *container.Weighted[storage.Head, *storage.Head] +} + +func TestMergerSuite(t *testing.T) { + suite.Run(t, new(MergerSuite)) +} + +func (s *MergerSuite) SetupSuite() { + s.baseCtx = context.Background() +} + +func (s *MergerSuite) SetupTest() { + s.activeHeadContainer = container.NewWeighted(s.createHead()) +} + +func (s *MergerSuite) createHead() *storage.Head { + shards := make([]*shard.Shard, shardsCount) + for shardID := range shardsCount { + shards[shardID] = s.createShardOnMemory(maxSegmentSize, uint16(shardID)) + } + + return head.NewHead( + "test-head-id", + shards, + shard.NewPerGoroutineShard[*storage.Wal], + nil, + 0, + nil, + ) +} + +func (s *MergerSuite) createShardOnMemory(maxSegmentSize uint32, shardID uint16) *shard.Shard { + s.segmentWriter = &mock.SegmentWriterMock{ + WriteFunc: func(*cppbridge.HeadEncodedSegment) error { return nil }, + FlushFunc: func() error { return nil }, + SyncFunc: func() error { return nil }, + CloseFunc: func() error { return nil }, + CurrentSizeFunc: func() int64 { return 0 }, + } + + lss := shard.NewLSS() + shardWalEncoder := cppbridge.NewHeadWalEncoder(shardID, 0, lss.Target()) + + return shard.NewShard( + lss, + shard.NewDataStorage(), + nil, + nil, + wal.NewWal(shardWalEncoder, s.segmentWriter, maxSegmentSize), + shardID, + ) +} + +func (s *MergerSuite) TestHappyPath() { + trigger := make(chan struct{}, 1) + start := make(chan struct{}) + mediator := &mock.MediatorMock{ + CFunc: func() <-chan struct{} { + close(start) + return trigger + }, + } + isNewHead := func(string) bool { return false } + merger := services.NewMerger(s.activeHeadContainer, mediator, isNewHead) + done := make(chan struct{}) + + s.T().Run("execute", func(t *testing.T) { + t.Parallel() + + err := merger.Execute(s.baseCtx) + close(done) + s.NoError(err) + }) + + s.T().Run("tick", func(t *testing.T) { + t.Parallel() + + <-start + trigger <- struct{}{} + close(trigger) + <-done + + s.Require().NoError(s.activeHeadContainer.Close()) + }) +} + +func (s *MergerSuite) TestSkipNewHead() { + s.T().Log("TestSkipNewHead") +} diff --git a/pp/go/storage/head/services/mock/mock.go b/pp/go/storage/head/services/mock/mock.go new file mode 100644 index 0000000000..e18cf9f9f1 --- /dev/null +++ b/pp/go/storage/head/services/mock/mock.go @@ -0,0 +1,12 @@ +package mock + +import ( + "github.com/prometheus/prometheus/pp/go/cppbridge" + "github.com/prometheus/prometheus/pp/go/storage/head/shard/wal" +) + +//go:generate -command moq go run github.com/matryer/moq --rm --skip-ensure --pkg mock --out +//go:generate moq segment_writer.go . SegmentWriter + +// SegmentWriter alias for [wal.SegmentWriter] with [cppbridge.HeadEncodedSegment]. +type SegmentWriter = wal.SegmentWriter[*cppbridge.HeadEncodedSegment] diff --git a/pp/go/storage/head/services/mock/segment_writer.go b/pp/go/storage/head/services/mock/segment_writer.go new file mode 100644 index 0000000000..7fa95a1871 --- /dev/null +++ b/pp/go/storage/head/services/mock/segment_writer.go @@ -0,0 +1,219 @@ +// Code generated by moq; DO NOT EDIT. +// github.com/matryer/moq + +package mock + +import ( + "github.com/prometheus/prometheus/pp/go/cppbridge" + "sync" +) + +// SegmentWriterMock is a mock implementation of SegmentWriter. +// +// func TestSomethingThatUsesSegmentWriter(t *testing.T) { +// +// // make and configure a mocked SegmentWriter +// mockedSegmentWriter := &SegmentWriterMock{ +// CloseFunc: func() error { +// panic("mock out the Close method") +// }, +// CurrentSizeFunc: func() int64 { +// panic("mock out the CurrentSize method") +// }, +// FlushFunc: func() error { +// panic("mock out the Flush method") +// }, +// SyncFunc: func() error { +// panic("mock out the Sync method") +// }, +// WriteFunc: func(segment *cppbridge.HeadEncodedSegment) error { +// panic("mock out the Write method") +// }, +// } +// +// // use mockedSegmentWriter in code that requires SegmentWriter +// // and then make assertions. +// +// } +type SegmentWriterMock struct { + // CloseFunc mocks the Close method. + CloseFunc func() error + + // CurrentSizeFunc mocks the CurrentSize method. + CurrentSizeFunc func() int64 + + // FlushFunc mocks the Flush method. + FlushFunc func() error + + // SyncFunc mocks the Sync method. + SyncFunc func() error + + // WriteFunc mocks the Write method. + WriteFunc func(segment *cppbridge.HeadEncodedSegment) error + + // calls tracks calls to the methods. + calls struct { + // Close holds details about calls to the Close method. + Close []struct { + } + // CurrentSize holds details about calls to the CurrentSize method. + CurrentSize []struct { + } + // Flush holds details about calls to the Flush method. + Flush []struct { + } + // Sync holds details about calls to the Sync method. + Sync []struct { + } + // Write holds details about calls to the Write method. + Write []struct { + // Segment is the segment argument value. + Segment *cppbridge.HeadEncodedSegment + } + } + lockClose sync.RWMutex + lockCurrentSize sync.RWMutex + lockFlush sync.RWMutex + lockSync sync.RWMutex + lockWrite sync.RWMutex +} + +// Close calls CloseFunc. +func (mock *SegmentWriterMock) Close() error { + if mock.CloseFunc == nil { + panic("SegmentWriterMock.CloseFunc: method is nil but SegmentWriter.Close was just called") + } + callInfo := struct { + }{} + mock.lockClose.Lock() + mock.calls.Close = append(mock.calls.Close, callInfo) + mock.lockClose.Unlock() + return mock.CloseFunc() +} + +// CloseCalls gets all the calls that were made to Close. +// Check the length with: +// +// len(mockedSegmentWriter.CloseCalls()) +func (mock *SegmentWriterMock) CloseCalls() []struct { +} { + var calls []struct { + } + mock.lockClose.RLock() + calls = mock.calls.Close + mock.lockClose.RUnlock() + return calls +} + +// CurrentSize calls CurrentSizeFunc. +func (mock *SegmentWriterMock) CurrentSize() int64 { + if mock.CurrentSizeFunc == nil { + panic("SegmentWriterMock.CurrentSizeFunc: method is nil but SegmentWriter.CurrentSize was just called") + } + callInfo := struct { + }{} + mock.lockCurrentSize.Lock() + mock.calls.CurrentSize = append(mock.calls.CurrentSize, callInfo) + mock.lockCurrentSize.Unlock() + return mock.CurrentSizeFunc() +} + +// CurrentSizeCalls gets all the calls that were made to CurrentSize. +// Check the length with: +// +// len(mockedSegmentWriter.CurrentSizeCalls()) +func (mock *SegmentWriterMock) CurrentSizeCalls() []struct { +} { + var calls []struct { + } + mock.lockCurrentSize.RLock() + calls = mock.calls.CurrentSize + mock.lockCurrentSize.RUnlock() + return calls +} + +// Flush calls FlushFunc. +func (mock *SegmentWriterMock) Flush() error { + if mock.FlushFunc == nil { + panic("SegmentWriterMock.FlushFunc: method is nil but SegmentWriter.Flush was just called") + } + callInfo := struct { + }{} + mock.lockFlush.Lock() + mock.calls.Flush = append(mock.calls.Flush, callInfo) + mock.lockFlush.Unlock() + return mock.FlushFunc() +} + +// FlushCalls gets all the calls that were made to Flush. +// Check the length with: +// +// len(mockedSegmentWriter.FlushCalls()) +func (mock *SegmentWriterMock) FlushCalls() []struct { +} { + var calls []struct { + } + mock.lockFlush.RLock() + calls = mock.calls.Flush + mock.lockFlush.RUnlock() + return calls +} + +// Sync calls SyncFunc. +func (mock *SegmentWriterMock) Sync() error { + if mock.SyncFunc == nil { + panic("SegmentWriterMock.SyncFunc: method is nil but SegmentWriter.Sync was just called") + } + callInfo := struct { + }{} + mock.lockSync.Lock() + mock.calls.Sync = append(mock.calls.Sync, callInfo) + mock.lockSync.Unlock() + return mock.SyncFunc() +} + +// SyncCalls gets all the calls that were made to Sync. +// Check the length with: +// +// len(mockedSegmentWriter.SyncCalls()) +func (mock *SegmentWriterMock) SyncCalls() []struct { +} { + var calls []struct { + } + mock.lockSync.RLock() + calls = mock.calls.Sync + mock.lockSync.RUnlock() + return calls +} + +// Write calls WriteFunc. +func (mock *SegmentWriterMock) Write(segment *cppbridge.HeadEncodedSegment) error { + if mock.WriteFunc == nil { + panic("SegmentWriterMock.WriteFunc: method is nil but SegmentWriter.Write was just called") + } + callInfo := struct { + Segment *cppbridge.HeadEncodedSegment + }{ + Segment: segment, + } + mock.lockWrite.Lock() + mock.calls.Write = append(mock.calls.Write, callInfo) + mock.lockWrite.Unlock() + return mock.WriteFunc(segment) +} + +// WriteCalls gets all the calls that were made to Write. +// Check the length with: +// +// len(mockedSegmentWriter.WriteCalls()) +func (mock *SegmentWriterMock) WriteCalls() []struct { + Segment *cppbridge.HeadEncodedSegment +} { + var calls []struct { + Segment *cppbridge.HeadEncodedSegment + } + mock.lockWrite.RLock() + calls = mock.calls.Write + mock.lockWrite.RUnlock() + return calls +} From 3a2c3a151cec220405614f8e514c8a8ba7f667b7 Mon Sep 17 00:00:00 2001 From: Alexandr Yudin Date: Thu, 9 Oct 2025 06:01:25 +0000 Subject: [PATCH 81/96] add merger test --- pp/go/storage/head/services/commiter_test.go | 101 +++-- pp/go/storage/head/services/merger_test.go | 184 +++++++- .../head/services/mock/file_storage.go | 413 ++++++++++++++++++ pp/go/storage/head/services/mock/mock.go | 5 + pp/go/storage/querier/querier_test.go | 40 +- 5 files changed, 663 insertions(+), 80 deletions(-) create mode 100644 pp/go/storage/head/services/mock/file_storage.go diff --git a/pp/go/storage/head/services/commiter_test.go b/pp/go/storage/head/services/commiter_test.go index 68b0527b90..db09b00089 100644 --- a/pp/go/storage/head/services/commiter_test.go +++ b/pp/go/storage/head/services/commiter_test.go @@ -19,9 +19,7 @@ import ( type CommitterSuite struct { suite.Suite - baseCtx context.Context - segmentWriter *mock.SegmentWriterMock - activeHeadContainer *container.Weighted[storage.Head, *storage.Head] + baseCtx context.Context } func TestCommitterSuite(t *testing.T) { @@ -32,14 +30,10 @@ func (s *CommitterSuite) SetupSuite() { s.baseCtx = context.Background() } -func (s *CommitterSuite) SetupTest() { - s.activeHeadContainer = container.NewWeighted(s.createHead()) -} - -func (s *CommitterSuite) createHead() *storage.Head { +func (s *CommitterSuite) createHead(segmentWriters []*mock.SegmentWriterMock) *storage.Head { shards := make([]*shard.Shard, shardsCount) - for shardID := range shardsCount { - shards[shardID] = s.createShardOnMemory(maxSegmentSize, uint16(shardID)) + for shardID, segmentWriter := range segmentWriters { + shards[shardID] = s.createShardOnMemory(segmentWriter, maxSegmentSize, uint16(shardID)) } return head.NewHead( @@ -52,15 +46,11 @@ func (s *CommitterSuite) createHead() *storage.Head { ) } -func (s *CommitterSuite) createShardOnMemory(maxSegmentSize uint32, shardID uint16) *shard.Shard { - s.segmentWriter = &mock.SegmentWriterMock{ - WriteFunc: func(*cppbridge.HeadEncodedSegment) error { return nil }, - FlushFunc: func() error { return nil }, - SyncFunc: func() error { return nil }, - CloseFunc: func() error { return nil }, - CurrentSizeFunc: func() int64 { return 0 }, - } - +func (*CommitterSuite) createShardOnMemory( + segmentWriter *mock.SegmentWriterMock, + maxSegmentSize uint32, + shardID uint16, +) *shard.Shard { lss := shard.NewLSS() // logShards is 0 for single encoder shardWalEncoder := cppbridge.NewHeadWalEncoder(shardID, 0, lss.Target()) @@ -70,7 +60,7 @@ func (s *CommitterSuite) createShardOnMemory(maxSegmentSize uint32, shardID uint shard.NewDataStorage(), nil, nil, - wal.NewWal(shardWalEncoder, s.segmentWriter, maxSegmentSize), + wal.NewWal(shardWalEncoder, segmentWriter, maxSegmentSize), shardID, ) } @@ -84,8 +74,21 @@ func (s *CommitterSuite) TestHappyPath() { return trigger }, } + + segmentWriters := make([]*mock.SegmentWriterMock, shardsCount) + for shardID := range shardsCount { + segmentWriters[shardID] = &mock.SegmentWriterMock{ + WriteFunc: func(*cppbridge.HeadEncodedSegment) error { return nil }, + FlushFunc: func() error { return nil }, + SyncFunc: func() error { return nil }, + CloseFunc: func() error { return nil }, + CurrentSizeFunc: func() int64 { return 0 }, + } + } + activeHeadContainer := container.NewWeighted(s.createHead(segmentWriters)) isNewHead := func(string) bool { return false } - committer := services.NewCommitter(s.activeHeadContainer, mediator, isNewHead) + + committer := services.NewCommitter(activeHeadContainer, mediator, isNewHead) done := make(chan struct{}) s.T().Run("execute", func(t *testing.T) { @@ -105,20 +108,22 @@ func (s *CommitterSuite) TestHappyPath() { close(trigger) <-done - s.Require().NoError(s.activeHeadContainer.Close()) - - if !s.Len(s.segmentWriter.WriteCalls(), 2) { - return - } - if !s.Len(s.segmentWriter.FlushCalls(), 2) { - return - } - if !s.Len(s.segmentWriter.SyncCalls(), 2) { - return - } - - for _, call := range s.segmentWriter.WriteCalls() { - s.Equal(uint32(0), call.Segment.Samples()) + s.Require().NoError(activeHeadContainer.Close()) + + for _, segmentWriter := range segmentWriters { + if !s.Len(segmentWriter.WriteCalls(), 2) { + return + } + if !s.Len(segmentWriter.FlushCalls(), 2) { + return + } + if !s.Len(segmentWriter.SyncCalls(), 2) { + return + } + + for _, call := range segmentWriter.WriteCalls() { + s.Equal(uint32(0), call.Segment.Samples()) + } } }) } @@ -132,8 +137,21 @@ func (s *CommitterSuite) TestSkipNewHead() { return trigger }, } + + segmentWriters := make([]*mock.SegmentWriterMock, shardsCount) + for shardID := range shardsCount { + segmentWriters[shardID] = &mock.SegmentWriterMock{ + WriteFunc: func(*cppbridge.HeadEncodedSegment) error { return nil }, + FlushFunc: func() error { return nil }, + SyncFunc: func() error { return nil }, + CloseFunc: func() error { return nil }, + CurrentSizeFunc: func() int64 { return 0 }, + } + } + activeHeadContainer := container.NewWeighted(s.createHead(segmentWriters)) + isNewHead := func(string) bool { return true } - committer := services.NewCommitter(s.activeHeadContainer, mediator, isNewHead) + committer := services.NewCommitter(activeHeadContainer, mediator, isNewHead) done := make(chan struct{}) s.T().Run("execute", func(t *testing.T) { @@ -149,13 +167,16 @@ func (s *CommitterSuite) TestSkipNewHead() { <-start trigger <- struct{}{} + trigger <- struct{}{} close(trigger) <-done - s.Empty(s.segmentWriter.WriteCalls()) - s.Empty(s.segmentWriter.FlushCalls()) - s.Empty(s.segmentWriter.SyncCalls()) + s.Require().NoError(activeHeadContainer.Close()) - s.Require().NoError(s.activeHeadContainer.Close()) + for _, segmentWriter := range segmentWriters { + s.Empty(segmentWriter.WriteCalls()) + s.Empty(segmentWriter.FlushCalls()) + s.Empty(segmentWriter.SyncCalls()) + } }) } diff --git a/pp/go/storage/head/services/merger_test.go b/pp/go/storage/head/services/merger_test.go index bcd5eb7e1c..46b7c98706 100644 --- a/pp/go/storage/head/services/merger_test.go +++ b/pp/go/storage/head/services/merger_test.go @@ -18,9 +18,7 @@ import ( type MergerSuite struct { suite.Suite - baseCtx context.Context - segmentWriter *mock.SegmentWriterMock - activeHeadContainer *container.Weighted[storage.Head, *storage.Head] + baseCtx context.Context } func TestMergerSuite(t *testing.T) { @@ -31,14 +29,18 @@ func (s *MergerSuite) SetupSuite() { s.baseCtx = context.Background() } -func (s *MergerSuite) SetupTest() { - s.activeHeadContainer = container.NewWeighted(s.createHead()) -} - -func (s *MergerSuite) createHead() *storage.Head { +func (s *MergerSuite) createHead( + unloadedFS []*mock.StorageFileMock, + queriedSeriesFS [][2]*mock.StorageFileMock, +) *storage.Head { shards := make([]*shard.Shard, shardsCount) - for shardID := range shardsCount { - shards[shardID] = s.createShardOnMemory(maxSegmentSize, uint16(shardID)) + for shardID := range unloadedFS { + shards[shardID] = s.createShardOnMemory( + unloadedFS[shardID], + queriedSeriesFS[shardID], + maxSegmentSize, + uint16(shardID), + ) } return head.NewHead( @@ -51,8 +53,16 @@ func (s *MergerSuite) createHead() *storage.Head { ) } -func (s *MergerSuite) createShardOnMemory(maxSegmentSize uint32, shardID uint16) *shard.Shard { - s.segmentWriter = &mock.SegmentWriterMock{ +func (*MergerSuite) createShardOnMemory( + unloadedFS *mock.StorageFileMock, + queriedSeriesFS [2]*mock.StorageFileMock, + maxSegmentSize uint32, + shardID uint16, +) *shard.Shard { + lss := shard.NewLSS() + shardWalEncoder := cppbridge.NewHeadWalEncoder(shardID, 0, lss.Target()) + + segmentWriter := &mock.SegmentWriterMock{ WriteFunc: func(*cppbridge.HeadEncodedSegment) error { return nil }, FlushFunc: func() error { return nil }, SyncFunc: func() error { return nil }, @@ -60,15 +70,18 @@ func (s *MergerSuite) createShardOnMemory(maxSegmentSize uint32, shardID uint16) CurrentSizeFunc: func() int64 { return 0 }, } - lss := shard.NewLSS() - shardWalEncoder := cppbridge.NewHeadWalEncoder(shardID, 0, lss.Target()) + unloadedDataStorage := shard.NewUnloadedDataStorage(unloadedFS) + queriedSeriesStorage := shard.NewQueriedSeriesStorage( + queriedSeriesFS[0], + queriedSeriesFS[1], + ) return shard.NewShard( lss, shard.NewDataStorage(), - nil, - nil, - wal.NewWal(shardWalEncoder, s.segmentWriter, maxSegmentSize), + unloadedDataStorage, + queriedSeriesStorage, + wal.NewWal(shardWalEncoder, segmentWriter, maxSegmentSize), shardID, ) } @@ -82,8 +95,41 @@ func (s *MergerSuite) TestHappyPath() { return trigger }, } + + unloadedFS := make([]*mock.StorageFileMock, shardsCount) + queriedSeriesFS := make([][2]*mock.StorageFileMock, shardsCount) + for shardID := range shardsCount { + unloadedFS[shardID] = &mock.StorageFileMock{ + CloseFunc: func() error { return nil }, + OpenFunc: func(int) error { return nil }, + SyncFunc: func() error { return nil }, + WriteFunc: func([]byte) (int, error) { return 0, nil }, + } + + queriedSeriesFS[shardID] = [2]*mock.StorageFileMock{ + { + CloseFunc: func() error { return nil }, + OpenFunc: func(int) error { return nil }, + SeekFunc: func(int64, int) (int64, error) { return 0, nil }, + SyncFunc: func() error { return nil }, + TruncateFunc: func(int64) error { return nil }, + WriteFunc: func([]byte) (int, error) { return 0, nil }, + }, + { + CloseFunc: func() error { return nil }, + OpenFunc: func(int) error { return nil }, + SeekFunc: func(int64, int) (int64, error) { return 0, nil }, + SyncFunc: func() error { return nil }, + TruncateFunc: func(int64) error { return nil }, + WriteFunc: func([]byte) (int, error) { return 0, nil }, + }, + } + } + + activeHeadContainer := container.NewWeighted(s.createHead(unloadedFS, queriedSeriesFS)) isNewHead := func(string) bool { return false } - merger := services.NewMerger(s.activeHeadContainer, mediator, isNewHead) + + merger := services.NewMerger(activeHeadContainer, mediator, isNewHead) done := make(chan struct{}) s.T().Run("execute", func(t *testing.T) { @@ -102,10 +148,108 @@ func (s *MergerSuite) TestHappyPath() { close(trigger) <-done - s.Require().NoError(s.activeHeadContainer.Close()) + s.Require().NoError(activeHeadContainer.Close()) + + for shardID := range unloadedFS { + s.Len(unloadedFS[shardID].OpenCalls(), 1) + s.Len(unloadedFS[shardID].SyncCalls(), 1) + s.Len(unloadedFS[shardID].WriteCalls(), 2) + + s.Len(queriedSeriesFS[shardID][0].OpenCalls(), 1) + s.Len(queriedSeriesFS[shardID][0].SeekCalls(), 1) + s.Len(queriedSeriesFS[shardID][0].SyncCalls(), 1) + s.Len(queriedSeriesFS[shardID][0].TruncateCalls(), 1) + s.Len(queriedSeriesFS[shardID][0].WriteCalls(), 2) + + s.Empty(queriedSeriesFS[shardID][1].OpenCalls()) + s.Empty(queriedSeriesFS[shardID][1].SeekCalls()) + s.Empty(queriedSeriesFS[shardID][1].SyncCalls()) + s.Empty(queriedSeriesFS[shardID][1].TruncateCalls()) + s.Empty(queriedSeriesFS[shardID][1].WriteCalls()) + } }) } func (s *MergerSuite) TestSkipNewHead() { - s.T().Log("TestSkipNewHead") + trigger := make(chan struct{}, 1) + start := make(chan struct{}) + mediator := &mock.MediatorMock{ + CFunc: func() <-chan struct{} { + close(start) + return trigger + }, + } + + unloadedFS := make([]*mock.StorageFileMock, shardsCount) + queriedSeriesFS := make([][2]*mock.StorageFileMock, shardsCount) + for shardID := range shardsCount { + unloadedFS[shardID] = &mock.StorageFileMock{ + CloseFunc: func() error { return nil }, + OpenFunc: func(int) error { return nil }, + SyncFunc: func() error { return nil }, + WriteFunc: func([]byte) (int, error) { return 0, nil }, + } + + queriedSeriesFS[shardID] = [2]*mock.StorageFileMock{ + { + CloseFunc: func() error { return nil }, + OpenFunc: func(int) error { return nil }, + SeekFunc: func(int64, int) (int64, error) { return 0, nil }, + SyncFunc: func() error { return nil }, + TruncateFunc: func(int64) error { return nil }, + WriteFunc: func([]byte) (int, error) { return 0, nil }, + }, + { + CloseFunc: func() error { return nil }, + OpenFunc: func(int) error { return nil }, + SeekFunc: func(int64, int) (int64, error) { return 0, nil }, + SyncFunc: func() error { return nil }, + TruncateFunc: func(int64) error { return nil }, + WriteFunc: func([]byte) (int, error) { return 0, nil }, + }, + } + } + + activeHeadContainer := container.NewWeighted(s.createHead(unloadedFS, queriedSeriesFS)) + isNewHead := func(string) bool { return true } + + merger := services.NewMerger(activeHeadContainer, mediator, isNewHead) + done := make(chan struct{}) + + s.T().Run("execute", func(t *testing.T) { + t.Parallel() + + err := merger.Execute(s.baseCtx) + close(done) + s.NoError(err) + }) + + s.T().Run("tick", func(t *testing.T) { + t.Parallel() + + <-start + trigger <- struct{}{} + close(trigger) + <-done + + s.Require().NoError(activeHeadContainer.Close()) + + for shardID := range unloadedFS { + s.Empty(unloadedFS[shardID].OpenCalls()) + s.Empty(unloadedFS[shardID].SyncCalls()) + s.Empty(unloadedFS[shardID].WriteCalls()) + + s.Empty(queriedSeriesFS[shardID][0].OpenCalls()) + s.Empty(queriedSeriesFS[shardID][0].SeekCalls()) + s.Empty(queriedSeriesFS[shardID][0].SyncCalls()) + s.Empty(queriedSeriesFS[shardID][0].TruncateCalls()) + s.Empty(queriedSeriesFS[shardID][0].WriteCalls()) + + s.Empty(queriedSeriesFS[shardID][1].OpenCalls()) + s.Empty(queriedSeriesFS[shardID][1].SeekCalls()) + s.Empty(queriedSeriesFS[shardID][1].SyncCalls()) + s.Empty(queriedSeriesFS[shardID][1].TruncateCalls()) + s.Empty(queriedSeriesFS[shardID][1].WriteCalls()) + } + }) } diff --git a/pp/go/storage/head/services/mock/file_storage.go b/pp/go/storage/head/services/mock/file_storage.go new file mode 100644 index 0000000000..ed97aae54e --- /dev/null +++ b/pp/go/storage/head/services/mock/file_storage.go @@ -0,0 +1,413 @@ +// Code generated by moq; DO NOT EDIT. +// github.com/matryer/moq + +package mock + +import ( + "sync" +) + +// StorageFileMock is a mock implementation of StorageFile. +// +// func TestSomethingThatUsesStorageFile(t *testing.T) { +// +// // make and configure a mocked StorageFile +// mockedStorageFile := &StorageFileMock{ +// CloseFunc: func() error { +// panic("mock out the Close method") +// }, +// IsEmptyFunc: func() bool { +// panic("mock out the IsEmpty method") +// }, +// OpenFunc: func(flags int) error { +// panic("mock out the Open method") +// }, +// ReadFunc: func(p []byte) (int, error) { +// panic("mock out the Read method") +// }, +// ReadAtFunc: func(p []byte, off int64) (int, error) { +// panic("mock out the ReadAt method") +// }, +// SeekFunc: func(offset int64, whence int) (int64, error) { +// panic("mock out the Seek method") +// }, +// SyncFunc: func() error { +// panic("mock out the Sync method") +// }, +// TruncateFunc: func(size int64) error { +// panic("mock out the Truncate method") +// }, +// WriteFunc: func(p []byte) (int, error) { +// panic("mock out the Write method") +// }, +// } +// +// // use mockedStorageFile in code that requires StorageFile +// // and then make assertions. +// +// } +type StorageFileMock struct { + // CloseFunc mocks the Close method. + CloseFunc func() error + + // IsEmptyFunc mocks the IsEmpty method. + IsEmptyFunc func() bool + + // OpenFunc mocks the Open method. + OpenFunc func(flags int) error + + // ReadFunc mocks the Read method. + ReadFunc func(p []byte) (int, error) + + // ReadAtFunc mocks the ReadAt method. + ReadAtFunc func(p []byte, off int64) (int, error) + + // SeekFunc mocks the Seek method. + SeekFunc func(offset int64, whence int) (int64, error) + + // SyncFunc mocks the Sync method. + SyncFunc func() error + + // TruncateFunc mocks the Truncate method. + TruncateFunc func(size int64) error + + // WriteFunc mocks the Write method. + WriteFunc func(p []byte) (int, error) + + // calls tracks calls to the methods. + calls struct { + // Close holds details about calls to the Close method. + Close []struct { + } + // IsEmpty holds details about calls to the IsEmpty method. + IsEmpty []struct { + } + // Open holds details about calls to the Open method. + Open []struct { + // Flags is the flags argument value. + Flags int + } + // Read holds details about calls to the Read method. + Read []struct { + // P is the p argument value. + P []byte + } + // ReadAt holds details about calls to the ReadAt method. + ReadAt []struct { + // P is the p argument value. + P []byte + // Off is the off argument value. + Off int64 + } + // Seek holds details about calls to the Seek method. + Seek []struct { + // Offset is the offset argument value. + Offset int64 + // Whence is the whence argument value. + Whence int + } + // Sync holds details about calls to the Sync method. + Sync []struct { + } + // Truncate holds details about calls to the Truncate method. + Truncate []struct { + // Size is the size argument value. + Size int64 + } + // Write holds details about calls to the Write method. + Write []struct { + // P is the p argument value. + P []byte + } + } + lockClose sync.RWMutex + lockIsEmpty sync.RWMutex + lockOpen sync.RWMutex + lockRead sync.RWMutex + lockReadAt sync.RWMutex + lockSeek sync.RWMutex + lockSync sync.RWMutex + lockTruncate sync.RWMutex + lockWrite sync.RWMutex +} + +// Close calls CloseFunc. +func (mock *StorageFileMock) Close() error { + if mock.CloseFunc == nil { + panic("StorageFileMock.CloseFunc: method is nil but StorageFile.Close was just called") + } + callInfo := struct { + }{} + mock.lockClose.Lock() + mock.calls.Close = append(mock.calls.Close, callInfo) + mock.lockClose.Unlock() + return mock.CloseFunc() +} + +// CloseCalls gets all the calls that were made to Close. +// Check the length with: +// +// len(mockedStorageFile.CloseCalls()) +func (mock *StorageFileMock) CloseCalls() []struct { +} { + var calls []struct { + } + mock.lockClose.RLock() + calls = mock.calls.Close + mock.lockClose.RUnlock() + return calls +} + +// IsEmpty calls IsEmptyFunc. +func (mock *StorageFileMock) IsEmpty() bool { + if mock.IsEmptyFunc == nil { + panic("StorageFileMock.IsEmptyFunc: method is nil but StorageFile.IsEmpty was just called") + } + callInfo := struct { + }{} + mock.lockIsEmpty.Lock() + mock.calls.IsEmpty = append(mock.calls.IsEmpty, callInfo) + mock.lockIsEmpty.Unlock() + return mock.IsEmptyFunc() +} + +// IsEmptyCalls gets all the calls that were made to IsEmpty. +// Check the length with: +// +// len(mockedStorageFile.IsEmptyCalls()) +func (mock *StorageFileMock) IsEmptyCalls() []struct { +} { + var calls []struct { + } + mock.lockIsEmpty.RLock() + calls = mock.calls.IsEmpty + mock.lockIsEmpty.RUnlock() + return calls +} + +// Open calls OpenFunc. +func (mock *StorageFileMock) Open(flags int) error { + if mock.OpenFunc == nil { + panic("StorageFileMock.OpenFunc: method is nil but StorageFile.Open was just called") + } + callInfo := struct { + Flags int + }{ + Flags: flags, + } + mock.lockOpen.Lock() + mock.calls.Open = append(mock.calls.Open, callInfo) + mock.lockOpen.Unlock() + return mock.OpenFunc(flags) +} + +// OpenCalls gets all the calls that were made to Open. +// Check the length with: +// +// len(mockedStorageFile.OpenCalls()) +func (mock *StorageFileMock) OpenCalls() []struct { + Flags int +} { + var calls []struct { + Flags int + } + mock.lockOpen.RLock() + calls = mock.calls.Open + mock.lockOpen.RUnlock() + return calls +} + +// Read calls ReadFunc. +func (mock *StorageFileMock) Read(p []byte) (int, error) { + if mock.ReadFunc == nil { + panic("StorageFileMock.ReadFunc: method is nil but StorageFile.Read was just called") + } + callInfo := struct { + P []byte + }{ + P: p, + } + mock.lockRead.Lock() + mock.calls.Read = append(mock.calls.Read, callInfo) + mock.lockRead.Unlock() + return mock.ReadFunc(p) +} + +// ReadCalls gets all the calls that were made to Read. +// Check the length with: +// +// len(mockedStorageFile.ReadCalls()) +func (mock *StorageFileMock) ReadCalls() []struct { + P []byte +} { + var calls []struct { + P []byte + } + mock.lockRead.RLock() + calls = mock.calls.Read + mock.lockRead.RUnlock() + return calls +} + +// ReadAt calls ReadAtFunc. +func (mock *StorageFileMock) ReadAt(p []byte, off int64) (int, error) { + if mock.ReadAtFunc == nil { + panic("StorageFileMock.ReadAtFunc: method is nil but StorageFile.ReadAt was just called") + } + callInfo := struct { + P []byte + Off int64 + }{ + P: p, + Off: off, + } + mock.lockReadAt.Lock() + mock.calls.ReadAt = append(mock.calls.ReadAt, callInfo) + mock.lockReadAt.Unlock() + return mock.ReadAtFunc(p, off) +} + +// ReadAtCalls gets all the calls that were made to ReadAt. +// Check the length with: +// +// len(mockedStorageFile.ReadAtCalls()) +func (mock *StorageFileMock) ReadAtCalls() []struct { + P []byte + Off int64 +} { + var calls []struct { + P []byte + Off int64 + } + mock.lockReadAt.RLock() + calls = mock.calls.ReadAt + mock.lockReadAt.RUnlock() + return calls +} + +// Seek calls SeekFunc. +func (mock *StorageFileMock) Seek(offset int64, whence int) (int64, error) { + if mock.SeekFunc == nil { + panic("StorageFileMock.SeekFunc: method is nil but StorageFile.Seek was just called") + } + callInfo := struct { + Offset int64 + Whence int + }{ + Offset: offset, + Whence: whence, + } + mock.lockSeek.Lock() + mock.calls.Seek = append(mock.calls.Seek, callInfo) + mock.lockSeek.Unlock() + return mock.SeekFunc(offset, whence) +} + +// SeekCalls gets all the calls that were made to Seek. +// Check the length with: +// +// len(mockedStorageFile.SeekCalls()) +func (mock *StorageFileMock) SeekCalls() []struct { + Offset int64 + Whence int +} { + var calls []struct { + Offset int64 + Whence int + } + mock.lockSeek.RLock() + calls = mock.calls.Seek + mock.lockSeek.RUnlock() + return calls +} + +// Sync calls SyncFunc. +func (mock *StorageFileMock) Sync() error { + if mock.SyncFunc == nil { + panic("StorageFileMock.SyncFunc: method is nil but StorageFile.Sync was just called") + } + callInfo := struct { + }{} + mock.lockSync.Lock() + mock.calls.Sync = append(mock.calls.Sync, callInfo) + mock.lockSync.Unlock() + return mock.SyncFunc() +} + +// SyncCalls gets all the calls that were made to Sync. +// Check the length with: +// +// len(mockedStorageFile.SyncCalls()) +func (mock *StorageFileMock) SyncCalls() []struct { +} { + var calls []struct { + } + mock.lockSync.RLock() + calls = mock.calls.Sync + mock.lockSync.RUnlock() + return calls +} + +// Truncate calls TruncateFunc. +func (mock *StorageFileMock) Truncate(size int64) error { + if mock.TruncateFunc == nil { + panic("StorageFileMock.TruncateFunc: method is nil but StorageFile.Truncate was just called") + } + callInfo := struct { + Size int64 + }{ + Size: size, + } + mock.lockTruncate.Lock() + mock.calls.Truncate = append(mock.calls.Truncate, callInfo) + mock.lockTruncate.Unlock() + return mock.TruncateFunc(size) +} + +// TruncateCalls gets all the calls that were made to Truncate. +// Check the length with: +// +// len(mockedStorageFile.TruncateCalls()) +func (mock *StorageFileMock) TruncateCalls() []struct { + Size int64 +} { + var calls []struct { + Size int64 + } + mock.lockTruncate.RLock() + calls = mock.calls.Truncate + mock.lockTruncate.RUnlock() + return calls +} + +// Write calls WriteFunc. +func (mock *StorageFileMock) Write(p []byte) (int, error) { + if mock.WriteFunc == nil { + panic("StorageFileMock.WriteFunc: method is nil but StorageFile.Write was just called") + } + callInfo := struct { + P []byte + }{ + P: p, + } + mock.lockWrite.Lock() + mock.calls.Write = append(mock.calls.Write, callInfo) + mock.lockWrite.Unlock() + return mock.WriteFunc(p) +} + +// WriteCalls gets all the calls that were made to Write. +// Check the length with: +// +// len(mockedStorageFile.WriteCalls()) +func (mock *StorageFileMock) WriteCalls() []struct { + P []byte +} { + var calls []struct { + P []byte + } + mock.lockWrite.RLock() + calls = mock.calls.Write + mock.lockWrite.RUnlock() + return calls +} diff --git a/pp/go/storage/head/services/mock/mock.go b/pp/go/storage/head/services/mock/mock.go index e18cf9f9f1..9bc502fe64 100644 --- a/pp/go/storage/head/services/mock/mock.go +++ b/pp/go/storage/head/services/mock/mock.go @@ -2,11 +2,16 @@ package mock import ( "github.com/prometheus/prometheus/pp/go/cppbridge" + "github.com/prometheus/prometheus/pp/go/storage/head/shard" "github.com/prometheus/prometheus/pp/go/storage/head/shard/wal" ) //go:generate -command moq go run github.com/matryer/moq --rm --skip-ensure --pkg mock --out //go:generate moq segment_writer.go . SegmentWriter +//go:generate moq file_storage.go . StorageFile // SegmentWriter alias for [wal.SegmentWriter] with [cppbridge.HeadEncodedSegment]. type SegmentWriter = wal.SegmentWriter[*cppbridge.HeadEncodedSegment] + +// StorageFile alias for [shard.StorageFile]. +type StorageFile = shard.StorageFile diff --git a/pp/go/storage/querier/querier_test.go b/pp/go/storage/querier/querier_test.go index f263405bc3..991c103cc8 100644 --- a/pp/go/storage/querier/querier_test.go +++ b/pp/go/storage/querier/querier_test.go @@ -31,18 +31,18 @@ const ( ) type Querier = querier.Querier[ - *task.Generic[*storage.PerGoroutineShard], + *task.Generic[*shard.PerGoroutineShard], *shard.DataStorage, *shard.LSS, - *storage.PerGoroutineShard, - *storage.HeadOnDisk, + *shard.PerGoroutineShard, + *storage.Head, ] type QuerierSuite struct { suite.Suite dataDir string context context.Context - head *storage.HeadOnDisk + head *storage.Head } func TestQuerierSuite(t *testing.T) { @@ -78,7 +78,7 @@ func (s *QuerierSuite) mustCreateCatalog() *catalog.Catalog { return c } -func (s *QuerierSuite) mustCreateHead(unloadDataStorageInterval time.Duration) *storage.HeadOnDisk { +func (s *QuerierSuite) mustCreateHead(unloadDataStorageInterval time.Duration) *storage.Head { h, err := storage.NewBuilder( s.mustCreateCatalog(), s.dataDir, @@ -112,11 +112,11 @@ func (s *QuerierSuite) TestRangeQuery() { } s.appendTimeSeries(timeSeries) - q := querier.NewQuerier[*task.Generic[*storage.PerGoroutineShard], + q := querier.NewQuerier[*task.Generic[*shard.PerGoroutineShard], *shard.DataStorage, *shard.LSS, - *storage.PerGoroutineShard, - *storage.HeadOnDisk, + *shard.PerGoroutineShard, + *storage.Head, ](s.head, querier.NewNoOpShardedDeduplicator, 0, 2, nil, nil) defer func() { _ = q.Close() }() matcher, _ := labels.NewMatcher(labels.MatchEqual, "__name__", "metric") @@ -140,11 +140,11 @@ func (s *QuerierSuite) TestRangeQueryWithoutMatching() { } s.appendTimeSeries(timeSeries) - q := querier.NewQuerier[*task.Generic[*storage.PerGoroutineShard], + q := querier.NewQuerier[*task.Generic[*shard.PerGoroutineShard], *shard.DataStorage, *shard.LSS, - *storage.PerGoroutineShard, - *storage.HeadOnDisk, + *shard.PerGoroutineShard, + *storage.Head, ](s.head, querier.NewNoOpShardedDeduplicator, 0, 2, nil, nil) defer func() { _ = q.Close() }() matcher, _ := labels.NewMatcher(labels.MatchEqual, "__name__", "unknown_metric") @@ -193,11 +193,11 @@ func (s *QuerierSuite) TestRangeQueryWithDataStorageLoading() { }, } - q := querier.NewQuerier[*task.Generic[*storage.PerGoroutineShard], + q := querier.NewQuerier[*task.Generic[*shard.PerGoroutineShard], *shard.DataStorage, *shard.LSS, - *storage.PerGoroutineShard, - *storage.HeadOnDisk, + *shard.PerGoroutineShard, + *storage.Head, ](s.head, querier.NewNoOpShardedDeduplicator, 0, 3, nil, nil) defer func() { _ = q.Close() }() matcher, _ := labels.NewMatcher(labels.MatchEqual, "__name__", "metric") @@ -231,11 +231,11 @@ func (s *QuerierSuite) TestInstantQuery() { } s.appendTimeSeries(timeSeries) - q := querier.NewQuerier[*task.Generic[*storage.PerGoroutineShard], + q := querier.NewQuerier[*task.Generic[*shard.PerGoroutineShard], *shard.DataStorage, *shard.LSS, - *storage.PerGoroutineShard, - *storage.HeadOnDisk, + *shard.PerGoroutineShard, + *storage.Head, ](s.head, querier.NewNoOpShardedDeduplicator, 0, 0, nil, nil) defer func() { _ = q.Close() }() matcher, _ := labels.NewMatcher(labels.MatchEqual, "__name__", "metric") @@ -284,11 +284,11 @@ func (s *QuerierSuite) TestInstantQueryWithDataStorageLoading() { }, } - q := querier.NewQuerier[*task.Generic[*storage.PerGoroutineShard], + q := querier.NewQuerier[*task.Generic[*shard.PerGoroutineShard], *shard.DataStorage, *shard.LSS, - *storage.PerGoroutineShard, - *storage.HeadOnDisk, + *shard.PerGoroutineShard, + *storage.Head, ](s.head, querier.NewNoOpShardedDeduplicator, 0, 0, nil, nil) defer func() { _ = q.Close() }() matcher, _ := labels.NewMatcher(labels.MatchEqual, "__name__", "metric") From bd6b4a3d4d46085c8837e83f5271e044f3a5a0ee Mon Sep 17 00:00:00 2001 From: Alexandr Yudin Date: Thu, 9 Oct 2025 06:22:20 +0000 Subject: [PATCH 82/96] for save --- .../head/services/metrics_updater_test.go | 26 +++++++++++++++++++ pp/go/storage/head/services/rotator_test.go | 26 +++++++++++++++++++ 2 files changed, 52 insertions(+) create mode 100644 pp/go/storage/head/services/metrics_updater_test.go create mode 100644 pp/go/storage/head/services/rotator_test.go diff --git a/pp/go/storage/head/services/metrics_updater_test.go b/pp/go/storage/head/services/metrics_updater_test.go new file mode 100644 index 0000000000..0ddb682a4c --- /dev/null +++ b/pp/go/storage/head/services/metrics_updater_test.go @@ -0,0 +1,26 @@ +package services + +import ( + "context" + "testing" + + "github.com/stretchr/testify/suite" +) + +type MetricsUpdaterSuite struct { + suite.Suite + + baseCtx context.Context +} + +func TestMetricsUpdaterSuite(t *testing.T) { + suite.Run(t, new(MetricsUpdaterSuite)) +} + +func (s *MetricsUpdaterSuite) SetupSuite() { + s.baseCtx = context.Background() +} + +func (s *MetricsUpdaterSuite) TestHappyPath() { + s.T().Log("MetricsUpdaterSuite TestHappyPath TODO") +} diff --git a/pp/go/storage/head/services/rotator_test.go b/pp/go/storage/head/services/rotator_test.go new file mode 100644 index 0000000000..0a9bc23929 --- /dev/null +++ b/pp/go/storage/head/services/rotator_test.go @@ -0,0 +1,26 @@ +package services_test + +import ( + "context" + "testing" + + "github.com/stretchr/testify/suite" +) + +type RotatorSuite struct { + suite.Suite + + baseCtx context.Context +} + +func TestRotatorSuite(t *testing.T) { + suite.Run(t, new(RotatorSuite)) +} + +func (s *RotatorSuite) SetupSuite() { + s.baseCtx = context.Background() +} + +func (s *RotatorSuite) TestHappyPath() { + s.T().Log("RotatorSuite TestHappyPath TODO") +} From fde36f83beb8da77b5fe0951add6fd4eebb08162 Mon Sep 17 00:00:00 2001 From: Gleb Shigin Date: Thu, 9 Oct 2025 11:49:31 +0300 Subject: [PATCH 83/96] Lss snapshot copy (#145) * add patch to btree * tracy headers update * lss copy benchmark * update benchmark * remove bool pointer comparator hack * count a total sum in benchmark * benchmark fix * fancy benchmark * snug reserve method fix * benchmark fix * relabler reserve method fix * entrypoint integration * fixes over tests * bitset copy fix + tests * MemoryControlBlockWithItemCount copy test * SliceControlBlock fix * DecodingTable reserve fix * tidy fix * tidy fix * review fixes * template name fix * remove entrypoint from benchmark * lss copy keeps label set order test * fix types * add commiter test * add merger test --------- Co-authored-by: glebshigin From 1f1acf5600a84ed8ee4926cec1e53ba45f4b3fcf Mon Sep 17 00:00:00 2001 From: cherep58 Date: Thu, 9 Oct 2025 13:17:12 +0300 Subject: [PATCH 84/96] Refactored relabeler classes (#162) * fixed roaring include error in CLion * fixed linter warnings * refactored tests for relabeler * refactored stateless relabeler * renamed LabelsBuilderStateMap to LabelsBuilder * refactored relabeler tests * rewritten test for PerGoroutineRelabelerFixture * refactored PerGoroutineRelabeler * fixed clang-tidy warning * used std::string instead of std::ostringstream * optimized hard_validate function * fixed bug * used reserve_and_write * fixed bug in RelabelConfig with mutable variable --- pp/WORKSPACE | 3 + pp/primitives/label_set.h | 22 +- pp/primitives/labels_builder.h | 154 +- pp/primitives/tests/labels_builder_tests.cpp | 77 +- pp/prometheus/relabeler.h | 736 +++---- pp/prometheus/stateless_relabeler.h | 489 ++--- pp/prometheus/tests/relabeler_tests.cpp | 1547 +++++--------- .../tests/stateless_relabeler_tests.cpp | 1890 ++++++----------- pp/wal/output_decoder.h | 16 +- 9 files changed, 1686 insertions(+), 3248 deletions(-) diff --git a/pp/WORKSPACE b/pp/WORKSPACE index 6e6ee41b6c..717cf53bd4 100644 --- a/pp/WORKSPACE +++ b/pp/WORKSPACE @@ -118,6 +118,9 @@ http_archive( "//third_party/patches/roaring:0001-disable-test-dependencies.patch", "//third_party/patches/roaring:0002-svacer_fixes.patch", ], + patch_cmds = [ + "cp cpp/* include/roaring" + ], sha256 = "a037e12a3f7c8c2abb3e81fc9669c23e274ffa2d8670d2034a2e05969e53689b", strip_prefix = "CRoaring-1.3.0/", url = "https://github.com/RoaringBitmap/CRoaring/archive/refs/tags/v1.3.0.zip", diff --git a/pp/primitives/label_set.h b/pp/primitives/label_set.h index c268fea8fa..4ff3609f89 100644 --- a/pp/primitives/label_set.h +++ b/pp/primitives/label_set.h @@ -8,8 +8,6 @@ namespace PromPP::Primitives { template class Container = BareBones::Vector> class BasicLabelSet { - Container labels_; - public: using label_type = LabelType; @@ -68,6 +66,17 @@ class BasicLabelSet { } } + template + PROMPP_ALWAYS_INLINE void append_unsorted(const LabelSet& label_set) { + labels_.reserve(labels_.size() + label_set.size()); + + for (const auto& label : label_set) { + append(label.first, label.second); + } + + sort(); + } + template PROMPP_ALWAYS_INLINE void add(const LabelSet& label_set) { labels_.reserve(labels_.size() + label_set.size()); @@ -102,6 +111,8 @@ class BasicLabelSet { [[nodiscard]] PROMPP_ALWAYS_INLINE const_iterator end() const noexcept { return labels_.end(); } PROMPP_ALWAYS_INLINE iterator end() noexcept { return labels_.end(); } + [[nodiscard]] PROMPP_ALWAYS_INLINE const LabelType& operator[](uint32_t index) const noexcept { return labels_[index]; } + [[nodiscard]] PROMPP_ALWAYS_INLINE uint32_t allocated_memory() const noexcept { return BareBones::mem::allocated_memory(labels_); } template @@ -169,6 +180,13 @@ class BasicLabelSet { }; [[nodiscard]] PROMPP_ALWAYS_INLINE Names names() const noexcept { return Names(*this); } + + private: + Container labels_; + + PROMPP_ALWAYS_INLINE void sort() noexcept { + std::ranges::sort(labels_, [](const auto& a, const auto& b) { return a.first < b.first; }); + } }; using LabelSet = BasicLabelSet; diff --git a/pp/primitives/labels_builder.h b/pp/primitives/labels_builder.h index d9253f0f72..cc458c9c8d 100644 --- a/pp/primitives/labels_builder.h +++ b/pp/primitives/labels_builder.h @@ -9,20 +9,11 @@ #include "primitives.h" namespace PromPP::Primitives { -class LabelsBuilderStateMap { - PromPP::Primitives::LabelViewSet building_buf_view_; - PromPP::Primitives::LabelSet building_buf_; - phmap::flat_hash_map buffer_; - template - void sort_labels(Labels& labels) { - std::ranges::sort(labels, [](const auto& a, const auto& b) { - if (a.first == b.first) { - return a.second < b.second; - } - return a.first < b.first; - }); - } +class LabelsBuilder { + LabelViewSet building_buf_view_; + LabelSet building_buf_; + phmap::flat_hash_map buffer_; public: // del add label name to remove from label set. @@ -33,8 +24,8 @@ class LabelsBuilderStateMap { // extract we extract(move) the lebel from the builder. PROMPP_ALWAYS_INLINE Label extract(const std::string_view& lname) { - if (auto it = buffer_.find(lname); it != buffer_.end()) { - auto node = buffer_.extract(it); + if (const auto it = buffer_.find(lname); it != buffer_.end()) { + auto&& node = buffer_.extract(it); return {std::move(const_cast(node.key())), std::move(node.mapped())}; } @@ -42,17 +33,17 @@ class LabelsBuilderStateMap { } // get returns the value for the label with the given name. Returns an empty string if the label doesn't exist. - PROMPP_ALWAYS_INLINE std::string_view get(const std::string_view lname) { - if (auto it = buffer_.find(lname); it != buffer_.end()) { - return (*it).second; + PROMPP_ALWAYS_INLINE std::string_view get(const std::string_view& lname) { + if (const auto it = buffer_.find(lname); it != buffer_.end()) { + return it->second; } return ""; } // contains check the given name if exist. - [[nodiscard]] PROMPP_ALWAYS_INLINE bool contains(const std::string_view lname) const noexcept { - if (auto it = buffer_.find(lname); it != buffer_.end()) { + [[nodiscard]] PROMPP_ALWAYS_INLINE bool contains(const std::string_view& lname) const noexcept { + if (const auto it = buffer_.find(lname); it != buffer_.end()) { return true; } @@ -68,7 +59,7 @@ class LabelsBuilderStateMap { } if (auto it = buffer_.find(lname); it != buffer_.end()) { - (*it).second = lvalue; + it->second = lvalue; return; } @@ -76,61 +67,34 @@ class LabelsBuilderStateMap { } // returns size of building labels. - PROMPP_ALWAYS_INLINE size_t size() { return buffer_.size(); } + [[nodiscard]] PROMPP_ALWAYS_INLINE size_t size() const { return buffer_.size(); } // returns true if ls represents an empty set of labels. - PROMPP_ALWAYS_INLINE bool is_empty() { return buffer_.size() == 0; } + [[nodiscard]] PROMPP_ALWAYS_INLINE bool is_empty() const { return buffer_.empty(); } // label_view_set - returns the label_view set from the builder. If no modifications were made, the original labels are returned. PROMPP_ALWAYS_INLINE const LabelViewSet& label_view_set() { building_buf_view_.clear(); - for (const auto& it : buffer_) { - if (it.second == "") [[unlikely]] { - continue; - } - - building_buf_view_.add(LabelView{it.first, it.second}); - } - - if (building_buf_view_.size() != 0) { - sort_labels(building_buf_view_); - } - + building_buf_view_.append_unsorted(buffer_); return building_buf_view_; } // label_set - returns the label set from the builder. If no modifications were made, the original labels are returned. PROMPP_ALWAYS_INLINE const LabelSet& label_set() { building_buf_.clear(); - - for (const auto& it : buffer_) { - if (it.second == "") [[unlikely]] { - continue; - } - - building_buf_.add(Label{it.first, it.second}); - } - - if (building_buf_.size() != 0) { - sort_labels(building_buf_); - } - + building_buf_.append_unsorted(buffer_); return building_buf_; } - // range - calls f on each label in the builder. - // TODO without copy buffer_, all changes in a another cycle. + // range - calls f on each label in the builder. Don't modify LabelsBuilderStateMap inside callback template - PROMPP_ALWAYS_INLINE void range(Callback func) { - // take a copy of add and del, so they are unaffected by calls to set() or del(). - phmap::flat_hash_map cbuffer_ = buffer_; - - for (const auto& it : cbuffer_) { - if (it.second == "") [[unlikely]] { + PROMPP_ALWAYS_INLINE void range(Callback func) const { + for (const auto& it : buffer_) { + if (it.second.empty()) [[unlikely]] { continue; } - if (bool ok = func(it.first, it.second); !ok) { + if (!func(it.first, it.second)) { return; } } @@ -145,83 +109,15 @@ class LabelsBuilderStateMap { // reset - clears all current state for the builder. template - PROMPP_ALWAYS_INLINE void reset(SomeLabelSet& base) { - building_buf_view_.clear(); - building_buf_.clear(); - buffer_.clear(); + PROMPP_ALWAYS_INLINE void reset(const SomeLabelSet& base) { + reset(); for (const auto& [lname, lvalue] : base) { - if (lvalue == "") { - continue; + if (!lvalue.empty()) [[likely]] { + buffer_[lname] = lvalue; } - - buffer_[lname] = lvalue; } } }; -// LabelsBuilder - builder for label set. -template -class LabelsBuilder { - BuilderState& state_; - - public: - PROMPP_ALWAYS_INLINE explicit LabelsBuilder(BuilderState& state) : state_(state) {} - - template - PROMPP_ALWAYS_INLINE explicit LabelsBuilder(BuilderState& state, SomeLabelSet* ls) : state_(state) { - reset(ls); - } - - // del - add label name to remove from label set. - template - PROMPP_ALWAYS_INLINE void del(const LNameType& lname) { - state_.del(lname); - } - - // extract we extract(move) the lebel from the builder. - PROMPP_ALWAYS_INLINE Label extract(const std::string_view& lname) { return state_.extract(lname); } - - // get - returns the value for the label with the given name. Returns an empty string if the label doesn't exist. - PROMPP_ALWAYS_INLINE std::string_view get(const std::string_view lname) { return state_.get(lname); } - - // contains check the given name if exist. - [[nodiscard]] PROMPP_ALWAYS_INLINE bool contains(const std::string_view lname) const noexcept { return state_.contains(lname); } - - // returns size of building labels. - PROMPP_ALWAYS_INLINE size_t size() { return state_.size(); } - - // returns true if ls represents an empty set of labels. - PROMPP_ALWAYS_INLINE bool is_empty() { return state_.is_empty(); } - - // label_view_set - returns the label_view set from the builder. If no modifications were made, the original labels are returned. - PROMPP_ALWAYS_INLINE const PromPP::Primitives::LabelViewSet& label_view_set() { return state_.label_view_set(); } - - // label_set - returns the label set from the builder. If no modifications were made, the original labels are returned. - PROMPP_ALWAYS_INLINE const PromPP::Primitives::LabelSet& label_set() { return state_.label_set(); } - - // range - calls f on each label in the builder. - template - PROMPP_ALWAYS_INLINE void range(Callback func) { - state_.range(func); - } - - // reset - clears all current state for the builder. - PROMPP_ALWAYS_INLINE void reset() { state_.reset(); } - - // reset - clears all current state for the builder and init from LabelSet. - template - PROMPP_ALWAYS_INLINE void reset(const SomeLabelSet& ls) { - state_.reset(ls); - } - - // set - the name/value pair as a label. A value of "" means delete that label. - template - PROMPP_ALWAYS_INLINE void set(const LNameType& lname, const LValueType& lvalue) { - state_.set(lname, lvalue); - } - - PROMPP_ALWAYS_INLINE LabelsBuilder(LabelsBuilder&&) noexcept = default; - PROMPP_ALWAYS_INLINE ~LabelsBuilder() = default; -}; } // namespace PromPP::Primitives \ No newline at end of file diff --git a/pp/primitives/tests/labels_builder_tests.cpp b/pp/primitives/tests/labels_builder_tests.cpp index 6454b4b487..8ef24b03cc 100644 --- a/pp/primitives/tests/labels_builder_tests.cpp +++ b/pp/primitives/tests/labels_builder_tests.cpp @@ -5,8 +5,7 @@ namespace { struct TestLabelsBuilder : public testing::Test { - PromPP::Primitives::LabelsBuilderStateMap builder_state_; - PromPP::Primitives::LabelsBuilder builder_{builder_state_}; + PromPP::Primitives::LabelsBuilder builder_; PromPP::Primitives::LabelViewSet ls_view_; PromPP::Primitives::LabelSet ls_; std::vector> DATA{{"qwe", "ewq"}, {"asd", "dsa"}, {"zxc", "cxz"}}; @@ -68,7 +67,7 @@ TEST_F(TestLabelsBuilder, SetEmpty) { ls_.add({DATA[i].first, DATA[i].second}); } - std::string empty = ""; + const std::string empty; builder_.set(DATA[0].first, empty); EXPECT_EQ(builder_.label_view_set(), ls_view_); @@ -85,7 +84,7 @@ TEST_F(TestLabelsBuilder, SetChange) { ls_.add({lname, lvalue}); } - std::string value = "zxcv"; + const std::string value = "zxcv"; builder_.set(DATA[0].first, value); EXPECT_NE(builder_.label_view_set(), ls_view_); @@ -110,7 +109,7 @@ TEST_F(TestLabelsBuilder, Extract) { builder_.set(lname, lvalue); } - auto l = builder_.extract(DATA[0].first); + const auto l = builder_.extract(DATA[0].first); EXPECT_EQ(DATA[0], l); EXPECT_EQ(builder_.get(l.first), ""); @@ -123,7 +122,7 @@ TEST_F(TestLabelsBuilder, Del) { builder_.del(DATA[0].first); - std::string_view b_lvalue = builder_.get(DATA[0].first); + const std::string_view b_lvalue = builder_.get(DATA[0].first); EXPECT_EQ(b_lvalue, ""); } @@ -142,72 +141,6 @@ TEST_F(TestLabelsBuilder, SetDelSet) { EXPECT_EQ(builder_.label_set(), ls_); } -TEST_F(TestLabelsBuilder, Range) { - for (auto& [lname, lvalue] : DATA) { - ls_view_.add({lvalue, lname}); - builder_.set(lname, lvalue); - ls_.add({lvalue, lname}); - } - - builder_.range([&](LNameType& lname, LValueType& lvalue) PROMPP_LAMBDA_INLINE -> bool { - builder_.del(lname); - builder_.set(lvalue, lname); - return true; - }); - - EXPECT_EQ(builder_.label_view_set(), ls_view_); - EXPECT_EQ(builder_.label_set(), ls_); -} - -TEST_F(TestLabelsBuilder, RangeFastExit) { - for (size_t i = 0; i < DATA.size(); ++i) { - builder_.set(DATA[i].first, DATA[i].second); - if (i == 2) { - // for last label not swap lname and lvalue - ls_view_.add({DATA[i].first, DATA[i].second}); - ls_.add({DATA[i].first, DATA[i].second}); - continue; - } - ls_view_.add({DATA[i].second, DATA[i].first}); - ls_.add({DATA[i].second, DATA[i].first}); - } - - size_t count{0}; - builder_.range([&](LNameType& lname, LValueType& lvalue) PROMPP_LAMBDA_INLINE -> bool { - builder_.del(lname); - builder_.set(lvalue, lname); - ++count; - if (count == 2) { - return false; - }; - return true; - }); - - EXPECT_EQ(count, 2); - EXPECT_EQ(builder_.label_view_set(), ls_view_); - EXPECT_EQ(builder_.label_set(), ls_); -} - -TEST_F(TestLabelsBuilder, ResetRange) { - PromPP::Primitives::LabelViewSet ls; - for (auto& [lname, lvalue] : DATA) { - ls.add({lname, lvalue}); - ls_view_.add({lvalue, lname}); - ls_.add({lvalue, lname}); - } - - builder_.reset(ls); - - builder_.range([&](LNameType& lname, LValueType& lvalue) PROMPP_LAMBDA_INLINE -> bool { - builder_.del(lname); - builder_.set(lvalue, lname); - return true; - }); - - EXPECT_EQ(builder_.label_view_set(), ls_view_); - EXPECT_EQ(builder_.label_set(), ls_); -} - TEST_F(TestLabelsBuilder, NotIsEmpty) { for (auto& [lname, lvalue] : DATA) { builder_.set(lname, lvalue); diff --git a/pp/prometheus/relabeler.h b/pp/prometheus/relabeler.h index 97ed91c1ce..f6bb8df606 100644 --- a/pp/prometheus/relabeler.h +++ b/pp/prometheus/relabeler.h @@ -1,6 +1,5 @@ #pragma once -#include #include #include #include @@ -29,14 +28,19 @@ struct MetricLimits { size_t label_value_length_limit{0}; size_t sample_limit{0}; - PROMPP_ALWAYS_INLINE bool label_limit_exceeded(size_t labels_count) { return label_limit > 0 && labels_count > label_limit; } - - PROMPP_ALWAYS_INLINE bool samples_limit_exceeded(size_t samples_count) { return sample_limit > 0 && samples_count >= sample_limit; } + [[nodiscard]] PROMPP_ALWAYS_INLINE bool label_limit_exceeded(size_t labels_count) const { return label_limit > 0 && labels_count > label_limit; } + [[nodiscard]] PROMPP_ALWAYS_INLINE bool samples_limit_exceeded(size_t samples_count) const { return sample_limit > 0 && samples_count >= sample_limit; } + [[nodiscard]] PROMPP_ALWAYS_INLINE bool label_name_length_limit_exceeded(size_t label_name_length) const { + return label_name_length_limit > 0 && label_name_length > label_name_length_limit; + } + [[nodiscard]] PROMPP_ALWAYS_INLINE bool label_value_length_limit_exceeded(size_t label_value_length) const { + return label_value_length_limit > 0 && label_value_length > label_value_length_limit; + } }; // hard_validate on empty, name label(__name__) mandatory, valid label name and value) validate label set. template -PROMPP_ALWAYS_INLINE void hard_validate(relabelStatus& rstatus, LabelsBuilder& builder, MetricLimits* limits) { +PROMPP_ALWAYS_INLINE void hard_validate(relabelStatus& rstatus, LabelsBuilder& builder, const MetricLimits* limits) { if (rstatus == rsDrop) { return; } @@ -54,13 +58,17 @@ PROMPP_ALWAYS_INLINE void hard_validate(relabelStatus& rstatus, LabelsBuilder& b } // validate labels - builder.range([&](LNameType& lname, LValueType& lvalue) PROMPP_LAMBDA_INLINE -> bool { - if (lname == kMetricLabelName && !metric_name_value_is_valid(lvalue)) { - rstatus = rsInvalid; - return false; + builder.range([&](const auto& lname, const auto& lvalue) PROMPP_LAMBDA_INLINE -> bool { + if (lname == kMetricLabelName) [[unlikely]] { + if (!metric_name_value_is_valid(lvalue)) [[unlikely]] { + rstatus = rsInvalid; + return false; + } + + return true; } - if (!label_name_is_valid(lname) || !label_value_is_valid(lvalue)) { + if (!label_name_is_valid(lname) || !label_value_is_valid(lvalue)) [[unlikely]] { rstatus = rsInvalid; return false; } @@ -86,13 +94,13 @@ PROMPP_ALWAYS_INLINE void hard_validate(relabelStatus& rstatus, LabelsBuilder& b } // check limit len label name and value - builder.range([&](LNameType& lname, LValueType& lvalue) PROMPP_LAMBDA_INLINE -> bool { - if (limits->label_name_length_limit > 0 && lname.size() > limits->label_name_length_limit) { + builder.range([&](const auto& lname, auto& lvalue) PROMPP_LAMBDA_INLINE -> bool { + if (limits->label_name_length_limit_exceeded(lname.size())) { rstatus = rsInvalid; return false; } - if (limits->label_value_length_limit > 0 && lvalue.size() > limits->label_value_length_limit) { + if (limits->label_value_length_limit_exceeded(lvalue.size())) { rstatus = rsInvalid; return false; } @@ -106,7 +114,7 @@ PROMPP_ALWAYS_INLINE void hard_validate(relabelStatus& rstatus, LabelsBuilder& b // samples - incoming samples; // ls_id - relabeling ls id from lss; struct InnerSerie { - PromPP::Primitives::Sample sample; + Primitives::Sample sample; uint32_t ls_id; PROMPP_ALWAYS_INLINE bool operator==(const InnerSerie& rt) const noexcept = default; @@ -121,17 +129,29 @@ class InnerSeries { BareBones::Vector data_; public: - PROMPP_ALWAYS_INLINE const BareBones::Vector& data() const { return data_; } + [[nodiscard]] PROMPP_ALWAYS_INLINE const BareBones::Vector& data() const { return data_; } - PROMPP_ALWAYS_INLINE size_t size() const { return size_; } + [[nodiscard]] PROMPP_ALWAYS_INLINE size_t size() const { return size_; } PROMPP_ALWAYS_INLINE void reserve(size_t n) { data_.reserve(n); } - PROMPP_ALWAYS_INLINE void emplace_back(const PromPP::Primitives::Sample& sample, const uint32_t& ls_id) { + PROMPP_ALWAYS_INLINE void emplace_back(const Primitives::Sample& sample, uint32_t ls_id) { data_.emplace_back(sample, ls_id); ++size_; } + PROMPP_ALWAYS_INLINE void emplace_back(auto const& samples, uint32_t ls_id) { + data_.reserve_and_write(samples.size(), [&](InnerSerie* series_buffer, uint32_t series_size) { + for (const auto& sample : samples) { + std::construct_at(series_buffer, sample, ls_id); + ++series_buffer; + } + return series_size; + }); + + size_ += samples.size(); + } + PROMPP_ALWAYS_INLINE void clear() noexcept { data_.clear(); size_ = 0; @@ -145,8 +165,8 @@ class InnerSeries { // hash - hash sum from ls; // ls_id - incoming ls id from lss; struct RelabeledSerie { - PromPP::Primitives::LabelSet ls; - BareBones::Vector samples; + Primitives::LabelSet ls; + BareBones::Vector samples; size_t hash; uint32_t ls_id; }; @@ -160,17 +180,25 @@ class RelabeledSeries { std::vector data_; public: - PROMPP_ALWAYS_INLINE const std::vector& data() const { return data_; } + [[nodiscard]] PROMPP_ALWAYS_INLINE const std::vector& data() const { return data_; } - PROMPP_ALWAYS_INLINE size_t size() const { return size_; } + [[nodiscard]] PROMPP_ALWAYS_INLINE size_t size() const { return size_; } - PROMPP_ALWAYS_INLINE void emplace_back(PromPP::Primitives::LabelSet& ls, - const BareBones::Vector& samples, + PROMPP_ALWAYS_INLINE void emplace_back(const Primitives::LabelSet& ls, + const BareBones::Vector& samples, const size_t hash, const uint32_t ls_id) { data_.emplace_back(ls, samples, hash, ls_id); ++size_; } + + PROMPP_ALWAYS_INLINE void emplace_back(const Primitives::LabelSet& ls, + BareBones::Vector&& samples, + const size_t hash, + const uint32_t ls_id) { + data_.emplace_back(ls, std::move(samples), hash, ls_id); + ++size_; + } }; // CacheValue - value for cache map. @@ -189,15 +217,15 @@ struct IncomingAndRelabeledLsID { }; // RelabelerStateUpdate - container for update states. -using RelabelerStateUpdate = PromPP::Primitives::Go::Slice; +using RelabelerStateUpdate = Primitives::Go::Slice; class NoOpStaleNaNsState { public: - PROMPP_ALWAYS_INLINE void add_input([[maybe_unused]] uint32_t id) {} - PROMPP_ALWAYS_INLINE void add_target([[maybe_unused]] uint32_t id) {} + PROMPP_ALWAYS_INLINE static void add_input([[maybe_unused]] uint32_t id) {} + PROMPP_ALWAYS_INLINE static void add_target([[maybe_unused]] uint32_t id) {} template - PROMPP_ALWAYS_INLINE void swap([[maybe_unused]] InputCallback input_fn, [[maybe_unused]] TargetCallback target_fn) {} + PROMPP_ALWAYS_INLINE static void swap([[maybe_unused]] InputCallback input_fn, [[maybe_unused]] TargetCallback target_fn) {} }; // StaleNaNsState state for stale nans. @@ -208,8 +236,6 @@ class StaleNaNsState { roaring::Roaring prev_target_bitset_{}; public: - PROMPP_ALWAYS_INLINE explicit StaleNaNsState() {} - PROMPP_ALWAYS_INLINE void add_input(uint32_t id) { input_bitset_.add(id); } PROMPP_ALWAYS_INLINE void add_target(uint32_t id) { target_bitset_.add(id); } @@ -235,20 +261,14 @@ class StaleNaNsState { // Cache stateless cache for relabeler. class Cache { size_t cache_allocated_memory_{0}; - phmap::parallel_flat_hash_map, - std::equal_to<>, - BareBones::Allocator>> - cache_relabel_{{}, {}, BareBones::Allocator>{cache_allocated_memory_}}; + phmap::parallel_flat_hash_map, std::equal_to<>, BareBones::Allocator>> + cache_relabel_{{}, {}, BareBones::Allocator>{cache_allocated_memory_}}; roaring::Roaring cache_keep_{}; roaring::Roaring cache_drop_{}; public: - PROMPP_ALWAYS_INLINE explicit Cache() {} - // allocated_memory return size of allocated memory for caches. - PROMPP_ALWAYS_INLINE size_t allocated_memory() const noexcept { + [[nodiscard]] PROMPP_ALWAYS_INLINE size_t allocated_memory() const noexcept { return cache_allocated_memory_ + cache_keep_.getSizeInBytes() + cache_drop_.getSizeInBytes(); } @@ -269,13 +289,13 @@ class Cache { cache_drop_.runOptimize(); } - PROMPP_ALWAYS_INLINE double part_of_drops() { + [[nodiscard]] PROMPP_ALWAYS_INLINE double part_of_drops() const { if (cache_drop_.cardinality() == 0) { return 0; } return std::bit_cast(cache_drop_.cardinality()) / - std::bit_cast(cache_drop_.cardinality() + cache_keep_.cardinality() + static_cast(cache_relabel_.size())); + std::bit_cast(cache_drop_.cardinality() + cache_keep_.cardinality() + cache_relabel_.size()); } struct CheckResult { @@ -293,13 +313,12 @@ class Cache { template PROMPP_ALWAYS_INLINE CheckResult check(const InputLSS& input_lss, const TargetLSS& target_lss, LabelSet& label_set, size_t hash) { - if (std::optional ls_id = input_lss.find(label_set, hash); ls_id.has_value()) { - auto res = check_input(ls_id.value()); - if (res.status != CheckResult::kNotFound) { + if (const auto ls_id = input_lss.find(label_set, hash); ls_id.has_value()) { + if (auto res = check_input(ls_id.value()); res.status != CheckResult::kNotFound) { return res; } } - if (std::optional ls_id = target_lss.find(label_set, hash); ls_id.has_value()) { + if (const auto ls_id = target_lss.find(label_set, hash); ls_id.has_value()) { return check_target(ls_id.value()); } return {}; @@ -310,14 +329,14 @@ class Cache { return {.status = CheckResult::Status::kDrop}; } - if (auto it = cache_relabel_.find(ls_id); it != cache_relabel_.end()) { + if (const auto it = cache_relabel_.find(ls_id); it != cache_relabel_.end()) { return {.status = CheckResult::Status::kRelabel, .shard_id = it->second.shard_id, .ls_id = it->second.ls_id, .source_ls_id = ls_id}; } return {}; } - PROMPP_ALWAYS_INLINE CheckResult check_target(uint32_t ls_id) { + [[nodiscard]] PROMPP_ALWAYS_INLINE CheckResult check_target(uint32_t ls_id) const { if (cache_keep_.contains(ls_id)) { return {.status = CheckResult::Status::kKeep, .ls_id = ls_id}; } @@ -335,7 +354,7 @@ class Cache { }; struct RelabelerOptions { - PromPP::Primitives::Go::SliceView> target_labels{}; + Primitives::Go::SliceView> target_labels{}; MetricLimits* metric_limits{nullptr}; bool honor_labels{false}; bool track_timestamps_staleness{false}; @@ -351,21 +370,20 @@ struct RelabelerOptions { // shard_id_ - current shard id; // log_shards_ - logarithm to the base 2 of total shards count; class PerShardRelabeler { - std::stringstream buf_; - PromPP::Primitives::LabelsBuilderStateMap builder_state_; - std::vector external_labels_{}; - PromPP::Primitives::TimeseriesSemiview timeseries_buf_; + std::string buf_; + Primitives::LabelsBuilder builder_; + std::vector external_labels_{}; + Primitives::TimeseriesSemiview timeseries_buf_; StatelessRelabeler* stateless_relabeler_; uint16_t number_of_shards_; uint16_t shard_id_; public: // PerShardRelabeler - constructor. Init only with pre-initialized LSS* and StatelessRelabeler*. - PROMPP_ALWAYS_INLINE PerShardRelabeler( - PromPP::Primitives::Go::SliceView>& external_labels, - StatelessRelabeler* stateless_relabeler, - const uint16_t number_of_shards, - const uint16_t shard_id) + PROMPP_ALWAYS_INLINE PerShardRelabeler(Primitives::Go::SliceView>& external_labels, + StatelessRelabeler* stateless_relabeler, + const uint16_t number_of_shards, + const uint16_t shard_id) : stateless_relabeler_(stateless_relabeler), number_of_shards_(number_of_shards), shard_id_(shard_id) { if (stateless_relabeler_ == nullptr) [[unlikely]] { throw BareBones::Exception(0xabd6db40882fd6aa, "stateless relabeler is null pointer"); @@ -378,18 +396,18 @@ class PerShardRelabeler { } private: - PROMPP_ALWAYS_INLINE bool resolve_timestamps(PromPP::Primitives::Timestamp def_timestamp, - BareBones::Vector& samples, - const RelabelerOptions& o) { + PROMPP_ALWAYS_INLINE static bool resolve_timestamps(Primitives::Timestamp def_timestamp, + BareBones::Vector& samples, + const RelabelerOptions& o) { // skip resolve without stalenans - if (def_timestamp == PromPP::Primitives::kNullTimestamp) { + if (def_timestamp == Primitives::kNullTimestamp) { return false; } bool track_staleness{true}; for (auto& sample : samples) { // replace null timestamp on def timestamp - if (sample.timestamp() == PromPP::Primitives::kNullTimestamp) { + if (sample.timestamp() == Primitives::kNullTimestamp) { sample.timestamp() = def_timestamp; continue; } @@ -426,14 +444,15 @@ class PerShardRelabeler { const Hashdex& hashdex, const RelabelerOptions& o, Stats& stats, - PromPP::Primitives::Go::SliceView& shards_inner_series, - PromPP::Primitives::Go::SliceView& shards_relabeled_series, + Primitives::Go::SliceView& shards_inner_series, + Primitives::Go::SliceView& shards_relabeled_series, StNaNsState& stale_nan_state, - PromPP::Primitives::Timestamp def_timestamp) { + Primitives::Timestamp def_timestamp) { assert(number_of_shards_ > 0); - size_t n = std::min(static_cast(hashdex.size()), static_cast((hashdex.size() * (1 - cache.part_of_drops()) * 1.1) / number_of_shards_)); - for (auto i = 0; i < number_of_shards_; ++i) { + const size_t n = + std::min(static_cast(hashdex.size()), static_cast((hashdex.size() * (1 - cache.part_of_drops()) * 1.1) / number_of_shards_)); + for (uint16_t i = 0; i < number_of_shards_; ++i) { if (shards_inner_series[i]->size() >= n) { continue; } @@ -441,7 +460,6 @@ class PerShardRelabeler { shards_inner_series[i]->reserve(n); } - PromPP::Primitives::LabelsBuilder builder{builder_state_}; size_t samples_count{0}; for (auto it = skip_shard_inner_series(hashdex, shards_inner_series[shard_id_]->size()); it != hashdex.end(); ++it) { @@ -455,14 +473,9 @@ class PerShardRelabeler { Cache::CheckResult check_result = cache.check(input_lss, target_lss, timeseries_buf_.label_set(), it->hash()); switch (check_result.status) { case Cache::CheckResult::kNotFound: { - builder.reset(timeseries_buf_.label_set()); - auto rstatus = relabel(o, builder); - switch (rstatus) { - case rsDrop: { - cache.add_drop(input_lss.find_or_emplace(timeseries_buf_.label_set(), it->hash())); - ++stats.series_drop; - continue; - } + builder_.reset(timeseries_buf_.label_set()); + switch (relabel(o, builder_)) { + case rsDrop: case rsInvalid: { cache.add_drop(input_lss.find_or_emplace(timeseries_buf_.label_set(), it->hash())); ++stats.series_drop; @@ -472,11 +485,11 @@ class PerShardRelabeler { auto ls_id = target_lss.find_or_emplace(timeseries_buf_.label_set(), it->hash()); cache.add_keep(ls_id); auto& samples = timeseries_buf_.samples(); - bool all_samples_reseted_to_scrape_ts = resolve_timestamps(def_timestamp, samples, o); + const bool all_samples_reseted_to_scrape_ts = resolve_timestamps(def_timestamp, samples, o); if (o.track_timestamps_staleness || all_samples_reseted_to_scrape_ts) { stale_nan_state.add_target(ls_id); } - for (const PromPP::Primitives::Sample& sample : samples) { + for (const Primitives::Sample& sample : samples) { shards_inner_series[shard_id_]->emplace_back(sample, ls_id); } ++stats.series_added; @@ -485,11 +498,11 @@ class PerShardRelabeler { } case rsRelabel: { auto ls_id = input_lss.find_or_emplace(timeseries_buf_.label_set(), it->hash()); - PromPP::Primitives::LabelSet new_label_set = builder.label_set(); + const auto& new_label_set = builder_.label_set(); size_t new_hash = hash_value(new_label_set); - size_t new_shard_id = new_hash % number_of_shards_; + const size_t new_shard_id = new_hash % number_of_shards_; auto& samples = timeseries_buf_.samples(); - bool all_samples_reseted_to_scrape_ts = resolve_timestamps(def_timestamp, samples, o); + const bool all_samples_reseted_to_scrape_ts = resolve_timestamps(def_timestamp, samples, o); if (o.track_timestamps_staleness || all_samples_reseted_to_scrape_ts) { stale_nan_state.add_input(ls_id); } @@ -504,11 +517,11 @@ class PerShardRelabeler { } case Cache::CheckResult::kKeep: { auto& samples = timeseries_buf_.samples(); - bool all_samples_reseted_to_scrape_ts = resolve_timestamps(def_timestamp, samples, o); + const bool all_samples_reseted_to_scrape_ts = resolve_timestamps(def_timestamp, samples, o); if (o.track_timestamps_staleness || all_samples_reseted_to_scrape_ts) { stale_nan_state.add_target(check_result.ls_id); } - for (const PromPP::Primitives::Sample& sample : samples) { + for (const Primitives::Sample& sample : samples) { shards_inner_series[shard_id_]->emplace_back(sample, check_result.ls_id); } @@ -516,11 +529,11 @@ class PerShardRelabeler { } case Cache::CheckResult::kRelabel: { auto& samples = timeseries_buf_.samples(); - bool all_samples_reseted_to_scrape_ts = resolve_timestamps(def_timestamp, samples, o); + const bool all_samples_reseted_to_scrape_ts = resolve_timestamps(def_timestamp, samples, o); if (o.track_timestamps_staleness || all_samples_reseted_to_scrape_ts) { stale_nan_state.add_input(check_result.source_ls_id); } - for (const PromPP::Primitives::Sample& sample : samples) { + for (const Primitives::Sample& sample : samples) { shards_inner_series[check_result.shard_id]->emplace_back(sample, check_result.ls_id); } @@ -530,7 +543,7 @@ class PerShardRelabeler { continue; } - stats.samples_added += static_cast(timeseries_buf_.samples().size()); + stats.samples_added += timeseries_buf_.samples().size(); if (o.metric_limits == nullptr) { continue; @@ -542,15 +555,15 @@ class PerShardRelabeler { } } - PromPP::Primitives::Sample smpl{def_timestamp, kStaleNan}; + const Primitives::Sample smpl{def_timestamp, kStaleNan}; stale_nan_state.swap( [&](uint32_t ls_id) { - if (auto res = cache.check_input(ls_id); res.status == Cache::CheckResult::kRelabel) { + if (const auto res = cache.check_input(ls_id); res.status == Cache::CheckResult::kRelabel) { shards_inner_series[res.shard_id]->emplace_back(smpl, res.ls_id); } }, [&](uint32_t ls_id) { - if (auto res = cache.check_target(ls_id); res.status == Cache::CheckResult::kKeep) { + if (const auto res = cache.check_target(ls_id); res.status == Cache::CheckResult::kKeep) { shards_inner_series[shard_id_]->emplace_back(smpl, res.ls_id); } }); @@ -564,18 +577,19 @@ class PerShardRelabeler { const Hashdex& hashdex, const RelabelerOptions& o, Stats& stats, - PromPP::Primitives::Go::SliceView& shards_inner_series, + Primitives::Go::SliceView& shards_inner_series, StNaNsState& stale_nan_state, - PromPP::Primitives::Timestamp def_timestamp) { + Primitives::Timestamp def_timestamp) { assert(number_of_shards_ > 0); - size_t n = std::min(static_cast(hashdex.size()), static_cast((hashdex.size() * (1 - cache.part_of_drops()) * 1.1) / number_of_shards_)); - for (auto i = 0; i < number_of_shards_; ++i) { + const size_t n = + std::min(static_cast(hashdex.size()), static_cast((hashdex.size() * (1 - cache.part_of_drops()) * 1.1) / number_of_shards_)); + for (uint16_t i = 0; i < number_of_shards_; ++i) { shards_inner_series[i]->reserve(n); } size_t samples_count{0}; - PromPP::Primitives::TimeseriesSemiview timeseries_buf; + Primitives::TimeseriesSemiview timeseries_buf; for (const auto& item : hashdex) { if ((item.hash() % number_of_shards_) != shard_id_) { @@ -584,18 +598,18 @@ class PerShardRelabeler { timeseries_buf.clear(); item.read(timeseries_buf); - Cache::CheckResult check_result = cache.check(input_lss, target_lss, timeseries_buf.label_set(), item.hash()); + const auto check_result = cache.check(input_lss, target_lss, timeseries_buf.label_set(), item.hash()); switch (check_result.status) { case Cache::CheckResult::kNotFound: { return false; }; case Cache::CheckResult::kKeep: { auto& samples = timeseries_buf.samples(); - bool all_samples_reseted_to_scrape_ts = resolve_timestamps(def_timestamp, samples, o); + const bool all_samples_reseted_to_scrape_ts = resolve_timestamps(def_timestamp, samples, o); if (o.track_timestamps_staleness || all_samples_reseted_to_scrape_ts) { stale_nan_state.add_target(check_result.ls_id); } - for (const PromPP::Primitives::Sample& sample : samples) { + for (const Primitives::Sample& sample : samples) { shards_inner_series[shard_id_]->emplace_back(sample, check_result.ls_id); } @@ -603,11 +617,11 @@ class PerShardRelabeler { } case Cache::CheckResult::kRelabel: { auto& samples = timeseries_buf.samples(); - bool all_samples_reseted_to_scrape_ts = resolve_timestamps(def_timestamp, samples, o); + const bool all_samples_reseted_to_scrape_ts = resolve_timestamps(def_timestamp, samples, o); if (o.track_timestamps_staleness || all_samples_reseted_to_scrape_ts) { stale_nan_state.add_input(check_result.source_ls_id); } - for (const PromPP::Primitives::Sample& sample : samples) { + for (const Primitives::Sample& sample : samples) { shards_inner_series[check_result.shard_id]->emplace_back(sample, check_result.ls_id); } @@ -617,7 +631,7 @@ class PerShardRelabeler { continue; } - stats.samples_added += static_cast(timeseries_buf.samples().size()); + stats.samples_added += timeseries_buf.samples().size(); if (o.metric_limits == nullptr) { continue; @@ -629,15 +643,15 @@ class PerShardRelabeler { } } - PromPP::Primitives::Sample smpl{def_timestamp, kStaleNan}; + const Primitives::Sample smpl{def_timestamp, kStaleNan}; stale_nan_state.swap( [&](uint32_t ls_id) { - if (auto res = cache.check_input(ls_id); res.status == Cache::CheckResult::kRelabel) { + if (const auto res = cache.check_input(ls_id); res.status == Cache::CheckResult::kRelabel) { shards_inner_series[res.shard_id]->emplace_back(smpl, res.ls_id); } }, [&](uint32_t ls_id) { - if (auto res = cache.check_target(ls_id); res.status == Cache::CheckResult::kKeep) { + if (const auto res = cache.check_target(ls_id); res.status == Cache::CheckResult::kKeep) { shards_inner_series[shard_id_]->emplace_back(smpl, res.ls_id); } }); @@ -647,7 +661,7 @@ class PerShardRelabeler { template PROMPP_ALWAYS_INLINE relabelStatus relabel(const RelabelerOptions& o, LabelsBuilder& builder) { - bool changed = inject_target_labels(builder, o); + const bool changed = inject_target_labels(builder, o); relabelStatus rstatus = stateless_relabeler_->relabeling_process(buf_, builder); hard_validate(rstatus, builder, o.metric_limits); @@ -659,7 +673,7 @@ class PerShardRelabeler { } // calculate_samples counts the number of samples excluding stale_nan. - PROMPP_ALWAYS_INLINE size_t calculate_samples(const BareBones::Vector& samples) noexcept { + PROMPP_ALWAYS_INLINE static size_t calculate_samples(const BareBones::Vector& samples) noexcept { size_t samples_count{0}; for (const auto smpl : samples) { if (is_stale_nan(smpl.value())) { @@ -692,9 +706,9 @@ class PerShardRelabeler { return changed; } - std::vector conflicting_exposed_labels; + std::vector conflicting_exposed_labels; for (const auto& [lname, lvalue] : o.target_labels) { - PromPP::Primitives::Label existing_label = target_builder.extract(static_cast(lname)); + Primitives::Label existing_label = target_builder.extract(static_cast(lname)); if (!existing_label.second.empty()) [[likely]] { conflicting_exposed_labels.emplace_back(std::move(existing_label)); } @@ -714,9 +728,9 @@ class PerShardRelabeler { // resolve_conflicting_exposed_labels add prefix to conflicting label name. template - PROMPP_ALWAYS_INLINE void resolve_conflicting_exposed_labels(LabelsBuilder& builder, std::vector& conflicting_exposed_labels) { + PROMPP_ALWAYS_INLINE void resolve_conflicting_exposed_labels(LabelsBuilder& builder, std::vector& conflicting_exposed_labels) { std::stable_sort(conflicting_exposed_labels.begin(), conflicting_exposed_labels.end(), - [](PromPP::Primitives::LabelView a, PromPP::Primitives::LabelView b) { return a.first.size() < b.first.size(); }); + [](const Primitives::LabelView& a, const Primitives::LabelView& b) { return a.first.size() < b.first.size(); }); for (auto& [ln, lv] : conflicting_exposed_labels) { while (true) { @@ -736,11 +750,10 @@ class PerShardRelabeler { const Hashdex& hashdex, const RelabelerOptions& o, Stats& stats, - PromPP::Primitives::Go::SliceView& shards_inner_series, - PromPP::Primitives::Go::SliceView& shards_relabeled_series) { + Primitives::Go::SliceView& shards_inner_series, + Primitives::Go::SliceView& shards_relabeled_series) { NoOpStaleNaNsState state{}; - input_relabeling_internal(input_lss, target_lss, cache, hashdex, o, stats, shards_inner_series, shards_relabeled_series, state, - PromPP::Primitives::kNullTimestamp); + input_relabeling_internal(input_lss, target_lss, cache, hashdex, o, stats, shards_inner_series, shards_relabeled_series, state, Primitives::kNullTimestamp); } template @@ -750,10 +763,10 @@ class PerShardRelabeler { const Hashdex& hashdex, const RelabelerOptions& o, Stats& stats, - PromPP::Primitives::Go::SliceView& shards_inner_series, - PromPP::Primitives::Go::SliceView& shards_relabeled_series, + Primitives::Go::SliceView& shards_inner_series, + Primitives::Go::SliceView& shards_relabeled_series, StaleNaNsState& state, - PromPP::Primitives::Timestamp def_timestamp) { + Primitives::Timestamp def_timestamp) { input_relabeling_internal(input_lss, target_lss, cache, hashdex, o, stats, shards_inner_series, shards_relabeled_series, state, def_timestamp); } @@ -764,10 +777,9 @@ class PerShardRelabeler { const Hashdex& hashdex, const RelabelerOptions& o, Stats& stats, - PromPP::Primitives::Go::SliceView& shards_inner_series) { + Primitives::Go::SliceView& shards_inner_series) { NoOpStaleNaNsState state{}; - return input_relabeling_from_cache_internal(input_lss, target_lss, cache, hashdex, o, stats, shards_inner_series, state, - PromPP::Primitives::kNullTimestamp); + return input_relabeling_from_cache_internal(input_lss, target_lss, cache, hashdex, o, stats, shards_inner_series, state, Primitives::kNullTimestamp); } template @@ -777,25 +789,25 @@ class PerShardRelabeler { const Hashdex& hashdex, const RelabelerOptions& o, Stats& stats, - PromPP::Primitives::Go::SliceView& shards_inner_series, + Primitives::Go::SliceView& shards_inner_series, StaleNaNsState& state, - PromPP::Primitives::Timestamp def_timestamp) { + Primitives::Timestamp def_timestamp) { return input_relabeling_from_cache_internal(input_lss, target_lss, cache, hashdex, o, stats, shards_inner_series, state, def_timestamp); } PROMPP_ALWAYS_INLINE void input_collect_stalenans(Cache& cache, - PromPP::Primitives::Go::SliceView& shards_inner_series, + Primitives::Go::SliceView& shards_inner_series, StaleNaNsState& state, - PromPP::Primitives::Timestamp stale_ts) { - PromPP::Primitives::Sample smpl{stale_ts, kStaleNan}; + Primitives::Timestamp stale_ts) const { + const Primitives::Sample smpl{stale_ts, kStaleNan}; state.swap( [&](uint32_t ls_id) { - if (auto res = cache.check_input(ls_id); res.status == Cache::CheckResult::kRelabel) { + if (const auto res = cache.check_input(ls_id); res.status == Cache::CheckResult::kRelabel) { shards_inner_series[res.shard_id]->emplace_back(smpl, res.ls_id); } }, [&](uint32_t ls_id) { - if (auto res = cache.check_target(ls_id); res.status == Cache::CheckResult::kKeep) { + if (const auto res = cache.check_target(ls_id); res.status == Cache::CheckResult::kKeep) { shards_inner_series[shard_id_]->emplace_back(smpl, res.ls_id); } }); @@ -804,10 +816,10 @@ class PerShardRelabeler { // append_relabeler_series add relabeled ls to lss, add to result and add to cache update(second stage). template - PROMPP_ALWAYS_INLINE void append_relabeler_series(LSS& lss, - InnerSeries* inner_series, - const RelabeledSeries* relabeled_series, - RelabelerStateUpdate* relabeler_state_update) { + PROMPP_ALWAYS_INLINE static void append_relabeler_series(LSS& lss, + InnerSeries* inner_series, + const RelabeledSeries* relabeled_series, + RelabelerStateUpdate* relabeler_state_update) { relabeler_state_update->reserve(relabeler_state_update->size() + relabeled_series->size()); inner_series->reserve(inner_series->size() + relabeled_series->size()); if constexpr (BareBones::concepts::has_reserve) { @@ -825,7 +837,7 @@ class PerShardRelabeler { } // update_relabeler_state - add to cache relabled data(third stage). - PROMPP_ALWAYS_INLINE void update_relabeler_state(Cache& cache, const RelabelerStateUpdate* relabeler_state_update, const uint16_t relabeled_shard_id) { + PROMPP_ALWAYS_INLINE static void update_relabeler_state(Cache& cache, const RelabelerStateUpdate* relabeler_state_update, const uint16_t relabeled_shard_id) { for (const auto& update : *relabeler_state_update) { cache.add_relabel(update.incoming_ls_id, update.relabeled_ls_id, relabeled_shard_id); } @@ -836,18 +848,15 @@ class PerShardRelabeler { PROMPP_ALWAYS_INLINE void output_relabeling(const LSS& lss, Cache& cache, RelabeledSeries* relabeled_series, - PromPP::Primitives::Go::SliceView& incoming_inner_series, - PromPP::Primitives::Go::SliceView& encoders_inner_series) { - std::ranges::for_each(incoming_inner_series, [&](const InnerSeries* inner_series) PROMPP_LAMBDA_INLINE { + Primitives::Go::SliceView& incoming_inner_series, + Primitives::Go::SliceView& encoders_inner_series) { + for (const auto inner_series : incoming_inner_series) { if (inner_series == nullptr || inner_series->size() == 0) { - return; + continue; } - // TODO move ctor builder from ranges for; - PromPP::Primitives::LabelsBuilder builder{builder_state_}; - std::ranges::for_each(inner_series->data(), [&](const InnerSerie& inner_serie) PROMPP_LAMBDA_INLINE { - auto res = cache.check_input(inner_serie.ls_id); + const auto res = cache.check_input(inner_serie.ls_id); if (res.status == Cache::CheckResult::kDrop) { return; } @@ -860,30 +869,27 @@ class PerShardRelabeler { if (inner_serie.ls_id >= lss.size()) [[unlikely]] { throw BareBones::Exception(0x7763a97e1717e835, "ls_id out of range: %d size: %d shard_id: %d", inner_serie.ls_id, lss.size(), shard_id_); } - typename LSS::value_type labels = lss[inner_serie.ls_id]; - builder.reset(labels); - process_external_labels(builder, external_labels_); + builder_.reset(lss[inner_serie.ls_id]); + process_external_labels(builder_, external_labels_); - relabelStatus rstatus = stateless_relabeler_->relabeling_process(buf_, builder); - soft_validate(rstatus, builder); + relabelStatus rstatus = stateless_relabeler_->relabeling_process(buf_, builder_); + soft_validate(rstatus, builder_); if (rstatus == rsDrop) { cache.add_drop(inner_serie.ls_id); return; } - PromPP::Primitives::LabelSet new_label_set = builder.label_set(); - relabeled_series->emplace_back(new_label_set, BareBones::Vector{inner_serie.sample}, hash_value(new_label_set), - inner_serie.ls_id); + const auto& new_label_set = builder_.label_set(); + relabeled_series->emplace_back(new_label_set, BareBones::Vector{inner_serie.sample}, hash_value(new_label_set), inner_serie.ls_id); }); - }); + } cache.optimize(); } // reset set new number_of_shards and external_labels. - PROMPP_ALWAYS_INLINE void reset_to( - const PromPP::Primitives::Go::SliceView>& external_labels, - const uint16_t number_of_shards) { + PROMPP_ALWAYS_INLINE void reset_to(const Primitives::Go::SliceView>& external_labels, + const uint16_t number_of_shards) { number_of_shards_ = number_of_shards; external_labels_.clear(); external_labels_.reserve(external_labels.size()); @@ -891,8 +897,6 @@ class PerShardRelabeler { external_labels_.emplace_back(static_cast(ln), static_cast(lv)); } } - - PROMPP_ALWAYS_INLINE ~PerShardRelabeler() = default; }; // @@ -901,9 +905,9 @@ class PerShardRelabeler { // PerGoroutineRelabeler stateful relabeler for shard goroutines. class PerGoroutineRelabeler { - std::stringstream buf_; - PromPP::Primitives::LabelsBuilderStateMap builder_state_; - PromPP::Primitives::TimeseriesSemiview timeseries_buf_; + std::string buf_; + Primitives::LabelsBuilder builder_; + Primitives::TimeseriesSemiview timeseries_buf_; uint16_t number_of_shards_; uint16_t shard_id_; @@ -913,28 +917,8 @@ class PerGoroutineRelabeler { : number_of_shards_(number_of_shards), shard_id_(shard_id) {} private: - // calculate_samples counts the number of samples excluding stale_nan. - PROMPP_ALWAYS_INLINE size_t calculate_samples(const BareBones::Vector& samples) noexcept { - size_t samples_count{0}; - for (const auto smpl : samples) { - if (is_stale_nan(smpl.value())) { - continue; - } - - ++samples_count; - } - - return samples_count; - } - - // check_target_lss check label_set in target lss. - template - PROMPP_ALWAYS_INLINE Cache::CheckResult check_target_lss(const TargetLSS& target_lss, LabelSet& label_set, size_t hash) { - if (std::optional ls_id = target_lss.find(label_set, hash); ls_id.has_value()) { - return {.status = Cache::CheckResult::Status::kKeep, .ls_id = ls_id.value()}; - } - - return {}; + PROMPP_ALWAYS_INLINE static size_t non_stale_nan_samples_count(const BareBones::Vector& samples) noexcept { + return std::ranges::count_if(samples, [](const Primitives::Sample& sample) { return !is_stale_nan(sample.value()); }); } // inject_target_labels add labels from target to builder. @@ -959,9 +943,9 @@ class PerGoroutineRelabeler { return changed; } - std::vector conflicting_exposed_labels; + std::vector conflicting_exposed_labels; for (const auto& [lname, lvalue] : options.target_labels) { - PromPP::Primitives::Label existing_label = target_builder.extract(static_cast(lname)); + Primitives::Label existing_label = target_builder.extract(static_cast(lname)); if (!existing_label.second.empty()) [[likely]] { conflicting_exposed_labels.emplace_back(std::move(existing_label)); } @@ -986,84 +970,63 @@ class PerGoroutineRelabeler { const Hashdex& hashdex, const RelabelerOptions& options, Stats& stats, - PromPP::Primitives::Go::SliceView& shards_inner_series, + Primitives::Go::SliceView& shards_inner_series, StNaNsState& stale_nan_state, - PromPP::Primitives::Timestamp def_timestamp) { - assert(number_of_shards_ > 0); - - size_t n = std::min(static_cast(hashdex.size()), static_cast((hashdex.size() * (1 - cache.part_of_drops()) * 1.1) / number_of_shards_)); - for (auto i = 0; i < number_of_shards_; ++i) { - shards_inner_series[i]->reserve(n); - } - - size_t samples_count{0}; - - for (const auto& item : hashdex) { - if ((item.hash() % number_of_shards_) != shard_id_) { - continue; - } - - timeseries_buf_.clear(); - item.read(timeseries_buf_); + Primitives::Timestamp def_timestamp) { + bool result{true}; + size_t samples_count{}; + fill_inner_series(hashdex, hashdex.begin(), shards_inner_series, [&](auto& item) { Cache::CheckResult check_result = cache.check(input_lss, target_lss, timeseries_buf_.label_set(), item.hash()); switch (check_result.status) { case Cache::CheckResult::kNotFound: { + result = false; return false; }; + case Cache::CheckResult::kKeep: { auto& samples = timeseries_buf_.samples(); - bool all_samples_reseted_to_scrape_ts = resolve_timestamps(def_timestamp, samples, options); + const bool all_samples_reseted_to_scrape_ts = resolve_timestamps(def_timestamp, samples, options); if (options.track_timestamps_staleness || all_samples_reseted_to_scrape_ts) { stale_nan_state.add_target(check_result.ls_id); } - for (const PromPP::Primitives::Sample& sample : samples) { - shards_inner_series[shard_id_]->emplace_back(sample, check_result.ls_id); - } + shards_inner_series[shard_id_]->emplace_back(samples, check_result.ls_id); break; } + case Cache::CheckResult::kRelabel: { auto& samples = timeseries_buf_.samples(); - bool all_samples_reseted_to_scrape_ts = resolve_timestamps(def_timestamp, samples, options); + const bool all_samples_reseted_to_scrape_ts = resolve_timestamps(def_timestamp, samples, options); if (options.track_timestamps_staleness || all_samples_reseted_to_scrape_ts) { stale_nan_state.add_input(check_result.source_ls_id); } - for (const PromPP::Primitives::Sample& sample : samples) { - shards_inner_series[check_result.shard_id]->emplace_back(sample, check_result.ls_id); - } + shards_inner_series[check_result.shard_id]->emplace_back(samples, check_result.ls_id); break; } - default: - continue; + + default: { + return true; + } } - stats.samples_added += static_cast(timeseries_buf_.samples().size()); + stats.samples_added += timeseries_buf_.samples().size(); - if (options.metric_limits == nullptr) { - continue; + if (options.metric_limits != nullptr) { + samples_count += non_stale_nan_samples_count(timeseries_buf_.samples()); + if (options.metric_limits->samples_limit_exceeded(samples_count)) { + return false; + } } - samples_count += calculate_samples(timeseries_buf_.samples()); - if (options.metric_limits->samples_limit_exceeded(samples_count)) { - break; - } - } + return true; + }); - PromPP::Primitives::Sample smpl{def_timestamp, kStaleNan}; - stale_nan_state.swap( - [&](uint32_t ls_id) { - if (auto res = cache.check_input(ls_id); res.status == Cache::CheckResult::kRelabel) { - shards_inner_series[res.shard_id]->emplace_back(smpl, res.ls_id); - } - }, - [&](uint32_t ls_id) { - if (auto res = cache.check_target(ls_id); res.status == Cache::CheckResult::kKeep) { - shards_inner_series[shard_id_]->emplace_back(smpl, res.ls_id); - } - }); + if (result) { + add_stale_nans(cache, shards_inner_series, stale_nan_state, def_timestamp); + } - return true; + return result; } template @@ -1074,74 +1037,46 @@ class PerGoroutineRelabeler { const RelabelerOptions& options, const StatelessRelabeler& stateless_relabeler, Stats& stats, - PromPP::Primitives::Go::SliceView& shards_inner_series, - PromPP::Primitives::Go::SliceView& shards_relabeled_series, + Primitives::Go::SliceView& shards_inner_series, + Primitives::Go::SliceView& shards_relabeled_series, StNaNsState& stale_nan_state, - PromPP::Primitives::Timestamp def_timestamp) { - assert(number_of_shards_ > 0); - - size_t n = std::min(static_cast(hashdex.size()), static_cast((hashdex.size() * (1 - cache.part_of_drops()) * 1.1) / number_of_shards_)); - for (auto i = 0; i < number_of_shards_; ++i) { - if (shards_inner_series[i]->size() >= n) { - continue; - } - - shards_inner_series[i]->reserve(n); - } - - PromPP::Primitives::LabelsBuilder builder{builder_state_}; - size_t samples_count{0}; - - for (auto it = skip_shard_inner_series(hashdex, shards_inner_series[shard_id_]->size()); it != hashdex.end(); ++it) { - if ((it->hash() % number_of_shards_) != shard_id_) { - continue; - } - - timeseries_buf_.clear(); - it->read(timeseries_buf_); - - Cache::CheckResult check_result = cache.check(input_lss, target_lss, timeseries_buf_.label_set(), it->hash()); + Primitives::Timestamp def_timestamp) { + size_t samples_count{}; + fill_inner_series(hashdex, skip_shard_inner_series(hashdex, shards_inner_series[shard_id_]->size()), shards_inner_series, [&](auto& item) { + Cache::CheckResult check_result = cache.check(input_lss, target_lss, timeseries_buf_.label_set(), item.hash()); switch (check_result.status) { case Cache::CheckResult::kNotFound: { - builder.reset(timeseries_buf_.label_set()); - auto rstatus = relabel(options, stateless_relabeler, builder); - switch (rstatus) { - case rsDrop: { - cache.add_drop(input_lss.find_or_emplace(timeseries_buf_.label_set(), it->hash())); - ++stats.series_drop; - - continue; - } + builder_.reset(timeseries_buf_.label_set()); + switch (relabel(options, stateless_relabeler, builder_)) { + case rsDrop: case rsInvalid: { - cache.add_drop(input_lss.find_or_emplace(timeseries_buf_.label_set(), it->hash())); + cache.add_drop(input_lss.find_or_emplace(timeseries_buf_.label_set(), item.hash())); ++stats.series_drop; - - continue; + return true; } + case rsKeep: { - auto ls_id = target_lss.find_or_emplace(timeseries_buf_.label_set(), it->hash()); + auto ls_id = target_lss.find_or_emplace(timeseries_buf_.label_set(), item.hash()); cache.add_keep(ls_id); auto& samples = timeseries_buf_.samples(); - bool all_samples_reseted_to_scrape_ts = resolve_timestamps(def_timestamp, samples, options); + const bool all_samples_reseted_to_scrape_ts = resolve_timestamps(def_timestamp, samples, options); if (options.track_timestamps_staleness || all_samples_reseted_to_scrape_ts) { stale_nan_state.add_target(ls_id); } - for (const PromPP::Primitives::Sample& sample : samples) { - shards_inner_series[shard_id_]->emplace_back(sample, ls_id); - } - + shards_inner_series[shard_id_]->emplace_back(samples, ls_id); ++stats.series_added; break; } + case rsRelabel: { - auto ls_id = input_lss.find_or_emplace(timeseries_buf_.label_set(), it->hash()); - PromPP::Primitives::LabelSet new_label_set = builder.label_set(); + auto ls_id = input_lss.find_or_emplace(timeseries_buf_.label_set(), item.hash()); + const auto& new_label_set = builder_.label_set(); size_t new_hash = hash_value(new_label_set); - size_t new_shard_id = new_hash % number_of_shards_; + const size_t new_shard_id = new_hash % number_of_shards_; auto& samples = timeseries_buf_.samples(); - bool all_samples_reseted_to_scrape_ts = resolve_timestamps(def_timestamp, samples, options); + const bool all_samples_reseted_to_scrape_ts = resolve_timestamps(def_timestamp, samples, options); if (options.track_timestamps_staleness || all_samples_reseted_to_scrape_ts) { stale_nan_state.add_input(ls_id); } @@ -1155,65 +1090,53 @@ class PerGoroutineRelabeler { break; } + case Cache::CheckResult::kKeep: { auto& samples = timeseries_buf_.samples(); - bool all_samples_reseted_to_scrape_ts = resolve_timestamps(def_timestamp, samples, options); + const bool all_samples_reseted_to_scrape_ts = resolve_timestamps(def_timestamp, samples, options); if (options.track_timestamps_staleness || all_samples_reseted_to_scrape_ts) { stale_nan_state.add_target(check_result.ls_id); } - for (const PromPP::Primitives::Sample& sample : samples) { - shards_inner_series[shard_id_]->emplace_back(sample, check_result.ls_id); - } - + shards_inner_series[shard_id_]->emplace_back(samples, check_result.ls_id); break; } + case Cache::CheckResult::kRelabel: { auto& samples = timeseries_buf_.samples(); - bool all_samples_reseted_to_scrape_ts = resolve_timestamps(def_timestamp, samples, options); + const bool all_samples_reseted_to_scrape_ts = resolve_timestamps(def_timestamp, samples, options); if (options.track_timestamps_staleness || all_samples_reseted_to_scrape_ts) { stale_nan_state.add_input(check_result.source_ls_id); } - for (const PromPP::Primitives::Sample& sample : samples) { - shards_inner_series[check_result.shard_id]->emplace_back(sample, check_result.ls_id); - } + shards_inner_series[check_result.shard_id]->emplace_back(samples, check_result.ls_id); + break; + } + + default: { break; } - default: - continue; } - stats.samples_added += static_cast(timeseries_buf_.samples().size()); + stats.samples_added += timeseries_buf_.samples().size(); - if (options.metric_limits == nullptr) { - continue; + if (options.metric_limits != nullptr) { + samples_count += non_stale_nan_samples_count(timeseries_buf_.samples()); + if (options.metric_limits->samples_limit_exceeded(samples_count)) { + return false; + } } - samples_count += calculate_samples(timeseries_buf_.samples()); - if (options.metric_limits->samples_limit_exceeded(samples_count)) { - break; - } - } + return true; + }); - PromPP::Primitives::Sample smpl{def_timestamp, kStaleNan}; - stale_nan_state.swap( - [&](uint32_t ls_id) { - if (auto res = cache.check_input(ls_id); res.status == Cache::CheckResult::kRelabel) { - shards_inner_series[res.shard_id]->emplace_back(smpl, res.ls_id); - } - }, - [&](uint32_t ls_id) { - if (auto res = cache.check_target(ls_id); res.status == Cache::CheckResult::kKeep) { - shards_inner_series[shard_id_]->emplace_back(smpl, res.ls_id); - } - }); + add_stale_nans(cache, shards_inner_series, stale_nan_state, def_timestamp); cache.optimize(); } template PROMPP_ALWAYS_INLINE relabelStatus relabel(const RelabelerOptions& options, const StatelessRelabeler& stateless_relabeler, LabelsBuilder& builder) { - bool changed = inject_target_labels(builder, options); + const bool changed = inject_target_labels(builder, options); relabelStatus rstatus = stateless_relabeler.relabeling_process(buf_, builder); hard_validate(rstatus, builder, options.metric_limits); @@ -1226,9 +1149,9 @@ class PerGoroutineRelabeler { // resolve_conflicting_exposed_labels add prefix to conflicting label name. template - PROMPP_ALWAYS_INLINE void resolve_conflicting_exposed_labels(LabelsBuilder& builder, std::vector& conflicting_exposed_labels) { + PROMPP_ALWAYS_INLINE void resolve_conflicting_exposed_labels(LabelsBuilder& builder, std::vector& conflicting_exposed_labels) { std::stable_sort(conflicting_exposed_labels.begin(), conflicting_exposed_labels.end(), - [](PromPP::Primitives::LabelView a, PromPP::Primitives::LabelView b) { return a.first.size() < b.first.size(); }); + [](const Primitives::LabelView& a, const Primitives::LabelView& b) { return a.first.size() < b.first.size(); }); for (auto& [ln, lv] : conflicting_exposed_labels) { while (true) { @@ -1241,18 +1164,18 @@ class PerGoroutineRelabeler { } } - PROMPP_ALWAYS_INLINE bool resolve_timestamps(PromPP::Primitives::Timestamp def_timestamp, - BareBones::Vector& samples, - const RelabelerOptions& options) { + PROMPP_ALWAYS_INLINE static bool resolve_timestamps(Primitives::Timestamp def_timestamp, + BareBones::Vector& samples, + const RelabelerOptions& options) { // skip resolve without stalenans - if (def_timestamp == PromPP::Primitives::kNullTimestamp) { + if (def_timestamp == Primitives::kNullTimestamp) { return false; } bool track_staleness{true}; for (auto& sample : samples) { // replace null timestamp on def timestamp - if (sample.timestamp() == PromPP::Primitives::kNullTimestamp) { + if (sample.timestamp() == Primitives::kNullTimestamp) { sample.timestamp() = def_timestamp; continue; } @@ -1292,11 +1215,11 @@ class PerGoroutineRelabeler { const RelabelerOptions& options, const StatelessRelabeler& stateless_relabeler, Stats& stats, - PromPP::Primitives::Go::SliceView& shards_inner_series, - PromPP::Primitives::Go::SliceView& shards_relabeled_series) { + Primitives::Go::SliceView& shards_inner_series, + Primitives::Go::SliceView& shards_relabeled_series) { NoOpStaleNaNsState state{}; input_relabeling_internal(input_lss, target_lss, cache, hashdex, options, stateless_relabeler, stats, shards_inner_series, shards_relabeled_series, state, - PromPP::Primitives::kNullTimestamp); + Primitives::kNullTimestamp); } template @@ -1306,10 +1229,9 @@ class PerGoroutineRelabeler { const Hashdex& hashdex, const RelabelerOptions& options, Stats& stats, - PromPP::Primitives::Go::SliceView& shards_inner_series) { + Primitives::Go::SliceView& shards_inner_series) { NoOpStaleNaNsState state{}; - return input_relabeling_from_cache_internal(input_lss, target_lss, cache, hashdex, options, stats, shards_inner_series, state, - PromPP::Primitives::kNullTimestamp); + return input_relabeling_from_cache_internal(input_lss, target_lss, cache, hashdex, options, stats, shards_inner_series, state, Primitives::kNullTimestamp); } template @@ -1320,10 +1242,10 @@ class PerGoroutineRelabeler { const RelabelerOptions& options, const StatelessRelabeler& stateless_relabeler, Stats& stats, - PromPP::Primitives::Go::SliceView& shards_inner_series, - PromPP::Primitives::Go::SliceView& shards_relabeled_series, + Primitives::Go::SliceView& shards_inner_series, + Primitives::Go::SliceView& shards_relabeled_series, StaleNaNsState& state, - PromPP::Primitives::Timestamp def_timestamp) { + Primitives::Timestamp def_timestamp) { input_relabeling_internal(input_lss, target_lss, cache, hashdex, options, stateless_relabeler, stats, shards_inner_series, shards_relabeled_series, state, def_timestamp); } @@ -1335,9 +1257,9 @@ class PerGoroutineRelabeler { const Hashdex& hashdex, const RelabelerOptions& options, Stats& stats, - PromPP::Primitives::Go::SliceView& shards_inner_series, + Primitives::Go::SliceView& shards_inner_series, StaleNaNsState& state, - PromPP::Primitives::Timestamp def_timestamp) { + Primitives::Timestamp def_timestamp) { return input_relabeling_from_cache_internal(input_lss, target_lss, cache, hashdex, options, stats, shards_inner_series, state, def_timestamp); } @@ -1346,58 +1268,18 @@ class PerGoroutineRelabeler { PROMPP_ALWAYS_INLINE void input_transition_relabeling(TargetLSS& target_lss, const Hashdex& hashdex, Stats& stats, - PromPP::Primitives::Go::SliceView& shards_inner_series) { - assert(number_of_shards_ > 0); - - size_t n = std::min(static_cast(hashdex.size()), static_cast((hashdex.size() * 1.1) / number_of_shards_)); - for (auto i = 0; i < number_of_shards_; ++i) { - if (shards_inner_series[i]->size() >= n) { - continue; - } - - shards_inner_series[i]->reserve(n); - } - - PromPP::Primitives::LabelsBuilder builder{builder_state_}; - - for (auto it = skip_shard_inner_series(hashdex, shards_inner_series[shard_id_]->size()); it != hashdex.end(); ++it) { - if ((it->hash() % number_of_shards_) != shard_id_) { - continue; - } - - timeseries_buf_.clear(); - it->read(timeseries_buf_); - - Cache::CheckResult check_result = check_target_lss(target_lss, timeseries_buf_.label_set(), it->hash()); - switch (check_result.status) { - case Cache::CheckResult::kNotFound: { - builder.reset(timeseries_buf_.label_set()); - auto ls_id = target_lss.find_or_emplace(timeseries_buf_.label_set(), it->hash()); - auto& samples = timeseries_buf_.samples(); - for (const PromPP::Primitives::Sample& sample : samples) { - shards_inner_series[shard_id_]->emplace_back(sample, ls_id); - } - - ++stats.series_added; - - break; - } - - case Cache::CheckResult::kKeep: { - auto& samples = timeseries_buf_.samples(); - for (const PromPP::Primitives::Sample& sample : samples) { - shards_inner_series[shard_id_]->emplace_back(sample, check_result.ls_id); - } - - break; - } - - default: - continue; + Primitives::Go::SliceView& shards_inner_series) { + fill_inner_series(hashdex, skip_shard_inner_series(hashdex, shards_inner_series[shard_id_]->size()), shards_inner_series, [&](auto& item) { + const auto previous_size = target_lss.size(); + auto ls_id = target_lss.find_or_emplace(timeseries_buf_.label_set(), item.hash()); + shards_inner_series[shard_id_]->emplace_back(timeseries_buf_.samples(), ls_id); + + if (target_lss.size() > previous_size) { + ++stats.series_added; } - - stats.samples_added += static_cast(timeseries_buf_.samples().size()); - } + stats.samples_added += timeseries_buf_.samples().size(); + return true; + }); } // input_transition_relabeling_from_cache transparent relabeling with only reading from the lss. @@ -1405,54 +1287,29 @@ class PerGoroutineRelabeler { PROMPP_ALWAYS_INLINE bool input_transition_relabeling_only_read(TargetLSS& target_lss, const Hashdex& hashdex, Stats& stats, - PromPP::Primitives::Go::SliceView& shards_inner_series) { - assert(number_of_shards_ > 0); - - size_t n = std::min(static_cast(hashdex.size()), static_cast((hashdex.size() * 1.1) / number_of_shards_)); - for (auto i = 0; i < number_of_shards_; ++i) { - shards_inner_series[i]->reserve(n); - } - - for (const auto& item : hashdex) { - if ((item.hash() % number_of_shards_) != shard_id_) { - continue; - } - - timeseries_buf_.clear(); - item.read(timeseries_buf_); - - Cache::CheckResult check_result = check_target_lss(target_lss, timeseries_buf_.label_set(), item.hash()); - switch (check_result.status) { - case Cache::CheckResult::kNotFound: { - return false; - }; - - case Cache::CheckResult::kKeep: { - auto& samples = timeseries_buf_.samples(); - for (const PromPP::Primitives::Sample& sample : samples) { - shards_inner_series[shard_id_]->emplace_back(sample, check_result.ls_id); - } - - break; - } - - default: - continue; + Primitives::Go::SliceView& shards_inner_series) { + bool result = true; + fill_inner_series(hashdex, hashdex.begin(), shards_inner_series, [&](auto& item) { + if (auto ls_id = target_lss.find(timeseries_buf_.label_set(), item.hash()); ls_id.has_value()) { + shards_inner_series[shard_id_]->emplace_back(timeseries_buf_.samples(), *ls_id); + stats.samples_added += timeseries_buf_.samples().size(); + return true; } - stats.samples_added += static_cast(timeseries_buf_.samples().size()); - } + result = false; + return false; + }); - return true; + return result; } // second stage // append_relabeler_series add relabeled ls to lss, add to result and add to cache update. template - PROMPP_ALWAYS_INLINE void append_relabeler_series(LSS& target_lss, - InnerSeries* inner_series, - const RelabeledSeries* relabeled_series, - RelabelerStateUpdate* relabeler_state_update) { + PROMPP_ALWAYS_INLINE static void append_relabeler_series(LSS& target_lss, + InnerSeries* inner_series, + const RelabeledSeries* relabeled_series, + RelabelerStateUpdate* relabeler_state_update) { relabeler_state_update->reserve(relabeler_state_update->size() + relabeled_series->size()); inner_series->reserve(inner_series->size() + relabeled_series->size()); if constexpr (BareBones::concepts::has_reserve) { @@ -1461,17 +1318,52 @@ class PerGoroutineRelabeler { for (const auto& relabeled_serie : relabeled_series->data()) { uint32_t ls_id = target_lss.find_or_emplace(relabeled_serie.ls, relabeled_serie.hash); + inner_series->emplace_back(relabeled_serie.samples, ls_id); + relabeler_state_update->emplace_back(relabeled_serie.ls_id, ls_id); + } + } - for (const Primitives::Sample& sample : relabeled_serie.samples) { - inner_series->emplace_back(sample, ls_id); + template + void fill_inner_series(const Hashdex& hashdex, auto hashdex_it, Primitives::Go::SliceView& shards_inner_series, Handler handler) { + assert(number_of_shards_ > 0); + + const size_t n = std::min(static_cast(hashdex.size()), static_cast((hashdex.size() * 1.1) / number_of_shards_)); + for (uint16_t i = 0; i < number_of_shards_; ++i) { + shards_inner_series[i]->reserve(n); + } + + for (; hashdex_it != hashdex.end(); ++hashdex_it) { + if ((hashdex_it->hash() % number_of_shards_) != shard_id_) { + continue; } - relabeler_state_update->emplace_back(relabeled_serie.ls_id, ls_id); + timeseries_buf_.clear(); + hashdex_it->read(timeseries_buf_); + + if (!handler(*hashdex_it)) { + break; + } } } - // destructor. - PROMPP_ALWAYS_INLINE ~PerGoroutineRelabeler() = default; + template + PROMPP_ALWAYS_INLINE void add_stale_nans(Cache& cache, + Primitives::Go::SliceView& shards_inner_series, + StNaNsState& stale_nan_state, + Primitives::Timestamp def_timestamp) { + const Primitives::Sample smpl{def_timestamp, kStaleNan}; + stale_nan_state.swap( + [&](uint32_t ls_id) { + if (const auto res = cache.check_input(ls_id); res.status == Cache::CheckResult::kRelabel) { + shards_inner_series[res.shard_id]->emplace_back(smpl, res.ls_id); + } + }, + [&](uint32_t ls_id) { + if (const auto res = cache.check_target(ls_id); res.status == Cache::CheckResult::kKeep) { + shards_inner_series[shard_id_]->emplace_back(smpl, res.ls_id); + } + }); + } }; } // namespace PromPP::Prometheus::Relabel diff --git a/pp/prometheus/stateless_relabeler.h b/pp/prometheus/stateless_relabeler.h index fa7bdb17c2..885e014109 100644 --- a/pp/prometheus/stateless_relabeler.h +++ b/pp/prometheus/stateless_relabeler.h @@ -1,20 +1,14 @@ #pragma once -#include #include #include #include #include #include -#pragma GCC diagnostic push -#pragma GCC diagnostic ignored "-Wold-style-cast" -#pragma GCC diagnostic ignored "-Wpedantic" -#pragma GCC diagnostic ignored "-Wswitch" -#include "re2/re2.h" -#pragma GCC diagnostic pop #include #include +#include "re2/re2.h" #include "bare_bones/bit.h" #include "bare_bones/exception.h" @@ -24,8 +18,8 @@ namespace PromPP::Prometheus::Relabel { // label_name_is_valid validate label name. -PROMPP_ALWAYS_INLINE bool label_name_is_valid(std::string_view name) { - if (name.size() == 0) { +PROMPP_ALWAYS_INLINE bool label_name_is_valid(const std::string_view& name) { + if (name.empty()) { return false; } @@ -41,13 +35,13 @@ PROMPP_ALWAYS_INLINE bool label_name_is_valid(std::string_view name) { } // label_value_is_valid validate label value. -PROMPP_ALWAYS_INLINE bool label_value_is_valid(std::string_view value) noexcept { +PROMPP_ALWAYS_INLINE bool label_value_is_valid(const std::string_view& value) noexcept { return simdutf::validate_utf8(value.data(), value.length()); } // metric_name_value_is_valid validate value for metric name(__name__). -PROMPP_ALWAYS_INLINE bool metric_name_value_is_valid(std::string_view value) { - if (value.size() == 0) { +PROMPP_ALWAYS_INLINE bool metric_name_value_is_valid(const std::string_view& value) { + if (value.empty()) { return false; } @@ -85,23 +79,13 @@ class PatternPart { PROMPP_ALWAYS_INLINE explicit PatternPart(int g) : type_(pGroup), data_{.group_ = g} {} // write - convert parts to out. - PROMPP_ALWAYS_INLINE void write(std::ostream& out, std::vector& groups) { - if (type_ == pGroup) { - out << groups[data_.group_]; - } else { - out << data_.string_; - } - } - - PROMPP_ALWAYS_INLINE void write(std::ostream& out, std::vector& groups) const { + PROMPP_ALWAYS_INLINE void write(std::string& out, const std::vector& groups) const { if (type_ == pGroup) { - out << groups[data_.group_]; + out += groups[data_.group_]; } else { - out << data_.string_; + out += data_.string_; } } - - ~PatternPart() = default; }; // Regexp - wrapper on re2. @@ -111,19 +95,14 @@ class Regexp { public: // Regexp - work without ("^(?:" + std::string(s) + ")$"). - PROMPP_ALWAYS_INLINE explicit Regexp(const std::string_view& s) noexcept : re_(std::make_unique(std::string(s))) {} - - PROMPP_ALWAYS_INLINE Regexp(Regexp&&) noexcept = default; - PROMPP_ALWAYS_INLINE ~Regexp() = default; + PROMPP_ALWAYS_INLINE explicit Regexp(const std::string_view& s) noexcept : re_(std::make_unique(s)) {} // number_of_capturing_groups - return the number of capturing sub-patterns, or -1 if the regexp wasn't valid on construction. The overall match ($0) does not // count. Use in test. - PROMPP_ALWAYS_INLINE int number_of_capturing_groups() { return re_->NumberOfCapturingGroups(); } - - PROMPP_ALWAYS_INLINE int number_of_capturing_groups() const { return re_->NumberOfCapturingGroups(); } + [[nodiscard]] PROMPP_ALWAYS_INLINE int number_of_capturing_groups() const { return re_->NumberOfCapturingGroups(); } // groups - get named capturing groups and number groups. - PROMPP_ALWAYS_INLINE std::map groups() { + [[nodiscard]] PROMPP_ALWAYS_INLINE std::map groups() const { // get named capturing groups std::map named_groups = re_->NamedCapturingGroups(); // add number groups to named capturing groups @@ -135,8 +114,8 @@ class Regexp { } // match_to_args - match expression and return result args. - PROMPP_ALWAYS_INLINE bool match_to_args(std::string_view src, std::vector& res) { - int n = number_of_capturing_groups(); + PROMPP_ALWAYS_INLINE bool match_to_args(std::string_view src, std::vector& res) const { + const int n = number_of_capturing_groups(); // search full match to args, where size - number of capturing groups res.resize(n + 1); @@ -149,108 +128,65 @@ class Regexp { re_args.emplace_back(&res[i]); re_args_ptr.emplace_back(&re_args[i - 1]); } - bool ok = RE2::FullMatchN(src, *re_.get(), &re_args_ptr[0], n); - return ok; - } - - PROMPP_ALWAYS_INLINE bool match_to_args(std::string_view src, std::vector& res) const { - int n = number_of_capturing_groups(); - // search full match to args, where size - number of capturing groups - res.resize(n + 1); - res[0] = src; - std::vector re_args; - re_args.reserve(n); - std::vector re_args_ptr; - re_args_ptr.reserve(n); - for (int i = 1; i <= n; ++i) { - re_args.emplace_back(&res[i]); - re_args_ptr.emplace_back(&re_args[i - 1]); + if (!RE2::FullMatchN(src, *re_, &re_args_ptr[0], n)) { + res.clear(); + return false; } - bool ok = RE2::FullMatchN(src, *re_.get(), &re_args_ptr[0], n); - return ok; + + return true; } // replace_with_args - replace in template with incoming args. - PROMPP_ALWAYS_INLINE std::string replace_with_args(std::stringstream& buf, std::vector& args, std::vector& tmpl) { - if (tmpl.size() == 0) [[unlikely]] { - // no template or source data - return ""; - } - - buf.str(""); - for (auto& val : tmpl) { - val.write(buf, args); - } - - return buf.str(); - } + PROMPP_ALWAYS_INLINE static void replace_with_args(std::string& buf, const std::vector& args, const std::vector& tmpl) { + buf.clear(); - PROMPP_ALWAYS_INLINE std::string replace_with_args(std::stringstream& buf, std::vector& args, const std::vector& tmpl) const { - if (tmpl.size() == 0) [[unlikely]] { + if (tmpl.empty()) [[unlikely]] { // no template or source data - return ""; + return; } - buf.str(""); for (auto& val : tmpl) { val.write(buf, args); } - - return buf.str(); } // replace_full - find match for source and replace in template. - PROMPP_ALWAYS_INLINE std::string replace_full(std::stringstream& out, std::string_view src, std::vector& tmpl) { - if (src.size() == 0 || tmpl.size() == 0) [[unlikely]] { - // no template or source data - return ""; - } - - std::vector res_args; - bool ok = match_to_args(src, res_args); - if (!ok) { - // no entries in regexp - return ""; - } + PROMPP_ALWAYS_INLINE void replace_full(std::string& out, std::string_view src, const std::vector& tmpl) const { + out.clear(); - return replace_with_args(out, res_args, tmpl); - } - - PROMPP_ALWAYS_INLINE std::string replace_full(std::stringstream& out, std::string_view src, const std::vector& tmpl) const { - if (src.size() == 0 || tmpl.size() == 0) [[unlikely]] { + if (src.empty() || tmpl.empty()) [[unlikely]] { // no template or source data - return ""; + return; } - std::vector res_args; - bool ok = match_to_args(src, res_args); - if (!ok) { + std::vector res_args; + if (!match_to_args(src, res_args)) { // no entries in regexp - return ""; + return; } - return replace_with_args(out, res_args, tmpl); + replace_with_args(out, res_args, tmpl); } // full_match - check text for full match regexp. - PROMPP_ALWAYS_INLINE bool full_match(std::string_view str) const { return RE2::FullMatch(str, *re_.get()); } + [[nodiscard]] PROMPP_ALWAYS_INLINE bool full_match(std::string_view str) const { return RE2::FullMatch(str, *re_); } }; struct GORelabelConfig { // source_labels - a list of labels from which values are taken and concatenated with the configured separator in order. - PromPP::Primitives::Go::SliceView source_labels; + Primitives::Go::SliceView source_labels; // separator - is the string between concatenated values from the source labels. - PromPP::Primitives::Go::String separator; + Primitives::Go::String separator; // regex - against which the concatenation is matched. - PromPP::Primitives::Go::String regex; + Primitives::Go::String regex; // modulus - to take of the hash of concatenated values from the source labels. uint64_t modulus; // target_label - is the label to which the resulting string is written in a replacement. // Regexp interpolation is allowed for the replace action. - PromPP::Primitives::Go::String target_label; + Primitives::Go::String target_label; // replacement - is the regex replacement pattern to be used. - PromPP::Primitives::Go::String replacement; + Primitives::Go::String replacement; // action - is the action to be performed for the relabeling. uint8_t action; }; @@ -318,27 +254,26 @@ class RelabelConfig { std::vector replacement_parts_; // extract - extract from source letter or digit value. - PROMPP_ALWAYS_INLINE std::string extract(re2::RE2& rgx_validate, std::string_view src) { + PROMPP_ALWAYS_INLINE static std::string extract(const re2::RE2& rgx_validate, const std::string_view& src) { std::string name; RE2::PartialMatch(src, rgx_validate, &name); return name; } // is_valid_name - validate source on letter or digit value. - PROMPP_ALWAYS_INLINE bool is_valid_name(re2::RE2& rgx_validate, std::string_view src) { return RE2::FullMatch(src, rgx_validate); } + PROMPP_ALWAYS_INLINE static bool is_valid_name(const re2::RE2& rgx_validate, std::string_view src) { return RE2::FullMatch(src, rgx_validate); } // parse - parse template on parts. - PROMPP_ALWAYS_INLINE void parse(Regexp& regexp, re2::RE2& rgx_validate, std::string_view tmpl, std::vector& src_parts) { + PROMPP_ALWAYS_INLINE static void parse(const Regexp& regexp, const re2::RE2& rgx_validate, std::string_view tmpl, std::vector& src_parts) { std::map groups = regexp.groups(); - std::string_view p = std::string_view(tmpl); + auto p = std::string_view(tmpl); while (true) { - if (p.size() == 0) { + if (p.empty()) { break; } // search '$' and cut before - size_t i = p.find('$'); - std::string_view substr_p = p.substr(0, i); - if (substr_p.size() != 0) { + const size_t i = p.find('$'); + if (std::string_view substr_p = p.substr(0, i); !substr_p.empty()) { src_parts.emplace_back(substr_p); } if (i == std::string_view::npos) { @@ -356,7 +291,7 @@ class RelabelConfig { // if contains '{...}' case '{': { p.remove_prefix(1); - size_t j = p.find('}'); + const size_t j = p.find('}'); if (j == std::string_view::npos) { // if '}' not found cut - "${" src_parts.emplace_back(tmpl.substr(tmpl.size() - p.size() - 2, 2)); @@ -364,16 +299,14 @@ class RelabelConfig { } std::string_view g_name = p.substr(0, j); - auto rec = groups.find(std::string(g_name)); - if (rec != groups.end()) { + if (auto rec = groups.find(std::string(g_name)); rec != groups.end()) { // if g_name found in map add as group(int) src_parts.emplace_back(rec->second); p.remove_prefix(g_name.size() + 1); continue; } - bool ok = is_valid_name(rgx_validate, g_name); - if (!ok) { + if (!is_valid_name(rgx_validate, g_name)) { // if g_name invalid add as is - "${" + std::string{g_name} + "}" src_parts.emplace_back(tmpl.substr(tmpl.size() - p.size() - 2, g_name.size() + 3)); p.remove_prefix(g_name.size() + 1); @@ -388,10 +321,10 @@ class RelabelConfig { default: { // search '$' and extract g_name - int j = p.find('$'); + const auto j = p.find('$'); std::string_view g_name = p.substr(0, j); std::string name = extract(rgx_validate, g_name); - if (name.size() == 0) { + if (name.empty()) { // if name invalid add as is - "$" src_parts.emplace_back(tmpl.substr(tmpl.size() - p.size() - 1, 1)); continue; @@ -401,7 +334,7 @@ class RelabelConfig { if (rec != groups.end()) { // if g_name found in map add as group(int) src_parts.emplace_back(rec->second); - if (substr_g_name.size() != 0) { + if (!substr_g_name.empty()) { src_parts.emplace_back(substr_g_name); } p.remove_prefix(g_name.size()); @@ -409,7 +342,7 @@ class RelabelConfig { } // if g_name not found in map cut g_name - if (substr_g_name.size() != 0) { + if (!substr_g_name.empty()) { src_parts.emplace_back(substr_g_name); } p.remove_prefix(g_name.size()); @@ -430,12 +363,28 @@ class RelabelConfig { return BareBones::Bit::be(*reinterpret_cast(&digest[shift])); } + template + PROMPP_ALWAYS_INLINE std::string get_value(LabelsBuilder& builder) const { + std::string value; + + for (size_t i = 0; i < source_labels_.size(); ++i) { + const auto lv = builder.get(source_labels_[i]); + if (i == 0) [[unlikely]] { + value += lv; + continue; + } + value += separator_; + value += lv; + } + + return value; + } + public: // RelabelConfig - constructor for RelabelConfig from go-config. template PROMPP_ALWAYS_INLINE explicit RelabelConfig(GORelabelConfig* go_rc) noexcept - : source_labels_{}, - separator_{static_cast(go_rc->separator)}, + : separator_{static_cast(go_rc->separator)}, regexp_(static_cast(go_rc->regex)), modulus_{go_rc->modulus}, target_label_{static_cast(go_rc->target_label)}, @@ -451,302 +400,168 @@ class RelabelConfig { parse(regexp_, rgx_validate, replacement_, replacement_parts_); } - PROMPP_ALWAYS_INLINE RelabelConfig(RelabelConfig&&) noexcept = default; - // source_labels - a list of labels from which values are taken and concatenated with the configured separator in order. - PROMPP_ALWAYS_INLINE const std::vector& source_labels() const noexcept { return source_labels_; } + [[nodiscard]] PROMPP_ALWAYS_INLINE const std::vector& source_labels() const noexcept { return source_labels_; } // separator - is the string between concatenated values from the source labels. - PROMPP_ALWAYS_INLINE const std::string_view& separator() const noexcept { return separator_; } + [[nodiscard]] PROMPP_ALWAYS_INLINE const std::string_view& separator() const noexcept { return separator_; } // regexp - against which the concatenation is matched. - PROMPP_ALWAYS_INLINE const Regexp& regexp() const noexcept { return regexp_; } + [[nodiscard]] PROMPP_ALWAYS_INLINE const Regexp& regexp() const noexcept { return regexp_; } // modulus - to take of the hash of concatenated values from the source labels. - PROMPP_ALWAYS_INLINE const uint64_t& modulus() const noexcept { return modulus_; } + [[nodiscard]] PROMPP_ALWAYS_INLINE uint64_t modulus() const noexcept { return modulus_; } // target_label - is the label to which the resulting string is written in a replacement. // Regexp interpolation is allowed for the replace action. - PROMPP_ALWAYS_INLINE const std::string_view& target_label() const noexcept { return target_label_; } + [[nodiscard]] PROMPP_ALWAYS_INLINE const std::string_view& target_label() const noexcept { return target_label_; } // replacement - is the regex replacement pattern to be used. - PROMPP_ALWAYS_INLINE const std::string_view& replacement() const noexcept { return replacement_; } + [[nodiscard]] PROMPP_ALWAYS_INLINE const std::string_view& replacement() const noexcept { return replacement_; } // action - is the action to be performed for the relabeling. - PROMPP_ALWAYS_INLINE const rAction& action() const noexcept { return action_; } + [[nodiscard]] PROMPP_ALWAYS_INLINE rAction action() const noexcept { return action_; } // target_label_parts - dismantled target_label. - PROMPP_ALWAYS_INLINE const std::vector& target_label_parts() const noexcept { return target_label_parts_; } + [[nodiscard]] PROMPP_ALWAYS_INLINE const std::vector& target_label_parts() const noexcept { return target_label_parts_; } // replacement_parts - dismantled replacement. - PROMPP_ALWAYS_INLINE const std::vector& replacement_parts() const noexcept { return replacement_parts_; } + [[nodiscard]] PROMPP_ALWAYS_INLINE const std::vector& replacement_parts() const noexcept { return replacement_parts_; } // relabel - building relabeling labels. template - PROMPP_ALWAYS_INLINE relabelStatus relabel(std::stringstream& buf, LabelsBuilder& builder) { - std::string value; - for (size_t i = 0; i < source_labels_.size(); ++i) { - std::string_view lv = builder.get(source_labels_[i]); - if (i == 0) { - value += std::string(lv); - continue; - } - value += separator_; - value += lv; - } - + PROMPP_ALWAYS_INLINE relabelStatus relabel(std::string& buf, LabelsBuilder& builder) const { switch (action_) { case rDrop: { - if (regexp_.full_match(value)) { + if (regexp_.full_match(get_value(builder))) { return rsDrop; } break; } case rKeep: { - if (!regexp_.full_match(value)) { + if (!regexp_.full_match(get_value(builder))) { return rsDrop; } break; } case rDropEqual: { - if (builder.get(target_label_) == value) { + if (builder.get(target_label_) == get_value(builder)) { return rsDrop; } break; } case rKeepEqual: { - if (builder.get(target_label_) != value) { + if (builder.get(target_label_) != get_value(builder)) { return rsDrop; } break; } case rReplace: { - std::vector res_args; - bool ok = regexp_.match_to_args(value, res_args); - if (!ok) { + const auto value = get_value(builder); + std::vector res_args; + if (!regexp_.match_to_args(value, res_args)) { break; } - std::string lname = regexp_.replace_with_args(buf, res_args, target_label_parts_); - if (!label_name_is_valid(lname)) { + Regexp::replace_with_args(buf, res_args, target_label_parts_); + if (!label_name_is_valid(buf)) { break; } - std::string lvalue = regexp_.replace_with_args(buf, res_args, replacement_parts_); - if (lvalue.size() == 0) { + std::string lname = buf; + + Regexp::replace_with_args(buf, res_args, replacement_parts_); + if (buf.empty()) { if (builder.contains(lname)) { builder.del(lname); return rsRelabel; } break; } - builder.set(lname, lvalue); + builder.set(lname, buf); return rsRelabel; } case rLowercase: { - std::string lvalue{value}; - std::ranges::transform(lvalue, lvalue.begin(), [](unsigned char c) { return std::tolower(c); }); - builder.set(target_label_, lvalue); + auto value = get_value(builder); + std::ranges::transform(value, value.begin(), [](unsigned char c) { return std::tolower(c); }); + builder.set(target_label_, value); return rsRelabel; } case rUppercase: { - std::string lvalue{value}; - std::ranges::transform(lvalue, lvalue.begin(), [](unsigned char c) { return std::toupper(c); }); - builder.set(target_label_, lvalue); + auto value = get_value(builder); + std::ranges::transform(value, value.begin(), [](unsigned char c) { return std::toupper(c); }); + builder.set(target_label_, value); return rsRelabel; } case rHashMod: { - std::string lvalue{std::to_string(make_hash_uint64(value) % modulus_)}; + std::string lvalue{std::to_string(make_hash_uint64(get_value(builder)) % modulus_)}; builder.set(target_label_, lvalue); return rsRelabel; } case rLabelMap: { - bool change{false}; - builder.range([&](LNameType& lname, LValueType& lvalue) PROMPP_LAMBDA_INLINE -> bool { + std::vector labels_for_set; + builder.range([&](const auto& lname, const auto& lvalue) PROMPP_LAMBDA_INLINE -> bool { if (regexp_.full_match(lname)) { - std::string rlname = regexp_.replace_full(buf, lname, replacement_parts_); - builder.set(rlname, lvalue); - change = true; + regexp_.replace_full(buf, lname, replacement_parts_); + labels_for_set.emplace_back(buf, lvalue); } return true; }); - if (change) { - return rsRelabel; - } - break; - } - case rLabelDrop: { - bool change{false}; - builder.range([&](LNameType& lname, [[maybe_unused]] LValueType& lvalue) PROMPP_LAMBDA_INLINE -> bool { - if (regexp_.full_match(lname)) { - builder.del(lname); - change = true; + if (!labels_for_set.empty()) { + for (const auto& label : labels_for_set) { + builder.set(label.first, label.second); } - return true; - }); - if (change) { - return rsRelabel; - } - break; - } - case rLabelKeep: { - bool change{false}; - builder.range([&](LNameType& lname, [[maybe_unused]] LValueType& lvalue) PROMPP_LAMBDA_INLINE -> bool { - if (!regexp_.full_match(lname)) { - builder.del(lname); - change = true; - } - return true; - }); - if (change) { - return rsRelabel; - } - break; - } - - default: { - throw BareBones::Exception(0x481dea53751b85c3, "unknown relabel action"); - } - } - - return rsKeep; - } - - template - PROMPP_ALWAYS_INLINE relabelStatus relabel(std::stringstream& buf, LabelsBuilder& builder) const { - std::string value; - for (size_t i = 0; i < source_labels_.size(); ++i) { - std::string_view lv = builder.get(source_labels_[i]); - if (i == 0) { - value += std::string(lv); - continue; - } - value += separator_; - value += lv; - } - - switch (action_) { - case rDrop: { - if (regexp_.full_match(value)) { - return rsDrop; - } - break; - } - case rKeep: { - if (!regexp_.full_match(value)) { - return rsDrop; - } - break; - } - - case rDropEqual: { - if (builder.get(target_label_) == value) { - return rsDrop; + return rsRelabel; } - break; - } - case rKeepEqual: { - if (builder.get(target_label_) != value) { - return rsDrop; - } break; } - case rReplace: { - std::vector res_args; - bool ok = regexp_.match_to_args(value, res_args); - if (!ok) { - break; - } - - std::string lname = regexp_.replace_with_args(buf, res_args, target_label_parts_); - if (!label_name_is_valid(lname)) { - break; - } - std::string lvalue = regexp_.replace_with_args(buf, res_args, replacement_parts_); - if (lvalue.size() == 0) { - if (builder.contains(lname)) { - builder.del(lname); - return rsRelabel; - } - break; - } - builder.set(lname, lvalue); - return rsRelabel; - } - - case rLowercase: { - std::string lvalue{value}; - std::ranges::transform(lvalue, lvalue.begin(), [](unsigned char c) { return std::tolower(c); }); - builder.set(target_label_, lvalue); - return rsRelabel; - } - - case rUppercase: { - std::string lvalue{value}; - std::ranges::transform(lvalue, lvalue.begin(), [](unsigned char c) { return std::toupper(c); }); - builder.set(target_label_, lvalue); - return rsRelabel; - } - - case rHashMod: { - std::string lvalue{std::to_string(make_hash_uint64(value) % modulus_)}; - builder.set(target_label_, lvalue); - return rsRelabel; - } - - case rLabelMap: { - bool change{false}; - builder.range([&](LNameType& lname, LValueType& lvalue) PROMPP_LAMBDA_INLINE -> bool { + case rLabelDrop: { + std::vector labels_for_del; + builder.range([&](const auto& lname, const auto&) PROMPP_LAMBDA_INLINE -> bool { if (regexp_.full_match(lname)) { - std::string rlname = regexp_.replace_full(buf, lname, replacement_parts_); - builder.set(rlname, lvalue); - change = true; + labels_for_del.emplace_back(lname); } return true; }); - if (change) { - return rsRelabel; - } - break; - } - case rLabelDrop: { - bool change{false}; - builder.range([&](LNameType& lname, [[maybe_unused]] LValueType& lvalue) PROMPP_LAMBDA_INLINE -> bool { - if (regexp_.full_match(lname)) { - builder.del(lname); - change = true; + if (!labels_for_del.empty()) { + for (const auto& name : labels_for_del) { + builder.del(name); } - return true; - }); - if (change) { + return rsRelabel; } + break; } case rLabelKeep: { - bool change{false}; - builder.range([&](LNameType& lname, [[maybe_unused]] LValueType& lvalue) PROMPP_LAMBDA_INLINE -> bool { + std::vector labels_for_del; + builder.range([&](const auto& lname, const auto&) PROMPP_LAMBDA_INLINE -> bool { if (!regexp_.full_match(lname)) { - builder.del(lname); - change = true; + labels_for_del.emplace_back(lname); } return true; }); - if (change) { + + if (!labels_for_del.empty()) { + for (const auto& name : labels_for_del) { + builder.del(name); + } + return rsRelabel; } + break; } @@ -757,9 +572,6 @@ class RelabelConfig { return rsKeep; } - - // ~RelabelConfig - destructor for RelabelConfig from go-config. - PROMPP_ALWAYS_INLINE ~RelabelConfig() = default; }; // StatelessRelabeler - stateless relabeler with relabel configs. @@ -772,34 +584,15 @@ class StatelessRelabeler { // StatelessRelabeler - constructor for StatelessRelabeler, converting go-config. template PROMPP_ALWAYS_INLINE explicit StatelessRelabeler(const GORelabelConfigs& go_rcfgs) noexcept { - configs_.reserve(go_rcfgs.size()); - for (const auto go_rcfg : go_rcfgs) { - configs_.emplace_back(go_rcfg); - } + reset_to(go_rcfgs); } // relabeling_process caller passes a LabelsBuilder containing the initial set of labels, which is mutated by the rules. template - PROMPP_ALWAYS_INLINE relabelStatus relabeling_process(std::stringstream& buf, LabelsBuilder& builder) { - relabelStatus rstatus{rsKeep}; - for (auto& rcfg : configs_) { - relabelStatus status = rcfg.relabel(buf, builder); - if (status == rsDrop) { - return rsDrop; - } - if (status == rsRelabel) { - rstatus = rsRelabel; - } - } - - return rstatus; - } - - template - PROMPP_ALWAYS_INLINE relabelStatus relabeling_process(std::stringstream& buf, LabelsBuilder& builder) const { + PROMPP_ALWAYS_INLINE relabelStatus relabeling_process(std::string& buf, LabelsBuilder& builder) const { relabelStatus rstatus{rsKeep}; for (auto& rcfg : configs_) { - relabelStatus status = rcfg.relabel(buf, builder); + const relabelStatus status = rcfg.relabel(buf, builder); if (status == rsDrop) { return rsDrop; } @@ -814,9 +607,8 @@ class StatelessRelabeler { // relabeling_process_with_soft_validate caller passes a LabelsBuilder containing the initial set of labels, which is mutated by the rules with soft(on empty) // validate label set. template - PROMPP_ALWAYS_INLINE relabelStatus relabeling_process_with_soft_validate(std::stringstream& buf, LabelsBuilder& builder) { - relabelStatus rstatus = relabeling_process(buf, builder); - + PROMPP_ALWAYS_INLINE relabelStatus relabeling_process_with_soft_validate(std::ostringstream& buf, LabelsBuilder& builder) { + const relabelStatus rstatus = relabeling_process(buf, builder); if (rstatus == rsDrop) { return rsDrop; } @@ -837,23 +629,21 @@ class StatelessRelabeler { configs_.emplace_back(go_rcfg); } } - - PROMPP_ALWAYS_INLINE ~StatelessRelabeler() = default; }; // processExternalLabels merges externalLabels into ls. If ls contains // a label in externalLabels, the value in ls wins. template -PROMPP_ALWAYS_INLINE void process_external_labels(LabelsBuilder& builder, ExternalLabels& external_labels) { +PROMPP_ALWAYS_INLINE void process_external_labels(LabelsBuilder& builder, const ExternalLabels& external_labels) { if (external_labels.size() == 0) { return; } std::size_t j{0}; - builder.range([&](LNameType& lname, [[maybe_unused]] LValueType& lvalue) PROMPP_LAMBDA_INLINE -> bool { - for (; j < external_labels.size() && lname > external_labels[j].first;) { - builder.set(external_labels[j].first, external_labels[j].second); - ++j; + std::vector indexes_for_set; + builder.range([&](const auto& lname, const auto&) PROMPP_LAMBDA_INLINE -> bool { + for (; j < external_labels.size() && lname > external_labels[j].first; ++j) { + indexes_for_set.emplace_back(j); } if (j < external_labels.size() && lname == external_labels[j].first) { @@ -862,6 +652,10 @@ PROMPP_ALWAYS_INLINE void process_external_labels(LabelsBuilder& builder, Extern return true; }); + for (auto index : indexes_for_set) { + builder.set(external_labels[index].first, external_labels[index].second); + } + for (; j < external_labels.size(); j++) { builder.set(external_labels[j].first, external_labels[j].second); } @@ -876,7 +670,6 @@ PROMPP_ALWAYS_INLINE void soft_validate(relabelStatus& rstatus, LabelsBuilder& b if (builder.is_empty()) [[unlikely]] { rstatus = rsDrop; - return; } }; diff --git a/pp/prometheus/tests/relabeler_tests.cpp b/pp/prometheus/tests/relabeler_tests.cpp index d34a1b4ddb..591fd4dff3 100644 --- a/pp/prometheus/tests/relabeler_tests.cpp +++ b/pp/prometheus/tests/relabeler_tests.cpp @@ -1,7 +1,3 @@ -#pragma GCC diagnostic push -#pragma GCC diagnostic ignored "-Wunknown-pragmas" -#pragma GCC optimize("no-var-tracking") // to speed up compilation - #include #include #include @@ -15,326 +11,36 @@ namespace { -using namespace PromPP::Prometheus; // NOLINT -using namespace PromPP::Primitives; // NOLINT - -using LabelViewForTest = std::pair; -using LabelForTest = std::pair; - -class SampleForTest { - private: - int64_t timestamp_; - double value_; - - public: - PROMPP_ALWAYS_INLINE SampleForTest(int64_t timestamp, double value) noexcept : timestamp_(timestamp), value_(value) {} - - PROMPP_ALWAYS_INLINE SampleForTest() noexcept = default; - - int64_t timestamp() const noexcept { return timestamp_; } - int64_t& timestamp() noexcept { return timestamp_; } - - double value() const noexcept { return value_; } - double& value() noexcept { return value_; } -}; - -class NamesSetForTest : public std::vector { - using Base = std::vector; - - public: - using Base::Base; - [[maybe_unused]] friend size_t hash_value(const NamesSetForTest& lns) { return PromPP::Primitives::hash::hash_of_string_list(lns); } -}; - -class LabelViewSetForTest : public std::vector { - using Base = std::vector; - - public: - using Base::Base; - - PROMPP_ALWAYS_INLINE void add(const LabelViewForTest& label) noexcept { - if (__builtin_expect(Base::empty() || std::get<0>(label) > std::get<0>(Base::back()), true)) { - Base::emplace_back(label); - } else if (__builtin_expect(std::get<0>(label) == std::get<0>(Base::back()), false)) { - std::get<1>(Base::back()) = std::get<1>(label); - } else { - auto i = std::lower_bound(Base::begin(), Base::end(), std::get<0>(label), - [](const LabelViewForTest& a, const std::string_view& b) { return std::get<0>(a) < b; }); - if (__builtin_expect(std::get<0>(*i) == std::get<0>(label), false)) { - std::get<1>(*i) = std::get<1>(label); - } else { - Base::insert(i, label); - } - } - } - - template - PROMPP_ALWAYS_INLINE SymbolType get(const SymbolType& label_name) noexcept { - for (const auto& [ln, lv] : *this) { - if (ln == label_name) [[unlikely]] { - return lv; - } - } - - return ""; - } - - NamesSetForTest names() const { - NamesSetForTest tns; - - for (auto [label_name, _] : *this) { - tns.push_back(label_name); - } - - return tns; - } - - [[maybe_unused]] friend size_t hash_value(const LabelViewSetForTest& tls) { return PromPP::Primitives::hash::hash_of_label_set(tls); } -}; - -class TimeseriesForTest { - LabelViewSetForTest label_set_; - std::vector samples_; - - public: - TimeseriesForTest() noexcept = default; - TimeseriesForTest(const TimeseriesForTest&) noexcept = default; - TimeseriesForTest& operator=(const TimeseriesForTest&) noexcept = default; - TimeseriesForTest(TimeseriesForTest&&) noexcept = default; - TimeseriesForTest& operator=(TimeseriesForTest&&) noexcept = default; - - TimeseriesForTest(const LabelViewSetForTest& label_set, const std::vector& samples) noexcept : label_set_(label_set), samples_(samples) {} - - PROMPP_ALWAYS_INLINE auto& label_set() noexcept { return label_set_; } - - PROMPP_ALWAYS_INLINE const auto& label_set() const noexcept { return label_set_; } - - PROMPP_ALWAYS_INLINE const auto& samples() const noexcept { return samples_; } - - PROMPP_ALWAYS_INLINE auto& samples() noexcept { return samples_; } - - PROMPP_ALWAYS_INLINE void clear() noexcept { - label_set().clear(); - samples().clear(); - } -}; - -// LabelsBuilderForTest - builder for label set. -template -class LabelsBuilderForTest { - LabelSet* base_ = nullptr; - LabelViewSetForTest buf_; - std::vector add_; - std::vector del_; - - public: - PROMPP_ALWAYS_INLINE LabelsBuilderForTest() noexcept {} - - // del - add label name to remove from label set. - PROMPP_ALWAYS_INLINE void del(std::string& lname) { - std::erase_if(add_, [lname](LabelForTest& lv) { return lv.first == lname; }); - - if (auto i = std::ranges::find_if(del_.begin(), del_.end(), [lname](const std::string_view& ln) { return ln == lname; }); i != del_.end()) { - return; - } - - del_.emplace_back(lname); - } - - // del - add label name to remove from label set. - PROMPP_ALWAYS_INLINE void del(std::string_view lname) { - std::erase_if(add_, [lname](LabelForTest& lv) { return lv.first == lname; }); - - if (auto i = std::ranges::find_if(del_.begin(), del_.end(), [lname](const std::string_view& ln) { return ln == lname; }); i != del_.end()) { - return; - } - - del_.emplace_back(lname); - } - - // get - returns the value for the label with the given name. Returns an empty string if the label doesn't exist. - PROMPP_ALWAYS_INLINE std::string_view get(std::string_view lname) { - if (auto i = std::ranges::find_if(add_.begin(), add_.end(), [lname](const LabelForTest& l) { return l.first == lname; }); i != add_.end()) { - return (*i).second; - } - - if (auto i = std::ranges::find_if(del_.begin(), del_.end(), [lname](const std::string_view& ln) { return ln == lname; }); i != del_.end()) { - return ""; - } - - if (base_ != nullptr) [[likely]] { - for (const auto& [ln, lv] : *base_) { - if (ln == lname) { - return lv; - } - } - } - - return ""; - } - - PROMPP_ALWAYS_INLINE bool contains(std::string_view lname) { - if (auto i = std::ranges::find_if(add_, [&lname](const LabelForTest& l) { return l.first == lname; }); i != add_.end()) { - return true; - } - - if (base_ != nullptr) [[likely]] { - for (const auto& [ln, lv] : *base_) { - if (ln == lname) { - return true; - } - } - } - - return false; - } - - // returns size of building labels. - PROMPP_ALWAYS_INLINE size_t size() { - size_t count{0}; - if (base_ != nullptr) [[likely]] { - for (const auto& ls : *base_) { - if (auto i = std::ranges::find_if(add_.begin(), add_.end(), [ls](const LabelForTest& l) { return l.first == ls.first; }); i != add_.end()) { - continue; - } - - if (auto i = std::ranges::find_if(del_.begin(), del_.end(), [ls](const std::string_view& ln) { return ln == ls.first; }); i != del_.end()) { - continue; - } - - ++count; - } - } - - count += add_.size(); - return count; - } - - // returns true if ls represents an empty set of labels. - PROMPP_ALWAYS_INLINE bool is_empty() { return size() == 0; } - - // labels - returns the labels from the builder. If no modifications were made, the original labels are returned. - PROMPP_ALWAYS_INLINE LabelViewSetForTest labels() { - if (base_ != nullptr) [[likely]] { - for (const auto& ls : *base_) { - if (auto i = std::ranges::find_if(add_.begin(), add_.end(), [ls](const LabelForTest& l) { return l.first == ls.first; }); i != add_.end()) { - continue; - } - - if (auto i = std::ranges::find_if(del_.begin(), del_.end(), [ls](const std::string_view& ln) { return ln == ls.first; }); i != del_.end()) { - continue; - } - - buf_.add(ls); - } - } - - if (add_.size() != 0) { - std::ranges::for_each(add_.begin(), add_.end(), [&](const LabelForTest& l) { buf_.add(l); }); - std::ranges::sort(buf_.begin(), buf_.end(), [](const LabelViewForTest& a, const LabelViewForTest& b) { - if (a.first == b.first) { - return a.second < b.second; - } - return a.first < b.first; - }); - } - - return buf_; - } - - // range - calls f on each label in the builder. - template - PROMPP_ALWAYS_INLINE void range(Callback func) { - // take a copy of add and del, so they are unaffected by calls to set() or del(). - std::vector cadd; - cadd.reserve(add_.size()); - std::ranges::copy(add_.begin(), add_.end(), std::back_inserter(cadd)); - - std::vector cdel; - cdel.reserve(del_.size()); - std::ranges::copy(cdel.begin(), cdel.end(), std::back_inserter(cdel)); - - if (__builtin_expect(base_ != nullptr, true)) { - for (const auto& ls : *base_) { - if (auto i = std::ranges::find_if(cadd.begin(), cadd.end(), [ls](const LabelViewForTest& l) { return l.first == ls.first; }); i != cadd.end()) { - continue; - } - - if (auto i = std::ranges::find_if(cdel.begin(), cdel.end(), [ls](const std::string_view& ln) { return ln == ls.first; }); i != cdel.end()) { - continue; - } - - func(ls.first, ls.second); - } - } - - std::ranges::for_each(cadd.begin(), cadd.end(), [&](const LabelViewForTest& l) { func(l.first, l.second); }); - } - - // reset - clears all current state for the builder. - PROMPP_ALWAYS_INLINE void reset() { - buf_.clear(); - add_.clear(); - del_.clear(); - base_ = nullptr; - } - - // reset - clears all current state for the builder and init from LabelSet. - PROMPP_ALWAYS_INLINE void reset(LabelSet& ls) { - reset(); - base_ = &ls; - } - - // set - the name/value pair as a label. A value of "" means delete that label. - PROMPP_ALWAYS_INLINE void set(std::string& lname, std::string& lvalue) { - if (__builtin_expect(lvalue.size() == 0, false)) { - del(lname); - return; - } - - if (auto i = std::ranges::find_if(add_.begin(), add_.end(), [lname](const LabelForTest& l) { return l.first == lname; }); i != add_.end()) { - (*i).second = lvalue; - return; - } - - add_.emplace_back(lname, lvalue); - } - - // set - the name/value pair as a label. A value of "" means delete that label. - PROMPP_ALWAYS_INLINE void set(std::string_view lname, std::string& lvalue) { - if (__builtin_expect(lvalue.size() == 0, false)) { - del(lname); - return; - } - - if (auto i = std::ranges::find_if(add_.begin(), add_.end(), [lname](const LabelForTest& l) { return l.first == lname; }); i != add_.end()) { - (*i).second = lvalue; - return; - } - - add_.emplace_back(lname, lvalue); - } - - // set - the name/value pair as a label. A value of "" means delete that label. - PROMPP_ALWAYS_INLINE void set(std::string& lname, std::string_view lvalue) { - if (__builtin_expect(lvalue.size() == 0, false)) { - del(lname); - return; - } - - if (auto i = std::ranges::find_if(add_.begin(), add_.end(), [lname](const LabelForTest& l) { return l.first == lname; }); i != add_.end()) { - (*i).second = lvalue; - return; - } - - add_.emplace_back(lname, lvalue); - } - - PROMPP_ALWAYS_INLINE LabelsBuilderForTest(LabelsBuilderForTest&&) noexcept = default; - PROMPP_ALWAYS_INLINE ~LabelsBuilderForTest() = default; -}; - -struct RelabelConfigTest { +using PromPP::Primitives::Label; +using PromPP::Primitives::LabelsBuilder; +using PromPP::Primitives::LabelSet; +using PromPP::Primitives::LabelView; +using PromPP::Primitives::LabelViewSet; +using PromPP::Primitives::Sample; +using PromPP::Primitives::Timestamp; +using PromPP::Primitives::Go::SliceView; +using PromPP::Primitives::SnugComposites::LabelSet::EncodingBimap; +using PromPP::Primitives::SnugComposites::LabelSet::OrderedEncodingBimap; +using PromPP::Prometheus::Relabel::hard_validate; +using PromPP::Prometheus::Relabel::InnerSerie; +using PromPP::Prometheus::Relabel::InnerSeries; +using PromPP::Prometheus::Relabel::MetricLimits; +using PromPP::Prometheus::Relabel::PerGoroutineRelabeler; +using PromPP::Prometheus::Relabel::PerShardRelabeler; +using PromPP::Prometheus::Relabel::RelabelerStateUpdate; +using PromPP::Prometheus::Relabel::relabelStatus; +using PromPP::Prometheus::Relabel::StaleNaNsState; +using PromPP::Prometheus::Relabel::StatelessRelabeler; +using enum PromPP::Prometheus::Relabel::rAction; +using enum relabelStatus; + +using GoString = PromPP::Primitives::Go::String; +using PromPP::Primitives::kNullTimestamp; +using PromPP::Prometheus::kStaleNan; + +using GoLabel = std::pair; + +struct RelabelConfig { std::vector source_labels{}; std::string_view separator{}; std::string_view regex{}; @@ -344,43 +50,25 @@ struct RelabelConfigTest { uint8_t action{0}; }; -PROMPP_ALWAYS_INLINE LabelViewSetForTest make_label_set(std::initializer_list lvs) { - LabelViewSetForTest labels; - for (const LabelViewForTest& lv : lvs) { - labels.add(lv); - } - - return labels; -} - -PROMPP_ALWAYS_INLINE std::vector make_samples(std::initializer_list samples) { - std::vector sampleses; - for (const SampleForTest& s : samples) { - sampleses.push_back(s); - } - - return sampleses; -} - -struct ItemTest { - size_t hash_; - LabelViewSetForTest labelview_set_; - std::vector samples_; +class ItemTest { + public: + ItemTest(LabelViewSet&& label_set, std::vector&& samples) + : label_set_(std::move(label_set)), samples_(std::move(samples)), hash_(hash_value(label_set_)) {} - PROMPP_ALWAYS_INLINE explicit ItemTest(size_t hash, LabelViewSetForTest& labelview_set, std::vector& samples) - : hash_(hash), labelview_set_(labelview_set), samples_(samples) {} - PROMPP_ALWAYS_INLINE size_t hash() const { return hash_; } + [[nodiscard]] PROMPP_ALWAYS_INLINE size_t hash() const { return hash_; } template PROMPP_ALWAYS_INLINE void read(Timeseries& timeseries) const { - for (const auto& labelview : labelview_set_) { - timeseries.label_set().add({labelview.first, labelview.second}); - } - + timeseries.label_set().add(label_set_); for (const auto& sample : samples_) { timeseries.samples().emplace_back(sample.timestamp(), sample.value()); } } + + private: + LabelViewSet label_set_; + std::vector samples_; + size_t hash_; }; class HashdexTest : public std::vector { @@ -389,6 +77,8 @@ class HashdexTest : public std::vector { public: using Base::Base; + void emplace_back(LabelViewSet&& label_set, std::vector&& samples) { Base::emplace_back(std::move(label_set), std::move(samples)); } + [[nodiscard]] PROMPP_ALWAYS_INLINE const auto& metrics() const noexcept { return *this; } [[nodiscard]] static PROMPP_ALWAYS_INLINE auto metadata() noexcept { struct Stub {}; @@ -398,109 +88,76 @@ class HashdexTest : public std::vector { static_assert(PromPP::Prometheus::hashdex::HashdexInterface); -PROMPP_ALWAYS_INLINE void make_hashdex(HashdexTest& hx, LabelViewSetForTest label_set, std::vector samples) { - hx.emplace_back(hash_value(label_set), label_set, samples); -} - -// -// TestValidate -// - -struct TestValidate : public testing::Test { - PromPP::Primitives::LabelsBuilderStateMap builder_state_; +struct HardValidateCase { + LabelViewSet labels; + std::optional limits{std::nullopt}; + relabelStatus expected; }; -TEST_F(TestValidate, HardValid) { - LabelViewSetForTest incoming_labels = make_label_set({{"__name__", "value"}, {"job", "abc"}}); - PromPP::Primitives::LabelsBuilder builder{builder_state_}; - builder.reset(incoming_labels); - - Relabel::relabelStatus rstatus{Relabel::rsKeep}; - Relabel::hard_validate(rstatus, builder, nullptr); - EXPECT_EQ(Relabel::rsKeep, rstatus); -} - -TEST_F(TestValidate, HardInvalid) { - LabelViewSetForTest incoming_labels = make_label_set({{"__value__", "value"}, {"job", "abc"}}); - PromPP::Primitives::LabelsBuilder builder{builder_state_}; - builder.reset(incoming_labels); - - Relabel::relabelStatus rstatus{Relabel::rsKeep}; - Relabel::hard_validate(rstatus, builder, nullptr); - EXPECT_EQ(Relabel::rsInvalid, rstatus); -} - -TEST_F(TestValidate, InvalidLabelLimit) { - LabelViewSetForTest incoming_labels = make_label_set({{"__name__", "value"}, {"job", "abc"}, {"jub", "buj"}}); - PromPP::Primitives::LabelsBuilder builder{builder_state_}; - builder.reset(incoming_labels); - - // label_limit 0 - Relabel::MetricLimits ll{}; - Relabel::relabelStatus rstatus{Relabel::rsKeep}; - Relabel::hard_validate(rstatus, builder, &ll); - EXPECT_EQ(Relabel::rsKeep, rstatus); - - // label_limit 2 - ll.label_limit = 2; - Relabel::hard_validate(rstatus, builder, &ll); - EXPECT_EQ(Relabel::rsInvalid, rstatus); - - // label_name_length_limit 3 - rstatus = Relabel::rsKeep; - ll.label_limit = 3; - ll.label_name_length_limit = 3; - Relabel::hard_validate(rstatus, builder, &ll); - EXPECT_EQ(Relabel::rsInvalid, rstatus); - - // label_value_length_limit 3 - rstatus = Relabel::rsKeep; - ll.label_limit = 3; - ll.label_name_length_limit = 10; - ll.label_value_length_limit = 3; - Relabel::hard_validate(rstatus, builder, &ll); - EXPECT_EQ(Relabel::rsInvalid, rstatus); -} +class HardValidateFixture : public testing::TestWithParam { + protected: + LabelsBuilder builder_; + relabelStatus rstatus_{rsKeep}; +}; -// -// PerShardRelabeler -// +TEST_P(HardValidateFixture, Test) { + // Arrange + builder_.reset(GetParam().labels); + + // Act + hard_validate(rstatus_, builder_, GetParam().limits ? &GetParam().limits.value() : nullptr); + + // Assert + EXPECT_EQ(GetParam().expected, rstatus_); +} + +INSTANTIATE_TEST_SUITE_P(Valid, HardValidateFixture, testing::Values(HardValidateCase{.labels = {{"__name__", "value"}, {"job", "abc"}}, .expected = rsKeep})); +INSTANTIATE_TEST_SUITE_P(Invalid, + HardValidateFixture, + testing::Values(HardValidateCase{.labels = {{"__value__", "value"}, {"job", "abc"}}, .expected = rsInvalid})); +INSTANTIATE_TEST_SUITE_P( + NoLimit, + HardValidateFixture, + testing::Values(HardValidateCase{.labels = {{"__name__", "value"}, {"job", "abc"}, {"jub", "buj"}}, .limits = MetricLimits{}, .expected = rsKeep})); +INSTANTIATE_TEST_SUITE_P(LabelCountLimitExceeded, + HardValidateFixture, + testing::Values(HardValidateCase{.labels = {{"__name__", "value"}, {"job", "abc"}, {"jub", "buj"}}, + .limits = MetricLimits{.label_limit = 2}, + .expected = rsInvalid})); +INSTANTIATE_TEST_SUITE_P(LabelNameLengthLimitExceeded, + HardValidateFixture, + testing::Values(HardValidateCase{.labels = {{"__name__", "value"}, {"job", "abc"}, {"jub", "buj"}}, + .limits = MetricLimits{.label_name_length_limit = 3}, + .expected = rsInvalid})); +INSTANTIATE_TEST_SUITE_P(LabelValueLengthLimitExceeded, + HardValidateFixture, + testing::Values(HardValidateCase{.labels = {{"__name__", "value"}, {"job", "abc"}, {"jub", "buj"}}, + .limits = MetricLimits{.label_value_length_limit = 3}, + .expected = rsInvalid})); struct Stats { uint32_t samples_added{0}; uint32_t series_added{0}; uint32_t series_drop{0}; -}; -struct TestPerShardRelabeler : public testing::Test { - // shards_inner_series - std::vector> vector_shards_inner_series_; - PromPP::Primitives::Go::SliceView shards_inner_series_; + bool operator==(const Stats& other) const noexcept = default; +}; - // relabeled_results - std::vector> vector_relabeled_results_; - PromPP::Primitives::Go::SliceView relabeled_results_; +class PerGoroutineRelabelerFixture : public testing::Test { + protected: + static constexpr uint16_t kNumberOfShards = 2; - // external_labels - std::vector> vector_external_labels_; - PromPP::Primitives::Go::SliceView> external_labels_; + std::vector> vector_shards_inner_series_; + SliceView shards_inner_series_{}; - // target_labels - std::vector> vector_target_labels_{}; + std::vector> vector_relabeled_results_; + SliceView relabeled_results_{}; - // Options + std::vector vector_target_labels_{}; PromPP::Prometheus::Relabel::RelabelerOptions o_; - - // Stats Stats stats_; - - // Hashdex HashdexTest hx_; - - // LSS - PromPP::Primitives::SnugComposites::LabelSet::EncodingBimap lss_; - - // Cache + EncodingBimap lss_; PromPP::Prometheus::Relabel::Cache cache_{}; void reset() { @@ -508,526 +165,432 @@ struct TestPerShardRelabeler : public testing::Test { SetUp(); } - void add_target_labels(std::vector>& list_target_labels) { - vector_target_labels_.resize(list_target_labels.size()); + void add_target_labels(const LabelViewSet& target_labels) { + vector_target_labels_.resize(target_labels.size()); for (size_t i = 0; i < vector_target_labels_.size(); i++) { - vector_target_labels_[i].first.reset_to(list_target_labels[i].first.data(), list_target_labels[i].first.size()); - vector_target_labels_[i].second.reset_to(list_target_labels[i].second.data(), list_target_labels[i].second.size()); + vector_target_labels_[i].first.reset_to(target_labels[i].first.data(), target_labels[i].first.size()); + vector_target_labels_[i].second.reset_to(target_labels[i].second.data(), target_labels[i].second.size()); } o_.target_labels.reset_to(vector_target_labels_.data(), vector_target_labels_.size(), vector_target_labels_.size()); } void SetUp() final { - // shards_inner_series - vector_shards_inner_series_.emplace_back(std::make_unique()); - vector_shards_inner_series_.emplace_back(std::make_unique()); - shards_inner_series_.reset_to(reinterpret_cast(vector_shards_inner_series_.data()), - vector_shards_inner_series_.size(), vector_shards_inner_series_.size()); + vector_shards_inner_series_.emplace_back(std::make_unique()); + vector_shards_inner_series_.emplace_back(std::make_unique()); + shards_inner_series_.reset_to(reinterpret_cast(vector_shards_inner_series_.data()), vector_shards_inner_series_.size(), + vector_shards_inner_series_.size()); - // relabeled_results vector_relabeled_results_.emplace_back(std::make_unique()); vector_relabeled_results_.emplace_back(std::make_unique()); relabeled_results_.reset_to(reinterpret_cast(vector_relabeled_results_.data()), vector_shards_inner_series_.size(), vector_shards_inner_series_.size()); - // external_labels - external_labels_.reset_to(vector_external_labels_.data(), vector_external_labels_.size(), vector_external_labels_.size()); - - // target_labels o_.target_labels.reset_to(vector_target_labels_.data(), vector_target_labels_.size(), vector_target_labels_.size()); } void TearDown() final { - // clear memory vector_shards_inner_series_.clear(); vector_relabeled_results_.clear(); } }; -TEST_F(TestPerShardRelabeler, KeepEQ) { - RelabelConfigTest rct{.source_labels = std::vector{"job"}, .regex = "abc", .action = 2}; // Keep - Relabel::StatelessRelabeler sr(std::vector{&rct}); - PromPP::Prometheus::Relabel::PerShardRelabeler prs(external_labels_, &sr, 2, 1); - make_hashdex(hx_, make_label_set({{"__name__", "value"}, {"job", "abc"}}), make_samples({{1712567046855, 0.1}})); +TEST_F(PerGoroutineRelabelerFixture, KeepOnNotFoundInCache) { + // Arrange + const RelabelConfig config{.source_labels = {{"job"}}, .regex = "abc", .action = rKeep}; + const StatelessRelabeler stateless_relabeler(std::initializer_list{&config}); + hx_.emplace_back({{"__name__", "value"}, {"job", "abc"}}, {{1000, 0.1}}); + PerGoroutineRelabeler relabeler{kNumberOfShards, 1}; - prs.input_relabeling(lss_, lss_, cache_, hx_, o_, stats_, shards_inner_series_, relabeled_results_); - // shard id 1 + // Act + relabeler.input_relabeling(lss_, lss_, cache_, hx_, o_, stateless_relabeler, stats_, shards_inner_series_, relabeled_results_); + + // Assert EXPECT_EQ(relabeled_results_[0]->size(), 0); EXPECT_EQ(relabeled_results_[1]->size(), 0); EXPECT_EQ(shards_inner_series_[0]->size(), 0); - EXPECT_EQ(shards_inner_series_[1]->size(), 1); - EXPECT_EQ(stats_.samples_added, 1); - EXPECT_EQ(stats_.series_added, 1); - - PromPP::Prometheus::Relabel::RelabelerStateUpdate update_data{}; - prs.append_relabeler_series(lss_, shards_inner_series_[1], relabeled_results_[1], &update_data); - EXPECT_EQ(shards_inner_series_[1]->size(), 1); - EXPECT_EQ(update_data.size(), 0); - - vector_shards_inner_series_[1] = std::make_unique(); - prs.input_relabeling(lss_, lss_, cache_, hx_, o_, stats_, shards_inner_series_, relabeled_results_); - EXPECT_EQ(shards_inner_series_[1]->size(), 1); - EXPECT_EQ(stats_.samples_added, 2); - EXPECT_EQ(stats_.series_added, 1); + EXPECT_TRUE(std::ranges::equal(std::vector{{.sample = Sample(1000, 0.1), .ls_id = 0}}, shards_inner_series_[1]->data())); + EXPECT_EQ((Stats{.samples_added = 1, .series_added = 1}), stats_); } -TEST_F(TestPerShardRelabeler, KeepEQ_OrderedEncodingBimap) { - PromPP::Primitives::SnugComposites::LabelSet::OrderedEncodingBimap lss; - RelabelConfigTest rct{.source_labels = std::vector{"job"}, .regex = "abc", .action = 2}; // Keep - Relabel::StatelessRelabeler sr(std::vector{&rct}); - PromPP::Prometheus::Relabel::PerShardRelabeler prs(external_labels_, &sr, 2, 1); - make_hashdex(hx_, make_label_set({{"__name__", "value"}, {"job", "abc"}}), make_samples({{1712567046855, 0.1}})); +TEST_F(PerGoroutineRelabelerFixture, InnerSeriesAlreadyAdded) { + // Arrange + const RelabelConfig config{.source_labels = {{"job"}}, .regex = "abc", .action = rKeep}; + const StatelessRelabeler stateless_relabeler(std::initializer_list{&config}); + hx_.emplace_back({{"__name__", "value"}, {"job", "abc"}}, {{1000, 0.1}}); + PerGoroutineRelabeler relabeler{kNumberOfShards, 1}; + + // Act + relabeler.input_relabeling(lss_, lss_, cache_, hx_, o_, stateless_relabeler, stats_, shards_inner_series_, relabeled_results_); + relabeler.input_relabeling(lss_, lss_, cache_, hx_, o_, stateless_relabeler, stats_, shards_inner_series_, relabeled_results_); - prs.input_relabeling(lss, lss, cache_, hx_, o_, stats_, shards_inner_series_, relabeled_results_); - // shard id 1 + // Assert EXPECT_EQ(relabeled_results_[0]->size(), 0); EXPECT_EQ(relabeled_results_[1]->size(), 0); EXPECT_EQ(shards_inner_series_[0]->size(), 0); - EXPECT_EQ(shards_inner_series_[1]->size(), 1); - EXPECT_EQ(stats_.samples_added, 1); - EXPECT_EQ(stats_.series_added, 1); - - PromPP::Prometheus::Relabel::RelabelerStateUpdate update_data{}; - prs.append_relabeler_series(lss_, shards_inner_series_[1], relabeled_results_[1], &update_data); - EXPECT_EQ(shards_inner_series_[1]->size(), 1); - EXPECT_EQ(update_data.size(), 0); - - vector_shards_inner_series_[1] = std::make_unique(); - prs.input_relabeling(lss, lss, cache_, hx_, o_, stats_, shards_inner_series_, relabeled_results_); - EXPECT_EQ(shards_inner_series_[1]->size(), 1); - EXPECT_EQ(stats_.samples_added, 2); - EXPECT_EQ(stats_.series_added, 1); + EXPECT_TRUE(std::ranges::equal(std::vector{{.sample = Sample(1000, 0.1), .ls_id = 0}}, shards_inner_series_[1]->data())); + EXPECT_EQ((Stats{.samples_added = 1, .series_added = 1}), stats_); } -TEST_F(TestPerShardRelabeler, KeepNE) { - RelabelConfigTest rct{.source_labels = std::vector{"job"}, .regex = "no-match", .action = 2}; // Keep - Relabel::StatelessRelabeler sr(std::vector{&rct}); - PromPP::Prometheus::Relabel::PerShardRelabeler prs(external_labels_, &sr, 2, 1); - make_hashdex(hx_, make_label_set({{"__name__", "value"}, {"job", "abc"}}), make_samples({{1712567046855, 0.1}})); +TEST_F(PerGoroutineRelabelerFixture, KeepOnFoundInCache) { + // Arrange + const RelabelConfig config{.source_labels = {{"job"}}, .regex = "abc", .action = rKeep}; + const StatelessRelabeler stateless_relabeler(std::initializer_list{&config}); + hx_.emplace_back({{"__name__", "value"}, {"job", "abc"}}, {{1000, 0.1}}); + PerGoroutineRelabeler relabeler{kNumberOfShards, 1}; + + // Act + relabeler.input_relabeling(lss_, lss_, cache_, hx_, o_, stateless_relabeler, stats_, shards_inner_series_, relabeled_results_); + shards_inner_series_[1]->clear(); + relabeler.input_relabeling(lss_, lss_, cache_, hx_, o_, stateless_relabeler, stats_, shards_inner_series_, relabeled_results_); - prs.input_relabeling(lss_, lss_, cache_, hx_, o_, stats_, shards_inner_series_, relabeled_results_); - // shard id 1 + // Assert EXPECT_EQ(relabeled_results_[0]->size(), 0); EXPECT_EQ(relabeled_results_[1]->size(), 0); EXPECT_EQ(shards_inner_series_[0]->size(), 0); - EXPECT_EQ(shards_inner_series_[1]->size(), 0); - EXPECT_EQ(stats_.samples_added, 0); - EXPECT_EQ(stats_.series_added, 0); + EXPECT_TRUE(std::ranges::equal(std::vector{{.sample = Sample(1000, 0.1), .ls_id = 0}}, shards_inner_series_[1]->data())); + EXPECT_EQ((Stats{.samples_added = 2, .series_added = 1}), stats_); } -TEST_F(TestPerShardRelabeler, KeepEQNE) { - RelabelConfigTest rct{.source_labels = std::vector{"job"}, .regex = "abc", .action = 2}; // Keep - Relabel::StatelessRelabeler sr(std::vector{&rct}); - PromPP::Prometheus::Relabel::PerShardRelabeler prs(external_labels_, &sr, 2, 1); - make_hashdex(hx_, make_label_set({{"__name__", "value"}, {"job", "abc"}}), make_samples({{1712567046855, 0.1}})); - - prs.input_relabeling(lss_, lss_, cache_, hx_, o_, stats_, shards_inner_series_, relabeled_results_); - // shard id 1 - EXPECT_EQ(relabeled_results_[0]->size(), 0); - EXPECT_EQ(relabeled_results_[1]->size(), 0); - EXPECT_EQ(shards_inner_series_[0]->size(), 0); - EXPECT_EQ(shards_inner_series_[1]->size(), 1); - EXPECT_EQ(stats_.samples_added, 1); - EXPECT_EQ(stats_.series_added, 1); +TEST_F(PerGoroutineRelabelerFixture, KeepNotEqual) { + // Arrange + const RelabelConfig config{.source_labels = {{"job"}}, .regex = "no-match", .action = rKeep}; + const StatelessRelabeler stateless_relabeler(std::initializer_list{&config}); + hx_.emplace_back({{"__name__", "value"}, {"job", "abc"}}, {{1000, 0.1}}); + PerGoroutineRelabeler relabeler{kNumberOfShards, 1}; - PromPP::Prometheus::Relabel::RelabelerStateUpdate update_data{}; - prs.append_relabeler_series(lss_, shards_inner_series_[1], relabeled_results_[1], &update_data); - EXPECT_EQ(shards_inner_series_[1]->size(), 1); - EXPECT_EQ(update_data.size(), 0); + // Act + relabeler.input_relabeling(lss_, lss_, cache_, hx_, o_, stateless_relabeler, stats_, shards_inner_series_, relabeled_results_); - reset(); - hx_.clear(); - make_hashdex(hx_, make_label_set({{"__name__", "value"}, {"job", "abcd"}}), make_samples({{1712567046855, 0.1}})); - prs.input_relabeling(lss_, lss_, cache_, hx_, o_, stats_, shards_inner_series_, relabeled_results_); - // shard id 1 skip + // Assert EXPECT_EQ(relabeled_results_[0]->size(), 0); EXPECT_EQ(relabeled_results_[1]->size(), 0); EXPECT_EQ(shards_inner_series_[0]->size(), 0); EXPECT_EQ(shards_inner_series_[1]->size(), 0); - EXPECT_EQ(stats_.samples_added, 1); - EXPECT_EQ(stats_.series_added, 1); + EXPECT_EQ(Stats{.series_drop = 1}, stats_); } -TEST_F(TestPerShardRelabeler, ReplaceToNewLS2) { - RelabelConfigTest rct{.source_labels = std::vector{"__name__"}, - .separator = ";", - .regex = ".*(o).*", - .target_label = "replaced", - .replacement = "$1", - .action = 5}; // Replace - Relabel::StatelessRelabeler sr(std::vector{&rct}); - PromPP::Prometheus::Relabel::PerShardRelabeler prs(external_labels_, &sr, 2, 1); - make_hashdex(hx_, make_label_set({{"__name__", "booom"}, {"jab", "baj"}, {"job", "baj"}}), make_samples({{1712567046855, 0.1}})); - - prs.input_relabeling(lss_, lss_, cache_, hx_, o_, stats_, shards_inner_series_, relabeled_results_); - // shard id 1 - EXPECT_EQ(relabeled_results_[0]->size(), 0); - EXPECT_EQ(relabeled_results_[1]->size(), 1); - EXPECT_EQ(shards_inner_series_[0]->size(), 0); - EXPECT_EQ(shards_inner_series_[1]->size(), 0); - EXPECT_EQ(stats_.samples_added, 1); - EXPECT_EQ(stats_.series_added, 1); - - PromPP::Prometheus::Relabel::RelabelerStateUpdate update_data{}; - prs.append_relabeler_series(lss_, shards_inner_series_[1], relabeled_results_[1], &update_data); - EXPECT_EQ(shards_inner_series_[1]->size(), 1); - EXPECT_EQ(update_data.size(), 1); - - prs.update_relabeler_state(cache_, &update_data, 1); +TEST_F(PerGoroutineRelabelerFixture, KeepEqualThenNotEqual) { + // Arrange + const RelabelConfig config{.source_labels = {{"job"}}, .regex = "abc", .action = rKeep}; + const StatelessRelabeler stateless_relabeler(std::initializer_list{&config}); + hx_.emplace_back({{"__name__", "value"}, {"job", "abc"}}, {{1000, 0.1}}); + PerGoroutineRelabeler relabeler{kNumberOfShards, 1}; - auto rlabels = lss_[update_data[0].relabeled_ls_id]; - LabelViewSetForTest expected_labels = make_label_set({{"__name__", "booom"}, {"jab", "baj"}, {"job", "baj"}, {"replaced", "o"}}); - - EXPECT_EQ(rlabels, expected_labels); -} - -TEST_F(TestPerShardRelabeler, ReplaceToNewLS3) { - RelabelConfigTest rct{.separator = ";", .regex = ".*", .target_label = "replaced", .replacement = "blabla", .action = 5}; // Replace - Relabel::StatelessRelabeler sr(std::vector{&rct}); - PromPP::Prometheus::Relabel::PerShardRelabeler prs(external_labels_, &sr, 2, 1); - make_hashdex(hx_, make_label_set({{"__name__", "booom"}, {"jab", "baj"}, {"job", "baj"}}), make_samples({{1712567046855, 0.1}})); + // Act + relabeler.input_relabeling(lss_, lss_, cache_, hx_, o_, stateless_relabeler, stats_, shards_inner_series_, relabeled_results_); + reset(); + hx_ = HashdexTest{{{{"__name__", "value"}, {"job", "abcd"}}, {{1000, 0.1}}}}; + relabeler.input_relabeling(lss_, lss_, cache_, hx_, o_, stateless_relabeler, stats_, shards_inner_series_, relabeled_results_); - prs.input_relabeling(lss_, lss_, cache_, hx_, o_, stats_, shards_inner_series_, relabeled_results_); - // shard id 1 - EXPECT_EQ(relabeled_results_[0]->size(), 1); + // Assert + EXPECT_EQ(relabeled_results_[0]->size(), 0); EXPECT_EQ(relabeled_results_[1]->size(), 0); EXPECT_EQ(shards_inner_series_[0]->size(), 0); EXPECT_EQ(shards_inner_series_[1]->size(), 0); - EXPECT_EQ(stats_.samples_added, 1); - EXPECT_EQ(stats_.series_added, 1); - - PromPP::Prometheus::Relabel::RelabelerStateUpdate update_data{}; - prs.append_relabeler_series(lss_, shards_inner_series_[0], relabeled_results_[0], &update_data); - EXPECT_EQ(shards_inner_series_[0]->size(), 1); - EXPECT_EQ(update_data.size(), 1); - - prs.update_relabeler_state(cache_, &update_data, 1); + EXPECT_EQ((Stats{.samples_added = 1, .series_added = 1, .series_drop = 1}), stats_); +} - auto rlabels = lss_[update_data[0].relabeled_ls_id]; - LabelViewSetForTest expected_labels = make_label_set({{"__name__", "booom"}, {"jab", "baj"}, {"job", "baj"}, {"replaced", "blabla"}}); +TEST_F(PerGoroutineRelabelerFixture, ReplaceToNewLS2) { + // Arrange + const RelabelConfig config{ + .source_labels = {{"__name__"}}, .separator = ";", .regex = ".*(o).*", .target_label = "replaced", .replacement = "$1", .action = rReplace}; + const StatelessRelabeler stateless_relabeler(std::initializer_list{&config}); + hx_.emplace_back({{"__name__", "booom"}, {"jab", "baj"}, {"job", "baj"}}, {{1000, 0.1}}); + PerGoroutineRelabeler relabeler{kNumberOfShards, 1}; + RelabelerStateUpdate update_data; - EXPECT_EQ(rlabels, expected_labels); -} + // Act + relabeler.input_relabeling(lss_, lss_, cache_, hx_, o_, stateless_relabeler, stats_, shards_inner_series_, relabeled_results_); + PerShardRelabeler::append_relabeler_series(lss_, shards_inner_series_[1], relabeled_results_[1], &update_data); -TEST_F(TestPerShardRelabeler, ReplaceToNewLS2_OrderedEncodingBimap) { - PromPP::Primitives::SnugComposites::LabelSet::OrderedEncodingBimap lss; - RelabelConfigTest rct{.source_labels = std::vector{"__name__"}, - .separator = ";", - .regex = ".*(o).*", - .target_label = "replaced", - .replacement = "$1", - .action = 5}; // Replace - Relabel::StatelessRelabeler sr(std::vector{&rct}); - PromPP::Prometheus::Relabel::PerShardRelabeler prs(external_labels_, &sr, 2, 1); - make_hashdex(hx_, make_label_set({{"__name__", "booom"}, {"jab", "baj"}, {"job", "baj"}}), make_samples({{1712567046855, 0.1}})); - - prs.input_relabeling(lss, lss, cache_, hx_, o_, stats_, shards_inner_series_, relabeled_results_); - // shard id 1 + // Assert EXPECT_EQ(relabeled_results_[0]->size(), 0); EXPECT_EQ(relabeled_results_[1]->size(), 1); EXPECT_EQ(shards_inner_series_[0]->size(), 0); + EXPECT_TRUE(std::ranges::equal(std::vector{{.sample = Sample(1000, 0.1), .ls_id = 1}}, shards_inner_series_[1]->data())); + EXPECT_EQ((Stats{.samples_added = 1, .series_added = 1}), stats_); + ASSERT_EQ(1U, update_data.size()); + EXPECT_EQ((LabelViewSet{{"__name__", "booom"}, {"jab", "baj"}, {"job", "baj"}, {"replaced", "o"}}), lss_[update_data[0].relabeled_ls_id]); +} + +TEST_F(PerGoroutineRelabelerFixture, ReplaceToNewLS3) { + // Arrange + const RelabelConfig config{.separator = ";", .regex = ".*", .target_label = "replaced", .replacement = "blabla", .action = rReplace}; + const StatelessRelabeler stateless_relabeler(std::initializer_list{&config}); + hx_.emplace_back({{"__name__", "booom"}, {"jab", "baj"}, {"job", "baj"}}, {{1000, 0.1}}); + PerGoroutineRelabeler relabeler{kNumberOfShards, 1}; + RelabelerStateUpdate update_data; + + // Act + relabeler.input_relabeling(lss_, lss_, cache_, hx_, o_, stateless_relabeler, stats_, shards_inner_series_, relabeled_results_); + PerShardRelabeler::append_relabeler_series(lss_, shards_inner_series_[0], relabeled_results_[0], &update_data); + PerShardRelabeler::update_relabeler_state(cache_, &update_data, 1); + + // Assert + EXPECT_EQ(relabeled_results_[0]->size(), 1); + EXPECT_EQ(relabeled_results_[1]->size(), 0); + EXPECT_TRUE(std::ranges::equal(std::vector{{.sample = Sample(1000, 0.1), .ls_id = 1}}, shards_inner_series_[0]->data())); EXPECT_EQ(shards_inner_series_[1]->size(), 0); - EXPECT_EQ(stats_.samples_added, 1); - EXPECT_EQ(stats_.series_added, 1); - - PromPP::Prometheus::Relabel::RelabelerStateUpdate update_data{}; - prs.append_relabeler_series(lss, shards_inner_series_[1], relabeled_results_[1], &update_data); - EXPECT_EQ(shards_inner_series_[1]->size(), 1); + EXPECT_EQ((Stats{.samples_added = 1, .series_added = 1}), stats_); EXPECT_EQ(update_data.size(), 1); - - prs.update_relabeler_state(cache_, &update_data, 1); - - auto rlabels = lss[update_data[0].relabeled_ls_id]; - LabelViewSetForTest expected_labels = make_label_set({{"__name__", "booom"}, {"jab", "baj"}, {"job", "baj"}, {"replaced", "o"}}); - - EXPECT_EQ(rlabels, expected_labels); + EXPECT_EQ((LabelViewSet{{"__name__", "booom"}, {"jab", "baj"}, {"job", "baj"}, {"replaced", "blabla"}}), lss_[update_data[0].relabeled_ls_id]); } -TEST_F(TestPerShardRelabeler, InputRelabelingWithStalenans_Default) { - RelabelConfigTest rct{.source_labels = std::vector{"job"}, .regex = "abc", .action = 2}; // Keep - Relabel::StatelessRelabeler sr(std::vector{&rct}); - PromPP::Prometheus::Relabel::PerShardRelabeler prs(external_labels_, &sr, 2, 1); - make_hashdex(hx_, make_label_set({{"__name__", "value"}, {"job", "abc"}}), make_samples({{PromPP::Primitives::kNullTimestamp, 0.1}})); - PromPP::Prometheus::Relabel::StaleNaNsState state{}; - PromPP::Primitives::Timestamp def_timestamp{1712567046955}; +TEST_F(PerGoroutineRelabelerFixture, InputRelabelingWithStalenans_Default) { + // Arrange + const RelabelConfig config{.source_labels = {{"job"}}, .regex = "abc", .action = rKeep}; + const StatelessRelabeler stateless_relabeler(std::initializer_list{&config}); + hx_.emplace_back({{"__name__", "value"}, {"job", "abc"}}, {{kNullTimestamp, 0.1}}); + PerGoroutineRelabeler relabeler{kNumberOfShards, 1}; + StaleNaNsState state; + RelabelerStateUpdate update_data; - prs.input_relabeling_with_stalenans(lss_, lss_, cache_, hx_, o_, stats_, shards_inner_series_, relabeled_results_, state, def_timestamp); + // Act + relabeler.input_relabeling_with_stalenans(lss_, lss_, cache_, hx_, o_, stateless_relabeler, stats_, shards_inner_series_, relabeled_results_, state, 1000); + PerShardRelabeler::append_relabeler_series(lss_, shards_inner_series_[1], relabeled_results_[1], &update_data); + relabeler.input_relabeling_with_stalenans(lss_, lss_, cache_, HashdexTest{}, o_, stateless_relabeler, stats_, shards_inner_series_, relabeled_results_, state, + 2000); - // shard id 1 + // Assert EXPECT_EQ(relabeled_results_[0]->size(), 0); EXPECT_EQ(relabeled_results_[1]->size(), 0); EXPECT_EQ(shards_inner_series_[0]->size(), 0); - EXPECT_EQ(shards_inner_series_[1]->size(), 1); - EXPECT_EQ(shards_inner_series_[1]->data()[0].sample.timestamp(), def_timestamp); - EXPECT_EQ(stats_.samples_added, 1); - EXPECT_EQ(stats_.series_added, 1); - - PromPP::Prometheus::Relabel::RelabelerStateUpdate update_data{}; - prs.append_relabeler_series(lss_, shards_inner_series_[1], relabeled_results_[1], &update_data); - EXPECT_EQ(shards_inner_series_[1]->size(), 1); + EXPECT_EQ(shards_inner_series_[1]->size(), 2); + EXPECT_TRUE(std::ranges::equal(std::vector{{.sample = Sample(1000, 0.1), .ls_id = 0}, {.sample = Sample(2000, kStaleNan), .ls_id = 0}}, + shards_inner_series_[1]->data())); + EXPECT_EQ((Stats{.samples_added = 1, .series_added = 1}), stats_); EXPECT_EQ(update_data.size(), 0); - - vector_shards_inner_series_[1] = std::make_unique(); - HashdexTest empty_hx; - PromPP::Primitives::Timestamp stale_ts = 1712567047055; - prs.input_relabeling_with_stalenans(lss_, lss_, cache_, empty_hx, o_, stats_, shards_inner_series_, relabeled_results_, state, stale_ts); - EXPECT_EQ(shards_inner_series_[1]->size(), 1); - EXPECT_EQ(PromPP::Primitives::Sample(stale_ts, PromPP::Prometheus::kStaleNan), shards_inner_series_[1]->data()[0].sample); - EXPECT_EQ(stats_.samples_added, 1); - EXPECT_EQ(stats_.series_added, 1); } -TEST_F(TestPerShardRelabeler, InputRelabelingWithStalenans_DefaultHonorTimestamps) { - RelabelConfigTest rct{.source_labels = std::vector{"job"}, .regex = "abc", .action = 2}; // Keep - Relabel::StatelessRelabeler sr(std::vector{&rct}); - PromPP::Prometheus::Relabel::PerShardRelabeler prs(external_labels_, &sr, 2, 1); - make_hashdex(hx_, make_label_set({{"__name__", "value"}, {"job", "abc"}}), make_samples({{PromPP::Primitives::kNullTimestamp, 0.1}})); - PromPP::Prometheus::Relabel::StaleNaNsState state{}; - PromPP::Primitives::Timestamp def_timestamp{1712567046955}; +TEST_F(PerGoroutineRelabelerFixture, InputRelabelingWithStalenans_DefaultHonorTimestamps) { + // Arrange + const RelabelConfig config{.source_labels = {{"job"}}, .regex = "abc", .action = rKeep}; + const StatelessRelabeler stateless_relabeler(std::initializer_list{&config}); + hx_.emplace_back({{"__name__", "value"}, {"job", "abc"}}, {{kNullTimestamp, 0.1}}); + PerGoroutineRelabeler relabeler{kNumberOfShards, 1}; + StaleNaNsState state; + RelabelerStateUpdate update_data; o_.honor_timestamps = true; - prs.input_relabeling_with_stalenans(lss_, lss_, cache_, hx_, o_, stats_, shards_inner_series_, relabeled_results_, state, def_timestamp); + // Act + relabeler.input_relabeling_with_stalenans(lss_, lss_, cache_, hx_, o_, stateless_relabeler, stats_, shards_inner_series_, relabeled_results_, state, 1000); + PerShardRelabeler::append_relabeler_series(lss_, shards_inner_series_[1], relabeled_results_[1], &update_data); + relabeler.input_relabeling_with_stalenans(lss_, lss_, cache_, HashdexTest{}, o_, stateless_relabeler, stats_, shards_inner_series_, relabeled_results_, state, + 2000); - // shard id 1 + // Assert EXPECT_EQ(relabeled_results_[0]->size(), 0); EXPECT_EQ(relabeled_results_[1]->size(), 0); EXPECT_EQ(shards_inner_series_[0]->size(), 0); - EXPECT_EQ(shards_inner_series_[1]->size(), 1); - EXPECT_EQ(shards_inner_series_[1]->data()[0].sample.timestamp(), def_timestamp); - EXPECT_EQ(stats_.samples_added, 1); - EXPECT_EQ(stats_.series_added, 1); - - PromPP::Prometheus::Relabel::RelabelerStateUpdate update_data{}; - prs.append_relabeler_series(lss_, shards_inner_series_[1], relabeled_results_[1], &update_data); - EXPECT_EQ(shards_inner_series_[1]->size(), 1); + EXPECT_EQ(shards_inner_series_[1]->size(), 2); + EXPECT_TRUE(std::ranges::equal(std::vector{{.sample = Sample(1000, 0.1), .ls_id = 0}, {.sample = Sample(2000, kStaleNan), .ls_id = 0}}, + shards_inner_series_[1]->data())); + EXPECT_EQ((Stats{.samples_added = 1, .series_added = 1}), stats_); EXPECT_EQ(update_data.size(), 0); - - vector_shards_inner_series_[1] = std::make_unique(); - HashdexTest empty_hx; - PromPP::Primitives::Timestamp stale_ts = 1712567047055; - prs.input_relabeling_with_stalenans(lss_, lss_, cache_, empty_hx, o_, stats_, shards_inner_series_, relabeled_results_, state, stale_ts); - EXPECT_EQ(shards_inner_series_[1]->size(), 1); - EXPECT_EQ(PromPP::Primitives::Sample(stale_ts, PromPP::Prometheus::kStaleNan), shards_inner_series_[1]->data()[0].sample); - EXPECT_EQ(stats_.samples_added, 1); - EXPECT_EQ(stats_.series_added, 1); } -TEST_F(TestPerShardRelabeler, InputRelabelingWithStalenans_WithMetricTimestamp) { - RelabelConfigTest rct{.source_labels = std::vector{"job"}, .regex = "abc", .action = 2}; // Keep - Relabel::StatelessRelabeler sr(std::vector{&rct}); - PromPP::Prometheus::Relabel::PerShardRelabeler prs(external_labels_, &sr, 2, 1); - make_hashdex(hx_, make_label_set({{"__name__", "value"}, {"job", "abc"}}), make_samples({{1712567046855, 0.1}})); - PromPP::Prometheus::Relabel::StaleNaNsState state{}; - PromPP::Primitives::Timestamp def_timestamp{1712567046955}; +TEST_F(PerGoroutineRelabelerFixture, InputRelabelingWithStalenans_WithMetricTimestamp) { + // Arrange + const RelabelConfig config{.source_labels = {{"job"}}, .regex = "abc", .action = rKeep}; + StatelessRelabeler stateless_relabeler(std::initializer_list{&config}); + hx_.emplace_back({{"__name__", "value"}, {"job", "abc"}}, {{1712567046855, 0.1}}); + PerGoroutineRelabeler relabeler{kNumberOfShards, 1}; + StaleNaNsState state; + RelabelerStateUpdate update_data; - prs.input_relabeling_with_stalenans(lss_, lss_, cache_, hx_, o_, stats_, shards_inner_series_, relabeled_results_, state, def_timestamp); + // Act + relabeler.input_relabeling_with_stalenans(lss_, lss_, cache_, hx_, o_, stateless_relabeler, stats_, shards_inner_series_, relabeled_results_, state, 1000); + PerShardRelabeler::append_relabeler_series(lss_, shards_inner_series_[1], relabeled_results_[1], &update_data); + relabeler.input_relabeling_with_stalenans(lss_, lss_, cache_, HashdexTest{}, o_, stateless_relabeler, stats_, shards_inner_series_, relabeled_results_, state, + 2000); - // shard id 1 + // Assert EXPECT_EQ(relabeled_results_[0]->size(), 0); EXPECT_EQ(relabeled_results_[1]->size(), 0); EXPECT_EQ(shards_inner_series_[0]->size(), 0); - EXPECT_EQ(shards_inner_series_[1]->size(), 1); - EXPECT_EQ(shards_inner_series_[1]->data()[0].sample.timestamp(), def_timestamp); - EXPECT_EQ(stats_.samples_added, 1); - EXPECT_EQ(stats_.series_added, 1); - - PromPP::Prometheus::Relabel::RelabelerStateUpdate update_data{}; - prs.append_relabeler_series(lss_, shards_inner_series_[1], relabeled_results_[1], &update_data); - EXPECT_EQ(shards_inner_series_[1]->size(), 1); + EXPECT_EQ(shards_inner_series_[1]->size(), 2); + EXPECT_TRUE(std::ranges::equal(std::vector{{.sample = Sample(1000, 0.1), .ls_id = 0}, {.sample = Sample(2000, kStaleNan), .ls_id = 0}}, + shards_inner_series_[1]->data())); + EXPECT_EQ((Stats{.samples_added = 1, .series_added = 1}), stats_); EXPECT_EQ(update_data.size(), 0); - - vector_shards_inner_series_[1] = std::make_unique(); - HashdexTest empty_hx; - PromPP::Primitives::Timestamp stale_ts = 1712567047055; - prs.input_relabeling_with_stalenans(lss_, lss_, cache_, empty_hx, o_, stats_, shards_inner_series_, relabeled_results_, state, stale_ts); - EXPECT_EQ(shards_inner_series_[1]->size(), 1); - EXPECT_EQ(PromPP::Primitives::Sample(stale_ts, PromPP::Prometheus::kStaleNan), shards_inner_series_[1]->data()[0].sample); - EXPECT_EQ(stats_.samples_added, 1); - EXPECT_EQ(stats_.series_added, 1); } -TEST_F(TestPerShardRelabeler, InputRelabelingWithStalenans_HonorTimestamps) { - RelabelConfigTest rct{.source_labels = std::vector{"job"}, .regex = "abc", .action = 2}; // Keep - Relabel::StatelessRelabeler sr(std::vector{&rct}); - PromPP::Prometheus::Relabel::PerShardRelabeler prs(external_labels_, &sr, 2, 1); - PromPP::Primitives::Timestamp metric_timestamp{1712567046955}; - make_hashdex(hx_, make_label_set({{"__name__", "value"}, {"job", "abc"}}), make_samples({{metric_timestamp, 0.1}})); - PromPP::Prometheus::Relabel::StaleNaNsState state{}; - PromPP::Primitives::Timestamp def_timestamp{1712567046955}; +TEST_F(PerGoroutineRelabelerFixture, InputRelabelingWithStalenans_HonorTimestamps) { + // Arrange + const RelabelConfig config{.source_labels = {{"job"}}, .regex = "abc", .action = rKeep}; + const StatelessRelabeler stateless_relabeler(std::initializer_list{&config}); + hx_.emplace_back({{"__name__", "value"}, {"job", "abc"}}, {{1500, 0.1}}); + PerGoroutineRelabeler relabeler{kNumberOfShards, 1}; o_.honor_timestamps = true; + StaleNaNsState state; + RelabelerStateUpdate update_data; - prs.input_relabeling_with_stalenans(lss_, lss_, cache_, hx_, o_, stats_, shards_inner_series_, relabeled_results_, state, def_timestamp); + // Act + relabeler.input_relabeling_with_stalenans(lss_, lss_, cache_, hx_, o_, stateless_relabeler, stats_, shards_inner_series_, relabeled_results_, state, 1000); + PerShardRelabeler::append_relabeler_series(lss_, shards_inner_series_[1], relabeled_results_[1], &update_data); + relabeler.input_relabeling_with_stalenans(lss_, lss_, cache_, HashdexTest{}, o_, stateless_relabeler, stats_, shards_inner_series_, relabeled_results_, state, + 2000); - // shard id 1 + // Assert EXPECT_EQ(relabeled_results_[0]->size(), 0); EXPECT_EQ(relabeled_results_[1]->size(), 0); EXPECT_EQ(shards_inner_series_[0]->size(), 0); EXPECT_EQ(shards_inner_series_[1]->size(), 1); - EXPECT_EQ(shards_inner_series_[1]->data()[0].sample.timestamp(), metric_timestamp); - EXPECT_EQ(stats_.samples_added, 1); - EXPECT_EQ(stats_.series_added, 1); - - PromPP::Prometheus::Relabel::RelabelerStateUpdate update_data{}; - prs.append_relabeler_series(lss_, shards_inner_series_[1], relabeled_results_[1], &update_data); - EXPECT_EQ(shards_inner_series_[1]->size(), 1); + EXPECT_TRUE(std::ranges::equal(std::vector{{.sample = Sample(1500, 0.1), .ls_id = 0}}, shards_inner_series_[1]->data())); + EXPECT_EQ((Stats{.samples_added = 1, .series_added = 1}), stats_); EXPECT_EQ(update_data.size(), 0); - - vector_shards_inner_series_[1] = std::make_unique(); - HashdexTest empty_hx; - PromPP::Primitives::Timestamp stale_ts = 1712567047055; - prs.input_relabeling_with_stalenans(lss_, lss_, cache_, empty_hx, o_, stats_, shards_inner_series_, relabeled_results_, state, stale_ts); - EXPECT_EQ(shards_inner_series_[1]->size(), 0); } -TEST_F(TestPerShardRelabeler, InputRelabelingWithStalenans_HonorTimestampsAndTrackStaleness) { - RelabelConfigTest rct{.source_labels = std::vector{"job"}, .regex = "abc", .action = 2}; // Keep - Relabel::StatelessRelabeler sr(std::vector{&rct}); - PromPP::Prometheus::Relabel::PerShardRelabeler prs(external_labels_, &sr, 2, 1); - PromPP::Primitives::Timestamp metric_timestamp{1712567046955}; - make_hashdex(hx_, make_label_set({{"__name__", "value"}, {"job", "abc"}}), make_samples({{metric_timestamp, 0.1}})); - PromPP::Prometheus::Relabel::StaleNaNsState state{}; - PromPP::Primitives::Timestamp def_timestamp{1712567046955}; +TEST_F(PerGoroutineRelabelerFixture, InputRelabelingWithStalenans_HonorTimestampsAndTrackStaleness) { + // Arrange + const RelabelConfig config{.source_labels = {{"job"}}, .regex = "abc", .action = rKeep}; + const StatelessRelabeler stateless_relabeler(std::initializer_list{&config}); + hx_.emplace_back({{"__name__", "value"}, {"job", "abc"}}, {{1500, 0.1}}); + PerGoroutineRelabeler relabeler{kNumberOfShards, 1}; o_.honor_timestamps = true; o_.track_timestamps_staleness = true; + StaleNaNsState state; + RelabelerStateUpdate update_data; - prs.input_relabeling_with_stalenans(lss_, lss_, cache_, hx_, o_, stats_, shards_inner_series_, relabeled_results_, state, def_timestamp); + // Act + relabeler.input_relabeling_with_stalenans(lss_, lss_, cache_, hx_, o_, stateless_relabeler, stats_, shards_inner_series_, relabeled_results_, state, 1000); + PerShardRelabeler::append_relabeler_series(lss_, shards_inner_series_[1], relabeled_results_[1], &update_data); + relabeler.input_relabeling_with_stalenans(lss_, lss_, cache_, HashdexTest{}, o_, stateless_relabeler, stats_, shards_inner_series_, relabeled_results_, state, + 2000); - // shard id 1 + // Assert EXPECT_EQ(relabeled_results_[0]->size(), 0); EXPECT_EQ(relabeled_results_[1]->size(), 0); EXPECT_EQ(shards_inner_series_[0]->size(), 0); - EXPECT_EQ(shards_inner_series_[1]->size(), 1); - EXPECT_EQ(shards_inner_series_[1]->data()[0].sample.timestamp(), metric_timestamp); - EXPECT_EQ(stats_.samples_added, 1); - EXPECT_EQ(stats_.series_added, 1); - - PromPP::Prometheus::Relabel::RelabelerStateUpdate update_data{}; - prs.append_relabeler_series(lss_, shards_inner_series_[1], relabeled_results_[1], &update_data); - EXPECT_EQ(shards_inner_series_[1]->size(), 1); + EXPECT_EQ(shards_inner_series_[1]->size(), 2); + EXPECT_TRUE(std::ranges::equal(std::vector{{.sample = Sample(1500, 0.1), .ls_id = 0}, {.sample = Sample(2000, kStaleNan), .ls_id = 0}}, + shards_inner_series_[1]->data())); + EXPECT_EQ((Stats{.samples_added = 1, .series_added = 1}), stats_); EXPECT_EQ(update_data.size(), 0); - - vector_shards_inner_series_[1] = std::make_unique(); - HashdexTest empty_hx; - PromPP::Primitives::Timestamp stale_ts = 1712567047055; - prs.input_relabeling_with_stalenans(lss_, lss_, cache_, empty_hx, o_, stats_, shards_inner_series_, relabeled_results_, state, stale_ts); - EXPECT_EQ(shards_inner_series_[1]->size(), 1); - EXPECT_EQ(PromPP::Primitives::Sample(stale_ts, PromPP::Prometheus::kStaleNan), shards_inner_series_[1]->data()[0].sample); - EXPECT_EQ(stats_.samples_added, 1); - EXPECT_EQ(stats_.series_added, 1); } -TEST_F(TestPerShardRelabeler, TargetLabels_HappyPath) { - RelabelConfigTest rct{.source_labels = std::vector{"job"}, .regex = "abc", .action = 2}; // Keep - Relabel::StatelessRelabeler sr(std::vector{&rct}); - PromPP::Prometheus::Relabel::PerShardRelabeler prs(external_labels_, &sr, 2, 0); - make_hashdex(hx_, make_label_set({{"__name__", "booom"}, {"jab", "baj"}, {"job", "abc"}}), make_samples({{1712567046855, 0.1}})); - std::vector> list_target_labels{{"a_name", "target_a_value"}, {"z_name", "target_z_value"}}; - add_target_labels(list_target_labels); +TEST_F(PerGoroutineRelabelerFixture, TargetLabels_HappyPath) { + // Arrange + const RelabelConfig config{.source_labels = {{"job"}}, .regex = "abc", .action = rKeep}; + const StatelessRelabeler stateless_relabeler(std::initializer_list{&config}); + hx_.emplace_back({{"__name__", "booom"}, {"jab", "baj"}, {"job", "abc"}}, {{1000, 0.1}}); + add_target_labels(LabelViewSet{{"a_name", "target_a_value"}, {"z_name", "target_z_value"}}); + PerGoroutineRelabeler relabeler{kNumberOfShards, 0}; + RelabelerStateUpdate update_data; - prs.input_relabeling(lss_, lss_, cache_, hx_, o_, stats_, shards_inner_series_, relabeled_results_); - // shard id 1 + // Act + relabeler.input_relabeling(lss_, lss_, cache_, hx_, o_, stateless_relabeler, stats_, shards_inner_series_, relabeled_results_); + PerShardRelabeler::append_relabeler_series(lss_, shards_inner_series_[1], relabeled_results_[1], &update_data); + PerShardRelabeler::update_relabeler_state(cache_, &update_data, 1); + + // Assert EXPECT_EQ(relabeled_results_[0]->size(), 0); EXPECT_EQ(relabeled_results_[1]->size(), 1); EXPECT_EQ(shards_inner_series_[0]->size(), 0); - EXPECT_EQ(shards_inner_series_[1]->size(), 0); - EXPECT_EQ(stats_.samples_added, 1); - EXPECT_EQ(stats_.series_added, 1); - - PromPP::Prometheus::Relabel::RelabelerStateUpdate update_data{}; - prs.append_relabeler_series(lss_, shards_inner_series_[1], relabeled_results_[1], &update_data); - EXPECT_EQ(shards_inner_series_[1]->size(), 1); + EXPECT_TRUE(std::ranges::equal(std::vector{{.sample = Sample(1000, 0.1), .ls_id = 1}}, shards_inner_series_[1]->data())); + EXPECT_EQ((Stats{.samples_added = 1, .series_added = 1}), stats_); EXPECT_EQ(update_data.size(), 1); - - prs.update_relabeler_state(cache_, &update_data, 1); - - auto rlabels = lss_[update_data[0].relabeled_ls_id]; - LabelViewSetForTest expected_labels = - make_label_set({{"__name__", "booom"}, {"a_name", "target_a_value"}, {"jab", "baj"}, {"job", "abc"}, {"z_name", "target_z_value"}}); - - EXPECT_EQ(rlabels, expected_labels); + EXPECT_EQ((LabelViewSet{{"__name__", "booom"}, {"a_name", "target_a_value"}, {"jab", "baj"}, {"job", "abc"}, {"z_name", "target_z_value"}}), + lss_[update_data[0].relabeled_ls_id]); } -TEST_F(TestPerShardRelabeler, TargetLabels_ExportedLabel) { - RelabelConfigTest rct{.source_labels = std::vector{"job"}, .regex = "abc", .action = 2}; // Keep - Relabel::StatelessRelabeler sr(std::vector{&rct}); - PromPP::Prometheus::Relabel::PerShardRelabeler prs(external_labels_, &sr, 2, 0); - make_hashdex(hx_, make_label_set({{"__name__", "booom"}, {"jab", "baj"}, {"job", "abc"}}), make_samples({{1712567046855, 0.1}})); - std::vector> list_target_labels{{"jab", "target_a_value"}, {"z_name", "target_z_value"}}; - add_target_labels(list_target_labels); +TEST_F(PerGoroutineRelabelerFixture, TargetLabels_ExportedLabel) { + // Arrange + const RelabelConfig config{.source_labels = {{"job"}}, .regex = "abc", .action = rsKeep}; + const StatelessRelabeler stateless_relabeler(std::initializer_list{&config}); + hx_.emplace_back({{"__name__", "booom"}, {"jab", "baj"}, {"job", "abc"}}, {{1000, 0.1}}); + add_target_labels(LabelViewSet{{"jab", "target_a_value"}, {"z_name", "target_z_value"}}); + PerGoroutineRelabeler relabeler{kNumberOfShards, 0}; + RelabelerStateUpdate update_data; - prs.input_relabeling(lss_, lss_, cache_, hx_, o_, stats_, shards_inner_series_, relabeled_results_); - // shard id 1 + // Act + relabeler.input_relabeling(lss_, lss_, cache_, hx_, o_, stateless_relabeler, stats_, shards_inner_series_, relabeled_results_); + PerShardRelabeler::append_relabeler_series(lss_, shards_inner_series_[0], relabeled_results_[0], &update_data); + PerShardRelabeler::update_relabeler_state(cache_, &update_data, 1); + + // Assert EXPECT_EQ(relabeled_results_[0]->size(), 1); EXPECT_EQ(relabeled_results_[1]->size(), 0); - EXPECT_EQ(shards_inner_series_[0]->size(), 0); + EXPECT_TRUE(std::ranges::equal(std::vector{{.sample = Sample(1000, 0.1), .ls_id = 1}}, shards_inner_series_[0]->data())); EXPECT_EQ(shards_inner_series_[1]->size(), 0); - EXPECT_EQ(stats_.samples_added, 1); - EXPECT_EQ(stats_.series_added, 1); - - PromPP::Prometheus::Relabel::RelabelerStateUpdate update_data{}; - prs.append_relabeler_series(lss_, shards_inner_series_[0], relabeled_results_[0], &update_data); - EXPECT_EQ(shards_inner_series_[0]->size(), 1); + EXPECT_EQ((Stats{.samples_added = 1, .series_added = 1}), stats_); EXPECT_EQ(update_data.size(), 1); - - prs.update_relabeler_state(cache_, &update_data, 1); - - auto rlabels = lss_[update_data[0].relabeled_ls_id]; - LabelViewSetForTest expected_labels = - make_label_set({{"__name__", "booom"}, {"exported_jab", "baj"}, {"jab", "target_a_value"}, {"job", "abc"}, {"z_name", "target_z_value"}}); - - EXPECT_EQ(rlabels, expected_labels); + EXPECT_EQ((LabelViewSet{{"__name__", "booom"}, {"exported_jab", "baj"}, {"jab", "target_a_value"}, {"job", "abc"}, {"z_name", "target_z_value"}}), + lss_[update_data[0].relabeled_ls_id]); } -TEST_F(TestPerShardRelabeler, TargetLabels_ExportedLabel_Honor) { - RelabelConfigTest rct{.source_labels = std::vector{"job"}, .regex = "abc", .action = 2}; // Keep - Relabel::StatelessRelabeler sr(std::vector{&rct}); - PromPP::Prometheus::Relabel::PerShardRelabeler prs(external_labels_, &sr, 2, 0); - make_hashdex(hx_, make_label_set({{"__name__", "booom"}, {"jab", "baj"}, {"job", "abc"}}), make_samples({{1712567046855, 0.1}})); - std::vector> list_target_labels{{"jab", "target_a_value"}, {"z_name", "target_z_value"}}; - add_target_labels(list_target_labels); +TEST_F(PerGoroutineRelabelerFixture, TargetLabels_ExportedLabel_Honor) { + // Arrange + const RelabelConfig config{.source_labels = {{"job"}}, .regex = "abc", .action = rsKeep}; + const StatelessRelabeler stateless_relabeler(std::initializer_list{&config}); + hx_.emplace_back({{"__name__", "booom"}, {"jab", "baj"}, {"job", "abc"}}, {{1000, 0.1}}); + add_target_labels(LabelViewSet{{"jab", "target_a_value"}, {"z_name", "target_z_value"}}); + PerGoroutineRelabeler relabeler{kNumberOfShards, 0}; o_.honor_labels = true; + RelabelerStateUpdate update_data; - prs.input_relabeling(lss_, lss_, cache_, hx_, o_, stats_, shards_inner_series_, relabeled_results_); - // shard id 1 + // Act + relabeler.input_relabeling(lss_, lss_, cache_, hx_, o_, stateless_relabeler, stats_, shards_inner_series_, relabeled_results_); + PerShardRelabeler::append_relabeler_series(lss_, shards_inner_series_[0], relabeled_results_[0], &update_data); + PerShardRelabeler::update_relabeler_state(cache_, &update_data, 1); + + // Assert EXPECT_EQ(relabeled_results_[0]->size(), 1); EXPECT_EQ(relabeled_results_[1]->size(), 0); - EXPECT_EQ(shards_inner_series_[0]->size(), 0); + EXPECT_TRUE(std::ranges::equal(std::vector{{.sample = Sample(1000, 0.1), .ls_id = 1}}, shards_inner_series_[0]->data())); EXPECT_EQ(shards_inner_series_[1]->size(), 0); - EXPECT_EQ(stats_.samples_added, 1); - EXPECT_EQ(stats_.series_added, 1); - - PromPP::Prometheus::Relabel::RelabelerStateUpdate update_data{}; - prs.append_relabeler_series(lss_, shards_inner_series_[0], relabeled_results_[0], &update_data); - EXPECT_EQ(shards_inner_series_[0]->size(), 1); + EXPECT_EQ((Stats{.samples_added = 1, .series_added = 1}), stats_); EXPECT_EQ(update_data.size(), 1); + EXPECT_EQ((LabelViewSet{{"__name__", "booom"}, {"jab", "baj"}, {"job", "abc"}, {"z_name", "target_z_value"}}), lss_[update_data[0].relabeled_ls_id]); +} - prs.update_relabeler_state(cache_, &update_data, 1); +TEST_F(PerGoroutineRelabelerFixture, SampleLimitExceeded) { + // Arrange + const RelabelConfig config{.source_labels = {{"job"}}, .regex = "abc", .action = rKeep}; + const StatelessRelabeler stateless_relabeler(std::initializer_list{&config}); + hx_ = HashdexTest{ + {{{"__name__", "value"}, {"job", "abc"}}, {{1000, kStaleNan}}}, + {{{"__name__", "value"}, {"job", "abc"}}, {{2000, 0.1}}}, + {{{"__name__", "value"}, {"job", "abc"}}, {{3000, 0.1}}}, + }; + MetricLimits limits{.sample_limit = 1}; + PerGoroutineRelabeler relabeler{kNumberOfShards, 1}; + o_.metric_limits = &limits; - auto rlabels = lss_[update_data[0].relabeled_ls_id]; - LabelViewSetForTest expected_labels = make_label_set({{"__name__", "booom"}, {"jab", "baj"}, {"job", "abc"}, {"z_name", "target_z_value"}}); + // Act + relabeler.input_relabeling(lss_, lss_, cache_, hx_, o_, stateless_relabeler, stats_, shards_inner_series_, relabeled_results_); - EXPECT_EQ(rlabels, expected_labels); + // Assert + EXPECT_EQ(relabeled_results_[0]->size(), 0); + EXPECT_EQ(relabeled_results_[1]->size(), 0); + EXPECT_EQ(shards_inner_series_[0]->size(), 0); + EXPECT_TRUE(std::ranges::equal(std::vector{{.sample = Sample(1000, kStaleNan), .ls_id = 0}, {.sample = Sample(2000, 0.1), .ls_id = 0}}, + shards_inner_series_[1]->data())); + EXPECT_EQ((Stats{.samples_added = 2, .series_added = 1}), stats_); } -struct TestTargetLabels : public testing::Test { - // target_labels - std::vector> vector_target_labels_; +class TargetLabelsFixture : public testing::Test { + protected: + static constexpr uint16_t kNumberOfShards = 2; + + std::vector vector_target_labels_; - // Options PromPP::Prometheus::Relabel::RelabelerOptions o_; + std::vector rcts_; + std::vector vector_external_labels_; + SliceView external_labels_{}; - // for init PerShardRelabeler - PromPP::Primitives::SnugComposites::LabelSet::EncodingBimap lss_; - std::vector rcts_; - std::vector> vector_external_labels_; - PromPP::Primitives::Go::SliceView> external_labels_; + LabelsBuilder builder_; + + StatelessRelabeler stateless_relabeler_{rcts_}; + PerShardRelabeler relabeler_{external_labels_, &stateless_relabeler_, kNumberOfShards, 1}; void SetUp() final { o_.target_labels.reset_to(vector_target_labels_.data(), vector_target_labels_.size(), vector_target_labels_.size()); external_labels_.reset_to(vector_external_labels_.data(), vector_external_labels_.size(), vector_external_labels_.size()); } - void add_target_labels(std::vector>& list_target_labels) { + void add_target_labels(const LabelViewSet& list_target_labels) { vector_target_labels_.resize(list_target_labels.size()); for (size_t i = 0; i < vector_target_labels_.size(); i++) { vector_target_labels_[i].first.reset_to(list_target_labels[i].first.data(), list_target_labels[i].first.size()); @@ -1037,226 +600,242 @@ struct TestTargetLabels : public testing::Test { } }; -TEST_F(TestTargetLabels, ResolveConflictingExposedLabels_EmptyConflictingLabels) { - auto labels = make_label_set({{"c_name", "c_value"}}); - PromPP::Primitives::LabelsBuilderStateMap builder_state; - PromPP::Primitives::LabelsBuilder builder{builder_state}; - builder.reset(labels); - Relabel::StatelessRelabeler sr(rcts_); - PromPP::Prometheus::Relabel::PerShardRelabeler prs(external_labels_, &sr, 2, 1); - std::vector conflicting_exposed_labels{}; - LabelViewSetForTest expected_labels = make_label_set({{"c_name", "c_value"}}); +TEST_F(TargetLabelsFixture, ResolveConflictingExposedLabels_EmptyConflictingLabels) { + // Arrange + const LabelViewSet labels{{"c_name", "c_value"}}; + + builder_.reset(labels); - prs.resolve_conflicting_exposed_labels(builder, conflicting_exposed_labels); + std::vector