From 8cb4eb7a091842acbac84ea59853e6c143847225 Mon Sep 17 00:00:00 2001 From: Pushkar Mishra Date: Sun, 12 May 2024 16:14:24 +0530 Subject: [PATCH 01/18] init Signed-off-by: Pushkar Mishra --- .../strategystore/adaptive/aggregator.go | 27 +- .../strategystore/adaptive/aggregator_test.go | 40 +- .../strategystore/adaptive/factory.go | 20 +- .../{processor.go => postaggregator.go} | 154 +-- .../strategystore/adaptive/processor_test.go | 896 ------------------ .../strategystore/adaptive/strategy_store.go | 173 +++- 6 files changed, 254 insertions(+), 1056 deletions(-) rename plugin/sampling/strategystore/adaptive/{processor.go => postaggregator.go} (72%) delete mode 100644 plugin/sampling/strategystore/adaptive/processor_test.go diff --git a/plugin/sampling/strategystore/adaptive/aggregator.go b/plugin/sampling/strategystore/adaptive/aggregator.go index 44d8c9906b9..ed1d17c1ab7 100644 --- a/plugin/sampling/strategystore/adaptive/aggregator.go +++ b/plugin/sampling/strategystore/adaptive/aggregator.go @@ -18,10 +18,14 @@ import ( "sync" "time" + "go.uber.org/zap" + "github.com/jaegertracing/jaeger/cmd/collector/app/sampling/model" "github.com/jaegertracing/jaeger/cmd/collector/app/sampling/strategystore" span_model "github.com/jaegertracing/jaeger/model" + "github.com/jaegertracing/jaeger/pkg/hostname" "github.com/jaegertracing/jaeger/pkg/metrics" + "github.com/jaegertracing/jaeger/plugin/sampling/leaderelection" "github.com/jaegertracing/jaeger/storage/samplingstore" ) @@ -35,6 +39,7 @@ type aggregator struct { operationsCounter metrics.Counter servicesCounter metrics.Counter currentThroughput serviceOperationThroughput + postAggregator *PostAggregator aggregationInterval time.Duration storage samplingstore.Store stop chan struct{} @@ -42,15 +47,27 @@ type aggregator struct { // NewAggregator creates a throughput aggregator that simply emits metrics // about the number of operations seen over the aggregationInterval. -func NewAggregator(metricsFactory metrics.Factory, interval time.Duration, storage samplingstore.Store) strategystore.Aggregator { +func NewAggregator(options Options, logger *zap.Logger, metricsFactory metrics.Factory, participant leaderelection.ElectionParticipant, store samplingstore.Store) (strategystore.Aggregator, error) { + hostname, err := hostname.AsIdentifier() + if err != nil { + return nil, err + } + logger.Info("Using unique participantName in adaptive sampling", zap.String("participantName", hostname)) + + postAgg, err := newPostAggregator(options, hostname, store, participant, metricsFactory, logger) + if err != nil { + return nil, err + } + return &aggregator{ operationsCounter: metricsFactory.Counter(metrics.Options{Name: "sampling_operations"}), servicesCounter: metricsFactory.Counter(metrics.Options{Name: "sampling_services"}), currentThroughput: make(serviceOperationThroughput), - aggregationInterval: interval, - storage: storage, + aggregationInterval: options.CalculationInterval, + postAggregator: postAgg, + storage: store, stop: make(chan struct{}), - } + }, nil } func (a *aggregator) runAggregationLoop() { @@ -112,9 +129,11 @@ func (a *aggregator) RecordThroughput(service, operation string, samplerType spa func (a *aggregator) Start() { go a.runAggregationLoop() + a.postAggregator.Start() } func (a *aggregator) Close() error { close(a.stop) + a.postAggregator.Close() return nil } diff --git a/plugin/sampling/strategystore/adaptive/aggregator_test.go b/plugin/sampling/strategystore/adaptive/aggregator_test.go index 781c51e72b0..84f3f66f170 100644 --- a/plugin/sampling/strategystore/adaptive/aggregator_test.go +++ b/plugin/sampling/strategystore/adaptive/aggregator_test.go @@ -20,9 +20,12 @@ import ( "github.com/stretchr/testify/assert" "github.com/stretchr/testify/mock" + "github.com/stretchr/testify/require" + "go.uber.org/zap" "github.com/jaegertracing/jaeger/internal/metricstest" "github.com/jaegertracing/jaeger/model" + epmocks "github.com/jaegertracing/jaeger/plugin/sampling/leaderelection/mocks" "github.com/jaegertracing/jaeger/storage/samplingstore/mocks" ) @@ -32,8 +35,19 @@ func TestAggregator(t *testing.T) { mockStorage := &mocks.Store{} mockStorage.On("InsertThroughput", mock.AnythingOfType("[]*model.Throughput")).Return(nil) + mockEP := &epmocks.ElectionParticipant{} + mockEP.On("Start").Return(nil) + mockEP.On("Close").Return(nil) + mockEP.On("IsLeader").Return(true) + testOpts := Options{ + CalculationInterval: 1 * time.Second, + AggregationBuckets: 1, + BucketsForCalculation: 1, + } + logger := zap.NewNop() - a := NewAggregator(metricsFactory, 5*time.Millisecond, mockStorage) + a, err := NewAggregator(testOpts, logger, metricsFactory, mockEP, mockStorage) + require.NoError(t, err) a.RecordThroughput("A", "GET", model.SamplerTypeProbabilistic, 0.001) a.RecordThroughput("B", "POST", model.SamplerTypeProbabilistic, 0.001) a.RecordThroughput("C", "GET", model.SamplerTypeProbabilistic, 0.001) @@ -60,15 +74,23 @@ func TestAggregator(t *testing.T) { func TestIncrementThroughput(t *testing.T) { metricsFactory := metricstest.NewFactory(0) mockStorage := &mocks.Store{} - - a := NewAggregator(metricsFactory, 5*time.Millisecond, mockStorage) + mockEP := &epmocks.ElectionParticipant{} + testOpts := Options{ + CalculationInterval: 1 * time.Second, + AggregationBuckets: 1, + BucketsForCalculation: 1, + } + logger := zap.NewNop() + a, err := NewAggregator(testOpts, logger, metricsFactory, mockEP, mockStorage) + require.NoError(t, err) // 20 different probabilities for i := 0; i < 20; i++ { a.RecordThroughput("A", "GET", model.SamplerTypeProbabilistic, 0.001*float64(i)) } assert.Len(t, a.(*aggregator).currentThroughput["A"]["GET"].Probabilities, 10) - a = NewAggregator(metricsFactory, 5*time.Millisecond, mockStorage) + a, err = NewAggregator(testOpts, logger, metricsFactory, mockEP, mockStorage) + require.NoError(t, err) // 20 of the same probabilities for i := 0; i < 20; i++ { a.RecordThroughput("A", "GET", model.SamplerTypeProbabilistic, 0.001) @@ -79,8 +101,16 @@ func TestIncrementThroughput(t *testing.T) { func TestLowerboundThroughput(t *testing.T) { metricsFactory := metricstest.NewFactory(0) mockStorage := &mocks.Store{} + mockEP := &epmocks.ElectionParticipant{} + testOpts := Options{ + CalculationInterval: 1 * time.Second, + AggregationBuckets: 1, + BucketsForCalculation: 1, + } + logger := zap.NewNop() - a := NewAggregator(metricsFactory, 5*time.Millisecond, mockStorage) + a, err := NewAggregator(testOpts, logger, metricsFactory, mockEP, mockStorage) + require.NoError(t, err) a.RecordThroughput("A", "GET", model.SamplerTypeLowerBound, 0.001) assert.EqualValues(t, 0, a.(*aggregator).currentThroughput["A"]["GET"].Count) assert.Empty(t, a.(*aggregator).currentThroughput["A"]["GET"].Probabilities["0.001000"]) diff --git a/plugin/sampling/strategystore/adaptive/factory.go b/plugin/sampling/strategystore/adaptive/factory.go index d45f8e61f9b..317644b6eea 100644 --- a/plugin/sampling/strategystore/adaptive/factory.go +++ b/plugin/sampling/strategystore/adaptive/factory.go @@ -25,6 +25,7 @@ import ( "github.com/jaegertracing/jaeger/pkg/distributedlock" "github.com/jaegertracing/jaeger/pkg/metrics" "github.com/jaegertracing/jaeger/plugin" + "github.com/jaegertracing/jaeger/plugin/sampling/leaderelection" "github.com/jaegertracing/jaeger/storage" "github.com/jaegertracing/jaeger/storage/samplingstore" ) @@ -83,12 +84,23 @@ func (f *Factory) Initialize(metricsFactory metrics.Factory, ssFactory storage.S // CreateStrategyStore implements strategystore.Factory func (f *Factory) CreateStrategyStore() (strategystore.StrategyStore, strategystore.Aggregator, error) { - p, err := NewStrategyStore(*f.options, f.metricsFactory, f.logger, f.lock, f.store) + participant := leaderelection.NewElectionParticipant(f.lock, defaultResourceName, leaderelection.ElectionParticipantOptions{ + FollowerLeaseRefreshInterval: f.options.FollowerLeaseRefreshInterval, + LeaderLeaseRefreshInterval: f.options.LeaderLeaseRefreshInterval, + Logger: f.logger, + }) + + ss, err := NewStrategyStore(*f.options, f.logger, participant, f.store) + if err != nil { + return nil, nil, err + } + ss.Start() + + a, err := NewAggregator(*f.options, f.logger, f.metricsFactory, participant, f.store) if err != nil { return nil, nil, err } - p.Start() - a := NewAggregator(f.metricsFactory, f.options.CalculationInterval, f.store) a.Start() - return p, a, nil + + return ss, a, nil } diff --git a/plugin/sampling/strategystore/adaptive/processor.go b/plugin/sampling/strategystore/adaptive/postaggregator.go similarity index 72% rename from plugin/sampling/strategystore/adaptive/processor.go rename to plugin/sampling/strategystore/adaptive/postaggregator.go index 7ac5c1834fa..69437cf907c 100644 --- a/plugin/sampling/strategystore/adaptive/processor.go +++ b/plugin/sampling/strategystore/adaptive/postaggregator.go @@ -1,4 +1,4 @@ -// Copyright (c) 2018 The Jaeger Authors. +// Copyright (c) 2024 The Jaeger Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. @@ -15,7 +15,6 @@ package adaptive import ( - "context" "errors" "math" "math/rand" @@ -28,7 +27,6 @@ import ( "github.com/jaegertracing/jaeger/pkg/metrics" "github.com/jaegertracing/jaeger/plugin/sampling/calculationstrategy" "github.com/jaegertracing/jaeger/plugin/sampling/leaderelection" - "github.com/jaegertracing/jaeger/proto-gen/api_v2" "github.com/jaegertracing/jaeger/storage/samplingstore" ) @@ -37,8 +35,6 @@ const ( getThroughputErrMsg = "failed to get throughput from storage" - defaultFollowerProbabilityInterval = 20 * time.Second - // The number of past entries for samplingCache the leader keeps in memory serviceCacheSize = 25 @@ -71,11 +67,11 @@ type throughputBucket struct { endTime time.Time } -// Processor retrieves service throughput over a look back interval and calculates sampling probabilities +// PostAggregator retrieves service throughput over a look back interval and calculates sampling probabilities // per operation such that each operation is sampled at a specified target QPS. It achieves this by // retrieving discrete buckets of operation throughput and doing a weighted average of the throughput // and generating a probability to match the targetQPS. -type Processor struct { +type PostAggregator struct { sync.RWMutex Options @@ -95,19 +91,10 @@ type Processor struct { // The latest throughput is stored at the head of the slice. throughputs []*throughputBucket - // strategyResponses is the cache of the sampling strategies for every service, in Thrift format. - // TODO change this to work with protobuf model instead, to support gRPC endpoint. - strategyResponses map[string]*api_v2.SamplingStrategyResponse - weightVectorCache *WeightVectorCache probabilityCalculator calculationstrategy.ProbabilityCalculator - // followerRefreshInterval determines how often the follower processor updates its probabilities. - // Given only the leader writes probabilities, the followers need to fetch the probabilities into - // cache. - followerRefreshInterval time.Duration - serviceCache []SamplingCache shutdown chan struct{} @@ -118,14 +105,14 @@ type Processor struct { } // newProcessor creates a new sampling processor that generates sampling rates for service operations. -func newProcessor( +func newPostAggregator( opts Options, hostname string, storage samplingstore.Store, electionParticipant leaderelection.ElectionParticipant, metricsFactory metrics.Factory, logger *zap.Logger, -) (*Processor, error) { +) (*PostAggregator, error) { if opts.CalculationInterval == 0 || opts.AggregationBuckets == 0 { return nil, errNonZero } @@ -133,50 +120,35 @@ func newProcessor( return nil, errBucketsForCalculation } metricsFactory = metricsFactory.Namespace(metrics.NSOptions{Name: "adaptive_sampling_processor"}) - return &Processor{ + return &PostAggregator{ Options: opts, storage: storage, probabilities: make(model.ServiceOperationProbabilities), qps: make(model.ServiceOperationQPS), hostname: hostname, - strategyResponses: make(map[string]*api_v2.SamplingStrategyResponse), logger: logger, electionParticipant: electionParticipant, // TODO make weightsCache and probabilityCalculator configurable weightVectorCache: NewWeightVectorCache(), probabilityCalculator: calculationstrategy.NewPercentageIncreaseCappedCalculator(1.0), - followerRefreshInterval: defaultFollowerProbabilityInterval, serviceCache: []SamplingCache{}, operationsCalculatedGauge: metricsFactory.Gauge(metrics.Options{Name: "operations_calculated"}), calculateProbabilitiesLatency: metricsFactory.Timer(metrics.TimerOptions{Name: "calculate_probabilities"}), }, nil } -// GetSamplingStrategy implements Thrift endpoint for retrieving sampling strategy for a service. -func (p *Processor) GetSamplingStrategy(_ context.Context, service string) (*api_v2.SamplingStrategyResponse, error) { - p.RLock() - defer p.RUnlock() - if strategy, ok := p.strategyResponses[service]; ok { - return strategy, nil - } - return p.generateDefaultSamplingStrategyResponse(), nil -} - // Start initializes and starts the sampling processor which regularly calculates sampling probabilities. -func (p *Processor) Start() error { +func (p *PostAggregator) Start() error { p.logger.Info("starting adaptive sampling processor") if err := p.electionParticipant.Start(); err != nil { return err } p.shutdown = make(chan struct{}) - p.loadProbabilities() - p.generateStrategyResponses() p.runBackground(p.runCalculationLoop) - p.runBackground(p.runUpdateProbabilitiesLoop) return nil } -func (p *Processor) runBackground(f func()) { +func (p *PostAggregator) runBackground(f func()) { p.bgFinished.Add(1) go func() { f() @@ -185,7 +157,7 @@ func (p *Processor) runBackground(f func()) { } // Close stops the processor from calculating probabilities. -func (p *Processor) Close() error { +func (p *PostAggregator) Close() error { p.logger.Info("stopping adaptive sampling processor") err := p.electionParticipant.Close() if p.shutdown != nil { @@ -195,44 +167,7 @@ func (p *Processor) Close() error { return err } -func (p *Processor) loadProbabilities() { - // TODO GetLatestProbabilities API can be changed to return the latest measured qps for initialization - probabilities, err := p.storage.GetLatestProbabilities() - if err != nil { - p.logger.Warn("failed to initialize probabilities", zap.Error(err)) - return - } - p.Lock() - defer p.Unlock() - p.probabilities = probabilities -} - -// runUpdateProbabilitiesLoop is a loop that reads probabilities from storage. -// The follower updates its local cache with the latest probabilities and serves them. -func (p *Processor) runUpdateProbabilitiesLoop() { - select { - case <-time.After(addJitter(p.followerRefreshInterval)): - case <-p.shutdown: - return - } - - ticker := time.NewTicker(p.followerRefreshInterval) - defer ticker.Stop() - for { - select { - case <-ticker.C: - // Only load probabilities if this processor doesn't hold the leader lock - if !p.isLeader() { - p.loadProbabilities() - p.generateStrategyResponses() - } - case <-p.shutdown: - return - } - } -} - -func (p *Processor) isLeader() bool { +func (p *PostAggregator) isLeader() bool { return p.electionParticipant.IsLeader() } @@ -244,7 +179,7 @@ func addJitter(jitterAmount time.Duration) time.Duration { return (jitterAmount / 2) + time.Duration(rand.Int63n(int64(jitterAmount/2))) } -func (p *Processor) runCalculationLoop() { +func (p *PostAggregator) runCalculationLoop() { lastCheckedTime := time.Now().Add(p.Delay * -1) p.initializeThroughput(lastCheckedTime) // NB: the first tick will be slightly delayed by the initializeThroughput call. @@ -278,13 +213,7 @@ func (p *Processor) runCalculationLoop() { p.probabilities = probabilities p.qps = qps p.Unlock() - // NB: This has the potential of running into a race condition if the CalculationInterval - // is set to an extremely low value. The worst case scenario is that probabilities is calculated - // and swapped more than once before generateStrategyResponses() and saveProbabilities() are called. - // This will result in one or more batches of probabilities not being saved which is completely - // fine. This race condition should not ever occur anyway since the calculation interval will - // be way longer than the time to run the calculations. - p.generateStrategyResponses() + p.calculateProbabilitiesLatency.Record(time.Since(startTime)) p.runBackground(p.saveProbabilitiesAndQPS) } @@ -294,7 +223,7 @@ func (p *Processor) runCalculationLoop() { } } -func (p *Processor) saveProbabilitiesAndQPS() { +func (p *PostAggregator) saveProbabilitiesAndQPS() { p.RLock() defer p.RUnlock() if err := p.storage.InsertProbabilitiesAndQPS(p.hostname, p.probabilities, p.qps); err != nil { @@ -302,7 +231,7 @@ func (p *Processor) saveProbabilitiesAndQPS() { } } -func (p *Processor) prependThroughputBucket(bucket *throughputBucket) { +func (p *PostAggregator) prependThroughputBucket(bucket *throughputBucket) { p.throughputs = append([]*throughputBucket{bucket}, p.throughputs...) if len(p.throughputs) > p.AggregationBuckets { p.throughputs = p.throughputs[0:p.AggregationBuckets] @@ -312,7 +241,7 @@ func (p *Processor) prependThroughputBucket(bucket *throughputBucket) { // aggregateThroughput aggregates operation throughput from different buckets into one. // All input buckets represent a single time range, but there are many of them because // they are all independently generated by different collector instances from inbound span traffic. -func (p *Processor) aggregateThroughput(throughputs []*model.Throughput) serviceOperationThroughput { +func (p *PostAggregator) aggregateThroughput(throughputs []*model.Throughput) serviceOperationThroughput { aggregatedThroughput := make(serviceOperationThroughput) for _, throughput := range throughputs { service := throughput.Service @@ -344,7 +273,7 @@ func copySet(in map[string]struct{}) map[string]struct{} { return out } -func (p *Processor) initializeThroughput(endTime time.Time) { +func (p *PostAggregator) initializeThroughput(endTime time.Time) { for i := 0; i < p.AggregationBuckets; i++ { startTime := endTime.Add(p.CalculationInterval * -1) throughput, err := p.storage.GetThroughput(startTime, endTime) @@ -366,7 +295,7 @@ func (p *Processor) initializeThroughput(endTime time.Time) { } // throughputToQPS converts raw throughput counts for all accumulated buckets to QPS values. -func (p *Processor) throughputToQPS() serviceOperationQPS { +func (p *PostAggregator) throughputToQPS() serviceOperationQPS { // TODO previous qps buckets have already been calculated, just need to calculate latest batch // and append them where necessary and throw out the oldest batch. // Edge case #buckets < p.AggregationBuckets, then we shouldn't throw out @@ -394,7 +323,7 @@ func calculateQPS(count int64, interval time.Duration) float64 { // calculateWeightedQPS calculates the weighted qps of the slice allQPS where weights are biased // towards more recent qps. This function assumes that the most recent qps is at the head of the slice. -func (p *Processor) calculateWeightedQPS(allQPS []float64) float64 { +func (p *PostAggregator) calculateWeightedQPS(allQPS []float64) float64 { if len(allQPS) == 0 { return 0 } @@ -406,14 +335,14 @@ func (p *Processor) calculateWeightedQPS(allQPS []float64) float64 { return qps } -func (p *Processor) prependServiceCache() { +func (p *PostAggregator) prependServiceCache() { p.serviceCache = append([]SamplingCache{make(SamplingCache)}, p.serviceCache...) if len(p.serviceCache) > serviceCacheSize { p.serviceCache = p.serviceCache[0:serviceCacheSize] } } -func (p *Processor) calculateProbabilitiesAndQPS() (model.ServiceOperationProbabilities, model.ServiceOperationQPS) { +func (p *PostAggregator) calculateProbabilitiesAndQPS() (model.ServiceOperationProbabilities, model.ServiceOperationQPS) { p.prependServiceCache() retProbabilities := make(model.ServiceOperationProbabilities) retQPS := make(model.ServiceOperationQPS) @@ -437,7 +366,7 @@ func (p *Processor) calculateProbabilitiesAndQPS() (model.ServiceOperationProbab return retProbabilities, retQPS } -func (p *Processor) calculateProbability(service, operation string, qps float64) float64 { +func (p *PostAggregator) calculateProbability(service, operation string, qps float64) float64 { oldProbability := p.InitialSamplingProbability // TODO: is this loop overly expensive? p.RLock() @@ -472,7 +401,7 @@ func (p *Processor) calculateProbability(service, operation string, qps float64) } // is actual value within p.DeltaTolerance percentage of expected value. -func (p *Processor) withinTolerance(actual, expected float64) bool { +func (p *PostAggregator) withinTolerance(actual, expected float64) bool { return math.Abs(actual-expected)/expected < p.DeltaTolerance } @@ -484,7 +413,7 @@ func merge(p1 map[string]struct{}, p2 map[string]struct{}) map[string]struct{} { return p1 } -func (p *Processor) isUsingAdaptiveSampling( +func (p *PostAggregator) isUsingAdaptiveSampling( probability float64, service string, operation string, @@ -511,40 +440,3 @@ func (p *Processor) isUsingAdaptiveSampling( } return false } - -// generateStrategyResponses generates and caches SamplingStrategyResponse from the calculated sampling probabilities. -func (p *Processor) generateStrategyResponses() { - p.RLock() - strategies := make(map[string]*api_v2.SamplingStrategyResponse) - for svc, opProbabilities := range p.probabilities { - opStrategies := make([]*api_v2.OperationSamplingStrategy, len(opProbabilities)) - var idx int - for op, probability := range opProbabilities { - opStrategies[idx] = &api_v2.OperationSamplingStrategy{ - Operation: op, - ProbabilisticSampling: &api_v2.ProbabilisticSamplingStrategy{ - SamplingRate: probability, - }, - } - idx++ - } - strategy := p.generateDefaultSamplingStrategyResponse() - strategy.OperationSampling.PerOperationStrategies = opStrategies - strategies[svc] = strategy - } - p.RUnlock() - - p.Lock() - defer p.Unlock() - p.strategyResponses = strategies -} - -func (p *Processor) generateDefaultSamplingStrategyResponse() *api_v2.SamplingStrategyResponse { - return &api_v2.SamplingStrategyResponse{ - StrategyType: api_v2.SamplingStrategyType_PROBABILISTIC, - OperationSampling: &api_v2.PerOperationSamplingStrategies{ - DefaultSamplingProbability: p.InitialSamplingProbability, - DefaultLowerBoundTracesPerSecond: p.MinSamplesPerSecond, - }, - } -} diff --git a/plugin/sampling/strategystore/adaptive/processor_test.go b/plugin/sampling/strategystore/adaptive/processor_test.go deleted file mode 100644 index 03ae6b7b7bb..00000000000 --- a/plugin/sampling/strategystore/adaptive/processor_test.go +++ /dev/null @@ -1,896 +0,0 @@ -// Copyright (c) 2018 The Jaeger Authors. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -package adaptive - -import ( - "context" - "errors" - "testing" - "time" - - "github.com/stretchr/testify/assert" - "github.com/stretchr/testify/mock" - "github.com/stretchr/testify/require" - "go.uber.org/zap" - - "github.com/jaegertracing/jaeger/cmd/collector/app/sampling/model" - "github.com/jaegertracing/jaeger/internal/metricstest" - "github.com/jaegertracing/jaeger/pkg/metrics" - "github.com/jaegertracing/jaeger/pkg/testutils" - "github.com/jaegertracing/jaeger/plugin/sampling/calculationstrategy" - epmocks "github.com/jaegertracing/jaeger/plugin/sampling/leaderelection/mocks" - "github.com/jaegertracing/jaeger/proto-gen/api_v2" - smocks "github.com/jaegertracing/jaeger/storage/samplingstore/mocks" -) - -func testThroughputs() []*model.Throughput { - return []*model.Throughput{ - {Service: "svcA", Operation: "GET", Count: 4, Probabilities: map[string]struct{}{"0.1": {}}}, - {Service: "svcA", Operation: "GET", Count: 4, Probabilities: map[string]struct{}{"0.2": {}}}, - {Service: "svcA", Operation: "PUT", Count: 5, Probabilities: map[string]struct{}{"0.1": {}}}, - {Service: "svcB", Operation: "GET", Count: 3, Probabilities: map[string]struct{}{"0.1": {}}}, - } -} - -func testThroughputBuckets() []*throughputBucket { - return []*throughputBucket{ - { - throughput: serviceOperationThroughput{ - "svcA": map[string]*model.Throughput{ - "GET": {Count: 45}, - "PUT": {Count: 60}, - }, - "svcB": map[string]*model.Throughput{ - "GET": {Count: 30}, - "PUT": {Count: 15}, - }, - }, - interval: 60 * time.Second, - }, - { - throughput: serviceOperationThroughput{ - "svcA": map[string]*model.Throughput{ - "GET": {Count: 30}, - }, - "svcB": map[string]*model.Throughput{ - "GET": {Count: 45}, - }, - }, - interval: 60 * time.Second, - }, - } -} - -func errTestStorage() error { - return errors.New("storage error") -} - -func testCalculator() calculationstrategy.ProbabilityCalculator { - return calculationstrategy.CalculateFunc(func(targetQPS, qps, oldProbability float64) float64 { - factor := targetQPS / qps - return oldProbability * factor - }) -} - -func TestAggregateThroughputInputsImmutability(t *testing.T) { - p := &Processor{} - in := testThroughputs() - _ = p.aggregateThroughput(in) - assert.Equal(t, in, testThroughputs()) -} - -func TestAggregateThroughput(t *testing.T) { - p := &Processor{} - aggregatedThroughput := p.aggregateThroughput(testThroughputs()) - require.Len(t, aggregatedThroughput, 2) - - throughput, ok := aggregatedThroughput["svcA"] - require.True(t, ok) - require.Len(t, throughput, 2) - - opThroughput, ok := throughput["GET"] - require.True(t, ok) - assert.Equal(t, int64(8), opThroughput.Count) - assert.Equal(t, map[string]struct{}{"0.1": {}, "0.2": {}}, opThroughput.Probabilities) - - opThroughput, ok = throughput["PUT"] - require.True(t, ok) - assert.Equal(t, int64(5), opThroughput.Count) - assert.Equal(t, map[string]struct{}{"0.1": {}}, opThroughput.Probabilities) - - throughput, ok = aggregatedThroughput["svcB"] - require.True(t, ok) - require.Len(t, throughput, 1) - - opThroughput, ok = throughput["GET"] - require.True(t, ok) - assert.Equal(t, int64(3), opThroughput.Count) - assert.Equal(t, map[string]struct{}{"0.1": {}}, opThroughput.Probabilities) -} - -func TestInitializeThroughput(t *testing.T) { - mockStorage := &smocks.Store{} - mockStorage.On("GetThroughput", time.Time{}.Add(time.Minute*19), time.Time{}.Add(time.Minute*20)). - Return(testThroughputs(), nil) - mockStorage.On("GetThroughput", time.Time{}.Add(time.Minute*18), time.Time{}.Add(time.Minute*19)). - Return([]*model.Throughput{{Service: "svcA", Operation: "GET", Count: 7}}, nil) - mockStorage.On("GetThroughput", time.Time{}.Add(time.Minute*17), time.Time{}.Add(time.Minute*18)). - Return([]*model.Throughput{}, nil) - p := &Processor{storage: mockStorage, Options: Options{CalculationInterval: time.Minute, AggregationBuckets: 3}} - p.initializeThroughput(time.Time{}.Add(time.Minute * 20)) - - require.Len(t, p.throughputs, 2) - require.Len(t, p.throughputs[0].throughput, 2) - assert.Equal(t, time.Minute, p.throughputs[0].interval) - assert.Equal(t, p.throughputs[0].endTime, time.Time{}.Add(time.Minute*20)) - require.Len(t, p.throughputs[1].throughput, 1) - assert.Equal(t, time.Minute, p.throughputs[1].interval) - assert.Equal(t, p.throughputs[1].endTime, time.Time{}.Add(time.Minute*19)) -} - -func TestInitializeThroughputFailure(t *testing.T) { - mockStorage := &smocks.Store{} - mockStorage.On("GetThroughput", time.Time{}.Add(time.Minute*19), time.Time{}.Add(time.Minute*20)). - Return(nil, errTestStorage()) - p := &Processor{storage: mockStorage, Options: Options{CalculationInterval: time.Minute, AggregationBuckets: 1}} - p.initializeThroughput(time.Time{}.Add(time.Minute * 20)) - - assert.Empty(t, p.throughputs) -} - -func TestCalculateQPS(t *testing.T) { - qps := calculateQPS(int64(90), 60*time.Second) - assert.Equal(t, 1.5, qps) - - qps = calculateQPS(int64(45), 60*time.Second) - assert.Equal(t, 0.75, qps) -} - -func TestGenerateOperationQPS(t *testing.T) { - p := &Processor{throughputs: testThroughputBuckets(), Options: Options{BucketsForCalculation: 10, AggregationBuckets: 10}} - svcOpQPS := p.throughputToQPS() - assert.Len(t, svcOpQPS, 2) - - opQPS, ok := svcOpQPS["svcA"] - require.True(t, ok) - require.Len(t, opQPS, 2) - - assert.Equal(t, []float64{0.75, 0.5}, opQPS["GET"]) - assert.Equal(t, []float64{1.0}, opQPS["PUT"]) - - opQPS, ok = svcOpQPS["svcB"] - require.True(t, ok) - require.Len(t, opQPS, 2) - - assert.Equal(t, []float64{0.5, 0.75}, opQPS["GET"]) - assert.Equal(t, []float64{0.25}, opQPS["PUT"]) - - // Test using the previous QPS if the throughput is not provided - p.prependThroughputBucket( - &throughputBucket{ - throughput: serviceOperationThroughput{ - "svcA": map[string]*model.Throughput{ - "GET": {Count: 30}, - }, - }, - interval: 60 * time.Second, - }, - ) - svcOpQPS = p.throughputToQPS() - require.Len(t, svcOpQPS, 2) - - opQPS, ok = svcOpQPS["svcA"] - require.True(t, ok) - require.Len(t, opQPS, 2) - - assert.Equal(t, []float64{0.5, 0.75, 0.5}, opQPS["GET"]) - assert.Equal(t, []float64{1.0}, opQPS["PUT"]) - - opQPS, ok = svcOpQPS["svcB"] - require.True(t, ok) - require.Len(t, opQPS, 2) - - assert.Equal(t, []float64{0.5, 0.75}, opQPS["GET"]) - assert.Equal(t, []float64{0.25}, opQPS["PUT"]) -} - -func TestGenerateOperationQPS_UseMostRecentBucketOnly(t *testing.T) { - p := &Processor{throughputs: testThroughputBuckets(), Options: Options{BucketsForCalculation: 1, AggregationBuckets: 10}} - svcOpQPS := p.throughputToQPS() - assert.Len(t, svcOpQPS, 2) - - opQPS, ok := svcOpQPS["svcA"] - require.True(t, ok) - require.Len(t, opQPS, 2) - - assert.Equal(t, []float64{0.75}, opQPS["GET"]) - assert.Equal(t, []float64{1.0}, opQPS["PUT"]) - - p.prependThroughputBucket( - &throughputBucket{ - throughput: serviceOperationThroughput{ - "svcA": map[string]*model.Throughput{ - "GET": {Count: 30}, - }, - }, - interval: 60 * time.Second, - }, - ) - - svcOpQPS = p.throughputToQPS() - require.Len(t, svcOpQPS, 2) - - opQPS, ok = svcOpQPS["svcA"] - require.True(t, ok) - require.Len(t, opQPS, 2) - - assert.Equal(t, []float64{0.5}, opQPS["GET"]) - assert.Equal(t, []float64{1.0}, opQPS["PUT"]) -} - -func TestCalculateWeightedQPS(t *testing.T) { - p := Processor{weightVectorCache: NewWeightVectorCache()} - assert.InDelta(t, 0.86735, p.calculateWeightedQPS([]float64{0.8, 1.2, 1.0}), 0.001) - assert.InDelta(t, 0.95197, p.calculateWeightedQPS([]float64{1.0, 1.0, 0.0, 0.0}), 0.001) - assert.Equal(t, 0.0, p.calculateWeightedQPS([]float64{})) -} - -func TestCalculateProbability(t *testing.T) { - throughputs := []*throughputBucket{ - { - throughput: serviceOperationThroughput{ - "svcA": map[string]*model.Throughput{ - "GET": {Probabilities: map[string]struct{}{"0.500000": {}}}, - }, - }, - }, - } - probabilities := model.ServiceOperationProbabilities{ - "svcA": map[string]float64{ - "GET": 0.5, - }, - } - cfg := Options{ - TargetSamplesPerSecond: 1.0, - DeltaTolerance: 0.2, - InitialSamplingProbability: 0.001, - MinSamplingProbability: 0.00001, - } - p := &Processor{ - Options: cfg, - probabilities: probabilities, - probabilityCalculator: testCalculator(), - throughputs: throughputs, - serviceCache: []SamplingCache{{"svcA": {}, "svcB": {}}}, - } - tests := []struct { - service string - operation string - qps float64 - expectedProbability float64 - errMsg string - }{ - {"svcA", "GET", 2.0, 0.25, "modify existing probability"}, - {"svcA", "PUT", 2.0, 0.0005, "modify default probability"}, - {"svcB", "GET", 0.9, 0.001, "qps within equivalence threshold"}, - {"svcB", "PUT", 0.000001, 1.0, "test max probability"}, - {"svcB", "DELETE", 1000000000, 0.00001, "test min probability"}, - {"svcB", "DELETE", 0.0, 0.002, "test 0 qps"}, - } - for _, test := range tests { - probability := p.calculateProbability(test.service, test.operation, test.qps) - assert.Equal(t, test.expectedProbability, probability, test.errMsg) - } -} - -func TestCalculateProbabilitiesAndQPS(t *testing.T) { - prevProbabilities := model.ServiceOperationProbabilities{ - "svcB": map[string]float64{ - "GET": 0.16, - "PUT": 0.03, - }, - } - qps := model.ServiceOperationQPS{ - "svcB": map[string]float64{ - "GET": 0.625, - }, - } - mets := metricstest.NewFactory(0) - p := &Processor{ - Options: Options{ - TargetSamplesPerSecond: 1.0, - DeltaTolerance: 0.2, - InitialSamplingProbability: 0.001, - BucketsForCalculation: 10, - }, - throughputs: testThroughputBuckets(), probabilities: prevProbabilities, qps: qps, - weightVectorCache: NewWeightVectorCache(), probabilityCalculator: testCalculator(), - operationsCalculatedGauge: mets.Gauge(metrics.Options{Name: "test"}), - } - probabilities, qps := p.calculateProbabilitiesAndQPS() - - require.Len(t, probabilities, 2) - assert.Equal(t, map[string]float64{"GET": 0.00136, "PUT": 0.001}, probabilities["svcA"]) - assert.Equal(t, map[string]float64{"GET": 0.16, "PUT": 0.03}, probabilities["svcB"]) - - require.Len(t, qps, 2) - assert.Equal(t, map[string]float64{"GET": 0.7352941176470588, "PUT": 1}, qps["svcA"]) - assert.Equal(t, map[string]float64{"GET": 0.5147058823529411, "PUT": 0.25}, qps["svcB"]) - - _, gauges := mets.Backend.Snapshot() - assert.EqualValues(t, 4, gauges["test"]) -} - -func TestRunCalculationLoop(t *testing.T) { - logger := zap.NewNop() - mockStorage := &smocks.Store{} - mockStorage.On("GetThroughput", mock.AnythingOfType("time.Time"), mock.AnythingOfType("time.Time")). - Return(testThroughputs(), nil) - mockStorage.On("GetLatestProbabilities").Return(model.ServiceOperationProbabilities{}, errTestStorage()) - mockStorage.On("InsertProbabilitiesAndQPS", "host", mock.AnythingOfType("model.ServiceOperationProbabilities"), - mock.AnythingOfType("model.ServiceOperationQPS")).Return(errTestStorage()) - mockEP := &epmocks.ElectionParticipant{} - mockEP.On("Start").Return(nil) - mockEP.On("Close").Return(nil) - mockEP.On("IsLeader").Return(true) - - cfg := Options{ - TargetSamplesPerSecond: 1.0, - DeltaTolerance: 0.1, - InitialSamplingProbability: 0.001, - CalculationInterval: time.Millisecond * 5, - AggregationBuckets: 2, - Delay: time.Millisecond * 5, - LeaderLeaseRefreshInterval: time.Millisecond, - FollowerLeaseRefreshInterval: time.Second, - BucketsForCalculation: 10, - } - p, err := newProcessor(cfg, "host", mockStorage, mockEP, metrics.NullFactory, logger) - require.NoError(t, err) - p.Start() - - for i := 0; i < 1000; i++ { - strategy, _ := p.GetSamplingStrategy(context.Background(), "svcA") - if len(strategy.OperationSampling.PerOperationStrategies) != 0 { - break - } - time.Sleep(time.Millisecond) - } - p.Close() - - strategy, err := p.GetSamplingStrategy(context.Background(), "svcA") - require.NoError(t, err) - assert.Len(t, strategy.OperationSampling.PerOperationStrategies, 2) -} - -func TestRunCalculationLoop_GetThroughputError(t *testing.T) { - logger, logBuffer := testutils.NewLogger() - mockStorage := &smocks.Store{} - mockStorage.On("GetThroughput", mock.AnythingOfType("time.Time"), mock.AnythingOfType("time.Time")). - Return(nil, errTestStorage()) - mockEP := &epmocks.ElectionParticipant{} - mockEP.On("Start").Return(nil) - mockEP.On("Close").Return(nil) - mockEP.On("IsLeader").Return(false) - - cfg := Options{ - CalculationInterval: time.Millisecond * 5, - AggregationBuckets: 2, - BucketsForCalculation: 10, - } - p, err := newProcessor(cfg, "host", mockStorage, mockEP, metrics.NullFactory, logger) - require.NoError(t, err) - p.shutdown = make(chan struct{}) - defer close(p.shutdown) - go p.runCalculationLoop() - - for i := 0; i < 1000; i++ { - // match logs specific to getThroughputErrMsg. We expect to see more than 2, once during - // initialization and one or more times during the loop. - if match, _ := testutils.LogMatcher(2, getThroughputErrMsg, logBuffer.Lines()); match { - break - } - time.Sleep(time.Millisecond) - } - match, errMsg := testutils.LogMatcher(2, getThroughputErrMsg, logBuffer.Lines()) - assert.True(t, match, errMsg) -} - -func TestLoadProbabilities(t *testing.T) { - mockStorage := &smocks.Store{} - mockStorage.On("GetLatestProbabilities").Return(make(model.ServiceOperationProbabilities), nil) - - p := &Processor{storage: mockStorage} - require.Nil(t, p.probabilities) - p.loadProbabilities() - require.NotNil(t, p.probabilities) -} - -func TestRunUpdateProbabilitiesLoop(t *testing.T) { - mockStorage := &smocks.Store{} - mockStorage.On("GetLatestProbabilities").Return(make(model.ServiceOperationProbabilities), nil) - mockEP := &epmocks.ElectionParticipant{} - mockEP.On("Start").Return(nil) - mockEP.On("Close").Return(nil) - mockEP.On("IsLeader").Return(false) - - p := &Processor{ - storage: mockStorage, - shutdown: make(chan struct{}), - followerRefreshInterval: time.Millisecond, - electionParticipant: mockEP, - } - defer close(p.shutdown) - require.Nil(t, p.probabilities) - require.Nil(t, p.strategyResponses) - go p.runUpdateProbabilitiesLoop() - - for i := 0; i < 1000; i++ { - p.RLock() - if p.probabilities != nil && p.strategyResponses != nil { - p.RUnlock() - break - } - p.RUnlock() - time.Sleep(time.Millisecond) - } - p.RLock() - assert.NotNil(t, p.probabilities) - assert.NotNil(t, p.strategyResponses) - p.RUnlock() -} - -func TestRealisticRunCalculationLoop(t *testing.T) { - t.Skip("Skipped realistic calculation loop test") - logger := zap.NewNop() - // NB: This is an extremely long test since it uses near realistic (1/6th scale) processor config values - testThroughputs := []*model.Throughput{ - {Service: "svcA", Operation: "GET", Count: 10}, - {Service: "svcA", Operation: "POST", Count: 9}, - {Service: "svcA", Operation: "PUT", Count: 5}, - {Service: "svcA", Operation: "DELETE", Count: 20}, - } - mockStorage := &smocks.Store{} - mockStorage.On("GetThroughput", mock.AnythingOfType("time.Time"), mock.AnythingOfType("time.Time")). - Return(testThroughputs, nil) - mockStorage.On("GetLatestProbabilities").Return(make(model.ServiceOperationProbabilities), nil) - mockStorage.On("InsertProbabilitiesAndQPS", "host", mock.AnythingOfType("model.ServiceOperationProbabilities"), - mock.AnythingOfType("model.ServiceOperationQPS")).Return(nil) - mockEP := &epmocks.ElectionParticipant{} - mockEP.On("Start").Return(nil) - mockEP.On("Close").Return(nil) - mockEP.On("IsLeader").Return(true) - cfg := Options{ - TargetSamplesPerSecond: 1.0, - DeltaTolerance: 0.2, - InitialSamplingProbability: 0.001, - CalculationInterval: time.Second * 10, - AggregationBuckets: 1, - Delay: time.Second * 10, - } - p, err := newProcessor(cfg, "host", mockStorage, mockEP, metrics.NullFactory, logger) - require.NoError(t, err) - p.Start() - - for i := 0; i < 100; i++ { - strategy, _ := p.GetSamplingStrategy(context.Background(), "svcA") - if len(strategy.OperationSampling.PerOperationStrategies) != 0 { - break - } - time.Sleep(250 * time.Millisecond) - } - p.Close() - - strategy, err := p.GetSamplingStrategy(context.Background(), "svcA") - require.NoError(t, err) - require.Len(t, strategy.OperationSampling.PerOperationStrategies, 4) - strategies := strategy.OperationSampling.PerOperationStrategies - - for _, s := range strategies { - switch s.Operation { - case "GET": - assert.Equal(t, 0.001, s.ProbabilisticSampling.SamplingRate, - "Already at 1QPS, no probability change") - case "POST": - assert.Equal(t, 0.001, s.ProbabilisticSampling.SamplingRate, - "Within epsilon of 1QPS, no probability change") - case "PUT": - assert.InEpsilon(t, 0.002, s.ProbabilisticSampling.SamplingRate, 0.025, - "Under sampled, double probability") - case "DELETE": - assert.InEpsilon(t, 0.0005, s.ProbabilisticSampling.SamplingRate, 0.025, - "Over sampled, halve probability") - } - } -} - -func TestPrependBucket(t *testing.T) { - p := &Processor{Options: Options{AggregationBuckets: 1}} - p.prependThroughputBucket(&throughputBucket{interval: time.Minute}) - require.Len(t, p.throughputs, 1) - assert.Equal(t, time.Minute, p.throughputs[0].interval) - - p.prependThroughputBucket(&throughputBucket{interval: 2 * time.Minute}) - require.Len(t, p.throughputs, 1) - assert.Equal(t, 2*time.Minute, p.throughputs[0].interval) -} - -func TestConstructorFailure(t *testing.T) { - logger := zap.NewNop() - - cfg := Options{ - TargetSamplesPerSecond: 1.0, - DeltaTolerance: 0.2, - InitialSamplingProbability: 0.001, - CalculationInterval: time.Second * 5, - AggregationBuckets: 0, - } - _, err := newProcessor(cfg, "host", nil, nil, metrics.NullFactory, logger) - require.EqualError(t, err, "CalculationInterval and AggregationBuckets must be greater than 0") - - cfg.CalculationInterval = 0 - _, err = newProcessor(cfg, "host", nil, nil, metrics.NullFactory, logger) - require.EqualError(t, err, "CalculationInterval and AggregationBuckets must be greater than 0") - - cfg.CalculationInterval = time.Millisecond - cfg.AggregationBuckets = 1 - cfg.BucketsForCalculation = -1 - _, err = newProcessor(cfg, "host", nil, nil, metrics.NullFactory, logger) - require.EqualError(t, err, "BucketsForCalculation cannot be less than 1") -} - -func TestGenerateStrategyResponses(t *testing.T) { - probabilities := model.ServiceOperationProbabilities{ - "svcA": map[string]float64{ - "GET": 0.5, - }, - } - p := &Processor{ - probabilities: probabilities, - Options: Options{ - InitialSamplingProbability: 0.001, - MinSamplesPerSecond: 0.0001, - }, - } - p.generateStrategyResponses() - - expectedResponse := map[string]*api_v2.SamplingStrategyResponse{ - "svcA": { - StrategyType: api_v2.SamplingStrategyType_PROBABILISTIC, - OperationSampling: &api_v2.PerOperationSamplingStrategies{ - DefaultSamplingProbability: 0.001, - DefaultLowerBoundTracesPerSecond: 0.0001, - PerOperationStrategies: []*api_v2.OperationSamplingStrategy{ - { - Operation: "GET", - ProbabilisticSampling: &api_v2.ProbabilisticSamplingStrategy{ - SamplingRate: 0.5, - }, - }, - }, - }, - }, - } - assert.Equal(t, expectedResponse, p.strategyResponses) -} - -func TestUsingAdaptiveSampling(t *testing.T) { - p := &Processor{} - throughput := serviceOperationThroughput{ - "svc": map[string]*model.Throughput{ - "op": {Probabilities: map[string]struct{}{"0.010000": {}}}, - }, - } - tests := []struct { - expected bool - probability float64 - service string - operation string - }{ - {expected: true, probability: 0.01, service: "svc", operation: "op"}, - {expected: true, probability: 0.0099999384, service: "svc", operation: "op"}, - {expected: false, probability: 0.01, service: "non-svc"}, - {expected: false, probability: 0.01, service: "svc", operation: "non-op"}, - {expected: false, probability: 0.01, service: "svc", operation: "non-op"}, - {expected: false, probability: 0.02, service: "svc", operation: "op"}, - {expected: false, probability: 0.0100009384, service: "svc", operation: "op"}, - } - for _, test := range tests { - assert.Equal(t, test.expected, p.isUsingAdaptiveSampling(test.probability, test.service, test.operation, throughput)) - } -} - -func TestPrependServiceCache(t *testing.T) { - p := &Processor{} - for i := 0; i < serviceCacheSize*2; i++ { - p.prependServiceCache() - } - assert.Len(t, p.serviceCache, serviceCacheSize) -} - -func TestCalculateProbabilitiesAndQPSMultiple(t *testing.T) { - buckets := []*throughputBucket{ - { - throughput: serviceOperationThroughput{ - "svcA": map[string]*model.Throughput{ - "GET": {Count: 3, Probabilities: map[string]struct{}{"0.001000": {}}}, - "PUT": {Count: 60, Probabilities: map[string]struct{}{"0.001000": {}}}, - }, - "svcB": map[string]*model.Throughput{ - "PUT": {Count: 15, Probabilities: map[string]struct{}{"0.001000": {}}}, - }, - }, - interval: 60 * time.Second, - }, - } - - p := &Processor{ - Options: Options{ - TargetSamplesPerSecond: 1.0, - DeltaTolerance: 0.002, - InitialSamplingProbability: 0.001, - BucketsForCalculation: 5, - AggregationBuckets: 10, - }, - throughputs: buckets, probabilities: make(model.ServiceOperationProbabilities), - qps: make(model.ServiceOperationQPS), weightVectorCache: NewWeightVectorCache(), - probabilityCalculator: calculationstrategy.NewPercentageIncreaseCappedCalculator(1.0), - serviceCache: []SamplingCache{}, - operationsCalculatedGauge: metrics.NullFactory.Gauge(metrics.Options{}), - } - - probabilities, qps := p.calculateProbabilitiesAndQPS() - - require.Len(t, probabilities, 2) - assert.Equal(t, map[string]float64{"GET": 0.002, "PUT": 0.001}, probabilities["svcA"]) - assert.Equal(t, map[string]float64{"PUT": 0.002}, probabilities["svcB"]) - - p.probabilities = probabilities - p.qps = qps - - // svcA:GET is no longer reported, we should not increase it's probability since we don't know if it's adaptively sampled - // until we get at least a lowerbound span or a probability span with the right probability. - // svcB:PUT is only reporting lowerbound, we should boost it's probability - p.prependThroughputBucket(&throughputBucket{ - throughput: serviceOperationThroughput{ - "svcA": map[string]*model.Throughput{ - "PUT": {Count: 60, Probabilities: map[string]struct{}{"0.001000": {}}}, - }, - "svcB": map[string]*model.Throughput{ - "GET": {Count: 30, Probabilities: map[string]struct{}{"0.001000": {}}}, - "PUT": {Count: 0, Probabilities: map[string]struct{}{"0.002000": {}}}, - }, - }, - interval: 60 * time.Second, - }) - - probabilities, qps = p.calculateProbabilitiesAndQPS() - - require.Len(t, probabilities, 2) - assert.Equal(t, map[string]float64{"GET": 0.002, "PUT": 0.001}, probabilities["svcA"]) - assert.Equal(t, map[string]float64{"PUT": 0.004, "GET": 0.002}, probabilities["svcB"]) - - p.probabilities = probabilities - p.qps = qps - - // svcA:GET is lower bound sampled, increase its probability - // svcB:PUT is not reported but we should boost it's probability since the previous calculation showed that - // it's using adaptive sampling - p.prependThroughputBucket(&throughputBucket{ - throughput: serviceOperationThroughput{ - "svcA": map[string]*model.Throughput{ - "GET": {Count: 0, Probabilities: map[string]struct{}{"0.002000": {}}}, - "PUT": {Count: 60, Probabilities: map[string]struct{}{"0.001000": {}}}, - }, - "svcB": map[string]*model.Throughput{ - "GET": {Count: 30, Probabilities: map[string]struct{}{"0.001000": {}}}, - }, - }, - interval: 60 * time.Second, - }) - - probabilities, qps = p.calculateProbabilitiesAndQPS() - - require.Len(t, probabilities, 2) - assert.Equal(t, map[string]float64{"GET": 0.004, "PUT": 0.001}, probabilities["svcA"]) - assert.Equal(t, map[string]float64{"PUT": 0.008, "GET": 0.002}, probabilities["svcB"]) - - p.probabilities = probabilities - p.qps = qps - - // svcA:GET is finally adaptively probabilistically sampled! - // svcB:PUT stopped using adaptive sampling - p.prependThroughputBucket(&throughputBucket{ - throughput: serviceOperationThroughput{ - "svcA": map[string]*model.Throughput{ - "GET": {Count: 1, Probabilities: map[string]struct{}{"0.004000": {}}}, - "PUT": {Count: 60, Probabilities: map[string]struct{}{"0.001000": {}}}, - }, - "svcB": map[string]*model.Throughput{ - "GET": {Count: 30, Probabilities: map[string]struct{}{"0.001000": {}}}, - "PUT": {Count: 15, Probabilities: map[string]struct{}{"0.001000": {}}}, - }, - }, - interval: 60 * time.Second, - }) - - probabilities, qps = p.calculateProbabilitiesAndQPS() - - require.Len(t, probabilities, 2) - assert.Equal(t, map[string]float64{"GET": 0.008, "PUT": 0.001}, probabilities["svcA"]) - assert.Equal(t, map[string]float64{"PUT": 0.008, "GET": 0.002}, probabilities["svcB"]) - - p.probabilities = probabilities - p.qps = qps - - // svcA:GET didn't report anything - p.prependThroughputBucket(&throughputBucket{ - throughput: serviceOperationThroughput{ - "svcA": map[string]*model.Throughput{ - "PUT": {Count: 30, Probabilities: map[string]struct{}{"0.001000": {}}}, - }, - "svcB": map[string]*model.Throughput{ - "GET": {Count: 30, Probabilities: map[string]struct{}{"0.001000": {}}}, - "PUT": {Count: 15, Probabilities: map[string]struct{}{"0.001000": {}}}, - }, - }, - interval: 60 * time.Second, - }) - - probabilities, qps = p.calculateProbabilitiesAndQPS() - - require.Len(t, probabilities, 2) - assert.Equal(t, map[string]float64{"GET": 0.016, "PUT": 0.001468867216804201}, probabilities["svcA"]) - assert.Equal(t, map[string]float64{"PUT": 0.008, "GET": 0.002}, probabilities["svcB"]) - - p.probabilities = probabilities - p.qps = qps - - // svcA:GET didn't report anything - // svcB:PUT starts to use adaptive sampling again - p.prependThroughputBucket(&throughputBucket{ - throughput: serviceOperationThroughput{ - "svcA": map[string]*model.Throughput{ - "PUT": {Count: 30, Probabilities: map[string]struct{}{"0.001000": {}}}, - }, - "svcB": map[string]*model.Throughput{ - "GET": {Count: 30, Probabilities: map[string]struct{}{"0.001000": {}}}, - "PUT": {Count: 1, Probabilities: map[string]struct{}{"0.008000": {}}}, - }, - }, - interval: 60 * time.Second, - }) - - probabilities, qps = p.calculateProbabilitiesAndQPS() - - require.Len(t, probabilities, 2) - assert.Equal(t, map[string]float64{"GET": 0.032, "PUT": 0.001468867216804201}, probabilities["svcA"]) - assert.Equal(t, map[string]float64{"PUT": 0.016, "GET": 0.002}, probabilities["svcB"]) - - p.probabilities = probabilities - p.qps = qps - - // svcA:GET didn't report anything - // svcB:PUT didn't report anything - p.prependThroughputBucket(&throughputBucket{ - throughput: serviceOperationThroughput{ - "svcA": map[string]*model.Throughput{ - "PUT": {Count: 30, Probabilities: map[string]struct{}{"0.001000": {}}}, - }, - "svcB": map[string]*model.Throughput{ - "GET": {Count: 15, Probabilities: map[string]struct{}{"0.001000": {}}}, - }, - }, - interval: 60 * time.Second, - }) - - probabilities, qps = p.calculateProbabilitiesAndQPS() - - require.Len(t, probabilities, 2) - assert.Equal(t, map[string]float64{"GET": 0.064, "PUT": 0.001468867216804201}, probabilities["svcA"]) - assert.Equal(t, map[string]float64{"PUT": 0.032, "GET": 0.002}, probabilities["svcB"]) - - p.probabilities = probabilities - p.qps = qps - - // svcA:GET didn't report anything - // svcB:PUT didn't report anything - p.prependThroughputBucket(&throughputBucket{ - throughput: serviceOperationThroughput{ - "svcA": map[string]*model.Throughput{ - "PUT": {Count: 20, Probabilities: map[string]struct{}{"0.001000": {}}}, - }, - "svcB": map[string]*model.Throughput{ - "GET": {Count: 10, Probabilities: map[string]struct{}{"0.001000": {}}}, - }, - }, - interval: 60 * time.Second, - }) - - probabilities, qps = p.calculateProbabilitiesAndQPS() - - require.Len(t, probabilities, 2) - assert.Equal(t, map[string]float64{"GET": 0.128, "PUT": 0.001468867216804201}, probabilities["svcA"]) - assert.Equal(t, map[string]float64{"PUT": 0.064, "GET": 0.002}, probabilities["svcB"]) - - p.probabilities = probabilities - p.qps = qps - - // svcA:GET didn't report anything - // svcB:PUT didn't report anything - p.prependThroughputBucket(&throughputBucket{ - throughput: serviceOperationThroughput{ - "svcA": map[string]*model.Throughput{ - "PUT": {Count: 20, Probabilities: map[string]struct{}{"0.001000": {}}}, - "GET": {Count: 120, Probabilities: map[string]struct{}{"0.128000": {}}}, - }, - "svcB": map[string]*model.Throughput{ - "PUT": {Count: 60, Probabilities: map[string]struct{}{"0.064000": {}}}, - "GET": {Count: 10, Probabilities: map[string]struct{}{"0.001000": {}}}, - }, - }, - interval: 60 * time.Second, - }) - - probabilities, qps = p.calculateProbabilitiesAndQPS() - - require.Len(t, probabilities, 2) - assert.Equal(t, map[string]float64{"GET": 0.0882586677054928, "PUT": 0.001468867216804201}, probabilities["svcA"]) - assert.Equal(t, map[string]float64{"PUT": 0.09587513707888091, "GET": 0.002}, probabilities["svcB"]) - - p.probabilities = probabilities - p.qps = qps -} - -func TestErrors(t *testing.T) { - mockStorage := &smocks.Store{} - mockStorage.On("GetLatestProbabilities").Return(model.ServiceOperationProbabilities{}, errTestStorage()) - mockStorage.On("GetThroughput", mock.AnythingOfType("time.Time"), mock.AnythingOfType("time.Time")). - Return(nil, nil) - - cfg := Options{ - TargetSamplesPerSecond: 1.0, - DeltaTolerance: 0.1, - InitialSamplingProbability: 0.001, - CalculationInterval: time.Millisecond * 5, - AggregationBuckets: 2, - Delay: time.Millisecond * 5, - LeaderLeaseRefreshInterval: time.Millisecond, - FollowerLeaseRefreshInterval: time.Second, - BucketsForCalculation: 10, - } - - // start errors - mockEP := &epmocks.ElectionParticipant{} - mockEP.On("Start").Return(errors.New("bad")) - mockEP.On("Close").Return(errors.New("also bad")) - mockEP.On("IsLeader").Return(false) - - p, err := newProcessor(cfg, "host", mockStorage, mockEP, metrics.NullFactory, zap.NewNop()) - require.NoError(t, err) - require.Error(t, p.Start()) - require.Error(t, p.Close()) - - // close errors - mockEP = &epmocks.ElectionParticipant{} - mockEP.On("Start").Return(nil) - mockEP.On("Close").Return(errors.New("still bad")) - mockEP.On("IsLeader").Return(false) - - p, err = newProcessor(cfg, "host", mockStorage, mockEP, metrics.NullFactory, zap.NewNop()) - require.NoError(t, err) - require.NoError(t, p.Start()) - require.Error(t, p.Close()) -} diff --git a/plugin/sampling/strategystore/adaptive/strategy_store.go b/plugin/sampling/strategystore/adaptive/strategy_store.go index 76d2b9f5d5e..fe2be9ac608 100644 --- a/plugin/sampling/strategystore/adaptive/strategy_store.go +++ b/plugin/sampling/strategystore/adaptive/strategy_store.go @@ -15,32 +15,173 @@ package adaptive import ( + "context" + "sync" + "time" + "go.uber.org/zap" - "github.com/jaegertracing/jaeger/pkg/distributedlock" - "github.com/jaegertracing/jaeger/pkg/hostname" - "github.com/jaegertracing/jaeger/pkg/metrics" + "github.com/jaegertracing/jaeger/cmd/collector/app/sampling/model" "github.com/jaegertracing/jaeger/plugin/sampling/leaderelection" + "github.com/jaegertracing/jaeger/proto-gen/api_v2" "github.com/jaegertracing/jaeger/storage/samplingstore" ) +const defaultFollowerProbabilityInterval = 20 * time.Second + +type StrategyStore struct { + sync.RWMutex + Options + + electionParticipant leaderelection.ElectionParticipant + storage samplingstore.Store + logger *zap.Logger + + // probabilities contains the latest calculated sampling probabilities for service operations. + probabilities model.ServiceOperationProbabilities + + // strategyResponses is the cache of the sampling strategies for every service, in Thrift format. + // TODO change this to work with protobuf model instead, to support gRPC endpoint. + strategyResponses map[string]*api_v2.SamplingStrategyResponse + + // followerRefreshInterval determines how often the follower processor updates its probabilities. + // Given only the leader writes probabilities, the followers need to fetch the probabilities into + // cache. + followerRefreshInterval time.Duration + + shutdown chan struct{} + bgFinished sync.WaitGroup +} + // NewStrategyStore creates a strategy store that holds adaptive sampling strategies. -func NewStrategyStore(options Options, metricsFactory metrics.Factory, logger *zap.Logger, lock distributedlock.Lock, store samplingstore.Store) (*Processor, error) { - hostname, err := hostname.AsIdentifier() - if err != nil { - return nil, err +func NewStrategyStore(options Options, logger *zap.Logger, participant leaderelection.ElectionParticipant, store samplingstore.Store) (*StrategyStore, error) { + return &StrategyStore{ + Options: options, + storage: store, + probabilities: make(model.ServiceOperationProbabilities), + strategyResponses: make(map[string]*api_v2.SamplingStrategyResponse), + logger: logger, + electionParticipant: participant, + followerRefreshInterval: defaultFollowerProbabilityInterval, + }, nil +} + +// Start initializes and starts the sampling processor which regularly calculates sampling probabilities. +func (ss *StrategyStore) Start() error { + ss.logger.Info("starting adaptive sampling processor") + if err := ss.electionParticipant.Start(); err != nil { + return err + } + ss.shutdown = make(chan struct{}) + ss.loadProbabilities() + ss.generateStrategyResponses() + ss.runBackground(ss.runUpdateProbabilitiesLoop) + return nil +} + +// GetSamplingStrategy implements Thrift endpoint for retrieving sampling strategy for a service. +func (ss *StrategyStore) GetSamplingStrategy(_ context.Context, service string) (*api_v2.SamplingStrategyResponse, error) { + ss.RLock() + defer ss.RUnlock() + if strategy, ok := ss.strategyResponses[service]; ok { + return strategy, nil } - logger.Info("Using unique participantName in adaptive sampling", zap.String("participantName", hostname)) + return ss.generateDefaultSamplingStrategyResponse(), nil +} - participant := leaderelection.NewElectionParticipant(lock, defaultResourceName, leaderelection.ElectionParticipantOptions{ - FollowerLeaseRefreshInterval: options.FollowerLeaseRefreshInterval, - LeaderLeaseRefreshInterval: options.LeaderLeaseRefreshInterval, - Logger: logger, - }) - p, err := newProcessor(options, hostname, store, participant, metricsFactory, logger) +func (ss *StrategyStore) loadProbabilities() { + // TODO GetLatestProbabilities API can be changed to return the latest measured qps for initialization + probabilities, err := ss.storage.GetLatestProbabilities() if err != nil { - return nil, err + ss.logger.Warn("failed to initialize probabilities", zap.Error(err)) + return + } + ss.Lock() + defer ss.Unlock() + ss.probabilities = probabilities +} + +// runUpdateProbabilitiesLoop is a loop that reads probabilities from storage. +// The follower updates its local cache with the latest probabilities and serves them. +func (ss *StrategyStore) runUpdateProbabilitiesLoop() { + select { + case <-time.After(addJitter(ss.followerRefreshInterval)): + case <-ss.shutdown: + return + } + + ticker := time.NewTicker(ss.followerRefreshInterval) + defer ticker.Stop() + for { + select { + case <-ticker.C: + // Only load probabilities if this processor doesn't hold the leader lock + if !ss.isLeader() { + ss.loadProbabilities() + ss.generateStrategyResponses() + } + case <-ss.shutdown: + return + } + } +} + +func (ss *StrategyStore) isLeader() bool { + return ss.electionParticipant.IsLeader() +} + +// generateStrategyResponses generates and caches SamplingStrategyResponse from the calculated sampling probabilities. +func (ss *StrategyStore) generateStrategyResponses() { + ss.RLock() + strategies := make(map[string]*api_v2.SamplingStrategyResponse) + for svc, opProbabilities := range ss.probabilities { + opStrategies := make([]*api_v2.OperationSamplingStrategy, len(opProbabilities)) + var idx int + for op, probability := range opProbabilities { + opStrategies[idx] = &api_v2.OperationSamplingStrategy{ + Operation: op, + ProbabilisticSampling: &api_v2.ProbabilisticSamplingStrategy{ + SamplingRate: probability, + }, + } + idx++ + } + strategy := ss.generateDefaultSamplingStrategyResponse() + strategy.OperationSampling.PerOperationStrategies = opStrategies + strategies[svc] = strategy } + ss.RUnlock() - return p, nil + ss.Lock() + defer ss.Unlock() + ss.strategyResponses = strategies +} + +func (ss *StrategyStore) generateDefaultSamplingStrategyResponse() *api_v2.SamplingStrategyResponse { + return &api_v2.SamplingStrategyResponse{ + StrategyType: api_v2.SamplingStrategyType_PROBABILISTIC, + OperationSampling: &api_v2.PerOperationSamplingStrategies{ + DefaultSamplingProbability: ss.InitialSamplingProbability, + DefaultLowerBoundTracesPerSecond: ss.MinSamplesPerSecond, + }, + } +} + +func (ss *StrategyStore) runBackground(f func()) { + ss.bgFinished.Add(1) + go func() { + f() + ss.bgFinished.Done() + }() +} + +// Close stops the processor from calculating probabilities. +func (ss *StrategyStore) Close() error { + ss.logger.Info("stopping adaptive sampling processor") + err := ss.electionParticipant.Close() + if ss.shutdown != nil { + close(ss.shutdown) + } + ss.bgFinished.Wait() + return err } From 890616e3299394d45b5addaca6c229177572ee33 Mon Sep 17 00:00:00 2001 From: Pushkar Mishra Date: Sun, 12 May 2024 16:38:13 +0530 Subject: [PATCH 02/18] fixing goroutine leak Signed-off-by: Pushkar Mishra --- .../strategystore/adaptive/aggregator.go | 18 +++++++++++++++--- 1 file changed, 15 insertions(+), 3 deletions(-) diff --git a/plugin/sampling/strategystore/adaptive/aggregator.go b/plugin/sampling/strategystore/adaptive/aggregator.go index ed1d17c1ab7..faffc31b26a 100644 --- a/plugin/sampling/strategystore/adaptive/aggregator.go +++ b/plugin/sampling/strategystore/adaptive/aggregator.go @@ -133,7 +133,19 @@ func (a *aggregator) Start() { } func (a *aggregator) Close() error { - close(a.stop) - a.postAggregator.Close() - return nil + a.Lock() + defer a.Unlock() + + // if err := a.postAggregator.Close(); err != nil { + // return err + // } + + // select { + // case <-a.stop: + // // a.stop is already closed, do nothing + // default: + // close(a.stop) + // } + + return a.postAggregator.Close() } From cd5cce637d5dd10fb8aa001b217a6df0165fd85f Mon Sep 17 00:00:00 2001 From: Pushkar Mishra Date: Sun, 12 May 2024 16:48:41 +0530 Subject: [PATCH 03/18] fix Signed-off-by: Pushkar Mishra --- plugin/sampling/strategystore/adaptive/aggregator.go | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/plugin/sampling/strategystore/adaptive/aggregator.go b/plugin/sampling/strategystore/adaptive/aggregator.go index faffc31b26a..36407751437 100644 --- a/plugin/sampling/strategystore/adaptive/aggregator.go +++ b/plugin/sampling/strategystore/adaptive/aggregator.go @@ -146,6 +146,7 @@ func (a *aggregator) Close() error { // default: // close(a.stop) // } + close(a.stop) - return a.postAggregator.Close() + return nil } From 618ce0318336e762ca203822d24bdb770018749f Mon Sep 17 00:00:00 2001 From: Pushkar Mishra Date: Sun, 12 May 2024 17:09:40 +0530 Subject: [PATCH 04/18] fix Signed-off-by: Pushkar Mishra --- .../strategystore/adaptive/aggregator.go | 21 +++++++++---------- 1 file changed, 10 insertions(+), 11 deletions(-) diff --git a/plugin/sampling/strategystore/adaptive/aggregator.go b/plugin/sampling/strategystore/adaptive/aggregator.go index 36407751437..76773e585db 100644 --- a/plugin/sampling/strategystore/adaptive/aggregator.go +++ b/plugin/sampling/strategystore/adaptive/aggregator.go @@ -136,17 +136,16 @@ func (a *aggregator) Close() error { a.Lock() defer a.Unlock() - // if err := a.postAggregator.Close(); err != nil { - // return err - // } - - // select { - // case <-a.stop: - // // a.stop is already closed, do nothing - // default: - // close(a.stop) - // } - close(a.stop) + if err := a.postAggregator.Close(); err != nil { + return err + } + + select { + case <-a.stop: + // a.stop is already closed, do nothing + default: + close(a.stop) + } return nil } From 71beb8d2cd97b20e1b49fd7be141a03404852d15 Mon Sep 17 00:00:00 2001 From: Pushkar Mishra Date: Mon, 13 May 2024 00:39:03 +0530 Subject: [PATCH 05/18] fix Signed-off-by: Pushkar Mishra --- .../strategystore/adaptive/aggregator.go | 31 +++++--- .../strategystore/adaptive/postaggregator.go | 72 ++++++++----------- 2 files changed, 50 insertions(+), 53 deletions(-) diff --git a/plugin/sampling/strategystore/adaptive/aggregator.go b/plugin/sampling/strategystore/adaptive/aggregator.go index 76773e585db..cd5145ce456 100644 --- a/plugin/sampling/strategystore/adaptive/aggregator.go +++ b/plugin/sampling/strategystore/adaptive/aggregator.go @@ -15,6 +15,7 @@ package adaptive import ( + "errors" "sync" "time" @@ -43,6 +44,7 @@ type aggregator struct { aggregationInterval time.Duration storage samplingstore.Store stop chan struct{} + bgFinished sync.WaitGroup } // NewAggregator creates a throughput aggregator that simply emits metrics @@ -72,12 +74,17 @@ func NewAggregator(options Options, logger *zap.Logger, metricsFactory metrics.F func (a *aggregator) runAggregationLoop() { ticker := time.NewTicker(a.aggregationInterval) + + // NB: the first tick will be slightly delayed by the initializeThroughput call. + a.postAggregator.lastCheckedTime = time.Now().Add(a.postAggregator.Delay * -1) + a.postAggregator.initializeThroughput(a.postAggregator.lastCheckedTime) for { select { case <-ticker.C: a.Lock() a.saveThroughput() a.currentThroughput = make(serviceOperationThroughput) + a.postAggregator.runCalculation() a.Unlock() case <-a.stop: ticker.Stop() @@ -128,24 +135,26 @@ func (a *aggregator) RecordThroughput(service, operation string, samplerType spa } func (a *aggregator) Start() { - go a.runAggregationLoop() a.postAggregator.Start() + a.runBackground(a.runAggregationLoop) } func (a *aggregator) Close() error { - a.Lock() - defer a.Unlock() - + var errs []error if err := a.postAggregator.Close(); err != nil { - return err + errs = append(errs, err) } - - select { - case <-a.stop: - // a.stop is already closed, do nothing - default: + if a.stop != nil { close(a.stop) } + a.bgFinished.Wait() + return errors.Join(errs...) +} - return nil +func (a *aggregator) runBackground(f func()) { + a.bgFinished.Add(1) + go func() { + f() + a.bgFinished.Done() + }() } diff --git a/plugin/sampling/strategystore/adaptive/postaggregator.go b/plugin/sampling/strategystore/adaptive/postaggregator.go index 69437cf907c..2e5c5463243 100644 --- a/plugin/sampling/strategystore/adaptive/postaggregator.go +++ b/plugin/sampling/strategystore/adaptive/postaggregator.go @@ -102,6 +102,7 @@ type PostAggregator struct { operationsCalculatedGauge metrics.Gauge calculateProbabilitiesLatency metrics.Timer + lastCheckedTime time.Time } // newProcessor creates a new sampling processor that generates sampling rates for service operations. @@ -144,7 +145,6 @@ func (p *PostAggregator) Start() error { return err } p.shutdown = make(chan struct{}) - p.runBackground(p.runCalculationLoop) return nil } @@ -179,47 +179,35 @@ func addJitter(jitterAmount time.Duration) time.Duration { return (jitterAmount / 2) + time.Duration(rand.Int63n(int64(jitterAmount/2))) } -func (p *PostAggregator) runCalculationLoop() { - lastCheckedTime := time.Now().Add(p.Delay * -1) - p.initializeThroughput(lastCheckedTime) - // NB: the first tick will be slightly delayed by the initializeThroughput call. - ticker := time.NewTicker(p.CalculationInterval) - defer ticker.Stop() - for { - select { - case <-ticker.C: - endTime := time.Now().Add(p.Delay * -1) - startTime := lastCheckedTime - throughput, err := p.storage.GetThroughput(startTime, endTime) - if err != nil { - p.logger.Error(getThroughputErrMsg, zap.Error(err)) - break - } - aggregatedThroughput := p.aggregateThroughput(throughput) - p.prependThroughputBucket(&throughputBucket{ - throughput: aggregatedThroughput, - interval: endTime.Sub(startTime), - endTime: endTime, - }) - lastCheckedTime = endTime - // Load the latest throughput so that if this host ever becomes leader, it - // has the throughput ready in memory. However, only run the actual calculations - // if this host becomes leader. - // TODO fill the throughput buffer only when we're leader - if p.isLeader() { - startTime := time.Now() - probabilities, qps := p.calculateProbabilitiesAndQPS() - p.Lock() - p.probabilities = probabilities - p.qps = qps - p.Unlock() - - p.calculateProbabilitiesLatency.Record(time.Since(startTime)) - p.runBackground(p.saveProbabilitiesAndQPS) - } - case <-p.shutdown: - return - } +func (p *PostAggregator) runCalculation() { + endTime := time.Now().Add(p.Delay * -1) + startTime := p.lastCheckedTime + throughput, err := p.storage.GetThroughput(startTime, endTime) + if err != nil { + p.logger.Error(getThroughputErrMsg, zap.Error(err)) + return + } + aggregatedThroughput := p.aggregateThroughput(throughput) + p.prependThroughputBucket(&throughputBucket{ + throughput: aggregatedThroughput, + interval: endTime.Sub(startTime), + endTime: endTime, + }) + p.lastCheckedTime = endTime + // Load the latest throughput so that if this host ever becomes leader, it + // has the throughput ready in memory. However, only run the actual calculations + // if this host becomes leader. + // TODO fill the throughput buffer only when we're leader + if p.isLeader() { + startTime := time.Now() + probabilities, qps := p.calculateProbabilitiesAndQPS() + p.Lock() + p.probabilities = probabilities + p.qps = qps + p.Unlock() + + p.calculateProbabilitiesLatency.Record(time.Since(startTime)) + p.runBackground(p.saveProbabilitiesAndQPS) } } From c0878ad89c82c0746eb38fcf5227d190c5340145 Mon Sep 17 00:00:00 2001 From: Pushkar Mishra Date: Mon, 13 May 2024 01:09:26 +0530 Subject: [PATCH 06/18] fix Signed-off-by: Pushkar Mishra --- plugin/sampling/strategystore/adaptive/aggregator.go | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/plugin/sampling/strategystore/adaptive/aggregator.go b/plugin/sampling/strategystore/adaptive/aggregator.go index cd5145ce456..1e211566d63 100644 --- a/plugin/sampling/strategystore/adaptive/aggregator.go +++ b/plugin/sampling/strategystore/adaptive/aggregator.go @@ -144,9 +144,9 @@ func (a *aggregator) Close() error { if err := a.postAggregator.Close(); err != nil { errs = append(errs, err) } - if a.stop != nil { - close(a.stop) - } + // if a.stop != nil { + // close(a.stop) + // } a.bgFinished.Wait() return errors.Join(errs...) } From 976ee74a4331c29385116b879a64814b0b8c226d Mon Sep 17 00:00:00 2001 From: Pushkar Mishra Date: Mon, 13 May 2024 01:23:49 +0530 Subject: [PATCH 07/18] fix Signed-off-by: Pushkar Mishra --- plugin/sampling/strategystore/adaptive/aggregator.go | 6 +++--- plugin/sampling/strategystore/adaptive/postaggregator.go | 3 --- 2 files changed, 3 insertions(+), 6 deletions(-) diff --git a/plugin/sampling/strategystore/adaptive/aggregator.go b/plugin/sampling/strategystore/adaptive/aggregator.go index 1e211566d63..cd5145ce456 100644 --- a/plugin/sampling/strategystore/adaptive/aggregator.go +++ b/plugin/sampling/strategystore/adaptive/aggregator.go @@ -144,9 +144,9 @@ func (a *aggregator) Close() error { if err := a.postAggregator.Close(); err != nil { errs = append(errs, err) } - // if a.stop != nil { - // close(a.stop) - // } + if a.stop != nil { + close(a.stop) + } a.bgFinished.Wait() return errors.Join(errs...) } diff --git a/plugin/sampling/strategystore/adaptive/postaggregator.go b/plugin/sampling/strategystore/adaptive/postaggregator.go index 2e5c5463243..539ed7b7106 100644 --- a/plugin/sampling/strategystore/adaptive/postaggregator.go +++ b/plugin/sampling/strategystore/adaptive/postaggregator.go @@ -160,9 +160,6 @@ func (p *PostAggregator) runBackground(f func()) { func (p *PostAggregator) Close() error { p.logger.Info("stopping adaptive sampling processor") err := p.electionParticipant.Close() - if p.shutdown != nil { - close(p.shutdown) - } p.bgFinished.Wait() return err } From b417348209f440d044d5cef7532c6a0fce3a9438 Mon Sep 17 00:00:00 2001 From: Pushkar Mishra Date: Wed, 15 May 2024 23:45:51 +0530 Subject: [PATCH 08/18] fix Signed-off-by: Pushkar Mishra --- .../{postaggregator.go => processor.go} | 100 +- .../strategystore/adaptive/processor_test.go | 896 ++++++++++++++++++ .../strategystore/adaptive/strategy_store.go | 97 -- 3 files changed, 995 insertions(+), 98 deletions(-) rename plugin/sampling/strategystore/adaptive/{postaggregator.go => processor.go} (83%) create mode 100644 plugin/sampling/strategystore/adaptive/processor_test.go diff --git a/plugin/sampling/strategystore/adaptive/postaggregator.go b/plugin/sampling/strategystore/adaptive/processor.go similarity index 83% rename from plugin/sampling/strategystore/adaptive/postaggregator.go rename to plugin/sampling/strategystore/adaptive/processor.go index 539ed7b7106..4f59330d515 100644 --- a/plugin/sampling/strategystore/adaptive/postaggregator.go +++ b/plugin/sampling/strategystore/adaptive/processor.go @@ -1,4 +1,4 @@ -// Copyright (c) 2024 The Jaeger Authors. +// Copyright (c) 2018 The Jaeger Authors. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. @@ -15,6 +15,7 @@ package adaptive import ( + "context" "errors" "math" "math/rand" @@ -27,6 +28,7 @@ import ( "github.com/jaegertracing/jaeger/pkg/metrics" "github.com/jaegertracing/jaeger/plugin/sampling/calculationstrategy" "github.com/jaegertracing/jaeger/plugin/sampling/leaderelection" + "github.com/jaegertracing/jaeger/proto-gen/api_v2" "github.com/jaegertracing/jaeger/storage/samplingstore" ) @@ -138,6 +140,16 @@ func newPostAggregator( }, nil } +// GetSamplingStrategy implements Thrift endpoint for retrieving sampling strategy for a service. +func (ss *StrategyStore) GetSamplingStrategy(_ context.Context, service string) (*api_v2.SamplingStrategyResponse, error) { + ss.RLock() + defer ss.RUnlock() + if strategy, ok := ss.strategyResponses[service]; ok { + return strategy, nil + } + return ss.generateDefaultSamplingStrategyResponse(), nil +} + // Start initializes and starts the sampling processor which regularly calculates sampling probabilities. func (p *PostAggregator) Start() error { p.logger.Info("starting adaptive sampling processor") @@ -156,6 +168,14 @@ func (p *PostAggregator) runBackground(f func()) { }() } +func (ss *StrategyStore) runBackground(f func()) { + ss.bgFinished.Add(1) + go func() { + f() + ss.bgFinished.Done() + }() +} + // Close stops the processor from calculating probabilities. func (p *PostAggregator) Close() error { p.logger.Info("stopping adaptive sampling processor") @@ -164,10 +184,51 @@ func (p *PostAggregator) Close() error { return err } +func (ss *StrategyStore) loadProbabilities() { + // TODO GetLatestProbabilities API can be changed to return the latest measured qps for initialization + probabilities, err := ss.storage.GetLatestProbabilities() + if err != nil { + ss.logger.Warn("failed to initialize probabilities", zap.Error(err)) + return + } + ss.Lock() + defer ss.Unlock() + ss.probabilities = probabilities +} + +// runUpdateProbabilitiesLoop is a loop that reads probabilities from storage. +// The follower updates its local cache with the latest probabilities and serves them. +func (ss *StrategyStore) runUpdateProbabilitiesLoop() { + select { + case <-time.After(addJitter(ss.followerRefreshInterval)): + case <-ss.shutdown: + return + } + + ticker := time.NewTicker(ss.followerRefreshInterval) + defer ticker.Stop() + for { + select { + case <-ticker.C: + // Only load probabilities if this processor doesn't hold the leader lock + if !ss.isLeader() { + ss.loadProbabilities() + ss.generateStrategyResponses() + } + case <-ss.shutdown: + return + } + } +} + func (p *PostAggregator) isLeader() bool { return p.electionParticipant.IsLeader() } +func (ss *StrategyStore) isLeader() bool { + return ss.electionParticipant.IsLeader() +} + // addJitter adds a random amount of time. Without jitter, if the host holding the leader // lock were to die, then all other collectors can potentially wait for a full cycle before // trying to acquire the lock. With jitter, we can reduce the average amount of time before a @@ -425,3 +486,40 @@ func (p *PostAggregator) isUsingAdaptiveSampling( } return false } + +// generateStrategyResponses generates and caches SamplingStrategyResponse from the calculated sampling probabilities. +func (ss *StrategyStore) generateStrategyResponses() { + ss.RLock() + strategies := make(map[string]*api_v2.SamplingStrategyResponse) + for svc, opProbabilities := range ss.probabilities { + opStrategies := make([]*api_v2.OperationSamplingStrategy, len(opProbabilities)) + var idx int + for op, probability := range opProbabilities { + opStrategies[idx] = &api_v2.OperationSamplingStrategy{ + Operation: op, + ProbabilisticSampling: &api_v2.ProbabilisticSamplingStrategy{ + SamplingRate: probability, + }, + } + idx++ + } + strategy := ss.generateDefaultSamplingStrategyResponse() + strategy.OperationSampling.PerOperationStrategies = opStrategies + strategies[svc] = strategy + } + ss.RUnlock() + + ss.Lock() + defer ss.Unlock() + ss.strategyResponses = strategies +} + +func (ss *StrategyStore) generateDefaultSamplingStrategyResponse() *api_v2.SamplingStrategyResponse { + return &api_v2.SamplingStrategyResponse{ + StrategyType: api_v2.SamplingStrategyType_PROBABILISTIC, + OperationSampling: &api_v2.PerOperationSamplingStrategies{ + DefaultSamplingProbability: ss.InitialSamplingProbability, + DefaultLowerBoundTracesPerSecond: ss.MinSamplesPerSecond, + }, + } +} diff --git a/plugin/sampling/strategystore/adaptive/processor_test.go b/plugin/sampling/strategystore/adaptive/processor_test.go new file mode 100644 index 00000000000..03ae6b7b7bb --- /dev/null +++ b/plugin/sampling/strategystore/adaptive/processor_test.go @@ -0,0 +1,896 @@ +// Copyright (c) 2018 The Jaeger Authors. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package adaptive + +import ( + "context" + "errors" + "testing" + "time" + + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/mock" + "github.com/stretchr/testify/require" + "go.uber.org/zap" + + "github.com/jaegertracing/jaeger/cmd/collector/app/sampling/model" + "github.com/jaegertracing/jaeger/internal/metricstest" + "github.com/jaegertracing/jaeger/pkg/metrics" + "github.com/jaegertracing/jaeger/pkg/testutils" + "github.com/jaegertracing/jaeger/plugin/sampling/calculationstrategy" + epmocks "github.com/jaegertracing/jaeger/plugin/sampling/leaderelection/mocks" + "github.com/jaegertracing/jaeger/proto-gen/api_v2" + smocks "github.com/jaegertracing/jaeger/storage/samplingstore/mocks" +) + +func testThroughputs() []*model.Throughput { + return []*model.Throughput{ + {Service: "svcA", Operation: "GET", Count: 4, Probabilities: map[string]struct{}{"0.1": {}}}, + {Service: "svcA", Operation: "GET", Count: 4, Probabilities: map[string]struct{}{"0.2": {}}}, + {Service: "svcA", Operation: "PUT", Count: 5, Probabilities: map[string]struct{}{"0.1": {}}}, + {Service: "svcB", Operation: "GET", Count: 3, Probabilities: map[string]struct{}{"0.1": {}}}, + } +} + +func testThroughputBuckets() []*throughputBucket { + return []*throughputBucket{ + { + throughput: serviceOperationThroughput{ + "svcA": map[string]*model.Throughput{ + "GET": {Count: 45}, + "PUT": {Count: 60}, + }, + "svcB": map[string]*model.Throughput{ + "GET": {Count: 30}, + "PUT": {Count: 15}, + }, + }, + interval: 60 * time.Second, + }, + { + throughput: serviceOperationThroughput{ + "svcA": map[string]*model.Throughput{ + "GET": {Count: 30}, + }, + "svcB": map[string]*model.Throughput{ + "GET": {Count: 45}, + }, + }, + interval: 60 * time.Second, + }, + } +} + +func errTestStorage() error { + return errors.New("storage error") +} + +func testCalculator() calculationstrategy.ProbabilityCalculator { + return calculationstrategy.CalculateFunc(func(targetQPS, qps, oldProbability float64) float64 { + factor := targetQPS / qps + return oldProbability * factor + }) +} + +func TestAggregateThroughputInputsImmutability(t *testing.T) { + p := &Processor{} + in := testThroughputs() + _ = p.aggregateThroughput(in) + assert.Equal(t, in, testThroughputs()) +} + +func TestAggregateThroughput(t *testing.T) { + p := &Processor{} + aggregatedThroughput := p.aggregateThroughput(testThroughputs()) + require.Len(t, aggregatedThroughput, 2) + + throughput, ok := aggregatedThroughput["svcA"] + require.True(t, ok) + require.Len(t, throughput, 2) + + opThroughput, ok := throughput["GET"] + require.True(t, ok) + assert.Equal(t, int64(8), opThroughput.Count) + assert.Equal(t, map[string]struct{}{"0.1": {}, "0.2": {}}, opThroughput.Probabilities) + + opThroughput, ok = throughput["PUT"] + require.True(t, ok) + assert.Equal(t, int64(5), opThroughput.Count) + assert.Equal(t, map[string]struct{}{"0.1": {}}, opThroughput.Probabilities) + + throughput, ok = aggregatedThroughput["svcB"] + require.True(t, ok) + require.Len(t, throughput, 1) + + opThroughput, ok = throughput["GET"] + require.True(t, ok) + assert.Equal(t, int64(3), opThroughput.Count) + assert.Equal(t, map[string]struct{}{"0.1": {}}, opThroughput.Probabilities) +} + +func TestInitializeThroughput(t *testing.T) { + mockStorage := &smocks.Store{} + mockStorage.On("GetThroughput", time.Time{}.Add(time.Minute*19), time.Time{}.Add(time.Minute*20)). + Return(testThroughputs(), nil) + mockStorage.On("GetThroughput", time.Time{}.Add(time.Minute*18), time.Time{}.Add(time.Minute*19)). + Return([]*model.Throughput{{Service: "svcA", Operation: "GET", Count: 7}}, nil) + mockStorage.On("GetThroughput", time.Time{}.Add(time.Minute*17), time.Time{}.Add(time.Minute*18)). + Return([]*model.Throughput{}, nil) + p := &Processor{storage: mockStorage, Options: Options{CalculationInterval: time.Minute, AggregationBuckets: 3}} + p.initializeThroughput(time.Time{}.Add(time.Minute * 20)) + + require.Len(t, p.throughputs, 2) + require.Len(t, p.throughputs[0].throughput, 2) + assert.Equal(t, time.Minute, p.throughputs[0].interval) + assert.Equal(t, p.throughputs[0].endTime, time.Time{}.Add(time.Minute*20)) + require.Len(t, p.throughputs[1].throughput, 1) + assert.Equal(t, time.Minute, p.throughputs[1].interval) + assert.Equal(t, p.throughputs[1].endTime, time.Time{}.Add(time.Minute*19)) +} + +func TestInitializeThroughputFailure(t *testing.T) { + mockStorage := &smocks.Store{} + mockStorage.On("GetThroughput", time.Time{}.Add(time.Minute*19), time.Time{}.Add(time.Minute*20)). + Return(nil, errTestStorage()) + p := &Processor{storage: mockStorage, Options: Options{CalculationInterval: time.Minute, AggregationBuckets: 1}} + p.initializeThroughput(time.Time{}.Add(time.Minute * 20)) + + assert.Empty(t, p.throughputs) +} + +func TestCalculateQPS(t *testing.T) { + qps := calculateQPS(int64(90), 60*time.Second) + assert.Equal(t, 1.5, qps) + + qps = calculateQPS(int64(45), 60*time.Second) + assert.Equal(t, 0.75, qps) +} + +func TestGenerateOperationQPS(t *testing.T) { + p := &Processor{throughputs: testThroughputBuckets(), Options: Options{BucketsForCalculation: 10, AggregationBuckets: 10}} + svcOpQPS := p.throughputToQPS() + assert.Len(t, svcOpQPS, 2) + + opQPS, ok := svcOpQPS["svcA"] + require.True(t, ok) + require.Len(t, opQPS, 2) + + assert.Equal(t, []float64{0.75, 0.5}, opQPS["GET"]) + assert.Equal(t, []float64{1.0}, opQPS["PUT"]) + + opQPS, ok = svcOpQPS["svcB"] + require.True(t, ok) + require.Len(t, opQPS, 2) + + assert.Equal(t, []float64{0.5, 0.75}, opQPS["GET"]) + assert.Equal(t, []float64{0.25}, opQPS["PUT"]) + + // Test using the previous QPS if the throughput is not provided + p.prependThroughputBucket( + &throughputBucket{ + throughput: serviceOperationThroughput{ + "svcA": map[string]*model.Throughput{ + "GET": {Count: 30}, + }, + }, + interval: 60 * time.Second, + }, + ) + svcOpQPS = p.throughputToQPS() + require.Len(t, svcOpQPS, 2) + + opQPS, ok = svcOpQPS["svcA"] + require.True(t, ok) + require.Len(t, opQPS, 2) + + assert.Equal(t, []float64{0.5, 0.75, 0.5}, opQPS["GET"]) + assert.Equal(t, []float64{1.0}, opQPS["PUT"]) + + opQPS, ok = svcOpQPS["svcB"] + require.True(t, ok) + require.Len(t, opQPS, 2) + + assert.Equal(t, []float64{0.5, 0.75}, opQPS["GET"]) + assert.Equal(t, []float64{0.25}, opQPS["PUT"]) +} + +func TestGenerateOperationQPS_UseMostRecentBucketOnly(t *testing.T) { + p := &Processor{throughputs: testThroughputBuckets(), Options: Options{BucketsForCalculation: 1, AggregationBuckets: 10}} + svcOpQPS := p.throughputToQPS() + assert.Len(t, svcOpQPS, 2) + + opQPS, ok := svcOpQPS["svcA"] + require.True(t, ok) + require.Len(t, opQPS, 2) + + assert.Equal(t, []float64{0.75}, opQPS["GET"]) + assert.Equal(t, []float64{1.0}, opQPS["PUT"]) + + p.prependThroughputBucket( + &throughputBucket{ + throughput: serviceOperationThroughput{ + "svcA": map[string]*model.Throughput{ + "GET": {Count: 30}, + }, + }, + interval: 60 * time.Second, + }, + ) + + svcOpQPS = p.throughputToQPS() + require.Len(t, svcOpQPS, 2) + + opQPS, ok = svcOpQPS["svcA"] + require.True(t, ok) + require.Len(t, opQPS, 2) + + assert.Equal(t, []float64{0.5}, opQPS["GET"]) + assert.Equal(t, []float64{1.0}, opQPS["PUT"]) +} + +func TestCalculateWeightedQPS(t *testing.T) { + p := Processor{weightVectorCache: NewWeightVectorCache()} + assert.InDelta(t, 0.86735, p.calculateWeightedQPS([]float64{0.8, 1.2, 1.0}), 0.001) + assert.InDelta(t, 0.95197, p.calculateWeightedQPS([]float64{1.0, 1.0, 0.0, 0.0}), 0.001) + assert.Equal(t, 0.0, p.calculateWeightedQPS([]float64{})) +} + +func TestCalculateProbability(t *testing.T) { + throughputs := []*throughputBucket{ + { + throughput: serviceOperationThroughput{ + "svcA": map[string]*model.Throughput{ + "GET": {Probabilities: map[string]struct{}{"0.500000": {}}}, + }, + }, + }, + } + probabilities := model.ServiceOperationProbabilities{ + "svcA": map[string]float64{ + "GET": 0.5, + }, + } + cfg := Options{ + TargetSamplesPerSecond: 1.0, + DeltaTolerance: 0.2, + InitialSamplingProbability: 0.001, + MinSamplingProbability: 0.00001, + } + p := &Processor{ + Options: cfg, + probabilities: probabilities, + probabilityCalculator: testCalculator(), + throughputs: throughputs, + serviceCache: []SamplingCache{{"svcA": {}, "svcB": {}}}, + } + tests := []struct { + service string + operation string + qps float64 + expectedProbability float64 + errMsg string + }{ + {"svcA", "GET", 2.0, 0.25, "modify existing probability"}, + {"svcA", "PUT", 2.0, 0.0005, "modify default probability"}, + {"svcB", "GET", 0.9, 0.001, "qps within equivalence threshold"}, + {"svcB", "PUT", 0.000001, 1.0, "test max probability"}, + {"svcB", "DELETE", 1000000000, 0.00001, "test min probability"}, + {"svcB", "DELETE", 0.0, 0.002, "test 0 qps"}, + } + for _, test := range tests { + probability := p.calculateProbability(test.service, test.operation, test.qps) + assert.Equal(t, test.expectedProbability, probability, test.errMsg) + } +} + +func TestCalculateProbabilitiesAndQPS(t *testing.T) { + prevProbabilities := model.ServiceOperationProbabilities{ + "svcB": map[string]float64{ + "GET": 0.16, + "PUT": 0.03, + }, + } + qps := model.ServiceOperationQPS{ + "svcB": map[string]float64{ + "GET": 0.625, + }, + } + mets := metricstest.NewFactory(0) + p := &Processor{ + Options: Options{ + TargetSamplesPerSecond: 1.0, + DeltaTolerance: 0.2, + InitialSamplingProbability: 0.001, + BucketsForCalculation: 10, + }, + throughputs: testThroughputBuckets(), probabilities: prevProbabilities, qps: qps, + weightVectorCache: NewWeightVectorCache(), probabilityCalculator: testCalculator(), + operationsCalculatedGauge: mets.Gauge(metrics.Options{Name: "test"}), + } + probabilities, qps := p.calculateProbabilitiesAndQPS() + + require.Len(t, probabilities, 2) + assert.Equal(t, map[string]float64{"GET": 0.00136, "PUT": 0.001}, probabilities["svcA"]) + assert.Equal(t, map[string]float64{"GET": 0.16, "PUT": 0.03}, probabilities["svcB"]) + + require.Len(t, qps, 2) + assert.Equal(t, map[string]float64{"GET": 0.7352941176470588, "PUT": 1}, qps["svcA"]) + assert.Equal(t, map[string]float64{"GET": 0.5147058823529411, "PUT": 0.25}, qps["svcB"]) + + _, gauges := mets.Backend.Snapshot() + assert.EqualValues(t, 4, gauges["test"]) +} + +func TestRunCalculationLoop(t *testing.T) { + logger := zap.NewNop() + mockStorage := &smocks.Store{} + mockStorage.On("GetThroughput", mock.AnythingOfType("time.Time"), mock.AnythingOfType("time.Time")). + Return(testThroughputs(), nil) + mockStorage.On("GetLatestProbabilities").Return(model.ServiceOperationProbabilities{}, errTestStorage()) + mockStorage.On("InsertProbabilitiesAndQPS", "host", mock.AnythingOfType("model.ServiceOperationProbabilities"), + mock.AnythingOfType("model.ServiceOperationQPS")).Return(errTestStorage()) + mockEP := &epmocks.ElectionParticipant{} + mockEP.On("Start").Return(nil) + mockEP.On("Close").Return(nil) + mockEP.On("IsLeader").Return(true) + + cfg := Options{ + TargetSamplesPerSecond: 1.0, + DeltaTolerance: 0.1, + InitialSamplingProbability: 0.001, + CalculationInterval: time.Millisecond * 5, + AggregationBuckets: 2, + Delay: time.Millisecond * 5, + LeaderLeaseRefreshInterval: time.Millisecond, + FollowerLeaseRefreshInterval: time.Second, + BucketsForCalculation: 10, + } + p, err := newProcessor(cfg, "host", mockStorage, mockEP, metrics.NullFactory, logger) + require.NoError(t, err) + p.Start() + + for i := 0; i < 1000; i++ { + strategy, _ := p.GetSamplingStrategy(context.Background(), "svcA") + if len(strategy.OperationSampling.PerOperationStrategies) != 0 { + break + } + time.Sleep(time.Millisecond) + } + p.Close() + + strategy, err := p.GetSamplingStrategy(context.Background(), "svcA") + require.NoError(t, err) + assert.Len(t, strategy.OperationSampling.PerOperationStrategies, 2) +} + +func TestRunCalculationLoop_GetThroughputError(t *testing.T) { + logger, logBuffer := testutils.NewLogger() + mockStorage := &smocks.Store{} + mockStorage.On("GetThroughput", mock.AnythingOfType("time.Time"), mock.AnythingOfType("time.Time")). + Return(nil, errTestStorage()) + mockEP := &epmocks.ElectionParticipant{} + mockEP.On("Start").Return(nil) + mockEP.On("Close").Return(nil) + mockEP.On("IsLeader").Return(false) + + cfg := Options{ + CalculationInterval: time.Millisecond * 5, + AggregationBuckets: 2, + BucketsForCalculation: 10, + } + p, err := newProcessor(cfg, "host", mockStorage, mockEP, metrics.NullFactory, logger) + require.NoError(t, err) + p.shutdown = make(chan struct{}) + defer close(p.shutdown) + go p.runCalculationLoop() + + for i := 0; i < 1000; i++ { + // match logs specific to getThroughputErrMsg. We expect to see more than 2, once during + // initialization and one or more times during the loop. + if match, _ := testutils.LogMatcher(2, getThroughputErrMsg, logBuffer.Lines()); match { + break + } + time.Sleep(time.Millisecond) + } + match, errMsg := testutils.LogMatcher(2, getThroughputErrMsg, logBuffer.Lines()) + assert.True(t, match, errMsg) +} + +func TestLoadProbabilities(t *testing.T) { + mockStorage := &smocks.Store{} + mockStorage.On("GetLatestProbabilities").Return(make(model.ServiceOperationProbabilities), nil) + + p := &Processor{storage: mockStorage} + require.Nil(t, p.probabilities) + p.loadProbabilities() + require.NotNil(t, p.probabilities) +} + +func TestRunUpdateProbabilitiesLoop(t *testing.T) { + mockStorage := &smocks.Store{} + mockStorage.On("GetLatestProbabilities").Return(make(model.ServiceOperationProbabilities), nil) + mockEP := &epmocks.ElectionParticipant{} + mockEP.On("Start").Return(nil) + mockEP.On("Close").Return(nil) + mockEP.On("IsLeader").Return(false) + + p := &Processor{ + storage: mockStorage, + shutdown: make(chan struct{}), + followerRefreshInterval: time.Millisecond, + electionParticipant: mockEP, + } + defer close(p.shutdown) + require.Nil(t, p.probabilities) + require.Nil(t, p.strategyResponses) + go p.runUpdateProbabilitiesLoop() + + for i := 0; i < 1000; i++ { + p.RLock() + if p.probabilities != nil && p.strategyResponses != nil { + p.RUnlock() + break + } + p.RUnlock() + time.Sleep(time.Millisecond) + } + p.RLock() + assert.NotNil(t, p.probabilities) + assert.NotNil(t, p.strategyResponses) + p.RUnlock() +} + +func TestRealisticRunCalculationLoop(t *testing.T) { + t.Skip("Skipped realistic calculation loop test") + logger := zap.NewNop() + // NB: This is an extremely long test since it uses near realistic (1/6th scale) processor config values + testThroughputs := []*model.Throughput{ + {Service: "svcA", Operation: "GET", Count: 10}, + {Service: "svcA", Operation: "POST", Count: 9}, + {Service: "svcA", Operation: "PUT", Count: 5}, + {Service: "svcA", Operation: "DELETE", Count: 20}, + } + mockStorage := &smocks.Store{} + mockStorage.On("GetThroughput", mock.AnythingOfType("time.Time"), mock.AnythingOfType("time.Time")). + Return(testThroughputs, nil) + mockStorage.On("GetLatestProbabilities").Return(make(model.ServiceOperationProbabilities), nil) + mockStorage.On("InsertProbabilitiesAndQPS", "host", mock.AnythingOfType("model.ServiceOperationProbabilities"), + mock.AnythingOfType("model.ServiceOperationQPS")).Return(nil) + mockEP := &epmocks.ElectionParticipant{} + mockEP.On("Start").Return(nil) + mockEP.On("Close").Return(nil) + mockEP.On("IsLeader").Return(true) + cfg := Options{ + TargetSamplesPerSecond: 1.0, + DeltaTolerance: 0.2, + InitialSamplingProbability: 0.001, + CalculationInterval: time.Second * 10, + AggregationBuckets: 1, + Delay: time.Second * 10, + } + p, err := newProcessor(cfg, "host", mockStorage, mockEP, metrics.NullFactory, logger) + require.NoError(t, err) + p.Start() + + for i := 0; i < 100; i++ { + strategy, _ := p.GetSamplingStrategy(context.Background(), "svcA") + if len(strategy.OperationSampling.PerOperationStrategies) != 0 { + break + } + time.Sleep(250 * time.Millisecond) + } + p.Close() + + strategy, err := p.GetSamplingStrategy(context.Background(), "svcA") + require.NoError(t, err) + require.Len(t, strategy.OperationSampling.PerOperationStrategies, 4) + strategies := strategy.OperationSampling.PerOperationStrategies + + for _, s := range strategies { + switch s.Operation { + case "GET": + assert.Equal(t, 0.001, s.ProbabilisticSampling.SamplingRate, + "Already at 1QPS, no probability change") + case "POST": + assert.Equal(t, 0.001, s.ProbabilisticSampling.SamplingRate, + "Within epsilon of 1QPS, no probability change") + case "PUT": + assert.InEpsilon(t, 0.002, s.ProbabilisticSampling.SamplingRate, 0.025, + "Under sampled, double probability") + case "DELETE": + assert.InEpsilon(t, 0.0005, s.ProbabilisticSampling.SamplingRate, 0.025, + "Over sampled, halve probability") + } + } +} + +func TestPrependBucket(t *testing.T) { + p := &Processor{Options: Options{AggregationBuckets: 1}} + p.prependThroughputBucket(&throughputBucket{interval: time.Minute}) + require.Len(t, p.throughputs, 1) + assert.Equal(t, time.Minute, p.throughputs[0].interval) + + p.prependThroughputBucket(&throughputBucket{interval: 2 * time.Minute}) + require.Len(t, p.throughputs, 1) + assert.Equal(t, 2*time.Minute, p.throughputs[0].interval) +} + +func TestConstructorFailure(t *testing.T) { + logger := zap.NewNop() + + cfg := Options{ + TargetSamplesPerSecond: 1.0, + DeltaTolerance: 0.2, + InitialSamplingProbability: 0.001, + CalculationInterval: time.Second * 5, + AggregationBuckets: 0, + } + _, err := newProcessor(cfg, "host", nil, nil, metrics.NullFactory, logger) + require.EqualError(t, err, "CalculationInterval and AggregationBuckets must be greater than 0") + + cfg.CalculationInterval = 0 + _, err = newProcessor(cfg, "host", nil, nil, metrics.NullFactory, logger) + require.EqualError(t, err, "CalculationInterval and AggregationBuckets must be greater than 0") + + cfg.CalculationInterval = time.Millisecond + cfg.AggregationBuckets = 1 + cfg.BucketsForCalculation = -1 + _, err = newProcessor(cfg, "host", nil, nil, metrics.NullFactory, logger) + require.EqualError(t, err, "BucketsForCalculation cannot be less than 1") +} + +func TestGenerateStrategyResponses(t *testing.T) { + probabilities := model.ServiceOperationProbabilities{ + "svcA": map[string]float64{ + "GET": 0.5, + }, + } + p := &Processor{ + probabilities: probabilities, + Options: Options{ + InitialSamplingProbability: 0.001, + MinSamplesPerSecond: 0.0001, + }, + } + p.generateStrategyResponses() + + expectedResponse := map[string]*api_v2.SamplingStrategyResponse{ + "svcA": { + StrategyType: api_v2.SamplingStrategyType_PROBABILISTIC, + OperationSampling: &api_v2.PerOperationSamplingStrategies{ + DefaultSamplingProbability: 0.001, + DefaultLowerBoundTracesPerSecond: 0.0001, + PerOperationStrategies: []*api_v2.OperationSamplingStrategy{ + { + Operation: "GET", + ProbabilisticSampling: &api_v2.ProbabilisticSamplingStrategy{ + SamplingRate: 0.5, + }, + }, + }, + }, + }, + } + assert.Equal(t, expectedResponse, p.strategyResponses) +} + +func TestUsingAdaptiveSampling(t *testing.T) { + p := &Processor{} + throughput := serviceOperationThroughput{ + "svc": map[string]*model.Throughput{ + "op": {Probabilities: map[string]struct{}{"0.010000": {}}}, + }, + } + tests := []struct { + expected bool + probability float64 + service string + operation string + }{ + {expected: true, probability: 0.01, service: "svc", operation: "op"}, + {expected: true, probability: 0.0099999384, service: "svc", operation: "op"}, + {expected: false, probability: 0.01, service: "non-svc"}, + {expected: false, probability: 0.01, service: "svc", operation: "non-op"}, + {expected: false, probability: 0.01, service: "svc", operation: "non-op"}, + {expected: false, probability: 0.02, service: "svc", operation: "op"}, + {expected: false, probability: 0.0100009384, service: "svc", operation: "op"}, + } + for _, test := range tests { + assert.Equal(t, test.expected, p.isUsingAdaptiveSampling(test.probability, test.service, test.operation, throughput)) + } +} + +func TestPrependServiceCache(t *testing.T) { + p := &Processor{} + for i := 0; i < serviceCacheSize*2; i++ { + p.prependServiceCache() + } + assert.Len(t, p.serviceCache, serviceCacheSize) +} + +func TestCalculateProbabilitiesAndQPSMultiple(t *testing.T) { + buckets := []*throughputBucket{ + { + throughput: serviceOperationThroughput{ + "svcA": map[string]*model.Throughput{ + "GET": {Count: 3, Probabilities: map[string]struct{}{"0.001000": {}}}, + "PUT": {Count: 60, Probabilities: map[string]struct{}{"0.001000": {}}}, + }, + "svcB": map[string]*model.Throughput{ + "PUT": {Count: 15, Probabilities: map[string]struct{}{"0.001000": {}}}, + }, + }, + interval: 60 * time.Second, + }, + } + + p := &Processor{ + Options: Options{ + TargetSamplesPerSecond: 1.0, + DeltaTolerance: 0.002, + InitialSamplingProbability: 0.001, + BucketsForCalculation: 5, + AggregationBuckets: 10, + }, + throughputs: buckets, probabilities: make(model.ServiceOperationProbabilities), + qps: make(model.ServiceOperationQPS), weightVectorCache: NewWeightVectorCache(), + probabilityCalculator: calculationstrategy.NewPercentageIncreaseCappedCalculator(1.0), + serviceCache: []SamplingCache{}, + operationsCalculatedGauge: metrics.NullFactory.Gauge(metrics.Options{}), + } + + probabilities, qps := p.calculateProbabilitiesAndQPS() + + require.Len(t, probabilities, 2) + assert.Equal(t, map[string]float64{"GET": 0.002, "PUT": 0.001}, probabilities["svcA"]) + assert.Equal(t, map[string]float64{"PUT": 0.002}, probabilities["svcB"]) + + p.probabilities = probabilities + p.qps = qps + + // svcA:GET is no longer reported, we should not increase it's probability since we don't know if it's adaptively sampled + // until we get at least a lowerbound span or a probability span with the right probability. + // svcB:PUT is only reporting lowerbound, we should boost it's probability + p.prependThroughputBucket(&throughputBucket{ + throughput: serviceOperationThroughput{ + "svcA": map[string]*model.Throughput{ + "PUT": {Count: 60, Probabilities: map[string]struct{}{"0.001000": {}}}, + }, + "svcB": map[string]*model.Throughput{ + "GET": {Count: 30, Probabilities: map[string]struct{}{"0.001000": {}}}, + "PUT": {Count: 0, Probabilities: map[string]struct{}{"0.002000": {}}}, + }, + }, + interval: 60 * time.Second, + }) + + probabilities, qps = p.calculateProbabilitiesAndQPS() + + require.Len(t, probabilities, 2) + assert.Equal(t, map[string]float64{"GET": 0.002, "PUT": 0.001}, probabilities["svcA"]) + assert.Equal(t, map[string]float64{"PUT": 0.004, "GET": 0.002}, probabilities["svcB"]) + + p.probabilities = probabilities + p.qps = qps + + // svcA:GET is lower bound sampled, increase its probability + // svcB:PUT is not reported but we should boost it's probability since the previous calculation showed that + // it's using adaptive sampling + p.prependThroughputBucket(&throughputBucket{ + throughput: serviceOperationThroughput{ + "svcA": map[string]*model.Throughput{ + "GET": {Count: 0, Probabilities: map[string]struct{}{"0.002000": {}}}, + "PUT": {Count: 60, Probabilities: map[string]struct{}{"0.001000": {}}}, + }, + "svcB": map[string]*model.Throughput{ + "GET": {Count: 30, Probabilities: map[string]struct{}{"0.001000": {}}}, + }, + }, + interval: 60 * time.Second, + }) + + probabilities, qps = p.calculateProbabilitiesAndQPS() + + require.Len(t, probabilities, 2) + assert.Equal(t, map[string]float64{"GET": 0.004, "PUT": 0.001}, probabilities["svcA"]) + assert.Equal(t, map[string]float64{"PUT": 0.008, "GET": 0.002}, probabilities["svcB"]) + + p.probabilities = probabilities + p.qps = qps + + // svcA:GET is finally adaptively probabilistically sampled! + // svcB:PUT stopped using adaptive sampling + p.prependThroughputBucket(&throughputBucket{ + throughput: serviceOperationThroughput{ + "svcA": map[string]*model.Throughput{ + "GET": {Count: 1, Probabilities: map[string]struct{}{"0.004000": {}}}, + "PUT": {Count: 60, Probabilities: map[string]struct{}{"0.001000": {}}}, + }, + "svcB": map[string]*model.Throughput{ + "GET": {Count: 30, Probabilities: map[string]struct{}{"0.001000": {}}}, + "PUT": {Count: 15, Probabilities: map[string]struct{}{"0.001000": {}}}, + }, + }, + interval: 60 * time.Second, + }) + + probabilities, qps = p.calculateProbabilitiesAndQPS() + + require.Len(t, probabilities, 2) + assert.Equal(t, map[string]float64{"GET": 0.008, "PUT": 0.001}, probabilities["svcA"]) + assert.Equal(t, map[string]float64{"PUT": 0.008, "GET": 0.002}, probabilities["svcB"]) + + p.probabilities = probabilities + p.qps = qps + + // svcA:GET didn't report anything + p.prependThroughputBucket(&throughputBucket{ + throughput: serviceOperationThroughput{ + "svcA": map[string]*model.Throughput{ + "PUT": {Count: 30, Probabilities: map[string]struct{}{"0.001000": {}}}, + }, + "svcB": map[string]*model.Throughput{ + "GET": {Count: 30, Probabilities: map[string]struct{}{"0.001000": {}}}, + "PUT": {Count: 15, Probabilities: map[string]struct{}{"0.001000": {}}}, + }, + }, + interval: 60 * time.Second, + }) + + probabilities, qps = p.calculateProbabilitiesAndQPS() + + require.Len(t, probabilities, 2) + assert.Equal(t, map[string]float64{"GET": 0.016, "PUT": 0.001468867216804201}, probabilities["svcA"]) + assert.Equal(t, map[string]float64{"PUT": 0.008, "GET": 0.002}, probabilities["svcB"]) + + p.probabilities = probabilities + p.qps = qps + + // svcA:GET didn't report anything + // svcB:PUT starts to use adaptive sampling again + p.prependThroughputBucket(&throughputBucket{ + throughput: serviceOperationThroughput{ + "svcA": map[string]*model.Throughput{ + "PUT": {Count: 30, Probabilities: map[string]struct{}{"0.001000": {}}}, + }, + "svcB": map[string]*model.Throughput{ + "GET": {Count: 30, Probabilities: map[string]struct{}{"0.001000": {}}}, + "PUT": {Count: 1, Probabilities: map[string]struct{}{"0.008000": {}}}, + }, + }, + interval: 60 * time.Second, + }) + + probabilities, qps = p.calculateProbabilitiesAndQPS() + + require.Len(t, probabilities, 2) + assert.Equal(t, map[string]float64{"GET": 0.032, "PUT": 0.001468867216804201}, probabilities["svcA"]) + assert.Equal(t, map[string]float64{"PUT": 0.016, "GET": 0.002}, probabilities["svcB"]) + + p.probabilities = probabilities + p.qps = qps + + // svcA:GET didn't report anything + // svcB:PUT didn't report anything + p.prependThroughputBucket(&throughputBucket{ + throughput: serviceOperationThroughput{ + "svcA": map[string]*model.Throughput{ + "PUT": {Count: 30, Probabilities: map[string]struct{}{"0.001000": {}}}, + }, + "svcB": map[string]*model.Throughput{ + "GET": {Count: 15, Probabilities: map[string]struct{}{"0.001000": {}}}, + }, + }, + interval: 60 * time.Second, + }) + + probabilities, qps = p.calculateProbabilitiesAndQPS() + + require.Len(t, probabilities, 2) + assert.Equal(t, map[string]float64{"GET": 0.064, "PUT": 0.001468867216804201}, probabilities["svcA"]) + assert.Equal(t, map[string]float64{"PUT": 0.032, "GET": 0.002}, probabilities["svcB"]) + + p.probabilities = probabilities + p.qps = qps + + // svcA:GET didn't report anything + // svcB:PUT didn't report anything + p.prependThroughputBucket(&throughputBucket{ + throughput: serviceOperationThroughput{ + "svcA": map[string]*model.Throughput{ + "PUT": {Count: 20, Probabilities: map[string]struct{}{"0.001000": {}}}, + }, + "svcB": map[string]*model.Throughput{ + "GET": {Count: 10, Probabilities: map[string]struct{}{"0.001000": {}}}, + }, + }, + interval: 60 * time.Second, + }) + + probabilities, qps = p.calculateProbabilitiesAndQPS() + + require.Len(t, probabilities, 2) + assert.Equal(t, map[string]float64{"GET": 0.128, "PUT": 0.001468867216804201}, probabilities["svcA"]) + assert.Equal(t, map[string]float64{"PUT": 0.064, "GET": 0.002}, probabilities["svcB"]) + + p.probabilities = probabilities + p.qps = qps + + // svcA:GET didn't report anything + // svcB:PUT didn't report anything + p.prependThroughputBucket(&throughputBucket{ + throughput: serviceOperationThroughput{ + "svcA": map[string]*model.Throughput{ + "PUT": {Count: 20, Probabilities: map[string]struct{}{"0.001000": {}}}, + "GET": {Count: 120, Probabilities: map[string]struct{}{"0.128000": {}}}, + }, + "svcB": map[string]*model.Throughput{ + "PUT": {Count: 60, Probabilities: map[string]struct{}{"0.064000": {}}}, + "GET": {Count: 10, Probabilities: map[string]struct{}{"0.001000": {}}}, + }, + }, + interval: 60 * time.Second, + }) + + probabilities, qps = p.calculateProbabilitiesAndQPS() + + require.Len(t, probabilities, 2) + assert.Equal(t, map[string]float64{"GET": 0.0882586677054928, "PUT": 0.001468867216804201}, probabilities["svcA"]) + assert.Equal(t, map[string]float64{"PUT": 0.09587513707888091, "GET": 0.002}, probabilities["svcB"]) + + p.probabilities = probabilities + p.qps = qps +} + +func TestErrors(t *testing.T) { + mockStorage := &smocks.Store{} + mockStorage.On("GetLatestProbabilities").Return(model.ServiceOperationProbabilities{}, errTestStorage()) + mockStorage.On("GetThroughput", mock.AnythingOfType("time.Time"), mock.AnythingOfType("time.Time")). + Return(nil, nil) + + cfg := Options{ + TargetSamplesPerSecond: 1.0, + DeltaTolerance: 0.1, + InitialSamplingProbability: 0.001, + CalculationInterval: time.Millisecond * 5, + AggregationBuckets: 2, + Delay: time.Millisecond * 5, + LeaderLeaseRefreshInterval: time.Millisecond, + FollowerLeaseRefreshInterval: time.Second, + BucketsForCalculation: 10, + } + + // start errors + mockEP := &epmocks.ElectionParticipant{} + mockEP.On("Start").Return(errors.New("bad")) + mockEP.On("Close").Return(errors.New("also bad")) + mockEP.On("IsLeader").Return(false) + + p, err := newProcessor(cfg, "host", mockStorage, mockEP, metrics.NullFactory, zap.NewNop()) + require.NoError(t, err) + require.Error(t, p.Start()) + require.Error(t, p.Close()) + + // close errors + mockEP = &epmocks.ElectionParticipant{} + mockEP.On("Start").Return(nil) + mockEP.On("Close").Return(errors.New("still bad")) + mockEP.On("IsLeader").Return(false) + + p, err = newProcessor(cfg, "host", mockStorage, mockEP, metrics.NullFactory, zap.NewNop()) + require.NoError(t, err) + require.NoError(t, p.Start()) + require.Error(t, p.Close()) +} diff --git a/plugin/sampling/strategystore/adaptive/strategy_store.go b/plugin/sampling/strategystore/adaptive/strategy_store.go index fe2be9ac608..5a6fc67f4f0 100644 --- a/plugin/sampling/strategystore/adaptive/strategy_store.go +++ b/plugin/sampling/strategystore/adaptive/strategy_store.go @@ -15,7 +15,6 @@ package adaptive import ( - "context" "sync" "time" @@ -79,102 +78,6 @@ func (ss *StrategyStore) Start() error { return nil } -// GetSamplingStrategy implements Thrift endpoint for retrieving sampling strategy for a service. -func (ss *StrategyStore) GetSamplingStrategy(_ context.Context, service string) (*api_v2.SamplingStrategyResponse, error) { - ss.RLock() - defer ss.RUnlock() - if strategy, ok := ss.strategyResponses[service]; ok { - return strategy, nil - } - return ss.generateDefaultSamplingStrategyResponse(), nil -} - -func (ss *StrategyStore) loadProbabilities() { - // TODO GetLatestProbabilities API can be changed to return the latest measured qps for initialization - probabilities, err := ss.storage.GetLatestProbabilities() - if err != nil { - ss.logger.Warn("failed to initialize probabilities", zap.Error(err)) - return - } - ss.Lock() - defer ss.Unlock() - ss.probabilities = probabilities -} - -// runUpdateProbabilitiesLoop is a loop that reads probabilities from storage. -// The follower updates its local cache with the latest probabilities and serves them. -func (ss *StrategyStore) runUpdateProbabilitiesLoop() { - select { - case <-time.After(addJitter(ss.followerRefreshInterval)): - case <-ss.shutdown: - return - } - - ticker := time.NewTicker(ss.followerRefreshInterval) - defer ticker.Stop() - for { - select { - case <-ticker.C: - // Only load probabilities if this processor doesn't hold the leader lock - if !ss.isLeader() { - ss.loadProbabilities() - ss.generateStrategyResponses() - } - case <-ss.shutdown: - return - } - } -} - -func (ss *StrategyStore) isLeader() bool { - return ss.electionParticipant.IsLeader() -} - -// generateStrategyResponses generates and caches SamplingStrategyResponse from the calculated sampling probabilities. -func (ss *StrategyStore) generateStrategyResponses() { - ss.RLock() - strategies := make(map[string]*api_v2.SamplingStrategyResponse) - for svc, opProbabilities := range ss.probabilities { - opStrategies := make([]*api_v2.OperationSamplingStrategy, len(opProbabilities)) - var idx int - for op, probability := range opProbabilities { - opStrategies[idx] = &api_v2.OperationSamplingStrategy{ - Operation: op, - ProbabilisticSampling: &api_v2.ProbabilisticSamplingStrategy{ - SamplingRate: probability, - }, - } - idx++ - } - strategy := ss.generateDefaultSamplingStrategyResponse() - strategy.OperationSampling.PerOperationStrategies = opStrategies - strategies[svc] = strategy - } - ss.RUnlock() - - ss.Lock() - defer ss.Unlock() - ss.strategyResponses = strategies -} - -func (ss *StrategyStore) generateDefaultSamplingStrategyResponse() *api_v2.SamplingStrategyResponse { - return &api_v2.SamplingStrategyResponse{ - StrategyType: api_v2.SamplingStrategyType_PROBABILISTIC, - OperationSampling: &api_v2.PerOperationSamplingStrategies{ - DefaultSamplingProbability: ss.InitialSamplingProbability, - DefaultLowerBoundTracesPerSecond: ss.MinSamplesPerSecond, - }, - } -} - -func (ss *StrategyStore) runBackground(f func()) { - ss.bgFinished.Add(1) - go func() { - f() - ss.bgFinished.Done() - }() -} - // Close stops the processor from calculating probabilities. func (ss *StrategyStore) Close() error { ss.logger.Info("stopping adaptive sampling processor") From d5b2bed60055d8bb83305984c108e81911f4871d Mon Sep 17 00:00:00 2001 From: Pushkar Mishra Date: Thu, 16 May 2024 00:04:16 +0530 Subject: [PATCH 09/18] comment updated Signed-off-by: Pushkar Mishra --- .../strategystore/adaptive/processor.go | 14 +++----------- .../strategystore/adaptive/strategy_store.go | 19 +++++++++++++------ 2 files changed, 16 insertions(+), 17 deletions(-) diff --git a/plugin/sampling/strategystore/adaptive/processor.go b/plugin/sampling/strategystore/adaptive/processor.go index 4f59330d515..5bd650cbd69 100644 --- a/plugin/sampling/strategystore/adaptive/processor.go +++ b/plugin/sampling/strategystore/adaptive/processor.go @@ -140,7 +140,7 @@ func newPostAggregator( }, nil } -// GetSamplingStrategy implements Thrift endpoint for retrieving sampling strategy for a service. +// GetSamplingStrategy implements protobuf endpoint for retrieving sampling strategy for a service. func (ss *StrategyStore) GetSamplingStrategy(_ context.Context, service string) (*api_v2.SamplingStrategyResponse, error) { ss.RLock() defer ss.RUnlock() @@ -150,7 +150,7 @@ func (ss *StrategyStore) GetSamplingStrategy(_ context.Context, service string) return ss.generateDefaultSamplingStrategyResponse(), nil } -// Start initializes and starts the sampling processor which regularly calculates sampling probabilities. +// Start initializes and starts the PostAggregator which regularly calculates sampling probabilities. func (p *PostAggregator) Start() error { p.logger.Info("starting adaptive sampling processor") if err := p.electionParticipant.Start(); err != nil { @@ -168,15 +168,7 @@ func (p *PostAggregator) runBackground(f func()) { }() } -func (ss *StrategyStore) runBackground(f func()) { - ss.bgFinished.Add(1) - go func() { - f() - ss.bgFinished.Done() - }() -} - -// Close stops the processor from calculating probabilities. +// Close stops the PostAggregator from calculating probabilities. func (p *PostAggregator) Close() error { p.logger.Info("stopping adaptive sampling processor") err := p.electionParticipant.Close() diff --git a/plugin/sampling/strategystore/adaptive/strategy_store.go b/plugin/sampling/strategystore/adaptive/strategy_store.go index 5a6fc67f4f0..284ad5092b8 100644 --- a/plugin/sampling/strategystore/adaptive/strategy_store.go +++ b/plugin/sampling/strategystore/adaptive/strategy_store.go @@ -39,8 +39,7 @@ type StrategyStore struct { // probabilities contains the latest calculated sampling probabilities for service operations. probabilities model.ServiceOperationProbabilities - // strategyResponses is the cache of the sampling strategies for every service, in Thrift format. - // TODO change this to work with protobuf model instead, to support gRPC endpoint. + // strategyResponses is the cache of the sampling strategies for every service, in protobuf format. strategyResponses map[string]*api_v2.SamplingStrategyResponse // followerRefreshInterval determines how often the follower processor updates its probabilities. @@ -65,9 +64,9 @@ func NewStrategyStore(options Options, logger *zap.Logger, participant leaderele }, nil } -// Start initializes and starts the sampling processor which regularly calculates sampling probabilities. +// Start initializes and starts the sampling service which regularly loads sampling probabilities and generates strategies. func (ss *StrategyStore) Start() error { - ss.logger.Info("starting adaptive sampling processor") + ss.logger.Info("starting adaptive sampling service") if err := ss.electionParticipant.Start(); err != nil { return err } @@ -78,9 +77,9 @@ func (ss *StrategyStore) Start() error { return nil } -// Close stops the processor from calculating probabilities. +// Close stops the service from loading probabilities and generating strategies. func (ss *StrategyStore) Close() error { - ss.logger.Info("stopping adaptive sampling processor") + ss.logger.Info("stopping adaptive sampling service") err := ss.electionParticipant.Close() if ss.shutdown != nil { close(ss.shutdown) @@ -88,3 +87,11 @@ func (ss *StrategyStore) Close() error { ss.bgFinished.Wait() return err } + +func (ss *StrategyStore) runBackground(f func()) { + ss.bgFinished.Add(1) + go func() { + f() + ss.bgFinished.Done() + }() +} From f1e8988a4e021283205aff674918a54b9626b934 Mon Sep 17 00:00:00 2001 From: Pushkar Mishra Date: Thu, 16 May 2024 17:26:04 +0530 Subject: [PATCH 10/18] fix Signed-off-by: Pushkar Mishra --- .../strategystore/adaptive/aggregator.go | 16 +-- .../strategystore/adaptive/processor.go | 130 +++++++++--------- .../strategystore/adaptive/strategy_store.go | 22 ++- 3 files changed, 81 insertions(+), 87 deletions(-) diff --git a/plugin/sampling/strategystore/adaptive/aggregator.go b/plugin/sampling/strategystore/adaptive/aggregator.go index cd5145ce456..536494bff87 100644 --- a/plugin/sampling/strategystore/adaptive/aggregator.go +++ b/plugin/sampling/strategystore/adaptive/aggregator.go @@ -40,7 +40,7 @@ type aggregator struct { operationsCounter metrics.Counter servicesCounter metrics.Counter currentThroughput serviceOperationThroughput - postAggregator *PostAggregator + processor *Processor aggregationInterval time.Duration storage samplingstore.Store stop chan struct{} @@ -56,7 +56,7 @@ func NewAggregator(options Options, logger *zap.Logger, metricsFactory metrics.F } logger.Info("Using unique participantName in adaptive sampling", zap.String("participantName", hostname)) - postAgg, err := newPostAggregator(options, hostname, store, participant, metricsFactory, logger) + processor, err := newProcessor(options, hostname, store, participant, metricsFactory, logger) if err != nil { return nil, err } @@ -66,7 +66,7 @@ func NewAggregator(options Options, logger *zap.Logger, metricsFactory metrics.F servicesCounter: metricsFactory.Counter(metrics.Options{Name: "sampling_services"}), currentThroughput: make(serviceOperationThroughput), aggregationInterval: options.CalculationInterval, - postAggregator: postAgg, + processor: processor, storage: store, stop: make(chan struct{}), }, nil @@ -74,17 +74,13 @@ func NewAggregator(options Options, logger *zap.Logger, metricsFactory metrics.F func (a *aggregator) runAggregationLoop() { ticker := time.NewTicker(a.aggregationInterval) - - // NB: the first tick will be slightly delayed by the initializeThroughput call. - a.postAggregator.lastCheckedTime = time.Now().Add(a.postAggregator.Delay * -1) - a.postAggregator.initializeThroughput(a.postAggregator.lastCheckedTime) for { select { case <-ticker.C: a.Lock() a.saveThroughput() a.currentThroughput = make(serviceOperationThroughput) - a.postAggregator.runCalculation() + a.processor.runCalculation() a.Unlock() case <-a.stop: ticker.Stop() @@ -135,13 +131,13 @@ func (a *aggregator) RecordThroughput(service, operation string, samplerType spa } func (a *aggregator) Start() { - a.postAggregator.Start() + a.processor.Start() a.runBackground(a.runAggregationLoop) } func (a *aggregator) Close() error { var errs []error - if err := a.postAggregator.Close(); err != nil { + if err := a.processor.Close(); err != nil { errs = append(errs, err) } if a.stop != nil { diff --git a/plugin/sampling/strategystore/adaptive/processor.go b/plugin/sampling/strategystore/adaptive/processor.go index 5bd650cbd69..5a8a7a3a7ad 100644 --- a/plugin/sampling/strategystore/adaptive/processor.go +++ b/plugin/sampling/strategystore/adaptive/processor.go @@ -69,11 +69,11 @@ type throughputBucket struct { endTime time.Time } -// PostAggregator retrieves service throughput over a look back interval and calculates sampling probabilities +// Processor retrieves service throughput over a look back interval and calculates sampling probabilities // per operation such that each operation is sampled at a specified target QPS. It achieves this by // retrieving discrete buckets of operation throughput and doing a weighted average of the throughput // and generating a probability to match the targetQPS. -type PostAggregator struct { +type Processor struct { sync.RWMutex Options @@ -108,14 +108,14 @@ type PostAggregator struct { } // newProcessor creates a new sampling processor that generates sampling rates for service operations. -func newPostAggregator( +func newProcessor( opts Options, hostname string, storage samplingstore.Store, electionParticipant leaderelection.ElectionParticipant, metricsFactory metrics.Factory, logger *zap.Logger, -) (*PostAggregator, error) { +) (*Processor, error) { if opts.CalculationInterval == 0 || opts.AggregationBuckets == 0 { return nil, errNonZero } @@ -123,7 +123,7 @@ func newPostAggregator( return nil, errBucketsForCalculation } metricsFactory = metricsFactory.Namespace(metrics.NSOptions{Name: "adaptive_sampling_processor"}) - return &PostAggregator{ + return &Processor{ Options: opts, storage: storage, probabilities: make(model.ServiceOperationProbabilities), @@ -137,88 +137,85 @@ func newPostAggregator( serviceCache: []SamplingCache{}, operationsCalculatedGauge: metricsFactory.Gauge(metrics.Options{Name: "operations_calculated"}), calculateProbabilitiesLatency: metricsFactory.Timer(metrics.TimerOptions{Name: "calculate_probabilities"}), + shutdown: make(chan struct{}), }, nil } // GetSamplingStrategy implements protobuf endpoint for retrieving sampling strategy for a service. -func (ss *StrategyStore) GetSamplingStrategy(_ context.Context, service string) (*api_v2.SamplingStrategyResponse, error) { - ss.RLock() - defer ss.RUnlock() - if strategy, ok := ss.strategyResponses[service]; ok { +func (p *StrategyStore) GetSamplingStrategy(_ context.Context, service string) (*api_v2.SamplingStrategyResponse, error) { + p.RLock() + defer p.RUnlock() + if strategy, ok := p.strategyResponses[service]; ok { return strategy, nil } - return ss.generateDefaultSamplingStrategyResponse(), nil + return p.generateDefaultSamplingStrategyResponse(), nil } -// Start initializes and starts the PostAggregator which regularly calculates sampling probabilities. -func (p *PostAggregator) Start() error { +// Start initializes and starts the sampling processor which regularly calculates sampling probabilities. +func (p *Processor) Start() error { p.logger.Info("starting adaptive sampling processor") if err := p.electionParticipant.Start(); err != nil { return err } - p.shutdown = make(chan struct{}) - return nil -} -func (p *PostAggregator) runBackground(f func()) { - p.bgFinished.Add(1) - go func() { - f() - p.bgFinished.Done() - }() + // NB: the first tick will be slightly delayed by the initializeThroughput call. + p.lastCheckedTime = time.Now().Add(p.Delay * -1) + p.initializeThroughput(p.lastCheckedTime) + return nil } -// Close stops the PostAggregator from calculating probabilities. -func (p *PostAggregator) Close() error { +// Close stops the Processor from calculating probabilities. +func (p *Processor) Close() error { p.logger.Info("stopping adaptive sampling processor") err := p.electionParticipant.Close() p.bgFinished.Wait() return err } -func (ss *StrategyStore) loadProbabilities() { +func (p *StrategyStore) loadProbabilities() { // TODO GetLatestProbabilities API can be changed to return the latest measured qps for initialization - probabilities, err := ss.storage.GetLatestProbabilities() + probabilities, err := p.storage.GetLatestProbabilities() if err != nil { - ss.logger.Warn("failed to initialize probabilities", zap.Error(err)) + p.logger.Warn("failed to initialize probabilities", zap.Error(err)) return } - ss.Lock() - defer ss.Unlock() - ss.probabilities = probabilities + p.Lock() + defer p.Unlock() + p.probabilities = probabilities } // runUpdateProbabilitiesLoop is a loop that reads probabilities from storage. // The follower updates its local cache with the latest probabilities and serves them. -func (ss *StrategyStore) runUpdateProbabilitiesLoop() { +func (p *StrategyStore) runUpdateProbabilitiesLoop() { select { - case <-time.After(addJitter(ss.followerRefreshInterval)): - case <-ss.shutdown: + case <-time.After(addJitter(p.followerRefreshInterval)): + // continue after jitter delay + case <-p.shutdown: return } - ticker := time.NewTicker(ss.followerRefreshInterval) + ticker := time.NewTicker(p.followerRefreshInterval) defer ticker.Stop() for { select { case <-ticker.C: // Only load probabilities if this processor doesn't hold the leader lock - if !ss.isLeader() { - ss.loadProbabilities() - ss.generateStrategyResponses() + if !p.isLeader() { + p.loadProbabilities() + p.generateStrategyResponses() } - case <-ss.shutdown: + case <-p.shutdown: return } } } -func (p *PostAggregator) isLeader() bool { +func (p *Processor) isLeader() bool { return p.electionParticipant.IsLeader() } -func (ss *StrategyStore) isLeader() bool { - return ss.electionParticipant.IsLeader() +func (p *StrategyStore) isLeader() bool { + return p.electionParticipant.IsLeader() } // addJitter adds a random amount of time. Without jitter, if the host holding the leader @@ -229,7 +226,7 @@ func addJitter(jitterAmount time.Duration) time.Duration { return (jitterAmount / 2) + time.Duration(rand.Int63n(int64(jitterAmount/2))) } -func (p *PostAggregator) runCalculation() { +func (p *Processor) runCalculation() { endTime := time.Now().Add(p.Delay * -1) startTime := p.lastCheckedTime throughput, err := p.storage.GetThroughput(startTime, endTime) @@ -257,11 +254,16 @@ func (p *PostAggregator) runCalculation() { p.Unlock() p.calculateProbabilitiesLatency.Record(time.Since(startTime)) - p.runBackground(p.saveProbabilitiesAndQPS) + + p.bgFinished.Add(1) + go func() { + p.saveProbabilitiesAndQPS() + p.bgFinished.Done() + }() } } -func (p *PostAggregator) saveProbabilitiesAndQPS() { +func (p *Processor) saveProbabilitiesAndQPS() { p.RLock() defer p.RUnlock() if err := p.storage.InsertProbabilitiesAndQPS(p.hostname, p.probabilities, p.qps); err != nil { @@ -269,7 +271,7 @@ func (p *PostAggregator) saveProbabilitiesAndQPS() { } } -func (p *PostAggregator) prependThroughputBucket(bucket *throughputBucket) { +func (p *Processor) prependThroughputBucket(bucket *throughputBucket) { p.throughputs = append([]*throughputBucket{bucket}, p.throughputs...) if len(p.throughputs) > p.AggregationBuckets { p.throughputs = p.throughputs[0:p.AggregationBuckets] @@ -279,7 +281,7 @@ func (p *PostAggregator) prependThroughputBucket(bucket *throughputBucket) { // aggregateThroughput aggregates operation throughput from different buckets into one. // All input buckets represent a single time range, but there are many of them because // they are all independently generated by different collector instances from inbound span traffic. -func (p *PostAggregator) aggregateThroughput(throughputs []*model.Throughput) serviceOperationThroughput { +func (p *Processor) aggregateThroughput(throughputs []*model.Throughput) serviceOperationThroughput { aggregatedThroughput := make(serviceOperationThroughput) for _, throughput := range throughputs { service := throughput.Service @@ -311,7 +313,7 @@ func copySet(in map[string]struct{}) map[string]struct{} { return out } -func (p *PostAggregator) initializeThroughput(endTime time.Time) { +func (p *Processor) initializeThroughput(endTime time.Time) { for i := 0; i < p.AggregationBuckets; i++ { startTime := endTime.Add(p.CalculationInterval * -1) throughput, err := p.storage.GetThroughput(startTime, endTime) @@ -333,7 +335,7 @@ func (p *PostAggregator) initializeThroughput(endTime time.Time) { } // throughputToQPS converts raw throughput counts for all accumulated buckets to QPS values. -func (p *PostAggregator) throughputToQPS() serviceOperationQPS { +func (p *Processor) throughputToQPS() serviceOperationQPS { // TODO previous qps buckets have already been calculated, just need to calculate latest batch // and append them where necessary and throw out the oldest batch. // Edge case #buckets < p.AggregationBuckets, then we shouldn't throw out @@ -361,7 +363,7 @@ func calculateQPS(count int64, interval time.Duration) float64 { // calculateWeightedQPS calculates the weighted qps of the slice allQPS where weights are biased // towards more recent qps. This function assumes that the most recent qps is at the head of the slice. -func (p *PostAggregator) calculateWeightedQPS(allQPS []float64) float64 { +func (p *Processor) calculateWeightedQPS(allQPS []float64) float64 { if len(allQPS) == 0 { return 0 } @@ -373,14 +375,14 @@ func (p *PostAggregator) calculateWeightedQPS(allQPS []float64) float64 { return qps } -func (p *PostAggregator) prependServiceCache() { +func (p *Processor) prependServiceCache() { p.serviceCache = append([]SamplingCache{make(SamplingCache)}, p.serviceCache...) if len(p.serviceCache) > serviceCacheSize { p.serviceCache = p.serviceCache[0:serviceCacheSize] } } -func (p *PostAggregator) calculateProbabilitiesAndQPS() (model.ServiceOperationProbabilities, model.ServiceOperationQPS) { +func (p *Processor) calculateProbabilitiesAndQPS() (model.ServiceOperationProbabilities, model.ServiceOperationQPS) { p.prependServiceCache() retProbabilities := make(model.ServiceOperationProbabilities) retQPS := make(model.ServiceOperationQPS) @@ -404,7 +406,7 @@ func (p *PostAggregator) calculateProbabilitiesAndQPS() (model.ServiceOperationP return retProbabilities, retQPS } -func (p *PostAggregator) calculateProbability(service, operation string, qps float64) float64 { +func (p *Processor) calculateProbability(service, operation string, qps float64) float64 { oldProbability := p.InitialSamplingProbability // TODO: is this loop overly expensive? p.RLock() @@ -439,7 +441,7 @@ func (p *PostAggregator) calculateProbability(service, operation string, qps flo } // is actual value within p.DeltaTolerance percentage of expected value. -func (p *PostAggregator) withinTolerance(actual, expected float64) bool { +func (p *Processor) withinTolerance(actual, expected float64) bool { return math.Abs(actual-expected)/expected < p.DeltaTolerance } @@ -451,7 +453,7 @@ func merge(p1 map[string]struct{}, p2 map[string]struct{}) map[string]struct{} { return p1 } -func (p *PostAggregator) isUsingAdaptiveSampling( +func (p *Processor) isUsingAdaptiveSampling( probability float64, service string, operation string, @@ -480,10 +482,10 @@ func (p *PostAggregator) isUsingAdaptiveSampling( } // generateStrategyResponses generates and caches SamplingStrategyResponse from the calculated sampling probabilities. -func (ss *StrategyStore) generateStrategyResponses() { - ss.RLock() +func (p *StrategyStore) generateStrategyResponses() { + p.RLock() strategies := make(map[string]*api_v2.SamplingStrategyResponse) - for svc, opProbabilities := range ss.probabilities { + for svc, opProbabilities := range p.probabilities { opStrategies := make([]*api_v2.OperationSamplingStrategy, len(opProbabilities)) var idx int for op, probability := range opProbabilities { @@ -495,23 +497,23 @@ func (ss *StrategyStore) generateStrategyResponses() { } idx++ } - strategy := ss.generateDefaultSamplingStrategyResponse() + strategy := p.generateDefaultSamplingStrategyResponse() strategy.OperationSampling.PerOperationStrategies = opStrategies strategies[svc] = strategy } - ss.RUnlock() + p.RUnlock() - ss.Lock() - defer ss.Unlock() - ss.strategyResponses = strategies + p.Lock() + defer p.Unlock() + p.strategyResponses = strategies } -func (ss *StrategyStore) generateDefaultSamplingStrategyResponse() *api_v2.SamplingStrategyResponse { +func (p *StrategyStore) generateDefaultSamplingStrategyResponse() *api_v2.SamplingStrategyResponse { return &api_v2.SamplingStrategyResponse{ StrategyType: api_v2.SamplingStrategyType_PROBABILISTIC, OperationSampling: &api_v2.PerOperationSamplingStrategies{ - DefaultSamplingProbability: ss.InitialSamplingProbability, - DefaultLowerBoundTracesPerSecond: ss.MinSamplesPerSecond, + DefaultSamplingProbability: p.InitialSamplingProbability, + DefaultLowerBoundTracesPerSecond: p.MinSamplesPerSecond, }, } } diff --git a/plugin/sampling/strategystore/adaptive/strategy_store.go b/plugin/sampling/strategystore/adaptive/strategy_store.go index 284ad5092b8..7539d32e579 100644 --- a/plugin/sampling/strategystore/adaptive/strategy_store.go +++ b/plugin/sampling/strategystore/adaptive/strategy_store.go @@ -61,6 +61,7 @@ func NewStrategyStore(options Options, logger *zap.Logger, participant leaderele logger: logger, electionParticipant: participant, followerRefreshInterval: defaultFollowerProbabilityInterval, + shutdown: make(chan struct{}), }, nil } @@ -70,10 +71,15 @@ func (ss *StrategyStore) Start() error { if err := ss.electionParticipant.Start(); err != nil { return err } - ss.shutdown = make(chan struct{}) ss.loadProbabilities() ss.generateStrategyResponses() - ss.runBackground(ss.runUpdateProbabilitiesLoop) + + ss.bgFinished.Add(1) + go func() { + ss.runUpdateProbabilitiesLoop() + ss.bgFinished.Done() + }() + return nil } @@ -81,17 +87,7 @@ func (ss *StrategyStore) Start() error { func (ss *StrategyStore) Close() error { ss.logger.Info("stopping adaptive sampling service") err := ss.electionParticipant.Close() - if ss.shutdown != nil { - close(ss.shutdown) - } + close(ss.shutdown) ss.bgFinished.Wait() return err } - -func (ss *StrategyStore) runBackground(f func()) { - ss.bgFinished.Add(1) - go func() { - f() - ss.bgFinished.Done() - }() -} From 5a5d5193ea10dc02d3151819a90e02e7b35e03c9 Mon Sep 17 00:00:00 2001 From: Pushkar Mishra Date: Thu, 16 May 2024 19:20:35 +0530 Subject: [PATCH 11/18] fix Signed-off-by: Pushkar Mishra --- .../sampling/strategystore/adaptive/aggregator.go | 15 ++++++--------- plugin/sampling/strategystore/adaptive/factory.go | 7 +++---- .../sampling/strategystore/adaptive/processor.go | 1 - 3 files changed, 9 insertions(+), 14 deletions(-) diff --git a/plugin/sampling/strategystore/adaptive/aggregator.go b/plugin/sampling/strategystore/adaptive/aggregator.go index 536494bff87..8ff561131c8 100644 --- a/plugin/sampling/strategystore/adaptive/aggregator.go +++ b/plugin/sampling/strategystore/adaptive/aggregator.go @@ -132,7 +132,12 @@ func (a *aggregator) RecordThroughput(service, operation string, samplerType spa func (a *aggregator) Start() { a.processor.Start() - a.runBackground(a.runAggregationLoop) + + a.bgFinished.Add(1) + go func() { + a.runAggregationLoop() + a.bgFinished.Done() + }() } func (a *aggregator) Close() error { @@ -146,11 +151,3 @@ func (a *aggregator) Close() error { a.bgFinished.Wait() return errors.Join(errs...) } - -func (a *aggregator) runBackground(f func()) { - a.bgFinished.Add(1) - go func() { - f() - a.bgFinished.Done() - }() -} diff --git a/plugin/sampling/strategystore/adaptive/factory.go b/plugin/sampling/strategystore/adaptive/factory.go index 317644b6eea..a237bdf7903 100644 --- a/plugin/sampling/strategystore/adaptive/factory.go +++ b/plugin/sampling/strategystore/adaptive/factory.go @@ -90,17 +90,16 @@ func (f *Factory) CreateStrategyStore() (strategystore.StrategyStore, strategyst Logger: f.logger, }) - ss, err := NewStrategyStore(*f.options, f.logger, participant, f.store) + p, err := NewStrategyStore(*f.options, f.logger, participant, f.store) if err != nil { return nil, nil, err } - ss.Start() - + p.Start() a, err := NewAggregator(*f.options, f.logger, f.metricsFactory, participant, f.store) if err != nil { return nil, nil, err } a.Start() - return ss, a, nil + return p, a, nil } diff --git a/plugin/sampling/strategystore/adaptive/processor.go b/plugin/sampling/strategystore/adaptive/processor.go index 5a8a7a3a7ad..63677e69d42 100644 --- a/plugin/sampling/strategystore/adaptive/processor.go +++ b/plugin/sampling/strategystore/adaptive/processor.go @@ -252,7 +252,6 @@ func (p *Processor) runCalculation() { p.probabilities = probabilities p.qps = qps p.Unlock() - p.calculateProbabilitiesLatency.Record(time.Since(startTime)) p.bgFinished.Add(1) From d4cfabc6339192dd93d3a72a43c328ba7710ac23 Mon Sep 17 00:00:00 2001 From: Pushkar Mishra Date: Thu, 16 May 2024 19:30:35 +0530 Subject: [PATCH 12/18] fix comment Signed-off-by: Pushkar Mishra --- plugin/sampling/strategystore/adaptive/processor.go | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/plugin/sampling/strategystore/adaptive/processor.go b/plugin/sampling/strategystore/adaptive/processor.go index 63677e69d42..18008ee7972 100644 --- a/plugin/sampling/strategystore/adaptive/processor.go +++ b/plugin/sampling/strategystore/adaptive/processor.go @@ -252,6 +252,13 @@ func (p *Processor) runCalculation() { p.probabilities = probabilities p.qps = qps p.Unlock() + // NB: This has the potential of running into a race condition if the CalculationInterval + // is set to an extremely low value. The worst case scenario is that probabilities is calculated + // and swapped more than once before generateStrategyResponses() and saveProbabilities() are called. + // This will result in one or more batches of probabilities not being saved which is completely + // fine. This race condition should not ever occur anyway since the calculation interval will + // be way longer than the time to run the calculations. + p.calculateProbabilitiesLatency.Record(time.Since(startTime)) p.bgFinished.Add(1) From 05d2e57ca05750ed542d483e5051d817475030ed Mon Sep 17 00:00:00 2001 From: Pushkar Mishra Date: Sat, 18 May 2024 16:52:45 +0530 Subject: [PATCH 13/18] fix Signed-off-by: Pushkar Mishra --- plugin/sampling/strategystore/adaptive/aggregator.go | 8 ++------ 1 file changed, 2 insertions(+), 6 deletions(-) diff --git a/plugin/sampling/strategystore/adaptive/aggregator.go b/plugin/sampling/strategystore/adaptive/aggregator.go index 8ff561131c8..4a673559148 100644 --- a/plugin/sampling/strategystore/adaptive/aggregator.go +++ b/plugin/sampling/strategystore/adaptive/aggregator.go @@ -142,12 +142,8 @@ func (a *aggregator) Start() { func (a *aggregator) Close() error { var errs []error - if err := a.processor.Close(); err != nil { - errs = append(errs, err) - } - if a.stop != nil { - close(a.stop) - } + errs = append(errs, a.processor.Close()) + close(a.stop) a.bgFinished.Wait() return errors.Join(errs...) } From f115849df788d57da70b3fc67ca41a7a96170168 Mon Sep 17 00:00:00 2001 From: pushkarm029 Date: Mon, 20 May 2024 18:10:00 +0530 Subject: [PATCH 14/18] fix Signed-off-by: pushkarm029 --- .../app/sampling/strategystore/factory.go | 2 ++ cmd/collector/main.go | 3 +++ .../strategystore/adaptive/aggregator.go | 5 +---- .../strategystore/adaptive/factory.go | 22 ++++++++++++------- .../strategystore/adaptive/processor.go | 12 ---------- .../strategystore/adaptive/strategy_store.go | 6 +---- plugin/sampling/strategystore/factory.go | 12 ++++++++++ .../sampling/strategystore/static/factory.go | 5 +++++ 8 files changed, 38 insertions(+), 29 deletions(-) diff --git a/cmd/collector/app/sampling/strategystore/factory.go b/cmd/collector/app/sampling/strategystore/factory.go index 5fab77b62bd..eb0371e9882 100644 --- a/cmd/collector/app/sampling/strategystore/factory.go +++ b/cmd/collector/app/sampling/strategystore/factory.go @@ -33,4 +33,6 @@ type Factory interface { // CreateStrategyStore initializes the StrategyStore and returns it. CreateStrategyStore() (StrategyStore, Aggregator, error) + + Close() error } diff --git a/cmd/collector/main.go b/cmd/collector/main.go index e8f1ca4db95..fdf85102e90 100644 --- a/cmd/collector/main.go +++ b/cmd/collector/main.go @@ -134,6 +134,9 @@ func main() { if err := storageFactory.Close(); err != nil { logger.Error("Failed to close storage factory", zap.Error(err)) } + if err := strategyStoreFactory.Close(); err != nil { + logger.Error("Failed to close sampling strategy store factory", zap.Error(err)) + } }) return nil }, diff --git a/plugin/sampling/strategystore/adaptive/aggregator.go b/plugin/sampling/strategystore/adaptive/aggregator.go index 4a673559148..f41bbd99ada 100644 --- a/plugin/sampling/strategystore/adaptive/aggregator.go +++ b/plugin/sampling/strategystore/adaptive/aggregator.go @@ -15,7 +15,6 @@ package adaptive import ( - "errors" "sync" "time" @@ -141,9 +140,7 @@ func (a *aggregator) Start() { } func (a *aggregator) Close() error { - var errs []error - errs = append(errs, a.processor.Close()) close(a.stop) a.bgFinished.Wait() - return errors.Join(errs...) + return nil } diff --git a/plugin/sampling/strategystore/adaptive/factory.go b/plugin/sampling/strategystore/adaptive/factory.go index a237bdf7903..fab9d207911 100644 --- a/plugin/sampling/strategystore/adaptive/factory.go +++ b/plugin/sampling/strategystore/adaptive/factory.go @@ -39,6 +39,7 @@ type Factory struct { metricsFactory metrics.Factory lock distributedlock.Lock store samplingstore.Store + participant *leaderelection.DistributedElectionParticipant } // NewFactory creates a new Factory. @@ -71,6 +72,11 @@ func (f *Factory) Initialize(metricsFactory metrics.Factory, ssFactory storage.S var err error f.logger = logger f.metricsFactory = metricsFactory + f.participant = leaderelection.NewElectionParticipant(f.lock, defaultResourceName, leaderelection.ElectionParticipantOptions{ + FollowerLeaseRefreshInterval: f.options.FollowerLeaseRefreshInterval, + LeaderLeaseRefreshInterval: f.options.LeaderLeaseRefreshInterval, + Logger: f.logger, + }) f.lock, err = ssFactory.CreateLock() if err != nil { return err @@ -84,18 +90,13 @@ func (f *Factory) Initialize(metricsFactory metrics.Factory, ssFactory storage.S // CreateStrategyStore implements strategystore.Factory func (f *Factory) CreateStrategyStore() (strategystore.StrategyStore, strategystore.Aggregator, error) { - participant := leaderelection.NewElectionParticipant(f.lock, defaultResourceName, leaderelection.ElectionParticipantOptions{ - FollowerLeaseRefreshInterval: f.options.FollowerLeaseRefreshInterval, - LeaderLeaseRefreshInterval: f.options.LeaderLeaseRefreshInterval, - Logger: f.logger, - }) - - p, err := NewStrategyStore(*f.options, f.logger, participant, f.store) + f.participant.Start() + p, err := NewStrategyStore(*f.options, f.logger, f.participant, f.store) if err != nil { return nil, nil, err } p.Start() - a, err := NewAggregator(*f.options, f.logger, f.metricsFactory, participant, f.store) + a, err := NewAggregator(*f.options, f.logger, f.metricsFactory, f.participant, f.store) if err != nil { return nil, nil, err } @@ -103,3 +104,8 @@ func (f *Factory) CreateStrategyStore() (strategystore.StrategyStore, strategyst return p, a, nil } + +// Closes the factory +func (f *Factory) Close() error { + return f.participant.Close() +} diff --git a/plugin/sampling/strategystore/adaptive/processor.go b/plugin/sampling/strategystore/adaptive/processor.go index 18008ee7972..b94ae4b447c 100644 --- a/plugin/sampling/strategystore/adaptive/processor.go +++ b/plugin/sampling/strategystore/adaptive/processor.go @@ -154,24 +154,12 @@ func (p *StrategyStore) GetSamplingStrategy(_ context.Context, service string) ( // Start initializes and starts the sampling processor which regularly calculates sampling probabilities. func (p *Processor) Start() error { p.logger.Info("starting adaptive sampling processor") - if err := p.electionParticipant.Start(); err != nil { - return err - } - // NB: the first tick will be slightly delayed by the initializeThroughput call. p.lastCheckedTime = time.Now().Add(p.Delay * -1) p.initializeThroughput(p.lastCheckedTime) return nil } -// Close stops the Processor from calculating probabilities. -func (p *Processor) Close() error { - p.logger.Info("stopping adaptive sampling processor") - err := p.electionParticipant.Close() - p.bgFinished.Wait() - return err -} - func (p *StrategyStore) loadProbabilities() { // TODO GetLatestProbabilities API can be changed to return the latest measured qps for initialization probabilities, err := p.storage.GetLatestProbabilities() diff --git a/plugin/sampling/strategystore/adaptive/strategy_store.go b/plugin/sampling/strategystore/adaptive/strategy_store.go index 7539d32e579..b9f9ae14993 100644 --- a/plugin/sampling/strategystore/adaptive/strategy_store.go +++ b/plugin/sampling/strategystore/adaptive/strategy_store.go @@ -68,9 +68,6 @@ func NewStrategyStore(options Options, logger *zap.Logger, participant leaderele // Start initializes and starts the sampling service which regularly loads sampling probabilities and generates strategies. func (ss *StrategyStore) Start() error { ss.logger.Info("starting adaptive sampling service") - if err := ss.electionParticipant.Start(); err != nil { - return err - } ss.loadProbabilities() ss.generateStrategyResponses() @@ -86,8 +83,7 @@ func (ss *StrategyStore) Start() error { // Close stops the service from loading probabilities and generating strategies. func (ss *StrategyStore) Close() error { ss.logger.Info("stopping adaptive sampling service") - err := ss.electionParticipant.Close() close(ss.shutdown) ss.bgFinished.Wait() - return err + return nil } diff --git a/plugin/sampling/strategystore/factory.go b/plugin/sampling/strategystore/factory.go index 9cc2693e7bf..d1506ed712d 100644 --- a/plugin/sampling/strategystore/factory.go +++ b/plugin/sampling/strategystore/factory.go @@ -15,6 +15,7 @@ package strategystore import ( + "errors" "flag" "fmt" @@ -113,3 +114,14 @@ func (f *Factory) CreateStrategyStore() (strategystore.StrategyStore, strategyst } return factory.CreateStrategyStore() } + +// Close closes all factories. +func (f *Factory) Close() error { + var errs []error + for _, factory := range f.factories { + if err := factory.Close(); err != nil { + errs = append(errs, err) + } + } + return errors.Join(errs...) +} diff --git a/plugin/sampling/strategystore/static/factory.go b/plugin/sampling/strategystore/static/factory.go index 91aa46e6d9c..16097860b6c 100644 --- a/plugin/sampling/strategystore/static/factory.go +++ b/plugin/sampling/strategystore/static/factory.go @@ -67,3 +67,8 @@ func (f *Factory) CreateStrategyStore() (strategystore.StrategyStore, strategyst return s, nil, nil } + +// Close closes the factory. +func (f *Factory) Close() error { + return nil +} From df0bceae00a95049888a81ccf981c4761a480637 Mon Sep 17 00:00:00 2001 From: pushkarm029 Date: Tue, 21 May 2024 20:55:03 +0530 Subject: [PATCH 15/18] fix Signed-off-by: pushkarm029 --- .../app/sampling/strategystore/factory.go | 1 + .../strategystore/adaptive/factory.go | 17 ++-- .../strategystore/adaptive/factory_test.go | 2 + .../strategystore/adaptive/processor_test.go | 82 ++++++------------- plugin/sampling/strategystore/factory_test.go | 9 ++ 5 files changed, 44 insertions(+), 67 deletions(-) diff --git a/cmd/collector/app/sampling/strategystore/factory.go b/cmd/collector/app/sampling/strategystore/factory.go index eb0371e9882..4a4059c4f5f 100644 --- a/cmd/collector/app/sampling/strategystore/factory.go +++ b/cmd/collector/app/sampling/strategystore/factory.go @@ -34,5 +34,6 @@ type Factory interface { // CreateStrategyStore initializes the StrategyStore and returns it. CreateStrategyStore() (StrategyStore, Aggregator, error) + // Close closes the factory Close() error } diff --git a/plugin/sampling/strategystore/adaptive/factory.go b/plugin/sampling/strategystore/adaptive/factory.go index fab9d207911..47f8751ed83 100644 --- a/plugin/sampling/strategystore/adaptive/factory.go +++ b/plugin/sampling/strategystore/adaptive/factory.go @@ -68,15 +68,9 @@ func (f *Factory) Initialize(metricsFactory metrics.Factory, ssFactory storage.S if ssFactory == nil { return errors.New("sampling store factory is nil. Please configure a backend that supports adaptive sampling") } - var err error f.logger = logger f.metricsFactory = metricsFactory - f.participant = leaderelection.NewElectionParticipant(f.lock, defaultResourceName, leaderelection.ElectionParticipantOptions{ - FollowerLeaseRefreshInterval: f.options.FollowerLeaseRefreshInterval, - LeaderLeaseRefreshInterval: f.options.LeaderLeaseRefreshInterval, - Logger: f.logger, - }) f.lock, err = ssFactory.CreateLock() if err != nil { return err @@ -85,21 +79,28 @@ func (f *Factory) Initialize(metricsFactory metrics.Factory, ssFactory storage.S if err != nil { return err } + f.participant = leaderelection.NewElectionParticipant(f.lock, defaultResourceName, leaderelection.ElectionParticipantOptions{ + FollowerLeaseRefreshInterval: f.options.FollowerLeaseRefreshInterval, + LeaderLeaseRefreshInterval: f.options.LeaderLeaseRefreshInterval, + Logger: f.logger, + }) + f.participant.Start() + return nil } // CreateStrategyStore implements strategystore.Factory func (f *Factory) CreateStrategyStore() (strategystore.StrategyStore, strategystore.Aggregator, error) { - f.participant.Start() p, err := NewStrategyStore(*f.options, f.logger, f.participant, f.store) if err != nil { return nil, nil, err } - p.Start() a, err := NewAggregator(*f.options, f.logger, f.metricsFactory, f.participant, f.store) if err != nil { return nil, nil, err } + + p.Start() a.Start() return p, a, nil diff --git a/plugin/sampling/strategystore/adaptive/factory_test.go b/plugin/sampling/strategystore/adaptive/factory_test.go index 99c02b8c1f2..39157f1f39c 100644 --- a/plugin/sampling/strategystore/adaptive/factory_test.go +++ b/plugin/sampling/strategystore/adaptive/factory_test.go @@ -76,6 +76,7 @@ func TestFactory(t *testing.T) { require.NoError(t, err) require.NoError(t, store.Close()) require.NoError(t, aggregator.Close()) + require.NoError(t, f.Close()) } func TestBadConfigFail(t *testing.T) { @@ -97,6 +98,7 @@ func TestBadConfigFail(t *testing.T) { require.NoError(t, f.Initialize(metrics.NullFactory, &mockSamplingStoreFactory{}, zap.NewNop())) _, _, err := f.CreateStrategyStore() require.Error(t, err) + require.NoError(t, f.Close()) } } diff --git a/plugin/sampling/strategystore/adaptive/processor_test.go b/plugin/sampling/strategystore/adaptive/processor_test.go index 03ae6b7b7bb..7fb3f2ed36e 100644 --- a/plugin/sampling/strategystore/adaptive/processor_test.go +++ b/plugin/sampling/strategystore/adaptive/processor_test.go @@ -339,8 +339,9 @@ func TestRunCalculationLoop(t *testing.T) { mockStorage.On("GetThroughput", mock.AnythingOfType("time.Time"), mock.AnythingOfType("time.Time")). Return(testThroughputs(), nil) mockStorage.On("GetLatestProbabilities").Return(model.ServiceOperationProbabilities{}, errTestStorage()) - mockStorage.On("InsertProbabilitiesAndQPS", "host", mock.AnythingOfType("model.ServiceOperationProbabilities"), + mockStorage.On("InsertProbabilitiesAndQPS", mock.AnythingOfType("string"), mock.AnythingOfType("model.ServiceOperationProbabilities"), mock.AnythingOfType("model.ServiceOperationQPS")).Return(errTestStorage()) + mockStorage.On("InsertThroughput", mock.AnythingOfType("[]*model.Throughput")).Return(errTestStorage()) mockEP := &epmocks.ElectionParticipant{} mockEP.On("Start").Return(nil) mockEP.On("Close").Return(nil) @@ -357,22 +358,23 @@ func TestRunCalculationLoop(t *testing.T) { FollowerLeaseRefreshInterval: time.Second, BucketsForCalculation: 10, } - p, err := newProcessor(cfg, "host", mockStorage, mockEP, metrics.NullFactory, logger) + agg, err := NewAggregator(cfg, logger, metrics.NullFactory, mockEP, mockStorage) require.NoError(t, err) - p.Start() + agg.Start() + defer agg.Close() for i := 0; i < 1000; i++ { - strategy, _ := p.GetSamplingStrategy(context.Background(), "svcA") - if len(strategy.OperationSampling.PerOperationStrategies) != 0 { + agg.(*aggregator).Lock() + probabilities := agg.(*aggregator).processor.probabilities + agg.(*aggregator).Unlock() + if len(probabilities) != 0 { break } time.Sleep(time.Millisecond) } - p.Close() - strategy, err := p.GetSamplingStrategy(context.Background(), "svcA") - require.NoError(t, err) - assert.Len(t, strategy.OperationSampling.PerOperationStrategies, 2) + probabilities := agg.(*aggregator).processor.probabilities + require.Len(t, probabilities["svcA"], 2) } func TestRunCalculationLoop_GetThroughputError(t *testing.T) { @@ -380,6 +382,11 @@ func TestRunCalculationLoop_GetThroughputError(t *testing.T) { mockStorage := &smocks.Store{} mockStorage.On("GetThroughput", mock.AnythingOfType("time.Time"), mock.AnythingOfType("time.Time")). Return(nil, errTestStorage()) + mockStorage.On("GetLatestProbabilities").Return(model.ServiceOperationProbabilities{}, errTestStorage()) + mockStorage.On("InsertProbabilitiesAndQPS", mock.AnythingOfType("string"), mock.AnythingOfType("model.ServiceOperationProbabilities"), + mock.AnythingOfType("model.ServiceOperationQPS")).Return(errTestStorage()) + mockStorage.On("InsertThroughput", mock.AnythingOfType("[]*model.Throughput")).Return(errTestStorage()) + mockEP := &epmocks.ElectionParticipant{} mockEP.On("Start").Return(nil) mockEP.On("Close").Return(nil) @@ -390,12 +397,9 @@ func TestRunCalculationLoop_GetThroughputError(t *testing.T) { AggregationBuckets: 2, BucketsForCalculation: 10, } - p, err := newProcessor(cfg, "host", mockStorage, mockEP, metrics.NullFactory, logger) + agg, err := NewAggregator(cfg, logger, metrics.NullFactory, mockEP, mockStorage) require.NoError(t, err) - p.shutdown = make(chan struct{}) - defer close(p.shutdown) - go p.runCalculationLoop() - + agg.Start() for i := 0; i < 1000; i++ { // match logs specific to getThroughputErrMsg. We expect to see more than 2, once during // initialization and one or more times during the loop. @@ -406,13 +410,14 @@ func TestRunCalculationLoop_GetThroughputError(t *testing.T) { } match, errMsg := testutils.LogMatcher(2, getThroughputErrMsg, logBuffer.Lines()) assert.True(t, match, errMsg) + require.NoError(t, agg.Close()) } func TestLoadProbabilities(t *testing.T) { mockStorage := &smocks.Store{} mockStorage.On("GetLatestProbabilities").Return(make(model.ServiceOperationProbabilities), nil) - p := &Processor{storage: mockStorage} + p := &StrategyStore{storage: mockStorage} require.Nil(t, p.probabilities) p.loadProbabilities() require.NotNil(t, p.probabilities) @@ -426,7 +431,7 @@ func TestRunUpdateProbabilitiesLoop(t *testing.T) { mockEP.On("Close").Return(nil) mockEP.On("IsLeader").Return(false) - p := &Processor{ + p := &StrategyStore{ storage: mockStorage, shutdown: make(chan struct{}), followerRefreshInterval: time.Millisecond, @@ -480,7 +485,7 @@ func TestRealisticRunCalculationLoop(t *testing.T) { AggregationBuckets: 1, Delay: time.Second * 10, } - p, err := newProcessor(cfg, "host", mockStorage, mockEP, metrics.NullFactory, logger) + p, err := NewStrategyStore(cfg, logger, mockEP, mockStorage) require.NoError(t, err) p.Start() @@ -557,7 +562,7 @@ func TestGenerateStrategyResponses(t *testing.T) { "GET": 0.5, }, } - p := &Processor{ + p := &StrategyStore{ probabilities: probabilities, Options: Options{ InitialSamplingProbability: 0.001, @@ -853,44 +858,3 @@ func TestCalculateProbabilitiesAndQPSMultiple(t *testing.T) { p.probabilities = probabilities p.qps = qps } - -func TestErrors(t *testing.T) { - mockStorage := &smocks.Store{} - mockStorage.On("GetLatestProbabilities").Return(model.ServiceOperationProbabilities{}, errTestStorage()) - mockStorage.On("GetThroughput", mock.AnythingOfType("time.Time"), mock.AnythingOfType("time.Time")). - Return(nil, nil) - - cfg := Options{ - TargetSamplesPerSecond: 1.0, - DeltaTolerance: 0.1, - InitialSamplingProbability: 0.001, - CalculationInterval: time.Millisecond * 5, - AggregationBuckets: 2, - Delay: time.Millisecond * 5, - LeaderLeaseRefreshInterval: time.Millisecond, - FollowerLeaseRefreshInterval: time.Second, - BucketsForCalculation: 10, - } - - // start errors - mockEP := &epmocks.ElectionParticipant{} - mockEP.On("Start").Return(errors.New("bad")) - mockEP.On("Close").Return(errors.New("also bad")) - mockEP.On("IsLeader").Return(false) - - p, err := newProcessor(cfg, "host", mockStorage, mockEP, metrics.NullFactory, zap.NewNop()) - require.NoError(t, err) - require.Error(t, p.Start()) - require.Error(t, p.Close()) - - // close errors - mockEP = &epmocks.ElectionParticipant{} - mockEP.On("Start").Return(nil) - mockEP.On("Close").Return(errors.New("still bad")) - mockEP.On("IsLeader").Return(false) - - p, err = newProcessor(cfg, "host", mockStorage, mockEP, metrics.NullFactory, zap.NewNop()) - require.NoError(t, err) - require.NoError(t, p.Start()) - require.Error(t, p.Close()) -} diff --git a/plugin/sampling/strategystore/factory_test.go b/plugin/sampling/strategystore/factory_test.go index f42a7a70861..75493a10b9e 100644 --- a/plugin/sampling/strategystore/factory_test.go +++ b/plugin/sampling/strategystore/factory_test.go @@ -79,12 +79,14 @@ func TestNewFactory(t *testing.T) { require.NoError(t, f.Initialize(metrics.NullFactory, mockSSFactory, zap.NewNop())) _, _, err = f.CreateStrategyStore() require.NoError(t, err) + require.NoError(t, f.Close()) // force the mock to return errors mock.retError = true require.EqualError(t, f.Initialize(metrics.NullFactory, mockSSFactory, zap.NewNop()), "error initializing store") _, _, err = f.CreateStrategyStore() require.EqualError(t, err, "error creating store") + require.EqualError(t, f.Close(), "error closing store") // request something that doesn't exist f.StrategyStoreType = "doesntexist" @@ -144,6 +146,13 @@ func (f *mockFactory) Initialize(metricsFactory metrics.Factory, ssFactory stora return nil } +func (f *mockFactory) Close() error { + if f.retError { + return errors.New("error closing store") + } + return nil +} + type mockSamplingStoreFactory struct{} func (m *mockSamplingStoreFactory) CreateLock() (distributedlock.Lock, error) { From 2b8c074649db92f2ca5a169620965d753a54c196 Mon Sep 17 00:00:00 2001 From: pushkarm029 Date: Tue, 21 May 2024 21:22:49 +0530 Subject: [PATCH 16/18] codecov fix Signed-off-by: pushkarm029 --- plugin/sampling/strategystore/static/factory_test.go | 1 + 1 file changed, 1 insertion(+) diff --git a/plugin/sampling/strategystore/static/factory_test.go b/plugin/sampling/strategystore/static/factory_test.go index f5d7b1880a3..11b169c4818 100644 --- a/plugin/sampling/strategystore/static/factory_test.go +++ b/plugin/sampling/strategystore/static/factory_test.go @@ -40,4 +40,5 @@ func TestFactory(t *testing.T) { require.NoError(t, f.Initialize(metrics.NullFactory, nil, zap.NewNop())) _, _, err := f.CreateStrategyStore() require.NoError(t, err) + require.NoError(t, f.Close()) } From 24f5a315bc909d8b29fe5c3d3a9aacbb367a7aae Mon Sep 17 00:00:00 2001 From: pushkarm029 Date: Tue, 21 May 2024 21:41:11 +0530 Subject: [PATCH 17/18] fix Signed-off-by: pushkarm029 --- plugin/sampling/strategystore/adaptive/processor.go | 10 ++-------- 1 file changed, 2 insertions(+), 8 deletions(-) diff --git a/plugin/sampling/strategystore/adaptive/processor.go b/plugin/sampling/strategystore/adaptive/processor.go index b94ae4b447c..6fe13d4ba88 100644 --- a/plugin/sampling/strategystore/adaptive/processor.go +++ b/plugin/sampling/strategystore/adaptive/processor.go @@ -99,8 +99,7 @@ type Processor struct { serviceCache []SamplingCache - shutdown chan struct{} - bgFinished sync.WaitGroup + shutdown chan struct{} operationsCalculatedGauge metrics.Gauge calculateProbabilitiesLatency metrics.Timer @@ -248,12 +247,7 @@ func (p *Processor) runCalculation() { // be way longer than the time to run the calculations. p.calculateProbabilitiesLatency.Record(time.Since(startTime)) - - p.bgFinished.Add(1) - go func() { - p.saveProbabilitiesAndQPS() - p.bgFinished.Done() - }() + p.saveProbabilitiesAndQPS() } } From 9c412a51dca58de7d749331c14e58bb4db246f24 Mon Sep 17 00:00:00 2001 From: pushkarm029 Date: Tue, 21 May 2024 23:39:42 +0530 Subject: [PATCH 18/18] fix Signed-off-by: pushkarm029 --- plugin/sampling/strategystore/adaptive/factory.go | 9 +++------ .../sampling/strategystore/adaptive/processor_test.go | 11 +++++------ .../sampling/strategystore/adaptive/strategy_store.go | 4 ++-- 3 files changed, 10 insertions(+), 14 deletions(-) diff --git a/plugin/sampling/strategystore/adaptive/factory.go b/plugin/sampling/strategystore/adaptive/factory.go index 47f8751ed83..ef875e2dcee 100644 --- a/plugin/sampling/strategystore/adaptive/factory.go +++ b/plugin/sampling/strategystore/adaptive/factory.go @@ -91,19 +91,16 @@ func (f *Factory) Initialize(metricsFactory metrics.Factory, ssFactory storage.S // CreateStrategyStore implements strategystore.Factory func (f *Factory) CreateStrategyStore() (strategystore.StrategyStore, strategystore.Aggregator, error) { - p, err := NewStrategyStore(*f.options, f.logger, f.participant, f.store) - if err != nil { - return nil, nil, err - } + s := NewStrategyStore(*f.options, f.logger, f.participant, f.store) a, err := NewAggregator(*f.options, f.logger, f.metricsFactory, f.participant, f.store) if err != nil { return nil, nil, err } - p.Start() + s.Start() a.Start() - return p, a, nil + return s, a, nil } // Closes the factory diff --git a/plugin/sampling/strategystore/adaptive/processor_test.go b/plugin/sampling/strategystore/adaptive/processor_test.go index 7fb3f2ed36e..1ff7b402de8 100644 --- a/plugin/sampling/strategystore/adaptive/processor_test.go +++ b/plugin/sampling/strategystore/adaptive/processor_test.go @@ -485,20 +485,19 @@ func TestRealisticRunCalculationLoop(t *testing.T) { AggregationBuckets: 1, Delay: time.Second * 10, } - p, err := NewStrategyStore(cfg, logger, mockEP, mockStorage) - require.NoError(t, err) - p.Start() + s := NewStrategyStore(cfg, logger, mockEP, mockStorage) + s.Start() for i := 0; i < 100; i++ { - strategy, _ := p.GetSamplingStrategy(context.Background(), "svcA") + strategy, _ := s.GetSamplingStrategy(context.Background(), "svcA") if len(strategy.OperationSampling.PerOperationStrategies) != 0 { break } time.Sleep(250 * time.Millisecond) } - p.Close() + s.Close() - strategy, err := p.GetSamplingStrategy(context.Background(), "svcA") + strategy, err := s.GetSamplingStrategy(context.Background(), "svcA") require.NoError(t, err) require.Len(t, strategy.OperationSampling.PerOperationStrategies, 4) strategies := strategy.OperationSampling.PerOperationStrategies diff --git a/plugin/sampling/strategystore/adaptive/strategy_store.go b/plugin/sampling/strategystore/adaptive/strategy_store.go index b9f9ae14993..6a8749c9dc8 100644 --- a/plugin/sampling/strategystore/adaptive/strategy_store.go +++ b/plugin/sampling/strategystore/adaptive/strategy_store.go @@ -52,7 +52,7 @@ type StrategyStore struct { } // NewStrategyStore creates a strategy store that holds adaptive sampling strategies. -func NewStrategyStore(options Options, logger *zap.Logger, participant leaderelection.ElectionParticipant, store samplingstore.Store) (*StrategyStore, error) { +func NewStrategyStore(options Options, logger *zap.Logger, participant leaderelection.ElectionParticipant, store samplingstore.Store) *StrategyStore { return &StrategyStore{ Options: options, storage: store, @@ -62,7 +62,7 @@ func NewStrategyStore(options Options, logger *zap.Logger, participant leaderele electionParticipant: participant, followerRefreshInterval: defaultFollowerProbabilityInterval, shutdown: make(chan struct{}), - }, nil + } } // Start initializes and starts the sampling service which regularly loads sampling probabilities and generates strategies.