diff --git a/aggregator/aggregator.go b/aggregator/aggregator.go index 9a5433e..a3ad3fa 100644 --- a/aggregator/aggregator.go +++ b/aggregator/aggregator.go @@ -24,6 +24,7 @@ import ( "context" "errors" "math" + "strconv" "sync" "sync/atomic" "time" @@ -200,7 +201,7 @@ func (agg *aggregator) AddForwarded( callEnd := agg.nowFn() agg.metrics.addForwarded.ReportSuccess(callEnd.Sub(callStart)) forwardingDelay := time.Duration(callEnd.UnixNano() - metric.TimeNanos) - agg.metrics.addForwarded.latencies.RecordDuration( + agg.metrics.addForwarded.ReportForwardingLatency( metadata.StoragePolicy.Resolution().Window, metadata.NumForwardedTimes, forwardingDelay, @@ -661,10 +662,18 @@ func (m *aggregatorAddUntimedMetrics) ReportError(err error) { m.aggregatorAddMetricMetrics.ReportError(err) } +type latencyBucketKey struct { + resolution time.Duration + numForwardedTimes int +} + type aggregatorAddForwardedMetrics struct { + sync.RWMutex aggregatorAddMetricMetrics - latencies *ForwardingLatencyHistograms + scope tally.Scope + maxAllowedForwardingDelayFn MaxAllowedForwardingDelayFn + forwardingLatency map[latencyBucketKey]tally.Histogram } func newAggregatorAddForwardedMetrics( @@ -672,27 +681,60 @@ func newAggregatorAddForwardedMetrics( samplingRate float64, maxAllowedForwardingDelayFn MaxAllowedForwardingDelayFn, ) aggregatorAddForwardedMetrics { - latencyScope := scope.Tagged(map[string]string{"latency-type": "current"}) - bucketsFn := func(key ForwardingLatencyBucketKey, numLatencyBuckets int) tally.Buckets { - maxForwardingDelayAllowed := maxAllowedForwardingDelayFn(key.Resolution, key.NumForwardedTimes) - latencyBucketSize := maxForwardingDelayAllowed * 2 / time.Duration(numLatencyBuckets) - return tally.MustMakeLinearDurationBuckets(0, latencyBucketSize, numLatencyBuckets) - } return aggregatorAddForwardedMetrics{ aggregatorAddMetricMetrics: newAggregatorAddMetricMetrics(scope, samplingRate), - latencies: NewForwardingLatencyHistograms(latencyScope, bucketsFn), + scope: scope, + maxAllowedForwardingDelayFn: maxAllowedForwardingDelayFn, + forwardingLatency: make(map[latencyBucketKey]tally.Histogram), } } -type tickMetricsForIncomingMetricType struct { +func (m *aggregatorAddForwardedMetrics) ReportForwardingLatency( + resolution time.Duration, + numForwardedTimes int, + duration time.Duration, +) { + key := latencyBucketKey{ + resolution: resolution, + numForwardedTimes: numForwardedTimes, + } + m.RLock() + histogram, exists := m.forwardingLatency[key] + m.RUnlock() + if exists { + histogram.RecordDuration(duration) + return + } + m.Lock() + histogram, exists = m.forwardingLatency[key] + if exists { + m.Unlock() + histogram.RecordDuration(duration) + return + } + maxForwardingDelayAllowed := m.maxAllowedForwardingDelayFn(resolution, numForwardedTimes) + maxLatencyBucketLimit := maxForwardingDelayAllowed * maxLatencyBucketLimitScaleFactor + latencyBucketSize := maxLatencyBucketLimit / time.Duration(numLatencyBuckets) + latencyBuckets := tally.MustMakeLinearDurationBuckets(0, latencyBucketSize, numLatencyBuckets) + histogram = m.scope.Tagged(map[string]string{ + "bucket-version": strconv.Itoa(latencyBucketVersion), + "resolution": resolution.String(), + "num-forwarded-times": strconv.Itoa(numForwardedTimes), + }).Histogram("forwarding-latency", latencyBuckets) + m.forwardingLatency[key] = histogram + m.Unlock() + histogram.RecordDuration(duration) +} + +type tickMetricsForMetricCategory struct { scope tally.Scope activeEntries tally.Gauge expiredEntries tally.Counter activeElems map[time.Duration]tally.Gauge } -func newTickMetricsForIncomingMetricType(scope tally.Scope) tickMetricsForIncomingMetricType { - return tickMetricsForIncomingMetricType{ +func newTickMetricsForMetricCategory(scope tally.Scope) tickMetricsForMetricCategory { + return tickMetricsForMetricCategory{ scope: scope, activeEntries: scope.Gauge("active-entries"), expiredEntries: scope.Counter("expired-entries"), @@ -700,7 +742,7 @@ func newTickMetricsForIncomingMetricType(scope tally.Scope) tickMetricsForIncomi } } -func (m tickMetricsForIncomingMetricType) Report(tickResult tickResultForIncomingMetricType) { +func (m tickMetricsForMetricCategory) Report(tickResult tickResultForMetricCategory) { m.activeEntries.Update(float64(tickResult.activeEntries)) m.expiredEntries.Inc(int64(tickResult.expiredEntries)) for dur, val := range tickResult.activeElems { @@ -718,8 +760,8 @@ func (m tickMetricsForIncomingMetricType) Report(tickResult tickResultForIncomin type aggregatorTickMetrics struct { flushTimesErrors tally.Counter duration tally.Timer - standard tickMetricsForIncomingMetricType - forwarded tickMetricsForIncomingMetricType + standard tickMetricsForMetricCategory + forwarded tickMetricsForMetricCategory } func newAggregatorTickMetrics(scope tally.Scope) aggregatorTickMetrics { @@ -728,8 +770,8 @@ func newAggregatorTickMetrics(scope tally.Scope) aggregatorTickMetrics { return aggregatorTickMetrics{ flushTimesErrors: scope.Counter("flush-times-errors"), duration: scope.Timer("duration"), - standard: newTickMetricsForIncomingMetricType(standardScope), - forwarded: newTickMetricsForIncomingMetricType(forwardedScope), + standard: newTickMetricsForMetricCategory(standardScope), + forwarded: newTickMetricsForMetricCategory(forwardedScope), } } @@ -850,3 +892,9 @@ const ( ) type sleepFn func(d time.Duration) + +const ( + latencyBucketVersion = 2 + numLatencyBuckets = 40 + maxLatencyBucketLimitScaleFactor = 2 +) diff --git a/aggregator/counter_elem.gen.go b/aggregator/counter_elem.gen.go index 6afb9b7..6a4da99 100644 --- a/aggregator/counter_elem.gen.go +++ b/aggregator/counter_elem.gen.go @@ -51,39 +51,9 @@ import ( type lockedCounterAggregation struct { sync.Mutex - closed bool - - // sourcesReady is only used for elements receiving forwarded metrics. - // It determines whether the current aggregation can use its source set - // to determine whether it has received data from all forwarding sources - // to perform eager forwarding if enabled. - sourcesReady bool - - // expectedSources is only used for elements receiving forwarded metrics. - // It keeps track of all the sources the current aggregation expect to receive - // data from. - expectedSources *bitset.BitSet - - // seenSources keeps track of all the sources the current aggregation has - // seen so far. - seenSources *bitset.BitSet - - // consumeState is only used for elements receiving forwarded metrics. It - // describes whether the current aggregation is ready to be consumed or has - // been consumed. This in turn determines whether the aggregation can be - // eagerly consumed, or should be skipped during consumption. - consumeState consumeState - aggregation counterAggregation -} - -func (lockedAgg *lockedCounterAggregation) close() { - if lockedAgg.closed { - return - } - lockedAgg.closed = true - lockedAgg.expectedSources = nil - lockedAgg.seenSources = nil - lockedAgg.aggregation.Close() + closed bool + sourcesSeen *bitset.BitSet + aggregation counterAggregation } type timedCounter struct { @@ -109,7 +79,6 @@ type CounterElem struct { // NewCounterElem creates a new element for the given metric type. func NewCounterElem( - incomingMetricType IncomingMetricType, id id.RawID, sp policy.StoragePolicy, aggTypes maggregation.Types, @@ -121,7 +90,7 @@ func NewCounterElem( elemBase: newElemBase(opts), values: make([]timedCounter, 0, defaultNumAggregations), // in most cases values will have two entries } - if err := e.ResetSetData(incomingMetricType, id, sp, aggTypes, pipeline, numForwardedTimes); err != nil { + if err := e.ResetSetData(id, sp, aggTypes, pipeline, numForwardedTimes); err != nil { return nil, err } return e, nil @@ -129,7 +98,6 @@ func NewCounterElem( // MustNewCounterElem creates a new element, or panics if the input is invalid. func MustNewCounterElem( - incomingMetricType IncomingMetricType, id id.RawID, sp policy.StoragePolicy, aggTypes maggregation.Types, @@ -137,7 +105,7 @@ func MustNewCounterElem( numForwardedTimes int, opts Options, ) *CounterElem { - elem, err := NewCounterElem(incomingMetricType, id, sp, aggTypes, pipeline, numForwardedTimes, opts) + elem, err := NewCounterElem(id, sp, aggTypes, pipeline, numForwardedTimes, opts) if err != nil { panic(fmt.Errorf("unable to create element: %v", err)) } @@ -146,7 +114,6 @@ func MustNewCounterElem( // ResetSetData resets the element and sets data. func (e *CounterElem) ResetSetData( - incomingMetricType IncomingMetricType, id id.RawID, sp policy.StoragePolicy, aggTypes maggregation.Types, @@ -157,7 +124,7 @@ func (e *CounterElem) ResetSetData( if useDefaultAggregation { aggTypes = e.DefaultAggregationTypes(e.aggTypesOpts) } - if err := e.elemBase.resetSetData(incomingMetricType, id, sp, aggTypes, useDefaultAggregation, pipeline, numForwardedTimes); err != nil { + if err := e.elemBase.resetSetData(id, sp, aggTypes, useDefaultAggregation, pipeline, numForwardedTimes); err != nil { return err } if err := e.counterElemBase.ResetSetData(e.aggTypesOpts, aggTypes, useDefaultAggregation); err != nil { @@ -182,7 +149,7 @@ func (e *CounterElem) ResetSetData( // AddUnion adds a metric value union at a given timestamp. func (e *CounterElem) AddUnion(timestamp time.Time, mu unaggregated.MetricUnion) error { alignedStart := timestamp.Truncate(e.sp.Resolution().Window).UnixNano() - lockedAgg, err := e.findOrCreate(alignedStart, sourcesOptions{}) + lockedAgg, err := e.findOrCreate(alignedStart, createAggregationOptions{}) if err != nil { return err } @@ -201,7 +168,7 @@ func (e *CounterElem) AddUnion(timestamp time.Time, mu unaggregated.MetricUnion) // same aggregation, the incoming value is discarded. func (e *CounterElem) AddUnique(timestamp time.Time, values []float64, sourceID uint32) error { alignedStart := timestamp.Truncate(e.sp.Resolution().Window).UnixNano() - lockedAgg, err := e.findOrCreate(alignedStart, sourcesOptions{updateSources: true, source: sourceID}) + lockedAgg, err := e.findOrCreate(alignedStart, createAggregationOptions{initSourceSet: true}) if err != nil { return err } @@ -211,26 +178,11 @@ func (e *CounterElem) AddUnique(timestamp time.Time, values []float64, sourceID return errAggregationClosed } source := uint(sourceID) - if lockedAgg.seenSources.Test(source) { + if lockedAgg.sourcesSeen.Test(source) { lockedAgg.Unlock() return errDuplicateForwardingSource } - lockedAgg.seenSources.Set(source) - if lockedAgg.sourcesReady { - // If the sources are ready, the expected sources will be a pre-filled - // bitset populated with sources the aggregation is expected to see data from. - // As such, we need to clear the source bit in the expected sources. - if lockedAgg.expectedSources.Test(source) { - // This source is never seen before and is in the expected source list, - // as a result, we need to clear the source bit. - lockedAgg.expectedSources.Clear(source) - if lockedAgg.expectedSources.None() { - lockedAgg.consumeState = readyToConsume - } - } - // New sources that are not in the expected source list are still allowed - // to go through. - } + lockedAgg.sourcesSeen.Set(source) for _, v := range values { lockedAgg.aggregation.Add(v) } @@ -245,7 +197,6 @@ func (e *CounterElem) AddUnique(timestamp time.Time, values []float64, sourceID // to avoid race conditions. func (e *CounterElem) Consume( targetNanos int64, - eagerForwardingMode eagerForwardingMode, isEarlierThanFn isEarlierThanFn, timestampNanosFn timestampNanosFn, flushLocalFn flushLocalMetricFn, @@ -261,8 +212,7 @@ func (e *CounterElem) Consume( idx := 0 for range e.values { // Bail as soon as the timestamp is no later than the target time. - timeNanos := timestampNanosFn(e.values[idx].startAtNanos, resolution) - if !isEarlierThanFn(timeNanos, targetNanos) { + if !isEarlierThanFn(e.values[idx].startAtNanos, resolution, targetNanos) { break } idx++ @@ -280,71 +230,31 @@ func (e *CounterElem) Consume( e.values = e.values[:n] } canCollect := len(e.values) == 0 && e.tombstoned - - // Additionally for elements receiving forwarded metrics and sending aggregated metrics - // to local backends, we also check if any aggregations are ready to be consumed. We however - // do not remove the aggregations as we do for aggregations whose timestamps are old enough, - // since for aggregations receiving forwarded metrics that are marked "consume ready", it is - // possible that metrics still go to the such aggregation bucket after they are marked "consume - // ready" due to delayed source re-delivery or new sources showing up, and removing such - // aggregation prematurely would mean the values from the delayed sources and/or new sources - // would be considered as the aggregated value for such aggregation bucket, which is incorrect. - // By keeping such aggregation buckets and only removing them when they are considered old enough - // (i.e., when their timestmaps are earlier than the target timestamp), we ensure no metrics may - // go to such aggregation buckets after they are consumed and therefore avoid the aformentioned - // problem. - aggregationIdxToCloseUntil := len(e.toConsume) - if e.incomingMetricType == ForwardedIncomingMetric && e.isSourcesSetReadyWithElemLock() { - e.maybeRefreshSourcesSetWithElemLock() - // We only attempt to consume if the outgoing metrics type is local instead of forwarded - // and eager forwarding is allowed (eager forwarding may be enabled and disallowed when - // the node is a follower and allowed when the node is a leader. This is because forwarded - // metrics are sent in batches and can only be sent when all sources in the same shard have - // been consumed, and as such is not well suited for pre-emptive consumption. - if e.outgoingMetricType() == localOutgoingMetric && eagerForwardingMode == allowEagerForwarding { - for i := 0; i < len(e.values); i++ { - // NB: This makes the logic easier to understand but it would be more efficient to use - // an atomic here to avoid locking aggregations. - e.values[i].lockedAgg.Lock() - if e.values[i].lockedAgg.consumeState == readyToConsume { - e.toConsume = append(e.toConsume, e.values[i]) - e.values[i].lockedAgg.consumeState = consuming - } - e.values[i].lockedAgg.Unlock() - } - } - } e.Unlock() // Process the aggregations that are ready for consumption. for i := range e.toConsume { timeNanos := timestampNanosFn(e.toConsume[i].startAtNanos, resolution) e.toConsume[i].lockedAgg.Lock() - if e.toConsume[i].lockedAgg.consumeState != consumed { - e.processValueWithAggregationLock( - timeNanos, eagerForwardingMode, e.toConsume[i].lockedAgg, - flushLocalFn, flushForwardedFn, - ) - } - e.toConsume[i].lockedAgg.consumeState = consumed - if i < aggregationIdxToCloseUntil { - if e.toConsume[i].lockedAgg.seenSources != nil { - e.sourcesLock.Lock() - // This is to make sure there aren't too many cached source sets taking up - // too much space. - if len(e.cachedSourceSets) < e.opts.MaxNumCachedSourceSets() { - e.cachedSourceSets = append(e.cachedSourceSets, e.toConsume[i].lockedAgg.seenSources) - e.toConsume[i].lockedAgg.seenSources = nil - } - e.sourcesLock.Unlock() + e.processValueWithAggregationLock(timeNanos, e.toConsume[i].lockedAgg, flushLocalFn, flushForwardedFn) + // Closes the aggregation object after it's processed. + e.toConsume[i].lockedAgg.closed = true + e.toConsume[i].lockedAgg.aggregation.Close() + if e.toConsume[i].lockedAgg.sourcesSeen != nil { + e.cachedSourceSetsLock.Lock() + // This is to make sure there aren't too many cached source sets taking up + // too much space. + if len(e.cachedSourceSets) < e.opts.MaxNumCachedSourceSets() { + e.cachedSourceSets = append(e.cachedSourceSets, e.toConsume[i].lockedAgg.sourcesSeen) } - e.toConsume[i].lockedAgg.close() + e.cachedSourceSetsLock.Unlock() + e.toConsume[i].lockedAgg.sourcesSeen = nil } e.toConsume[i].lockedAgg.Unlock() e.toConsume[i].Reset() } - if e.outgoingMetricType() == forwardedOutgoingMetric { + if e.parsedPipeline.HasRollup { forwardedAggregationKey, _ := e.ForwardedAggregationKey() onForwardedFlushedFn(e.onForwardedAggregationWrittenFn, forwardedAggregationKey) } @@ -364,15 +274,14 @@ func (e *CounterElem) Close() { e.parsedPipeline = parsedPipeline{} e.writeForwardedMetricFn = nil e.onForwardedAggregationWrittenFn = nil - e.sourcesHeartbeat = nil - e.sourcesSet = nil for idx := range e.cachedSourceSets { e.cachedSourceSets[idx] = nil } e.cachedSourceSets = nil for idx := range e.values { // Close the underlying aggregation objects. - e.values[idx].lockedAgg.close() + e.values[idx].lockedAgg.sourcesSeen = nil + e.values[idx].lockedAgg.aggregation.Close() e.values[idx].Reset() } e.values = e.values[:0] @@ -393,16 +302,13 @@ func (e *CounterElem) Close() { // if it doesn't exist. func (e *CounterElem) findOrCreate( alignedStart int64, - sourcesOpts sourcesOptions, + createOpts createAggregationOptions, ) (*lockedCounterAggregation, error) { e.RLock() if e.closed { e.RUnlock() return nil, errElemClosed } - if sourcesOpts.updateSources { - e.updateSources(sourcesOpts.source) - } idx, found := e.indexOfWithLock(alignedStart) if found { agg := e.values[idx].lockedAgg @@ -428,36 +334,24 @@ func (e *CounterElem) findOrCreate( e.values = append(e.values, timedCounter{}) copy(e.values[idx+1:numValues+1], e.values[idx:numValues]) - var ( - sourcesReady = e.isSourcesSetReadyWithElemLock() - expectedSources *bitset.BitSet - seenSources *bitset.BitSet - ) - if sourcesOpts.updateSources { - e.sourcesLock.Lock() - // If the sources set is ready, we clone it ane use the clone to - // determine when we have received from all the expected sources. - if sourcesReady { - expectedSources = e.sourcesSet.Clone() - } + var sourcesSeen *bitset.BitSet + if createOpts.initSourceSet { + e.cachedSourceSetsLock.Lock() if numCachedSourceSets := len(e.cachedSourceSets); numCachedSourceSets > 0 { - seenSources = e.cachedSourceSets[numCachedSourceSets-1] + sourcesSeen = e.cachedSourceSets[numCachedSourceSets-1] e.cachedSourceSets[numCachedSourceSets-1] = nil e.cachedSourceSets = e.cachedSourceSets[:numCachedSourceSets-1] - seenSources.ClearAll() + sourcesSeen.ClearAll() } else { - seenSources = bitset.New(defaultNumSources) + sourcesSeen = bitset.New(defaultNumSources) } - e.sourcesLock.Unlock() + e.cachedSourceSetsLock.Unlock() } - e.values[idx] = timedCounter{ startAtNanos: alignedStart, lockedAgg: &lockedCounterAggregation{ - sourcesReady: sourcesReady, - expectedSources: expectedSources, - seenSources: seenSources, - aggregation: e.NewAggregation(e.opts, e.aggOpts), + sourcesSeen: sourcesSeen, + aggregation: e.NewAggregation(e.opts, e.aggOpts), }, } agg := e.values[idx].lockedAgg @@ -496,14 +390,10 @@ func (e *CounterElem) indexOfWithLock(alignedStart int64) (int, bool) { func (e *CounterElem) processValueWithAggregationLock( timeNanos int64, - eagerForwardingMode eagerForwardingMode, lockedAgg *lockedCounterAggregation, flushLocalFn flushLocalMetricFn, flushForwardedFn flushForwardedMetricFn, ) { - if lockedAgg.aggregation.Count() == 0 { - return - } var ( fullPrefix = e.FullPrefix(e.opts) transformations = e.parsedPipeline.Transformations @@ -533,7 +423,7 @@ func (e *CounterElem) processValueWithAggregationLock( if discardNaNValues && math.IsNaN(value) { continue } - if e.outgoingMetricType() == localOutgoingMetric { + if !e.parsedPipeline.HasRollup { flushLocalFn(fullPrefix, e.id, e.TypeStringFor(e.aggTypesOpts, aggType), timeNanos, value, e.sp) } else { forwardedAggregationKey, _ := e.ForwardedAggregationKey() @@ -541,73 +431,4 @@ func (e *CounterElem) processValueWithAggregationLock( } } e.lastConsumedAtNanos = timeNanos - - // Emit latency metrics for forwarded metrics going to local backends - // when eager forwarding is allowed. - if eagerForwardingMode == allowEagerForwarding && - e.incomingMetricType == ForwardedIncomingMetric && - e.outgoingMetricType() == localOutgoingMetric { - e.opts.FullForwardingLatencyHistograms().RecordDuration( - e.sp.Resolution().Window, - e.numForwardedTimes, - time.Duration(e.nowFn().UnixNano()-timeNanos), - ) - } -} - -func (e *CounterElem) outgoingMetricType() outgoingMetricType { - if !e.parsedPipeline.HasRollup { - return localOutgoingMetric - } - return forwardedOutgoingMetric -} - -func (e *CounterElem) isSourcesSetReadyWithElemLock() bool { - if !e.opts.EnableEagerForwarding() { - return false - } - if e.buildingSourcesAtNanos == 0 { - return false - } - // NB: Allow TTL for the source set to build up. - return e.nowFn().UnixNano() >= e.buildingSourcesAtNanos+e.sourcesTTLNanos -} - -func (e *CounterElem) maybeRefreshSourcesSetWithElemLock() { - if !e.opts.EnableEagerForwarding() { - return - } - nowNanos := e.nowFn().UnixNano() - if nowNanos-e.lastSourcesRefreshNanos < e.sourcesTTLNanos { - return - } - e.sourcesLock.Lock() - for sourceID, lastHeartbeatNanos := range e.sourcesHeartbeat { - if nowNanos-lastHeartbeatNanos >= e.sourcesTTLNanos { - delete(e.sourcesHeartbeat, sourceID) - e.sourcesSet.Clear(uint(sourceID)) - } - } - e.lastSourcesRefreshNanos = nowNanos - e.sourcesLock.Unlock() -} - -func (e *CounterElem) updateSources(source uint32) { - if !e.opts.EnableEagerForwarding() { - return - } - nowNanos := e.nowFn().UnixNano() - e.sourcesLock.Lock() - // First time a source is received. - if e.sourcesHeartbeat == nil { - e.sourcesHeartbeat = make(map[uint32]int64, defaultNumSources) - e.sourcesSet = bitset.New(defaultNumSources) - e.buildingSourcesAtNanos = nowNanos - e.lastSourcesRefreshNanos = nowNanos - } - if v, exists := e.sourcesHeartbeat[source]; !exists || v < nowNanos { - e.sourcesHeartbeat[source] = nowNanos - } - e.sourcesSet.Set(uint(source)) - e.sourcesLock.Unlock() } diff --git a/aggregator/election_mgr.go b/aggregator/election_mgr.go index 794f9ff..257aea5 100644 --- a/aggregator/election_mgr.go +++ b/aggregator/election_mgr.go @@ -125,7 +125,6 @@ func (state ElectionState) String() string { case LeaderState: return "leader" default: - // nolint: goconst return "unknown" } } diff --git a/aggregator/elem_base.go b/aggregator/elem_base.go index fabaccb..ba1231a 100644 --- a/aggregator/elem_base.go +++ b/aggregator/elem_base.go @@ -35,7 +35,6 @@ import ( mpipeline "github.com/m3db/m3metrics/pipeline" "github.com/m3db/m3metrics/pipeline/applied" "github.com/m3db/m3metrics/policy" - "github.com/m3db/m3x/clock" "github.com/m3db/m3x/pool" "github.com/willf/bitset" @@ -64,29 +63,17 @@ var ( // isEarlierThanFn determines whether the timestamps of the metrics in a given // aggregation window are earlier than the given target time. -type isEarlierThanFn func(sourceNanos int64, targetNanos int64) bool +type isEarlierThanFn func(windowStartNanos int64, resolution time.Duration, targetNanos int64) bool // timestampNanosFn determines the final timestamps of metrics in a given aggregation // window with a given resolution. type timestampNanosFn func(windowStartNanos int64, resolution time.Duration) int64 -type sourcesOptions struct { - // updateSource determines whether to update the source set. - updateSources bool - // source is the source of the metric. - source uint32 +type createAggregationOptions struct { + // initSourceSet determines whether to initialize the source set. + initSourceSet bool } -type consumeState int - -const ( - // nolint: megacheck - notReadyToConsume consumeState = iota - readyToConsume - consuming - consumed -) - // metricElem is the common interface for metric elements. type metricElem interface { // Type returns the metric type. @@ -103,7 +90,6 @@ type metricElem interface { // ResetSetData resets the element and sets data. ResetSetData( - incomingMetricType IncomingMetricType, id id.RawID, sp policy.StoragePolicy, aggTypes maggregation.Types, @@ -130,7 +116,6 @@ type metricElem interface { // the element can be collected after the consumption is completed. Consume( targetNanos int64, - eagerForwardingMode eagerForwardingMode, isEarlierThanFn isEarlierThanFn, timestampNanosFn timestampNanosFn, flushLocalFn flushLocalMetricFn, @@ -151,9 +136,7 @@ type elemBase struct { // Immutable states. opts Options - nowFn clock.NowFn aggTypesOpts maggregation.TypesOptions - incomingMetricType IncomingMetricType id id.RawID sp policy.StoragePolicy useDefaultAggregation bool @@ -165,21 +148,15 @@ type elemBase struct { onForwardedAggregationWrittenFn onForwardedAggregationDoneFn // Mutable states. - tombstoned bool - closed bool - sourcesLock sync.Mutex // nolint: structcheck - cachedSourceSets []*bitset.BitSet // nolint: structcheck - sourcesHeartbeat map[uint32]int64 - sourcesSet *bitset.BitSet - sourcesTTLNanos int64 - buildingSourcesAtNanos int64 - lastSourcesRefreshNanos int64 + tombstoned bool + closed bool + cachedSourceSetsLock sync.Mutex // nolint: structcheck + cachedSourceSets []*bitset.BitSet // nolint: structcheck } func newElemBase(opts Options) elemBase { return elemBase{ opts: opts, - nowFn: opts.ClockOptions().NowFn(), aggTypesOpts: opts.AggregationTypesOptions(), aggOpts: raggregation.NewOptions(), } @@ -187,7 +164,6 @@ func newElemBase(opts Options) elemBase { // resetSetData resets the element base and sets data. func (e *elemBase) resetSetData( - incomingMetricType IncomingMetricType, id id.RawID, sp policy.StoragePolicy, aggTypes maggregation.Types, @@ -199,7 +175,6 @@ func (e *elemBase) resetSetData( if err != nil { return err } - e.incomingMetricType = incomingMetricType e.id = id e.sp = sp e.aggTypes = aggTypes @@ -209,11 +184,6 @@ func (e *elemBase) resetSetData( e.numForwardedTimes = numForwardedTimes e.tombstoned = false e.closed = false - e.sourcesHeartbeat = nil - e.sourcesSet = nil - e.sourcesTTLNanos = e.opts.ForwardingSourcesTTLFn()(sp.Resolution().Window).Nanoseconds() - e.buildingSourcesAtNanos = 0 - e.lastSourcesRefreshNanos = 0 return nil } diff --git a/aggregator/elem_base_test.go b/aggregator/elem_base_test.go index cf5934b..242d19a 100644 --- a/aggregator/elem_base_test.go +++ b/aggregator/elem_base_test.go @@ -23,12 +23,12 @@ package aggregator import ( "strings" "testing" - "time" + + "github.com/m3db/m3metrics/metric/id" raggregation "github.com/m3db/m3aggregator/aggregation" maggregation "github.com/m3db/m3metrics/aggregation" "github.com/m3db/m3metrics/metric" - "github.com/m3db/m3metrics/metric/id" "github.com/m3db/m3metrics/metric/unaggregated" "github.com/m3db/m3metrics/pipeline" "github.com/m3db/m3metrics/pipeline/applied" @@ -38,8 +38,8 @@ import ( ) func TestElemBaseID(t *testing.T) { - e := newElemBase(NewOptions()) - e.resetSetData(StandardIncomingMetric, testCounterID, testStoragePolicy, maggregation.DefaultTypes, true, applied.DefaultPipeline, 0) + e := &elemBase{} + e.resetSetData(testCounterID, testStoragePolicy, maggregation.DefaultTypes, true, applied.DefaultPipeline, 0) require.Equal(t, testCounterID, e.ID()) } @@ -71,12 +71,8 @@ func TestElemBaseResetSetData(t *testing.T) { }, }), } - opts := NewOptions().SetForwardingSourcesTTLFn(func(resolution time.Duration) time.Duration { - return 2 * resolution - }) - e := newElemBase(opts) - e.resetSetData(ForwardedIncomingMetric, testCounterID, testStoragePolicy, testAggregationTypesExpensive, false, testPipeline, 3) - require.Equal(t, ForwardedIncomingMetric, e.incomingMetricType) + e := &elemBase{} + e.resetSetData(testCounterID, testStoragePolicy, testAggregationTypesExpensive, false, testPipeline, 3) require.Equal(t, testCounterID, e.id) require.Equal(t, testStoragePolicy, e.sp) require.Equal(t, testAggregationTypesExpensive, e.aggTypes) @@ -86,11 +82,6 @@ func TestElemBaseResetSetData(t *testing.T) { require.Equal(t, 3, e.numForwardedTimes) require.False(t, e.tombstoned) require.False(t, e.closed) - require.Nil(t, e.sourcesHeartbeat) - require.Nil(t, e.sourcesSet) - require.Equal(t, 20*time.Second.Nanoseconds(), e.sourcesTTLNanos) - require.Equal(t, int64(0), e.buildingSourcesAtNanos) - require.Equal(t, int64(0), e.lastSourcesRefreshNanos) } func TestElemBaseResetSetDataInvalidPipeline(t *testing.T) { @@ -100,8 +91,8 @@ func TestElemBaseResetSetDataInvalidPipeline(t *testing.T) { Transformation: pipeline.TransformationOp{Type: transformation.Absolute}, }, }) - e := newElemBase(NewOptions()) - err := e.resetSetData(StandardIncomingMetric, testCounterID, testStoragePolicy, testAggregationTypes, false, invalidPipeline, 0) + e := &elemBase{} + err := e.resetSetData(testCounterID, testStoragePolicy, testAggregationTypes, false, invalidPipeline, 0) require.Error(t, err) require.True(t, strings.Contains(err.Error(), "has no rollup operations")) } @@ -113,8 +104,8 @@ func TestElemBaseForwardedIDWithDefaultPipeline(t *testing.T) { } func TestElemBaseForwardedIDWithCustomPipeline(t *testing.T) { - e := newElemBase(NewOptions()) - e.resetSetData(StandardIncomingMetric, testCounterID, testStoragePolicy, testAggregationTypesExpensive, false, testPipeline, 3) + e := &elemBase{} + e.resetSetData(testCounterID, testStoragePolicy, testAggregationTypesExpensive, false, testPipeline, 3) fid, ok := e.ForwardedID() require.True(t, ok) require.Equal(t, id.RawID("foo.bar"), fid) @@ -127,8 +118,8 @@ func TestElemBaseForwardedAggregationKeyWithDefaultPipeline(t *testing.T) { } func TestElemBaseForwardedAggregationKeyWithCustomPipeline(t *testing.T) { - e := newElemBase(NewOptions()) - e.resetSetData(StandardIncomingMetric, testCounterID, testStoragePolicy, testAggregationTypesExpensive, false, testPipeline, 3) + e := &elemBase{} + e.resetSetData(testCounterID, testStoragePolicy, testAggregationTypesExpensive, false, testPipeline, 3) aggKey, ok := e.ForwardedAggregationKey() require.True(t, ok) expected := aggregationKey{ diff --git a/aggregator/elem_pool_test.go b/aggregator/elem_pool_test.go index 3ceae0d..e05be73 100644 --- a/aggregator/elem_pool_test.go +++ b/aggregator/elem_pool_test.go @@ -34,12 +34,12 @@ import ( func TestCounterElemPool(t *testing.T) { p := NewCounterElemPool(pool.NewObjectPoolOptions().SetSize(1)) p.Init(func() *CounterElem { - return MustNewCounterElem(UnknownIncomingMetric, nil, policy.EmptyStoragePolicy, aggregation.DefaultTypes, applied.DefaultPipeline, 0, NewOptions()) + return MustNewCounterElem(nil, policy.EmptyStoragePolicy, aggregation.DefaultTypes, applied.DefaultPipeline, 0, NewOptions()) }) // Retrieve an element from the pool. element := p.Get() - require.NoError(t, element.ResetSetData(StandardIncomingMetric, testCounterID, testStoragePolicy, aggregation.DefaultTypes, applied.DefaultPipeline, 0)) + require.NoError(t, element.ResetSetData(testCounterID, testStoragePolicy, aggregation.DefaultTypes, applied.DefaultPipeline, 0)) require.Equal(t, testCounterID, element.id) require.Equal(t, testStoragePolicy, element.sp) @@ -55,12 +55,12 @@ func TestCounterElemPool(t *testing.T) { func TestTimerElemPool(t *testing.T) { p := NewTimerElemPool(pool.NewObjectPoolOptions().SetSize(1)) p.Init(func() *TimerElem { - return MustNewTimerElem(UnknownIncomingMetric, nil, policy.EmptyStoragePolicy, aggregation.DefaultTypes, applied.DefaultPipeline, 0, NewOptions()) + return MustNewTimerElem(nil, policy.EmptyStoragePolicy, aggregation.DefaultTypes, applied.DefaultPipeline, 0, NewOptions()) }) // Retrieve an element from the pool. element := p.Get() - require.NoError(t, element.ResetSetData(StandardIncomingMetric, testBatchTimerID, testStoragePolicy, aggregation.DefaultTypes, applied.DefaultPipeline, 0)) + require.NoError(t, element.ResetSetData(testBatchTimerID, testStoragePolicy, aggregation.DefaultTypes, applied.DefaultPipeline, 0)) require.Equal(t, testBatchTimerID, element.id) require.Equal(t, testStoragePolicy, element.sp) @@ -76,12 +76,12 @@ func TestTimerElemPool(t *testing.T) { func TestGaugeElemPool(t *testing.T) { p := NewGaugeElemPool(pool.NewObjectPoolOptions().SetSize(1)) p.Init(func() *GaugeElem { - return MustNewGaugeElem(UnknownIncomingMetric, nil, policy.EmptyStoragePolicy, aggregation.DefaultTypes, applied.DefaultPipeline, 0, NewOptions()) + return MustNewGaugeElem(nil, policy.EmptyStoragePolicy, aggregation.DefaultTypes, applied.DefaultPipeline, 0, NewOptions()) }) // Retrieve an element from the pool. element := p.Get() - require.NoError(t, element.ResetSetData(StandardIncomingMetric, testGaugeID, testStoragePolicy, aggregation.DefaultTypes, applied.DefaultPipeline, 0)) + require.NoError(t, element.ResetSetData(testGaugeID, testStoragePolicy, aggregation.DefaultTypes, applied.DefaultPipeline, 0)) require.Equal(t, testGaugeID, element.id) require.Equal(t, testStoragePolicy, element.sp) diff --git a/aggregator/elem_test.go b/aggregator/elem_test.go index 4fd53f4..d121b33 100644 --- a/aggregator/elem_test.go +++ b/aggregator/elem_test.go @@ -35,12 +35,11 @@ import ( "github.com/m3db/m3metrics/pipeline/applied" "github.com/m3db/m3metrics/policy" "github.com/m3db/m3metrics/transformation" - "github.com/m3db/m3x/clock" "github.com/m3db/m3x/pool" xtime "github.com/m3db/m3x/time" - "github.com/willf/bitset" "github.com/stretchr/testify/require" + "github.com/willf/bitset" ) var ( @@ -105,16 +104,15 @@ var ( func TestCounterResetSetData(t *testing.T) { opts := NewOptions() - ce, err := NewCounterElem(StandardIncomingMetric, nil, policy.EmptyStoragePolicy, maggregation.DefaultTypes, applied.DefaultPipeline, 1, opts) + ce, err := NewCounterElem(nil, policy.EmptyStoragePolicy, maggregation.DefaultTypes, applied.DefaultPipeline, 1, opts) require.NoError(t, err) - require.Equal(t, StandardIncomingMetric, ce.incomingMetricType) require.Equal(t, opts.AggregationTypesOptions().DefaultCounterAggregationTypes(), ce.aggTypes) require.True(t, ce.useDefaultAggregation) require.False(t, ce.aggOpts.HasExpensiveAggregations) require.Equal(t, 1, ce.numForwardedTimes) // Reset element with a default pipeline. - err = ce.ResetSetData(StandardIncomingMetric, testCounterID, testStoragePolicy, testAggregationTypesExpensive, applied.DefaultPipeline, 2) + err = ce.ResetSetData(testCounterID, testStoragePolicy, testAggregationTypesExpensive, applied.DefaultPipeline, 2) require.NoError(t, err) require.Equal(t, testCounterID, ce.id) require.Equal(t, testStoragePolicy, ce.sp) @@ -155,7 +153,7 @@ func TestCounterResetSetData(t *testing.T) { }, }), } - err = ce.ResetSetData(StandardIncomingMetric, testCounterID, testStoragePolicy, testAggregationTypesExpensive, testPipeline, 0) + err = ce.ResetSetData(testCounterID, testStoragePolicy, testAggregationTypesExpensive, testPipeline, 0) require.NoError(t, err) require.Equal(t, expectedParsedPipeline, ce.parsedPipeline) require.Equal(t, len(testAggregationTypesExpensive), len(ce.lastConsumedValues)) @@ -166,14 +164,14 @@ func TestCounterResetSetData(t *testing.T) { func TestCounterResetSetDataInvalidAggregationType(t *testing.T) { opts := NewOptions() - ce := MustNewCounterElem(StandardIncomingMetric, nil, policy.EmptyStoragePolicy, maggregation.DefaultTypes, applied.DefaultPipeline, testNumForwardedTimes, opts) - err := ce.ResetSetData(StandardIncomingMetric, testCounterID, testStoragePolicy, maggregation.Types{maggregation.Last}, applied.DefaultPipeline, 0) + ce := MustNewCounterElem(nil, policy.EmptyStoragePolicy, maggregation.DefaultTypes, applied.DefaultPipeline, testNumForwardedTimes, opts) + err := ce.ResetSetData(testCounterID, testStoragePolicy, maggregation.Types{maggregation.Last}, applied.DefaultPipeline, 0) require.Error(t, err) } func TestCounterResetSetDataInvalidPipeline(t *testing.T) { opts := NewOptions() - ce := MustNewCounterElem(StandardIncomingMetric, nil, policy.EmptyStoragePolicy, maggregation.DefaultTypes, applied.DefaultPipeline, testNumForwardedTimes, opts) + ce := MustNewCounterElem(nil, policy.EmptyStoragePolicy, maggregation.DefaultTypes, applied.DefaultPipeline, testNumForwardedTimes, opts) invalidPipeline := applied.NewPipeline([]applied.OpUnion{ { @@ -181,12 +179,12 @@ func TestCounterResetSetDataInvalidPipeline(t *testing.T) { Transformation: pipeline.TransformationOp{Type: transformation.Absolute}, }, }) - err := ce.ResetSetData(StandardIncomingMetric, testCounterID, testStoragePolicy, maggregation.DefaultTypes, invalidPipeline, 0) + err := ce.ResetSetData(testCounterID, testStoragePolicy, maggregation.DefaultTypes, invalidPipeline, 0) require.Error(t, err) } func TestCounterElemAddUnion(t *testing.T) { - e, err := NewCounterElem(StandardIncomingMetric, testCounterID, testStoragePolicy, maggregation.DefaultTypes, applied.DefaultPipeline, testNumForwardedTimes, NewOptions()) + e, err := NewCounterElem(testCounterID, testStoragePolicy, maggregation.DefaultTypes, applied.DefaultPipeline, testNumForwardedTimes, NewOptions()) require.NoError(t, err) // Add a counter metric. @@ -222,7 +220,7 @@ func TestCounterElemAddUnion(t *testing.T) { } func TestCounterElemAddUnionWithCustomAggregation(t *testing.T) { - e, err := NewCounterElem(StandardIncomingMetric, testCounterID, testStoragePolicy, testAggregationTypesExpensive, applied.DefaultPipeline, testNumForwardedTimes, NewOptions()) + e, err := NewCounterElem(testCounterID, testStoragePolicy, testAggregationTypesExpensive, applied.DefaultPipeline, testNumForwardedTimes, NewOptions()) require.NoError(t, err) // Add a counter metric. @@ -255,9 +253,8 @@ func TestCounterElemAddUnionWithCustomAggregation(t *testing.T) { require.Equal(t, errElemClosed, e.AddUnion(testTimestamps[2], testCounter)) } -func TestCounterElemAddUniqueSourcesNotReady(t *testing.T) { - opts := NewOptions().SetEnableEagerForwarding(true) - e, err := NewCounterElem(ForwardedIncomingMetric, testCounterID, testStoragePolicy, maggregation.DefaultTypes, applied.DefaultPipeline, testNumForwardedTimes, opts) +func TestCounterElemAddUnique(t *testing.T) { + e, err := NewCounterElem(testCounterID, testStoragePolicy, maggregation.DefaultTypes, applied.DefaultPipeline, testNumForwardedTimes, NewOptions()) require.NoError(t, err) // Add a metric. @@ -268,8 +265,7 @@ func TestCounterElemAddUniqueSourcesNotReady(t *testing.T) { require.Equal(t, int64(345), e.values[0].lockedAgg.aggregation.Sum()) require.Equal(t, int64(1), e.values[0].lockedAgg.aggregation.Count()) require.Equal(t, int64(0), e.values[0].lockedAgg.aggregation.SumSq()) - require.Nil(t, e.values[0].lockedAgg.expectedSources) - require.True(t, e.values[0].lockedAgg.seenSources.Test(uint(source1))) + require.True(t, e.values[0].lockedAgg.sourcesSeen.Test(uint(source1))) // Add another metric at slightly different time but still within the // same aggregation interval with a different source. @@ -280,8 +276,7 @@ func TestCounterElemAddUniqueSourcesNotReady(t *testing.T) { require.Equal(t, int64(845), e.values[0].lockedAgg.aggregation.Sum()) require.Equal(t, int64(2), e.values[0].lockedAgg.aggregation.Count()) require.Equal(t, int64(0), e.values[0].lockedAgg.aggregation.SumSq()) - require.Nil(t, e.values[0].lockedAgg.expectedSources) - require.True(t, e.values[0].lockedAgg.seenSources.Test(uint(source2))) + require.True(t, e.values[0].lockedAgg.sourcesSeen.Test(uint(source2))) // Add the counter metric in the next aggregation interval. require.NoError(t, e.AddUnique(testTimestamps[2], []float64{278}, source1)) @@ -292,8 +287,7 @@ func TestCounterElemAddUniqueSourcesNotReady(t *testing.T) { require.Equal(t, int64(278), e.values[1].lockedAgg.aggregation.Sum()) require.Equal(t, int64(1), e.values[1].lockedAgg.aggregation.Count()) require.Equal(t, int64(0), e.values[1].lockedAgg.aggregation.SumSq()) - require.Nil(t, e.values[1].lockedAgg.expectedSources) - require.True(t, e.values[1].lockedAgg.seenSources.Test(uint(source1))) + require.True(t, e.values[1].lockedAgg.sourcesSeen.Test(uint(source1))) // Add the counter metric in the same aggregation interval with the same // source results in an error. @@ -305,92 +299,15 @@ func TestCounterElemAddUniqueSourcesNotReady(t *testing.T) { require.Equal(t, int64(278), e.values[1].lockedAgg.aggregation.Sum()) require.Equal(t, int64(1), e.values[1].lockedAgg.aggregation.Count()) require.Equal(t, int64(0), e.values[1].lockedAgg.aggregation.SumSq()) - require.Nil(t, e.values[1].lockedAgg.expectedSources) - require.True(t, e.values[1].lockedAgg.seenSources.Test(uint(source1))) + require.True(t, e.values[1].lockedAgg.sourcesSeen.Test(uint(source1))) // Adding the counter metric to a closed element results in an error. e.closed = true require.Equal(t, errElemClosed, e.AddUnique(testTimestamps[2], []float64{100}, 1376)) } -func TestCounterElemAddUniqueSourcesReady(t *testing.T) { - opts := NewOptions(). - SetEnableEagerForwarding(true). - SetForwardingSourcesTTLFn(func(time.Duration) time.Duration { return 0 }) - e, err := NewCounterElem(ForwardedIncomingMetric, testCounterID, testStoragePolicy, maggregation.DefaultTypes, applied.DefaultPipeline, testNumForwardedTimes, opts) - require.NoError(t, err) - - // Source is not ready the first time a metric is added. - source1 := uint32(1234) - require.NoError(t, e.AddUnique(testTimestamps[0], []float64{345}, source1)) - require.Equal(t, 1, len(e.values)) - require.Equal(t, testAlignedStarts[0], e.values[0].startAtNanos) - require.Equal(t, int64(345), e.values[0].lockedAgg.aggregation.Sum()) - require.Equal(t, int64(1), e.values[0].lockedAgg.aggregation.Count()) - require.Equal(t, int64(0), e.values[0].lockedAgg.aggregation.SumSq()) - require.NotNil(t, e.values[0].lockedAgg.expectedSources) - require.True(t, e.values[0].lockedAgg.expectedSources.None()) - require.True(t, e.values[0].lockedAgg.seenSources.Test(uint(source1))) - require.Equal(t, readyToConsume, e.values[0].lockedAgg.consumeState) - - // Add the metric in the next aggregation interval. - source2 := uint32(5678) - require.NoError(t, e.AddUnique(testTimestamps[2], []float64{278}, source2)) - require.Equal(t, 2, len(e.values)) - for i := 0; i < len(e.values); i++ { - require.Equal(t, testAlignedStarts[i], e.values[i].startAtNanos) - } - require.Equal(t, int64(278), e.values[1].lockedAgg.aggregation.Sum()) - require.Equal(t, int64(1), e.values[1].lockedAgg.aggregation.Count()) - require.Equal(t, int64(0), e.values[1].lockedAgg.aggregation.SumSq()) - require.True(t, e.values[1].lockedAgg.expectedSources.Test(uint(source1))) - require.False(t, e.values[1].lockedAgg.expectedSources.Test(uint(source2))) - require.False(t, e.values[1].lockedAgg.seenSources.Test(uint(source1))) - require.True(t, e.values[1].lockedAgg.seenSources.Test(uint(source2))) - require.Equal(t, notReadyToConsume, e.values[1].lockedAgg.consumeState) - - // Now add another metric within the same aggregation interval with a different source. - require.NoError(t, e.AddUnique(testTimestamps[2], []float64{500}, source1)) - require.Equal(t, 2, len(e.values)) - require.Equal(t, testAlignedStarts[1], e.values[1].startAtNanos) - require.Equal(t, int64(778), e.values[1].lockedAgg.aggregation.Sum()) - require.Equal(t, int64(2), e.values[1].lockedAgg.aggregation.Count()) - require.Equal(t, int64(0), e.values[1].lockedAgg.aggregation.SumSq()) - require.True(t, e.values[1].lockedAgg.expectedSources.None()) - require.True(t, e.values[1].lockedAgg.seenSources.Test(uint(source1))) - require.True(t, e.values[1].lockedAgg.seenSources.Test(uint(source2))) - require.Equal(t, readyToConsume, e.values[1].lockedAgg.consumeState) - - // Add the metric with the same source results in an error. - require.Equal(t, errDuplicateForwardingSource, e.AddUnique(testTimestamps[2], []float64{500}, source1)) - require.Equal(t, 2, len(e.values)) - require.Equal(t, testAlignedStarts[1], e.values[1].startAtNanos) - require.Equal(t, int64(778), e.values[1].lockedAgg.aggregation.Sum()) - require.Equal(t, int64(2), e.values[1].lockedAgg.aggregation.Count()) - require.Equal(t, int64(0), e.values[1].lockedAgg.aggregation.SumSq()) - require.True(t, e.values[1].lockedAgg.expectedSources.None()) - require.True(t, e.values[1].lockedAgg.seenSources.Test(uint(source1))) - require.True(t, e.values[1].lockedAgg.seenSources.Test(uint(source2))) - require.Equal(t, readyToConsume, e.values[1].lockedAgg.consumeState) - - // Add the metric with a new source does not result in an error and updates the source set. - source3 := uint32(1987) - require.NoError(t, e.AddUnique(testTimestamps[2], []float64{500}, source3)) - require.Equal(t, 2, len(e.values)) - require.Equal(t, testAlignedStarts[1], e.values[1].startAtNanos) - require.Equal(t, int64(1278), e.values[1].lockedAgg.aggregation.Sum()) - require.Equal(t, int64(3), e.values[1].lockedAgg.aggregation.Count()) - require.Equal(t, int64(0), e.values[1].lockedAgg.aggregation.SumSq()) - require.True(t, e.sourcesSet.Test(uint(source3))) - require.True(t, e.values[1].lockedAgg.expectedSources.None()) - require.True(t, e.values[1].lockedAgg.seenSources.Test(uint(source1))) - require.True(t, e.values[1].lockedAgg.seenSources.Test(uint(source2))) - require.True(t, e.values[1].lockedAgg.seenSources.Test(uint(source3))) - require.Equal(t, readyToConsume, e.values[1].lockedAgg.consumeState) -} - func TestCounterElemAddUniqueWithCustomAggregation(t *testing.T) { - e, err := NewCounterElem(ForwardedIncomingMetric, testCounterID, testStoragePolicy, testAggregationTypesExpensive, applied.DefaultPipeline, testNumForwardedTimes, NewOptions()) + e, err := NewCounterElem(testCounterID, testStoragePolicy, testAggregationTypesExpensive, applied.DefaultPipeline, testNumForwardedTimes, NewOptions()) require.NoError(t, err) // Add a counter metric. @@ -401,7 +318,7 @@ func TestCounterElemAddUniqueWithCustomAggregation(t *testing.T) { require.Equal(t, int64(12), e.values[0].lockedAgg.aggregation.Sum()) require.Equal(t, int64(12), e.values[0].lockedAgg.aggregation.Max()) require.Equal(t, int64(144), e.values[0].lockedAgg.aggregation.SumSq()) - require.True(t, e.values[0].lockedAgg.seenSources.Test(uint(source1))) + require.True(t, e.values[0].lockedAgg.sourcesSeen.Test(uint(source1))) // Add the counter metric at slightly different time // but still within the same aggregation interval. @@ -432,7 +349,7 @@ func TestCounterElemAddUniqueWithCustomAggregation(t *testing.T) { require.Equal(t, int64(20), e.values[1].lockedAgg.aggregation.Sum()) require.Equal(t, int64(1), e.values[1].lockedAgg.aggregation.Count()) require.Equal(t, int64(400), e.values[1].lockedAgg.aggregation.SumSq()) - require.True(t, e.values[1].lockedAgg.seenSources.Test(uint(source1))) + require.True(t, e.values[1].lockedAgg.sourcesSeen.Test(uint(source1))) // Adding the counter metric to a closed element results in an error. e.closed = true @@ -449,7 +366,7 @@ func TestCounterElemConsumeDefaultAggregationDefaultPipeline(t *testing.T) { localFn, localRes := testFlushLocalMetricFn() forwardFn, forwardRes := testFlushForwardedMetricFn() onForwardedFlushedFn, onForwardedFlushedRes := testOnForwardedFlushedFn() - require.False(t, e.Consume(0, allowEagerForwarding, isEarlierThanFn, timestampNanosFn, localFn, forwardFn, onForwardedFlushedFn)) + require.False(t, e.Consume(0, isEarlierThanFn, timestampNanosFn, localFn, forwardFn, onForwardedFlushedFn)) require.Equal(t, 0, len(*localRes)) require.Equal(t, 0, len(*forwardRes)) require.Equal(t, 0, len(*onForwardedFlushedRes)) @@ -459,7 +376,7 @@ func TestCounterElemConsumeDefaultAggregationDefaultPipeline(t *testing.T) { localFn, localRes = testFlushLocalMetricFn() forwardFn, forwardRes = testFlushForwardedMetricFn() onForwardedFlushedFn, onForwardedFlushedRes = testOnForwardedFlushedFn() - require.False(t, e.Consume(testAlignedStarts[1], allowEagerForwarding, isEarlierThanFn, timestampNanosFn, localFn, forwardFn, onForwardedFlushedFn)) + require.False(t, e.Consume(testAlignedStarts[1], isEarlierThanFn, timestampNanosFn, localFn, forwardFn, onForwardedFlushedFn)) require.Equal(t, expectedLocalMetricsForCounter(testAlignedStarts[1], testStoragePolicy, maggregation.DefaultTypes), *localRes) require.Equal(t, 0, len(*forwardRes)) require.Equal(t, 0, len(*onForwardedFlushedRes)) @@ -469,7 +386,7 @@ func TestCounterElemConsumeDefaultAggregationDefaultPipeline(t *testing.T) { localFn, localRes = testFlushLocalMetricFn() forwardFn, forwardRes = testFlushForwardedMetricFn() onForwardedFlushedFn, onForwardedFlushedRes = testOnForwardedFlushedFn() - require.False(t, e.Consume(testAlignedStarts[2], allowEagerForwarding, isEarlierThanFn, timestampNanosFn, localFn, forwardFn, onForwardedFlushedFn)) + require.False(t, e.Consume(testAlignedStarts[2], isEarlierThanFn, timestampNanosFn, localFn, forwardFn, onForwardedFlushedFn)) require.Equal(t, expectedLocalMetricsForCounter(testAlignedStarts[2], testStoragePolicy, maggregation.DefaultTypes), *localRes) require.Equal(t, 0, len(*forwardRes)) require.Equal(t, 0, len(*onForwardedFlushedRes)) @@ -480,7 +397,7 @@ func TestCounterElemConsumeDefaultAggregationDefaultPipeline(t *testing.T) { localFn, localRes = testFlushLocalMetricFn() forwardFn, forwardRes = testFlushForwardedMetricFn() onForwardedFlushedFn, onForwardedFlushedRes = testOnForwardedFlushedFn() - require.True(t, e.Consume(testAlignedStarts[2], allowEagerForwarding, isEarlierThanFn, timestampNanosFn, localFn, forwardFn, onForwardedFlushedFn)) + require.True(t, e.Consume(testAlignedStarts[2], isEarlierThanFn, timestampNanosFn, localFn, forwardFn, onForwardedFlushedFn)) require.Equal(t, 0, len(*localRes)) require.Equal(t, 0, len(*forwardRes)) require.Equal(t, 0, len(*onForwardedFlushedRes)) @@ -491,7 +408,7 @@ func TestCounterElemConsumeDefaultAggregationDefaultPipeline(t *testing.T) { localFn, localRes = testFlushLocalMetricFn() forwardFn, forwardRes = testFlushForwardedMetricFn() onForwardedFlushedFn, onForwardedFlushedRes = testOnForwardedFlushedFn() - require.False(t, e.Consume(testAlignedStarts[2], allowEagerForwarding, isEarlierThanFn, timestampNanosFn, localFn, forwardFn, onForwardedFlushedFn)) + require.False(t, e.Consume(testAlignedStarts[2], isEarlierThanFn, timestampNanosFn, localFn, forwardFn, onForwardedFlushedFn)) require.Equal(t, 0, len(*localRes)) require.Equal(t, 0, len(*forwardRes)) require.Equal(t, 0, len(*onForwardedFlushedRes)) @@ -508,7 +425,7 @@ func TestCounterElemConsumeCustomAggregationDefaultPipeline(t *testing.T) { localFn, localRes := testFlushLocalMetricFn() forwardFn, forwardRes := testFlushForwardedMetricFn() onForwardedFlushedFn, onForwardedFlushedRes := testOnForwardedFlushedFn() - require.False(t, e.Consume(0, allowEagerForwarding, isEarlierThanFn, timestampNanosFn, localFn, forwardFn, onForwardedFlushedFn)) + require.False(t, e.Consume(0, isEarlierThanFn, timestampNanosFn, localFn, forwardFn, onForwardedFlushedFn)) require.Equal(t, 0, len(*localRes)) require.Equal(t, 0, len(*forwardRes)) require.Equal(t, 0, len(*onForwardedFlushedRes)) @@ -518,7 +435,7 @@ func TestCounterElemConsumeCustomAggregationDefaultPipeline(t *testing.T) { localFn, localRes = testFlushLocalMetricFn() forwardFn, forwardRes = testFlushForwardedMetricFn() onForwardedFlushedFn, onForwardedFlushedRes = testOnForwardedFlushedFn() - require.False(t, e.Consume(testAlignedStarts[1], allowEagerForwarding, isEarlierThanFn, timestampNanosFn, localFn, forwardFn, onForwardedFlushedFn)) + require.False(t, e.Consume(testAlignedStarts[1], isEarlierThanFn, timestampNanosFn, localFn, forwardFn, onForwardedFlushedFn)) require.Equal(t, expectedLocalMetricsForCounter(testAlignedStarts[1], testStoragePolicy, testAggregationTypes), *localRes) require.Equal(t, 0, len(*forwardRes)) require.Equal(t, 0, len(*onForwardedFlushedRes)) @@ -528,7 +445,7 @@ func TestCounterElemConsumeCustomAggregationDefaultPipeline(t *testing.T) { localFn, localRes = testFlushLocalMetricFn() forwardFn, forwardRes = testFlushForwardedMetricFn() onForwardedFlushedFn, onForwardedFlushedRes = testOnForwardedFlushedFn() - require.False(t, e.Consume(testAlignedStarts[2], allowEagerForwarding, isEarlierThanFn, timestampNanosFn, localFn, forwardFn, onForwardedFlushedFn)) + require.False(t, e.Consume(testAlignedStarts[2], isEarlierThanFn, timestampNanosFn, localFn, forwardFn, onForwardedFlushedFn)) require.Equal(t, expectedLocalMetricsForCounter(testAlignedStarts[2], testStoragePolicy, testAggregationTypes), *localRes) require.Equal(t, 0, len(*forwardRes)) require.Equal(t, 0, len(*onForwardedFlushedRes)) @@ -539,7 +456,7 @@ func TestCounterElemConsumeCustomAggregationDefaultPipeline(t *testing.T) { localFn, localRes = testFlushLocalMetricFn() forwardFn, forwardRes = testFlushForwardedMetricFn() onForwardedFlushedFn, onForwardedFlushedRes = testOnForwardedFlushedFn() - require.True(t, e.Consume(testAlignedStarts[2], allowEagerForwarding, isEarlierThanFn, timestampNanosFn, localFn, forwardFn, onForwardedFlushedFn)) + require.True(t, e.Consume(testAlignedStarts[2], isEarlierThanFn, timestampNanosFn, localFn, forwardFn, onForwardedFlushedFn)) require.Equal(t, 0, len(*localRes)) require.Equal(t, 0, len(*forwardRes)) require.Equal(t, 0, len(*onForwardedFlushedRes)) @@ -550,7 +467,7 @@ func TestCounterElemConsumeCustomAggregationDefaultPipeline(t *testing.T) { localFn, localRes = testFlushLocalMetricFn() forwardFn, forwardRes = testFlushForwardedMetricFn() onForwardedFlushedFn, onForwardedFlushedRes = testOnForwardedFlushedFn() - require.False(t, e.Consume(testAlignedStarts[2], allowEagerForwarding, isEarlierThanFn, timestampNanosFn, localFn, forwardFn, onForwardedFlushedFn)) + require.False(t, e.Consume(testAlignedStarts[2], isEarlierThanFn, timestampNanosFn, localFn, forwardFn, onForwardedFlushedFn)) require.Equal(t, 0, len(*localRes)) require.Equal(t, 0, len(*forwardRes)) require.Equal(t, 0, len(*onForwardedFlushedRes)) @@ -595,7 +512,7 @@ func TestCounterElemConsumeCustomAggregationCustomPipeline(t *testing.T) { localFn, localRes := testFlushLocalMetricFn() forwardFn, forwardRes := testFlushForwardedMetricFn() onForwardedFlushedFn, onForwardedFlushedRes := testOnForwardedFlushedFn() - require.False(t, e.Consume(0, allowEagerForwarding, isEarlierThanFn, timestampNanosFn, localFn, forwardFn, onForwardedFlushedFn)) + require.False(t, e.Consume(0, isEarlierThanFn, timestampNanosFn, localFn, forwardFn, onForwardedFlushedFn)) verifyOnForwardedFlushResult(t, expectedOnFlushedRes, *onForwardedFlushedRes) require.Equal(t, 0, len(*localRes)) require.Equal(t, 0, len(*forwardRes)) @@ -612,7 +529,7 @@ func TestCounterElemConsumeCustomAggregationCustomPipeline(t *testing.T) { localFn, localRes = testFlushLocalMetricFn() forwardFn, forwardRes = testFlushForwardedMetricFn() onForwardedFlushedFn, onForwardedFlushedRes = testOnForwardedFlushedFn() - require.False(t, e.Consume(alignedstartAtNanos[1], allowEagerForwarding, isEarlierThanFn, timestampNanosFn, localFn, forwardFn, onForwardedFlushedFn)) + require.False(t, e.Consume(alignedstartAtNanos[1], isEarlierThanFn, timestampNanosFn, localFn, forwardFn, onForwardedFlushedFn)) verifyForwardedMetrics(t, expectedForwardedRes, *forwardRes) verifyOnForwardedFlushResult(t, expectedOnFlushedRes, *onForwardedFlushedRes) require.Equal(t, 0, len(*localRes)) @@ -636,7 +553,7 @@ func TestCounterElemConsumeCustomAggregationCustomPipeline(t *testing.T) { localFn, localRes = testFlushLocalMetricFn() forwardFn, forwardRes = testFlushForwardedMetricFn() onForwardedFlushedFn, onForwardedFlushedRes = testOnForwardedFlushedFn() - require.False(t, e.Consume(alignedstartAtNanos[3], allowEagerForwarding, isEarlierThanFn, timestampNanosFn, localFn, forwardFn, onForwardedFlushedFn)) + require.False(t, e.Consume(alignedstartAtNanos[3], isEarlierThanFn, timestampNanosFn, localFn, forwardFn, onForwardedFlushedFn)) verifyForwardedMetrics(t, expectedForwardedRes, *forwardRes) verifyOnForwardedFlushResult(t, expectedOnFlushedRes, *onForwardedFlushedRes) require.Equal(t, 0, len(*localRes)) @@ -649,7 +566,7 @@ func TestCounterElemConsumeCustomAggregationCustomPipeline(t *testing.T) { localFn, localRes = testFlushLocalMetricFn() forwardFn, forwardRes = testFlushForwardedMetricFn() onForwardedFlushedFn, onForwardedFlushedRes = testOnForwardedFlushedFn() - require.True(t, e.Consume(alignedstartAtNanos[3], allowEagerForwarding, isEarlierThanFn, timestampNanosFn, localFn, forwardFn, onForwardedFlushedFn)) + require.True(t, e.Consume(alignedstartAtNanos[3], isEarlierThanFn, timestampNanosFn, localFn, forwardFn, onForwardedFlushedFn)) verifyOnForwardedFlushResult(t, expectedOnFlushedRes, *onForwardedFlushedRes) require.Equal(t, 0, len(*localRes)) require.Equal(t, 0, len(*forwardRes)) @@ -660,266 +577,13 @@ func TestCounterElemConsumeCustomAggregationCustomPipeline(t *testing.T) { localFn, localRes = testFlushLocalMetricFn() forwardFn, forwardRes = testFlushForwardedMetricFn() onForwardedFlushedFn, onForwardedFlushedRes = testOnForwardedFlushedFn() - require.False(t, e.Consume(alignedstartAtNanos[3], allowEagerForwarding, isEarlierThanFn, timestampNanosFn, localFn, forwardFn, onForwardedFlushedFn)) + require.False(t, e.Consume(alignedstartAtNanos[3], isEarlierThanFn, timestampNanosFn, localFn, forwardFn, onForwardedFlushedFn)) require.Equal(t, 0, len(*localRes)) require.Equal(t, 0, len(*forwardRes)) require.Equal(t, 0, len(*onForwardedFlushedRes)) require.Equal(t, 0, len(e.values)) } -func TestCounterElemConsumeSourcesReadyAllowEagerForwarding(t *testing.T) { - alignedstartAtNanos := []int64{ - time.Unix(210, 0).UnixNano(), - time.Unix(220, 0).UnixNano(), - time.Unix(230, 0).UnixNano(), - time.Unix(240, 0).UnixNano(), - time.Unix(250, 0).UnixNano(), - } - counterVal := testCounter.CounterVal - counterVals := []int64{counterVal, counterVal, counterVal, counterVal} - isEarlierThanFn := isStandardMetricEarlierThan - timestampNanosFn := standardMetricTimestampNanos - nowFn := func() time.Time { return time.Unix(200, 0) } - clockOpts := clock.NewOptions().SetNowFn(nowFn) - opts := NewOptions(). - SetClockOptions(clockOpts). - SetDiscardNaNAggregatedValues(false). - SetEnableEagerForwarding(true). - SetForwardingSourcesTTLFn(func(time.Duration) time.Duration { return time.Minute }). - SetMaxNumCachedSourceSets(2) - e := testCounterElem(alignedstartAtNanos[:len(alignedstartAtNanos)-1], counterVals, maggregation.DefaultTypes, applied.DefaultPipeline, opts) - e.incomingMetricType = ForwardedIncomingMetric - e.buildingSourcesAtNanos = time.Unix(10, 0).UnixNano() - e.values[1].lockedAgg.consumeState = readyToConsume - e.sourcesHeartbeat = map[uint32]int64{ - 1: time.Unix(200, 0).UnixNano(), // still live - 2: time.Unix(100, 0).UnixNano(), // stale - 3: time.Unix(150, 0).UnixNano(), // still live - } - e.sourcesSet = bitset.New(10) - for k := range e.sourcesHeartbeat { - e.sourcesSet.Set(uint(k)) - } - - // Consume one value. - localFn, localRes := testFlushLocalMetricFn() - forwardFn, forwardRes := testFlushForwardedMetricFn() - onForwardedFlushedFn, onForwardedFlushedRes := testOnForwardedFlushedFn() - require.False(t, e.Consume(alignedstartAtNanos[1], allowEagerForwarding, isEarlierThanFn, timestampNanosFn, localFn, forwardFn, onForwardedFlushedFn)) - - var expectedMetrics []testLocalMetricWithMetadata - expectedMetrics = append(expectedMetrics, expectedLocalMetricsForCounter(alignedstartAtNanos[1], testStoragePolicy, maggregation.DefaultTypes)...) - expectedMetrics = append(expectedMetrics, expectedLocalMetricsForCounter(alignedstartAtNanos[2], testStoragePolicy, maggregation.DefaultTypes)...) - expectedHeartbeat := map[uint32]int64{ - 1: time.Unix(200, 0).UnixNano(), // still live - 3: time.Unix(150, 0).UnixNano(), // still live - } - expectedSourceSet := bitset.New(e.sourcesSet.Len()) - expectedSourceSet.Set(1) - expectedSourceSet.Set(3) - require.Equal(t, expectedMetrics, *localRes) - require.Equal(t, 0, len(*forwardRes)) - require.Equal(t, 0, len(*onForwardedFlushedRes)) - require.Equal(t, 3, len(e.values)) - require.Equal(t, expectedHeartbeat, e.sourcesHeartbeat) - require.True(t, expectedSourceSet.Equal(e.sourcesSet)) - require.Equal(t, nowFn().UnixNano(), e.lastSourcesRefreshNanos) - require.Equal(t, consumed, e.values[0].lockedAgg.consumeState) - for _, v := range e.values { - require.NotNil(t, v.lockedAgg) - } - require.Equal(t, 1, len(e.cachedSourceSets)) - for _, v := range e.cachedSourceSets { - require.NotNil(t, v) - } - - // Disable eager forwarding and consume another value and expect that - // the aggregations are not eagerly consumed. - e.opts = e.opts.SetEnableEagerForwarding(false) - e.values[2].lockedAgg.consumeState = readyToConsume - localFn, localRes = testFlushLocalMetricFn() - forwardFn, forwardRes = testFlushForwardedMetricFn() - onForwardedFlushedFn, onForwardedFlushedRes = testOnForwardedFlushedFn() - require.False(t, e.Consume(alignedstartAtNanos[3], allowEagerForwarding, isEarlierThanFn, timestampNanosFn, localFn, forwardFn, onForwardedFlushedFn)) - require.Equal(t, expectedLocalMetricsForCounter(alignedstartAtNanos[3], testStoragePolicy, maggregation.DefaultTypes), *localRes) - require.Equal(t, 0, len(*forwardRes)) - require.Equal(t, 0, len(*onForwardedFlushedRes)) - require.Equal(t, 1, len(e.values)) - require.Equal(t, 2, len(e.cachedSourceSets)) - for _, v := range e.cachedSourceSets { - require.NotNil(t, v) - } - - // Now enable eager forwarding and consume another value. - e.opts = e.opts.SetEnableEagerForwarding(true) - localFn, localRes = testFlushLocalMetricFn() - forwardFn, forwardRes = testFlushForwardedMetricFn() - onForwardedFlushedFn, onForwardedFlushedRes = testOnForwardedFlushedFn() - require.False(t, e.Consume(alignedstartAtNanos[2], allowEagerForwarding, isEarlierThanFn, timestampNanosFn, localFn, forwardFn, onForwardedFlushedFn)) - require.Equal(t, expectedLocalMetricsForCounter(alignedstartAtNanos[4], testStoragePolicy, maggregation.DefaultTypes), *localRes) - require.Equal(t, 0, len(*forwardRes)) - require.Equal(t, 0, len(*onForwardedFlushedRes)) - require.Equal(t, 1, len(e.values)) - require.Equal(t, 2, len(e.cachedSourceSets)) - for _, v := range e.cachedSourceSets { - require.NotNil(t, v) - } - - // Tombstone the element and discard all values. - e.tombstoned = true - localFn, localRes = testFlushLocalMetricFn() - forwardFn, forwardRes = testFlushForwardedMetricFn() - onForwardedFlushedFn, onForwardedFlushedRes = testOnForwardedFlushedFn() - require.True(t, e.Consume(alignedstartAtNanos[4], allowEagerForwarding, isEarlierThanFn, timestampNanosFn, localFn, forwardFn, onForwardedFlushedFn)) - require.Equal(t, 0, len(*localRes)) - require.Equal(t, 0, len(*forwardRes)) - require.Equal(t, 0, len(*onForwardedFlushedRes)) - require.Equal(t, 0, len(e.values)) - require.Equal(t, 2, len(e.cachedSourceSets)) - for _, v := range e.cachedSourceSets { - require.NotNil(t, v) - } -} - -func TestCounterElemConsumeSourcesReadyDisallowEagerForwarding(t *testing.T) { - alignedstartAtNanos := []int64{ - time.Unix(210, 0).UnixNano(), - time.Unix(220, 0).UnixNano(), - time.Unix(230, 0).UnixNano(), - time.Unix(240, 0).UnixNano(), - time.Unix(250, 0).UnixNano(), - } - counterVal := testCounter.CounterVal - counterVals := []int64{counterVal, counterVal, counterVal, counterVal} - isEarlierThanFn := isStandardMetricEarlierThan - timestampNanosFn := standardMetricTimestampNanos - nowFn := func() time.Time { return time.Unix(200, 0) } - clockOpts := clock.NewOptions().SetNowFn(nowFn) - opts := NewOptions(). - SetClockOptions(clockOpts). - SetDiscardNaNAggregatedValues(false). - SetEnableEagerForwarding(true). - SetForwardingSourcesTTLFn(func(time.Duration) time.Duration { return time.Minute }). - SetMaxNumCachedSourceSets(2) - e := testCounterElem(alignedstartAtNanos[:len(alignedstartAtNanos)-1], counterVals, maggregation.DefaultTypes, applied.DefaultPipeline, opts) - e.incomingMetricType = ForwardedIncomingMetric - e.buildingSourcesAtNanos = time.Unix(10, 0).UnixNano() - e.values[1].lockedAgg.consumeState = readyToConsume - e.sourcesHeartbeat = map[uint32]int64{ - 1: time.Unix(200, 0).UnixNano(), // still live - 2: time.Unix(100, 0).UnixNano(), // stale - 3: time.Unix(150, 0).UnixNano(), // still live - } - e.sourcesSet = bitset.New(10) - for k := range e.sourcesHeartbeat { - e.sourcesSet.Set(uint(k)) - } - - // Consume one value. - localFn, localRes := testFlushLocalMetricFn() - forwardFn, forwardRes := testFlushForwardedMetricFn() - onForwardedFlushedFn, onForwardedFlushedRes := testOnForwardedFlushedFn() - require.False(t, e.Consume(alignedstartAtNanos[1], disAllowEagerForwarding, isEarlierThanFn, timestampNanosFn, localFn, forwardFn, onForwardedFlushedFn)) - - var expectedMetrics []testLocalMetricWithMetadata - expectedMetrics = append(expectedMetrics, expectedLocalMetricsForCounter(alignedstartAtNanos[1], testStoragePolicy, maggregation.DefaultTypes)...) - expectedHeartbeat := map[uint32]int64{ - 1: time.Unix(200, 0).UnixNano(), // still live - 3: time.Unix(150, 0).UnixNano(), // still live - } - expectedSourceSet := bitset.New(e.sourcesSet.Len()) - expectedSourceSet.Set(1) - expectedSourceSet.Set(3) - require.Equal(t, expectedMetrics, *localRes) - require.Equal(t, 0, len(*forwardRes)) - require.Equal(t, 0, len(*onForwardedFlushedRes)) - require.Equal(t, 3, len(e.values)) - require.Equal(t, expectedHeartbeat, e.sourcesHeartbeat) - require.True(t, expectedSourceSet.Equal(e.sourcesSet)) - require.Equal(t, nowFn().UnixNano(), e.lastSourcesRefreshNanos) - require.Equal(t, readyToConsume, e.values[0].lockedAgg.consumeState) - for _, v := range e.values { - require.NotNil(t, v.lockedAgg) - } - require.Equal(t, 1, len(e.cachedSourceSets)) - for _, v := range e.cachedSourceSets { - require.NotNil(t, v) - } - - // Disable eager forwarding and consume another value and expect that - // the aggregations are not eagerly consumed. - e.opts = e.opts.SetEnableEagerForwarding(false) - e.values[2].lockedAgg.consumeState = readyToConsume - localFn, localRes = testFlushLocalMetricFn() - forwardFn, forwardRes = testFlushForwardedMetricFn() - onForwardedFlushedFn, onForwardedFlushedRes = testOnForwardedFlushedFn() - require.False(t, e.Consume(alignedstartAtNanos[3], disAllowEagerForwarding, isEarlierThanFn, timestampNanosFn, localFn, forwardFn, onForwardedFlushedFn)) - - expectedMetrics = nil - expectedMetrics = append(expectedMetrics, expectedLocalMetricsForCounter(alignedstartAtNanos[2], testStoragePolicy, maggregation.DefaultTypes)...) - expectedMetrics = append(expectedMetrics, expectedLocalMetricsForCounter(alignedstartAtNanos[3], testStoragePolicy, maggregation.DefaultTypes)...) - require.Equal(t, expectedMetrics, *localRes) - require.Equal(t, 0, len(*forwardRes)) - require.Equal(t, 0, len(*onForwardedFlushedRes)) - require.Equal(t, 1, len(e.values)) - require.Equal(t, 2, len(e.cachedSourceSets)) - for _, v := range e.cachedSourceSets { - require.NotNil(t, v) - } - - // Now enable eager forwarding and consume another value. - e.opts = e.opts.SetEnableEagerForwarding(true) - localFn, localRes = testFlushLocalMetricFn() - forwardFn, forwardRes = testFlushForwardedMetricFn() - onForwardedFlushedFn, onForwardedFlushedRes = testOnForwardedFlushedFn() - require.False(t, e.Consume(alignedstartAtNanos[2], disAllowEagerForwarding, isEarlierThanFn, timestampNanosFn, localFn, forwardFn, onForwardedFlushedFn)) - require.Nil(t, *localRes) - require.Equal(t, 0, len(*forwardRes)) - require.Equal(t, 0, len(*onForwardedFlushedRes)) - require.Equal(t, 1, len(e.values)) - require.Equal(t, 2, len(e.cachedSourceSets)) - for _, v := range e.cachedSourceSets { - require.NotNil(t, v) - } - - // Tombstone the element and discard all values. - e.tombstoned = true - localFn, localRes = testFlushLocalMetricFn() - forwardFn, forwardRes = testFlushForwardedMetricFn() - onForwardedFlushedFn, onForwardedFlushedRes = testOnForwardedFlushedFn() - require.True(t, e.Consume(alignedstartAtNanos[4], allowEagerForwarding, isEarlierThanFn, timestampNanosFn, localFn, forwardFn, onForwardedFlushedFn)) - require.Equal(t, expectedLocalMetricsForCounter(alignedstartAtNanos[4], testStoragePolicy, maggregation.DefaultTypes), *localRes) - require.Equal(t, 0, len(*forwardRes)) - require.Equal(t, 0, len(*onForwardedFlushedRes)) - require.Equal(t, 0, len(e.values)) - require.Equal(t, 2, len(e.cachedSourceSets)) - for _, v := range e.cachedSourceSets { - require.NotNil(t, v) - } -} - -func TestCounterElemOutgoingMetricType(t *testing.T) { - inputs := []struct { - pipeline applied.Pipeline - expected outgoingMetricType - }{ - { - pipeline: applied.DefaultPipeline, - expected: localOutgoingMetric, - }, - { - pipeline: testPipeline, - expected: forwardedOutgoingMetric, - }, - } - - for _, input := range inputs { - e, err := NewCounterElem(StandardIncomingMetric, testCounterID, testStoragePolicy, maggregation.DefaultTypes, input.pipeline, testNumForwardedTimes, NewOptions()) - require.NoError(t, err) - require.Equal(t, input.expected, e.outgoingMetricType()) - } -} - func TestCounterElemClose(t *testing.T) { e := testCounterElem(testAlignedStarts[:len(testAlignedStarts)-1], testCounterVals, maggregation.DefaultTypes, applied.DefaultPipeline, NewOptions()) require.False(t, e.closed) @@ -935,8 +599,6 @@ func TestCounterElemClose(t *testing.T) { require.Equal(t, parsedPipeline{}, e.parsedPipeline) require.Nil(t, e.writeForwardedMetricFn) require.Nil(t, e.onForwardedAggregationWrittenFn) - require.Nil(t, e.sourcesHeartbeat) - require.Nil(t, e.sourcesSet) require.Nil(t, e.cachedSourceSets) require.Equal(t, 0, len(e.values)) require.Equal(t, 0, len(e.toConsume)) @@ -945,7 +607,7 @@ func TestCounterElemClose(t *testing.T) { } func TestCounterFindOrCreateNoSourceSet(t *testing.T) { - e, err := NewCounterElem(StandardIncomingMetric, testCounterID, testStoragePolicy, maggregation.DefaultTypes, applied.DefaultPipeline, testNumForwardedTimes, NewOptions()) + e, err := NewCounterElem(testCounterID, testStoragePolicy, maggregation.DefaultTypes, applied.DefaultPipeline, testNumForwardedTimes, NewOptions()) require.NoError(t, err) inputs := []int64{10, 10, 20, 10, 15} @@ -957,27 +619,22 @@ func TestCounterFindOrCreateNoSourceSet(t *testing.T) { {index: 1, data: []int64{10, 15, 20}}, } for idx, input := range inputs { - res, err := e.findOrCreate(input, sourcesOptions{}) + res, err := e.findOrCreate(input, createAggregationOptions{initSourceSet: false}) require.NoError(t, err) var times []int64 for _, v := range e.values { times = append(times, v.startAtNanos) } require.Equal(t, e.values[expected[idx].index].lockedAgg, res) - require.Nil(t, e.values[expected[idx].index].lockedAgg.seenSources) - require.Nil(t, e.values[expected[idx].index].lockedAgg.expectedSources) + require.Nil(t, e.values[expected[idx].index].lockedAgg.sourcesSeen) require.Equal(t, expected[idx].data, times) } } -func TestCounterFindOrCreateWithCachedSourceSetNoEagerForwarding(t *testing.T) { - opts := NewOptions().SetEnableEagerForwarding(false) - e, err := NewCounterElem(StandardIncomingMetric, testCounterID, testStoragePolicy, maggregation.DefaultTypes, applied.DefaultPipeline, testNumForwardedTimes, opts) +func TestCounterFindOrCreateWithSourceSet(t *testing.T) { + e, err := NewCounterElem(testCounterID, testStoragePolicy, maggregation.DefaultTypes, applied.DefaultPipeline, testNumForwardedTimes, NewOptions()) require.NoError(t, err) - - // Intentionally set up cached source sets. - e.cachedSourceSets = []*bitset.BitSet{bitset.From([]uint64{1, 2})} - require.False(t, e.cachedSourceSets[0].None()) + e.cachedSourceSets = []*bitset.BitSet{bitset.New(0)} inputs := []int64{10, 20} expected := []testIndexData{ @@ -985,7 +642,7 @@ func TestCounterFindOrCreateWithCachedSourceSetNoEagerForwarding(t *testing.T) { {index: 1, data: []int64{10, 20}}, } for idx, input := range inputs { - res, err := e.findOrCreate(input, sourcesOptions{updateSources: true, source: 1234}) + res, err := e.findOrCreate(input, createAggregationOptions{initSourceSet: true}) require.NoError(t, err) var times []int64 for _, v := range e.values { @@ -993,67 +650,23 @@ func TestCounterFindOrCreateWithCachedSourceSetNoEagerForwarding(t *testing.T) { } require.Equal(t, e.values[expected[idx].index].lockedAgg, res) require.Equal(t, expected[idx].data, times) - require.Nil(t, e.values[expected[idx].index].lockedAgg.expectedSources) - require.NotNil(t, e.values[expected[idx].index].lockedAgg.seenSources) - require.True(t, e.values[expected[idx].index].lockedAgg.seenSources.None()) + require.NotNil(t, e.values[expected[idx].index].lockedAgg.sourcesSeen) } require.Equal(t, 0, len(e.cachedSourceSets)) } -func TestCounterFindOrCreateWithNoCachedSourceSetWithEagerForwarding(t *testing.T) { - forwardingSourcesTTLFn := func(time.Duration) time.Duration { return 10 * time.Second } - now := time.Unix(1234, 0) - nowFn := func() time.Time { return now } - clockOpts := clock.NewOptions().SetNowFn(nowFn) - opts := NewOptions(). - SetClockOptions(clockOpts). - SetEnableEagerForwarding(true). - SetForwardingSourcesTTLFn(forwardingSourcesTTLFn) - e, err := NewCounterElem(StandardIncomingMetric, testCounterID, testStoragePolicy, maggregation.DefaultTypes, applied.DefaultPipeline, testNumForwardedTimes, opts) - require.NoError(t, err) - require.Nil(t, e.sourcesHeartbeat) - require.Equal(t, int64(0), e.buildingSourcesAtNanos) - - // First time should not clone the source set. - sourceOpts := sourcesOptions{updateSources: true, source: 1234} - res, err := e.findOrCreate(1230*time.Second.Nanoseconds(), sourceOpts) - require.NoError(t, err) - require.Equal(t, int64(1234*time.Second.Nanoseconds()), e.buildingSourcesAtNanos) - require.Equal(t, int64(1234*time.Second.Nanoseconds()), e.lastSourcesRefreshNanos) - require.True(t, e.sourcesSet.Test(uint(1234))) - require.Nil(t, res.expectedSources) - require.NotNil(t, res.seenSources) - require.True(t, res.seenSources.None()) - - // Move time forward and create a second time should clone the source set. - now = time.Unix(1300, 0) - sourceOpts = sourcesOptions{updateSources: true, source: 5678} - res, err = e.findOrCreate(1240*time.Second.Nanoseconds(), sourceOpts) - require.NoError(t, err) - require.Equal(t, int64(1234*time.Second.Nanoseconds()), e.buildingSourcesAtNanos) - require.Equal(t, int64(1234*time.Second.Nanoseconds()), e.lastSourcesRefreshNanos) - require.True(t, e.sourcesSet.Test(uint(1234))) - require.True(t, e.sourcesSet.Test(uint(5678))) - require.NotNil(t, res.expectedSources) - require.True(t, res.expectedSources.Test(uint(1234))) - require.True(t, res.expectedSources.Test(uint(5678))) - require.NotNil(t, res.seenSources) - require.True(t, res.seenSources.None()) -} - func TestTimerResetSetData(t *testing.T) { opts := NewOptions() - te, err := NewTimerElem(StandardIncomingMetric, nil, policy.EmptyStoragePolicy, maggregation.DefaultTypes, applied.DefaultPipeline, testNumForwardedTimes, opts) + te, err := NewTimerElem(nil, policy.EmptyStoragePolicy, maggregation.DefaultTypes, applied.DefaultPipeline, testNumForwardedTimes, opts) require.NoError(t, err) require.Nil(t, te.quantilesPool) require.NotNil(t, te.quantiles) require.True(t, te.aggOpts.HasExpensiveAggregations) - require.Equal(t, StandardIncomingMetric, te.incomingMetricType) require.Equal(t, opts.AggregationTypesOptions().DefaultTimerAggregationTypes(), te.aggTypes) require.True(t, te.useDefaultAggregation) // Reset element with a default pipeline. - err = te.ResetSetData(StandardIncomingMetric, testBatchTimerID, testStoragePolicy, maggregation.Types{maggregation.Max, maggregation.P999}, applied.DefaultPipeline, 0) + err = te.ResetSetData(testBatchTimerID, testStoragePolicy, maggregation.Types{maggregation.Max, maggregation.P999}, applied.DefaultPipeline, 0) require.NoError(t, err) require.Equal(t, testBatchTimerID, te.id) require.Equal(t, testStoragePolicy, te.sp) @@ -1095,7 +708,7 @@ func TestTimerResetSetData(t *testing.T) { }, }), } - err = te.ResetSetData(StandardIncomingMetric, testBatchTimerID, testStoragePolicy, testAggregationTypesExpensive, testPipeline, 0) + err = te.ResetSetData(testBatchTimerID, testStoragePolicy, testAggregationTypesExpensive, testPipeline, 0) require.NoError(t, err) require.Equal(t, expectedParsedPipeline, te.parsedPipeline) require.Equal(t, len(testAggregationTypesExpensive), len(te.lastConsumedValues)) @@ -1106,14 +719,14 @@ func TestTimerResetSetData(t *testing.T) { func TestTimerResetSetDataInvalidAggregationType(t *testing.T) { opts := NewOptions() - te := MustNewTimerElem(StandardIncomingMetric, nil, policy.EmptyStoragePolicy, maggregation.DefaultTypes, applied.DefaultPipeline, testNumForwardedTimes, opts) - err := te.ResetSetData(StandardIncomingMetric, testBatchTimerID, testStoragePolicy, maggregation.Types{maggregation.Last}, applied.DefaultPipeline, 0) + te := MustNewTimerElem(nil, policy.EmptyStoragePolicy, maggregation.DefaultTypes, applied.DefaultPipeline, testNumForwardedTimes, opts) + err := te.ResetSetData(testBatchTimerID, testStoragePolicy, maggregation.Types{maggregation.Last}, applied.DefaultPipeline, 0) require.Error(t, err) } func TestTimerResetSetDataInvalidPipeline(t *testing.T) { opts := NewOptions() - te := MustNewTimerElem(StandardIncomingMetric, nil, policy.EmptyStoragePolicy, maggregation.DefaultTypes, applied.DefaultPipeline, testNumForwardedTimes, opts) + te := MustNewTimerElem(nil, policy.EmptyStoragePolicy, maggregation.DefaultTypes, applied.DefaultPipeline, testNumForwardedTimes, opts) invalidPipeline := applied.NewPipeline([]applied.OpUnion{ { @@ -1121,12 +734,12 @@ func TestTimerResetSetDataInvalidPipeline(t *testing.T) { Transformation: pipeline.TransformationOp{Type: transformation.Absolute}, }, }) - err := te.ResetSetData(StandardIncomingMetric, testBatchTimerID, testStoragePolicy, maggregation.DefaultTypes, invalidPipeline, 0) + err := te.ResetSetData(testBatchTimerID, testStoragePolicy, maggregation.DefaultTypes, invalidPipeline, 0) require.Error(t, err) } func TestTimerElemAddUnion(t *testing.T) { - e, err := NewTimerElem(StandardIncomingMetric, testBatchTimerID, testStoragePolicy, maggregation.DefaultTypes, applied.DefaultPipeline, testNumForwardedTimes, NewOptions()) + e, err := NewTimerElem(testBatchTimerID, testStoragePolicy, maggregation.DefaultTypes, applied.DefaultPipeline, testNumForwardedTimes, NewOptions()) require.NoError(t, err) // Add a timer metric. @@ -1170,9 +783,8 @@ func TestTimerElemAddUnion(t *testing.T) { require.Equal(t, errElemClosed, e.AddUnion(testTimestamps[2], testBatchTimer)) } -func TestTimerElemAddUniqueSourcesNotReady(t *testing.T) { - opts := NewOptions().SetEnableEagerForwarding(true) - e, err := NewTimerElem(ForwardedIncomingMetric, testBatchTimerID, testStoragePolicy, maggregation.DefaultTypes, applied.DefaultPipeline, testNumForwardedTimes, opts) +func TestTimerElemAddUnique(t *testing.T) { + e, err := NewTimerElem(testBatchTimerID, testStoragePolicy, maggregation.DefaultTypes, applied.DefaultPipeline, testNumForwardedTimes, NewOptions()) require.NoError(t, err) // Add a metric. @@ -1185,10 +797,6 @@ func TestTimerElemAddUniqueSourcesNotReady(t *testing.T) { require.Equal(t, int64(3), timer.Count()) require.InEpsilon(t, 36.6, timer.Sum(), 1e-10) require.Equal(t, 12.2, timer.Quantile(0.5)) - require.Nil(t, e.values[0].lockedAgg.expectedSources) - require.True(t, e.values[0].lockedAgg.seenSources.Test(1)) - require.True(t, e.values[0].lockedAgg.seenSources.Test(2)) - require.True(t, e.values[0].lockedAgg.seenSources.Test(3)) // Add another metric at slightly different time but still within the // same aggregation interval with a different source. @@ -1198,8 +806,6 @@ func TestTimerElemAddUniqueSourcesNotReady(t *testing.T) { timer = e.values[0].lockedAgg.aggregation require.Equal(t, int64(4), timer.Count()) require.InEpsilon(t, 51, timer.Sum(), 1e-10) - require.Nil(t, e.values[0].lockedAgg.expectedSources) - require.True(t, e.values[0].lockedAgg.seenSources.Test(uint(4))) // Add the metric in the next aggregation interval. require.NoError(t, e.AddUnique(testTimestamps[2], []float64{20.0}, 1)) @@ -1210,8 +816,6 @@ func TestTimerElemAddUniqueSourcesNotReady(t *testing.T) { require.Equal(t, 20.0, e.values[1].lockedAgg.aggregation.Sum()) require.Equal(t, int64(1), e.values[1].lockedAgg.aggregation.Count()) require.Equal(t, 20.0, e.values[1].lockedAgg.aggregation.Sum()) - require.Nil(t, e.values[1].lockedAgg.expectedSources) - require.True(t, e.values[1].lockedAgg.seenSources.Test(1)) // Add the metric in the same aggregation interval with the same // source results in an error. @@ -1223,85 +827,13 @@ func TestTimerElemAddUniqueSourcesNotReady(t *testing.T) { require.Equal(t, 20.0, e.values[1].lockedAgg.aggregation.Sum()) require.Equal(t, int64(1), e.values[1].lockedAgg.aggregation.Count()) require.InEpsilon(t, 400.0, e.values[1].lockedAgg.aggregation.SumSq(), 1e-10) - require.Nil(t, e.values[1].lockedAgg.expectedSources) - require.True(t, e.values[1].lockedAgg.seenSources.Test(1)) + require.True(t, e.values[1].lockedAgg.sourcesSeen.Test(1)) // Adding the timer metric to a closed element results in an error. e.closed = true require.Equal(t, errElemClosed, e.AddUnique(testTimestamps[2], []float64{100}, 3)) } -func TestTimerElemAddUniqueSourcesReady(t *testing.T) { - opts := NewOptions(). - SetEnableEagerForwarding(true). - SetForwardingSourcesTTLFn(func(time.Duration) time.Duration { return 0 }) - e, err := NewTimerElem(ForwardedIncomingMetric, testBatchTimerID, testStoragePolicy, maggregation.DefaultTypes, applied.DefaultPipeline, testNumForwardedTimes, opts) - require.NoError(t, err) - - // Source is not ready the first time a metric is added. - source1 := uint32(1234) - require.NoError(t, e.AddUnique(testTimestamps[0], []float64{345}, source1)) - require.Equal(t, 1, len(e.values)) - require.Equal(t, testAlignedStarts[0], e.values[0].startAtNanos) - require.Equal(t, float64(345), e.values[0].lockedAgg.aggregation.Sum()) - require.Equal(t, int64(1), e.values[0].lockedAgg.aggregation.Count()) - require.NotNil(t, e.values[0].lockedAgg.expectedSources) - require.True(t, e.values[0].lockedAgg.expectedSources.None()) - require.True(t, e.values[0].lockedAgg.seenSources.Test(uint(source1))) - require.Equal(t, readyToConsume, e.values[0].lockedAgg.consumeState) - - // Add the metric in the next aggregation interval. - source2 := uint32(5678) - require.NoError(t, e.AddUnique(testTimestamps[2], []float64{278}, source2)) - require.Equal(t, 2, len(e.values)) - for i := 0; i < len(e.values); i++ { - require.Equal(t, testAlignedStarts[i], e.values[i].startAtNanos) - } - require.Equal(t, float64(278), e.values[1].lockedAgg.aggregation.Sum()) - require.Equal(t, int64(1), e.values[1].lockedAgg.aggregation.Count()) - require.True(t, e.values[1].lockedAgg.expectedSources.Test(uint(source1))) - require.False(t, e.values[1].lockedAgg.expectedSources.Test(uint(source2))) - require.False(t, e.values[1].lockedAgg.seenSources.Test(uint(source1))) - require.True(t, e.values[1].lockedAgg.seenSources.Test(uint(source2))) - require.Equal(t, notReadyToConsume, e.values[1].lockedAgg.consumeState) - - // Now add another metric within the same aggregation interval with a different source. - require.NoError(t, e.AddUnique(testTimestamps[2], []float64{500}, source1)) - require.Equal(t, 2, len(e.values)) - require.Equal(t, testAlignedStarts[1], e.values[1].startAtNanos) - require.Equal(t, float64(778), e.values[1].lockedAgg.aggregation.Sum()) - require.Equal(t, int64(2), e.values[1].lockedAgg.aggregation.Count()) - require.True(t, e.values[1].lockedAgg.expectedSources.None()) - require.True(t, e.values[1].lockedAgg.seenSources.Test(uint(source1))) - require.True(t, e.values[1].lockedAgg.seenSources.Test(uint(source2))) - require.Equal(t, readyToConsume, e.values[1].lockedAgg.consumeState) - - // Add the metric with the same source results in an error. - require.Equal(t, errDuplicateForwardingSource, e.AddUnique(testTimestamps[2], []float64{500}, source1)) - require.Equal(t, 2, len(e.values)) - require.Equal(t, testAlignedStarts[1], e.values[1].startAtNanos) - require.Equal(t, float64(778), e.values[1].lockedAgg.aggregation.Sum()) - require.Equal(t, int64(2), e.values[1].lockedAgg.aggregation.Count()) - require.True(t, e.values[1].lockedAgg.expectedSources.None()) - require.True(t, e.values[1].lockedAgg.seenSources.Test(uint(source1))) - require.True(t, e.values[1].lockedAgg.seenSources.Test(uint(source2))) - require.Equal(t, readyToConsume, e.values[1].lockedAgg.consumeState) - - // Add the metric with a new source does not result in an error and updates the source set. - source3 := uint32(1987) - require.NoError(t, e.AddUnique(testTimestamps[2], []float64{500}, source3)) - require.Equal(t, 2, len(e.values)) - require.Equal(t, testAlignedStarts[1], e.values[1].startAtNanos) - require.Equal(t, float64(1278), e.values[1].lockedAgg.aggregation.Sum()) - require.Equal(t, int64(3), e.values[1].lockedAgg.aggregation.Count()) - require.True(t, e.sourcesSet.Test(uint(source3))) - require.True(t, e.values[1].lockedAgg.expectedSources.None()) - require.True(t, e.values[1].lockedAgg.seenSources.Test(uint(source1))) - require.True(t, e.values[1].lockedAgg.seenSources.Test(uint(source2))) - require.True(t, e.values[1].lockedAgg.seenSources.Test(uint(source3))) - require.Equal(t, readyToConsume, e.values[1].lockedAgg.consumeState) -} - func TestTimerElemConsumeDefaultAggregationDefaultPipeline(t *testing.T) { // Set up stream options. streamOpts, p, numAlloc := testStreamOptions(t, len(testAlignedStarts)-1) @@ -1317,7 +849,7 @@ func TestTimerElemConsumeDefaultAggregationDefaultPipeline(t *testing.T) { localFn, localRes := testFlushLocalMetricFn() forwardFn, forwardRes := testFlushForwardedMetricFn() onForwardedFlushedFn, onForwardedFlushedRes := testOnForwardedFlushedFn() - require.False(t, e.Consume(0, allowEagerForwarding, isEarlierThanFn, timestampNanosFn, localFn, forwardFn, onForwardedFlushedFn)) + require.False(t, e.Consume(0, isEarlierThanFn, timestampNanosFn, localFn, forwardFn, onForwardedFlushedFn)) require.Equal(t, 0, len(*localRes)) require.Equal(t, 0, len(*forwardRes)) require.Equal(t, 0, len(*onForwardedFlushedRes)) @@ -1327,7 +859,7 @@ func TestTimerElemConsumeDefaultAggregationDefaultPipeline(t *testing.T) { localFn, localRes = testFlushLocalMetricFn() forwardFn, forwardRes = testFlushForwardedMetricFn() onForwardedFlushedFn, onForwardedFlushedRes = testOnForwardedFlushedFn() - require.False(t, e.Consume(testAlignedStarts[1], allowEagerForwarding, isEarlierThanFn, timestampNanosFn, localFn, forwardFn, onForwardedFlushedFn)) + require.False(t, e.Consume(testAlignedStarts[1], isEarlierThanFn, timestampNanosFn, localFn, forwardFn, onForwardedFlushedFn)) require.Equal(t, expectedLocalMetricsForTimer(testAlignedStarts[1], testStoragePolicy, maggregation.DefaultTypes), *localRes) require.Equal(t, 0, len(*forwardRes)) require.Equal(t, 0, len(*onForwardedFlushedRes)) @@ -1337,7 +869,7 @@ func TestTimerElemConsumeDefaultAggregationDefaultPipeline(t *testing.T) { localFn, localRes = testFlushLocalMetricFn() forwardFn, forwardRes = testFlushForwardedMetricFn() onForwardedFlushedFn, onForwardedFlushedRes = testOnForwardedFlushedFn() - require.False(t, e.Consume(testAlignedStarts[2], allowEagerForwarding, isEarlierThanFn, timestampNanosFn, localFn, forwardFn, onForwardedFlushedFn)) + require.False(t, e.Consume(testAlignedStarts[2], isEarlierThanFn, timestampNanosFn, localFn, forwardFn, onForwardedFlushedFn)) require.Equal(t, expectedLocalMetricsForTimer(testAlignedStarts[2], testStoragePolicy, maggregation.DefaultTypes), *localRes) require.Equal(t, 0, len(*forwardRes)) require.Equal(t, 0, len(*onForwardedFlushedRes)) @@ -1348,7 +880,7 @@ func TestTimerElemConsumeDefaultAggregationDefaultPipeline(t *testing.T) { localFn, localRes = testFlushLocalMetricFn() forwardFn, forwardRes = testFlushForwardedMetricFn() onForwardedFlushedFn, onForwardedFlushedRes = testOnForwardedFlushedFn() - require.True(t, e.Consume(testAlignedStarts[2], allowEagerForwarding, isEarlierThanFn, timestampNanosFn, localFn, forwardFn, onForwardedFlushedFn)) + require.True(t, e.Consume(testAlignedStarts[2], isEarlierThanFn, timestampNanosFn, localFn, forwardFn, onForwardedFlushedFn)) require.Equal(t, 0, len(*localRes)) require.Equal(t, 0, len(*forwardRes)) require.Equal(t, 0, len(*onForwardedFlushedRes)) @@ -1359,7 +891,7 @@ func TestTimerElemConsumeDefaultAggregationDefaultPipeline(t *testing.T) { localFn, localRes = testFlushLocalMetricFn() forwardFn, forwardRes = testFlushForwardedMetricFn() onForwardedFlushedFn, onForwardedFlushedRes = testOnForwardedFlushedFn() - require.False(t, e.Consume(testAlignedStarts[2], allowEagerForwarding, isEarlierThanFn, timestampNanosFn, localFn, forwardFn, onForwardedFlushedFn)) + require.False(t, e.Consume(testAlignedStarts[2], isEarlierThanFn, timestampNanosFn, localFn, forwardFn, onForwardedFlushedFn)) require.Equal(t, 0, len(*localRes)) require.Equal(t, 0, len(*forwardRes)) require.Equal(t, 0, len(*onForwardedFlushedRes)) @@ -1384,7 +916,7 @@ func TestTimerElemConsumeCustomAggregationDefaultPipeline(t *testing.T) { localFn, localRes := testFlushLocalMetricFn() forwardFn, forwardRes := testFlushForwardedMetricFn() onForwardedFlushedFn, onForwardedFlushedRes := testOnForwardedFlushedFn() - require.False(t, e.Consume(0, allowEagerForwarding, isEarlierThanFn, timestampNanosFn, localFn, forwardFn, onForwardedFlushedFn)) + require.False(t, e.Consume(0, isEarlierThanFn, timestampNanosFn, localFn, forwardFn, onForwardedFlushedFn)) require.Equal(t, 0, len(*localRes)) require.Equal(t, 0, len(*forwardRes)) require.Equal(t, 0, len(*onForwardedFlushedRes)) @@ -1394,7 +926,7 @@ func TestTimerElemConsumeCustomAggregationDefaultPipeline(t *testing.T) { localFn, localRes = testFlushLocalMetricFn() forwardFn, forwardRes = testFlushForwardedMetricFn() onForwardedFlushedFn, onForwardedFlushedRes = testOnForwardedFlushedFn() - require.False(t, e.Consume(testAlignedStarts[1], allowEagerForwarding, isEarlierThanFn, timestampNanosFn, localFn, forwardFn, onForwardedFlushedFn)) + require.False(t, e.Consume(testAlignedStarts[1], isEarlierThanFn, timestampNanosFn, localFn, forwardFn, onForwardedFlushedFn)) require.Equal(t, expectedLocalMetricsForTimer(testAlignedStarts[1], testStoragePolicy, testTimerAggregationTypes), *localRes) require.Equal(t, 0, len(*forwardRes)) require.Equal(t, 0, len(*onForwardedFlushedRes)) @@ -1404,7 +936,7 @@ func TestTimerElemConsumeCustomAggregationDefaultPipeline(t *testing.T) { localFn, localRes = testFlushLocalMetricFn() forwardFn, forwardRes = testFlushForwardedMetricFn() onForwardedFlushedFn, onForwardedFlushedRes = testOnForwardedFlushedFn() - require.False(t, e.Consume(testAlignedStarts[2], allowEagerForwarding, isEarlierThanFn, timestampNanosFn, localFn, forwardFn, onForwardedFlushedFn)) + require.False(t, e.Consume(testAlignedStarts[2], isEarlierThanFn, timestampNanosFn, localFn, forwardFn, onForwardedFlushedFn)) require.Equal(t, expectedLocalMetricsForTimer(testAlignedStarts[2], testStoragePolicy, testTimerAggregationTypes), *localRes) require.Equal(t, 0, len(*forwardRes)) require.Equal(t, 0, len(*onForwardedFlushedRes)) @@ -1415,7 +947,7 @@ func TestTimerElemConsumeCustomAggregationDefaultPipeline(t *testing.T) { localFn, localRes = testFlushLocalMetricFn() forwardFn, forwardRes = testFlushForwardedMetricFn() onForwardedFlushedFn, onForwardedFlushedRes = testOnForwardedFlushedFn() - require.True(t, e.Consume(testAlignedStarts[2], allowEagerForwarding, isEarlierThanFn, timestampNanosFn, localFn, forwardFn, onForwardedFlushedFn)) + require.True(t, e.Consume(testAlignedStarts[2], isEarlierThanFn, timestampNanosFn, localFn, forwardFn, onForwardedFlushedFn)) require.Equal(t, 0, len(*localRes)) require.Equal(t, 0, len(*forwardRes)) require.Equal(t, 0, len(*onForwardedFlushedRes)) @@ -1426,7 +958,7 @@ func TestTimerElemConsumeCustomAggregationDefaultPipeline(t *testing.T) { localFn, localRes = testFlushLocalMetricFn() forwardFn, forwardRes = testFlushForwardedMetricFn() onForwardedFlushedFn, onForwardedFlushedRes = testOnForwardedFlushedFn() - require.False(t, e.Consume(testAlignedStarts[2], allowEagerForwarding, isEarlierThanFn, timestampNanosFn, localFn, forwardFn, onForwardedFlushedFn)) + require.False(t, e.Consume(testAlignedStarts[2], isEarlierThanFn, timestampNanosFn, localFn, forwardFn, onForwardedFlushedFn)) require.Equal(t, 0, len(*localRes)) require.Equal(t, 0, len(*forwardRes)) require.Equal(t, 0, len(*onForwardedFlushedRes)) @@ -1478,7 +1010,7 @@ func TestTimerElemConsumeCustomAggregationCustomPipeline(t *testing.T) { localFn, localRes := testFlushLocalMetricFn() forwardFn, forwardRes := testFlushForwardedMetricFn() onForwardedFlushedFn, onForwardedFlushedRes := testOnForwardedFlushedFn() - require.False(t, e.Consume(0, allowEagerForwarding, isEarlierThanFn, timestampNanosFn, localFn, forwardFn, onForwardedFlushedFn)) + require.False(t, e.Consume(0, isEarlierThanFn, timestampNanosFn, localFn, forwardFn, onForwardedFlushedFn)) verifyOnForwardedFlushResult(t, expectedOnFlushedRes, *onForwardedFlushedRes) require.Equal(t, 0, len(*localRes)) require.Equal(t, 0, len(*forwardRes)) @@ -1495,7 +1027,7 @@ func TestTimerElemConsumeCustomAggregationCustomPipeline(t *testing.T) { localFn, localRes = testFlushLocalMetricFn() forwardFn, forwardRes = testFlushForwardedMetricFn() onForwardedFlushedFn, onForwardedFlushedRes = testOnForwardedFlushedFn() - require.False(t, e.Consume(alignedstartAtNanos[1], allowEagerForwarding, isEarlierThanFn, timestampNanosFn, localFn, forwardFn, onForwardedFlushedFn)) + require.False(t, e.Consume(alignedstartAtNanos[1], isEarlierThanFn, timestampNanosFn, localFn, forwardFn, onForwardedFlushedFn)) verifyForwardedMetrics(t, expectedForwardedRes, *forwardRes) verifyOnForwardedFlushResult(t, expectedOnFlushedRes, *onForwardedFlushedRes) require.Equal(t, 0, len(*localRes)) @@ -1519,7 +1051,7 @@ func TestTimerElemConsumeCustomAggregationCustomPipeline(t *testing.T) { localFn, localRes = testFlushLocalMetricFn() forwardFn, forwardRes = testFlushForwardedMetricFn() onForwardedFlushedFn, onForwardedFlushedRes = testOnForwardedFlushedFn() - require.False(t, e.Consume(alignedstartAtNanos[3], allowEagerForwarding, isEarlierThanFn, timestampNanosFn, localFn, forwardFn, onForwardedFlushedFn)) + require.False(t, e.Consume(alignedstartAtNanos[3], isEarlierThanFn, timestampNanosFn, localFn, forwardFn, onForwardedFlushedFn)) verifyForwardedMetrics(t, expectedForwardedRes, *forwardRes) verifyOnForwardedFlushResult(t, expectedOnFlushedRes, *onForwardedFlushedRes) require.Equal(t, 0, len(*localRes)) @@ -1532,7 +1064,7 @@ func TestTimerElemConsumeCustomAggregationCustomPipeline(t *testing.T) { localFn, localRes = testFlushLocalMetricFn() forwardFn, forwardRes = testFlushForwardedMetricFn() onForwardedFlushedFn, onForwardedFlushedRes = testOnForwardedFlushedFn() - require.True(t, e.Consume(alignedstartAtNanos[3], allowEagerForwarding, isEarlierThanFn, timestampNanosFn, localFn, forwardFn, onForwardedFlushedFn)) + require.True(t, e.Consume(alignedstartAtNanos[3], isEarlierThanFn, timestampNanosFn, localFn, forwardFn, onForwardedFlushedFn)) verifyOnForwardedFlushResult(t, expectedOnFlushedRes, *onForwardedFlushedRes) require.Equal(t, 0, len(*localRes)) require.Equal(t, 0, len(*forwardRes)) @@ -1543,264 +1075,11 @@ func TestTimerElemConsumeCustomAggregationCustomPipeline(t *testing.T) { localFn, localRes = testFlushLocalMetricFn() forwardFn, forwardRes = testFlushForwardedMetricFn() onForwardedFlushedFn, onForwardedFlushedRes = testOnForwardedFlushedFn() - require.False(t, e.Consume(alignedstartAtNanos[3], allowEagerForwarding, isEarlierThanFn, timestampNanosFn, localFn, forwardFn, onForwardedFlushedFn)) - require.Equal(t, 0, len(*localRes)) - require.Equal(t, 0, len(*forwardRes)) - require.Equal(t, 0, len(*onForwardedFlushedRes)) - require.Equal(t, 0, len(e.values)) -} - -func TestTimerElemConsumeSourcesReadyAllowEagerForwarding(t *testing.T) { - alignedstartAtNanos := []int64{ - time.Unix(210, 0).UnixNano(), - time.Unix(220, 0).UnixNano(), - time.Unix(230, 0).UnixNano(), - time.Unix(240, 0).UnixNano(), - time.Unix(250, 0).UnixNano(), - } - timerVal := testBatchTimer.BatchTimerVal - timerVals := [][]float64{timerVal, timerVal, timerVal, timerVal} - isEarlierThanFn := isStandardMetricEarlierThan - timestampNanosFn := standardMetricTimestampNanos - nowFn := func() time.Time { return time.Unix(200, 0) } - clockOpts := clock.NewOptions().SetNowFn(nowFn) - opts := NewOptions(). - SetClockOptions(clockOpts). - SetDiscardNaNAggregatedValues(false). - SetEnableEagerForwarding(true). - SetForwardingSourcesTTLFn(func(time.Duration) time.Duration { return time.Minute }). - SetMaxNumCachedSourceSets(2) - e := testTimerElem(alignedstartAtNanos[:len(alignedstartAtNanos)-1], timerVals, maggregation.DefaultTypes, applied.DefaultPipeline, opts) - e.incomingMetricType = ForwardedIncomingMetric - e.buildingSourcesAtNanos = time.Unix(10, 0).UnixNano() - e.values[1].lockedAgg.consumeState = readyToConsume - e.sourcesHeartbeat = map[uint32]int64{ - 1: time.Unix(200, 0).UnixNano(), // still live - 2: time.Unix(100, 0).UnixNano(), // stale - 3: time.Unix(150, 0).UnixNano(), // still live - } - e.sourcesSet = bitset.New(10) - for k := range e.sourcesHeartbeat { - e.sourcesSet.Set(uint(k)) - } - - // Consume one value. - localFn, localRes := testFlushLocalMetricFn() - forwardFn, forwardRes := testFlushForwardedMetricFn() - onForwardedFlushedFn, onForwardedFlushedRes := testOnForwardedFlushedFn() - require.False(t, e.Consume(alignedstartAtNanos[1], allowEagerForwarding, isEarlierThanFn, timestampNanosFn, localFn, forwardFn, onForwardedFlushedFn)) - - var expectedMetrics []testLocalMetricWithMetadata - expectedMetrics = append(expectedMetrics, expectedLocalMetricsForTimer(alignedstartAtNanos[1], testStoragePolicy, maggregation.DefaultTypes)...) - expectedMetrics = append(expectedMetrics, expectedLocalMetricsForTimer(alignedstartAtNanos[2], testStoragePolicy, maggregation.DefaultTypes)...) - expectedHeartbeat := map[uint32]int64{ - 1: time.Unix(200, 0).UnixNano(), // still live - 3: time.Unix(150, 0).UnixNano(), // still live - } - expectedSourceSet := bitset.New(e.sourcesSet.Len()) - expectedSourceSet.Set(1) - expectedSourceSet.Set(3) - require.Equal(t, expectedMetrics, *localRes) - require.Equal(t, 0, len(*forwardRes)) - require.Equal(t, 0, len(*onForwardedFlushedRes)) - require.Equal(t, 3, len(e.values)) - require.Equal(t, expectedHeartbeat, e.sourcesHeartbeat) - require.True(t, expectedSourceSet.Equal(e.sourcesSet)) - require.Equal(t, nowFn().UnixNano(), e.lastSourcesRefreshNanos) - require.Equal(t, consumed, e.values[0].lockedAgg.consumeState) - for _, v := range e.values { - require.NotNil(t, v.lockedAgg) - } - require.Equal(t, 1, len(e.cachedSourceSets)) - for _, v := range e.cachedSourceSets { - require.NotNil(t, v) - } - - // Disable eager forwarding and consume another value and expect that - // the aggregations are not eagerly consumed. - e.opts = e.opts.SetEnableEagerForwarding(false) - e.values[2].lockedAgg.consumeState = readyToConsume - localFn, localRes = testFlushLocalMetricFn() - forwardFn, forwardRes = testFlushForwardedMetricFn() - onForwardedFlushedFn, onForwardedFlushedRes = testOnForwardedFlushedFn() - require.False(t, e.Consume(alignedstartAtNanos[3], allowEagerForwarding, isEarlierThanFn, timestampNanosFn, localFn, forwardFn, onForwardedFlushedFn)) - require.Equal(t, expectedLocalMetricsForTimer(alignedstartAtNanos[3], testStoragePolicy, maggregation.DefaultTypes), *localRes) - require.Equal(t, 0, len(*forwardRes)) - require.Equal(t, 0, len(*onForwardedFlushedRes)) - require.Equal(t, 1, len(e.values)) - require.Equal(t, 2, len(e.cachedSourceSets)) - for _, v := range e.cachedSourceSets { - require.NotNil(t, v) - } - - // Now enable eager forwarding and consume another value. - e.opts = e.opts.SetEnableEagerForwarding(true) - localFn, localRes = testFlushLocalMetricFn() - forwardFn, forwardRes = testFlushForwardedMetricFn() - onForwardedFlushedFn, onForwardedFlushedRes = testOnForwardedFlushedFn() - require.False(t, e.Consume(alignedstartAtNanos[2], allowEagerForwarding, isEarlierThanFn, timestampNanosFn, localFn, forwardFn, onForwardedFlushedFn)) - require.Equal(t, expectedLocalMetricsForTimer(alignedstartAtNanos[4], testStoragePolicy, maggregation.DefaultTypes), *localRes) - require.Equal(t, 0, len(*forwardRes)) - require.Equal(t, 0, len(*onForwardedFlushedRes)) - require.Equal(t, 1, len(e.values)) - require.Equal(t, 2, len(e.cachedSourceSets)) - for _, v := range e.cachedSourceSets { - require.NotNil(t, v) - } - - // Tombstone the element and discard all values. - e.tombstoned = true - localFn, localRes = testFlushLocalMetricFn() - forwardFn, forwardRes = testFlushForwardedMetricFn() - onForwardedFlushedFn, onForwardedFlushedRes = testOnForwardedFlushedFn() - require.True(t, e.Consume(alignedstartAtNanos[4], allowEagerForwarding, isEarlierThanFn, timestampNanosFn, localFn, forwardFn, onForwardedFlushedFn)) + require.False(t, e.Consume(alignedstartAtNanos[3], isEarlierThanFn, timestampNanosFn, localFn, forwardFn, onForwardedFlushedFn)) require.Equal(t, 0, len(*localRes)) require.Equal(t, 0, len(*forwardRes)) require.Equal(t, 0, len(*onForwardedFlushedRes)) require.Equal(t, 0, len(e.values)) - require.Equal(t, 2, len(e.cachedSourceSets)) - for _, v := range e.cachedSourceSets { - require.NotNil(t, v) - } -} - -func TestTimerElemConsumeSourcesReadyDisallowEagerForwarding(t *testing.T) { - alignedstartAtNanos := []int64{ - time.Unix(210, 0).UnixNano(), - time.Unix(220, 0).UnixNano(), - time.Unix(230, 0).UnixNano(), - time.Unix(240, 0).UnixNano(), - time.Unix(250, 0).UnixNano(), - } - timerVal := testBatchTimer.BatchTimerVal - timerVals := [][]float64{timerVal, timerVal, timerVal, timerVal} - isEarlierThanFn := isStandardMetricEarlierThan - timestampNanosFn := standardMetricTimestampNanos - nowFn := func() time.Time { return time.Unix(200, 0) } - clockOpts := clock.NewOptions().SetNowFn(nowFn) - opts := NewOptions(). - SetClockOptions(clockOpts). - SetDiscardNaNAggregatedValues(false). - SetEnableEagerForwarding(true). - SetForwardingSourcesTTLFn(func(time.Duration) time.Duration { return time.Minute }). - SetMaxNumCachedSourceSets(2) - e := testTimerElem(alignedstartAtNanos[:len(alignedstartAtNanos)-1], timerVals, maggregation.DefaultTypes, applied.DefaultPipeline, opts) - e.incomingMetricType = ForwardedIncomingMetric - e.buildingSourcesAtNanos = time.Unix(10, 0).UnixNano() - e.values[1].lockedAgg.consumeState = readyToConsume - e.sourcesHeartbeat = map[uint32]int64{ - 1: time.Unix(200, 0).UnixNano(), // still live - 2: time.Unix(100, 0).UnixNano(), // stale - 3: time.Unix(150, 0).UnixNano(), // still live - } - e.sourcesSet = bitset.New(10) - for k := range e.sourcesHeartbeat { - e.sourcesSet.Set(uint(k)) - } - - // Consume one value. - localFn, localRes := testFlushLocalMetricFn() - forwardFn, forwardRes := testFlushForwardedMetricFn() - onForwardedFlushedFn, onForwardedFlushedRes := testOnForwardedFlushedFn() - require.False(t, e.Consume(alignedstartAtNanos[1], disAllowEagerForwarding, isEarlierThanFn, timestampNanosFn, localFn, forwardFn, onForwardedFlushedFn)) - - var expectedMetrics []testLocalMetricWithMetadata - expectedMetrics = append(expectedMetrics, expectedLocalMetricsForTimer(alignedstartAtNanos[1], testStoragePolicy, maggregation.DefaultTypes)...) - expectedHeartbeat := map[uint32]int64{ - 1: time.Unix(200, 0).UnixNano(), // still live - 3: time.Unix(150, 0).UnixNano(), // still live - } - expectedSourceSet := bitset.New(e.sourcesSet.Len()) - expectedSourceSet.Set(1) - expectedSourceSet.Set(3) - require.Equal(t, expectedMetrics, *localRes) - require.Equal(t, 0, len(*forwardRes)) - require.Equal(t, 0, len(*onForwardedFlushedRes)) - require.Equal(t, 3, len(e.values)) - require.Equal(t, expectedHeartbeat, e.sourcesHeartbeat) - require.True(t, expectedSourceSet.Equal(e.sourcesSet)) - require.Equal(t, nowFn().UnixNano(), e.lastSourcesRefreshNanos) - require.Equal(t, readyToConsume, e.values[0].lockedAgg.consumeState) - for _, v := range e.values { - require.NotNil(t, v.lockedAgg) - } - require.Equal(t, 1, len(e.cachedSourceSets)) - for _, v := range e.cachedSourceSets { - require.NotNil(t, v) - } - - // Disable eager forwarding and consume another value and expect that - // the aggregations are not eagerly consumed. - e.opts = e.opts.SetEnableEagerForwarding(false) - e.values[2].lockedAgg.consumeState = readyToConsume - localFn, localRes = testFlushLocalMetricFn() - forwardFn, forwardRes = testFlushForwardedMetricFn() - onForwardedFlushedFn, onForwardedFlushedRes = testOnForwardedFlushedFn() - require.False(t, e.Consume(alignedstartAtNanos[3], disAllowEagerForwarding, isEarlierThanFn, timestampNanosFn, localFn, forwardFn, onForwardedFlushedFn)) - - expectedMetrics = nil - expectedMetrics = append(expectedMetrics, expectedLocalMetricsForTimer(alignedstartAtNanos[2], testStoragePolicy, maggregation.DefaultTypes)...) - expectedMetrics = append(expectedMetrics, expectedLocalMetricsForTimer(alignedstartAtNanos[3], testStoragePolicy, maggregation.DefaultTypes)...) - require.Equal(t, expectedMetrics, *localRes) - require.Equal(t, 0, len(*forwardRes)) - require.Equal(t, 0, len(*onForwardedFlushedRes)) - require.Equal(t, 1, len(e.values)) - require.Equal(t, 2, len(e.cachedSourceSets)) - for _, v := range e.cachedSourceSets { - require.NotNil(t, v) - } - - // Now enable eager forwarding and consume another value. - e.opts = e.opts.SetEnableEagerForwarding(true) - localFn, localRes = testFlushLocalMetricFn() - forwardFn, forwardRes = testFlushForwardedMetricFn() - onForwardedFlushedFn, onForwardedFlushedRes = testOnForwardedFlushedFn() - require.False(t, e.Consume(alignedstartAtNanos[2], disAllowEagerForwarding, isEarlierThanFn, timestampNanosFn, localFn, forwardFn, onForwardedFlushedFn)) - require.Nil(t, *localRes) - require.Equal(t, 0, len(*forwardRes)) - require.Equal(t, 0, len(*onForwardedFlushedRes)) - require.Equal(t, 1, len(e.values)) - require.Equal(t, 2, len(e.cachedSourceSets)) - for _, v := range e.cachedSourceSets { - require.NotNil(t, v) - } - - // Tombstone the element and discard all values. - e.tombstoned = true - localFn, localRes = testFlushLocalMetricFn() - forwardFn, forwardRes = testFlushForwardedMetricFn() - onForwardedFlushedFn, onForwardedFlushedRes = testOnForwardedFlushedFn() - require.True(t, e.Consume(alignedstartAtNanos[4], allowEagerForwarding, isEarlierThanFn, timestampNanosFn, localFn, forwardFn, onForwardedFlushedFn)) - require.Equal(t, expectedLocalMetricsForTimer(alignedstartAtNanos[4], testStoragePolicy, maggregation.DefaultTypes), *localRes) - require.Equal(t, 0, len(*forwardRes)) - require.Equal(t, 0, len(*onForwardedFlushedRes)) - require.Equal(t, 0, len(e.values)) - require.Equal(t, 2, len(e.cachedSourceSets)) - for _, v := range e.cachedSourceSets { - require.NotNil(t, v) - } -} - -func TestTimerElemOutgoingMetricType(t *testing.T) { - inputs := []struct { - pipeline applied.Pipeline - expected outgoingMetricType - }{ - { - pipeline: applied.DefaultPipeline, - expected: localOutgoingMetric, - }, - { - pipeline: testPipeline, - expected: forwardedOutgoingMetric, - }, - } - - for _, input := range inputs { - e, err := NewTimerElem(StandardIncomingMetric, testBatchTimerID, testStoragePolicy, maggregation.DefaultTypes, input.pipeline, testNumForwardedTimes, NewOptions()) - require.NoError(t, err) - require.Equal(t, input.expected, e.outgoingMetricType()) - } } func TestTimerElemClose(t *testing.T) { @@ -1825,8 +1104,6 @@ func TestTimerElemClose(t *testing.T) { require.Equal(t, parsedPipeline{}, e.parsedPipeline) require.Nil(t, e.writeForwardedMetricFn) require.Nil(t, e.onForwardedAggregationWrittenFn) - require.Nil(t, e.sourcesHeartbeat) - require.Nil(t, e.sourcesSet) require.Nil(t, e.cachedSourceSets) require.Equal(t, 0, len(e.values)) require.Equal(t, 0, len(e.toConsume)) @@ -1838,7 +1115,7 @@ func TestTimerElemClose(t *testing.T) { } func TestTimerFindOrCreateNoSourceSet(t *testing.T) { - e, err := NewTimerElem(StandardIncomingMetric, testBatchTimerID, testStoragePolicy, maggregation.DefaultTypes, applied.DefaultPipeline, testNumForwardedTimes, NewOptions()) + e, err := NewTimerElem(testBatchTimerID, testStoragePolicy, maggregation.DefaultTypes, applied.DefaultPipeline, testNumForwardedTimes, NewOptions()) require.NoError(t, err) inputs := []int64{10, 10, 20, 10, 15} @@ -1850,27 +1127,21 @@ func TestTimerFindOrCreateNoSourceSet(t *testing.T) { {index: 1, data: []int64{10, 15, 20}}, } for idx, input := range inputs { - res, err := e.findOrCreate(input, sourcesOptions{}) + res, err := e.findOrCreate(input, createAggregationOptions{initSourceSet: false}) require.NoError(t, err) var times []int64 for _, v := range e.values { times = append(times, v.startAtNanos) } require.Equal(t, e.values[expected[idx].index].lockedAgg, res) - require.Nil(t, e.values[expected[idx].index].lockedAgg.seenSources) - require.Nil(t, e.values[expected[idx].index].lockedAgg.expectedSources) require.Equal(t, expected[idx].data, times) } } -func TestTimerFindOrCreateWithCachedSourceSetNoEagerForwarding(t *testing.T) { - opts := NewOptions().SetEnableEagerForwarding(false) - e, err := NewTimerElem(StandardIncomingMetric, testBatchTimerID, testStoragePolicy, maggregation.DefaultTypes, applied.DefaultPipeline, testNumForwardedTimes, opts) +func TestTimerFindOrCreateWithSourceSet(t *testing.T) { + e, err := NewTimerElem(testBatchTimerID, testStoragePolicy, maggregation.DefaultTypes, applied.DefaultPipeline, testNumForwardedTimes, NewOptions()) require.NoError(t, err) - - // Intentionally set up cached source sets. - e.cachedSourceSets = []*bitset.BitSet{bitset.From([]uint64{1, 2})} - require.False(t, e.cachedSourceSets[0].None()) + e.cachedSourceSets = []*bitset.BitSet{bitset.New(0)} inputs := []int64{10, 20} expected := []testIndexData{ @@ -1878,7 +1149,7 @@ func TestTimerFindOrCreateWithCachedSourceSetNoEagerForwarding(t *testing.T) { {index: 1, data: []int64{10, 20}}, } for idx, input := range inputs { - res, err := e.findOrCreate(input, sourcesOptions{updateSources: true, source: 1234}) + res, err := e.findOrCreate(input, createAggregationOptions{initSourceSet: true}) require.NoError(t, err) var times []int64 for _, v := range e.values { @@ -1886,65 +1157,21 @@ func TestTimerFindOrCreateWithCachedSourceSetNoEagerForwarding(t *testing.T) { } require.Equal(t, e.values[expected[idx].index].lockedAgg, res) require.Equal(t, expected[idx].data, times) - require.Nil(t, e.values[expected[idx].index].lockedAgg.expectedSources) - require.NotNil(t, e.values[expected[idx].index].lockedAgg.seenSources) - require.True(t, e.values[expected[idx].index].lockedAgg.seenSources.None()) + require.NotNil(t, e.values[expected[idx].index].lockedAgg.sourcesSeen) } require.Equal(t, 0, len(e.cachedSourceSets)) } -func TestTimerFindOrCreateWithNoCachedSourceSetWithEagerForwarding(t *testing.T) { - forwardingSourcesTTLFn := func(time.Duration) time.Duration { return 10 * time.Second } - now := time.Unix(1234, 0) - nowFn := func() time.Time { return now } - clockOpts := clock.NewOptions().SetNowFn(nowFn) - opts := NewOptions(). - SetClockOptions(clockOpts). - SetEnableEagerForwarding(true). - SetForwardingSourcesTTLFn(forwardingSourcesTTLFn) - e, err := NewTimerElem(StandardIncomingMetric, testBatchTimerID, testStoragePolicy, maggregation.DefaultTypes, applied.DefaultPipeline, testNumForwardedTimes, opts) - require.NoError(t, err) - require.Nil(t, e.sourcesHeartbeat) - require.Equal(t, int64(0), e.buildingSourcesAtNanos) - - // First time should not clone the source set. - sourceOpts := sourcesOptions{updateSources: true, source: 1234} - res, err := e.findOrCreate(1230*time.Second.Nanoseconds(), sourceOpts) - require.NoError(t, err) - require.Equal(t, int64(1234*time.Second.Nanoseconds()), e.buildingSourcesAtNanos) - require.Equal(t, int64(1234*time.Second.Nanoseconds()), e.lastSourcesRefreshNanos) - require.True(t, e.sourcesSet.Test(uint(1234))) - require.Nil(t, res.expectedSources) - require.NotNil(t, res.seenSources) - require.True(t, res.seenSources.None()) - - // Move time forward and create a second time should clone the source set. - now = time.Unix(1300, 0) - sourceOpts = sourcesOptions{updateSources: true, source: 5678} - res, err = e.findOrCreate(1240*time.Second.Nanoseconds(), sourceOpts) - require.NoError(t, err) - require.Equal(t, int64(1234*time.Second.Nanoseconds()), e.buildingSourcesAtNanos) - require.Equal(t, int64(1234*time.Second.Nanoseconds()), e.lastSourcesRefreshNanos) - require.True(t, e.sourcesSet.Test(uint(1234))) - require.True(t, e.sourcesSet.Test(uint(5678))) - require.NotNil(t, res.expectedSources) - require.True(t, res.expectedSources.Test(uint(1234))) - require.True(t, res.expectedSources.Test(uint(5678))) - require.NotNil(t, res.seenSources) - require.True(t, res.seenSources.None()) -} - func TestGaugeResetSetData(t *testing.T) { opts := NewOptions() - ge, err := NewGaugeElem(StandardIncomingMetric, nil, policy.EmptyStoragePolicy, maggregation.DefaultTypes, applied.DefaultPipeline, testNumForwardedTimes, opts) + ge, err := NewGaugeElem(nil, policy.EmptyStoragePolicy, maggregation.DefaultTypes, applied.DefaultPipeline, testNumForwardedTimes, opts) require.NoError(t, err) - require.Equal(t, StandardIncomingMetric, ge.incomingMetricType) require.Equal(t, opts.AggregationTypesOptions().DefaultGaugeAggregationTypes(), ge.aggTypes) require.True(t, ge.useDefaultAggregation) require.False(t, ge.aggOpts.HasExpensiveAggregations) // Reset element with a default pipeline. - err = ge.ResetSetData(StandardIncomingMetric, testGaugeID, testStoragePolicy, testAggregationTypesExpensive, applied.DefaultPipeline, 0) + err = ge.ResetSetData(testGaugeID, testStoragePolicy, testAggregationTypesExpensive, applied.DefaultPipeline, 0) require.NoError(t, err) require.Equal(t, testGaugeID, ge.id) require.Equal(t, testStoragePolicy, ge.sp) @@ -1984,7 +1211,7 @@ func TestGaugeResetSetData(t *testing.T) { }, }), } - err = ge.ResetSetData(StandardIncomingMetric, testGaugeID, testStoragePolicy, testAggregationTypesExpensive, testPipeline, 0) + err = ge.ResetSetData(testGaugeID, testStoragePolicy, testAggregationTypesExpensive, testPipeline, 0) require.NoError(t, err) require.Equal(t, expectedParsedPipeline, ge.parsedPipeline) require.Equal(t, len(testAggregationTypesExpensive), len(ge.lastConsumedValues)) @@ -1994,7 +1221,7 @@ func TestGaugeResetSetData(t *testing.T) { } func TestGaugeElemAddUnion(t *testing.T) { - e, err := NewGaugeElem(StandardIncomingMetric, testGaugeID, testStoragePolicy, maggregation.DefaultTypes, applied.DefaultPipeline, testNumForwardedTimes, NewOptions()) + e, err := NewGaugeElem(testGaugeID, testStoragePolicy, maggregation.DefaultTypes, applied.DefaultPipeline, testNumForwardedTimes, NewOptions()) require.NoError(t, err) // Add a gauge metric. @@ -2030,7 +1257,7 @@ func TestGaugeElemAddUnion(t *testing.T) { } func TestGaugeElemAddUnionWithCustomAggregation(t *testing.T) { - e, err := NewGaugeElem(StandardIncomingMetric, testGaugeID, testStoragePolicy, testAggregationTypesExpensive, applied.DefaultPipeline, testNumForwardedTimes, NewOptions()) + e, err := NewGaugeElem(testGaugeID, testStoragePolicy, testAggregationTypesExpensive, applied.DefaultPipeline, testNumForwardedTimes, NewOptions()) require.NoError(t, err) // Add a gauge metric. @@ -2067,9 +1294,8 @@ func TestGaugeElemAddUnionWithCustomAggregation(t *testing.T) { require.Equal(t, errElemClosed, e.AddUnion(testTimestamps[2], testGauge)) } -func TestGaugeElemAddUniqueSourcesNotReady(t *testing.T) { - opts := NewOptions().SetEnableEagerForwarding(true) - e, err := NewGaugeElem(ForwardedIncomingMetric, testGaugeID, testStoragePolicy, maggregation.DefaultTypes, applied.DefaultPipeline, testNumForwardedTimes, opts) +func TestGaugeElemAddUnique(t *testing.T) { + e, err := NewGaugeElem(testGaugeID, testStoragePolicy, maggregation.DefaultTypes, applied.DefaultPipeline, testNumForwardedTimes, NewOptions()) require.NoError(t, err) // Add a metric. @@ -2080,8 +1306,7 @@ func TestGaugeElemAddUniqueSourcesNotReady(t *testing.T) { require.Equal(t, 46.8, e.values[0].lockedAgg.aggregation.Sum()) require.Equal(t, int64(2), e.values[0].lockedAgg.aggregation.Count()) require.Equal(t, 0.0, e.values[0].lockedAgg.aggregation.SumSq()) - require.Nil(t, e.values[0].lockedAgg.expectedSources) - require.True(t, e.values[0].lockedAgg.seenSources.Test(uint(source1))) + require.True(t, e.values[0].lockedAgg.sourcesSeen.Test(uint(source1))) // Add another metric at slightly different time but still within the // same aggregation interval with a different source. @@ -2092,8 +1317,7 @@ func TestGaugeElemAddUniqueSourcesNotReady(t *testing.T) { require.Equal(t, 96.8, e.values[0].lockedAgg.aggregation.Sum()) require.Equal(t, int64(3), e.values[0].lockedAgg.aggregation.Count()) require.Equal(t, 0.0, e.values[0].lockedAgg.aggregation.SumSq()) - require.Nil(t, e.values[0].lockedAgg.expectedSources) - require.True(t, e.values[0].lockedAgg.seenSources.Test(uint(source2))) + require.True(t, e.values[0].lockedAgg.sourcesSeen.Test(uint(source2))) // Add the metric in the next aggregation interval. require.NoError(t, e.AddUnique(testTimestamps[2], []float64{27.8}, source1)) @@ -2104,8 +1328,7 @@ func TestGaugeElemAddUniqueSourcesNotReady(t *testing.T) { require.Equal(t, 27.8, e.values[1].lockedAgg.aggregation.Sum()) require.Equal(t, int64(1), e.values[1].lockedAgg.aggregation.Count()) require.Equal(t, 0.0, e.values[1].lockedAgg.aggregation.SumSq()) - require.Nil(t, e.values[1].lockedAgg.expectedSources) - require.True(t, e.values[1].lockedAgg.seenSources.Test(uint(source1))) + require.True(t, e.values[1].lockedAgg.sourcesSeen.Test(uint(source1))) // Add the gauge metric in the same aggregation interval with the same // source results in an error. @@ -2117,92 +1340,15 @@ func TestGaugeElemAddUniqueSourcesNotReady(t *testing.T) { require.Equal(t, 27.8, e.values[1].lockedAgg.aggregation.Sum()) require.Equal(t, int64(1), e.values[1].lockedAgg.aggregation.Count()) require.Equal(t, 0.0, e.values[1].lockedAgg.aggregation.SumSq()) - require.Nil(t, e.values[1].lockedAgg.expectedSources) - require.True(t, e.values[1].lockedAgg.seenSources.Test(uint(source1))) + require.True(t, e.values[1].lockedAgg.sourcesSeen.Test(uint(source1))) // Adding the gauge metric to a closed element results in an error. e.closed = true require.Equal(t, errElemClosed, e.AddUnique(testTimestamps[2], []float64{10.0}, 3)) } -func TestGaugeElemAddUniqueSourcesReady(t *testing.T) { - opts := NewOptions(). - SetEnableEagerForwarding(true). - SetForwardingSourcesTTLFn(func(time.Duration) time.Duration { return 0 }) - e, err := NewGaugeElem(ForwardedIncomingMetric, testGaugeID, testStoragePolicy, maggregation.DefaultTypes, applied.DefaultPipeline, testNumForwardedTimes, opts) - require.NoError(t, err) - - // Source is not ready the first time a metric is added. - source1 := uint32(1234) - require.NoError(t, e.AddUnique(testTimestamps[0], []float64{345}, source1)) - require.Equal(t, 1, len(e.values)) - require.Equal(t, testAlignedStarts[0], e.values[0].startAtNanos) - require.Equal(t, float64(345), e.values[0].lockedAgg.aggregation.Sum()) - require.Equal(t, int64(1), e.values[0].lockedAgg.aggregation.Count()) - require.Equal(t, float64(0), e.values[0].lockedAgg.aggregation.SumSq()) - require.NotNil(t, e.values[0].lockedAgg.expectedSources) - require.True(t, e.values[0].lockedAgg.expectedSources.None()) - require.True(t, e.values[0].lockedAgg.seenSources.Test(uint(source1))) - require.Equal(t, readyToConsume, e.values[0].lockedAgg.consumeState) - - // Add the metric in the next aggregation interval. - source2 := uint32(5678) - require.NoError(t, e.AddUnique(testTimestamps[2], []float64{278}, source2)) - require.Equal(t, 2, len(e.values)) - for i := 0; i < len(e.values); i++ { - require.Equal(t, testAlignedStarts[i], e.values[i].startAtNanos) - } - require.Equal(t, float64(278), e.values[1].lockedAgg.aggregation.Sum()) - require.Equal(t, int64(1), e.values[1].lockedAgg.aggregation.Count()) - require.Equal(t, float64(0), e.values[1].lockedAgg.aggregation.SumSq()) - require.True(t, e.values[1].lockedAgg.expectedSources.Test(uint(source1))) - require.False(t, e.values[1].lockedAgg.expectedSources.Test(uint(source2))) - require.False(t, e.values[1].lockedAgg.seenSources.Test(uint(source1))) - require.True(t, e.values[1].lockedAgg.seenSources.Test(uint(source2))) - require.Equal(t, notReadyToConsume, e.values[1].lockedAgg.consumeState) - - // Now add another metric within the same aggregation interval with a different source. - require.NoError(t, e.AddUnique(testTimestamps[2], []float64{500}, source1)) - require.Equal(t, 2, len(e.values)) - require.Equal(t, testAlignedStarts[1], e.values[1].startAtNanos) - require.Equal(t, float64(778), e.values[1].lockedAgg.aggregation.Sum()) - require.Equal(t, int64(2), e.values[1].lockedAgg.aggregation.Count()) - require.Equal(t, float64(0), e.values[1].lockedAgg.aggregation.SumSq()) - require.True(t, e.values[1].lockedAgg.expectedSources.None()) - require.True(t, e.values[1].lockedAgg.seenSources.Test(uint(source1))) - require.True(t, e.values[1].lockedAgg.seenSources.Test(uint(source2))) - require.Equal(t, readyToConsume, e.values[1].lockedAgg.consumeState) - - // Add the metric with the same source results in an error. - require.Equal(t, errDuplicateForwardingSource, e.AddUnique(testTimestamps[2], []float64{500}, source1)) - require.Equal(t, 2, len(e.values)) - require.Equal(t, testAlignedStarts[1], e.values[1].startAtNanos) - require.Equal(t, float64(778), e.values[1].lockedAgg.aggregation.Sum()) - require.Equal(t, int64(2), e.values[1].lockedAgg.aggregation.Count()) - require.Equal(t, float64(0), e.values[1].lockedAgg.aggregation.SumSq()) - require.True(t, e.values[1].lockedAgg.expectedSources.None()) - require.True(t, e.values[1].lockedAgg.seenSources.Test(uint(source1))) - require.True(t, e.values[1].lockedAgg.seenSources.Test(uint(source2))) - require.Equal(t, readyToConsume, e.values[1].lockedAgg.consumeState) - - // Add the metric with a new source does not result in an error and updates the source set. - source3 := uint32(1987) - require.NoError(t, e.AddUnique(testTimestamps[2], []float64{500}, source3)) - require.Equal(t, 2, len(e.values)) - require.Equal(t, testAlignedStarts[1], e.values[1].startAtNanos) - require.Equal(t, float64(1278), e.values[1].lockedAgg.aggregation.Sum()) - require.Equal(t, int64(3), e.values[1].lockedAgg.aggregation.Count()) - require.Equal(t, float64(0), e.values[1].lockedAgg.aggregation.SumSq()) - require.True(t, e.sourcesSet.Test(uint(source3))) - require.True(t, e.values[1].lockedAgg.expectedSources.None()) - require.True(t, e.values[1].lockedAgg.seenSources.Test(uint(source1))) - require.True(t, e.values[1].lockedAgg.seenSources.Test(uint(source2))) - require.True(t, e.values[1].lockedAgg.seenSources.Test(uint(source3))) - require.Equal(t, readyToConsume, e.values[1].lockedAgg.consumeState) -} - func TestGaugeElemAddUniqueWithCustomAggregation(t *testing.T) { - e, err := NewGaugeElem(ForwardedIncomingMetric, testGaugeID, testStoragePolicy, testAggregationTypesExpensive, applied.DefaultPipeline, testNumForwardedTimes, NewOptions()) + e, err := NewGaugeElem(testGaugeID, testStoragePolicy, testAggregationTypesExpensive, applied.DefaultPipeline, testNumForwardedTimes, NewOptions()) require.NoError(t, err) // Add a gauge metric. @@ -2213,7 +1359,7 @@ func TestGaugeElemAddUniqueWithCustomAggregation(t *testing.T) { require.Equal(t, 1.2, e.values[0].lockedAgg.aggregation.Sum()) require.Equal(t, 1.2, e.values[0].lockedAgg.aggregation.Max()) require.Equal(t, 1.44, e.values[0].lockedAgg.aggregation.SumSq()) - require.True(t, e.values[0].lockedAgg.seenSources.Test(uint(source1))) + require.True(t, e.values[0].lockedAgg.sourcesSeen.Test(uint(source1))) // Add the gauge metric at slightly different time // but still within the same aggregation interval. @@ -2244,7 +1390,7 @@ func TestGaugeElemAddUniqueWithCustomAggregation(t *testing.T) { require.Equal(t, 2.0, e.values[1].lockedAgg.aggregation.Sum()) require.Equal(t, 2.0, e.values[1].lockedAgg.aggregation.Max()) require.Equal(t, 4.0, e.values[1].lockedAgg.aggregation.SumSq()) - require.True(t, e.values[1].lockedAgg.seenSources.Test(uint(source1))) + require.True(t, e.values[1].lockedAgg.sourcesSeen.Test(uint(source1))) // Adding the gauge metric to a closed element results in an error. e.closed = true @@ -2261,7 +1407,7 @@ func TestGaugeElemConsumeDefaultAggregationDefaultPipeline(t *testing.T) { localFn, localRes := testFlushLocalMetricFn() forwardFn, forwardRes := testFlushForwardedMetricFn() onForwardedFlushedFn, onForwardedFlushedRes := testOnForwardedFlushedFn() - require.False(t, e.Consume(0, allowEagerForwarding, isEarlierThanFn, timestampNanosFn, localFn, forwardFn, onForwardedFlushedFn)) + require.False(t, e.Consume(0, isEarlierThanFn, timestampNanosFn, localFn, forwardFn, onForwardedFlushedFn)) require.Equal(t, 0, len(*localRes)) require.Equal(t, 0, len(*forwardRes)) require.Equal(t, 0, len(*onForwardedFlushedRes)) @@ -2271,7 +1417,7 @@ func TestGaugeElemConsumeDefaultAggregationDefaultPipeline(t *testing.T) { localFn, localRes = testFlushLocalMetricFn() forwardFn, forwardRes = testFlushForwardedMetricFn() onForwardedFlushedFn, onForwardedFlushedRes = testOnForwardedFlushedFn() - require.False(t, e.Consume(testAlignedStarts[1], allowEagerForwarding, isEarlierThanFn, timestampNanosFn, localFn, forwardFn, onForwardedFlushedFn)) + require.False(t, e.Consume(testAlignedStarts[1], isEarlierThanFn, timestampNanosFn, localFn, forwardFn, onForwardedFlushedFn)) require.Equal(t, expectedLocalMetricsForGauge(testAlignedStarts[1], testStoragePolicy, maggregation.DefaultTypes), *localRes) require.Equal(t, 0, len(*forwardRes)) require.Equal(t, 0, len(*onForwardedFlushedRes)) @@ -2281,7 +1427,7 @@ func TestGaugeElemConsumeDefaultAggregationDefaultPipeline(t *testing.T) { localFn, localRes = testFlushLocalMetricFn() forwardFn, forwardRes = testFlushForwardedMetricFn() onForwardedFlushedFn, onForwardedFlushedRes = testOnForwardedFlushedFn() - require.False(t, e.Consume(testAlignedStarts[2], allowEagerForwarding, isEarlierThanFn, timestampNanosFn, localFn, forwardFn, onForwardedFlushedFn)) + require.False(t, e.Consume(testAlignedStarts[2], isEarlierThanFn, timestampNanosFn, localFn, forwardFn, onForwardedFlushedFn)) require.Equal(t, expectedLocalMetricsForGauge(testAlignedStarts[2], testStoragePolicy, maggregation.DefaultTypes), *localRes) require.Equal(t, 0, len(*forwardRes)) require.Equal(t, 0, len(*onForwardedFlushedRes)) @@ -2292,7 +1438,7 @@ func TestGaugeElemConsumeDefaultAggregationDefaultPipeline(t *testing.T) { localFn, localRes = testFlushLocalMetricFn() forwardFn, forwardRes = testFlushForwardedMetricFn() onForwardedFlushedFn, onForwardedFlushedRes = testOnForwardedFlushedFn() - require.True(t, e.Consume(testAlignedStarts[2], allowEagerForwarding, isEarlierThanFn, timestampNanosFn, localFn, forwardFn, onForwardedFlushedFn)) + require.True(t, e.Consume(testAlignedStarts[2], isEarlierThanFn, timestampNanosFn, localFn, forwardFn, onForwardedFlushedFn)) require.Equal(t, 0, len(*localRes)) require.Equal(t, 0, len(*forwardRes)) require.Equal(t, 0, len(*onForwardedFlushedRes)) @@ -2303,7 +1449,7 @@ func TestGaugeElemConsumeDefaultAggregationDefaultPipeline(t *testing.T) { localFn, localRes = testFlushLocalMetricFn() forwardFn, forwardRes = testFlushForwardedMetricFn() onForwardedFlushedFn, onForwardedFlushedRes = testOnForwardedFlushedFn() - require.False(t, e.Consume(testAlignedStarts[2], allowEagerForwarding, isEarlierThanFn, timestampNanosFn, localFn, forwardFn, onForwardedFlushedFn)) + require.False(t, e.Consume(testAlignedStarts[2], isEarlierThanFn, timestampNanosFn, localFn, forwardFn, onForwardedFlushedFn)) require.Equal(t, 0, len(*localRes)) require.Equal(t, 0, len(*forwardRes)) require.Equal(t, 0, len(*onForwardedFlushedRes)) @@ -2320,7 +1466,7 @@ func TestGaugeElemConsumeCustomAggregationDefaultPipeline(t *testing.T) { localFn, localRes := testFlushLocalMetricFn() forwardFn, forwardRes := testFlushForwardedMetricFn() onForwardedFlushedFn, onForwardedFlushedRes := testOnForwardedFlushedFn() - require.False(t, e.Consume(0, allowEagerForwarding, isEarlierThanFn, timestampNanosFn, localFn, forwardFn, onForwardedFlushedFn)) + require.False(t, e.Consume(0, isEarlierThanFn, timestampNanosFn, localFn, forwardFn, onForwardedFlushedFn)) require.Equal(t, 0, len(*localRes)) require.Equal(t, 0, len(*forwardRes)) require.Equal(t, 0, len(*onForwardedFlushedRes)) @@ -2330,7 +1476,7 @@ func TestGaugeElemConsumeCustomAggregationDefaultPipeline(t *testing.T) { localFn, localRes = testFlushLocalMetricFn() forwardFn, forwardRes = testFlushForwardedMetricFn() onForwardedFlushedFn, onForwardedFlushedRes = testOnForwardedFlushedFn() - require.False(t, e.Consume(testAlignedStarts[1], allowEagerForwarding, isEarlierThanFn, timestampNanosFn, localFn, forwardFn, onForwardedFlushedFn)) + require.False(t, e.Consume(testAlignedStarts[1], isEarlierThanFn, timestampNanosFn, localFn, forwardFn, onForwardedFlushedFn)) require.Equal(t, expectedLocalMetricsForGauge(testAlignedStarts[1], testStoragePolicy, testAggregationTypes), *localRes) require.Equal(t, 0, len(*forwardRes)) require.Equal(t, 0, len(*onForwardedFlushedRes)) @@ -2340,7 +1486,7 @@ func TestGaugeElemConsumeCustomAggregationDefaultPipeline(t *testing.T) { localFn, localRes = testFlushLocalMetricFn() forwardFn, forwardRes = testFlushForwardedMetricFn() onForwardedFlushedFn, onForwardedFlushedRes = testOnForwardedFlushedFn() - require.False(t, e.Consume(testAlignedStarts[2], allowEagerForwarding, isEarlierThanFn, timestampNanosFn, localFn, forwardFn, onForwardedFlushedFn)) + require.False(t, e.Consume(testAlignedStarts[2], isEarlierThanFn, timestampNanosFn, localFn, forwardFn, onForwardedFlushedFn)) require.Equal(t, expectedLocalMetricsForGauge(testAlignedStarts[2], testStoragePolicy, testAggregationTypes), *localRes) require.Equal(t, 0, len(*forwardRes)) require.Equal(t, 0, len(*onForwardedFlushedRes)) @@ -2351,7 +1497,7 @@ func TestGaugeElemConsumeCustomAggregationDefaultPipeline(t *testing.T) { localFn, localRes = testFlushLocalMetricFn() forwardFn, forwardRes = testFlushForwardedMetricFn() onForwardedFlushedFn, onForwardedFlushedRes = testOnForwardedFlushedFn() - require.True(t, e.Consume(testAlignedStarts[2], allowEagerForwarding, isEarlierThanFn, timestampNanosFn, localFn, forwardFn, onForwardedFlushedFn)) + require.True(t, e.Consume(testAlignedStarts[2], isEarlierThanFn, timestampNanosFn, localFn, forwardFn, onForwardedFlushedFn)) require.Equal(t, 0, len(*localRes)) require.Equal(t, 0, len(*forwardRes)) require.Equal(t, 0, len(*onForwardedFlushedRes)) @@ -2362,7 +1508,7 @@ func TestGaugeElemConsumeCustomAggregationDefaultPipeline(t *testing.T) { localFn, localRes = testFlushLocalMetricFn() forwardFn, forwardRes = testFlushForwardedMetricFn() onForwardedFlushedFn, onForwardedFlushedRes = testOnForwardedFlushedFn() - require.False(t, e.Consume(testAlignedStarts[2], allowEagerForwarding, isEarlierThanFn, timestampNanosFn, localFn, forwardFn, onForwardedFlushedFn)) + require.False(t, e.Consume(testAlignedStarts[2], isEarlierThanFn, timestampNanosFn, localFn, forwardFn, onForwardedFlushedFn)) require.Equal(t, 0, len(*localRes)) require.Equal(t, 0, len(*forwardRes)) require.Equal(t, 0, len(*onForwardedFlushedRes)) @@ -2407,7 +1553,7 @@ func TestGaugeElemConsumeCustomAggregationCustomPipeline(t *testing.T) { localFn, localRes := testFlushLocalMetricFn() forwardFn, forwardRes := testFlushForwardedMetricFn() onForwardedFlushedFn, onForwardedFlushedRes := testOnForwardedFlushedFn() - require.False(t, e.Consume(0, allowEagerForwarding, isEarlierThanFn, timestampNanosFn, localFn, forwardFn, onForwardedFlushedFn)) + require.False(t, e.Consume(0, isEarlierThanFn, timestampNanosFn, localFn, forwardFn, onForwardedFlushedFn)) verifyOnForwardedFlushResult(t, expectedOnFlushedRes, *onForwardedFlushedRes) require.Equal(t, 0, len(*localRes)) require.Equal(t, 0, len(*forwardRes)) @@ -2424,7 +1570,7 @@ func TestGaugeElemConsumeCustomAggregationCustomPipeline(t *testing.T) { localFn, localRes = testFlushLocalMetricFn() forwardFn, forwardRes = testFlushForwardedMetricFn() onForwardedFlushedFn, onForwardedFlushedRes = testOnForwardedFlushedFn() - require.False(t, e.Consume(alignedstartAtNanos[1], allowEagerForwarding, isEarlierThanFn, timestampNanosFn, localFn, forwardFn, onForwardedFlushedFn)) + require.False(t, e.Consume(alignedstartAtNanos[1], isEarlierThanFn, timestampNanosFn, localFn, forwardFn, onForwardedFlushedFn)) verifyForwardedMetrics(t, expectedForwardedRes, *forwardRes) verifyOnForwardedFlushResult(t, expectedOnFlushedRes, *onForwardedFlushedRes) require.Equal(t, 0, len(*localRes)) @@ -2448,7 +1594,7 @@ func TestGaugeElemConsumeCustomAggregationCustomPipeline(t *testing.T) { localFn, localRes = testFlushLocalMetricFn() forwardFn, forwardRes = testFlushForwardedMetricFn() onForwardedFlushedFn, onForwardedFlushedRes = testOnForwardedFlushedFn() - require.False(t, e.Consume(alignedstartAtNanos[3], allowEagerForwarding, isEarlierThanFn, timestampNanosFn, localFn, forwardFn, onForwardedFlushedFn)) + require.False(t, e.Consume(alignedstartAtNanos[3], isEarlierThanFn, timestampNanosFn, localFn, forwardFn, onForwardedFlushedFn)) verifyForwardedMetrics(t, expectedForwardedRes, *forwardRes) verifyOnForwardedFlushResult(t, expectedOnFlushedRes, *onForwardedFlushedRes) require.Equal(t, 0, len(*localRes)) @@ -2461,7 +1607,7 @@ func TestGaugeElemConsumeCustomAggregationCustomPipeline(t *testing.T) { localFn, localRes = testFlushLocalMetricFn() forwardFn, forwardRes = testFlushForwardedMetricFn() onForwardedFlushedFn, onForwardedFlushedRes = testOnForwardedFlushedFn() - require.True(t, e.Consume(alignedstartAtNanos[3], allowEagerForwarding, isEarlierThanFn, timestampNanosFn, localFn, forwardFn, onForwardedFlushedFn)) + require.True(t, e.Consume(alignedstartAtNanos[3], isEarlierThanFn, timestampNanosFn, localFn, forwardFn, onForwardedFlushedFn)) verifyOnForwardedFlushResult(t, expectedOnFlushedRes, *onForwardedFlushedRes) require.Equal(t, 0, len(*localRes)) require.Equal(t, 0, len(*forwardRes)) @@ -2472,266 +1618,13 @@ func TestGaugeElemConsumeCustomAggregationCustomPipeline(t *testing.T) { localFn, localRes = testFlushLocalMetricFn() forwardFn, forwardRes = testFlushForwardedMetricFn() onForwardedFlushedFn, onForwardedFlushedRes = testOnForwardedFlushedFn() - require.False(t, e.Consume(alignedstartAtNanos[3], allowEagerForwarding, isEarlierThanFn, timestampNanosFn, localFn, forwardFn, onForwardedFlushedFn)) + require.False(t, e.Consume(alignedstartAtNanos[3], isEarlierThanFn, timestampNanosFn, localFn, forwardFn, onForwardedFlushedFn)) require.Equal(t, 0, len(*localRes)) require.Equal(t, 0, len(*forwardRes)) require.Equal(t, 0, len(*onForwardedFlushedRes)) require.Equal(t, 0, len(e.values)) } -func TestGaugeElemConsumeSourcesReadyAllowEagerForwarding(t *testing.T) { - alignedstartAtNanos := []int64{ - time.Unix(210, 0).UnixNano(), - time.Unix(220, 0).UnixNano(), - time.Unix(230, 0).UnixNano(), - time.Unix(240, 0).UnixNano(), - time.Unix(250, 0).UnixNano(), - } - gaugeVal := testGauge.GaugeVal - gaugeVals := []float64{gaugeVal, gaugeVal, gaugeVal, gaugeVal} - isEarlierThanFn := isStandardMetricEarlierThan - timestampNanosFn := standardMetricTimestampNanos - nowFn := func() time.Time { return time.Unix(200, 0) } - clockOpts := clock.NewOptions().SetNowFn(nowFn) - opts := NewOptions(). - SetClockOptions(clockOpts). - SetDiscardNaNAggregatedValues(false). - SetEnableEagerForwarding(true). - SetForwardingSourcesTTLFn(func(time.Duration) time.Duration { return time.Minute }). - SetMaxNumCachedSourceSets(2) - e := testGaugeElem(alignedstartAtNanos[:len(alignedstartAtNanos)-1], gaugeVals, maggregation.DefaultTypes, applied.DefaultPipeline, opts) - e.incomingMetricType = ForwardedIncomingMetric - e.buildingSourcesAtNanos = time.Unix(10, 0).UnixNano() - e.values[1].lockedAgg.consumeState = readyToConsume - e.sourcesHeartbeat = map[uint32]int64{ - 1: time.Unix(200, 0).UnixNano(), // still live - 2: time.Unix(100, 0).UnixNano(), // stale - 3: time.Unix(150, 0).UnixNano(), // still live - } - e.sourcesSet = bitset.New(10) - for k := range e.sourcesHeartbeat { - e.sourcesSet.Set(uint(k)) - } - - // Consume one value. - localFn, localRes := testFlushLocalMetricFn() - forwardFn, forwardRes := testFlushForwardedMetricFn() - onForwardedFlushedFn, onForwardedFlushedRes := testOnForwardedFlushedFn() - require.False(t, e.Consume(alignedstartAtNanos[1], allowEagerForwarding, isEarlierThanFn, timestampNanosFn, localFn, forwardFn, onForwardedFlushedFn)) - - var expectedMetrics []testLocalMetricWithMetadata - expectedMetrics = append(expectedMetrics, expectedLocalMetricsForGauge(alignedstartAtNanos[1], testStoragePolicy, maggregation.DefaultTypes)...) - expectedMetrics = append(expectedMetrics, expectedLocalMetricsForGauge(alignedstartAtNanos[2], testStoragePolicy, maggregation.DefaultTypes)...) - expectedHeartbeat := map[uint32]int64{ - 1: time.Unix(200, 0).UnixNano(), // still live - 3: time.Unix(150, 0).UnixNano(), // still live - } - expectedSourceSet := bitset.New(e.sourcesSet.Len()) - expectedSourceSet.Set(1) - expectedSourceSet.Set(3) - require.Equal(t, expectedMetrics, *localRes) - require.Equal(t, 0, len(*forwardRes)) - require.Equal(t, 0, len(*onForwardedFlushedRes)) - require.Equal(t, 3, len(e.values)) - require.Equal(t, expectedHeartbeat, e.sourcesHeartbeat) - require.True(t, expectedSourceSet.Equal(e.sourcesSet)) - require.Equal(t, nowFn().UnixNano(), e.lastSourcesRefreshNanos) - require.Equal(t, consumed, e.values[0].lockedAgg.consumeState) - for _, v := range e.values { - require.NotNil(t, v.lockedAgg) - } - require.Equal(t, 1, len(e.cachedSourceSets)) - for _, v := range e.cachedSourceSets { - require.NotNil(t, v) - } - - // Disable eager forwarding and consume another value and expect that - // the aggregations are not eagerly consumed. - e.opts = e.opts.SetEnableEagerForwarding(false) - e.values[2].lockedAgg.consumeState = readyToConsume - localFn, localRes = testFlushLocalMetricFn() - forwardFn, forwardRes = testFlushForwardedMetricFn() - onForwardedFlushedFn, onForwardedFlushedRes = testOnForwardedFlushedFn() - require.False(t, e.Consume(alignedstartAtNanos[3], allowEagerForwarding, isEarlierThanFn, timestampNanosFn, localFn, forwardFn, onForwardedFlushedFn)) - require.Equal(t, expectedLocalMetricsForGauge(alignedstartAtNanos[3], testStoragePolicy, maggregation.DefaultTypes), *localRes) - require.Equal(t, 0, len(*forwardRes)) - require.Equal(t, 0, len(*onForwardedFlushedRes)) - require.Equal(t, 1, len(e.values)) - require.Equal(t, 2, len(e.cachedSourceSets)) - for _, v := range e.cachedSourceSets { - require.NotNil(t, v) - } - - // Now enable eager forwarding and consume another value. - e.opts = e.opts.SetEnableEagerForwarding(true) - localFn, localRes = testFlushLocalMetricFn() - forwardFn, forwardRes = testFlushForwardedMetricFn() - onForwardedFlushedFn, onForwardedFlushedRes = testOnForwardedFlushedFn() - require.False(t, e.Consume(alignedstartAtNanos[2], allowEagerForwarding, isEarlierThanFn, timestampNanosFn, localFn, forwardFn, onForwardedFlushedFn)) - require.Equal(t, expectedLocalMetricsForGauge(alignedstartAtNanos[4], testStoragePolicy, maggregation.DefaultTypes), *localRes) - require.Equal(t, 0, len(*forwardRes)) - require.Equal(t, 0, len(*onForwardedFlushedRes)) - require.Equal(t, 1, len(e.values)) - require.Equal(t, 2, len(e.cachedSourceSets)) - for _, v := range e.cachedSourceSets { - require.NotNil(t, v) - } - - // Tombstone the element and discard all values. - e.tombstoned = true - localFn, localRes = testFlushLocalMetricFn() - forwardFn, forwardRes = testFlushForwardedMetricFn() - onForwardedFlushedFn, onForwardedFlushedRes = testOnForwardedFlushedFn() - require.True(t, e.Consume(alignedstartAtNanos[4], allowEagerForwarding, isEarlierThanFn, timestampNanosFn, localFn, forwardFn, onForwardedFlushedFn)) - require.Equal(t, 0, len(*localRes)) - require.Equal(t, 0, len(*forwardRes)) - require.Equal(t, 0, len(*onForwardedFlushedRes)) - require.Equal(t, 0, len(e.values)) - require.Equal(t, 2, len(e.cachedSourceSets)) - for _, v := range e.cachedSourceSets { - require.NotNil(t, v) - } -} - -func TestGaugeElemConsumeSourcesReadyDisallowEagerForwarding(t *testing.T) { - alignedstartAtNanos := []int64{ - time.Unix(210, 0).UnixNano(), - time.Unix(220, 0).UnixNano(), - time.Unix(230, 0).UnixNano(), - time.Unix(240, 0).UnixNano(), - time.Unix(250, 0).UnixNano(), - } - gaugeVal := testGauge.GaugeVal - gaugeVals := []float64{gaugeVal, gaugeVal, gaugeVal, gaugeVal} - isEarlierThanFn := isStandardMetricEarlierThan - timestampNanosFn := standardMetricTimestampNanos - nowFn := func() time.Time { return time.Unix(200, 0) } - clockOpts := clock.NewOptions().SetNowFn(nowFn) - opts := NewOptions(). - SetClockOptions(clockOpts). - SetDiscardNaNAggregatedValues(false). - SetEnableEagerForwarding(true). - SetForwardingSourcesTTLFn(func(time.Duration) time.Duration { return time.Minute }). - SetMaxNumCachedSourceSets(2) - e := testGaugeElem(alignedstartAtNanos[:len(alignedstartAtNanos)-1], gaugeVals, maggregation.DefaultTypes, applied.DefaultPipeline, opts) - e.incomingMetricType = ForwardedIncomingMetric - e.buildingSourcesAtNanos = time.Unix(10, 0).UnixNano() - e.values[1].lockedAgg.consumeState = readyToConsume - e.sourcesHeartbeat = map[uint32]int64{ - 1: time.Unix(200, 0).UnixNano(), // still live - 2: time.Unix(100, 0).UnixNano(), // stale - 3: time.Unix(150, 0).UnixNano(), // still live - } - e.sourcesSet = bitset.New(10) - for k := range e.sourcesHeartbeat { - e.sourcesSet.Set(uint(k)) - } - - // Consume one value. - localFn, localRes := testFlushLocalMetricFn() - forwardFn, forwardRes := testFlushForwardedMetricFn() - onForwardedFlushedFn, onForwardedFlushedRes := testOnForwardedFlushedFn() - require.False(t, e.Consume(alignedstartAtNanos[1], disAllowEagerForwarding, isEarlierThanFn, timestampNanosFn, localFn, forwardFn, onForwardedFlushedFn)) - - var expectedMetrics []testLocalMetricWithMetadata - expectedMetrics = append(expectedMetrics, expectedLocalMetricsForGauge(alignedstartAtNanos[1], testStoragePolicy, maggregation.DefaultTypes)...) - expectedHeartbeat := map[uint32]int64{ - 1: time.Unix(200, 0).UnixNano(), // still live - 3: time.Unix(150, 0).UnixNano(), // still live - } - expectedSourceSet := bitset.New(e.sourcesSet.Len()) - expectedSourceSet.Set(1) - expectedSourceSet.Set(3) - require.Equal(t, expectedMetrics, *localRes) - require.Equal(t, 0, len(*forwardRes)) - require.Equal(t, 0, len(*onForwardedFlushedRes)) - require.Equal(t, 3, len(e.values)) - require.Equal(t, expectedHeartbeat, e.sourcesHeartbeat) - require.True(t, expectedSourceSet.Equal(e.sourcesSet)) - require.Equal(t, nowFn().UnixNano(), e.lastSourcesRefreshNanos) - require.Equal(t, readyToConsume, e.values[0].lockedAgg.consumeState) - for _, v := range e.values { - require.NotNil(t, v.lockedAgg) - } - require.Equal(t, 1, len(e.cachedSourceSets)) - for _, v := range e.cachedSourceSets { - require.NotNil(t, v) - } - - // Disable eager forwarding and consume another value and expect that - // the aggregations are not eagerly consumed. - e.opts = e.opts.SetEnableEagerForwarding(false) - e.values[2].lockedAgg.consumeState = readyToConsume - localFn, localRes = testFlushLocalMetricFn() - forwardFn, forwardRes = testFlushForwardedMetricFn() - onForwardedFlushedFn, onForwardedFlushedRes = testOnForwardedFlushedFn() - require.False(t, e.Consume(alignedstartAtNanos[3], disAllowEagerForwarding, isEarlierThanFn, timestampNanosFn, localFn, forwardFn, onForwardedFlushedFn)) - - expectedMetrics = nil - expectedMetrics = append(expectedMetrics, expectedLocalMetricsForGauge(alignedstartAtNanos[2], testStoragePolicy, maggregation.DefaultTypes)...) - expectedMetrics = append(expectedMetrics, expectedLocalMetricsForGauge(alignedstartAtNanos[3], testStoragePolicy, maggregation.DefaultTypes)...) - require.Equal(t, expectedMetrics, *localRes) - require.Equal(t, 0, len(*forwardRes)) - require.Equal(t, 0, len(*onForwardedFlushedRes)) - require.Equal(t, 1, len(e.values)) - require.Equal(t, 2, len(e.cachedSourceSets)) - for _, v := range e.cachedSourceSets { - require.NotNil(t, v) - } - - // Now enable eager forwarding and consume another value. - e.opts = e.opts.SetEnableEagerForwarding(true) - localFn, localRes = testFlushLocalMetricFn() - forwardFn, forwardRes = testFlushForwardedMetricFn() - onForwardedFlushedFn, onForwardedFlushedRes = testOnForwardedFlushedFn() - require.False(t, e.Consume(alignedstartAtNanos[2], disAllowEagerForwarding, isEarlierThanFn, timestampNanosFn, localFn, forwardFn, onForwardedFlushedFn)) - require.Nil(t, *localRes) - require.Equal(t, 0, len(*forwardRes)) - require.Equal(t, 0, len(*onForwardedFlushedRes)) - require.Equal(t, 1, len(e.values)) - require.Equal(t, 2, len(e.cachedSourceSets)) - for _, v := range e.cachedSourceSets { - require.NotNil(t, v) - } - - // Tombstone the element and discard all values. - e.tombstoned = true - localFn, localRes = testFlushLocalMetricFn() - forwardFn, forwardRes = testFlushForwardedMetricFn() - onForwardedFlushedFn, onForwardedFlushedRes = testOnForwardedFlushedFn() - require.True(t, e.Consume(alignedstartAtNanos[4], allowEagerForwarding, isEarlierThanFn, timestampNanosFn, localFn, forwardFn, onForwardedFlushedFn)) - require.Equal(t, expectedLocalMetricsForGauge(alignedstartAtNanos[4], testStoragePolicy, maggregation.DefaultTypes), *localRes) - require.Equal(t, 0, len(*forwardRes)) - require.Equal(t, 0, len(*onForwardedFlushedRes)) - require.Equal(t, 0, len(e.values)) - require.Equal(t, 2, len(e.cachedSourceSets)) - for _, v := range e.cachedSourceSets { - require.NotNil(t, v) - } -} - -func TestGaugeElemOutgoingMetricType(t *testing.T) { - inputs := []struct { - pipeline applied.Pipeline - expected outgoingMetricType - }{ - { - pipeline: applied.DefaultPipeline, - expected: localOutgoingMetric, - }, - { - pipeline: testPipeline, - expected: forwardedOutgoingMetric, - }, - } - - for _, input := range inputs { - e, err := NewGaugeElem(StandardIncomingMetric, testGaugeID, testStoragePolicy, maggregation.DefaultTypes, input.pipeline, testNumForwardedTimes, NewOptions()) - require.NoError(t, err) - require.Equal(t, input.expected, e.outgoingMetricType()) - } -} - func TestGaugeElemClose(t *testing.T) { e := testGaugeElem(testAlignedStarts[:len(testAlignedStarts)-1], testGaugeVals, maggregation.DefaultTypes, applied.DefaultPipeline, NewOptions()) require.False(t, e.closed) @@ -2747,8 +1640,6 @@ func TestGaugeElemClose(t *testing.T) { require.Equal(t, parsedPipeline{}, e.parsedPipeline) require.Nil(t, e.writeForwardedMetricFn) require.Nil(t, e.onForwardedAggregationWrittenFn) - require.Nil(t, e.sourcesHeartbeat) - require.Nil(t, e.sourcesSet) require.Nil(t, e.cachedSourceSets) require.Equal(t, 0, len(e.values)) require.Equal(t, 0, len(e.toConsume)) @@ -2757,7 +1648,7 @@ func TestGaugeElemClose(t *testing.T) { } func TestGaugeFindOrCreateNoSourceSet(t *testing.T) { - e, err := NewGaugeElem(StandardIncomingMetric, testGaugeID, testStoragePolicy, maggregation.DefaultTypes, applied.DefaultPipeline, testNumForwardedTimes, NewOptions()) + e, err := NewGaugeElem(testGaugeID, testStoragePolicy, maggregation.DefaultTypes, applied.DefaultPipeline, testNumForwardedTimes, NewOptions()) require.NoError(t, err) inputs := []int64{10, 10, 20, 10, 15} @@ -2769,27 +1660,21 @@ func TestGaugeFindOrCreateNoSourceSet(t *testing.T) { {index: 1, data: []int64{10, 15, 20}}, } for idx, input := range inputs { - res, err := e.findOrCreate(input, sourcesOptions{}) + res, err := e.findOrCreate(input, createAggregationOptions{initSourceSet: false}) require.NoError(t, err) var times []int64 for _, v := range e.values { times = append(times, v.startAtNanos) } require.Equal(t, e.values[expected[idx].index].lockedAgg, res) - require.Nil(t, e.values[expected[idx].index].lockedAgg.seenSources) - require.Nil(t, e.values[expected[idx].index].lockedAgg.expectedSources) require.Equal(t, expected[idx].data, times) } } -func TestGaugeFindOrCreateWithCachedSourceSetNoEagerForwarding(t *testing.T) { - opts := NewOptions().SetEnableEagerForwarding(false) - e, err := NewGaugeElem(StandardIncomingMetric, testGaugeID, testStoragePolicy, maggregation.DefaultTypes, applied.DefaultPipeline, testNumForwardedTimes, opts) +func TestGaugeFindOrCreateWithSourceSet(t *testing.T) { + e, err := NewGaugeElem(testGaugeID, testStoragePolicy, maggregation.DefaultTypes, applied.DefaultPipeline, testNumForwardedTimes, NewOptions()) require.NoError(t, err) - - // Intentionally set up cached source sets. - e.cachedSourceSets = []*bitset.BitSet{bitset.From([]uint64{1, 2})} - require.False(t, e.cachedSourceSets[0].None()) + e.cachedSourceSets = []*bitset.BitSet{bitset.New(0)} inputs := []int64{10, 20} expected := []testIndexData{ @@ -2797,7 +1682,7 @@ func TestGaugeFindOrCreateWithCachedSourceSetNoEagerForwarding(t *testing.T) { {index: 1, data: []int64{10, 20}}, } for idx, input := range inputs { - res, err := e.findOrCreate(input, sourcesOptions{updateSources: true, source: 1234}) + res, err := e.findOrCreate(input, createAggregationOptions{initSourceSet: true}) require.NoError(t, err) var times []int64 for _, v := range e.values { @@ -2805,54 +1690,11 @@ func TestGaugeFindOrCreateWithCachedSourceSetNoEagerForwarding(t *testing.T) { } require.Equal(t, e.values[expected[idx].index].lockedAgg, res) require.Equal(t, expected[idx].data, times) - require.Nil(t, e.values[expected[idx].index].lockedAgg.expectedSources) - require.NotNil(t, e.values[expected[idx].index].lockedAgg.seenSources) - require.True(t, e.values[expected[idx].index].lockedAgg.seenSources.None()) + require.NotNil(t, e.values[expected[idx].index].lockedAgg.sourcesSeen) } require.Equal(t, 0, len(e.cachedSourceSets)) } -func TestGaugeFindOrCreateWithNoCachedSourceSetWithEagerForwarding(t *testing.T) { - forwardingSourcesTTLFn := func(time.Duration) time.Duration { return 10 * time.Second } - now := time.Unix(1234, 0) - nowFn := func() time.Time { return now } - clockOpts := clock.NewOptions().SetNowFn(nowFn) - opts := NewOptions(). - SetClockOptions(clockOpts). - SetEnableEagerForwarding(true). - SetForwardingSourcesTTLFn(forwardingSourcesTTLFn) - e, err := NewGaugeElem(StandardIncomingMetric, testGaugeID, testStoragePolicy, maggregation.DefaultTypes, applied.DefaultPipeline, testNumForwardedTimes, opts) - require.NoError(t, err) - require.Nil(t, e.sourcesHeartbeat) - require.Equal(t, int64(0), e.buildingSourcesAtNanos) - - // First time should not clone the source set. - sourceOpts := sourcesOptions{updateSources: true, source: 1234} - res, err := e.findOrCreate(1230*time.Second.Nanoseconds(), sourceOpts) - require.NoError(t, err) - require.Equal(t, int64(1234*time.Second.Nanoseconds()), e.buildingSourcesAtNanos) - require.Equal(t, int64(1234*time.Second.Nanoseconds()), e.lastSourcesRefreshNanos) - require.True(t, e.sourcesSet.Test(uint(1234))) - require.Nil(t, res.expectedSources) - require.NotNil(t, res.seenSources) - require.True(t, res.seenSources.None()) - - // Move time forward and create a second time should clone the source set. - now = time.Unix(1300, 0) - sourceOpts = sourcesOptions{updateSources: true, source: 5678} - res, err = e.findOrCreate(1240*time.Second.Nanoseconds(), sourceOpts) - require.NoError(t, err) - require.Equal(t, int64(1234*time.Second.Nanoseconds()), e.buildingSourcesAtNanos) - require.Equal(t, int64(1234*time.Second.Nanoseconds()), e.lastSourcesRefreshNanos) - require.True(t, e.sourcesSet.Test(uint(1234))) - require.True(t, e.sourcesSet.Test(uint(5678))) - require.NotNil(t, res.expectedSources) - require.True(t, res.expectedSources.Test(uint(1234))) - require.True(t, res.expectedSources.Test(uint(5678))) - require.NotNil(t, res.seenSources) - require.True(t, res.seenSources.None()) -} - type testIndexData struct { index int data []int64 @@ -2959,12 +1801,9 @@ func testCounterElem( pipeline applied.Pipeline, opts Options, ) *CounterElem { - e := MustNewCounterElem(StandardIncomingMetric, testCounterID, testStoragePolicy, aggTypes, pipeline, testNumForwardedTimes, opts) + e := MustNewCounterElem(testCounterID, testStoragePolicy, aggTypes, pipeline, testNumForwardedTimes, opts) for i, aligned := range alignedstartAtNanos { - counter := &lockedCounterAggregation{ - aggregation: newCounterAggregation(raggregation.NewCounter(e.aggOpts)), - seenSources: bitset.New(10), - } + counter := &lockedCounterAggregation{aggregation: newCounterAggregation(raggregation.NewCounter(e.aggOpts))} counter.aggregation.Update(counterVals[i]) e.values = append(e.values, timedCounter{ startAtNanos: aligned, @@ -2981,13 +1820,10 @@ func testTimerElem( pipeline applied.Pipeline, opts Options, ) *TimerElem { - e := MustNewTimerElem(StandardIncomingMetric, testBatchTimerID, testStoragePolicy, aggTypes, pipeline, testNumForwardedTimes, opts) + e := MustNewTimerElem(testBatchTimerID, testStoragePolicy, aggTypes, pipeline, testNumForwardedTimes, opts) for i, aligned := range alignedstartAtNanos { newTimer := raggregation.NewTimer(opts.AggregationTypesOptions().Quantiles(), opts.StreamOptions(), e.aggOpts) - timer := &lockedTimerAggregation{ - aggregation: newTimerAggregation(newTimer), - seenSources: bitset.New(10), - } + timer := &lockedTimerAggregation{aggregation: newTimerAggregation(newTimer)} timer.aggregation.AddBatch(timerBatches[i]) e.values = append(e.values, timedTimer{ startAtNanos: aligned, @@ -3004,12 +1840,9 @@ func testGaugeElem( pipeline applied.Pipeline, opts Options, ) *GaugeElem { - e := MustNewGaugeElem(StandardIncomingMetric, testGaugeID, testStoragePolicy, aggTypes, pipeline, testNumForwardedTimes, opts) + e := MustNewGaugeElem(testGaugeID, testStoragePolicy, aggTypes, pipeline, testNumForwardedTimes, opts) for i, aligned := range alignedstartAtNanos { - gauge := &lockedGaugeAggregation{ - aggregation: newGaugeAggregation(raggregation.NewGauge(e.aggOpts)), - seenSources: bitset.New(10), - } + gauge := &lockedGaugeAggregation{aggregation: newGaugeAggregation(raggregation.NewGauge(e.aggOpts))} gauge.aggregation.Update(gaugeVals[i]) e.values = append(e.values, timedGauge{ startAtNanos: aligned, diff --git a/aggregator/entry.go b/aggregator/entry.go index 5c17c50..fd94706 100644 --- a/aggregator/entry.go +++ b/aggregator/entry.go @@ -505,7 +505,7 @@ func (e *Entry) addNewAggregationKey( } // NB: The pipeline may not be owned by us and as such we need to make a copy here. key.pipeline = key.pipeline.Clone() - if err = newElem.ResetSetData(listID.incomingMetricType, metricID, key.storagePolicy, aggTypes, key.pipeline, key.numForwardedTimes); err != nil { + if err = newElem.ResetSetData(metricID, key.storagePolicy, aggTypes, key.pipeline, key.numForwardedTimes); err != nil { return nil, err } list, err := e.lists.FindOrCreate(listID) diff --git a/aggregator/entry_test.go b/aggregator/entry_test.go index f02cf47..f380744 100644 --- a/aggregator/entry_test.go +++ b/aggregator/entry_test.go @@ -1589,7 +1589,7 @@ func populateTestUntimedAggregations( require.Fail(t, fmt.Sprintf("unrecognized metric type: %v", typ)) } aggTypes := e.decompressor.MustDecompress(aggKey.aggregationID) - newElem.ResetSetData(StandardIncomingMetric, testID, aggKey.storagePolicy, aggTypes, aggKey.pipeline, 0) + newElem.ResetSetData(testID, aggKey.storagePolicy, aggTypes, aggKey.pipeline, 0) listID := standardMetricListID{ resolution: aggKey.storagePolicy.Resolution().Window, }.toMetricListID() diff --git a/aggregator/flush.go b/aggregator/flush.go index fcce4d1..f95b301 100644 --- a/aggregator/flush.go +++ b/aggregator/flush.go @@ -42,10 +42,10 @@ type flushingMetricList interface { LastFlushedNanos() int64 // Flush performs a flush for a given request. - Flush(req flushRequest, eagerForwardingMode eagerForwardingMode) + Flush(req flushRequest) // DiscardBefore discards all metrics before a given timestamp. - DiscardBefore(beforeNanos int64, eagerForwardingMode eagerForwardingMode) + DiscardBefore(beforeNanos int64) } // fixedOffsetFlushingMetricList is a flushing metric list that flushes at fixed offset @@ -70,13 +70,6 @@ type flushRequest struct { BufferAfterCutoff time.Duration } -type eagerForwardingMode int - -const ( - allowEagerForwarding eagerForwardingMode = iota - disAllowEagerForwarding -) - type flushType int const ( diff --git a/aggregator/flush_mgr.go b/aggregator/flush_mgr.go index b39c910..6ffe67f 100644 --- a/aggregator/flush_mgr.go +++ b/aggregator/flush_mgr.go @@ -278,23 +278,16 @@ func (mgr *flushManager) findOrCreateBucketWithLock(l flushingMetricList) (*flus // immediately after we have waited long enough, whereas a standard metric list has a flexible // flush offset to more evenly spread out the load during flushing. var ( - resolution time.Duration - flushOffset time.Duration flushInterval = l.FlushInterval() + flushOffset time.Duration ) - if bucketID.incomingMetricType == StandardIncomingMetric { - resolution = bucketID.standard.resolution - } else { - resolution = bucketID.forwarded.resolution - } if fl, ok := l.(fixedOffsetFlushingMetricList); ok { flushOffset = fl.FlushOffset() } else { flushOffset = mgr.computeFlushIntervalOffset(flushInterval) } scope := mgr.scope.SubScope("bucket").Tagged(map[string]string{ - "bucket-type": bucketID.incomingMetricType.String(), - "resolution": resolution.String(), + "bucket-type": bucketID.listType.String(), "interval": flushInterval.String(), }) bucket = newBucket(bucketID, flushInterval, flushOffset, scope) diff --git a/aggregator/flush_mock.go b/aggregator/flush_mock.go index 41fa297..068d3d6 100644 --- a/aggregator/flush_mock.go +++ b/aggregator/flush_mock.go @@ -90,20 +90,20 @@ func (_mr *_MockflushingMetricListRecorder) LastFlushedNanos() *gomock.Call { return _mr.mock.ctrl.RecordCall(_mr.mock, "LastFlushedNanos") } -func (_m *MockflushingMetricList) Flush(req flushRequest, eagerForwardingMode eagerForwardingMode) { - _m.ctrl.Call(_m, "Flush", req, eagerForwardingMode) +func (_m *MockflushingMetricList) Flush(req flushRequest) { + _m.ctrl.Call(_m, "Flush", req) } -func (_mr *_MockflushingMetricListRecorder) Flush(arg0, arg1 interface{}) *gomock.Call { - return _mr.mock.ctrl.RecordCall(_mr.mock, "Flush", arg0, arg1) +func (_mr *_MockflushingMetricListRecorder) Flush(arg0 interface{}) *gomock.Call { + return _mr.mock.ctrl.RecordCall(_mr.mock, "Flush", arg0) } -func (_m *MockflushingMetricList) DiscardBefore(beforeNanos int64, eagerForwardingMode eagerForwardingMode) { - _m.ctrl.Call(_m, "DiscardBefore", beforeNanos, eagerForwardingMode) +func (_m *MockflushingMetricList) DiscardBefore(beforeNanos int64) { + _m.ctrl.Call(_m, "DiscardBefore", beforeNanos) } -func (_mr *_MockflushingMetricListRecorder) DiscardBefore(arg0, arg1 interface{}) *gomock.Call { - return _mr.mock.ctrl.RecordCall(_mr.mock, "DiscardBefore", arg0, arg1) +func (_mr *_MockflushingMetricListRecorder) DiscardBefore(arg0 interface{}) *gomock.Call { + return _mr.mock.ctrl.RecordCall(_mr.mock, "DiscardBefore", arg0) } // Mock of fixedOffsetFlushingMetricList interface @@ -167,20 +167,20 @@ func (_mr *_MockfixedOffsetFlushingMetricListRecorder) LastFlushedNanos() *gomoc return _mr.mock.ctrl.RecordCall(_mr.mock, "LastFlushedNanos") } -func (_m *MockfixedOffsetFlushingMetricList) Flush(req flushRequest, eagerForwardingMode eagerForwardingMode) { - _m.ctrl.Call(_m, "Flush", req, eagerForwardingMode) +func (_m *MockfixedOffsetFlushingMetricList) Flush(req flushRequest) { + _m.ctrl.Call(_m, "Flush", req) } -func (_mr *_MockfixedOffsetFlushingMetricListRecorder) Flush(arg0, arg1 interface{}) *gomock.Call { - return _mr.mock.ctrl.RecordCall(_mr.mock, "Flush", arg0, arg1) +func (_mr *_MockfixedOffsetFlushingMetricListRecorder) Flush(arg0 interface{}) *gomock.Call { + return _mr.mock.ctrl.RecordCall(_mr.mock, "Flush", arg0) } -func (_m *MockfixedOffsetFlushingMetricList) DiscardBefore(beforeNanos int64, eagerForwardingMode eagerForwardingMode) { - _m.ctrl.Call(_m, "DiscardBefore", beforeNanos, eagerForwardingMode) +func (_m *MockfixedOffsetFlushingMetricList) DiscardBefore(beforeNanos int64) { + _m.ctrl.Call(_m, "DiscardBefore", beforeNanos) } -func (_mr *_MockfixedOffsetFlushingMetricListRecorder) DiscardBefore(arg0, arg1 interface{}) *gomock.Call { - return _mr.mock.ctrl.RecordCall(_mr.mock, "DiscardBefore", arg0, arg1) +func (_mr *_MockfixedOffsetFlushingMetricListRecorder) DiscardBefore(arg0 interface{}) *gomock.Call { + return _mr.mock.ctrl.RecordCall(_mr.mock, "DiscardBefore", arg0) } func (_m *MockfixedOffsetFlushingMetricList) FlushOffset() time.Duration { diff --git a/aggregator/follower_flush_mgr.go b/aggregator/follower_flush_mgr.go index 086e96e..d942071 100644 --- a/aggregator/follower_flush_mgr.go +++ b/aggregator/follower_flush_mgr.go @@ -289,10 +289,10 @@ func (mgr *followerFlushManager) flushersFromKVUpdateWithLock(buckets []*flushBu bucketID := bucket.bucketID flushersByInterval[i].interval = bucket.interval flushersByInterval[i].duration = bucket.duration - switch bucketID.incomingMetricType { - case StandardIncomingMetric: + switch bucketID.listType { + case standardMetricListType: flushersByInterval[i].flushers = mgr.standardFlushersFromKVUpdateWithLock(bucketID.standard, bucket.flushers) - case ForwardedIncomingMetric: + case forwardedMetricListType: flushersByInterval[i].flushers = mgr.forwardedFlushersFromKVUpdateWithLock(bucketID.forwarded, bucket.flushers) default: panic("should never get here") @@ -461,13 +461,7 @@ func (t *followerFlushTask) Run() { flusherWithTime := flusherWithTime wgWorkers.Add(1) mgr.workers.Go(func() { - // Eager forwarding is not allowed as the follower should be flushing - // aggregated metrics based on timestamps persisted in KV indicating - // what have been flushed to downstream by the leader peer and should - // not eagerly consume forwarded metrics on its own pace, or otherwise - // the follower may discard data prematurely and cause metrics loss - // during a leader re-election. - flusherWithTime.flusher.DiscardBefore(flusherWithTime.flushBeforeNanos, disAllowEagerForwarding) + flusherWithTime.flusher.DiscardBefore(flusherWithTime.flushBeforeNanos) wgWorkers.Done() }) } diff --git a/aggregator/follower_flush_mgr_test.go b/aggregator/follower_flush_mgr_test.go index 7532393..618be67 100644 --- a/aggregator/follower_flush_mgr_test.go +++ b/aggregator/follower_flush_mgr_test.go @@ -404,19 +404,15 @@ func TestFollowerFlushTaskRun(t *testing.T) { ctrl := gomock.NewController(t) defer ctrl.Finish() - var ( - flushedBefore = make([]int64, 3) - eagerForwardingModes = make([]eagerForwardingMode, 3) - ) + flushedBefore := make([]int64, 3) flushers := make([]flushingMetricList, 3) for i := 0; i < 3; i++ { i := i flusher := NewMockflushingMetricList(ctrl) flusher.EXPECT(). - DiscardBefore(gomock.Any(), disAllowEagerForwarding). - Do(func(beforeNanos int64, eagerForwardingMode eagerForwardingMode) { + DiscardBefore(gomock.Any()). + Do(func(beforeNanos int64) { flushedBefore[i] = beforeNanos - eagerForwardingModes[i] = eagerForwardingMode }) flushers[i] = flusher } @@ -454,11 +450,4 @@ func TestFollowerFlushTaskRun(t *testing.T) { } flushTask.Run() require.Equal(t, []int64{1234, 2345, 3456}, flushedBefore) - - expectedEagerForwardingModes := []eagerForwardingMode{ - disAllowEagerForwarding, - disAllowEagerForwarding, - disAllowEagerForwarding, - } - require.Equal(t, expectedEagerForwardingModes, eagerForwardingModes) } diff --git a/aggregator/forwarded_writer.go b/aggregator/forwarded_writer.go index d754911..dbf91c1 100644 --- a/aggregator/forwarded_writer.go +++ b/aggregator/forwarded_writer.go @@ -30,7 +30,6 @@ import ( "github.com/m3db/m3metrics/metric" "github.com/m3db/m3metrics/metric/aggregated" "github.com/m3db/m3metrics/metric/id" - "github.com/m3db/m3x/clock" xerrors "github.com/m3db/m3x/errors" "github.com/uber-go/tally" @@ -73,8 +72,8 @@ type forwardedMetricWriter interface { aggKey aggregationKey, ) error - // Prepare prepares the writer for a new write cycle flushing for given timestamp. - Prepare(targetNanos int64) + // Prepare prepares the writer for a new write cycle. + Prepare() // Flush flushes any data buffered in the writer. Flush() error @@ -132,12 +131,8 @@ func newForwardedWriterMetrics(scope tally.Scope) forwardedWriterMetrics { // associated with the (shard, listID) combination, which is used for value // deduplication during leadership re-elections on the destination server. type forwardedWriter struct { - shard uint32 - client client.AdminClient - eagerForwardingEnabled bool - maxForwardingWindows int - isEarlierThanFn isEarlierThanFn - nowFn clock.NowFn + shard uint32 + client client.AdminClient closed bool aggregations map[idKey]*forwardedAggregation // Aggregations for each forward metric id @@ -148,22 +143,14 @@ type forwardedWriter struct { func newForwardedWriter( shard uint32, client client.AdminClient, - eagerForwardingEnabled bool, - maxForwardingWindows int, - isEarlierThanFn isEarlierThanFn, - nowFn clock.NowFn, scope tally.Scope, ) forwardedMetricWriter { return &forwardedWriter{ - shard: shard, - client: client, - eagerForwardingEnabled: eagerForwardingEnabled, - maxForwardingWindows: maxForwardingWindows, - isEarlierThanFn: isEarlierThanFn, - nowFn: nowFn, - aggregations: make(map[idKey]*forwardedAggregation), - metrics: newForwardedWriterMetrics(scope), - aggregationMetrics: newForwardedAggregationMetrics(scope.SubScope("aggregations")), + shard: shard, + client: client, + aggregations: make(map[idKey]*forwardedAggregation), + metrics: newForwardedWriterMetrics(scope), + aggregationMetrics: newForwardedAggregationMetrics(scope.SubScope("aggregations")), } } @@ -181,10 +168,7 @@ func (w *forwardedWriter) Register( key := newIDKey(metricType, metricID) fa, exists := w.aggregations[key] if !exists { - fa = newForwardedAggregation( - metricType, metricID, w.shard, w.client, w.eagerForwardingEnabled, - w.maxForwardingWindows, w.isEarlierThanFn, w.nowFn, w.aggregationMetrics, - ) + fa = newForwardedAggregation(metricType, metricID, w.shard, w.client, w.aggregationMetrics) w.aggregations[key] = fa } fa.add(aggKey) @@ -220,9 +204,9 @@ func (w *forwardedWriter) Unregister( return nil } -func (w *forwardedWriter) Prepare(targetFlushNanos int64) { +func (w *forwardedWriter) Prepare() { for _, agg := range w.aggregations { - agg.reset(targetFlushNanos) + agg.reset() } w.metrics.prepare.Inc(1) } @@ -286,40 +270,26 @@ type forwardedAggregationWithKey struct { // is called. currRefCnt int cachedValueArrays [][]float64 - bucketsByTimeAsc forwardedAggregationBuckets - // lastWriteNanos is the last timestamp of the aggregation written to the destination server. - lastWriteNanos int64 + buckets forwardedAggregationBuckets } -// NB: Intetionally do not reset last write nanos so we can use it to figure out the last written -// timestamp and determine when we start writing forwarded metrics. func (agg *forwardedAggregationWithKey) reset() { agg.currRefCnt = 0 - for i := 0; i < len(agg.bucketsByTimeAsc); i++ { - agg.bucketsByTimeAsc[i].values = agg.bucketsByTimeAsc[i].values[:0] - agg.cachedValueArrays = append(agg.cachedValueArrays, agg.bucketsByTimeAsc[i].values) - agg.bucketsByTimeAsc[i].values = nil + for i := 0; i < len(agg.buckets); i++ { + agg.buckets[i].values = agg.buckets[i].values[:0] + agg.cachedValueArrays = append(agg.cachedValueArrays, agg.buckets[i].values) + agg.buckets[i].values = nil } - agg.bucketsByTimeAsc = agg.bucketsByTimeAsc[:0] + agg.buckets = agg.buckets[:0] } func (agg *forwardedAggregationWithKey) add(timeNanos int64, value float64) { - var idx int - for idx = 0; idx < len(agg.bucketsByTimeAsc); idx++ { - if agg.bucketsByTimeAsc[idx].timeNanos == timeNanos { - agg.bucketsByTimeAsc[idx].values = append(agg.bucketsByTimeAsc[idx].values, value) + for i := 0; i < len(agg.buckets); i++ { + if agg.buckets[i].timeNanos == timeNanos { + agg.buckets[i].values = append(agg.buckets[i].values, value) return } - if agg.bucketsByTimeAsc[idx].timeNanos > timeNanos { - break - } } - - // Bucket not found. - numBuckets := len(agg.bucketsByTimeAsc) - agg.bucketsByTimeAsc = append(agg.bucketsByTimeAsc, forwardedAggregationBucket{}) - copy(agg.bucketsByTimeAsc[idx+1:numBuckets+1], agg.bucketsByTimeAsc[idx:numBuckets]) - var values []float64 if numCachedValueArrays := len(agg.cachedValueArrays); numCachedValueArrays > 0 { values = agg.cachedValueArrays[numCachedValueArrays-1] @@ -329,10 +299,11 @@ func (agg *forwardedAggregationWithKey) add(timeNanos int64, value float64) { values = make([]float64, 0, initialValueArrayCapacity) } values = append(values, value) - agg.bucketsByTimeAsc[idx] = forwardedAggregationBucket{ + bucket := forwardedAggregationBucket{ timeNanos: timeNanos, values: values, } + agg.buckets = append(agg.buckets, bucket) } type forwardedAggregationMetrics struct { @@ -342,8 +313,6 @@ type forwardedAggregationMetrics struct { onDoneNoWrite tally.Counter onDoneWriteSuccess tally.Counter onDoneWriteErrors tally.Counter - onDoneHeartbeatSuccess tally.Counter - onDoneHeartbeatErrors tally.Counter onDoneUnexpectedRefCnt tally.Counter } @@ -355,27 +324,20 @@ func newForwardedAggregationMetrics(scope tally.Scope) *forwardedAggregationMetr onDoneNoWrite: scope.Counter("on-done-not-write"), onDoneWriteSuccess: scope.Counter("on-done-write-success"), onDoneWriteErrors: scope.Counter("on-done-write-errors"), - onDoneHeartbeatSuccess: scope.Counter("on-done-heartbeat-success"), - onDoneHeartbeatErrors: scope.Counter("on-done-heartbeat-errors"), onDoneUnexpectedRefCnt: scope.Counter("on-done-unexpected-refcnt"), } } type forwardedAggregation struct { - metricType metric.Type - metricID id.RawID - shard uint32 - client client.AdminClient - eagerForwardingEnabled bool - maxForwardingWindows int - isEarlierThanFn isEarlierThanFn - nowFn clock.NowFn - - targetFlushNanos int64 - byKey []forwardedAggregationWithKey - metrics *forwardedAggregationMetrics - writeFn writeForwardedMetricFn - onDoneFn onForwardedAggregationDoneFn + metricType metric.Type + metricID id.RawID + shard uint32 + client client.AdminClient + + byKey []forwardedAggregationWithKey + metrics *forwardedAggregationMetrics + writeFn writeForwardedMetricFn + onDoneFn onForwardedAggregationDoneFn } func newForwardedAggregation( @@ -383,23 +345,15 @@ func newForwardedAggregation( metricID id.RawID, shard uint32, client client.AdminClient, - eagerForwardingEnabled bool, - maxForwardingWindows int, - isEarlierThanFn isEarlierThanFn, - nowFn clock.NowFn, fm *forwardedAggregationMetrics, ) *forwardedAggregation { agg := &forwardedAggregation{ - metricType: metricType, - metricID: metricID, - shard: shard, - client: client, - eagerForwardingEnabled: eagerForwardingEnabled, - maxForwardingWindows: maxForwardingWindows, - isEarlierThanFn: isEarlierThanFn, - nowFn: nowFn, - byKey: make([]forwardedAggregationWithKey, 0, 2), - metrics: fm, + metricType: metricType, + metricID: metricID, + shard: shard, + client: client, + byKey: make([]forwardedAggregationWithKey, 0, 2), + metrics: fm, } agg.writeFn = agg.write agg.onDoneFn = agg.onDone @@ -416,8 +370,7 @@ func (agg *forwardedAggregation) onAggregationKeyDoneFn() onForwardedAggregation func (agg *forwardedAggregation) clear() { *agg = forwardedAggregation{} } -func (agg *forwardedAggregation) reset(targetFlushNanos int64) { - agg.targetFlushNanos = targetFlushNanos +func (agg *forwardedAggregation) reset() { for i := 0; i < len(agg.byKey); i++ { agg.byKey[i].reset() } @@ -432,10 +385,10 @@ func (agg *forwardedAggregation) add(key aggregationKey) { return } aggregation := forwardedAggregationWithKey{ - key: key, - totalRefCnt: 1, - currRefCnt: 0, - bucketsByTimeAsc: make(forwardedAggregationBuckets, 0, 2), + key: key, + totalRefCnt: 1, + currRefCnt: 0, + buckets: make(forwardedAggregationBuckets, 0, 2), } agg.byKey = append(agg.byKey, aggregation) agg.metrics.added.Inc(1) @@ -443,8 +396,6 @@ func (agg *forwardedAggregation) add(key aggregationKey) { // remove removes the aggregation key from the set of aggregations, returning // the remaining number of aggregations, and whether the removal is successful. -// NB: could unregister metric on destination server when key is removed to -// facilitate faster flushing for forwarded metric. func (agg *forwardedAggregation) remove(key aggregationKey) (int, bool) { idx := agg.index(key) if idx < 0 { @@ -478,31 +429,21 @@ func (agg *forwardedAggregation) onDone(key aggregationKey) error { agg.metrics.onDoneNoWrite.Inc(1) return nil } - if agg.byKey[idx].currRefCnt > agg.byKey[idx].totalRefCnt { - // If the current ref count is higher than total, this is likely a logical error. - agg.metrics.onDoneUnexpectedRefCnt.Inc(1) - return fmt.Errorf("unexpected refcount: current=%d, total=%d", agg.byKey[idx].currRefCnt, agg.byKey[idx].totalRefCnt) - } - var ( - multiErr = xerrors.NewMultiError() - meta = metadata.ForwardMetadata{ - AggregationID: key.aggregationID, - StoragePolicy: key.storagePolicy, - Pipeline: key.pipeline, - SourceID: agg.shard, - NumForwardedTimes: key.numForwardedTimes, - } - resolutionNanos = key.storagePolicy.Resolution().Window.Nanoseconds() - numAggregationWindows = int((agg.targetFlushNanos - agg.byKey[idx].lastWriteNanos) / resolutionNanos) - lastWriteNanos int64 - ) - if !agg.eagerForwardingEnabled || agg.byKey[idx].lastWriteNanos == 0 || numAggregationWindows > agg.maxForwardingWindows { - // If this is first time this aggregation key is flushed or the number of aggregation windows - // that we need to send empty batches for exceeds the threshold, we only send what are stored - // in the aggregation buckets to initialize lastWriteNanos and rely on the maxForwardingDelay - // on the destination server to flush the forward metric for aggregation windows where no - // batch is sent. - for _, b := range agg.byKey[idx].bucketsByTimeAsc { + if agg.byKey[idx].currRefCnt == agg.byKey[idx].totalRefCnt { + var ( + multiErr = xerrors.NewMultiError() + meta = metadata.ForwardMetadata{ + AggregationID: key.aggregationID, + StoragePolicy: key.storagePolicy, + Pipeline: key.pipeline, + SourceID: agg.shard, + NumForwardedTimes: key.numForwardedTimes, + } + ) + for _, b := range agg.byKey[idx].buckets { + if len(b.values) == 0 { + continue + } metric := aggregated.ForwardedMetric{ Type: agg.metricType, ID: agg.metricID, @@ -515,86 +456,12 @@ func (agg *forwardedAggregation) onDone(key aggregationKey) error { } else { agg.metrics.onDoneWriteSuccess.Inc(1) } - lastWriteNanos = b.timeNanos - } - } else { - // We have flushed before, so we ensure we send a batch for every aggregation - // in between so the destination server knows when it has received from all - // sources and can perform its flush as soon as possible. - var ( - currWriteNanos = agg.byKey[idx].lastWriteNanos + resolutionNanos - bucketIdx = 0 - ) - lastWriteNanos = agg.byKey[idx].lastWriteNanos - for agg.isEarlierThanFn(currWriteNanos, agg.targetFlushNanos) || bucketIdx < len(agg.byKey[idx].bucketsByTimeAsc) { - compareResult := agg.compareTimes(currWriteNanos, idx, bucketIdx) - - var metric aggregated.ForwardedMetric - if compareResult == 0 { - metric = aggregated.ForwardedMetric{ - Type: agg.metricType, - ID: agg.metricID, - TimeNanos: agg.byKey[idx].bucketsByTimeAsc[bucketIdx].timeNanos, - Values: agg.byKey[idx].bucketsByTimeAsc[bucketIdx].values, - } - currWriteNanos += resolutionNanos - bucketIdx++ - } else if compareResult < 0 { - metric = aggregated.ForwardedMetric{ - Type: agg.metricType, - ID: agg.metricID, - TimeNanos: currWriteNanos, - } - currWriteNanos += resolutionNanos - } else { - metric = aggregated.ForwardedMetric{ - Type: agg.metricType, - ID: agg.metricID, - TimeNanos: agg.byKey[idx].bucketsByTimeAsc[bucketIdx].timeNanos, - Values: agg.byKey[idx].bucketsByTimeAsc[bucketIdx].values, - } - bucketIdx++ - } - if err := agg.client.WriteForwarded(metric, meta); err != nil { - multiErr = multiErr.Add(err) - agg.metrics.onDoneWriteErrors.Inc(1) - } else { - agg.metrics.onDoneWriteSuccess.Inc(1) - } - lastWriteNanos = metric.TimeNanos } + return multiErr.FinalError() } - agg.byKey[idx].lastWriteNanos = lastWriteNanos - return multiErr.FinalError() -} - -// compare compares the current timestamp derived from the last written timestamp -// with the timestamp from the buckets populated in the current flush cycle, and -// determines which timestamp should be chosen in the current iteration. -// * If all buckets have been exhausted, return -1. -// * If the current timestamp exceeds the flush target time, return 1. -// * Otherwise -// * If the current write timestamp is the same as the bucket timestamp, return 0. -// * If the current write timestamp is earlier than the bucket timestamp, return -1. -// * If the current write timestamp is later than the bucket timestamp, return 1. -func (agg *forwardedAggregation) compareTimes( - currWriteNanos int64, - aggKeyIdx int, - bucketIdx int, -) int { - if bucketIdx >= len(agg.byKey[aggKeyIdx].bucketsByTimeAsc) { - return -1 - } - if !agg.isEarlierThanFn(currWriteNanos, agg.targetFlushNanos) { - return 1 - } - if currWriteNanos == agg.byKey[aggKeyIdx].bucketsByTimeAsc[bucketIdx].timeNanos { - return 0 - } - if currWriteNanos < agg.byKey[aggKeyIdx].bucketsByTimeAsc[bucketIdx].timeNanos { - return -1 - } - return 1 + // If the current ref count is higher than total, this is likely a logical error. + agg.metrics.onDoneUnexpectedRefCnt.Inc(1) + return fmt.Errorf("unexpected refcount: current=%d, total=%d", agg.byKey[idx].currRefCnt, agg.byKey[idx].totalRefCnt) } func (agg *forwardedAggregation) index(key aggregationKey) int { diff --git a/aggregator/forwarded_writer_test.go b/aggregator/forwarded_writer_test.go index 17dd21c..4f9c4c5 100644 --- a/aggregator/forwarded_writer_test.go +++ b/aggregator/forwarded_writer_test.go @@ -22,18 +22,19 @@ package aggregator import ( "testing" - "time" + + "github.com/m3db/m3metrics/metadata" + "github.com/m3db/m3metrics/metric/aggregated" + + "github.com/stretchr/testify/require" "github.com/m3db/m3aggregator/client" "github.com/m3db/m3metrics/aggregation" - "github.com/m3db/m3metrics/metadata" "github.com/m3db/m3metrics/metric" - "github.com/m3db/m3metrics/metric/aggregated" "github.com/m3db/m3metrics/metric/id" "github.com/m3db/m3metrics/policy" "github.com/golang/mock/gomock" - "github.com/stretchr/testify/require" "github.com/uber-go/tally" ) @@ -51,7 +52,7 @@ func TestForwardedWriterRegisterWriterClosed(t *testing.T) { var ( c = client.NewMockAdminClient(ctrl) - w = newForwardedWriter(0, c, true, 10, isStandardMetricEarlierThan, time.Now, tally.NoopScope) + w = newForwardedWriter(0, c, tally.NoopScope) mt = metric.CounterType mid = id.RawID("foo") aggKey = testForwardedWriterAggregationKey @@ -68,7 +69,7 @@ func TestForwardedWriterRegisterNewAggregation(t *testing.T) { var ( c = client.NewMockAdminClient(ctrl) - w = newForwardedWriter(0, c, true, 10, isStandardMetricEarlierThan, time.Now, tally.NoopScope) + w = newForwardedWriter(0, c, tally.NoopScope) mt = metric.GaugeType mid = id.RawID("foo") aggKey = testForwardedWriterAggregationKey @@ -95,23 +96,23 @@ func TestForwardedWriterRegisterNewAggregation(t *testing.T) { require.Equal(t, 1, len(agg.byKey)) require.Equal(t, 1, agg.byKey[0].totalRefCnt) require.True(t, aggKey.Equal(agg.byKey[0].key)) - require.Equal(t, 0, len(agg.byKey[0].bucketsByTimeAsc)) + require.Equal(t, 0, len(agg.byKey[0].buckets)) // Validate that writeFn can be used to write data to the aggregation. writeFn(aggKey, 1234, 5.67) - require.Equal(t, 1, len(agg.byKey[0].bucketsByTimeAsc)) - require.Equal(t, int64(1234), agg.byKey[0].bucketsByTimeAsc[0].timeNanos) - require.Equal(t, []float64{5.67}, agg.byKey[0].bucketsByTimeAsc[0].values) + require.Equal(t, 1, len(agg.byKey[0].buckets)) + require.Equal(t, int64(1234), agg.byKey[0].buckets[0].timeNanos) + require.Equal(t, []float64{5.67}, agg.byKey[0].buckets[0].values) writeFn(aggKey, 1234, 1.78) - require.Equal(t, 1, len(agg.byKey[0].bucketsByTimeAsc)) - require.Equal(t, int64(1234), agg.byKey[0].bucketsByTimeAsc[0].timeNanos) - require.Equal(t, []float64{5.67, 1.78}, agg.byKey[0].bucketsByTimeAsc[0].values) + require.Equal(t, 1, len(agg.byKey[0].buckets)) + require.Equal(t, int64(1234), agg.byKey[0].buckets[0].timeNanos) + require.Equal(t, []float64{5.67, 1.78}, agg.byKey[0].buckets[0].values) writeFn(aggKey, 1240, -2.95) - require.Equal(t, 2, len(agg.byKey[0].bucketsByTimeAsc)) - require.Equal(t, int64(1240), agg.byKey[0].bucketsByTimeAsc[1].timeNanos) - require.Equal(t, []float64{-2.95}, agg.byKey[0].bucketsByTimeAsc[1].values) + require.Equal(t, 2, len(agg.byKey[0].buckets)) + require.Equal(t, int64(1240), agg.byKey[0].buckets[1].timeNanos) + require.Equal(t, []float64{-2.95}, agg.byKey[0].buckets[1].values) // Validate that onDoneFn can be used to flush data out. expectedMetric1 := aggregated.ForwardedMetric{ @@ -144,7 +145,7 @@ func TestForwardedWriterRegisterExistingAggregation(t *testing.T) { var ( c = client.NewMockAdminClient(ctrl) - w = newForwardedWriter(0, c, true, 10, isStandardMetricEarlierThan, time.Now, tally.NoopScope) + w = newForwardedWriter(0, c, tally.NoopScope) mt = metric.GaugeType mid = id.RawID("foo") aggKey = testForwardedWriterAggregationKey @@ -180,7 +181,7 @@ func TestForwardedWriterUnregisterWriterClosed(t *testing.T) { var ( c = client.NewMockAdminClient(ctrl) - w = newForwardedWriter(0, c, true, 10, isStandardMetricEarlierThan, time.Now, tally.NoopScope) + w = newForwardedWriter(0, c, tally.NoopScope) mt = metric.GaugeType mid = id.RawID("foo") aggKey = testForwardedWriterAggregationKey @@ -196,7 +197,7 @@ func TestForwardedWriterUnregisterMetricNotFound(t *testing.T) { var ( c = client.NewMockAdminClient(ctrl) - w = newForwardedWriter(0, c, true, 10, isStandardMetricEarlierThan, time.Now, tally.NoopScope) + w = newForwardedWriter(0, c, tally.NoopScope) mt = metric.GaugeType mid = id.RawID("foo") aggKey = testForwardedWriterAggregationKey @@ -211,7 +212,7 @@ func TestForwardedWriterUnregisterAggregationKeyNotFound(t *testing.T) { var ( c = client.NewMockAdminClient(ctrl) - w = newForwardedWriter(0, c, true, 10, isStandardMetricEarlierThan, time.Now, tally.NoopScope) + w = newForwardedWriter(0, c, tally.NoopScope) mt = metric.GaugeType mid = id.RawID("foo") aggKey = testForwardedWriterAggregationKey @@ -233,7 +234,7 @@ func TestForwardedWriterUnregister(t *testing.T) { var ( c = client.NewMockAdminClient(ctrl) - w = newForwardedWriter(0, c, true, 10, isStandardMetricEarlierThan, time.Now, tally.NoopScope) + w = newForwardedWriter(0, c, tally.NoopScope) mt = metric.GaugeType mid = id.RawID("foo") aggKey = testForwardedWriterAggregationKey @@ -265,7 +266,7 @@ func TestForwardedWriterPrepare(t *testing.T) { var ( c = client.NewMockAdminClient(ctrl) - w = newForwardedWriter(0, c, true, 10, isStandardMetricEarlierThan, time.Now, tally.NoopScope) + w = newForwardedWriter(0, c, tally.NoopScope) mt = metric.GaugeType mid = id.RawID("foo") mid2 = id.RawID("bar") @@ -331,38 +332,32 @@ func TestForwardedWriterPrepare(t *testing.T) { agg, exists := fw.aggregations[newIDKey(mt, mid)] require.True(t, exists) require.Equal(t, 1, len(agg.byKey)) - require.Equal(t, 2, len(agg.byKey[0].bucketsByTimeAsc)) + require.Equal(t, 2, len(agg.byKey[0].buckets)) require.Equal(t, 1, agg.byKey[0].currRefCnt) require.Equal(t, 0, len(agg.byKey[0].cachedValueArrays)) - require.Equal(t, int64(0), agg.targetFlushNanos) - require.Equal(t, int64(1240), agg.byKey[0].lastWriteNanos) agg, exists = fw.aggregations[newIDKey(mt, mid2)] require.True(t, exists) require.Equal(t, 1, len(agg.byKey)) - require.Equal(t, 2, len(agg.byKey[0].bucketsByTimeAsc)) + require.Equal(t, 2, len(agg.byKey[0].buckets)) require.Equal(t, 1, agg.byKey[0].currRefCnt) require.Equal(t, 0, len(agg.byKey[0].cachedValueArrays)) - require.Equal(t, int64(0), agg.targetFlushNanos) - require.Equal(t, int64(1239), agg.byKey[0].lastWriteNanos) - w.Prepare(1234) + w.Prepare() // Assert the internal state has been reset. require.Equal(t, 2, len(fw.aggregations)) agg, exists = fw.aggregations[newIDKey(mt, mid)] require.True(t, exists) require.Equal(t, 1, len(agg.byKey)) - require.Equal(t, 0, len(agg.byKey[0].bucketsByTimeAsc)) + require.Equal(t, 0, len(agg.byKey[0].buckets)) require.Equal(t, 0, agg.byKey[0].currRefCnt) require.Equal(t, 2, len(agg.byKey[0].cachedValueArrays)) - require.Equal(t, int64(1234), agg.targetFlushNanos) agg, exists = fw.aggregations[newIDKey(mt, mid2)] require.True(t, exists) require.Equal(t, 1, len(agg.byKey)) - require.Equal(t, 0, len(agg.byKey[0].bucketsByTimeAsc)) + require.Equal(t, 0, len(agg.byKey[0].buckets)) require.Equal(t, 0, agg.byKey[0].currRefCnt) require.Equal(t, 2, len(agg.byKey[0].cachedValueArrays)) - require.Equal(t, int64(1234), agg.targetFlushNanos) // Write datapoints again. writeFn(aggKey, 1234, 3.4) @@ -377,138 +372,15 @@ func TestForwardedWriterPrepare(t *testing.T) { agg, exists = fw.aggregations[newIDKey(mt, mid)] require.True(t, exists) require.Equal(t, 1, len(agg.byKey)) - require.Equal(t, 2, len(agg.byKey[0].bucketsByTimeAsc)) + require.Equal(t, 2, len(agg.byKey[0].buckets)) require.Equal(t, 1, agg.byKey[0].currRefCnt) require.Equal(t, 0, len(agg.byKey[0].cachedValueArrays)) - require.Equal(t, int64(1240), agg.byKey[0].lastWriteNanos) agg, exists = fw.aggregations[newIDKey(mt, mid2)] require.True(t, exists) require.Equal(t, 1, len(agg.byKey)) - require.Equal(t, 2, len(agg.byKey[0].bucketsByTimeAsc)) + require.Equal(t, 2, len(agg.byKey[0].buckets)) require.Equal(t, 1, agg.byKey[0].currRefCnt) require.Equal(t, 0, len(agg.byKey[0].cachedValueArrays)) - require.Equal(t, int64(1239), agg.byKey[0].lastWriteNanos) -} - -func TestForwardedWriterOnDoneWithEagerForwardingEnabled(t *testing.T) { - ctrl := gomock.NewController(t) - defer ctrl.Finish() - - var ( - now = time.Unix(1234, 0) - nowFn = func() time.Time { return now } - c = client.NewMockAdminClient(ctrl) - w = newForwardedWriter(0, c, true, 10, isStandardMetricEarlierThan, nowFn, tally.NoopScope) - mt = metric.GaugeType - mid = id.RawID("foo") - mid2 = id.RawID("bar") - aggKey = testForwardedWriterAggregationKey - ) - - // Register an aggregation. - writeFn, onDoneFn, err := w.Register(mt, mid, aggKey) - require.NoError(t, err) - - // Write some datapoints. - writeFn(aggKey, 1234*time.Second.Nanoseconds(), 3.4) - writeFn(aggKey, 1234*time.Second.Nanoseconds(), 3.5) - writeFn(aggKey, 1240*time.Second.Nanoseconds(), 98.2) - - // Register another aggregation. - writeFn2, onDoneFn2, err := w.Register(mt, mid2, aggKey) - require.NoError(t, err) - - // Write some more datapoints. - writeFn2(aggKey, 1238*time.Second.Nanoseconds(), 3.4) - writeFn2(aggKey, 1240*time.Second.Nanoseconds(), 3.5) - - expectedMetric1 := aggregated.ForwardedMetric{ - Type: mt, - ID: mid, - TimeNanos: 1234 * time.Second.Nanoseconds(), - Values: []float64{3.4, 3.5}, - } - expectedMetric2 := aggregated.ForwardedMetric{ - Type: mt, - ID: mid, - TimeNanos: 1240 * time.Second.Nanoseconds(), - Values: []float64{98.2}, - } - expectedMetric3 := aggregated.ForwardedMetric{ - Type: mt, - ID: mid2, - TimeNanos: 1238 * time.Second.Nanoseconds(), - Values: []float64{3.4}, - } - expectedMetric4 := aggregated.ForwardedMetric{ - Type: mt, - ID: mid2, - TimeNanos: 1240 * time.Second.Nanoseconds(), - Values: []float64{3.5}, - } - expectedMeta := metadata.ForwardMetadata{ - AggregationID: aggregation.MustCompressTypes(aggregation.Count), - StoragePolicy: policy.MustParseStoragePolicy("10s:2d"), - SourceID: 0, - NumForwardedTimes: 1, - } - c.EXPECT().WriteForwarded(expectedMetric1, expectedMeta).Return(nil).Times(1) - c.EXPECT().WriteForwarded(expectedMetric2, expectedMeta).Return(nil).Times(1) - c.EXPECT().WriteForwarded(expectedMetric3, expectedMeta).Return(nil).Times(1) - c.EXPECT().WriteForwarded(expectedMetric4, expectedMeta).Return(nil).Times(1) - require.NoError(t, onDoneFn(aggKey)) - require.NoError(t, onDoneFn2(aggKey)) - - expectedMetrics := []aggregated.ForwardedMetric{ - { - Type: mt, - ID: mid, - TimeNanos: 1240 * time.Second.Nanoseconds(), - Values: []float64{12.4, 18.5}, - }, - { - Type: mt, - ID: mid, - TimeNanos: 1250 * time.Second.Nanoseconds(), - }, - { - Type: mt, - ID: mid, - TimeNanos: 1260 * time.Second.Nanoseconds(), - Values: []float64{78.2}, - }, - { - Type: mt, - ID: mid2, - TimeNanos: 1250 * time.Second.Nanoseconds(), - Values: []float64{9.4}, - }, - { - Type: mt, - ID: mid2, - TimeNanos: 1260 * time.Second.Nanoseconds(), - }, - { - Type: mt, - ID: mid2, - TimeNanos: 1280 * time.Second.Nanoseconds(), - Values: []float64{-12.5}, - }, - } - for _, expectedMetric := range expectedMetrics { - c.EXPECT().WriteForwarded(expectedMetric, expectedMeta).Return(nil).Times(1) - } - - // Prepare for another write. - w.Prepare(1263 * time.Second.Nanoseconds()) - - writeFn(aggKey, 1240*time.Second.Nanoseconds(), 12.4) - writeFn(aggKey, 1240*time.Second.Nanoseconds(), 18.5) - writeFn(aggKey, 1260*time.Second.Nanoseconds(), 78.2) - writeFn2(aggKey, 1250*time.Second.Nanoseconds(), 9.4) - writeFn2(aggKey, 1280*time.Second.Nanoseconds(), -12.5) - require.NoError(t, onDoneFn(aggKey)) - require.NoError(t, onDoneFn2(aggKey)) } func TestForwardedWriterCloseWriterClosed(t *testing.T) { @@ -517,7 +389,7 @@ func TestForwardedWriterCloseWriterClosed(t *testing.T) { var ( c = client.NewMockAdminClient(ctrl) - w = newForwardedWriter(0, c, true, 10, isStandardMetricEarlierThan, time.Now, tally.NoopScope) + w = newForwardedWriter(0, c, tally.NoopScope) ) // Close the writer and validate that the fields are nil'ed out. diff --git a/aggregator/forwarding_latency_histogram.go b/aggregator/forwarding_latency_histogram.go deleted file mode 100644 index 59b741f..0000000 --- a/aggregator/forwarding_latency_histogram.go +++ /dev/null @@ -1,102 +0,0 @@ -// Copyright (c) 2018 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - -package aggregator - -import ( - "strconv" - "sync" - "time" - - "github.com/uber-go/tally" -) - -const ( - forwardingLatencyBucketVersion = 5 - numForwardingLatencyBuckets = 40 -) - -// ForwardingLatencyBucketsFn creates forwarding latency buckets. -type ForwardingLatencyBucketsFn func( - key ForwardingLatencyBucketKey, - numLatencyBuckets int, -) tally.Buckets - -// ForwardingLatencyBucketKey is the key for a forwarding latency bucket. -type ForwardingLatencyBucketKey struct { - Resolution time.Duration - NumForwardedTimes int -} - -// ForwardingLatencyHistograms record forwarded latencies as histograms. -type ForwardingLatencyHistograms struct { - sync.RWMutex - - scope tally.Scope - bucketsFn ForwardingLatencyBucketsFn - histograms map[ForwardingLatencyBucketKey]tally.Histogram -} - -// NewForwardingLatencyHistograms create a new set of forwarding latency histograms. -func NewForwardingLatencyHistograms( - scope tally.Scope, - bucketsFn ForwardingLatencyBucketsFn, -) *ForwardingLatencyHistograms { - return &ForwardingLatencyHistograms{ - scope: scope, - bucketsFn: bucketsFn, - histograms: make(map[ForwardingLatencyBucketKey]tally.Histogram), - } -} - -// RecordDuration record a forwarding latency. -func (m *ForwardingLatencyHistograms) RecordDuration( - resolution time.Duration, - numForwardedTimes int, - duration time.Duration, -) { - key := ForwardingLatencyBucketKey{ - Resolution: resolution, - NumForwardedTimes: numForwardedTimes, - } - m.RLock() - histogram, exists := m.histograms[key] - m.RUnlock() - if exists { - histogram.RecordDuration(duration) - return - } - m.Lock() - histogram, exists = m.histograms[key] - if exists { - m.Unlock() - histogram.RecordDuration(duration) - return - } - buckets := m.bucketsFn(key, numForwardingLatencyBuckets) - histogram = m.scope.Tagged(map[string]string{ - "bucket-version": strconv.Itoa(forwardingLatencyBucketVersion), - "resolution": resolution.String(), - "num-forwarded-times": strconv.Itoa(numForwardedTimes), - }).Histogram("forwarding-latency", buckets) - m.histograms[key] = histogram - m.Unlock() - histogram.RecordDuration(duration) -} diff --git a/aggregator/forwarding_latency_histogram_test.go b/aggregator/forwarding_latency_histogram_test.go deleted file mode 100644 index cae95af..0000000 --- a/aggregator/forwarding_latency_histogram_test.go +++ /dev/null @@ -1,85 +0,0 @@ -// Copyright (c) 2018 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - -package aggregator - -import ( - "testing" - "time" - - "github.com/stretchr/testify/require" - "github.com/uber-go/tally" -) - -func TestForwardingLatencyHistogramsRecordDuration(t *testing.T) { - s := tally.NewTestScope("testScope", nil) - bucketsFn := func(key ForwardingLatencyBucketKey, numLatencyBuckets int) tally.Buckets { - bucketWidth := 2 * key.Resolution / time.Duration(numLatencyBuckets) - return tally.MustMakeLinearDurationBuckets(0, bucketWidth, numLatencyBuckets) - } - histograms := NewForwardingLatencyHistograms(s, bucketsFn) - histograms.RecordDuration(10*time.Second, 1, 2*time.Second) - histograms.RecordDuration(10*time.Second, 2, 2*time.Second) - histograms.RecordDuration(time.Minute, 2, 20*time.Second) - histograms.RecordDuration(10*time.Second, 1, 5*time.Second) - histograms.RecordDuration(10*time.Second, 1, 2*time.Second) - snapshot := s.Snapshot() - histogramSnapshots := snapshot.Histograms() - - require.Equal(t, 3, len(histogramSnapshots)) - for _, input := range []struct { - id string - expectedName string - expectedDurations map[time.Duration]int64 - }{ - { - id: "testScope.forwarding-latency+bucket-version=5,num-forwarded-times=1,resolution=10s", - expectedName: "testScope.forwarding-latency", - expectedDurations: map[time.Duration]int64{ - 2 * time.Second: 2, - 5 * time.Second: 1, - }, - }, - { - id: "testScope.forwarding-latency+bucket-version=5,num-forwarded-times=2,resolution=10s", - expectedName: "testScope.forwarding-latency", - expectedDurations: map[time.Duration]int64{ - 2 * time.Second: 1, - }, - }, - { - id: "testScope.forwarding-latency+bucket-version=5,num-forwarded-times=2,resolution=1m0s", - expectedName: "testScope.forwarding-latency", - expectedDurations: map[time.Duration]int64{ - 21 * time.Second: 1, - }, - }, - } { - h, exists := histogramSnapshots[input.id] - require.True(t, exists) - require.Equal(t, input.expectedName, h.Name()) - actualDurations := h.Durations() - for k, v := range input.expectedDurations { - actual, exists := actualDurations[k] - require.True(t, exists) - require.Equal(t, v, actual) - } - } -} diff --git a/aggregator/gauge_elem.gen.go b/aggregator/gauge_elem.gen.go index 7f2719f..63151e9 100644 --- a/aggregator/gauge_elem.gen.go +++ b/aggregator/gauge_elem.gen.go @@ -51,39 +51,9 @@ import ( type lockedGaugeAggregation struct { sync.Mutex - closed bool - - // sourcesReady is only used for elements receiving forwarded metrics. - // It determines whether the current aggregation can use its source set - // to determine whether it has received data from all forwarding sources - // to perform eager forwarding if enabled. - sourcesReady bool - - // expectedSources is only used for elements receiving forwarded metrics. - // It keeps track of all the sources the current aggregation expect to receive - // data from. - expectedSources *bitset.BitSet - - // seenSources keeps track of all the sources the current aggregation has - // seen so far. - seenSources *bitset.BitSet - - // consumeState is only used for elements receiving forwarded metrics. It - // describes whether the current aggregation is ready to be consumed or has - // been consumed. This in turn determines whether the aggregation can be - // eagerly consumed, or should be skipped during consumption. - consumeState consumeState - aggregation gaugeAggregation -} - -func (lockedAgg *lockedGaugeAggregation) close() { - if lockedAgg.closed { - return - } - lockedAgg.closed = true - lockedAgg.expectedSources = nil - lockedAgg.seenSources = nil - lockedAgg.aggregation.Close() + closed bool + sourcesSeen *bitset.BitSet + aggregation gaugeAggregation } type timedGauge struct { @@ -109,7 +79,6 @@ type GaugeElem struct { // NewGaugeElem creates a new element for the given metric type. func NewGaugeElem( - incomingMetricType IncomingMetricType, id id.RawID, sp policy.StoragePolicy, aggTypes maggregation.Types, @@ -121,7 +90,7 @@ func NewGaugeElem( elemBase: newElemBase(opts), values: make([]timedGauge, 0, defaultNumAggregations), // in most cases values will have two entries } - if err := e.ResetSetData(incomingMetricType, id, sp, aggTypes, pipeline, numForwardedTimes); err != nil { + if err := e.ResetSetData(id, sp, aggTypes, pipeline, numForwardedTimes); err != nil { return nil, err } return e, nil @@ -129,7 +98,6 @@ func NewGaugeElem( // MustNewGaugeElem creates a new element, or panics if the input is invalid. func MustNewGaugeElem( - incomingMetricType IncomingMetricType, id id.RawID, sp policy.StoragePolicy, aggTypes maggregation.Types, @@ -137,7 +105,7 @@ func MustNewGaugeElem( numForwardedTimes int, opts Options, ) *GaugeElem { - elem, err := NewGaugeElem(incomingMetricType, id, sp, aggTypes, pipeline, numForwardedTimes, opts) + elem, err := NewGaugeElem(id, sp, aggTypes, pipeline, numForwardedTimes, opts) if err != nil { panic(fmt.Errorf("unable to create element: %v", err)) } @@ -146,7 +114,6 @@ func MustNewGaugeElem( // ResetSetData resets the element and sets data. func (e *GaugeElem) ResetSetData( - incomingMetricType IncomingMetricType, id id.RawID, sp policy.StoragePolicy, aggTypes maggregation.Types, @@ -157,7 +124,7 @@ func (e *GaugeElem) ResetSetData( if useDefaultAggregation { aggTypes = e.DefaultAggregationTypes(e.aggTypesOpts) } - if err := e.elemBase.resetSetData(incomingMetricType, id, sp, aggTypes, useDefaultAggregation, pipeline, numForwardedTimes); err != nil { + if err := e.elemBase.resetSetData(id, sp, aggTypes, useDefaultAggregation, pipeline, numForwardedTimes); err != nil { return err } if err := e.gaugeElemBase.ResetSetData(e.aggTypesOpts, aggTypes, useDefaultAggregation); err != nil { @@ -182,7 +149,7 @@ func (e *GaugeElem) ResetSetData( // AddUnion adds a metric value union at a given timestamp. func (e *GaugeElem) AddUnion(timestamp time.Time, mu unaggregated.MetricUnion) error { alignedStart := timestamp.Truncate(e.sp.Resolution().Window).UnixNano() - lockedAgg, err := e.findOrCreate(alignedStart, sourcesOptions{}) + lockedAgg, err := e.findOrCreate(alignedStart, createAggregationOptions{}) if err != nil { return err } @@ -201,7 +168,7 @@ func (e *GaugeElem) AddUnion(timestamp time.Time, mu unaggregated.MetricUnion) e // same aggregation, the incoming value is discarded. func (e *GaugeElem) AddUnique(timestamp time.Time, values []float64, sourceID uint32) error { alignedStart := timestamp.Truncate(e.sp.Resolution().Window).UnixNano() - lockedAgg, err := e.findOrCreate(alignedStart, sourcesOptions{updateSources: true, source: sourceID}) + lockedAgg, err := e.findOrCreate(alignedStart, createAggregationOptions{initSourceSet: true}) if err != nil { return err } @@ -211,26 +178,11 @@ func (e *GaugeElem) AddUnique(timestamp time.Time, values []float64, sourceID ui return errAggregationClosed } source := uint(sourceID) - if lockedAgg.seenSources.Test(source) { + if lockedAgg.sourcesSeen.Test(source) { lockedAgg.Unlock() return errDuplicateForwardingSource } - lockedAgg.seenSources.Set(source) - if lockedAgg.sourcesReady { - // If the sources are ready, the expected sources will be a pre-filled - // bitset populated with sources the aggregation is expected to see data from. - // As such, we need to clear the source bit in the expected sources. - if lockedAgg.expectedSources.Test(source) { - // This source is never seen before and is in the expected source list, - // as a result, we need to clear the source bit. - lockedAgg.expectedSources.Clear(source) - if lockedAgg.expectedSources.None() { - lockedAgg.consumeState = readyToConsume - } - } - // New sources that are not in the expected source list are still allowed - // to go through. - } + lockedAgg.sourcesSeen.Set(source) for _, v := range values { lockedAgg.aggregation.Add(v) } @@ -245,7 +197,6 @@ func (e *GaugeElem) AddUnique(timestamp time.Time, values []float64, sourceID ui // to avoid race conditions. func (e *GaugeElem) Consume( targetNanos int64, - eagerForwardingMode eagerForwardingMode, isEarlierThanFn isEarlierThanFn, timestampNanosFn timestampNanosFn, flushLocalFn flushLocalMetricFn, @@ -261,8 +212,7 @@ func (e *GaugeElem) Consume( idx := 0 for range e.values { // Bail as soon as the timestamp is no later than the target time. - timeNanos := timestampNanosFn(e.values[idx].startAtNanos, resolution) - if !isEarlierThanFn(timeNanos, targetNanos) { + if !isEarlierThanFn(e.values[idx].startAtNanos, resolution, targetNanos) { break } idx++ @@ -280,71 +230,31 @@ func (e *GaugeElem) Consume( e.values = e.values[:n] } canCollect := len(e.values) == 0 && e.tombstoned - - // Additionally for elements receiving forwarded metrics and sending aggregated metrics - // to local backends, we also check if any aggregations are ready to be consumed. We however - // do not remove the aggregations as we do for aggregations whose timestamps are old enough, - // since for aggregations receiving forwarded metrics that are marked "consume ready", it is - // possible that metrics still go to the such aggregation bucket after they are marked "consume - // ready" due to delayed source re-delivery or new sources showing up, and removing such - // aggregation prematurely would mean the values from the delayed sources and/or new sources - // would be considered as the aggregated value for such aggregation bucket, which is incorrect. - // By keeping such aggregation buckets and only removing them when they are considered old enough - // (i.e., when their timestmaps are earlier than the target timestamp), we ensure no metrics may - // go to such aggregation buckets after they are consumed and therefore avoid the aformentioned - // problem. - aggregationIdxToCloseUntil := len(e.toConsume) - if e.incomingMetricType == ForwardedIncomingMetric && e.isSourcesSetReadyWithElemLock() { - e.maybeRefreshSourcesSetWithElemLock() - // We only attempt to consume if the outgoing metrics type is local instead of forwarded - // and eager forwarding is allowed (eager forwarding may be enabled and disallowed when - // the node is a follower and allowed when the node is a leader. This is because forwarded - // metrics are sent in batches and can only be sent when all sources in the same shard have - // been consumed, and as such is not well suited for pre-emptive consumption. - if e.outgoingMetricType() == localOutgoingMetric && eagerForwardingMode == allowEagerForwarding { - for i := 0; i < len(e.values); i++ { - // NB: This makes the logic easier to understand but it would be more efficient to use - // an atomic here to avoid locking aggregations. - e.values[i].lockedAgg.Lock() - if e.values[i].lockedAgg.consumeState == readyToConsume { - e.toConsume = append(e.toConsume, e.values[i]) - e.values[i].lockedAgg.consumeState = consuming - } - e.values[i].lockedAgg.Unlock() - } - } - } e.Unlock() // Process the aggregations that are ready for consumption. for i := range e.toConsume { timeNanos := timestampNanosFn(e.toConsume[i].startAtNanos, resolution) e.toConsume[i].lockedAgg.Lock() - if e.toConsume[i].lockedAgg.consumeState != consumed { - e.processValueWithAggregationLock( - timeNanos, eagerForwardingMode, e.toConsume[i].lockedAgg, - flushLocalFn, flushForwardedFn, - ) - } - e.toConsume[i].lockedAgg.consumeState = consumed - if i < aggregationIdxToCloseUntil { - if e.toConsume[i].lockedAgg.seenSources != nil { - e.sourcesLock.Lock() - // This is to make sure there aren't too many cached source sets taking up - // too much space. - if len(e.cachedSourceSets) < e.opts.MaxNumCachedSourceSets() { - e.cachedSourceSets = append(e.cachedSourceSets, e.toConsume[i].lockedAgg.seenSources) - e.toConsume[i].lockedAgg.seenSources = nil - } - e.sourcesLock.Unlock() + e.processValueWithAggregationLock(timeNanos, e.toConsume[i].lockedAgg, flushLocalFn, flushForwardedFn) + // Closes the aggregation object after it's processed. + e.toConsume[i].lockedAgg.closed = true + e.toConsume[i].lockedAgg.aggregation.Close() + if e.toConsume[i].lockedAgg.sourcesSeen != nil { + e.cachedSourceSetsLock.Lock() + // This is to make sure there aren't too many cached source sets taking up + // too much space. + if len(e.cachedSourceSets) < e.opts.MaxNumCachedSourceSets() { + e.cachedSourceSets = append(e.cachedSourceSets, e.toConsume[i].lockedAgg.sourcesSeen) } - e.toConsume[i].lockedAgg.close() + e.cachedSourceSetsLock.Unlock() + e.toConsume[i].lockedAgg.sourcesSeen = nil } e.toConsume[i].lockedAgg.Unlock() e.toConsume[i].Reset() } - if e.outgoingMetricType() == forwardedOutgoingMetric { + if e.parsedPipeline.HasRollup { forwardedAggregationKey, _ := e.ForwardedAggregationKey() onForwardedFlushedFn(e.onForwardedAggregationWrittenFn, forwardedAggregationKey) } @@ -364,15 +274,14 @@ func (e *GaugeElem) Close() { e.parsedPipeline = parsedPipeline{} e.writeForwardedMetricFn = nil e.onForwardedAggregationWrittenFn = nil - e.sourcesHeartbeat = nil - e.sourcesSet = nil for idx := range e.cachedSourceSets { e.cachedSourceSets[idx] = nil } e.cachedSourceSets = nil for idx := range e.values { // Close the underlying aggregation objects. - e.values[idx].lockedAgg.close() + e.values[idx].lockedAgg.sourcesSeen = nil + e.values[idx].lockedAgg.aggregation.Close() e.values[idx].Reset() } e.values = e.values[:0] @@ -393,16 +302,13 @@ func (e *GaugeElem) Close() { // if it doesn't exist. func (e *GaugeElem) findOrCreate( alignedStart int64, - sourcesOpts sourcesOptions, + createOpts createAggregationOptions, ) (*lockedGaugeAggregation, error) { e.RLock() if e.closed { e.RUnlock() return nil, errElemClosed } - if sourcesOpts.updateSources { - e.updateSources(sourcesOpts.source) - } idx, found := e.indexOfWithLock(alignedStart) if found { agg := e.values[idx].lockedAgg @@ -428,36 +334,24 @@ func (e *GaugeElem) findOrCreate( e.values = append(e.values, timedGauge{}) copy(e.values[idx+1:numValues+1], e.values[idx:numValues]) - var ( - sourcesReady = e.isSourcesSetReadyWithElemLock() - expectedSources *bitset.BitSet - seenSources *bitset.BitSet - ) - if sourcesOpts.updateSources { - e.sourcesLock.Lock() - // If the sources set is ready, we clone it ane use the clone to - // determine when we have received from all the expected sources. - if sourcesReady { - expectedSources = e.sourcesSet.Clone() - } + var sourcesSeen *bitset.BitSet + if createOpts.initSourceSet { + e.cachedSourceSetsLock.Lock() if numCachedSourceSets := len(e.cachedSourceSets); numCachedSourceSets > 0 { - seenSources = e.cachedSourceSets[numCachedSourceSets-1] + sourcesSeen = e.cachedSourceSets[numCachedSourceSets-1] e.cachedSourceSets[numCachedSourceSets-1] = nil e.cachedSourceSets = e.cachedSourceSets[:numCachedSourceSets-1] - seenSources.ClearAll() + sourcesSeen.ClearAll() } else { - seenSources = bitset.New(defaultNumSources) + sourcesSeen = bitset.New(defaultNumSources) } - e.sourcesLock.Unlock() + e.cachedSourceSetsLock.Unlock() } - e.values[idx] = timedGauge{ startAtNanos: alignedStart, lockedAgg: &lockedGaugeAggregation{ - sourcesReady: sourcesReady, - expectedSources: expectedSources, - seenSources: seenSources, - aggregation: e.NewAggregation(e.opts, e.aggOpts), + sourcesSeen: sourcesSeen, + aggregation: e.NewAggregation(e.opts, e.aggOpts), }, } agg := e.values[idx].lockedAgg @@ -496,14 +390,10 @@ func (e *GaugeElem) indexOfWithLock(alignedStart int64) (int, bool) { func (e *GaugeElem) processValueWithAggregationLock( timeNanos int64, - eagerForwardingMode eagerForwardingMode, lockedAgg *lockedGaugeAggregation, flushLocalFn flushLocalMetricFn, flushForwardedFn flushForwardedMetricFn, ) { - if lockedAgg.aggregation.Count() == 0 { - return - } var ( fullPrefix = e.FullPrefix(e.opts) transformations = e.parsedPipeline.Transformations @@ -533,7 +423,7 @@ func (e *GaugeElem) processValueWithAggregationLock( if discardNaNValues && math.IsNaN(value) { continue } - if e.outgoingMetricType() == localOutgoingMetric { + if !e.parsedPipeline.HasRollup { flushLocalFn(fullPrefix, e.id, e.TypeStringFor(e.aggTypesOpts, aggType), timeNanos, value, e.sp) } else { forwardedAggregationKey, _ := e.ForwardedAggregationKey() @@ -541,73 +431,4 @@ func (e *GaugeElem) processValueWithAggregationLock( } } e.lastConsumedAtNanos = timeNanos - - // Emit latency metrics for forwarded metrics going to local backends - // when eager forwarding is allowed. - if eagerForwardingMode == allowEagerForwarding && - e.incomingMetricType == ForwardedIncomingMetric && - e.outgoingMetricType() == localOutgoingMetric { - e.opts.FullForwardingLatencyHistograms().RecordDuration( - e.sp.Resolution().Window, - e.numForwardedTimes, - time.Duration(e.nowFn().UnixNano()-timeNanos), - ) - } -} - -func (e *GaugeElem) outgoingMetricType() outgoingMetricType { - if !e.parsedPipeline.HasRollup { - return localOutgoingMetric - } - return forwardedOutgoingMetric -} - -func (e *GaugeElem) isSourcesSetReadyWithElemLock() bool { - if !e.opts.EnableEagerForwarding() { - return false - } - if e.buildingSourcesAtNanos == 0 { - return false - } - // NB: Allow TTL for the source set to build up. - return e.nowFn().UnixNano() >= e.buildingSourcesAtNanos+e.sourcesTTLNanos -} - -func (e *GaugeElem) maybeRefreshSourcesSetWithElemLock() { - if !e.opts.EnableEagerForwarding() { - return - } - nowNanos := e.nowFn().UnixNano() - if nowNanos-e.lastSourcesRefreshNanos < e.sourcesTTLNanos { - return - } - e.sourcesLock.Lock() - for sourceID, lastHeartbeatNanos := range e.sourcesHeartbeat { - if nowNanos-lastHeartbeatNanos >= e.sourcesTTLNanos { - delete(e.sourcesHeartbeat, sourceID) - e.sourcesSet.Clear(uint(sourceID)) - } - } - e.lastSourcesRefreshNanos = nowNanos - e.sourcesLock.Unlock() -} - -func (e *GaugeElem) updateSources(source uint32) { - if !e.opts.EnableEagerForwarding() { - return - } - nowNanos := e.nowFn().UnixNano() - e.sourcesLock.Lock() - // First time a source is received. - if e.sourcesHeartbeat == nil { - e.sourcesHeartbeat = make(map[uint32]int64, defaultNumSources) - e.sourcesSet = bitset.New(defaultNumSources) - e.buildingSourcesAtNanos = nowNanos - e.lastSourcesRefreshNanos = nowNanos - } - if v, exists := e.sourcesHeartbeat[source]; !exists || v < nowNanos { - e.sourcesHeartbeat[source] = nowNanos - } - e.sourcesSet.Set(uint(source)) - e.sourcesLock.Unlock() } diff --git a/aggregator/generic_elem.go b/aggregator/generic_elem.go index 1607a9d..4bde6f6 100644 --- a/aggregator/generic_elem.go +++ b/aggregator/generic_elem.go @@ -48,9 +48,6 @@ type typeSpecificAggregation interface { // AddUnion adds a new metric value union. AddUnion(mu unaggregated.MetricUnion) - // Count returns the number of values in the aggregation. - Count() int64 - // ValueOf returns the value for the given aggregation type. ValueOf(aggType maggregation.Type) float64 @@ -100,39 +97,9 @@ type typeSpecificElemBase interface { type lockedAggregation struct { sync.Mutex - closed bool - - // sourcesReady is only used for elements receiving forwarded metrics. - // It determines whether the current aggregation can use its source set - // to determine whether it has received data from all forwarding sources - // to perform eager forwarding if enabled. - sourcesReady bool - - // expectedSources is only used for elements receiving forwarded metrics. - // It keeps track of all the sources the current aggregation expect to receive - // data from. - expectedSources *bitset.BitSet - - // seenSources keeps track of all the sources the current aggregation has - // seen so far. - seenSources *bitset.BitSet - - // consumeState is only used for elements receiving forwarded metrics. It - // describes whether the current aggregation is ready to be consumed or has - // been consumed. This in turn determines whether the aggregation can be - // eagerly consumed, or should be skipped during consumption. - consumeState consumeState - aggregation typeSpecificAggregation -} - -func (lockedAgg *lockedAggregation) close() { - if lockedAgg.closed { - return - } - lockedAgg.closed = true - lockedAgg.expectedSources = nil - lockedAgg.seenSources = nil - lockedAgg.aggregation.Close() + closed bool + sourcesSeen *bitset.BitSet + aggregation typeSpecificAggregation } type timedAggregation struct { @@ -158,7 +125,6 @@ type GenericElem struct { // NewGenericElem creates a new element for the given metric type. func NewGenericElem( - incomingMetricType IncomingMetricType, id id.RawID, sp policy.StoragePolicy, aggTypes maggregation.Types, @@ -170,7 +136,7 @@ func NewGenericElem( elemBase: newElemBase(opts), values: make([]timedAggregation, 0, defaultNumAggregations), // in most cases values will have two entries } - if err := e.ResetSetData(incomingMetricType, id, sp, aggTypes, pipeline, numForwardedTimes); err != nil { + if err := e.ResetSetData(id, sp, aggTypes, pipeline, numForwardedTimes); err != nil { return nil, err } return e, nil @@ -178,7 +144,6 @@ func NewGenericElem( // MustNewGenericElem creates a new element, or panics if the input is invalid. func MustNewGenericElem( - incomingMetricType IncomingMetricType, id id.RawID, sp policy.StoragePolicy, aggTypes maggregation.Types, @@ -186,7 +151,7 @@ func MustNewGenericElem( numForwardedTimes int, opts Options, ) *GenericElem { - elem, err := NewGenericElem(incomingMetricType, id, sp, aggTypes, pipeline, numForwardedTimes, opts) + elem, err := NewGenericElem(id, sp, aggTypes, pipeline, numForwardedTimes, opts) if err != nil { panic(fmt.Errorf("unable to create element: %v", err)) } @@ -195,7 +160,6 @@ func MustNewGenericElem( // ResetSetData resets the element and sets data. func (e *GenericElem) ResetSetData( - incomingMetricType IncomingMetricType, id id.RawID, sp policy.StoragePolicy, aggTypes maggregation.Types, @@ -206,7 +170,7 @@ func (e *GenericElem) ResetSetData( if useDefaultAggregation { aggTypes = e.DefaultAggregationTypes(e.aggTypesOpts) } - if err := e.elemBase.resetSetData(incomingMetricType, id, sp, aggTypes, useDefaultAggregation, pipeline, numForwardedTimes); err != nil { + if err := e.elemBase.resetSetData(id, sp, aggTypes, useDefaultAggregation, pipeline, numForwardedTimes); err != nil { return err } if err := e.typeSpecificElemBase.ResetSetData(e.aggTypesOpts, aggTypes, useDefaultAggregation); err != nil { @@ -231,7 +195,7 @@ func (e *GenericElem) ResetSetData( // AddUnion adds a metric value union at a given timestamp. func (e *GenericElem) AddUnion(timestamp time.Time, mu unaggregated.MetricUnion) error { alignedStart := timestamp.Truncate(e.sp.Resolution().Window).UnixNano() - lockedAgg, err := e.findOrCreate(alignedStart, sourcesOptions{}) + lockedAgg, err := e.findOrCreate(alignedStart, createAggregationOptions{}) if err != nil { return err } @@ -250,7 +214,7 @@ func (e *GenericElem) AddUnion(timestamp time.Time, mu unaggregated.MetricUnion) // same aggregation, the incoming value is discarded. func (e *GenericElem) AddUnique(timestamp time.Time, values []float64, sourceID uint32) error { alignedStart := timestamp.Truncate(e.sp.Resolution().Window).UnixNano() - lockedAgg, err := e.findOrCreate(alignedStart, sourcesOptions{updateSources: true, source: sourceID}) + lockedAgg, err := e.findOrCreate(alignedStart, createAggregationOptions{initSourceSet: true}) if err != nil { return err } @@ -260,26 +224,11 @@ func (e *GenericElem) AddUnique(timestamp time.Time, values []float64, sourceID return errAggregationClosed } source := uint(sourceID) - if lockedAgg.seenSources.Test(source) { + if lockedAgg.sourcesSeen.Test(source) { lockedAgg.Unlock() return errDuplicateForwardingSource } - lockedAgg.seenSources.Set(source) - if lockedAgg.sourcesReady { - // If the sources are ready, the expected sources will be a pre-filled - // bitset populated with sources the aggregation is expected to see data from. - // As such, we need to clear the source bit in the expected sources. - if lockedAgg.expectedSources.Test(source) { - // This source is never seen before and is in the expected source list, - // as a result, we need to clear the source bit. - lockedAgg.expectedSources.Clear(source) - if lockedAgg.expectedSources.None() { - lockedAgg.consumeState = readyToConsume - } - } - // New sources that are not in the expected source list are still allowed - // to go through. - } + lockedAgg.sourcesSeen.Set(source) for _, v := range values { lockedAgg.aggregation.Add(v) } @@ -294,7 +243,6 @@ func (e *GenericElem) AddUnique(timestamp time.Time, values []float64, sourceID // to avoid race conditions. func (e *GenericElem) Consume( targetNanos int64, - eagerForwardingMode eagerForwardingMode, isEarlierThanFn isEarlierThanFn, timestampNanosFn timestampNanosFn, flushLocalFn flushLocalMetricFn, @@ -310,8 +258,7 @@ func (e *GenericElem) Consume( idx := 0 for range e.values { // Bail as soon as the timestamp is no later than the target time. - timeNanos := timestampNanosFn(e.values[idx].startAtNanos, resolution) - if !isEarlierThanFn(timeNanos, targetNanos) { + if !isEarlierThanFn(e.values[idx].startAtNanos, resolution, targetNanos) { break } idx++ @@ -329,71 +276,31 @@ func (e *GenericElem) Consume( e.values = e.values[:n] } canCollect := len(e.values) == 0 && e.tombstoned - - // Additionally for elements receiving forwarded metrics and sending aggregated metrics - // to local backends, we also check if any aggregations are ready to be consumed. We however - // do not remove the aggregations as we do for aggregations whose timestamps are old enough, - // since for aggregations receiving forwarded metrics that are marked "consume ready", it is - // possible that metrics still go to the such aggregation bucket after they are marked "consume - // ready" due to delayed source re-delivery or new sources showing up, and removing such - // aggregation prematurely would mean the values from the delayed sources and/or new sources - // would be considered as the aggregated value for such aggregation bucket, which is incorrect. - // By keeping such aggregation buckets and only removing them when they are considered old enough - // (i.e., when their timestmaps are earlier than the target timestamp), we ensure no metrics may - // go to such aggregation buckets after they are consumed and therefore avoid the aformentioned - // problem. - aggregationIdxToCloseUntil := len(e.toConsume) - if e.incomingMetricType == ForwardedIncomingMetric && e.isSourcesSetReadyWithElemLock() { - e.maybeRefreshSourcesSetWithElemLock() - // We only attempt to consume if the outgoing metrics type is local instead of forwarded - // and eager forwarding is allowed (eager forwarding may be enabled and disallowed when - // the node is a follower and allowed when the node is a leader. This is because forwarded - // metrics are sent in batches and can only be sent when all sources in the same shard have - // been consumed, and as such is not well suited for pre-emptive consumption. - if e.outgoingMetricType() == localOutgoingMetric && eagerForwardingMode == allowEagerForwarding { - for i := 0; i < len(e.values); i++ { - // NB: This makes the logic easier to understand but it would be more efficient to use - // an atomic here to avoid locking aggregations. - e.values[i].lockedAgg.Lock() - if e.values[i].lockedAgg.consumeState == readyToConsume { - e.toConsume = append(e.toConsume, e.values[i]) - e.values[i].lockedAgg.consumeState = consuming - } - e.values[i].lockedAgg.Unlock() - } - } - } e.Unlock() // Process the aggregations that are ready for consumption. for i := range e.toConsume { timeNanos := timestampNanosFn(e.toConsume[i].startAtNanos, resolution) e.toConsume[i].lockedAgg.Lock() - if e.toConsume[i].lockedAgg.consumeState != consumed { - e.processValueWithAggregationLock( - timeNanos, eagerForwardingMode, e.toConsume[i].lockedAgg, - flushLocalFn, flushForwardedFn, - ) - } - e.toConsume[i].lockedAgg.consumeState = consumed - if i < aggregationIdxToCloseUntil { - if e.toConsume[i].lockedAgg.seenSources != nil { - e.sourcesLock.Lock() - // This is to make sure there aren't too many cached source sets taking up - // too much space. - if len(e.cachedSourceSets) < e.opts.MaxNumCachedSourceSets() { - e.cachedSourceSets = append(e.cachedSourceSets, e.toConsume[i].lockedAgg.seenSources) - e.toConsume[i].lockedAgg.seenSources = nil - } - e.sourcesLock.Unlock() + e.processValueWithAggregationLock(timeNanos, e.toConsume[i].lockedAgg, flushLocalFn, flushForwardedFn) + // Closes the aggregation object after it's processed. + e.toConsume[i].lockedAgg.closed = true + e.toConsume[i].lockedAgg.aggregation.Close() + if e.toConsume[i].lockedAgg.sourcesSeen != nil { + e.cachedSourceSetsLock.Lock() + // This is to make sure there aren't too many cached source sets taking up + // too much space. + if len(e.cachedSourceSets) < e.opts.MaxNumCachedSourceSets() { + e.cachedSourceSets = append(e.cachedSourceSets, e.toConsume[i].lockedAgg.sourcesSeen) } - e.toConsume[i].lockedAgg.close() + e.cachedSourceSetsLock.Unlock() + e.toConsume[i].lockedAgg.sourcesSeen = nil } e.toConsume[i].lockedAgg.Unlock() e.toConsume[i].Reset() } - if e.outgoingMetricType() == forwardedOutgoingMetric { + if e.parsedPipeline.HasRollup { forwardedAggregationKey, _ := e.ForwardedAggregationKey() onForwardedFlushedFn(e.onForwardedAggregationWrittenFn, forwardedAggregationKey) } @@ -413,15 +320,14 @@ func (e *GenericElem) Close() { e.parsedPipeline = parsedPipeline{} e.writeForwardedMetricFn = nil e.onForwardedAggregationWrittenFn = nil - e.sourcesHeartbeat = nil - e.sourcesSet = nil for idx := range e.cachedSourceSets { e.cachedSourceSets[idx] = nil } e.cachedSourceSets = nil for idx := range e.values { // Close the underlying aggregation objects. - e.values[idx].lockedAgg.close() + e.values[idx].lockedAgg.sourcesSeen = nil + e.values[idx].lockedAgg.aggregation.Close() e.values[idx].Reset() } e.values = e.values[:0] @@ -442,16 +348,13 @@ func (e *GenericElem) Close() { // if it doesn't exist. func (e *GenericElem) findOrCreate( alignedStart int64, - sourcesOpts sourcesOptions, + createOpts createAggregationOptions, ) (*lockedAggregation, error) { e.RLock() if e.closed { e.RUnlock() return nil, errElemClosed } - if sourcesOpts.updateSources { - e.updateSources(sourcesOpts.source) - } idx, found := e.indexOfWithLock(alignedStart) if found { agg := e.values[idx].lockedAgg @@ -477,36 +380,24 @@ func (e *GenericElem) findOrCreate( e.values = append(e.values, timedAggregation{}) copy(e.values[idx+1:numValues+1], e.values[idx:numValues]) - var ( - sourcesReady = e.isSourcesSetReadyWithElemLock() - expectedSources *bitset.BitSet - seenSources *bitset.BitSet - ) - if sourcesOpts.updateSources { - e.sourcesLock.Lock() - // If the sources set is ready, we clone it ane use the clone to - // determine when we have received from all the expected sources. - if sourcesReady { - expectedSources = e.sourcesSet.Clone() - } + var sourcesSeen *bitset.BitSet + if createOpts.initSourceSet { + e.cachedSourceSetsLock.Lock() if numCachedSourceSets := len(e.cachedSourceSets); numCachedSourceSets > 0 { - seenSources = e.cachedSourceSets[numCachedSourceSets-1] + sourcesSeen = e.cachedSourceSets[numCachedSourceSets-1] e.cachedSourceSets[numCachedSourceSets-1] = nil e.cachedSourceSets = e.cachedSourceSets[:numCachedSourceSets-1] - seenSources.ClearAll() + sourcesSeen.ClearAll() } else { - seenSources = bitset.New(defaultNumSources) + sourcesSeen = bitset.New(defaultNumSources) } - e.sourcesLock.Unlock() + e.cachedSourceSetsLock.Unlock() } - e.values[idx] = timedAggregation{ startAtNanos: alignedStart, lockedAgg: &lockedAggregation{ - sourcesReady: sourcesReady, - expectedSources: expectedSources, - seenSources: seenSources, - aggregation: e.NewAggregation(e.opts, e.aggOpts), + sourcesSeen: sourcesSeen, + aggregation: e.NewAggregation(e.opts, e.aggOpts), }, } agg := e.values[idx].lockedAgg @@ -545,14 +436,10 @@ func (e *GenericElem) indexOfWithLock(alignedStart int64) (int, bool) { func (e *GenericElem) processValueWithAggregationLock( timeNanos int64, - eagerForwardingMode eagerForwardingMode, lockedAgg *lockedAggregation, flushLocalFn flushLocalMetricFn, flushForwardedFn flushForwardedMetricFn, ) { - if lockedAgg.aggregation.Count() == 0 { - return - } var ( fullPrefix = e.FullPrefix(e.opts) transformations = e.parsedPipeline.Transformations @@ -582,7 +469,7 @@ func (e *GenericElem) processValueWithAggregationLock( if discardNaNValues && math.IsNaN(value) { continue } - if e.outgoingMetricType() == localOutgoingMetric { + if !e.parsedPipeline.HasRollup { flushLocalFn(fullPrefix, e.id, e.TypeStringFor(e.aggTypesOpts, aggType), timeNanos, value, e.sp) } else { forwardedAggregationKey, _ := e.ForwardedAggregationKey() @@ -590,73 +477,4 @@ func (e *GenericElem) processValueWithAggregationLock( } } e.lastConsumedAtNanos = timeNanos - - // Emit latency metrics for forwarded metrics going to local backends - // when eager forwarding is allowed. - if eagerForwardingMode == allowEagerForwarding && - e.incomingMetricType == ForwardedIncomingMetric && - e.outgoingMetricType() == localOutgoingMetric { - e.opts.FullForwardingLatencyHistograms().RecordDuration( - e.sp.Resolution().Window, - e.numForwardedTimes, - time.Duration(e.nowFn().UnixNano()-timeNanos), - ) - } -} - -func (e *GenericElem) outgoingMetricType() outgoingMetricType { - if !e.parsedPipeline.HasRollup { - return localOutgoingMetric - } - return forwardedOutgoingMetric -} - -func (e *GenericElem) isSourcesSetReadyWithElemLock() bool { - if !e.opts.EnableEagerForwarding() { - return false - } - if e.buildingSourcesAtNanos == 0 { - return false - } - // NB: Allow TTL for the source set to build up. - return e.nowFn().UnixNano() >= e.buildingSourcesAtNanos+e.sourcesTTLNanos -} - -func (e *GenericElem) maybeRefreshSourcesSetWithElemLock() { - if !e.opts.EnableEagerForwarding() { - return - } - nowNanos := e.nowFn().UnixNano() - if nowNanos-e.lastSourcesRefreshNanos < e.sourcesTTLNanos { - return - } - e.sourcesLock.Lock() - for sourceID, lastHeartbeatNanos := range e.sourcesHeartbeat { - if nowNanos-lastHeartbeatNanos >= e.sourcesTTLNanos { - delete(e.sourcesHeartbeat, sourceID) - e.sourcesSet.Clear(uint(sourceID)) - } - } - e.lastSourcesRefreshNanos = nowNanos - e.sourcesLock.Unlock() -} - -func (e *GenericElem) updateSources(source uint32) { - if !e.opts.EnableEagerForwarding() { - return - } - nowNanos := e.nowFn().UnixNano() - e.sourcesLock.Lock() - // First time a source is received. - if e.sourcesHeartbeat == nil { - e.sourcesHeartbeat = make(map[uint32]int64, defaultNumSources) - e.sourcesSet = bitset.New(defaultNumSources) - e.buildingSourcesAtNanos = nowNanos - e.lastSourcesRefreshNanos = nowNanos - } - if v, exists := e.sourcesHeartbeat[source]; !exists || v < nowNanos { - e.sourcesHeartbeat[source] = nowNanos - } - e.sourcesSet.Set(uint(source)) - e.sourcesLock.Unlock() } diff --git a/aggregator/leader_flush_mgr.go b/aggregator/leader_flush_mgr.go index 45ce999..2eef21b 100644 --- a/aggregator/leader_flush_mgr.go +++ b/aggregator/leader_flush_mgr.go @@ -264,10 +264,10 @@ func (mgr *leaderFlushManager) updateFlushTimesWithLock( } for _, bucket := range buckets { bucketID := bucket.bucketID - switch bucketID.incomingMetricType { - case StandardIncomingMetric: + switch bucketID.listType { + case standardMetricListType: mgr.updateStandardFlushTimesWithLock(bucketID.standard, bucket.flushers) - case ForwardedIncomingMetric: + case forwardedMetricListType: mgr.updateForwardedFlushTimesWithLock(bucketID.forwarded, bucket.flushers) default: panic("should never get here") @@ -413,7 +413,7 @@ func (t *leaderFlushTask) Run() { flusher := flusher wgWorkers.Add(1) mgr.workers.Go(func() { - flusher.Flush(req, allowEagerForwarding) + flusher.Flush(req) wgWorkers.Done() }) } diff --git a/aggregator/leader_flush_mgr_test.go b/aggregator/leader_flush_mgr_test.go index 8b1b638..7755fe1 100644 --- a/aggregator/leader_flush_mgr_test.go +++ b/aggregator/leader_flush_mgr_test.go @@ -478,8 +478,8 @@ func TestLeaderFlushTaskRunShardNotFound(t *testing.T) { flusher := NewMockflushingMetricList(ctrl) flusher.EXPECT().Shard().Return(uint32(2)).AnyTimes() flusher.EXPECT(). - Flush(gomock.Any(), allowEagerForwarding). - Do(func(req flushRequest, _ eagerForwardingMode) { + Flush(gomock.Any()). + Do(func(req flushRequest) { request = &req }) @@ -514,15 +514,15 @@ func TestLeaderFlushTaskRunWithFlushes(t *testing.T) { flusher1 := NewMockflushingMetricList(ctrl) flusher1.EXPECT().Shard().Return(uint32(0)).AnyTimes() flusher1.EXPECT(). - Flush(gomock.Any(), allowEagerForwarding). - Do(func(req flushRequest, _ eagerForwardingMode) { + Flush(gomock.Any()). + Do(func(req flushRequest) { requests[0] = req }) flusher2 := NewMockflushingMetricList(ctrl) flusher2.EXPECT().Shard().Return(uint32(1)).AnyTimes() flusher2.EXPECT(). - Flush(gomock.Any(), allowEagerForwarding). - Do(func(req flushRequest, _ eagerForwardingMode) { + Flush(gomock.Any()). + Do(func(req flushRequest) { requests[1] = req }) diff --git a/aggregator/list.go b/aggregator/list.go index 6f7557f..32a5448 100644 --- a/aggregator/list.go +++ b/aggregator/list.go @@ -147,11 +147,7 @@ func newMetricListMetrics(scope tally.Scope) baseMetricListMetrics { // of aggregation windows that are eligible for flushing. type targetNanosFn func(nowNanos int64) int64 -type flushBeforeFn func( - beforeNanos int64, - flushType flushType, - eagerForwardingMode eagerForwardingMode, -) +type flushBeforeFn func(beforeNanos int64, flushType flushType) // baseMetricList is a metric list storing aggregations at a given resolution and // flushing them periodically. @@ -166,7 +162,6 @@ type baseMetricList struct { localWriter writer.Writer forwardedWriter forwardedMetricWriter resolution time.Duration - flushInterval time.Duration targetNanosFn targetNanosFn isEarlierThanFn isEarlierThanFn timestampNanosFn timestampNanosFn @@ -189,7 +184,6 @@ type baseMetricList struct { func newBaseMetricList( shard uint32, resolution time.Duration, - flushInterval time.Duration, targetNanosFn targetNanosFn, isEarlierThanFn isEarlierThanFn, timestampNanosFn timestampNanosFn, @@ -205,12 +199,7 @@ func newBaseMetricList( return nil, err } forwardedWriterScope := scope.Tagged(map[string]string{"writer-type": "forwarded"}).SubScope("writer") - maxForwardingWindows := opts.MaxAggregationWindowsForEagerForwarding() - nowFn := opts.ClockOptions().NowFn() - forwardedWriter := newForwardedWriter( - shard, opts.AdminClient(), opts.EnableEagerForwarding(), - maxForwardingWindows, isEarlierThanFn, nowFn, forwardedWriterScope, - ) + forwardedWriter := newForwardedWriter(shard, opts.AdminClient(), forwardedWriterScope) l := &baseMetricList{ shard: shard, opts: opts, @@ -220,7 +209,6 @@ func newBaseMetricList( localWriter: localWriter, forwardedWriter: forwardedWriter, resolution: resolution, - flushInterval: flushInterval, targetNanosFn: targetNanosFn, isEarlierThanFn: isEarlierThanFn, timestampNanosFn: timestampNanosFn, @@ -240,7 +228,7 @@ func newBaseMetricList( func (l *baseMetricList) Shard() uint32 { return l.shard } func (l *baseMetricList) Resolution() time.Duration { return l.resolution } -func (l *baseMetricList) FlushInterval() time.Duration { return l.flushInterval } +func (l *baseMetricList) FlushInterval() time.Duration { return l.resolution } func (l *baseMetricList) LastFlushedNanos() int64 { return atomic.LoadInt64(&l.lastFlushedNanos) } // Len returns the number of elements in the list. @@ -304,7 +292,7 @@ func (l *baseMetricList) Close() bool { return true } -func (l *baseMetricList) Flush(req flushRequest, eagerForwardingMode eagerForwardingMode) { +func (l *baseMetricList) Flush(req flushRequest) { start := l.nowFn() defer func() { @@ -322,23 +310,23 @@ func (l *baseMetricList) Flush(req flushRequest, eagerForwardingMode eagerForwar // Metrics before shard cutover are discarded. if targetNanos <= req.CutoverNanos { - l.flushBeforeFn(targetNanos, discardType, eagerForwardingMode) + l.flushBeforeFn(targetNanos, discardType) l.metrics.flushBeforeCutover.Inc(1) return } // Metrics between shard cutover and shard cutoff are consumed. if req.CutoverNanos > 0 { - l.flushBeforeFn(req.CutoverNanos, discardType, eagerForwardingMode) + l.flushBeforeFn(req.CutoverNanos, discardType) } if targetNanos <= req.CutoffNanos { - l.flushBeforeFn(targetNanos, consumeType, eagerForwardingMode) + l.flushBeforeFn(targetNanos, consumeType) l.metrics.flushBetweenCutoverCutoff.Inc(1) return } // Metrics after now-keepAfterCutoff are retained. - l.flushBeforeFn(req.CutoffNanos, consumeType, eagerForwardingMode) + l.flushBeforeFn(req.CutoffNanos, consumeType) bufferEndNanos := targetNanos - int64(req.BufferAfterCutoff) if bufferEndNanos <= req.CutoffNanos { l.metrics.flushBetweenCutoffBufferEnd.Inc(1) @@ -346,22 +334,18 @@ func (l *baseMetricList) Flush(req flushRequest, eagerForwardingMode eagerForwar } // Metrics between cutoff and now-bufferAfterCutoff are discarded. - l.flushBeforeFn(bufferEndNanos, discardType, eagerForwardingMode) + l.flushBeforeFn(bufferEndNanos, discardType) l.metrics.flushAfterBufferEnd.Inc(1) } -func (l *baseMetricList) DiscardBefore(beforeNanos int64, eagerForwardingMode eagerForwardingMode) { - l.flushBeforeFn(beforeNanos, discardType, eagerForwardingMode) +func (l *baseMetricList) DiscardBefore(beforeNanos int64) { + l.flushBeforeFn(beforeNanos, discardType) l.metrics.discardBefore.Inc(1) } // flushBefore flushes or discards data before a given time based on the flush type. // It is not thread-safe. -func (l *baseMetricList) flushBefore( - beforeNanos int64, - flushType flushType, - eagerForwardingMode eagerForwardingMode, -) { +func (l *baseMetricList) flushBefore(beforeNanos int64, flushType flushType) { if l.LastFlushedNanos() >= beforeNanos { l.metrics.flushBeforeStale.Inc(1) return @@ -387,14 +371,13 @@ func (l *baseMetricList) flushBefore( // NB: Ensure the elements are consumed within a read lock so that the // refcounts of forwarded metrics tracked in the forwarded writer do not // change so no elements may be added or removed while holding the lock. - l.forwardedWriter.Prepare(beforeNanos) + l.forwardedWriter.Prepare() for e := l.aggregations.Front(); e != nil; e = e.Next() { // If the element is eligible for collection after the values are // processed, add it to the list of elements to collect. elem := e.Value.(metricElem) if elem.Consume( beforeNanos, - eagerForwardingMode, l.isEarlierThanFn, l.timestampNanosFn, flushLocalFn, @@ -529,13 +512,15 @@ func (l *baseMetricList) onForwardingElemDiscarded( // Standard metrics whose timestamps are earlier than current time can be flushed. func standardMetricTargetNanos(nowNanos int64) int64 { return nowNanos } -// Only the aggregation windows whose timestamps are no later than the target time -// can be flushed. +// The timestamp of a standard metric is the end time boundary of the aggregation +// window when the metric is flushed. As such, only the aggregation windows whose +// end time boundaries are no later than the target time can be flushed. func isStandardMetricEarlierThan( - timestampNanos int64, + windowStartNanos int64, + resolution time.Duration, targetNanos int64, ) bool { - return timestampNanos <= targetNanos + return windowStartNanos+resolution.Nanoseconds() <= targetNanos } // The timestamp of a standard metric is set to the end time boundary of the aggregation @@ -550,10 +535,7 @@ type standardMetricListID struct { } func (id standardMetricListID) toMetricListID() metricListID { - return metricListID{ - incomingMetricType: StandardIncomingMetric, - standard: id, - } + return metricListID{listType: standardMetricListType, standard: id} } // standardMetricList is a list storing aggregations of incoming standard metrics @@ -572,11 +554,9 @@ func newStandardMetricList( ) (*standardMetricList, error) { iOpts := opts.InstrumentOptions() listScope := iOpts.MetricsScope().Tagged(map[string]string{"list-type": "standard"}) - flushInterval := opts.FlushIntervalFn()(StandardIncomingMetric, id.resolution) l, err := newBaseMetricList( shard, id.resolution, - flushInterval, standardMetricTargetNanos, isStandardMetricEarlierThan, standardMetricTimestampNanos, @@ -610,10 +590,11 @@ func (l *standardMetricList) Close() { // NB: the targetNanos for forwarded metrics has already taken into account the maximum // amount of lateness that is tolerated. func isForwardedMetricEarlierThan( - timestampNanos int64, + windowStartNanos int64, + _ time.Duration, targetNanos int64, ) bool { - return timestampNanos < targetNanos + return windowStartNanos < targetNanos } // The timestamp of a forwarded metric is set to the start time boundary of the aggregation @@ -631,10 +612,7 @@ type forwardedMetricListID struct { } func (id forwardedMetricListID) toMetricListID() metricListID { - return metricListID{ - incomingMetricType: ForwardedIncomingMetric, - forwarded: id, - } + return metricListID{listType: forwardedMetricListType, forwarded: id} } // forwardedMetricList is a list storing aggregations of incoming forwarded metrics @@ -655,7 +633,6 @@ func newForwardedMetricList( ) (*forwardedMetricList, error) { var ( resolution = id.resolution - flushInterval = opts.FlushIntervalFn()(ForwardedIncomingMetric, resolution) numForwardedTimes = id.numForwardedTimes maxLatenessAllowedFn = opts.MaxAllowedForwardingDelayFn() maxLatenessAllowed = maxLatenessAllowedFn(resolution, numForwardedTimes) @@ -670,7 +647,6 @@ func newForwardedMetricList( l, err := newBaseMetricList( shard, resolution, - flushInterval, targetNanosFn, isForwardedMetricEarlierThan, forwardedMetricTimestampNanos, @@ -712,20 +688,39 @@ func (l *forwardedMetricList) Close() { } } +type metricListType int + +const ( + standardMetricListType metricListType = iota + forwardedMetricListType +) + +func (t metricListType) String() string { + switch t { + case standardMetricListType: + return "standard" + case forwardedMetricListType: + return "forwarded" + default: + // Should never get here. + return "unknown" + } +} + type metricListID struct { - incomingMetricType IncomingMetricType - standard standardMetricListID - forwarded forwardedMetricListID + listType metricListType + standard standardMetricListID + forwarded forwardedMetricListID } func newMetricList(shard uint32, id metricListID, opts Options) (metricList, error) { - switch id.incomingMetricType { - case StandardIncomingMetric: + switch id.listType { + case standardMetricListType: return newStandardMetricList(shard, id.standard, opts) - case ForwardedIncomingMetric: + case forwardedMetricListType: return newForwardedMetricList(shard, id.forwarded, opts) default: - return nil, fmt.Errorf("unknown incoming metric type for list: %v", id.incomingMetricType) + return nil, fmt.Errorf("unknown list type: %v", id.listType) } } @@ -806,10 +801,10 @@ func (l *metricLists) Tick() listsTickResult { for id, list := range l.lists { resolution := list.Resolution() numElems := list.Len() - switch id.incomingMetricType { - case StandardIncomingMetric: + switch id.listType { + case standardMetricListType: res.standard[resolution] += numElems - case ForwardedIncomingMetric: + case forwardedMetricListType: res.forwarded[resolution] += numElems } } diff --git a/aggregator/list_test.go b/aggregator/list_test.go index d83dead..65de19a 100644 --- a/aggregator/list_test.go +++ b/aggregator/list_test.go @@ -49,9 +49,9 @@ func TestBaseMetricListPushBackElemWithDefaultPipeline(t *testing.T) { ctrl := gomock.NewController(t) defer ctrl.Finish() - l, err := newBaseMetricList(testShard, time.Second, time.Second, nil, nil, nil, testOptions(ctrl)) + l, err := newBaseMetricList(testShard, time.Second, nil, nil, nil, testOptions(ctrl)) require.NoError(t, err) - elem, err := NewCounterElem(StandardIncomingMetric, nil, policy.EmptyStoragePolicy, aggregation.DefaultTypes, applied.DefaultPipeline, 0, l.opts) + elem, err := NewCounterElem(nil, policy.EmptyStoragePolicy, aggregation.DefaultTypes, applied.DefaultPipeline, 0, l.opts) require.NoError(t, err) // Push a counter to the list. @@ -75,9 +75,9 @@ func TestBaseMetricListPushBackElemWithForwardingPipeline(t *testing.T) { ctrl := gomock.NewController(t) defer ctrl.Finish() - l, err := newBaseMetricList(testShard, time.Second, time.Second, nil, nil, nil, testOptions(ctrl)) + l, err := newBaseMetricList(testShard, time.Second, nil, nil, nil, testOptions(ctrl)) require.NoError(t, err) - elem, err := NewCounterElem(StandardIncomingMetric, nil, policy.EmptyStoragePolicy, aggregation.DefaultTypes, testPipeline, 0, l.opts) + elem, err := NewCounterElem(nil, policy.EmptyStoragePolicy, aggregation.DefaultTypes, testPipeline, 0, l.opts) require.NoError(t, err) // Push a counter to the list. @@ -94,7 +94,7 @@ func TestBaseMetricListClose(t *testing.T) { defer ctrl.Finish() opts := testOptions(ctrl) - l, err := newBaseMetricList(testShard, time.Second, time.Second, nil, nil, nil, opts) + l, err := newBaseMetricList(testShard, time.Second, nil, nil, nil, opts) require.NoError(t, err) l.RLock() @@ -122,24 +122,18 @@ func TestBaseMetricListFlushWithRequests(t *testing.T) { results []flushBeforeResult ) opts := testOptions(ctrl).SetClockOptions(clock.NewOptions().SetNowFn(nowFn)) - l, err := newBaseMetricList(testShard, time.Second, time.Second, targetNanosFn, isEarlierThanFn, timestampNanosFn, opts) + l, err := newBaseMetricList(testShard, time.Second, targetNanosFn, isEarlierThanFn, timestampNanosFn, opts) require.NoError(t, err) - l.flushBeforeFn = func( - beforeNanos int64, - flushType flushType, - eagerForwardingMode eagerForwardingMode, - ) { + l.flushBeforeFn = func(beforeNanos int64, flushType flushType) { results = append(results, flushBeforeResult{ - beforeNanos: beforeNanos, - flushType: flushType, - eagerForwardingMode: eagerForwardingMode, + beforeNanos: beforeNanos, + flushType: flushType, }) } inputs := []struct { - request flushRequest - eagerForwardingMode eagerForwardingMode - expected []flushBeforeResult + request flushRequest + expected []flushBeforeResult }{ { request: flushRequest{ @@ -147,7 +141,6 @@ func TestBaseMetricListFlushWithRequests(t *testing.T) { CutoffNanos: 30000 * int64(time.Second), BufferAfterCutoff: time.Second, }, - eagerForwardingMode: allowEagerForwarding, expected: []flushBeforeResult{ { beforeNanos: 12345 * int64(time.Second), @@ -161,7 +154,6 @@ func TestBaseMetricListFlushWithRequests(t *testing.T) { CutoffNanos: 30000 * int64(time.Second), BufferAfterCutoff: time.Second, }, - eagerForwardingMode: allowEagerForwarding, expected: []flushBeforeResult{ { beforeNanos: 10000 * int64(time.Second), @@ -179,7 +171,6 @@ func TestBaseMetricListFlushWithRequests(t *testing.T) { CutoffNanos: 12300 * int64(time.Second), BufferAfterCutoff: time.Minute, }, - eagerForwardingMode: allowEagerForwarding, expected: []flushBeforeResult{ { beforeNanos: 10000 * int64(time.Second), @@ -197,7 +188,6 @@ func TestBaseMetricListFlushWithRequests(t *testing.T) { CutoffNanos: 12300 * int64(time.Second), BufferAfterCutoff: 10 * time.Second, }, - eagerForwardingMode: allowEagerForwarding, expected: []flushBeforeResult{ { beforeNanos: 10000 * int64(time.Second), @@ -219,7 +209,6 @@ func TestBaseMetricListFlushWithRequests(t *testing.T) { CutoffNanos: 30000 * int64(time.Second), BufferAfterCutoff: time.Second, }, - eagerForwardingMode: allowEagerForwarding, expected: []flushBeforeResult{ { beforeNanos: 12345 * int64(time.Second), @@ -230,7 +219,7 @@ func TestBaseMetricListFlushWithRequests(t *testing.T) { } for _, input := range inputs { results = results[:0] - l.Flush(input.request, input.eagerForwardingMode) + l.Flush(input.request) require.Equal(t, input.expected, results) } } @@ -245,10 +234,10 @@ func TestBaseMetricListFlushBeforeStale(t *testing.T) { timestampNanosFn = standardMetricTimestampNanos opts = testOptions(ctrl) ) - l, err := newBaseMetricList(testShard, 0, 0, targetNanosFn, isEarlierThanFn, timestampNanosFn, opts) + l, err := newBaseMetricList(testShard, 0, targetNanosFn, isEarlierThanFn, timestampNanosFn, opts) require.NoError(t, err) l.lastFlushedNanos = 1234 - l.flushBefore(1000, discardType, allowEagerForwarding) + l.flushBefore(1000, discardType) require.Equal(t, int64(1234), l.LastFlushedNanos()) } @@ -263,8 +252,8 @@ func TestStandardMetricListID(t *testing.T) { require.NoError(t, err) expectedListID := metricListID{ - incomingMetricType: StandardIncomingMetric, - standard: listID, + listType: standardMetricListType, + standard: listID, } require.Equal(t, expectedListID, l.ID()) } @@ -322,15 +311,15 @@ func TestStandardMetricListFlushConsumingAndCollectingLocalMetrics(t *testing.T) metric unaggregated.MetricUnion }{ { - elem: MustNewCounterElem(StandardIncomingMetric, testCounterID, testStoragePolicy, aggregation.DefaultTypes, applied.DefaultPipeline, 0, opts), + elem: MustNewCounterElem(testCounterID, testStoragePolicy, aggregation.DefaultTypes, applied.DefaultPipeline, 0, opts), metric: testCounter, }, { - elem: MustNewTimerElem(StandardIncomingMetric, testBatchTimerID, testStoragePolicy, aggregation.DefaultTypes, applied.DefaultPipeline, 0, opts), + elem: MustNewTimerElem(testBatchTimerID, testStoragePolicy, aggregation.DefaultTypes, applied.DefaultPipeline, 0, opts), metric: testBatchTimer, }, { - elem: MustNewGaugeElem(StandardIncomingMetric, testGaugeID, testStoragePolicy, aggregation.DefaultTypes, applied.DefaultPipeline, 0, opts), + elem: MustNewGaugeElem(testGaugeID, testStoragePolicy, aggregation.DefaultTypes, applied.DefaultPipeline, 0, opts), metric: testGauge, }, } @@ -346,7 +335,7 @@ func TestStandardMetricListFlushConsumingAndCollectingLocalMetrics(t *testing.T) l.Flush(flushRequest{ CutoverNanos: cutoverNanos, CutoffNanos: cutoffNanos, - }, allowEagerForwarding) + }) // Assert nothing has been flushed. flushLock.Lock() @@ -362,7 +351,7 @@ func TestStandardMetricListFlushConsumingAndCollectingLocalMetrics(t *testing.T) l.Flush(flushRequest{ CutoverNanos: cutoverNanos, CutoffNanos: cutoffNanos, - }, allowEagerForwarding) + }) var expected []testLocalMetricWithMetadata alignedStart := nowTs.Truncate(l.resolution).UnixNano() @@ -391,7 +380,7 @@ func TestStandardMetricListFlushConsumingAndCollectingLocalMetrics(t *testing.T) l.Flush(flushRequest{ CutoverNanos: cutoverNanos, CutoffNanos: cutoffNanos, - }, allowEagerForwarding) + }) // Assert nothing has been flushed. flushLock.Lock() @@ -410,7 +399,7 @@ func TestStandardMetricListFlushConsumingAndCollectingLocalMetrics(t *testing.T) l.Flush(flushRequest{ CutoverNanos: cutoverNanos, CutoffNanos: cutoffNanos, - }, allowEagerForwarding) + }) // Assert all elements have been collected. require.Equal(t, 0, l.aggregations.Len()) @@ -473,8 +462,8 @@ func TestForwardedMetricListID(t *testing.T) { require.NoError(t, err) expectedListID := metricListID{ - incomingMetricType: ForwardedIncomingMetric, - forwarded: listID, + listType: forwardedMetricListType, + forwarded: listID, } require.Equal(t, expectedListID, l.ID()) } @@ -542,8 +531,7 @@ func TestForwardedMetricListFlushConsumingAndCollectingForwardedMetrics(t *testi opts := testOptions(ctrl). SetClockOptions(clockOpts). SetAdminClient(client). - SetMaxAllowedForwardingDelayFn(maxForwardingDelayFn). - SetEnableEagerForwarding(false) + SetMaxAllowedForwardingDelayFn(maxForwardingDelayFn) listID := forwardedMetricListID{ resolution: resolution, @@ -567,7 +555,7 @@ func TestForwardedMetricListFlushConsumingAndCollectingForwardedMetrics(t *testi metric aggregated.ForwardedMetric }{ { - elem: MustNewCounterElem(ForwardedIncomingMetric, []byte("testForwardedCounter"), testStoragePolicy, aggregation.DefaultTypes, pipeline, testNumForwardedTimes, opts), + elem: MustNewCounterElem([]byte("testForwardedCounter"), testStoragePolicy, aggregation.DefaultTypes, pipeline, testNumForwardedTimes, opts), metric: aggregated.ForwardedMetric{ Type: metric.CounterType, ID: []byte("testForwardedCounter"), @@ -576,7 +564,7 @@ func TestForwardedMetricListFlushConsumingAndCollectingForwardedMetrics(t *testi }, }, { - elem: MustNewGaugeElem(ForwardedIncomingMetric, []byte("testForwardedGauge"), testStoragePolicy, aggregation.DefaultTypes, pipeline, testNumForwardedTimes, opts), + elem: MustNewGaugeElem([]byte("testForwardedGauge"), testStoragePolicy, aggregation.DefaultTypes, pipeline, testNumForwardedTimes, opts), metric: aggregated.ForwardedMetric{ Type: metric.GaugeType, ID: []byte("testForwardedGauge"), @@ -599,7 +587,7 @@ func TestForwardedMetricListFlushConsumingAndCollectingForwardedMetrics(t *testi l.Flush(flushRequest{ CutoverNanos: cutoverNanos, CutoffNanos: cutoffNanos, - }, allowEagerForwarding) + }) // Assert nothing has been flushed. flushLock.Lock() @@ -615,7 +603,7 @@ func TestForwardedMetricListFlushConsumingAndCollectingForwardedMetrics(t *testi l.Flush(flushRequest{ CutoverNanos: cutoverNanos, CutoffNanos: cutoffNanos, - }, allowEagerForwarding) + }) var expected []aggregated.ForwardedMetricWithMetadata alignedStart := (nowTs.Add(-maxLatenessAllowed)).Truncate(l.resolution).UnixNano() @@ -660,7 +648,7 @@ func TestForwardedMetricListFlushConsumingAndCollectingForwardedMetrics(t *testi l.Flush(flushRequest{ CutoverNanos: cutoverNanos, CutoffNanos: cutoffNanos, - }, allowEagerForwarding) + }) // Assert nothing has been flushed. flushLock.Lock() @@ -679,7 +667,7 @@ func TestForwardedMetricListFlushConsumingAndCollectingForwardedMetrics(t *testi l.Flush(flushRequest{ CutoverNanos: cutoverNanos, CutoffNanos: cutoffNanos, - }, allowEagerForwarding) + }) // Assert all elements have been collected. require.Equal(t, 0, l.aggregations.Len()) @@ -811,7 +799,6 @@ func validateLocalFlushed( } type flushBeforeResult struct { - beforeNanos int64 - flushType flushType - eagerForwardingMode eagerForwardingMode + beforeNanos int64 + flushType flushType } diff --git a/aggregator/map.go b/aggregator/map.go index fd18936..5a72d75 100644 --- a/aggregator/map.go +++ b/aggregator/map.go @@ -51,10 +51,19 @@ var ( errWriteNewMetricRateLimitExceeded = errors.New("write new metric rate limit is exceeded") ) +type metricCategory int + +const ( + // nolint: megacheck + unknownMetricCategory metricCategory = iota + untimedMetric + forwardedMetric +) + type entryKey struct { - incomingMetricType IncomingMetricType - metricType metric.Type - idHash hash.Hash128 + metricCategory metricCategory + metricType metric.Type + idHash hash.Hash128 } type hashedEntry struct { @@ -136,9 +145,9 @@ func (m *metricMap) AddUntimed( metadatas metadata.StagedMetadatas, ) error { key := entryKey{ - incomingMetricType: StandardIncomingMetric, - metricType: metric.Type, - idHash: hash.Murmur3Hash128(metric.ID), + metricCategory: untimedMetric, + metricType: metric.Type, + idHash: hash.Murmur3Hash128(metric.ID), } entry, err := m.findOrCreate(key) if err != nil { @@ -154,9 +163,9 @@ func (m *metricMap) AddForwarded( metadata metadata.ForwardMetadata, ) error { key := entryKey{ - incomingMetricType: ForwardedIncomingMetric, - metricType: metric.Type, - idHash: hash.Murmur3Hash128(metric.ID), + metricCategory: forwardedMetric, + metricType: metric.Type, + idHash: hash.Murmur3Hash128(metric.ID), } entry, err := m.findOrCreate(key) if err != nil { @@ -295,10 +304,10 @@ func (m *metricMap) tick(target time.Duration) tickResult { m.sleepFn(targetDeadline.Sub(now)) } } - switch entry.key.incomingMetricType { - case StandardIncomingMetric: + switch entry.key.metricCategory { + case untimedMetric: numStandardActive++ - case ForwardedIncomingMetric: + case forwardedMetric: numForwardedActive++ } if entry.entry.ShouldExpire(now) { @@ -324,11 +333,11 @@ func (m *metricMap) tick(target time.Duration) tickResult { numStandardExpired += standardExpired numForwardedExpired += forwardedExpired return tickResult{ - standard: tickResultForIncomingMetricType{ + standard: tickResultForMetricCategory{ activeEntries: numStandardActive - numStandardExpired, expiredEntries: numStandardExpired, }, - forwarded: tickResultForIncomingMetricType{ + forwarded: tickResultForMetricCategory{ activeEntries: numForwardedActive - numForwardedExpired, expiredEntries: numForwardedExpired, }, @@ -347,10 +356,10 @@ func (m *metricMap) purgeExpired( for i := range entries { if entries[i].entry.TryExpire(now) { key := entries[i].key - switch key.incomingMetricType { - case StandardIncomingMetric: + switch key.metricCategory { + case untimedMetric: numStandardExpired++ - case ForwardedIncomingMetric: + case forwardedMetric: numForwardedExpired++ } elem := m.entries[key] diff --git a/aggregator/map_test.go b/aggregator/map_test.go index 419bae9..6957047 100644 --- a/aggregator/map_test.go +++ b/aggregator/map_test.go @@ -85,9 +85,9 @@ func TestMetricMapAddUntimedNoRateLimit(t *testing.T) { // Add a counter metric and assert there is one entry afterwards. key := entryKey{ - incomingMetricType: StandardIncomingMetric, - metricType: metric.CounterType, - idHash: hash.Murmur3Hash128(testCounterID), + metricCategory: untimedMetric, + metricType: metric.CounterType, + idHash: hash.Murmur3Hash128(testCounterID), } require.NoError(t, m.AddUntimed(testCounter, policies)) require.Equal(t, 1, len(m.entries)) @@ -114,9 +114,9 @@ func TestMetricMapAddUntimedNoRateLimit(t *testing.T) { // Add a metric with different type and assert there are // now two entries. key2 := entryKey{ - incomingMetricType: StandardIncomingMetric, - metricType: metric.GaugeType, - idHash: hash.Murmur3Hash128(testCounterID), + metricCategory: untimedMetric, + metricType: metric.GaugeType, + idHash: hash.Murmur3Hash128(testCounterID), } metricWithDifferentType := unaggregated.MetricUnion{ Type: metric.GaugeType, @@ -273,9 +273,9 @@ func TestMetricMapAddForwardedNoRateLimit(t *testing.T) { Values: []float64{76109}, } key := entryKey{ - incomingMetricType: ForwardedIncomingMetric, - metricType: metric.CounterType, - idHash: hash.Murmur3Hash128(am.ID), + metricCategory: forwardedMetric, + metricType: metric.CounterType, + idHash: hash.Murmur3Hash128(am.ID), } require.NoError(t, m.AddForwarded(am, testForwardMetadata)) require.Equal(t, 1, len(m.entries)) @@ -306,9 +306,9 @@ func TestMetricMapAddForwardedNoRateLimit(t *testing.T) { CounterVal: 123, } key2 := entryKey{ - incomingMetricType: StandardIncomingMetric, - metricType: metric.CounterType, - idHash: hash.Murmur3Hash128(um.ID), + metricCategory: untimedMetric, + metricType: metric.CounterType, + idHash: hash.Murmur3Hash128(um.ID), } require.NoError(t, m.AddUntimed(um, testStagedMetadatas)) require.Equal(t, 2, len(m.entries)) @@ -328,9 +328,9 @@ func TestMetricMapAddForwardedNoRateLimit(t *testing.T) { Values: []float64{123.456}, } key3 := entryKey{ - incomingMetricType: ForwardedIncomingMetric, - metricType: metric.GaugeType, - idHash: hash.Murmur3Hash128(metricWithDifferentType.ID), + metricCategory: forwardedMetric, + metricType: metric.GaugeType, + idHash: hash.Murmur3Hash128(metricWithDifferentType.ID), } require.NoError(t, m.AddForwarded(metricWithDifferentType, testForwardMetadata)) require.Equal(t, 3, len(m.entries)) @@ -348,9 +348,9 @@ func TestMetricMapAddForwardedNoRateLimit(t *testing.T) { Values: []float64{123.456}, } key4 := entryKey{ - incomingMetricType: ForwardedIncomingMetric, - metricType: metric.GaugeType, - idHash: hash.Murmur3Hash128(metricWithDifferentID.ID), + metricCategory: forwardedMetric, + metricType: metric.GaugeType, + idHash: hash.Murmur3Hash128(metricWithDifferentID.ID), } require.NoError(t, m.AddForwarded(metricWithDifferentID, testForwardMetadata)) require.Equal(t, 4, len(m.entries)) diff --git a/aggregator/metric_type.go b/aggregator/metric_type.go deleted file mode 100644 index 8684304..0000000 --- a/aggregator/metric_type.go +++ /dev/null @@ -1,73 +0,0 @@ -// Copyright (c) 2018 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - -package aggregator - -// IncomingMetricType describes the type of an incoming metric. -type IncomingMetricType int - -const ( - // UnknownIncomingMetric is an incoming metric with unknown type. - UnknownIncomingMetric IncomingMetricType = iota - - // StandardIncomingMetric is a standard (currently untimed) incoming metric. - StandardIncomingMetric - - // ForwardedIncomingMetric is a forwarded incoming metric. - ForwardedIncomingMetric -) - -func (t IncomingMetricType) String() string { - switch t { - case StandardIncomingMetric: - return "standardIncomingMetric" - case ForwardedIncomingMetric: - return "forwardedIncomingMetric" - default: - // nolint: goconst - // Should never get here. - return "unknown" - } -} - -type outgoingMetricType int - -const ( - // localOutgoingMetric is an outgoing metric that gets flushed to backends - // locally known to the server. - localOutgoingMetric outgoingMetricType = iota - - // forwardedOutgoingMetric is an outgoing metric that gets forwarded to - // other aggregation servers for further aggregation and rollup. - forwardedOutgoingMetric -) - -func (t outgoingMetricType) String() string { - switch t { - case localOutgoingMetric: - return "localOutgoingMetric" - case forwardedOutgoingMetric: - return "forwardedOutgoingMetric" - default: - // nolint: goconst - // Should never get here. - return "unknown" - } -} diff --git a/aggregator/options.go b/aggregator/options.go index 1ab1c39..4bbf6e3 100644 --- a/aggregator/options.go +++ b/aggregator/options.go @@ -35,27 +35,21 @@ import ( "github.com/m3db/m3x/clock" "github.com/m3db/m3x/instrument" xtime "github.com/m3db/m3x/time" - - "github.com/uber-go/tally" ) var ( - defaultMetricPrefix = []byte("stats.") - defaultCounterPrefix = []byte("counts.") - defaultTimerPrefix = []byte("timers.") - defaultGaugePrefix = []byte("gauges.") - defaultEntryTTL = 24 * time.Hour - defaultEntryCheckInterval = time.Hour - defaultEntryCheckBatchPercent = 0.01 - defaultMaxTimerBatchSizePerWrite = 0 - defaultEnableEagerForwarding = true - defaultMaxForwardingWindows = 10 - defaultForwardingSourcesTTLByResolution = 10 - defaultForwardingSourcesTTLByDuration = 5 * time.Minute - defaultMaxNumCachedSourceSets = 2 - defaultDiscardNaNAggregatedValues = true - defaultResignTimeout = 5 * time.Minute - defaultDefaultStoragePolicies = []policy.StoragePolicy{ + defaultMetricPrefix = []byte("stats.") + defaultCounterPrefix = []byte("counts.") + defaultTimerPrefix = []byte("timers.") + defaultGaugePrefix = []byte("gauges.") + defaultEntryTTL = 24 * time.Hour + defaultEntryCheckInterval = time.Hour + defaultEntryCheckBatchPercent = 0.01 + defaultMaxTimerBatchSizePerWrite = 0 + defaultMaxNumCachedSourceSets = 2 + defaultDiscardNaNAggregatedValues = true + defaultResignTimeout = 5 * time.Minute + defaultDefaultStoragePolicies = []policy.StoragePolicy{ policy.NewStoragePolicy(10*time.Second, xtime.Second, 2*24*time.Hour), policy.NewStoragePolicy(time.Minute, xtime.Minute, 40*24*time.Hour), } @@ -70,10 +64,6 @@ var ( defaultBufferDurationAfterShardCutoff = time.Hour ) -// FlushIntervalFn determines the flush interval for lists containing metrics of -// the given type associated with the given resolution. -type FlushIntervalFn func(metricType IncomingMetricType, resolution time.Duration) time.Duration - // MaxAllowedForwardingDelayFn returns the maximum allowed forwarding delay given // the metric resolution and number of times the metric has been forwarded. The forwarding // delay refers to the maximum tolerated delay between when a metric is ready to be @@ -85,9 +75,6 @@ type FlushIntervalFn func(metricType IncomingMetricType, resolution time.Duratio // destination server, and ingestion delay at the destination server. type MaxAllowedForwardingDelayFn func(resolution time.Duration, numForwardedTimes int) time.Duration -// ForwardingSourcesTTLFn returns the ttl for forwarding sources given a resolution. -type ForwardingSourcesTTLFn func(resolution time.Duration) time.Duration - // Options provide a set of base and derived options for the aggregator. type Options interface { /// Read-write base options. @@ -242,12 +229,6 @@ type Options interface { // ResignTimeout returns the resign timeout. ResignTimeout() time.Duration - // SetFlushIntervalFn sets the flush interval function. - SetFlushIntervalFn(value FlushIntervalFn) Options - - // FlushIntervalFn returns the flush interval function. - FlushIntervalFn() FlushIntervalFn - // SetMaxAllowedForwardingDelayFn sets the function that determines the maximum forwarding // delay for given metric resolution and number of times the metric has been forwarded. SetMaxAllowedForwardingDelayFn(value MaxAllowedForwardingDelayFn) Options @@ -262,42 +243,6 @@ type Options interface { // MaxNumCachedSourceSets returns the maximum number of cached source sets. MaxNumCachedSourceSets() int - // SetEnableEagerForwarding determines whether eager forwarding is enabled. - SetEnableEagerForwarding(value bool) Options - - // EnableEagerForwarding return whether eager forwarding is enabled. - EnableEagerForwarding() bool - - // SetMaxAggregationWindowsForEagerForwarding sets the maximum aggregation windows - // for which we always forward metrics to notify the destination server where the forwarded - // metrics are stored of the liveliness of the source for the purpose of eager flushing. - // This is so that when the destination server has received results from all expected sources, - // it can flush the forwarded metric as soon as possible. - SetMaxAggregationWindowsForEagerForwarding(value int) Options - - // MaxAggregationWindowsForEagerForwarding returns the maximum aggregation windows - // for which we always forward metrics to notify the destination server where the forwarded - // metrics are stored of the liveliness of the source for the purpose of eager flushing. - // This is so that when the destination server has received results from all expected sources - // for an aggregation window, it can flush the forwarded metric as soon as possible. - MaxAggregationWindowsForEagerForwarding() int - - // SetForwardingSourcesTTLFn sets the ttl function for forwarding sources. - SetForwardingSourcesTTLFn(value ForwardingSourcesTTLFn) Options - - // ForwardingSourcesTTLFn returns the ttl function for forwarding sources. - ForwardingSourcesTTLFn() ForwardingSourcesTTLFn - - // SetFullForwardingLatencyHistograms sets the histograms for recording the full - // latency of forwarded metrics between when the metric is timestamped and when - // the metric is flushed to backends. - SetFullForwardingLatencyHistograms(value *ForwardingLatencyHistograms) Options - - // FullForwardingLatencyHistograms returns the histograms for recording the full - // latency of forwarded metrics between when the metric is timestamped and when - // the metric is flushed to backends. - FullForwardingLatencyHistograms() *ForwardingLatencyHistograms - // SetDiscardNaNAggregatedValues determines whether NaN aggregated values are discarded. SetDiscardNaNAggregatedValues(value bool) Options @@ -367,14 +312,9 @@ type options struct { flushTimesManager FlushTimesManager electionManager ElectionManager resignTimeout time.Duration - flushIntervalFn FlushIntervalFn maxAllowedForwardingDelayFn MaxAllowedForwardingDelayFn maxNumCachedSourceSets int - enableEagerForwarding bool - maxForwardingWindows int - forwardingSourcesTTLFn ForwardingSourcesTTLFn discardNaNAggregatedValues bool - forwardingLatencyHistograms *ForwardingLatencyHistograms entryPool EntryPool counterElemPool CounterElemPool timerElemPool TimerElemPool @@ -413,14 +353,9 @@ func NewOptions() Options { maxTimerBatchSizePerWrite: defaultMaxTimerBatchSizePerWrite, defaultStoragePolicies: defaultDefaultStoragePolicies, resignTimeout: defaultResignTimeout, - flushIntervalFn: defaultFlushIntervalFn, maxAllowedForwardingDelayFn: defaultMaxAllowedForwardingDelayFn, maxNumCachedSourceSets: defaultMaxNumCachedSourceSets, - enableEagerForwarding: defaultEnableEagerForwarding, - maxForwardingWindows: defaultMaxForwardingWindows, - forwardingSourcesTTLFn: defaultForwardingSourcesTTLFn, discardNaNAggregatedValues: defaultDiscardNaNAggregatedValues, - forwardingLatencyHistograms: NewForwardingLatencyHistograms(tally.NoopScope, defaultForwardingLatencyBucketsFn), } // Initialize pools. @@ -686,16 +621,6 @@ func (o *options) ResignTimeout() time.Duration { return o.resignTimeout } -func (o *options) SetFlushIntervalFn(value FlushIntervalFn) Options { - opts := *o - opts.flushIntervalFn = value - return &opts -} - -func (o *options) FlushIntervalFn() FlushIntervalFn { - return o.flushIntervalFn -} - func (o *options) SetMaxAllowedForwardingDelayFn(value MaxAllowedForwardingDelayFn) Options { opts := *o opts.maxAllowedForwardingDelayFn = value @@ -716,46 +641,6 @@ func (o *options) MaxNumCachedSourceSets() int { return o.maxNumCachedSourceSets } -func (o *options) SetEnableEagerForwarding(value bool) Options { - opts := *o - opts.enableEagerForwarding = value - return &opts -} - -func (o *options) EnableEagerForwarding() bool { - return o.enableEagerForwarding -} - -func (o *options) SetMaxAggregationWindowsForEagerForwarding(value int) Options { - opts := *o - opts.maxForwardingWindows = value - return &opts -} - -func (o *options) MaxAggregationWindowsForEagerForwarding() int { - return o.maxForwardingWindows -} - -func (o *options) SetForwardingSourcesTTLFn(value ForwardingSourcesTTLFn) Options { - opts := *o - opts.forwardingSourcesTTLFn = value - return &opts -} - -func (o *options) ForwardingSourcesTTLFn() ForwardingSourcesTTLFn { - return o.forwardingSourcesTTLFn -} - -func (o *options) SetFullForwardingLatencyHistograms(value *ForwardingLatencyHistograms) Options { - opts := *o - opts.forwardingLatencyHistograms = value - return &opts -} - -func (o *options) FullForwardingLatencyHistograms() *ForwardingLatencyHistograms { - return o.forwardingLatencyHistograms -} - func (o *options) SetDiscardNaNAggregatedValues(value bool) Options { opts := *o opts.discardNaNAggregatedValues = value @@ -831,17 +716,17 @@ func (o *options) initPools() { o.counterElemPool = NewCounterElemPool(nil) o.counterElemPool.Init(func() *CounterElem { - return MustNewCounterElem(UnknownIncomingMetric, nil, policy.EmptyStoragePolicy, aggregation.DefaultTypes, applied.DefaultPipeline, 0, o) + return MustNewCounterElem(nil, policy.EmptyStoragePolicy, aggregation.DefaultTypes, applied.DefaultPipeline, 0, o) }) o.timerElemPool = NewTimerElemPool(nil) o.timerElemPool.Init(func() *TimerElem { - return MustNewTimerElem(UnknownIncomingMetric, nil, policy.EmptyStoragePolicy, aggregation.DefaultTypes, applied.DefaultPipeline, 0, o) + return MustNewTimerElem(nil, policy.EmptyStoragePolicy, aggregation.DefaultTypes, applied.DefaultPipeline, 0, o) }) o.gaugeElemPool = NewGaugeElemPool(nil) o.gaugeElemPool.Init(func() *GaugeElem { - return MustNewGaugeElem(UnknownIncomingMetric, nil, policy.EmptyStoragePolicy, aggregation.DefaultTypes, applied.DefaultPipeline, 0, o) + return MustNewGaugeElem(nil, policy.EmptyStoragePolicy, aggregation.DefaultTypes, applied.DefaultPipeline, 0, o) }) } @@ -876,29 +761,9 @@ func (o *options) computeFullGaugePrefix() { o.fullGaugePrefix = fullGaugePrefix } -func defaultFlushIntervalFn(_ IncomingMetricType, resolution time.Duration) time.Duration { - return resolution -} - func defaultMaxAllowedForwardingDelayFn( resolution time.Duration, numForwardedTimes int, ) time.Duration { return resolution * time.Duration(numForwardedTimes) } - -func defaultForwardingSourcesTTLFn(resolution time.Duration) time.Duration { - dur := time.Duration(defaultForwardingSourcesTTLByResolution) * resolution - if dur >= defaultForwardingSourcesTTLByDuration { - return dur - } - return defaultForwardingSourcesTTLByDuration -} - -func defaultForwardingLatencyBucketsFn( - key ForwardingLatencyBucketKey, - numLatencyBuckets int, -) tally.Buckets { - bucketWidth := 2 * key.Resolution / time.Duration(numLatencyBuckets) - return tally.MustMakeLinearDurationBuckets(0, bucketWidth, numLatencyBuckets) -} diff --git a/aggregator/options_test.go b/aggregator/options_test.go index 6097940..1f12ff8 100644 --- a/aggregator/options_test.go +++ b/aggregator/options_test.go @@ -34,7 +34,6 @@ import ( "github.com/golang/mock/gomock" "github.com/stretchr/testify/require" - "github.com/uber-go/tally" ) func validateDerivedPrefix( @@ -177,42 +176,6 @@ func TestSetEntryCheckBatchPercent(t *testing.T) { require.Equal(t, value, o.EntryCheckBatchPercent()) } -func TestSetFlushIntervalFn(t *testing.T) { - value := func(metricType IncomingMetricType, resolution time.Duration) time.Duration { - if metricType == StandardIncomingMetric { - return time.Second - } - return time.Minute - } - o := NewOptions().SetFlushIntervalFn(value) - require.Equal(t, time.Second, o.FlushIntervalFn()(StandardIncomingMetric, time.Second)) - require.Equal(t, time.Minute, o.FlushIntervalFn()(ForwardedIncomingMetric, time.Second)) -} - -func TestSetEnableEagerForwarding(t *testing.T) { - o := NewOptions().SetEnableEagerForwarding(false) - require.Equal(t, false, o.EnableEagerForwarding()) -} - -func TestSetMaxAggregationWindowsForEagerForwarding(t *testing.T) { - o := NewOptions().SetMaxAggregationWindowsForEagerForwarding(5) - require.Equal(t, 5, o.MaxAggregationWindowsForEagerForwarding()) -} - -func TestSetForwardingSourcesTTLFn(t *testing.T) { - value := func(resolution time.Duration) time.Duration { - return resolution + time.Second - } - o := NewOptions().SetForwardingSourcesTTLFn(value) - require.Equal(t, 2*time.Second, o.ForwardingSourcesTTLFn()(time.Second)) -} - -func TestSetFullForwardingLatencyHistograms(t *testing.T) { - value := NewForwardingLatencyHistograms(tally.NoopScope, nil) - o := NewOptions().SetFullForwardingLatencyHistograms(value) - require.NotNil(t, o.FullForwardingLatencyHistograms()) -} - func TestSetEntryPool(t *testing.T) { value := NewEntryPool(nil) o := NewOptions().SetEntryPool(value) diff --git a/aggregator/tick_result.go b/aggregator/tick_result.go index f636621..0e752cb 100644 --- a/aggregator/tick_result.go +++ b/aggregator/tick_result.go @@ -22,16 +22,16 @@ package aggregator import "time" -type tickResultForIncomingMetricType struct { +type tickResultForMetricCategory struct { activeEntries int expiredEntries int activeElems map[time.Duration]int } -func (r *tickResultForIncomingMetricType) merge( - other tickResultForIncomingMetricType, -) tickResultForIncomingMetricType { - res := tickResultForIncomingMetricType{ +func (r *tickResultForMetricCategory) merge( + other tickResultForMetricCategory, +) tickResultForMetricCategory { + res := tickResultForMetricCategory{ activeEntries: r.activeEntries + other.activeEntries, expiredEntries: r.expiredEntries + other.expiredEntries, } @@ -51,8 +51,8 @@ func (r *tickResultForIncomingMetricType) merge( } type tickResult struct { - standard tickResultForIncomingMetricType - forwarded tickResultForIncomingMetricType + standard tickResultForMetricCategory + forwarded tickResultForMetricCategory } // merge merges two results. Both input results may become invalid after merge is called. diff --git a/aggregator/timer_elem.gen.go b/aggregator/timer_elem.gen.go index 7c230e8..402d969 100644 --- a/aggregator/timer_elem.gen.go +++ b/aggregator/timer_elem.gen.go @@ -51,39 +51,9 @@ import ( type lockedTimerAggregation struct { sync.Mutex - closed bool - - // sourcesReady is only used for elements receiving forwarded metrics. - // It determines whether the current aggregation can use its source set - // to determine whether it has received data from all forwarding sources - // to perform eager forwarding if enabled. - sourcesReady bool - - // expectedSources is only used for elements receiving forwarded metrics. - // It keeps track of all the sources the current aggregation expect to receive - // data from. - expectedSources *bitset.BitSet - - // seenSources keeps track of all the sources the current aggregation has - // seen so far. - seenSources *bitset.BitSet - - // consumeState is only used for elements receiving forwarded metrics. It - // describes whether the current aggregation is ready to be consumed or has - // been consumed. This in turn determines whether the aggregation can be - // eagerly consumed, or should be skipped during consumption. - consumeState consumeState - aggregation timerAggregation -} - -func (lockedAgg *lockedTimerAggregation) close() { - if lockedAgg.closed { - return - } - lockedAgg.closed = true - lockedAgg.expectedSources = nil - lockedAgg.seenSources = nil - lockedAgg.aggregation.Close() + closed bool + sourcesSeen *bitset.BitSet + aggregation timerAggregation } type timedTimer struct { @@ -109,7 +79,6 @@ type TimerElem struct { // NewTimerElem creates a new element for the given metric type. func NewTimerElem( - incomingMetricType IncomingMetricType, id id.RawID, sp policy.StoragePolicy, aggTypes maggregation.Types, @@ -121,7 +90,7 @@ func NewTimerElem( elemBase: newElemBase(opts), values: make([]timedTimer, 0, defaultNumAggregations), // in most cases values will have two entries } - if err := e.ResetSetData(incomingMetricType, id, sp, aggTypes, pipeline, numForwardedTimes); err != nil { + if err := e.ResetSetData(id, sp, aggTypes, pipeline, numForwardedTimes); err != nil { return nil, err } return e, nil @@ -129,7 +98,6 @@ func NewTimerElem( // MustNewTimerElem creates a new element, or panics if the input is invalid. func MustNewTimerElem( - incomingMetricType IncomingMetricType, id id.RawID, sp policy.StoragePolicy, aggTypes maggregation.Types, @@ -137,7 +105,7 @@ func MustNewTimerElem( numForwardedTimes int, opts Options, ) *TimerElem { - elem, err := NewTimerElem(incomingMetricType, id, sp, aggTypes, pipeline, numForwardedTimes, opts) + elem, err := NewTimerElem(id, sp, aggTypes, pipeline, numForwardedTimes, opts) if err != nil { panic(fmt.Errorf("unable to create element: %v", err)) } @@ -146,7 +114,6 @@ func MustNewTimerElem( // ResetSetData resets the element and sets data. func (e *TimerElem) ResetSetData( - incomingMetricType IncomingMetricType, id id.RawID, sp policy.StoragePolicy, aggTypes maggregation.Types, @@ -157,7 +124,7 @@ func (e *TimerElem) ResetSetData( if useDefaultAggregation { aggTypes = e.DefaultAggregationTypes(e.aggTypesOpts) } - if err := e.elemBase.resetSetData(incomingMetricType, id, sp, aggTypes, useDefaultAggregation, pipeline, numForwardedTimes); err != nil { + if err := e.elemBase.resetSetData(id, sp, aggTypes, useDefaultAggregation, pipeline, numForwardedTimes); err != nil { return err } if err := e.timerElemBase.ResetSetData(e.aggTypesOpts, aggTypes, useDefaultAggregation); err != nil { @@ -182,7 +149,7 @@ func (e *TimerElem) ResetSetData( // AddUnion adds a metric value union at a given timestamp. func (e *TimerElem) AddUnion(timestamp time.Time, mu unaggregated.MetricUnion) error { alignedStart := timestamp.Truncate(e.sp.Resolution().Window).UnixNano() - lockedAgg, err := e.findOrCreate(alignedStart, sourcesOptions{}) + lockedAgg, err := e.findOrCreate(alignedStart, createAggregationOptions{}) if err != nil { return err } @@ -201,7 +168,7 @@ func (e *TimerElem) AddUnion(timestamp time.Time, mu unaggregated.MetricUnion) e // same aggregation, the incoming value is discarded. func (e *TimerElem) AddUnique(timestamp time.Time, values []float64, sourceID uint32) error { alignedStart := timestamp.Truncate(e.sp.Resolution().Window).UnixNano() - lockedAgg, err := e.findOrCreate(alignedStart, sourcesOptions{updateSources: true, source: sourceID}) + lockedAgg, err := e.findOrCreate(alignedStart, createAggregationOptions{initSourceSet: true}) if err != nil { return err } @@ -211,26 +178,11 @@ func (e *TimerElem) AddUnique(timestamp time.Time, values []float64, sourceID ui return errAggregationClosed } source := uint(sourceID) - if lockedAgg.seenSources.Test(source) { + if lockedAgg.sourcesSeen.Test(source) { lockedAgg.Unlock() return errDuplicateForwardingSource } - lockedAgg.seenSources.Set(source) - if lockedAgg.sourcesReady { - // If the sources are ready, the expected sources will be a pre-filled - // bitset populated with sources the aggregation is expected to see data from. - // As such, we need to clear the source bit in the expected sources. - if lockedAgg.expectedSources.Test(source) { - // This source is never seen before and is in the expected source list, - // as a result, we need to clear the source bit. - lockedAgg.expectedSources.Clear(source) - if lockedAgg.expectedSources.None() { - lockedAgg.consumeState = readyToConsume - } - } - // New sources that are not in the expected source list are still allowed - // to go through. - } + lockedAgg.sourcesSeen.Set(source) for _, v := range values { lockedAgg.aggregation.Add(v) } @@ -245,7 +197,6 @@ func (e *TimerElem) AddUnique(timestamp time.Time, values []float64, sourceID ui // to avoid race conditions. func (e *TimerElem) Consume( targetNanos int64, - eagerForwardingMode eagerForwardingMode, isEarlierThanFn isEarlierThanFn, timestampNanosFn timestampNanosFn, flushLocalFn flushLocalMetricFn, @@ -261,8 +212,7 @@ func (e *TimerElem) Consume( idx := 0 for range e.values { // Bail as soon as the timestamp is no later than the target time. - timeNanos := timestampNanosFn(e.values[idx].startAtNanos, resolution) - if !isEarlierThanFn(timeNanos, targetNanos) { + if !isEarlierThanFn(e.values[idx].startAtNanos, resolution, targetNanos) { break } idx++ @@ -280,71 +230,31 @@ func (e *TimerElem) Consume( e.values = e.values[:n] } canCollect := len(e.values) == 0 && e.tombstoned - - // Additionally for elements receiving forwarded metrics and sending aggregated metrics - // to local backends, we also check if any aggregations are ready to be consumed. We however - // do not remove the aggregations as we do for aggregations whose timestamps are old enough, - // since for aggregations receiving forwarded metrics that are marked "consume ready", it is - // possible that metrics still go to the such aggregation bucket after they are marked "consume - // ready" due to delayed source re-delivery or new sources showing up, and removing such - // aggregation prematurely would mean the values from the delayed sources and/or new sources - // would be considered as the aggregated value for such aggregation bucket, which is incorrect. - // By keeping such aggregation buckets and only removing them when they are considered old enough - // (i.e., when their timestmaps are earlier than the target timestamp), we ensure no metrics may - // go to such aggregation buckets after they are consumed and therefore avoid the aformentioned - // problem. - aggregationIdxToCloseUntil := len(e.toConsume) - if e.incomingMetricType == ForwardedIncomingMetric && e.isSourcesSetReadyWithElemLock() { - e.maybeRefreshSourcesSetWithElemLock() - // We only attempt to consume if the outgoing metrics type is local instead of forwarded - // and eager forwarding is allowed (eager forwarding may be enabled and disallowed when - // the node is a follower and allowed when the node is a leader. This is because forwarded - // metrics are sent in batches and can only be sent when all sources in the same shard have - // been consumed, and as such is not well suited for pre-emptive consumption. - if e.outgoingMetricType() == localOutgoingMetric && eagerForwardingMode == allowEagerForwarding { - for i := 0; i < len(e.values); i++ { - // NB: This makes the logic easier to understand but it would be more efficient to use - // an atomic here to avoid locking aggregations. - e.values[i].lockedAgg.Lock() - if e.values[i].lockedAgg.consumeState == readyToConsume { - e.toConsume = append(e.toConsume, e.values[i]) - e.values[i].lockedAgg.consumeState = consuming - } - e.values[i].lockedAgg.Unlock() - } - } - } e.Unlock() // Process the aggregations that are ready for consumption. for i := range e.toConsume { timeNanos := timestampNanosFn(e.toConsume[i].startAtNanos, resolution) e.toConsume[i].lockedAgg.Lock() - if e.toConsume[i].lockedAgg.consumeState != consumed { - e.processValueWithAggregationLock( - timeNanos, eagerForwardingMode, e.toConsume[i].lockedAgg, - flushLocalFn, flushForwardedFn, - ) - } - e.toConsume[i].lockedAgg.consumeState = consumed - if i < aggregationIdxToCloseUntil { - if e.toConsume[i].lockedAgg.seenSources != nil { - e.sourcesLock.Lock() - // This is to make sure there aren't too many cached source sets taking up - // too much space. - if len(e.cachedSourceSets) < e.opts.MaxNumCachedSourceSets() { - e.cachedSourceSets = append(e.cachedSourceSets, e.toConsume[i].lockedAgg.seenSources) - e.toConsume[i].lockedAgg.seenSources = nil - } - e.sourcesLock.Unlock() + e.processValueWithAggregationLock(timeNanos, e.toConsume[i].lockedAgg, flushLocalFn, flushForwardedFn) + // Closes the aggregation object after it's processed. + e.toConsume[i].lockedAgg.closed = true + e.toConsume[i].lockedAgg.aggregation.Close() + if e.toConsume[i].lockedAgg.sourcesSeen != nil { + e.cachedSourceSetsLock.Lock() + // This is to make sure there aren't too many cached source sets taking up + // too much space. + if len(e.cachedSourceSets) < e.opts.MaxNumCachedSourceSets() { + e.cachedSourceSets = append(e.cachedSourceSets, e.toConsume[i].lockedAgg.sourcesSeen) } - e.toConsume[i].lockedAgg.close() + e.cachedSourceSetsLock.Unlock() + e.toConsume[i].lockedAgg.sourcesSeen = nil } e.toConsume[i].lockedAgg.Unlock() e.toConsume[i].Reset() } - if e.outgoingMetricType() == forwardedOutgoingMetric { + if e.parsedPipeline.HasRollup { forwardedAggregationKey, _ := e.ForwardedAggregationKey() onForwardedFlushedFn(e.onForwardedAggregationWrittenFn, forwardedAggregationKey) } @@ -364,15 +274,14 @@ func (e *TimerElem) Close() { e.parsedPipeline = parsedPipeline{} e.writeForwardedMetricFn = nil e.onForwardedAggregationWrittenFn = nil - e.sourcesHeartbeat = nil - e.sourcesSet = nil for idx := range e.cachedSourceSets { e.cachedSourceSets[idx] = nil } e.cachedSourceSets = nil for idx := range e.values { // Close the underlying aggregation objects. - e.values[idx].lockedAgg.close() + e.values[idx].lockedAgg.sourcesSeen = nil + e.values[idx].lockedAgg.aggregation.Close() e.values[idx].Reset() } e.values = e.values[:0] @@ -393,16 +302,13 @@ func (e *TimerElem) Close() { // if it doesn't exist. func (e *TimerElem) findOrCreate( alignedStart int64, - sourcesOpts sourcesOptions, + createOpts createAggregationOptions, ) (*lockedTimerAggregation, error) { e.RLock() if e.closed { e.RUnlock() return nil, errElemClosed } - if sourcesOpts.updateSources { - e.updateSources(sourcesOpts.source) - } idx, found := e.indexOfWithLock(alignedStart) if found { agg := e.values[idx].lockedAgg @@ -428,36 +334,24 @@ func (e *TimerElem) findOrCreate( e.values = append(e.values, timedTimer{}) copy(e.values[idx+1:numValues+1], e.values[idx:numValues]) - var ( - sourcesReady = e.isSourcesSetReadyWithElemLock() - expectedSources *bitset.BitSet - seenSources *bitset.BitSet - ) - if sourcesOpts.updateSources { - e.sourcesLock.Lock() - // If the sources set is ready, we clone it ane use the clone to - // determine when we have received from all the expected sources. - if sourcesReady { - expectedSources = e.sourcesSet.Clone() - } + var sourcesSeen *bitset.BitSet + if createOpts.initSourceSet { + e.cachedSourceSetsLock.Lock() if numCachedSourceSets := len(e.cachedSourceSets); numCachedSourceSets > 0 { - seenSources = e.cachedSourceSets[numCachedSourceSets-1] + sourcesSeen = e.cachedSourceSets[numCachedSourceSets-1] e.cachedSourceSets[numCachedSourceSets-1] = nil e.cachedSourceSets = e.cachedSourceSets[:numCachedSourceSets-1] - seenSources.ClearAll() + sourcesSeen.ClearAll() } else { - seenSources = bitset.New(defaultNumSources) + sourcesSeen = bitset.New(defaultNumSources) } - e.sourcesLock.Unlock() + e.cachedSourceSetsLock.Unlock() } - e.values[idx] = timedTimer{ startAtNanos: alignedStart, lockedAgg: &lockedTimerAggregation{ - sourcesReady: sourcesReady, - expectedSources: expectedSources, - seenSources: seenSources, - aggregation: e.NewAggregation(e.opts, e.aggOpts), + sourcesSeen: sourcesSeen, + aggregation: e.NewAggregation(e.opts, e.aggOpts), }, } agg := e.values[idx].lockedAgg @@ -496,14 +390,10 @@ func (e *TimerElem) indexOfWithLock(alignedStart int64) (int, bool) { func (e *TimerElem) processValueWithAggregationLock( timeNanos int64, - eagerForwardingMode eagerForwardingMode, lockedAgg *lockedTimerAggregation, flushLocalFn flushLocalMetricFn, flushForwardedFn flushForwardedMetricFn, ) { - if lockedAgg.aggregation.Count() == 0 { - return - } var ( fullPrefix = e.FullPrefix(e.opts) transformations = e.parsedPipeline.Transformations @@ -533,7 +423,7 @@ func (e *TimerElem) processValueWithAggregationLock( if discardNaNValues && math.IsNaN(value) { continue } - if e.outgoingMetricType() == localOutgoingMetric { + if !e.parsedPipeline.HasRollup { flushLocalFn(fullPrefix, e.id, e.TypeStringFor(e.aggTypesOpts, aggType), timeNanos, value, e.sp) } else { forwardedAggregationKey, _ := e.ForwardedAggregationKey() @@ -541,73 +431,4 @@ func (e *TimerElem) processValueWithAggregationLock( } } e.lastConsumedAtNanos = timeNanos - - // Emit latency metrics for forwarded metrics going to local backends - // when eager forwarding is allowed. - if eagerForwardingMode == allowEagerForwarding && - e.incomingMetricType == ForwardedIncomingMetric && - e.outgoingMetricType() == localOutgoingMetric { - e.opts.FullForwardingLatencyHistograms().RecordDuration( - e.sp.Resolution().Window, - e.numForwardedTimes, - time.Duration(e.nowFn().UnixNano()-timeNanos), - ) - } -} - -func (e *TimerElem) outgoingMetricType() outgoingMetricType { - if !e.parsedPipeline.HasRollup { - return localOutgoingMetric - } - return forwardedOutgoingMetric -} - -func (e *TimerElem) isSourcesSetReadyWithElemLock() bool { - if !e.opts.EnableEagerForwarding() { - return false - } - if e.buildingSourcesAtNanos == 0 { - return false - } - // NB: Allow TTL for the source set to build up. - return e.nowFn().UnixNano() >= e.buildingSourcesAtNanos+e.sourcesTTLNanos -} - -func (e *TimerElem) maybeRefreshSourcesSetWithElemLock() { - if !e.opts.EnableEagerForwarding() { - return - } - nowNanos := e.nowFn().UnixNano() - if nowNanos-e.lastSourcesRefreshNanos < e.sourcesTTLNanos { - return - } - e.sourcesLock.Lock() - for sourceID, lastHeartbeatNanos := range e.sourcesHeartbeat { - if nowNanos-lastHeartbeatNanos >= e.sourcesTTLNanos { - delete(e.sourcesHeartbeat, sourceID) - e.sourcesSet.Clear(uint(sourceID)) - } - } - e.lastSourcesRefreshNanos = nowNanos - e.sourcesLock.Unlock() -} - -func (e *TimerElem) updateSources(source uint32) { - if !e.opts.EnableEagerForwarding() { - return - } - nowNanos := e.nowFn().UnixNano() - e.sourcesLock.Lock() - // First time a source is received. - if e.sourcesHeartbeat == nil { - e.sourcesHeartbeat = make(map[uint32]int64, defaultNumSources) - e.sourcesSet = bitset.New(defaultNumSources) - e.buildingSourcesAtNanos = nowNanos - e.lastSourcesRefreshNanos = nowNanos - } - if v, exists := e.sourcesHeartbeat[source]; !exists || v < nowNanos { - e.sourcesHeartbeat[source] = nowNanos - } - e.sourcesSet.Set(uint(source)) - e.sourcesLock.Unlock() } diff --git a/integration/setup.go b/integration/setup.go index 4020c43..cdd793a 100644 --- a/integration/setup.go +++ b/integration/setup.go @@ -183,19 +183,19 @@ func newTestServerSetup(t *testing.T, opts testServerOptions) *testServerSetup { counterElemPool := aggregator.NewCounterElemPool(nil) aggregatorOpts = aggregatorOpts.SetCounterElemPool(counterElemPool) counterElemPool.Init(func() *aggregator.CounterElem { - return aggregator.MustNewCounterElem(aggregator.UnknownIncomingMetric, nil, policy.EmptyStoragePolicy, aggregation.DefaultTypes, applied.DefaultPipeline, 0, aggregatorOpts) + return aggregator.MustNewCounterElem(nil, policy.EmptyStoragePolicy, aggregation.DefaultTypes, applied.DefaultPipeline, 0, aggregatorOpts) }) timerElemPool := aggregator.NewTimerElemPool(nil) aggregatorOpts = aggregatorOpts.SetTimerElemPool(timerElemPool) timerElemPool.Init(func() *aggregator.TimerElem { - return aggregator.MustNewTimerElem(aggregator.UnknownIncomingMetric, nil, policy.EmptyStoragePolicy, aggregation.DefaultTypes, applied.DefaultPipeline, 0, aggregatorOpts) + return aggregator.MustNewTimerElem(nil, policy.EmptyStoragePolicy, aggregation.DefaultTypes, applied.DefaultPipeline, 0, aggregatorOpts) }) gaugeElemPool := aggregator.NewGaugeElemPool(nil) aggregatorOpts = aggregatorOpts.SetGaugeElemPool(gaugeElemPool) gaugeElemPool.Init(func() *aggregator.GaugeElem { - return aggregator.MustNewGaugeElem(aggregator.UnknownIncomingMetric, nil, policy.EmptyStoragePolicy, aggregation.DefaultTypes, applied.DefaultPipeline, 0, aggregatorOpts) + return aggregator.MustNewGaugeElem(nil, policy.EmptyStoragePolicy, aggregation.DefaultTypes, applied.DefaultPipeline, 0, aggregatorOpts) }) return &testServerSetup{ diff --git a/services/m3aggregator/config/aggregator.go b/services/m3aggregator/config/aggregator.go index e174dfb..d8f2dbc 100644 --- a/services/m3aggregator/config/aggregator.go +++ b/services/m3aggregator/config/aggregator.go @@ -47,8 +47,6 @@ import ( "github.com/m3db/m3x/pool" "github.com/m3db/m3x/retry" "github.com/m3db/m3x/sync" - - "github.com/uber-go/tally" ) var ( @@ -177,6 +175,7 @@ func (c *AggregatorConfiguration) NewAggregatorOptions( opts = opts.SetStreamOptions(streamOpts) // Set administrative client. + // TODO(xichen): client retry threshold likely needs to be low for faster retries. iOpts = instrumentOpts.SetMetricsScope(scope.SubScope("client")) adminClient, err := c.Client.NewAdminClient(client, clock.NewOptions(), iOpts) if err != nil { @@ -271,10 +270,11 @@ func (c *AggregatorConfiguration) NewAggregatorOptions( } opts = opts.SetFlushHandler(flushHandler) - // Apply forwarding configuration. + // Set max allowed forwarding delay function. jitterEnabled := flushManagerOpts.JitterEnabled() maxJitterFn := flushManagerOpts.MaxJitterFn() - opts = c.Forwarding.ApplyForwardingConfiguration(opts, jitterEnabled, maxJitterFn, scope) + maxAllowedForwardingDelayFn := c.Forwarding.MaxAllowedForwardingDelayFn(jitterEnabled, maxJitterFn) + opts = opts.SetMaxAllowedForwardingDelayFn(maxAllowedForwardingDelayFn) // Set entry options. if c.EntryTTL != 0 { @@ -295,7 +295,7 @@ func (c *AggregatorConfiguration) NewAggregatorOptions( copy(storagePolicies, c.DefaultStoragePolicies) opts = opts.SetDefaultStoragePolicies(storagePolicies) - // Set max number of cached source sets. + // Set cached source sets options. if c.MaxNumCachedSourceSets != nil { opts = opts.SetMaxNumCachedSourceSets(*c.MaxNumCachedSourceSets) } @@ -311,7 +311,7 @@ func (c *AggregatorConfiguration) NewAggregatorOptions( counterElemPool := aggregator.NewCounterElemPool(counterElemPoolOpts) opts = opts.SetCounterElemPool(counterElemPool) counterElemPool.Init(func() *aggregator.CounterElem { - return aggregator.MustNewCounterElem(aggregator.UnknownIncomingMetric, nil, policy.EmptyStoragePolicy, aggregation.DefaultTypes, applied.DefaultPipeline, 0, opts) + return aggregator.MustNewCounterElem(nil, policy.EmptyStoragePolicy, aggregation.DefaultTypes, applied.DefaultPipeline, 0, opts) }) // Set timer elem pool. @@ -320,7 +320,7 @@ func (c *AggregatorConfiguration) NewAggregatorOptions( timerElemPool := aggregator.NewTimerElemPool(timerElemPoolOpts) opts = opts.SetTimerElemPool(timerElemPool) timerElemPool.Init(func() *aggregator.TimerElem { - return aggregator.MustNewTimerElem(aggregator.UnknownIncomingMetric, nil, policy.EmptyStoragePolicy, aggregation.DefaultTypes, applied.DefaultPipeline, 0, opts) + return aggregator.MustNewTimerElem(nil, policy.EmptyStoragePolicy, aggregation.DefaultTypes, applied.DefaultPipeline, 0, opts) }) // Set gauge elem pool. @@ -329,7 +329,7 @@ func (c *AggregatorConfiguration) NewAggregatorOptions( gaugeElemPool := aggregator.NewGaugeElemPool(gaugeElemPoolOpts) opts = opts.SetGaugeElemPool(gaugeElemPool) gaugeElemPool.Init(func() *aggregator.GaugeElem { - return aggregator.MustNewGaugeElem(aggregator.UnknownIncomingMetric, nil, policy.EmptyStoragePolicy, aggregation.DefaultTypes, applied.DefaultPipeline, 0, opts) + return aggregator.MustNewGaugeElem(nil, policy.EmptyStoragePolicy, aggregation.DefaultTypes, applied.DefaultPipeline, 0, opts) }) // Set entry pool. @@ -436,53 +436,7 @@ func (c placementManagerConfiguration) NewPlacementManager( type forwardingConfiguration struct { // MaxSingleDelay is the maximum delay for a single forward step. - MaxSingleDelay time.Duration `yaml:"maxSingleDelay" validate:"nonzero"` - EnableEagerForwarding *bool `yaml:"enableEagerForwarding"` - MaxAggregationWindows *int `yaml:"maxAggregationWindows"` - SourcesTTL *forwardingSourcesTTLConfiguration `yaml:"sourcesTTL"` - FlushIntervalOverrides map[time.Duration]time.Duration `yaml:"flushIntervalOverrides"` -} - -func (c forwardingConfiguration) ApplyForwardingConfiguration( - opts aggregator.Options, - jitterEnabled bool, - maxJitterFn aggregator.FlushJitterFn, - scope tally.Scope, -) aggregator.Options { - // Set max allowed forwarding delay function. - maxAllowedForwardingDelayFn := c.MaxAllowedForwardingDelayFn(jitterEnabled, maxJitterFn) - opts = opts.SetMaxAllowedForwardingDelayFn(maxAllowedForwardingDelayFn) - - // Set whether to enable eager forwarding. - if c.EnableEagerForwarding != nil { - opts = opts.SetEnableEagerForwarding(*c.EnableEagerForwarding) - } - - // Set flush interval function. - flushIntervalFn := c.FlushIntervalFn() - opts = opts.SetFlushIntervalFn(flushIntervalFn) - - // Set max number of aggregation windows for eager forwarding. - if c.MaxAggregationWindows != nil { - opts = opts.SetMaxAggregationWindowsForEagerForwarding(*c.MaxAggregationWindows) - } - - // Set forwarding sources TTL function. - if c.SourcesTTL != nil { - opts = opts.SetForwardingSourcesTTLFn(c.SourcesTTL.ForwardingSourcesTTLFn()) - } - - // Set full forwarding latency histograms. - latencyScope := scope.Tagged(map[string]string{ - "latency-type": "full", - }) - latencyBucketsFn := func(key aggregator.ForwardingLatencyBucketKey, numLatencyBuckets int) tally.Buckets { - maxForwardingDelayAllowed := maxAllowedForwardingDelayFn(key.Resolution, key.NumForwardedTimes) - latencyBucketSize := maxForwardingDelayAllowed * 2 / time.Duration(numLatencyBuckets) - return tally.MustMakeLinearDurationBuckets(0, latencyBucketSize, numLatencyBuckets) - } - latencyHistograms := aggregator.NewForwardingLatencyHistograms(latencyScope, latencyBucketsFn) - return opts.SetFullForwardingLatencyHistograms(latencyHistograms) + MaxSingleDelay time.Duration `yaml:"maxSingleDelay"` } func (c forwardingConfiguration) MaxAllowedForwardingDelayFn( @@ -501,34 +455,6 @@ func (c forwardingConfiguration) MaxAllowedForwardingDelayFn( } } -func (c forwardingConfiguration) FlushIntervalFn() aggregator.FlushIntervalFn { - return func(metricType aggregator.IncomingMetricType, resolution time.Duration) time.Duration { - if metricType != aggregator.ForwardedIncomingMetric { - return resolution - } - override, exists := c.FlushIntervalOverrides[resolution] - if !exists { - return resolution - } - return override - } -} - -type forwardingSourcesTTLConfiguration struct { - ByResolution int `yaml:"byResolution"` - ByDuration time.Duration `yaml:"byDuration"` -} - -func (c forwardingSourcesTTLConfiguration) ForwardingSourcesTTLFn() aggregator.ForwardingSourcesTTLFn { - return func(resolution time.Duration) time.Duration { - dur := resolution * time.Duration(c.ByResolution) - if dur >= c.ByDuration { - return dur - } - return c.ByDuration - } -} - type flushTimesManagerConfiguration struct { // KV Configuration. KVConfig kv.OverrideConfiguration `yaml:"kvConfig"` diff --git a/services/m3aggregator/config/aggregator_test.go b/services/m3aggregator/config/aggregator_test.go index 4e7c173..1672519 100644 --- a/services/m3aggregator/config/aggregator_test.go +++ b/services/m3aggregator/config/aggregator_test.go @@ -24,8 +24,6 @@ import ( "testing" "time" - "github.com/m3db/m3aggregator/aggregator" - "github.com/stretchr/testify/require" yaml "gopkg.in/yaml.v2" ) @@ -120,67 +118,3 @@ func TestMaxAllowedForwardingDelayFnJitterDisabled(t *testing.T) { require.Equal(t, input.expected, fn(input.resolution, input.numForwardedTimes)) } } - -func TestFlushIntervalFn(t *testing.T) { - overrides := map[time.Duration]time.Duration{ - 10 * time.Second: 5 * time.Second, - time.Minute: 20 * time.Second, - } - c := forwardingConfiguration{FlushIntervalOverrides: overrides} - flushIntervalFn := c.FlushIntervalFn() - inputs := []struct { - metricType aggregator.IncomingMetricType - resolution time.Duration - expected time.Duration - }{ - { - metricType: aggregator.StandardIncomingMetric, - resolution: 10 * time.Second, - expected: 10 * time.Second, - }, - { - metricType: aggregator.ForwardedIncomingMetric, - resolution: 10 * time.Second, - expected: 5 * time.Second, - }, - { - metricType: aggregator.ForwardedIncomingMetric, - resolution: time.Minute, - expected: 20 * time.Second, - }, - { - metricType: aggregator.ForwardedIncomingMetric, - resolution: 10 * time.Minute, - expected: 10 * time.Minute, - }, - } - - for _, input := range inputs { - require.Equal(t, input.expected, flushIntervalFn(input.metricType, input.resolution)) - } -} - -func TestForwardingSourcesTTLFn(t *testing.T) { - c := forwardingSourcesTTLConfiguration{ - ByResolution: 10, - ByDuration: 5 * time.Minute, - } - ttlFn := c.ForwardingSourcesTTLFn() - inputs := []struct { - resolution time.Duration - expected time.Duration - }{ - { - resolution: 10 * time.Second, - expected: 5 * time.Minute, - }, - { - resolution: time.Minute, - expected: 10 * time.Minute, - }, - } - - for _, input := range inputs { - require.Equal(t, input.expected, ttlFn(input.resolution)) - } -}