From c6949062b694a0f8dd3a19f87b011426e37e7722 Mon Sep 17 00:00:00 2001 From: Ryan Hall Date: Thu, 28 Oct 2021 21:24:04 -0700 Subject: [PATCH 1/6] Fix race when checking for dirty aggregations This fixes the following race: 1. Writer creates a new aggregation and adds it to the drity set. 2. Flusher processes dirty set. 3. Writer updates the aggregation value. This race was introduced with the recent refactor of resending to use to an explicit dirty set. However, this race existed historically before resending introduced the notion of a dirty bit. The fix is to ensure the individual aggregation lock is held when read/writing the dirty bit. It was starting to get really confusing what state truly needs to be guarded by the aggregation lock. This also refactors the code to introduce an aggFlushState of flusher local state that doesn't need any synchronization. Now all fields on the lockedAggregation must be accessed with the lock. --- src/aggregator/aggregation/common.go | 4 +- src/aggregator/aggregation/counter.go | 2 +- src/aggregator/aggregation/gauge.go | 2 +- src/aggregator/aggregation/timer.go | 2 +- src/aggregator/aggregator/counter_elem_gen.go | 339 ++++++++++-------- src/aggregator/aggregator/elem_base.go | 41 ++- src/aggregator/aggregator/elem_test.go | 223 ++++++------ src/aggregator/aggregator/gauge_elem_gen.go | 339 ++++++++++-------- src/aggregator/aggregator/generic_elem.go | 338 +++++++++-------- src/aggregator/aggregator/timer_elem_gen.go | 339 ++++++++++-------- 10 files changed, 936 insertions(+), 693 deletions(-) diff --git a/src/aggregator/aggregation/common.go b/src/aggregator/aggregation/common.go index bf0d1332b3..18fd59b243 100644 --- a/src/aggregator/aggregation/common.go +++ b/src/aggregator/aggregation/common.go @@ -44,7 +44,9 @@ func isExpensive(aggTypes aggregation.Types) bool { return false } -func maybeReplaceAnnotation(currentAnnotation, newAnnotation []byte) []byte { +// MaybeReplaceAnnotation replaces the current annotation with the new annotation, returning the updated ref for +// current annotation. +func MaybeReplaceAnnotation(currentAnnotation, newAnnotation []byte) []byte { if len(newAnnotation) == 0 { return currentAnnotation } diff --git a/src/aggregator/aggregation/counter.go b/src/aggregator/aggregation/counter.go index 5ece529062..58d1a373e8 100644 --- a/src/aggregator/aggregation/counter.go +++ b/src/aggregator/aggregation/counter.go @@ -74,7 +74,7 @@ func (c *Counter) Update(timestamp time.Time, value int64, annotation []byte) { c.sumSq += value * value } - c.annotation = maybeReplaceAnnotation(c.annotation, annotation) + c.annotation = MaybeReplaceAnnotation(c.annotation, annotation) } // LastAt returns the time of the last value received. diff --git a/src/aggregator/aggregation/gauge.go b/src/aggregator/aggregation/gauge.go index 8892ff4c44..6a3f1e29d1 100644 --- a/src/aggregator/aggregation/gauge.go +++ b/src/aggregator/aggregation/gauge.go @@ -52,7 +52,7 @@ func NewGauge(opts Options) Gauge { // Update updates the gauge value. func (g *Gauge) Update(timestamp time.Time, value float64, annotation []byte) { - g.annotation = maybeReplaceAnnotation(g.annotation, annotation) + g.annotation = MaybeReplaceAnnotation(g.annotation, annotation) g.updateTotals(timestamp, value) // min/max cannot be updated by an update to a value. if math.IsNaN(g.max) || g.max < value { diff --git a/src/aggregator/aggregation/timer.go b/src/aggregator/aggregation/timer.go index 44c84bdd1f..34ccd5b8b5 100644 --- a/src/aggregator/aggregation/timer.go +++ b/src/aggregator/aggregation/timer.go @@ -72,7 +72,7 @@ func (t *Timer) AddBatch(timestamp time.Time, values []float64, annotation []byt t.stream.AddBatch(values) - t.annotation = maybeReplaceAnnotation(t.annotation, annotation) + t.annotation = MaybeReplaceAnnotation(t.annotation, annotation) } func (t *Timer) recordLastAt(timestamp time.Time) { diff --git a/src/aggregator/aggregator/counter_elem_gen.go b/src/aggregator/aggregator/counter_elem_gen.go index 1d072d0493..a739a16d82 100644 --- a/src/aggregator/aggregator/counter_elem_gen.go +++ b/src/aggregator/aggregator/counter_elem_gen.go @@ -30,6 +30,7 @@ import ( "sync" "time" + raggregation "github.com/m3db/m3/src/aggregator/aggregation" "github.com/m3db/m3/src/metrics/metadata" "github.com/m3db/m3/src/metrics/metric/aggregated" "github.com/m3db/m3/src/metrics/metric/unaggregated" @@ -45,26 +46,21 @@ type lockedCounterAggregation struct { sync.Mutex dirty bool - flushed bool closed bool sourcesSeen map[uint32]*bitset.BitSet aggregation counterAggregation - prevValues []float64 // the previously emitted values (one per aggregation type). } type timedCounter struct { startAtNanos xtime.UnixNano // start time of an aggregation window lockedAgg *lockedCounterAggregation resendEnabled bool - - // this is mutable data for specifying on each Consume which previous value the - // current agg can reference (i.e. for binary ops). it must be mutable since the - // set of vals within the buffer past can change and so on each consume a given agg's - // previous depends on the state of values preceding the current at that point in time. - previousTimeNanos xtime.UnixNano + inDirtySet bool } -func (ta *timedCounter) Release() { +// close is called when the aggregation has been expired or the element is being closed. +func (ta *timedCounter) close() { + ta.lockedAgg.aggregation.Close() ta.lockedAgg = nil } @@ -75,25 +71,26 @@ type CounterElem struct { // startTime -> agg (new one per every resolution) values map[xtime.UnixNano]timedCounter + // startTime -> state. this is local state to the flusher and does not need to guarded with a lock. + // values and flushState should always have the exact same key set. + flushState map[xtime.UnixNano]aggFlushState // sorted start aligned times that have been written to since the last flush dirty []xtime.UnixNano // min time in the values map. allow for iterating through map. minStartTime xtime.UnixNano // internal consume state that does not need to be synchronized. - toConsume []timedCounter // small buffer to avoid memory allocations during consumption - toExpire []timedCounter // small buffer to avoid memory allocations during consumption - // map of the previous consumed values for each timestamp in the buffer. needed to support binary transforms that - // need the value from the previous timestamp. - consumedValues valuesByTime + toConsume []aggFlushState // small buffer to avoid memory allocations during consumption + flushStateToExpire []xtime.UnixNano // small buffer to avoid memory allocations during consumption } // NewCounterElem returns a new CounterElem. func NewCounterElem(data ElemData, opts ElemOptions) (*CounterElem, error) { e := &CounterElem{ - elemBase: newElemBase(opts), - dirty: make([]xtime.UnixNano, 0, defaultNumAggregations), // in most cases values will have two entries - values: make(map[xtime.UnixNano]timedCounter), + elemBase: newElemBase(opts), + dirty: make([]xtime.UnixNano, 0, defaultNumAggregations), // in most cases values will have two entries + values: make(map[xtime.UnixNano]timedCounter), + flushState: make(map[xtime.UnixNano]aggFlushState), } if err := e.ResetSetData(data); err != nil { return nil, err @@ -137,6 +134,7 @@ func (e *CounterElem) AddUnion(timestamp time.Time, mu unaggregated.MetricUnion, return errAggregationClosed } lockedAgg.aggregation.AddUnion(timestamp, mu) + lockedAgg.dirty = true lockedAgg.Unlock() return nil } @@ -154,6 +152,7 @@ func (e *CounterElem) AddValue(timestamp time.Time, value float64, annotation [] return errAggregationClosed } lockedAgg.aggregation.Add(timestamp, value, annotation) + lockedAgg.dirty = true lockedAgg.Unlock() return nil } @@ -205,6 +204,7 @@ func (e *CounterElem) AddUnique( lockedAgg.aggregation.Add(timestamp, v, metric.Annotation) } } + lockedAgg.dirty = true lockedAgg.Unlock() return nil } @@ -212,9 +212,8 @@ func (e *CounterElem) AddUnique( // remove expired aggregations from the values map. func (e *CounterElem) expireValuesWithLock( targetNanos int64, - timestampNanosFn timestampNanosFn, isEarlierThanFn isEarlierThanFn) { - e.toExpire = e.toExpire[:0] + e.flushStateToExpire = e.flushStateToExpire[:0] if len(e.values) == 0 { return } @@ -232,22 +231,54 @@ func (e *CounterElem) expireValuesWithLock( break } } - // if this current value is eligible to be expired it will no longer be written to. this means it's safe - // to remove the _previous_ value since it will no longer be needed for binary transformations. when the + + // close the agg to prevent any more writes. + dirty := false + currV.lockedAgg.Lock() + currV.lockedAgg.closed = true + dirty = currV.lockedAgg.dirty + currV.lockedAgg.Unlock() + if dirty { + // a race occurred and a write happened before we could close the aggregation. will expire next time. + break + } + + // if this current value is closed and clean it will no longer be flushed. this means it's safe + // to remove the previous value since it will no longer be needed for binary transformations. when the // next value is eligible to be expired, this current value will actually be removed. if prevV, ok := e.values[e.minStartTime]; ok { - // Previous times are used to key into consumedValues, which are non-start-aligned. And so - // we convert from startAligned here when setting previous. - prevV.previousTimeNanos = xtime.UnixNano(timestampNanosFn(int64(e.minStartTime), resolution)) - e.toExpire = append(e.toExpire, prevV) + // can't expire flush state until after the flushing, so we save the time to expire later. + e.flushStateToExpire = append(e.flushStateToExpire, e.minStartTime) delete(e.values, e.minStartTime) e.minStartTime = currStart + + // it's safe to access this outside the agg lock since it was closed in a previous iteration. + // This is to make sure there aren't too many cached source sets taking up + // too much space. + if prevV.lockedAgg.sourcesSeen != nil && len(e.cachedSourceSets) < e.opts.MaxNumCachedSourceSets() { + e.cachedSourceSets = append(e.cachedSourceSets, prevV.lockedAgg.sourcesSeen) + } + prevV.close() } } currStart = currStart.Add(resolution) } } +func (e *CounterElem) expireFlushState() { + for _, t := range e.flushStateToExpire { + prevFlushState, ok := e.flushState[t] + if !ok { + ts := t.ToTime() + instrument.EmitAndLogInvariantViolation(e.opts.InstrumentOptions(), func(l *zap.Logger) { + l.Error("expire time not in state map", zap.Time("ts", ts)) + }) + } + prevFlushState.close() + delete(e.flushState, t) + } +} + // return the timestamp in the values map that is before the provided time. returns false if the provided time is the // smallest time or the map is empty. func (e *CounterElem) previousStartAlignedWithLock(timestamp xtime.UnixNano) (xtime.UnixNano, bool) { @@ -255,6 +286,7 @@ func (e *CounterElem) previousStartAlignedWithLock(timestamp xtime.UnixNano) (xt return 0, false } resolution := e.sp.Resolution().Window + // ensure the input is start aligned and then calculate the previous start time. startAligned := timestamp.Truncate(resolution).Add(-resolution) for !startAligned.Before(e.minStartTime) { _, ok := e.values[startAligned] @@ -308,80 +340,55 @@ func (e *CounterElem) Consume( return false } e.toConsume = e.toConsume[:0] - // Evaluate and GC expired items. dirtyTimes := e.dirty e.dirty = e.dirty[:0] - for _, dirtyTime := range dirtyTimes { - agg, ok := e.values[dirtyTime] - if !ok { - dirtyTime := dirtyTime - instrument.EmitAndLogInvariantViolation(e.opts.InstrumentOptions(), func(l *zap.Logger) { - l.Error("dirty timestamp not in map", zap.Time("ts", dirtyTime.ToTime())) - }) - } - - if !isEarlierThanFn(int64(dirtyTime), resolution, targetNanos) { + for i, dirtyTime := range dirtyTimes { + flushState, ready := e.updateFlushStateWithLock(dirtyTime, targetNanos, isEarlierThanFn) + if !ready || !flushState.dirty { + // there is a race where the value was added to the dirty set, but the writer didn't actually update the + // value yet (by marking dirty). add back to the dirty set so it can be processed in the next round once + // the value has been updated. e.dirty = append(e.dirty, dirtyTime) continue } + val := e.values[dirtyTime] + val.inDirtySet = false + e.values[dirtyTime] = val + e.toConsume = append(e.toConsume, flushState) - agg.lockedAgg.dirty = false - previousStartAligned, ok := e.previousStartAlignedWithLock(dirtyTime) - if ok { - agg.previousTimeNanos = xtime.UnixNano(timestampNanosFn(int64(previousStartAligned), resolution)) - } - e.toConsume = append(e.toConsume, agg) - - // add the nextAgg to the dirty set as well in case we need to cascade the value. + // potentially consume the nextAgg as well in case we need to cascade an update due to a resend. nextAgg, ok := e.nextAggWithLock(dirtyTime, targetNanos, isEarlierThanFn) - // only add nextAgg if not already in the dirty set - if ok && !nextAgg.lockedAgg.dirty { - nextAgg.previousTimeNanos = xtime.UnixNano(timestampNanosFn(int64(dirtyTime), resolution)) - e.toConsume = append(e.toConsume, nextAgg) + if ok && (i == len(dirtyTimes)-1 || dirtyTimes[i+1] != nextAgg.startAtNanos) { + // if not already in the dirty set. + flushState, ready := e.updateFlushStateWithLock(nextAgg.startAtNanos, targetNanos, isEarlierThanFn) + if ready { + e.toConsume = append(e.toConsume, flushState) + } } } - e.expireValuesWithLock(targetNanos, timestampNanosFn, isEarlierThanFn) + // expire the values and aggregations while we still hold the lock. + e.expireValuesWithLock(targetNanos, isEarlierThanFn) canCollect := len(e.dirty) == 0 && e.tombstoned e.Unlock() // Process the aggregations that are ready for consumption. - for i := range e.toConsume { - timeNanos := xtime.UnixNano(timestampNanosFn(int64(e.toConsume[i].startAtNanos), resolution)) - e.toConsume[i].lockedAgg.Lock() - _ = e.processValueWithAggregationLock( - timeNanos, - e.toConsume[i].previousTimeNanos, - e.toConsume[i].resendEnabled, - e.toConsume[i].lockedAgg, + for _, flushState := range e.toConsume { + flushState.timestamp = xtime.UnixNano(timestampNanosFn(int64(flushState.startAt), resolution)) + flushState = e.processValueWithAggregation( + flushState, flushLocalFn, flushForwardedFn, resolution, latenessAllowed, ) - e.toConsume[i].lockedAgg.flushed = true - e.toConsume[i].lockedAgg.Unlock() - } - - // Cleanup expired item after consuming since consuming still has a ref to the locked aggregation. - for i := range e.toExpire { - e.toExpire[i].lockedAgg.closed = true - e.toExpire[i].lockedAgg.aggregation.Close() - if e.toExpire[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.toExpire[i].lockedAgg.sourcesSeen) - } - e.cachedSourceSetsLock.Unlock() - e.toExpire[i].lockedAgg.sourcesSeen = nil - } - e.toExpire[i].Release() - - delete(e.consumedValues, e.toExpire[i].previousTimeNanos) + flushState.flushed = true + e.flushState[flushState.startAt] = flushState } + // expire the flush state after processing since it's needed in the processing. + e.expireFlushState() + if e.parsedPipeline.HasRollup { forwardedAggregationKey, _ := e.ForwardedAggregationKey() onForwardedFlushedFn(e.onForwardedAggregationWrittenFn, forwardedAggregationKey) @@ -390,6 +397,46 @@ func (e *CounterElem) Consume( return canCollect } +func (e *CounterElem) updateFlushStateWithLock(dirtyTime xtime.UnixNano, targetNanos int64, + isEarlierThanFn isEarlierThanFn) (aggFlushState, bool) { + resolution := e.sp.Resolution().Window + agg, ok := e.values[dirtyTime] + if !ok { + ts := dirtyTime.ToTime() + instrument.EmitAndLogInvariantViolation(e.opts.InstrumentOptions(), func(l *zap.Logger) { + l.Error("dirty timestamp not in values map", zap.Time("ts", ts)) + }) + } + if !isEarlierThanFn(int64(dirtyTime), resolution, targetNanos) { + return aggFlushState{}, false + } + + // note: flushState might be empty for the first flush + flushState := e.flushState[dirtyTime] + // copy the lockedAgg data to the flushState while holding the lock. + agg.lockedAgg.Lock() + flushState.dirty = agg.lockedAgg.dirty + flushState.values = flushState.values[:0] + for _, aggType := range e.aggTypes { + flushState.values = append(flushState.values, agg.lockedAgg.aggregation.ValueOf(aggType)) + } + flushState.annotation = raggregation.MaybeReplaceAnnotation( + flushState.annotation, agg.lockedAgg.aggregation.Annotation()) + agg.lockedAgg.dirty = false + agg.lockedAgg.Unlock() + + // update the flushState with everything else. + previousStartAligned, ok := e.previousStartAlignedWithLock(dirtyTime) + if ok { + flushState.prevStartTime = previousStartAligned + } else { + flushState.prevStartTime = 0 + } + flushState.resendEnabled = agg.resendEnabled + flushState.startAt = dirtyTime + return flushState, true +} + // Close closes the element. func (e *CounterElem) Close() { e.Lock() @@ -407,39 +454,44 @@ func (e *CounterElem) Close() { } e.cachedSourceSets = nil - resolution := e.sp.Resolution().Window // note: this is not in the hot path so it's ok to iterate over the map. // this allows to catch any bugs with unexpected entries still in the map. - for k := range e.values { - if k < e.minStartTime { + minStartTime := e.minStartTime + for k, v := range e.values { + if k < minStartTime { k := k + ts := e.minStartTime.ToTime() instrument.EmitAndLogInvariantViolation(e.opts.InstrumentOptions(), func(l *zap.Logger) { - l.Error("aggregate timestamp is less than min", + l.Error("value timestamp is less than min start time", zap.Time("ts", k.ToTime()), - zap.Time("min", e.minStartTime.ToTime())) + zap.Time("min", ts)) }) } + v.close() delete(e.values, k) - // Close the underlying aggregation objects. - if v, ok := e.values[e.minStartTime]; ok { - v.lockedAgg.sourcesSeen = nil - v.lockedAgg.aggregation.Close() - v.Release() - delete(e.values, e.minStartTime) + flushState, ok := e.flushState[k] + if ok { + flushState.close() } - e.minStartTime = e.minStartTime.Add(resolution) + delete(e.flushState, k) + } + // clean up any dangling flush state that should never exist. + for k, v := range e.flushState { + ts := k.ToTime() + instrument.EmitAndLogInvariantViolation(e.opts.InstrumentOptions(), func(l *zap.Logger) { + l.Error("dangling state timestamp", zap.Time("ts", ts)) + }) + v.close() + delete(e.flushState, k) } e.counterElemBase.Close() aggTypesPool := e.aggTypesOpts.TypesPool() pool := e.ElemPool(e.opts) - e.Unlock() - - // internal consumption state that doesn't need to be synchronized. - e.toConsume = e.toConsume[:0] e.dirty = e.dirty[:0] - e.toExpire = e.toExpire[:0] - e.consumedValues = nil + e.toConsume = e.toConsume[:0] + e.flushStateToExpire = e.flushStateToExpire[:0] e.minStartTime = 0 + e.Unlock() if !e.useDefaultAggregation { aggTypesPool.Put(e.aggTypes) @@ -479,20 +531,19 @@ func (e *CounterElem) insertDirty(alignedStart xtime.UnixNano) { // find finds the aggregation for a given time, or returns nil. //nolint: dupl -func (e *CounterElem) find(alignedStartNanos xtime.UnixNano) (timedCounter, bool, error) { +func (e *CounterElem) find(alignedStartNanos xtime.UnixNano) (timedCounter, error) { e.RLock() if e.closed { e.RUnlock() - return timedCounter{}, false, errElemClosed + return timedCounter{}, errElemClosed } timedAgg, ok := e.values[alignedStartNanos] if ok { - dirty := timedAgg.lockedAgg.dirty e.RUnlock() - return timedAgg, dirty, nil + return timedAgg, nil } e.RUnlock() - return timedCounter{}, false, nil + return timedCounter{}, nil } // findOrCreate finds the aggregation for a given time, or creates one @@ -503,12 +554,12 @@ func (e *CounterElem) findOrCreate( createOpts createAggregationOptions, ) (*lockedCounterAggregation, error) { alignedStart := xtime.UnixNano(alignedStartNanos) - found, isDirty, err := e.find(alignedStart) + found, err := e.find(alignedStart) if err != nil { return nil, err } // if the aggregation is found and does not need to be updated, return as is. - if found.lockedAgg != nil && isDirty && found.resendEnabled == createOpts.resendEnabled { + if found.lockedAgg != nil && found.inDirtySet && found.resendEnabled == createOpts.resendEnabled { return found.lockedAgg, err } @@ -520,9 +571,9 @@ func (e *CounterElem) findOrCreate( timedAgg, ok := e.values[alignedStart] if ok { - // if the agg is not dirty, mark it dirty so it will be flushed. - if !timedAgg.lockedAgg.dirty { - timedAgg.lockedAgg.dirty = true + // add to dirty set so it will be flushed. + if !timedAgg.inDirtySet { + timedAgg.inDirtySet = true e.insertDirty(alignedStart) } // ensure the resendEnabled state is the latest. @@ -534,7 +585,6 @@ func (e *CounterElem) findOrCreate( var sourcesSeen map[uint32]*bitset.BitSet if createOpts.initSourceSet { - e.cachedSourceSetsLock.Lock() if numCachedSourceSets := len(e.cachedSourceSets); numCachedSourceSets > 0 { sourcesSeen = e.cachedSourceSets[numCachedSourceSets-1] e.cachedSourceSets[numCachedSourceSets-1] = nil @@ -545,17 +595,15 @@ func (e *CounterElem) findOrCreate( } else { sourcesSeen = make(map[uint32]*bitset.BitSet) } - e.cachedSourceSetsLock.Unlock() } timedAgg = timedCounter{ startAtNanos: alignedStart, lockedAgg: &lockedCounterAggregation{ sourcesSeen: sourcesSeen, aggregation: e.NewAggregation(e.opts, e.aggOpts), - prevValues: make([]float64, len(e.aggTypes)), - dirty: true, }, resendEnabled: createOpts.resendEnabled, + inDirtySet: true, } e.values[alignedStart] = timedAgg e.insertDirty(alignedStart) @@ -567,23 +615,19 @@ func (e *CounterElem) findOrCreate( } // returns true if a datapoint is emitted. -func (e *CounterElem) processValueWithAggregationLock( - timeNanos xtime.UnixNano, - prevTimeNanos xtime.UnixNano, - resendEnabled bool, - lockedAgg *lockedCounterAggregation, +func (e *CounterElem) processValueWithAggregation( + flushState aggFlushState, flushLocalFn flushLocalMetricFn, flushForwardedFn flushForwardedMetricFn, resolution time.Duration, - latenessAllowed time.Duration) bool { + latenessAllowed time.Duration) aggFlushState { var ( transformations = e.parsedPipeline.Transformations discardNaNValues = e.opts.DiscardNaNAggregatedValues() - emitted bool ) for aggTypeIdx, aggType := range e.aggTypes { var extraDp transformation.Datapoint - value := lockedAgg.aggregation.ValueOf(aggType) + value := flushState.values[aggTypeIdx] for _, transformOp := range transformations { unaryOp, isUnaryOp := transformOp.UnaryTransform() binaryOp, isBinaryOp := transformOp.BinaryTransform() @@ -591,7 +635,7 @@ func (e *CounterElem) processValueWithAggregationLock( switch { case isUnaryOp: curr := transformation.Datapoint{ - TimeNanos: int64(timeNanos), + TimeNanos: int64(flushState.timestamp), Value: value, } @@ -603,15 +647,21 @@ func (e *CounterElem) processValueWithAggregationLock( prev := transformation.Datapoint{ Value: nan, } - // lazily construct consumedValues since they are only needed by binary transforms. - if e.consumedValues == nil { - e.consumedValues = make(valuesByTime) - } - if _, ok := e.consumedValues[prevTimeNanos]; ok { - prev = e.consumedValues[prevTimeNanos][aggTypeIdx] + if flushState.prevStartTime > 0 { + prevFlushState, ok := e.flushState[flushState.prevStartTime] + if !ok { + ts := flushState.prevStartTime.ToTime() + instrument.EmitAndLogInvariantViolation(e.opts.InstrumentOptions(), func(l *zap.Logger) { + l.Error("previous start time not in state map", + zap.Time("ts", ts)) + }) + } else { + prev.Value = prevFlushState.consumedValues[aggTypeIdx] + prev.TimeNanos = int64(prevFlushState.timestamp) + } } curr := transformation.Datapoint{ - TimeNanos: int64(timeNanos), + TimeNanos: int64(flushState.timestamp), Value: value, } res := binaryOp.Evaluate(prev, curr, transformation.FeatureFlags{}) @@ -621,16 +671,15 @@ func (e *CounterElem) processValueWithAggregationLock( // need to keep one value. In the future if we need to support higher-order // derivative transformations, we need to store an array of values here. if !math.IsNaN(curr.Value) { - if e.consumedValues[timeNanos] == nil { - e.consumedValues[timeNanos] = make([]transformation.Datapoint, len(e.aggTypes)) + if flushState.consumedValues == nil { + flushState.consumedValues = make([]float64, len(e.aggTypes)) } - e.consumedValues[timeNanos][aggTypeIdx] = curr + flushState.consumedValues[aggTypeIdx] = curr.Value } - value = res.Value case isUnaryMultiOp: curr := transformation.Datapoint{ - TimeNanos: int64(timeNanos), + TimeNanos: int64(flushState.timestamp), Value: value, } @@ -646,20 +695,24 @@ func (e *CounterElem) processValueWithAggregationLock( // It's ok to send a 0 prevValue on the first forward because it's not used in AddUnique unless it's a // resend (version > 0) - prevValue := lockedAgg.prevValues[aggTypeIdx] - lockedAgg.prevValues[aggTypeIdx] = value - if lockedAgg.flushed { + var prevValue float64 + if flushState.emittedValues == nil { + flushState.emittedValues = make([]float64, len(e.aggTypes)) + } else { + prevValue = flushState.emittedValues[aggTypeIdx] + } + flushState.emittedValues[aggTypeIdx] = value + if flushState.flushed { // no need to resend a value that hasn't changed. if (math.IsNaN(prevValue) && math.IsNaN(value)) || (prevValue == value) { continue } } - emitted = true if !e.parsedPipeline.HasRollup { toFlush := make([]transformation.Datapoint, 0, 2) toFlush = append(toFlush, transformation.Datapoint{ - TimeNanos: int64(timeNanos), + TimeNanos: int64(flushState.timestamp), Value: value, }) if extraDp.TimeNanos != 0 { @@ -668,22 +721,24 @@ func (e *CounterElem) processValueWithAggregationLock( for _, point := range toFlush { switch e.idPrefixSuffixType { case NoPrefixNoSuffix: - flushLocalFn(nil, e.id, nil, point.TimeNanos, point.Value, lockedAgg.aggregation.Annotation(), e.sp) + flushLocalFn(nil, e.id, nil, point.TimeNanos, point.Value, flushState.annotation, + e.sp) case WithPrefixWithSuffix: flushLocalFn(e.FullPrefix(e.opts), e.id, e.TypeStringFor(e.aggTypesOpts, aggType), - point.TimeNanos, point.Value, lockedAgg.aggregation.Annotation(), e.sp) + point.TimeNanos, point.Value, flushState.annotation, e.sp) } } } else { forwardedAggregationKey, _ := e.ForwardedAggregationKey() // only record lag for the initial flush (not resends) - if !lockedAgg.flushed { + if !flushState.flushed { // latenessAllowed is not due to processing delay, so it remove it from lag calc. - e.forwardLagMetric(resolution).RecordDuration(time.Since(timeNanos.ToTime().Add(-latenessAllowed))) + e.forwardLagMetric(resolution).RecordDuration( + time.Since(flushState.timestamp.ToTime().Add(-latenessAllowed))) } flushForwardedFn(e.writeForwardedMetricFn, forwardedAggregationKey, - int64(timeNanos), value, prevValue, lockedAgg.aggregation.Annotation(), resendEnabled) + int64(flushState.timestamp), value, prevValue, flushState.annotation, flushState.resendEnabled) } } - return emitted + return flushState } diff --git a/src/aggregator/aggregator/elem_base.go b/src/aggregator/aggregator/elem_base.go index 53d7ca9e5f..89b31f9071 100644 --- a/src/aggregator/aggregator/elem_base.go +++ b/src/aggregator/aggregator/elem_base.go @@ -176,15 +176,46 @@ type elemBase struct { listType metricListType // Mutable states. - tombstoned bool - closed bool - cachedSourceSetsLock sync.Mutex // nolint: structcheck - cachedSourceSets []map[uint32]*bitset.BitSet // nolint: structcheck + tombstoned bool + closed bool + cachedSourceSets []map[uint32]*bitset.BitSet // nolint: structcheck // a cache of the lag metrics that don't require grabbing a lock to access. forwardLagMetrics map[forwardLagKey]tally.Histogram } -type valuesByTime map[xtime.UnixNano][]transformation.Datapoint +// mutable state for a timedAggregation that is local to the flusher. does not need to be synchronized. +type aggFlushState struct { + // the annotation copied from the lockedAgg. + annotation []byte + // the values copied from the lockedAgg. + values []float64 + // the consumed values from the previous flush. used for binary transformations. note these are the values before + // transformation. emittedValues are after transformation. + consumedValues []float64 + // the emitted values from the previous flush. used to determine if the emitted values have not changed and + // can be skipped. + emittedValues []float64 + // the start time of the aggregation. immutable. + startAt xtime.UnixNano + // the timestamp of the aggregation. effectively immutable, but lazily set at flush time. + timestamp xtime.UnixNano + // the start aligned timestamp of the previous aggregation. used to lookup the consumedValues of the previous + // aggregation for binary transformations. + prevStartTime xtime.UnixNano + // true if this aggregation has ever been flushed. + flushed bool + // the dirty bit copied from the lockedAgg. + dirty bool + // copied from the timedAggregation + resendEnabled bool +} + +// close is called when the aggregation has expired and is no longer needed. +func (a *aggFlushState) close() { + a.values = a.values[:0] + a.consumedValues = a.consumedValues[:0] + a.emittedValues = a.emittedValues[:0] +} type elemMetrics struct { sync.RWMutex diff --git a/src/aggregator/aggregator/elem_test.go b/src/aggregator/aggregator/elem_test.go index 1dfc73bb84..2d155f3b5a 100644 --- a/src/aggregator/aggregator/elem_test.go +++ b/src/aggregator/aggregator/elem_test.go @@ -149,7 +149,7 @@ func TestCounterResetSetData(t *testing.T) { require.False(t, ce.closed) require.False(t, ce.useDefaultAggregation) require.True(t, ce.aggOpts.HasExpensiveAggregations) - require.Nil(t, ce.consumedValues) + require.Empty(t, ce.flushState) require.Equal(t, 2, ce.numForwardedTimes) // Reset element with a pipeline containing a derivative transformation. @@ -179,7 +179,7 @@ func TestCounterResetSetData(t *testing.T) { err = ce.ResetSetData(elemData) require.NoError(t, err) requirePipelinesMatch(t, expectedParsedPipeline, ce.parsedPipeline) - require.Empty(t, ce.consumedValues) + require.Empty(t, ce.flushState) } func TestCounterResetSetDataInvalidAggregationType(t *testing.T) { @@ -215,7 +215,7 @@ func TestCounterElemAddUnion(t *testing.T) { testCounter.Annotation = []byte{1} require.NoError(t, e.AddUnion(testTimestamps[0], testCounter, false)) require.Equal(t, 1, len(e.values)) - a, _, err := e.find(xtime.UnixNano(testAlignedStarts[0])) + a, err := e.find(xtime.UnixNano(testAlignedStarts[0])) require.NoError(t, err) v := a.lockedAgg require.Equal(t, testCounter.CounterVal, v.aggregation.Sum()) @@ -228,7 +228,7 @@ func TestCounterElemAddUnion(t *testing.T) { testCounter.Annotation = []byte{} require.NoError(t, e.AddUnion(testTimestamps[1], testCounter, false)) require.Equal(t, 1, len(e.values)) - a, _, err = e.find(xtime.UnixNano(testAlignedStarts[0])) + a, err = e.find(xtime.UnixNano(testAlignedStarts[0])) require.NoError(t, err) v = a.lockedAgg require.Equal(t, 2*testCounter.CounterVal, v.aggregation.Sum()) @@ -240,7 +240,7 @@ func TestCounterElemAddUnion(t *testing.T) { testCounter.Annotation = []byte{2} require.NoError(t, e.AddUnion(testTimestamps[2], testCounter, false)) require.Equal(t, 2, len(e.values)) - a1, _, err := e.find(xtime.UnixNano(testAlignedStarts[1])) + a1, err := e.find(xtime.UnixNano(testAlignedStarts[1])) require.NoError(t, err) v1 := a1.lockedAgg require.Equal(t, testCounter.CounterVal, v1.aggregation.Sum()) @@ -263,7 +263,7 @@ func TestCounterElemAddUnionWithCustomAggregation(t *testing.T) { testCounter.Annotation = []byte{1} require.NoError(t, e.AddUnion(testTimestamps[0], testCounter, false)) require.Equal(t, 1, len(e.values)) - a, _, err := e.find(xtime.UnixNano(testAlignedStarts[0])) + a, err := e.find(xtime.UnixNano(testAlignedStarts[0])) require.NoError(t, err) v := a.lockedAgg require.Equal(t, testCounter.CounterVal, v.aggregation.Sum()) @@ -276,7 +276,7 @@ func TestCounterElemAddUnionWithCustomAggregation(t *testing.T) { testCounter.Annotation = []byte{} require.NoError(t, e.AddUnion(testTimestamps[1], testCounter, false)) require.Equal(t, 1, len(e.values)) - a, _, err = e.find(xtime.UnixNano(testAlignedStarts[0])) + a, err = e.find(xtime.UnixNano(testAlignedStarts[0])) require.NoError(t, err) v = a.lockedAgg require.Equal(t, 2*testCounter.CounterVal, v.aggregation.Sum()) @@ -289,7 +289,7 @@ func TestCounterElemAddUnionWithCustomAggregation(t *testing.T) { for i := 0; i < len(e.values); i++ { require.Equal(t, xtime.UnixNano(testAlignedStarts[i]), e.values[e.dirty[i]].startAtNanos) } - a, _, err = e.find(xtime.UnixNano(testAlignedStarts[1])) + a, err = e.find(xtime.UnixNano(testAlignedStarts[1])) require.NoError(t, err) v = a.lockedAgg require.Equal(t, testCounter.CounterVal, v.aggregation.Sum()) @@ -310,7 +310,7 @@ func TestCounterElemAddUnique(t *testing.T) { aggregated.ForwardedMetric{Values: []float64{345}}, metadata.ForwardMetadata{SourceID: source1})) require.Equal(t, 1, len(e.values)) - a, _, err := e.find(xtime.UnixNano(testAlignedStarts[0])) + a, err := e.find(xtime.UnixNano(testAlignedStarts[0])) require.NoError(t, err) v := a.lockedAgg require.Equal(t, int64(345), v.aggregation.Sum()) @@ -328,7 +328,7 @@ func TestCounterElemAddUnique(t *testing.T) { aggregated.ForwardedMetric{Values: []float64{500}, Annotation: testAnnot}, metadata.ForwardMetadata{SourceID: source2})) require.Equal(t, 1, len(e.values)) - a, _, err = e.find(xtime.UnixNano(testAlignedStarts[0])) + a, err = e.find(xtime.UnixNano(testAlignedStarts[0])) require.NoError(t, err) v = a.lockedAgg require.Equal(t, int64(845), v.aggregation.Sum()) @@ -347,7 +347,7 @@ func TestCounterElemAddUnique(t *testing.T) { for i := 0; i < len(e.values); i++ { require.Equal(t, xtime.UnixNano(testAlignedStarts[i]), e.values[e.dirty[i]].startAtNanos) } - a1, _, err := e.find(xtime.UnixNano(testAlignedStarts[1])) + a1, err := e.find(xtime.UnixNano(testAlignedStarts[1])) require.NoError(t, err) v1 := a1.lockedAgg require.Equal(t, int64(278), v1.aggregation.Sum()) @@ -364,7 +364,7 @@ func TestCounterElemAddUnique(t *testing.T) { aggregated.ForwardedMetric{Values: []float64{278}}, metadata.ForwardMetadata{SourceID: source1})) require.Equal(t, 2, len(e.values)) - a1, _, err = e.find(xtime.UnixNano(testAlignedStarts[1])) + a1, err = e.find(xtime.UnixNano(testAlignedStarts[1])) require.NoError(t, err) v1 = a1.lockedAgg for i := 0; i < len(e.values); i++ { @@ -396,7 +396,7 @@ func TestCounterElemAddUniqueWithCustomAggregation(t *testing.T) { aggregated.ForwardedMetric{Values: []float64{12}}, metadata.ForwardMetadata{SourceID: source1})) require.Equal(t, 1, len(e.values)) - a, _, err := e.find(xtime.UnixNano(testAlignedStarts[0])) + a, err := e.find(xtime.UnixNano(testAlignedStarts[0])) require.NoError(t, err) v := a.lockedAgg require.Equal(t, int64(12), v.aggregation.Sum()) @@ -413,7 +413,7 @@ func TestCounterElemAddUniqueWithCustomAggregation(t *testing.T) { aggregated.ForwardedMetric{Values: []float64{14}}, metadata.ForwardMetadata{SourceID: source2})) require.Equal(t, 1, len(e.values)) - a, _, err = e.find(xtime.UnixNano(testAlignedStarts[0])) + a, err = e.find(xtime.UnixNano(testAlignedStarts[0])) require.NoError(t, err) v = a.lockedAgg require.Equal(t, int64(26), v.aggregation.Sum()) @@ -424,7 +424,7 @@ func TestCounterElemAddUniqueWithCustomAggregation(t *testing.T) { aggregated.ForwardedMetric{Values: []float64{20}}, metadata.ForwardMetadata{SourceID: source1})) require.Equal(t, 2, len(e.values)) - a1, _, err := e.find(xtime.UnixNano(testAlignedStarts[1])) + a1, err := e.find(xtime.UnixNano(testAlignedStarts[1])) require.NoError(t, err) v1 := a1.lockedAgg for i := 0; i < len(e.values); i++ { @@ -440,7 +440,7 @@ func TestCounterElemAddUniqueWithCustomAggregation(t *testing.T) { aggregated.ForwardedMetric{Values: []float64{30}}, metadata.ForwardMetadata{SourceID: source1})) require.Equal(t, 2, len(e.values)) - a1, _, err = e.find(xtime.UnixNano(testAlignedStarts[1])) + a1, err = e.find(xtime.UnixNano(testAlignedStarts[1])) require.NoError(t, err) v1 = a1.lockedAgg for i := 0; i < len(e.values); i++ { @@ -652,11 +652,12 @@ func TestCounterElemConsumeCustomAggregationCustomPipeline(t *testing.T) { verifyOnForwardedFlushResult(t, expectedOnFlushedRes, *onForwardedFlushedRes) require.Equal(t, 0, len(*localRes)) require.Equal(t, 3, len(e.values)) - require.Len(t, e.consumedValues, 1) - consumedVal := e.consumedValues[xtime.UnixNano(expectedForwardedRes[0].timeNanos)] + require.Len(t, e.flushState, 1) + flushState := e.flushState[xtime.UnixNano(alignedstartAtNanos[0])] + consumedVal := flushState.consumedValues require.Len(t, consumedVal, 1) - require.Equal(t, 123.0, consumedVal[0].Value) - require.Equal(t, time.Unix(220, 0).UnixNano(), consumedVal[0].TimeNanos) + require.Equal(t, 123.0, consumedVal[0]) + require.Equal(t, xtime.FromSeconds(220), flushState.timestamp) // Consume all values. expectedForwardedRes = []testForwardedMetricWithMetadata{ @@ -684,11 +685,12 @@ func TestCounterElemConsumeCustomAggregationCustomPipeline(t *testing.T) { verifyOnForwardedFlushResult(t, expectedOnFlushedRes, *onForwardedFlushedRes) require.Equal(t, 0, len(*localRes)) require.Equal(t, 1, len(e.values)) - require.Len(t, e.consumedValues, 1) - consumedVal = e.consumedValues[xtime.UnixNano(expectedForwardedRes[1].timeNanos)] + require.Len(t, e.flushState, 1) + flushState = e.flushState[xtime.UnixNano(alignedstartAtNanos[2])] + consumedVal = flushState.consumedValues require.Len(t, consumedVal, 1) - require.Equal(t, 589.0, consumedVal[0].Value) - require.Equal(t, time.Unix(240, 0).UnixNano(), consumedVal[0].TimeNanos) + require.Equal(t, 589.0, consumedVal[0]) + require.Equal(t, xtime.FromSeconds(240), flushState.timestamp) // Tombstone the element and discard all values. e.tombstoned = true @@ -734,7 +736,7 @@ func TestCounterElemClose(t *testing.T) { require.Nil(t, e.cachedSourceSets) require.Equal(t, 0, len(e.values)) require.Equal(t, 0, len(e.toConsume)) - require.Equal(t, 0, len(e.consumedValues)) + require.Equal(t, 0, len(e.flushState)) require.NotNil(t, e.values) } @@ -806,7 +808,7 @@ func TestTimerResetSetData(t *testing.T) { require.False(t, te.aggOpts.HasExpensiveAggregations) require.Equal(t, []float64{0.999}, te.quantiles) require.NotNil(t, te.quantilesPool) - require.Nil(t, te.consumedValues) + require.Empty(t, te.flushState) // Reset element with a pipeline containing a derivative transformation. expectedParsedPipeline := parsedPipeline{ @@ -834,7 +836,7 @@ func TestTimerResetSetData(t *testing.T) { err = te.ResetSetData(elemData) require.NoError(t, err) requirePipelinesMatch(t, expectedParsedPipeline, te.parsedPipeline) - require.Empty(t, te.consumedValues) + require.Empty(t, te.flushState) } func TestTimerResetSetDataInvalidAggregationType(t *testing.T) { @@ -869,7 +871,7 @@ func TestTimerElemAddUnion(t *testing.T) { // Add a timer metric. require.NoError(t, e.AddUnion(testTimestamps[0], testBatchTimer, false)) require.Equal(t, 1, len(e.values)) - a, _, err := e.find(xtime.UnixNano(testAlignedStarts[0])) + a, err := e.find(xtime.UnixNano(testAlignedStarts[0])) require.NoError(t, err) v := a.lockedAgg timer := v.aggregation @@ -884,7 +886,7 @@ func TestTimerElemAddUnion(t *testing.T) { // but still within the same aggregation interval. require.NoError(t, e.AddUnion(testTimestamps[1], testBatchTimer, false)) require.Equal(t, 1, len(e.values)) - a, _, err = e.find(xtime.UnixNano(testAlignedStarts[0])) + a, err = e.find(xtime.UnixNano(testAlignedStarts[0])) require.NoError(t, err) v = a.lockedAgg timer = v.aggregation @@ -897,7 +899,7 @@ func TestTimerElemAddUnion(t *testing.T) { // Add the timer metric in the next aggregation interval. require.NoError(t, e.AddUnion(testTimestamps[2], testBatchTimer, false)) require.Equal(t, 2, len(e.values)) - a1, _, err := e.find(xtime.UnixNano(testAlignedStarts[1])) + a1, err := e.find(xtime.UnixNano(testAlignedStarts[1])) require.NoError(t, err) v1 := a1.lockedAgg for i := 0; i < len(e.values); i++ { @@ -930,7 +932,7 @@ func TestTimerElemAddUnique(t *testing.T) { aggregated.ForwardedMetric{Values: []float64{13.3}}, metadata.ForwardMetadata{SourceID: 3})) require.Equal(t, 1, len(e.values)) - a, _, err := e.find(xtime.UnixNano(testAlignedStarts[0])) + a, err := e.find(xtime.UnixNano(testAlignedStarts[0])) require.NoError(t, err) v := a.lockedAgg timer := v.aggregation @@ -944,7 +946,7 @@ func TestTimerElemAddUnique(t *testing.T) { aggregated.ForwardedMetric{Values: []float64{14.4}}, metadata.ForwardMetadata{SourceID: 4})) require.Equal(t, 1, len(e.values)) - a, _, err = e.find(xtime.UnixNano(testAlignedStarts[0])) + a, err = e.find(xtime.UnixNano(testAlignedStarts[0])) require.NoError(t, err) v = a.lockedAgg timer = v.aggregation @@ -956,7 +958,7 @@ func TestTimerElemAddUnique(t *testing.T) { aggregated.ForwardedMetric{Values: []float64{20.0}}, metadata.ForwardMetadata{SourceID: 1})) require.Equal(t, 2, len(e.values)) - a1, _, err := e.find(xtime.UnixNano(testAlignedStarts[1])) + a1, err := e.find(xtime.UnixNano(testAlignedStarts[1])) require.NoError(t, err) v1 := a1.lockedAgg for i := 0; i < len(e.values); i++ { @@ -1192,11 +1194,12 @@ func TestTimerElemConsumeCustomAggregationCustomPipeline(t *testing.T) { verifyOnForwardedFlushResult(t, expectedOnFlushedRes, *onForwardedFlushedRes) require.Equal(t, 0, len(*localRes)) require.Equal(t, 3, len(e.values)) - require.Len(t, e.consumedValues, 1) - consumedVal := e.consumedValues[xtime.UnixNano(expectedForwardedRes[0].timeNanos)] + require.Len(t, e.flushState, 1) + flushState := e.flushState[xtime.UnixNano(alignedstartAtNanos[0])] + consumedVal := flushState.consumedValues require.Len(t, consumedVal, 1) - require.Equal(t, 123.0, consumedVal[0].Value) - require.Equal(t, time.Unix(220, 0).UnixNano(), consumedVal[0].TimeNanos) + require.Equal(t, 123.0, consumedVal[0]) + require.Equal(t, xtime.FromSeconds(220), flushState.timestamp) // Consume all values. expectedForwardedRes = []testForwardedMetricWithMetadata{ @@ -1220,11 +1223,12 @@ func TestTimerElemConsumeCustomAggregationCustomPipeline(t *testing.T) { verifyOnForwardedFlushResult(t, expectedOnFlushedRes, *onForwardedFlushedRes) require.Equal(t, 0, len(*localRes)) require.Equal(t, 1, len(e.values)) - require.Len(t, e.consumedValues, 1) - consumedVal = e.consumedValues[xtime.UnixNano(expectedForwardedRes[1].timeNanos)] + require.Len(t, e.flushState, 1) + flushState = e.flushState[xtime.UnixNano(alignedstartAtNanos[2])] + consumedVal = flushState.consumedValues require.Len(t, consumedVal, 1) - require.Equal(t, 589.0, consumedVal[0].Value) - require.Equal(t, time.Unix(240, 0).UnixNano(), consumedVal[0].TimeNanos) + require.Equal(t, 589.0, consumedVal[0]) + require.Equal(t, xtime.FromSeconds(240), flushState.timestamp) // Tombstone the element and discard all values. e.tombstoned = true @@ -1275,7 +1279,7 @@ func TestTimerElemClose(t *testing.T) { require.Nil(t, e.cachedSourceSets) require.Equal(t, 0, len(e.values)) require.Equal(t, 0, len(e.toConsume)) - require.Equal(t, 0, len(e.consumedValues)) + require.Equal(t, 0, len(e.flushState)) require.NotNil(t, e.values) } @@ -1337,7 +1341,7 @@ func TestGaugeResetSetData(t *testing.T) { require.False(t, ge.closed) require.False(t, ge.useDefaultAggregation) require.True(t, ge.aggOpts.HasExpensiveAggregations) - require.Nil(t, ge.consumedValues) + require.Empty(t, ge.flushState) // Reset element with a pipeline containing a derivative transformation. expectedParsedPipeline := parsedPipeline{ @@ -1365,7 +1369,7 @@ func TestGaugeResetSetData(t *testing.T) { err = ge.ResetSetData(elemData) require.NoError(t, err) requirePipelinesMatch(t, expectedParsedPipeline, ge.parsedPipeline) - require.Empty(t, ge.consumedValues) + require.Empty(t, ge.flushState) } func TestGaugeElemAddUnion(t *testing.T) { @@ -1375,7 +1379,7 @@ func TestGaugeElemAddUnion(t *testing.T) { // Add a gauge metric. require.NoError(t, e.AddUnion(testTimestamps[0], testGauge, false)) require.Equal(t, 1, len(e.values)) - a, _, err := e.find(xtime.UnixNano(testAlignedStarts[0])) + a, err := e.find(xtime.UnixNano(testAlignedStarts[0])) require.NoError(t, err) v := a.lockedAgg require.Equal(t, testGauge.GaugeVal, v.aggregation.Last()) @@ -1386,7 +1390,7 @@ func TestGaugeElemAddUnion(t *testing.T) { // but still within the same aggregation interval. require.NoError(t, e.AddUnion(testTimestamps[1], testGauge, false)) require.Equal(t, 1, len(e.values)) - a, _, err = e.find(xtime.UnixNano(testAlignedStarts[0])) + a, err = e.find(xtime.UnixNano(testAlignedStarts[0])) require.NoError(t, err) v = a.lockedAgg require.Equal(t, testGauge.GaugeVal, v.aggregation.Last()) @@ -1396,7 +1400,7 @@ func TestGaugeElemAddUnion(t *testing.T) { // Add the gauge metric in the next aggregation interval. require.NoError(t, e.AddUnion(testTimestamps[2], testGauge, false)) require.Equal(t, 2, len(e.values)) - a1, _, err := e.find(xtime.UnixNano(testAlignedStarts[1])) + a1, err := e.find(xtime.UnixNano(testAlignedStarts[1])) require.NoError(t, err) v1 := a1.lockedAgg for i := 0; i < len(e.values); i++ { @@ -1420,7 +1424,7 @@ func TestGaugeElemAddUnionWithCustomAggregation(t *testing.T) { // Add a gauge metric. require.NoError(t, e.AddUnion(testTimestamps[0], testGauge, false)) require.Equal(t, 1, len(e.values)) - a, _, err := e.find(xtime.UnixNano(testAlignedStarts[0])) + a, err := e.find(xtime.UnixNano(testAlignedStarts[0])) require.NoError(t, err) v := a.lockedAgg require.Equal(t, testGauge.GaugeVal, v.aggregation.Last()) @@ -1433,7 +1437,7 @@ func TestGaugeElemAddUnionWithCustomAggregation(t *testing.T) { // but still within the same aggregation interval. require.NoError(t, e.AddUnion(testTimestamps[1], testGauge, false)) require.Equal(t, 1, len(e.values)) - a, _, err = e.find(xtime.UnixNano(testAlignedStarts[0])) + a, err = e.find(xtime.UnixNano(testAlignedStarts[0])) require.NoError(t, err) v = a.lockedAgg require.Equal(t, testGauge.GaugeVal, v.aggregation.Last()) @@ -1444,7 +1448,7 @@ func TestGaugeElemAddUnionWithCustomAggregation(t *testing.T) { // Add the gauge metric in the next aggregation interval. require.NoError(t, e.AddUnion(testTimestamps[2], testGauge, false)) require.Equal(t, 2, len(e.values)) - a1, _, err := e.find(xtime.UnixNano(testAlignedStarts[1])) + a1, err := e.find(xtime.UnixNano(testAlignedStarts[1])) require.NoError(t, err) v1 := a1.lockedAgg for i := 0; i < len(e.values); i++ { @@ -1468,7 +1472,7 @@ func TestGaugeElemAddUnique(t *testing.T) { aggregated.ForwardedMetric{Values: []float64{12.3, 34.5}}, metadata.ForwardMetadata{SourceID: source1})) require.Equal(t, 1, len(e.values)) - a, _, err := e.find(xtime.UnixNano(testAlignedStarts[0])) + a, err := e.find(xtime.UnixNano(testAlignedStarts[0])) require.NoError(t, err) v := a.lockedAgg require.Equal(t, 46.8, v.aggregation.Sum()) @@ -1486,7 +1490,7 @@ func TestGaugeElemAddUnique(t *testing.T) { aggregated.ForwardedMetric{Values: []float64{50}, Annotation: testAnnot}, metadata.ForwardMetadata{SourceID: source2})) require.Equal(t, 1, len(e.values)) - a, _, err = e.find(xtime.UnixNano(testAlignedStarts[0])) + a, err = e.find(xtime.UnixNano(testAlignedStarts[0])) require.NoError(t, err) v = a.lockedAgg require.Equal(t, 96.8, v.aggregation.Sum()) @@ -1503,7 +1507,7 @@ func TestGaugeElemAddUnique(t *testing.T) { metadata.ForwardMetadata{SourceID: source1})) require.Equal(t, testAnnot, v.aggregation.Annotation()) require.Equal(t, 2, len(e.values)) - a1, _, err := e.find(xtime.UnixNano(testAlignedStarts[1])) + a1, err := e.find(xtime.UnixNano(testAlignedStarts[1])) require.NoError(t, err) v1 := a1.lockedAgg for i := 0; i < len(e.values); i++ { @@ -1569,7 +1573,7 @@ func TestGaugeElemAddUniqueWithCustomAggregation(t *testing.T) { aggregated.ForwardedMetric{Values: []float64{1.2}}, metadata.ForwardMetadata{SourceID: source1})) require.Equal(t, 1, len(e.values)) - a, _, err := e.find(xtime.UnixNano(testAlignedStarts[0])) + a, err := e.find(xtime.UnixNano(testAlignedStarts[0])) require.NoError(t, err) v := a.lockedAgg require.Equal(t, 1.2, v.aggregation.Sum()) @@ -1586,7 +1590,7 @@ func TestGaugeElemAddUniqueWithCustomAggregation(t *testing.T) { aggregated.ForwardedMetric{Values: []float64{1.4}}, metadata.ForwardMetadata{SourceID: source2})) require.Equal(t, 1, len(e.values)) - a, _, err = e.find(xtime.UnixNano(testAlignedStarts[0])) + a, err = e.find(xtime.UnixNano(testAlignedStarts[0])) require.NoError(t, err) v = a.lockedAgg require.InEpsilon(t, 2.6, v.aggregation.Sum(), 1e-10) @@ -1597,7 +1601,7 @@ func TestGaugeElemAddUniqueWithCustomAggregation(t *testing.T) { aggregated.ForwardedMetric{Values: []float64{2.0}}, metadata.ForwardMetadata{SourceID: source1})) require.Equal(t, 2, len(e.values)) - a1, _, err := e.find(xtime.UnixNano(testAlignedStarts[1])) + a1, err := e.find(xtime.UnixNano(testAlignedStarts[1])) require.NoError(t, err) v1 := a1.lockedAgg for i := 0; i < len(e.values); i++ { @@ -1729,16 +1733,16 @@ func TestGaugeElemConsumeResendBuffer(t *testing.T) { require.Equal(t, 0, len(*onForwardedFlushedRes)) require.Equal(t, 2, len(e.values)) - a, _, err := e.find(xtime.UnixNano(testAlignedStarts[0])) + a, err := e.find(xtime.UnixNano(testAlignedStarts[0])) require.NoError(t, err) v := a.lockedAgg - a1, _, err := e.find(xtime.UnixNano(testAlignedStarts[1])) + a1, err := e.find(xtime.UnixNano(testAlignedStarts[1])) v1 := a1.lockedAgg require.NoError(t, err) require.False(t, v.dirty) - require.True(t, v.flushed) + require.True(t, e.flushState[xtime.UnixNano(testAlignedStarts[0])].flushed) require.True(t, v1.dirty) - require.False(t, v1.flushed) + require.False(t, e.flushState[xtime.UnixNano(testAlignedStarts[1])].flushed) // Consume all values. localFn, localRes = testFlushLocalMetricFn() @@ -1755,9 +1759,9 @@ func TestGaugeElemConsumeResendBuffer(t *testing.T) { require.Equal(t, 0, len(*onForwardedFlushedRes)) require.Equal(t, 2, len(e.values)) require.False(t, v.dirty) - require.True(t, v.flushed) + require.True(t, e.flushState[xtime.UnixNano(testAlignedStarts[0])].flushed) require.False(t, v1.dirty) - require.True(t, v1.flushed) + require.True(t, e.flushState[xtime.UnixNano(testAlignedStarts[1])].flushed) // Update the first value after flushing updatedVal := testGaugeVals[0] - 1.0 @@ -1779,9 +1783,9 @@ func TestGaugeElemConsumeResendBuffer(t *testing.T) { require.Equal(t, 0, len(*onForwardedFlushedRes)) require.Equal(t, 2, len(e.values)) require.False(t, v.dirty) - require.True(t, v.flushed) + require.True(t, e.flushState[xtime.UnixNano(testAlignedStarts[0])].flushed) require.False(t, v1.dirty) - require.True(t, v1.flushed) + require.True(t, e.flushState[xtime.UnixNano(testAlignedStarts[1])].flushed) // expire the values past the buffer. localFn, localRes = testFlushLocalMetricFn() @@ -1934,11 +1938,12 @@ func TestGaugeElemConsumeCustomAggregationCustomPipeline(t *testing.T) { verifyOnForwardedFlushResult(t, expectedOnFlushedRes, *onForwardedFlushedRes) require.Equal(t, 0, len(*localRes)) require.Equal(t, 3, len(e.values)) - require.Len(t, e.consumedValues, 1) - consumedVal := e.consumedValues[xtime.UnixNano(expectedForwardedRes[0].timeNanos)] + require.Len(t, e.flushState, 1) + flushedState := e.flushState[xtime.UnixNano(alignedstartAtNanos[0])] + consumedVal := flushedState.consumedValues require.Len(t, consumedVal, 1) - require.Equal(t, 123.0, consumedVal[0].Value) - require.Equal(t, time.Unix(220, 0).UnixNano(), consumedVal[0].TimeNanos) + require.Equal(t, 123.0, consumedVal[0]) + require.Equal(t, xtime.FromSeconds(220), flushedState.timestamp) // Consume all values. expectedForwardedRes = []testForwardedMetricWithMetadata{ @@ -1962,11 +1967,12 @@ func TestGaugeElemConsumeCustomAggregationCustomPipeline(t *testing.T) { verifyOnForwardedFlushResult(t, expectedOnFlushedRes, *onForwardedFlushedRes) require.Equal(t, 0, len(*localRes)) require.Equal(t, 1, len(e.values)) - require.Len(t, e.consumedValues, 1) - consumedVal = e.consumedValues[xtime.UnixNano(expectedForwardedRes[1].timeNanos)] + require.Len(t, e.flushState, 1) + flushedState = e.flushState[xtime.UnixNano(alignedstartAtNanos[2])] + consumedVal = flushedState.consumedValues require.Len(t, consumedVal, 1) - require.Equal(t, 589.0, consumedVal[0].Value) - require.Equal(t, time.Unix(240, 0).UnixNano(), consumedVal[0].TimeNanos) + require.Equal(t, 589.0, consumedVal[0]) + require.Equal(t, xtime.FromSeconds(240), flushedState.timestamp) // Tombstone the element and discard all values. e.tombstoned = true @@ -2034,16 +2040,16 @@ func TestGaugeElemResendSumReset(t *testing.T) { require.Equal(t, 0, len(*forwardRes)) require.Equal(t, 0, len(*onForwardedFlushedRes)) require.Equal(t, 2, len(e.values)) - a, _, err := e.find(xtime.UnixNano(testAlignedStarts[0])) + a, err := e.find(xtime.UnixNano(testAlignedStarts[0])) require.NoError(t, err) v := a.lockedAgg - a1, _, err := e.find(xtime.UnixNano(testAlignedStarts[1])) + a1, err := e.find(xtime.UnixNano(testAlignedStarts[1])) v1 := a1.lockedAgg require.NoError(t, err) require.False(t, v.dirty) - require.True(t, v.flushed) + require.True(t, e.flushState[xtime.UnixNano(testAlignedStarts[0])].flushed) require.True(t, v1.dirty) - require.False(t, v1.flushed) + require.False(t, e.flushState[xtime.UnixNano(testAlignedStarts[1])].flushed) // Update the first value after flushing require.NoError(t, e.AddUnique(time.Unix(0, testAlignedStarts[0]), aggregated.ForwardedMetric{ @@ -2068,9 +2074,9 @@ func TestGaugeElemResendSumReset(t *testing.T) { require.Equal(t, 0, len(*onForwardedFlushedRes)) require.Equal(t, 2, len(e.values)) require.False(t, v.dirty) - require.True(t, v.flushed) + require.True(t, e.flushState[xtime.UnixNano(testAlignedStarts[0])].flushed) require.True(t, v1.dirty) - require.False(t, v1.flushed) + require.False(t, e.flushState[xtime.UnixNano(testAlignedStarts[1])].flushed) } func TestGaugeElemResendBufferForwarding(t *testing.T) { @@ -2149,13 +2155,12 @@ func TestGaugeElemResendBufferForwarding(t *testing.T) { verifyOnForwardedFlushResult(t, expectedOnFlushedRes, *onForwardedFlushedRes) require.Equal(t, 0, len(*localRes)) require.Equal(t, 3, len(e.values)) - require.Len(t, e.consumedValues, 1) - consumedVal := e.consumedValues[xtime.UnixNano(expectedForwardedRes[0].timeNanos)] + require.Len(t, e.flushState, 1) + flushState := e.flushState[xtime.UnixNano(alignedstartAtNanos[0])] + consumedVal := flushState.consumedValues require.Len(t, consumedVal, 1) - require.Equal(t, transformation.Datapoint{ - Value: 123.0, - TimeNanos: time.Unix(220, 0).UnixNano(), - }, consumedVal[0]) + require.Equal(t, 123.0, consumedVal[0]) + require.Equal(t, xtime.FromSeconds(220), flushState.timestamp) // Consume all values. expectedForwardedRes = []testForwardedMetricWithMetadata{ @@ -2180,19 +2185,10 @@ func TestGaugeElemResendBufferForwarding(t *testing.T) { verifyOnForwardedFlushResult(t, expectedOnFlushedRes, *onForwardedFlushedRes) require.Equal(t, 0, len(*localRes)) require.Equal(t, 3, len(e.values)) - require.Len(t, e.consumedValues, 3) - require.Equal(t, transformation.Datapoint{ - Value: 123.0, - TimeNanos: time.Unix(220, 0).UnixNano(), - }, e.consumedValues[xtime.ToUnixNano(time.Unix(220, 0))][0]) - require.Equal(t, transformation.Datapoint{ - Value: 456.0, - TimeNanos: time.Unix(230, 0).UnixNano(), - }, e.consumedValues[xtime.ToUnixNano(time.Unix(230, 0))][0]) - require.Equal(t, transformation.Datapoint{ - Value: 589.0, - TimeNanos: time.Unix(240, 0).UnixNano(), - }, e.consumedValues[xtime.ToUnixNano(time.Unix(240, 0))][0]) + require.Len(t, e.flushState, 3) + require.Equal(t, 123.0, e.flushState[xtime.ToUnixNano(time.Unix(210, 0))].consumedValues[0]) + require.Equal(t, 456.0, e.flushState[xtime.ToUnixNano(time.Unix(220, 0))].consumedValues[0]) + require.Equal(t, 589.0, e.flushState[xtime.ToUnixNano(time.Unix(230, 0))].consumedValues[0]) // Update a previous value require.NoError(t, e.AddValue(time.Unix(210, 0), 124.0, nil)) @@ -2218,19 +2214,10 @@ func TestGaugeElemResendBufferForwarding(t *testing.T) { verifyOnForwardedFlushResult(t, expectedOnFlushedRes, *onForwardedFlushedRes) require.Equal(t, 0, len(*localRes)) require.Equal(t, 3, len(e.values)) - require.Len(t, e.consumedValues, 3) - require.Equal(t, transformation.Datapoint{ - Value: 124.0, - TimeNanos: time.Unix(220, 0).UnixNano(), - }, e.consumedValues[xtime.ToUnixNano(time.Unix(220, 0))][0]) - require.Equal(t, transformation.Datapoint{ - Value: 456.0, - TimeNanos: time.Unix(230, 0).UnixNano(), - }, e.consumedValues[xtime.ToUnixNano(time.Unix(230, 0))][0]) - require.Equal(t, transformation.Datapoint{ - Value: 589.0, - TimeNanos: time.Unix(240, 0).UnixNano(), - }, e.consumedValues[xtime.ToUnixNano(time.Unix(240, 0))][0]) + require.Len(t, e.flushState, 3) + require.Equal(t, 124.0, e.flushState[xtime.ToUnixNano(time.Unix(210, 0))].consumedValues[0]) + require.Equal(t, 456.0, e.flushState[xtime.ToUnixNano(time.Unix(220, 0))].consumedValues[0]) + require.Equal(t, 589.0, e.flushState[xtime.ToUnixNano(time.Unix(230, 0))].consumedValues[0]) } func TestGaugeElemReset(t *testing.T) { @@ -2279,7 +2266,10 @@ func TestGaugeElemReset(t *testing.T) { require.Equal(t, 0, len(*forwardRes)) require.Equal(t, 0, len(*onForwardedFlushedRes)) require.Equal(t, 3, len(e.values)) - require.Equal(t, 0, len(e.consumedValues)) + for _, f := range e.flushState { + require.Equal(t, 0, len(f.consumedValues)) + } + // Consume all values. localFn, localRes = testFlushLocalMetricFn() @@ -2298,7 +2288,9 @@ func TestGaugeElemReset(t *testing.T) { require.Equal(t, 0, len(*forwardRes)) require.Equal(t, 0, len(*onForwardedFlushedRes)) require.Equal(t, 1, len(e.values)) - require.Equal(t, 0, len(e.consumedValues)) + for _, f := range e.flushState { + require.Equal(t, 0, len(f.consumedValues)) + } // Tombstone the element and discard all values. e.tombstoned = true @@ -2345,7 +2337,9 @@ func TestGaugeElemClose(t *testing.T) { require.Nil(t, e.cachedSourceSets) require.Equal(t, 0, len(e.values)) require.Equal(t, 0, len(e.toConsume)) - require.Equal(t, 0, len(e.consumedValues)) + for _, f := range e.flushState { + require.Equal(t, 0, len(f.consumedValues)) + } require.NotNil(t, e.values) } @@ -2495,7 +2489,6 @@ func testCounterElem( for i, aligned := range alignedstartAtNanos { counter := &lockedCounterAggregation{ aggregation: newCounterAggregation(raggregation.NewCounter(e.aggOpts)), - prevValues: make([]float64, len(e.aggTypes)), } counter.dirty = true counter.aggregation.Update(time.Unix(0, aligned), counterVals[i], nil) @@ -2525,7 +2518,6 @@ func testTimerElem( newTimer := raggregation.NewTimer(opts.AggregationTypesOptions().Quantiles(), opts.StreamOptions(), e.aggOpts) timer := &lockedTimerAggregation{ aggregation: newTimerAggregation(newTimer), - prevValues: make([]float64, len(e.aggTypes)), } timer.dirty = true timer.aggregation.AddBatch(time.Now(), timerBatches[i], nil) @@ -2573,7 +2565,6 @@ func testGaugeElemWithData( for i, aligned := range alignedstartAtNanos { gauge := &lockedGaugeAggregation{ aggregation: newGaugeAggregation(raggregation.NewGauge(e.aggOpts)), - prevValues: make([]float64, len(e.aggTypes)), sourcesSeen: make(map[uint32]*bitset.BitSet), } gauge.dirty = true diff --git a/src/aggregator/aggregator/gauge_elem_gen.go b/src/aggregator/aggregator/gauge_elem_gen.go index fdcda286fa..de49954d6d 100644 --- a/src/aggregator/aggregator/gauge_elem_gen.go +++ b/src/aggregator/aggregator/gauge_elem_gen.go @@ -30,6 +30,7 @@ import ( "sync" "time" + raggregation "github.com/m3db/m3/src/aggregator/aggregation" "github.com/m3db/m3/src/metrics/metadata" "github.com/m3db/m3/src/metrics/metric/aggregated" "github.com/m3db/m3/src/metrics/metric/unaggregated" @@ -45,26 +46,21 @@ type lockedGaugeAggregation struct { sync.Mutex dirty bool - flushed bool closed bool sourcesSeen map[uint32]*bitset.BitSet aggregation gaugeAggregation - prevValues []float64 // the previously emitted values (one per aggregation type). } type timedGauge struct { startAtNanos xtime.UnixNano // start time of an aggregation window lockedAgg *lockedGaugeAggregation resendEnabled bool - - // this is mutable data for specifying on each Consume which previous value the - // current agg can reference (i.e. for binary ops). it must be mutable since the - // set of vals within the buffer past can change and so on each consume a given agg's - // previous depends on the state of values preceding the current at that point in time. - previousTimeNanos xtime.UnixNano + inDirtySet bool } -func (ta *timedGauge) Release() { +// close is called when the aggregation has been expired or the element is being closed. +func (ta *timedGauge) close() { + ta.lockedAgg.aggregation.Close() ta.lockedAgg = nil } @@ -75,25 +71,26 @@ type GaugeElem struct { // startTime -> agg (new one per every resolution) values map[xtime.UnixNano]timedGauge + // startTime -> state. this is local state to the flusher and does not need to guarded with a lock. + // values and flushState should always have the exact same key set. + flushState map[xtime.UnixNano]aggFlushState // sorted start aligned times that have been written to since the last flush dirty []xtime.UnixNano // min time in the values map. allow for iterating through map. minStartTime xtime.UnixNano // internal consume state that does not need to be synchronized. - toConsume []timedGauge // small buffer to avoid memory allocations during consumption - toExpire []timedGauge // small buffer to avoid memory allocations during consumption - // map of the previous consumed values for each timestamp in the buffer. needed to support binary transforms that - // need the value from the previous timestamp. - consumedValues valuesByTime + toConsume []aggFlushState // small buffer to avoid memory allocations during consumption + flushStateToExpire []xtime.UnixNano // small buffer to avoid memory allocations during consumption } // NewGaugeElem returns a new GaugeElem. func NewGaugeElem(data ElemData, opts ElemOptions) (*GaugeElem, error) { e := &GaugeElem{ - elemBase: newElemBase(opts), - dirty: make([]xtime.UnixNano, 0, defaultNumAggregations), // in most cases values will have two entries - values: make(map[xtime.UnixNano]timedGauge), + elemBase: newElemBase(opts), + dirty: make([]xtime.UnixNano, 0, defaultNumAggregations), // in most cases values will have two entries + values: make(map[xtime.UnixNano]timedGauge), + flushState: make(map[xtime.UnixNano]aggFlushState), } if err := e.ResetSetData(data); err != nil { return nil, err @@ -137,6 +134,7 @@ func (e *GaugeElem) AddUnion(timestamp time.Time, mu unaggregated.MetricUnion, r return errAggregationClosed } lockedAgg.aggregation.AddUnion(timestamp, mu) + lockedAgg.dirty = true lockedAgg.Unlock() return nil } @@ -154,6 +152,7 @@ func (e *GaugeElem) AddValue(timestamp time.Time, value float64, annotation []by return errAggregationClosed } lockedAgg.aggregation.Add(timestamp, value, annotation) + lockedAgg.dirty = true lockedAgg.Unlock() return nil } @@ -205,6 +204,7 @@ func (e *GaugeElem) AddUnique( lockedAgg.aggregation.Add(timestamp, v, metric.Annotation) } } + lockedAgg.dirty = true lockedAgg.Unlock() return nil } @@ -212,9 +212,8 @@ func (e *GaugeElem) AddUnique( // remove expired aggregations from the values map. func (e *GaugeElem) expireValuesWithLock( targetNanos int64, - timestampNanosFn timestampNanosFn, isEarlierThanFn isEarlierThanFn) { - e.toExpire = e.toExpire[:0] + e.flushStateToExpire = e.flushStateToExpire[:0] if len(e.values) == 0 { return } @@ -232,22 +231,54 @@ func (e *GaugeElem) expireValuesWithLock( break } } - // if this current value is eligible to be expired it will no longer be written to. this means it's safe - // to remove the _previous_ value since it will no longer be needed for binary transformations. when the + + // close the agg to prevent any more writes. + dirty := false + currV.lockedAgg.Lock() + currV.lockedAgg.closed = true + dirty = currV.lockedAgg.dirty + currV.lockedAgg.Unlock() + if dirty { + // a race occurred and a write happened before we could close the aggregation. will expire next time. + break + } + + // if this current value is closed and clean it will no longer be flushed. this means it's safe + // to remove the previous value since it will no longer be needed for binary transformations. when the // next value is eligible to be expired, this current value will actually be removed. if prevV, ok := e.values[e.minStartTime]; ok { - // Previous times are used to key into consumedValues, which are non-start-aligned. And so - // we convert from startAligned here when setting previous. - prevV.previousTimeNanos = xtime.UnixNano(timestampNanosFn(int64(e.minStartTime), resolution)) - e.toExpire = append(e.toExpire, prevV) + // can't expire flush state until after the flushing, so we save the time to expire later. + e.flushStateToExpire = append(e.flushStateToExpire, e.minStartTime) delete(e.values, e.minStartTime) e.minStartTime = currStart + + // it's safe to access this outside the agg lock since it was closed in a previous iteration. + // This is to make sure there aren't too many cached source sets taking up + // too much space. + if prevV.lockedAgg.sourcesSeen != nil && len(e.cachedSourceSets) < e.opts.MaxNumCachedSourceSets() { + e.cachedSourceSets = append(e.cachedSourceSets, prevV.lockedAgg.sourcesSeen) + } + prevV.close() } } currStart = currStart.Add(resolution) } } +func (e *GaugeElem) expireFlushState() { + for _, t := range e.flushStateToExpire { + prevFlushState, ok := e.flushState[t] + if !ok { + ts := t.ToTime() + instrument.EmitAndLogInvariantViolation(e.opts.InstrumentOptions(), func(l *zap.Logger) { + l.Error("expire time not in state map", zap.Time("ts", ts)) + }) + } + prevFlushState.close() + delete(e.flushState, t) + } +} + // return the timestamp in the values map that is before the provided time. returns false if the provided time is the // smallest time or the map is empty. func (e *GaugeElem) previousStartAlignedWithLock(timestamp xtime.UnixNano) (xtime.UnixNano, bool) { @@ -255,6 +286,7 @@ func (e *GaugeElem) previousStartAlignedWithLock(timestamp xtime.UnixNano) (xtim return 0, false } resolution := e.sp.Resolution().Window + // ensure the input is start aligned and then calculate the previous start time. startAligned := timestamp.Truncate(resolution).Add(-resolution) for !startAligned.Before(e.minStartTime) { _, ok := e.values[startAligned] @@ -308,80 +340,55 @@ func (e *GaugeElem) Consume( return false } e.toConsume = e.toConsume[:0] - // Evaluate and GC expired items. dirtyTimes := e.dirty e.dirty = e.dirty[:0] - for _, dirtyTime := range dirtyTimes { - agg, ok := e.values[dirtyTime] - if !ok { - dirtyTime := dirtyTime - instrument.EmitAndLogInvariantViolation(e.opts.InstrumentOptions(), func(l *zap.Logger) { - l.Error("dirty timestamp not in map", zap.Time("ts", dirtyTime.ToTime())) - }) - } - - if !isEarlierThanFn(int64(dirtyTime), resolution, targetNanos) { + for i, dirtyTime := range dirtyTimes { + flushState, ready := e.updateFlushStateWithLock(dirtyTime, targetNanos, isEarlierThanFn) + if !ready || !flushState.dirty { + // there is a race where the value was added to the dirty set, but the writer didn't actually update the + // value yet (by marking dirty). add back to the dirty set so it can be processed in the next round once + // the value has been updated. e.dirty = append(e.dirty, dirtyTime) continue } + val := e.values[dirtyTime] + val.inDirtySet = false + e.values[dirtyTime] = val + e.toConsume = append(e.toConsume, flushState) - agg.lockedAgg.dirty = false - previousStartAligned, ok := e.previousStartAlignedWithLock(dirtyTime) - if ok { - agg.previousTimeNanos = xtime.UnixNano(timestampNanosFn(int64(previousStartAligned), resolution)) - } - e.toConsume = append(e.toConsume, agg) - - // add the nextAgg to the dirty set as well in case we need to cascade the value. + // potentially consume the nextAgg as well in case we need to cascade an update due to a resend. nextAgg, ok := e.nextAggWithLock(dirtyTime, targetNanos, isEarlierThanFn) - // only add nextAgg if not already in the dirty set - if ok && !nextAgg.lockedAgg.dirty { - nextAgg.previousTimeNanos = xtime.UnixNano(timestampNanosFn(int64(dirtyTime), resolution)) - e.toConsume = append(e.toConsume, nextAgg) + if ok && (i == len(dirtyTimes)-1 || dirtyTimes[i+1] != nextAgg.startAtNanos) { + // if not already in the dirty set. + flushState, ready := e.updateFlushStateWithLock(nextAgg.startAtNanos, targetNanos, isEarlierThanFn) + if ready { + e.toConsume = append(e.toConsume, flushState) + } } } - e.expireValuesWithLock(targetNanos, timestampNanosFn, isEarlierThanFn) + // expire the values and aggregations while we still hold the lock. + e.expireValuesWithLock(targetNanos, isEarlierThanFn) canCollect := len(e.dirty) == 0 && e.tombstoned e.Unlock() // Process the aggregations that are ready for consumption. - for i := range e.toConsume { - timeNanos := xtime.UnixNano(timestampNanosFn(int64(e.toConsume[i].startAtNanos), resolution)) - e.toConsume[i].lockedAgg.Lock() - _ = e.processValueWithAggregationLock( - timeNanos, - e.toConsume[i].previousTimeNanos, - e.toConsume[i].resendEnabled, - e.toConsume[i].lockedAgg, + for _, flushState := range e.toConsume { + flushState.timestamp = xtime.UnixNano(timestampNanosFn(int64(flushState.startAt), resolution)) + flushState = e.processValueWithAggregation( + flushState, flushLocalFn, flushForwardedFn, resolution, latenessAllowed, ) - e.toConsume[i].lockedAgg.flushed = true - e.toConsume[i].lockedAgg.Unlock() - } - - // Cleanup expired item after consuming since consuming still has a ref to the locked aggregation. - for i := range e.toExpire { - e.toExpire[i].lockedAgg.closed = true - e.toExpire[i].lockedAgg.aggregation.Close() - if e.toExpire[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.toExpire[i].lockedAgg.sourcesSeen) - } - e.cachedSourceSetsLock.Unlock() - e.toExpire[i].lockedAgg.sourcesSeen = nil - } - e.toExpire[i].Release() - - delete(e.consumedValues, e.toExpire[i].previousTimeNanos) + flushState.flushed = true + e.flushState[flushState.startAt] = flushState } + // expire the flush state after processing since it's needed in the processing. + e.expireFlushState() + if e.parsedPipeline.HasRollup { forwardedAggregationKey, _ := e.ForwardedAggregationKey() onForwardedFlushedFn(e.onForwardedAggregationWrittenFn, forwardedAggregationKey) @@ -390,6 +397,46 @@ func (e *GaugeElem) Consume( return canCollect } +func (e *GaugeElem) updateFlushStateWithLock(dirtyTime xtime.UnixNano, targetNanos int64, + isEarlierThanFn isEarlierThanFn) (aggFlushState, bool) { + resolution := e.sp.Resolution().Window + agg, ok := e.values[dirtyTime] + if !ok { + ts := dirtyTime.ToTime() + instrument.EmitAndLogInvariantViolation(e.opts.InstrumentOptions(), func(l *zap.Logger) { + l.Error("dirty timestamp not in values map", zap.Time("ts", ts)) + }) + } + if !isEarlierThanFn(int64(dirtyTime), resolution, targetNanos) { + return aggFlushState{}, false + } + + // note: flushState might be empty for the first flush + flushState := e.flushState[dirtyTime] + // copy the lockedAgg data to the flushState while holding the lock. + agg.lockedAgg.Lock() + flushState.dirty = agg.lockedAgg.dirty + flushState.values = flushState.values[:0] + for _, aggType := range e.aggTypes { + flushState.values = append(flushState.values, agg.lockedAgg.aggregation.ValueOf(aggType)) + } + flushState.annotation = raggregation.MaybeReplaceAnnotation( + flushState.annotation, agg.lockedAgg.aggregation.Annotation()) + agg.lockedAgg.dirty = false + agg.lockedAgg.Unlock() + + // update the flushState with everything else. + previousStartAligned, ok := e.previousStartAlignedWithLock(dirtyTime) + if ok { + flushState.prevStartTime = previousStartAligned + } else { + flushState.prevStartTime = 0 + } + flushState.resendEnabled = agg.resendEnabled + flushState.startAt = dirtyTime + return flushState, true +} + // Close closes the element. func (e *GaugeElem) Close() { e.Lock() @@ -407,39 +454,44 @@ func (e *GaugeElem) Close() { } e.cachedSourceSets = nil - resolution := e.sp.Resolution().Window // note: this is not in the hot path so it's ok to iterate over the map. // this allows to catch any bugs with unexpected entries still in the map. - for k := range e.values { - if k < e.minStartTime { + minStartTime := e.minStartTime + for k, v := range e.values { + if k < minStartTime { k := k + ts := e.minStartTime.ToTime() instrument.EmitAndLogInvariantViolation(e.opts.InstrumentOptions(), func(l *zap.Logger) { - l.Error("aggregate timestamp is less than min", + l.Error("value timestamp is less than min start time", zap.Time("ts", k.ToTime()), - zap.Time("min", e.minStartTime.ToTime())) + zap.Time("min", ts)) }) } + v.close() delete(e.values, k) - // Close the underlying aggregation objects. - if v, ok := e.values[e.minStartTime]; ok { - v.lockedAgg.sourcesSeen = nil - v.lockedAgg.aggregation.Close() - v.Release() - delete(e.values, e.minStartTime) + flushState, ok := e.flushState[k] + if ok { + flushState.close() } - e.minStartTime = e.minStartTime.Add(resolution) + delete(e.flushState, k) + } + // clean up any dangling flush state that should never exist. + for k, v := range e.flushState { + ts := k.ToTime() + instrument.EmitAndLogInvariantViolation(e.opts.InstrumentOptions(), func(l *zap.Logger) { + l.Error("dangling state timestamp", zap.Time("ts", ts)) + }) + v.close() + delete(e.flushState, k) } e.gaugeElemBase.Close() aggTypesPool := e.aggTypesOpts.TypesPool() pool := e.ElemPool(e.opts) - e.Unlock() - - // internal consumption state that doesn't need to be synchronized. - e.toConsume = e.toConsume[:0] e.dirty = e.dirty[:0] - e.toExpire = e.toExpire[:0] - e.consumedValues = nil + e.toConsume = e.toConsume[:0] + e.flushStateToExpire = e.flushStateToExpire[:0] e.minStartTime = 0 + e.Unlock() if !e.useDefaultAggregation { aggTypesPool.Put(e.aggTypes) @@ -479,20 +531,19 @@ func (e *GaugeElem) insertDirty(alignedStart xtime.UnixNano) { // find finds the aggregation for a given time, or returns nil. //nolint: dupl -func (e *GaugeElem) find(alignedStartNanos xtime.UnixNano) (timedGauge, bool, error) { +func (e *GaugeElem) find(alignedStartNanos xtime.UnixNano) (timedGauge, error) { e.RLock() if e.closed { e.RUnlock() - return timedGauge{}, false, errElemClosed + return timedGauge{}, errElemClosed } timedAgg, ok := e.values[alignedStartNanos] if ok { - dirty := timedAgg.lockedAgg.dirty e.RUnlock() - return timedAgg, dirty, nil + return timedAgg, nil } e.RUnlock() - return timedGauge{}, false, nil + return timedGauge{}, nil } // findOrCreate finds the aggregation for a given time, or creates one @@ -503,12 +554,12 @@ func (e *GaugeElem) findOrCreate( createOpts createAggregationOptions, ) (*lockedGaugeAggregation, error) { alignedStart := xtime.UnixNano(alignedStartNanos) - found, isDirty, err := e.find(alignedStart) + found, err := e.find(alignedStart) if err != nil { return nil, err } // if the aggregation is found and does not need to be updated, return as is. - if found.lockedAgg != nil && isDirty && found.resendEnabled == createOpts.resendEnabled { + if found.lockedAgg != nil && found.inDirtySet && found.resendEnabled == createOpts.resendEnabled { return found.lockedAgg, err } @@ -520,9 +571,9 @@ func (e *GaugeElem) findOrCreate( timedAgg, ok := e.values[alignedStart] if ok { - // if the agg is not dirty, mark it dirty so it will be flushed. - if !timedAgg.lockedAgg.dirty { - timedAgg.lockedAgg.dirty = true + // add to dirty set so it will be flushed. + if !timedAgg.inDirtySet { + timedAgg.inDirtySet = true e.insertDirty(alignedStart) } // ensure the resendEnabled state is the latest. @@ -534,7 +585,6 @@ func (e *GaugeElem) findOrCreate( var sourcesSeen map[uint32]*bitset.BitSet if createOpts.initSourceSet { - e.cachedSourceSetsLock.Lock() if numCachedSourceSets := len(e.cachedSourceSets); numCachedSourceSets > 0 { sourcesSeen = e.cachedSourceSets[numCachedSourceSets-1] e.cachedSourceSets[numCachedSourceSets-1] = nil @@ -545,17 +595,15 @@ func (e *GaugeElem) findOrCreate( } else { sourcesSeen = make(map[uint32]*bitset.BitSet) } - e.cachedSourceSetsLock.Unlock() } timedAgg = timedGauge{ startAtNanos: alignedStart, lockedAgg: &lockedGaugeAggregation{ sourcesSeen: sourcesSeen, aggregation: e.NewAggregation(e.opts, e.aggOpts), - prevValues: make([]float64, len(e.aggTypes)), - dirty: true, }, resendEnabled: createOpts.resendEnabled, + inDirtySet: true, } e.values[alignedStart] = timedAgg e.insertDirty(alignedStart) @@ -567,23 +615,19 @@ func (e *GaugeElem) findOrCreate( } // returns true if a datapoint is emitted. -func (e *GaugeElem) processValueWithAggregationLock( - timeNanos xtime.UnixNano, - prevTimeNanos xtime.UnixNano, - resendEnabled bool, - lockedAgg *lockedGaugeAggregation, +func (e *GaugeElem) processValueWithAggregation( + flushState aggFlushState, flushLocalFn flushLocalMetricFn, flushForwardedFn flushForwardedMetricFn, resolution time.Duration, - latenessAllowed time.Duration) bool { + latenessAllowed time.Duration) aggFlushState { var ( transformations = e.parsedPipeline.Transformations discardNaNValues = e.opts.DiscardNaNAggregatedValues() - emitted bool ) for aggTypeIdx, aggType := range e.aggTypes { var extraDp transformation.Datapoint - value := lockedAgg.aggregation.ValueOf(aggType) + value := flushState.values[aggTypeIdx] for _, transformOp := range transformations { unaryOp, isUnaryOp := transformOp.UnaryTransform() binaryOp, isBinaryOp := transformOp.BinaryTransform() @@ -591,7 +635,7 @@ func (e *GaugeElem) processValueWithAggregationLock( switch { case isUnaryOp: curr := transformation.Datapoint{ - TimeNanos: int64(timeNanos), + TimeNanos: int64(flushState.timestamp), Value: value, } @@ -603,15 +647,21 @@ func (e *GaugeElem) processValueWithAggregationLock( prev := transformation.Datapoint{ Value: nan, } - // lazily construct consumedValues since they are only needed by binary transforms. - if e.consumedValues == nil { - e.consumedValues = make(valuesByTime) - } - if _, ok := e.consumedValues[prevTimeNanos]; ok { - prev = e.consumedValues[prevTimeNanos][aggTypeIdx] + if flushState.prevStartTime > 0 { + prevFlushState, ok := e.flushState[flushState.prevStartTime] + if !ok { + ts := flushState.prevStartTime.ToTime() + instrument.EmitAndLogInvariantViolation(e.opts.InstrumentOptions(), func(l *zap.Logger) { + l.Error("previous start time not in state map", + zap.Time("ts", ts)) + }) + } else { + prev.Value = prevFlushState.consumedValues[aggTypeIdx] + prev.TimeNanos = int64(prevFlushState.timestamp) + } } curr := transformation.Datapoint{ - TimeNanos: int64(timeNanos), + TimeNanos: int64(flushState.timestamp), Value: value, } res := binaryOp.Evaluate(prev, curr, transformation.FeatureFlags{}) @@ -621,16 +671,15 @@ func (e *GaugeElem) processValueWithAggregationLock( // need to keep one value. In the future if we need to support higher-order // derivative transformations, we need to store an array of values here. if !math.IsNaN(curr.Value) { - if e.consumedValues[timeNanos] == nil { - e.consumedValues[timeNanos] = make([]transformation.Datapoint, len(e.aggTypes)) + if flushState.consumedValues == nil { + flushState.consumedValues = make([]float64, len(e.aggTypes)) } - e.consumedValues[timeNanos][aggTypeIdx] = curr + flushState.consumedValues[aggTypeIdx] = curr.Value } - value = res.Value case isUnaryMultiOp: curr := transformation.Datapoint{ - TimeNanos: int64(timeNanos), + TimeNanos: int64(flushState.timestamp), Value: value, } @@ -646,20 +695,24 @@ func (e *GaugeElem) processValueWithAggregationLock( // It's ok to send a 0 prevValue on the first forward because it's not used in AddUnique unless it's a // resend (version > 0) - prevValue := lockedAgg.prevValues[aggTypeIdx] - lockedAgg.prevValues[aggTypeIdx] = value - if lockedAgg.flushed { + var prevValue float64 + if flushState.emittedValues == nil { + flushState.emittedValues = make([]float64, len(e.aggTypes)) + } else { + prevValue = flushState.emittedValues[aggTypeIdx] + } + flushState.emittedValues[aggTypeIdx] = value + if flushState.flushed { // no need to resend a value that hasn't changed. if (math.IsNaN(prevValue) && math.IsNaN(value)) || (prevValue == value) { continue } } - emitted = true if !e.parsedPipeline.HasRollup { toFlush := make([]transformation.Datapoint, 0, 2) toFlush = append(toFlush, transformation.Datapoint{ - TimeNanos: int64(timeNanos), + TimeNanos: int64(flushState.timestamp), Value: value, }) if extraDp.TimeNanos != 0 { @@ -668,22 +721,24 @@ func (e *GaugeElem) processValueWithAggregationLock( for _, point := range toFlush { switch e.idPrefixSuffixType { case NoPrefixNoSuffix: - flushLocalFn(nil, e.id, nil, point.TimeNanos, point.Value, lockedAgg.aggregation.Annotation(), e.sp) + flushLocalFn(nil, e.id, nil, point.TimeNanos, point.Value, flushState.annotation, + e.sp) case WithPrefixWithSuffix: flushLocalFn(e.FullPrefix(e.opts), e.id, e.TypeStringFor(e.aggTypesOpts, aggType), - point.TimeNanos, point.Value, lockedAgg.aggregation.Annotation(), e.sp) + point.TimeNanos, point.Value, flushState.annotation, e.sp) } } } else { forwardedAggregationKey, _ := e.ForwardedAggregationKey() // only record lag for the initial flush (not resends) - if !lockedAgg.flushed { + if !flushState.flushed { // latenessAllowed is not due to processing delay, so it remove it from lag calc. - e.forwardLagMetric(resolution).RecordDuration(time.Since(timeNanos.ToTime().Add(-latenessAllowed))) + e.forwardLagMetric(resolution).RecordDuration( + time.Since(flushState.timestamp.ToTime().Add(-latenessAllowed))) } flushForwardedFn(e.writeForwardedMetricFn, forwardedAggregationKey, - int64(timeNanos), value, prevValue, lockedAgg.aggregation.Annotation(), resendEnabled) + int64(flushState.timestamp), value, prevValue, flushState.annotation, flushState.resendEnabled) } } - return emitted + return flushState } diff --git a/src/aggregator/aggregator/generic_elem.go b/src/aggregator/aggregator/generic_elem.go index 17fc47d1c1..feb324da1a 100644 --- a/src/aggregator/aggregator/generic_elem.go +++ b/src/aggregator/aggregator/generic_elem.go @@ -109,26 +109,21 @@ type lockedAggregation struct { sync.Mutex dirty bool - flushed bool closed bool sourcesSeen map[uint32]*bitset.BitSet aggregation typeSpecificAggregation - prevValues []float64 // the previously emitted values (one per aggregation type). } type timedAggregation struct { startAtNanos xtime.UnixNano // start time of an aggregation window lockedAgg *lockedAggregation resendEnabled bool - - // this is mutable data for specifying on each Consume which previous value the - // current agg can reference (i.e. for binary ops). it must be mutable since the - // set of vals within the buffer past can change and so on each consume a given agg's - // previous depends on the state of values preceding the current at that point in time. - previousTimeNanos xtime.UnixNano + inDirtySet bool } -func (ta *timedAggregation) Release() { +// close is called when the aggregation has been expired or the element is being closed. +func (ta *timedAggregation) close() { + ta.lockedAgg.aggregation.Close() ta.lockedAgg = nil } @@ -139,25 +134,26 @@ type GenericElem struct { // startTime -> agg (new one per every resolution) values map[xtime.UnixNano]timedAggregation + // startTime -> state. this is local state to the flusher and does not need to guarded with a lock. + // values and flushState should always have the exact same key set. + flushState map[xtime.UnixNano]aggFlushState // sorted start aligned times that have been written to since the last flush dirty []xtime.UnixNano // min time in the values map. allow for iterating through map. minStartTime xtime.UnixNano // internal consume state that does not need to be synchronized. - toConsume []timedAggregation // small buffer to avoid memory allocations during consumption - toExpire []timedAggregation // small buffer to avoid memory allocations during consumption - // map of the previous consumed values for each timestamp in the buffer. needed to support binary transforms that - // need the value from the previous timestamp. - consumedValues valuesByTime + toConsume []aggFlushState // small buffer to avoid memory allocations during consumption + flushStateToExpire []xtime.UnixNano // small buffer to avoid memory allocations during consumption } // NewGenericElem returns a new GenericElem. func NewGenericElem(data ElemData, opts ElemOptions) (*GenericElem, error) { e := &GenericElem{ - elemBase: newElemBase(opts), - dirty: make([]xtime.UnixNano, 0, defaultNumAggregations), // in most cases values will have two entries - values: make(map[xtime.UnixNano]timedAggregation), + elemBase: newElemBase(opts), + dirty: make([]xtime.UnixNano, 0, defaultNumAggregations), // in most cases values will have two entries + values: make(map[xtime.UnixNano]timedAggregation), + flushState: make(map[xtime.UnixNano]aggFlushState), } if err := e.ResetSetData(data); err != nil { return nil, err @@ -201,6 +197,7 @@ func (e *GenericElem) AddUnion(timestamp time.Time, mu unaggregated.MetricUnion, return errAggregationClosed } lockedAgg.aggregation.AddUnion(timestamp, mu) + lockedAgg.dirty = true lockedAgg.Unlock() return nil } @@ -218,6 +215,7 @@ func (e *GenericElem) AddValue(timestamp time.Time, value float64, annotation [] return errAggregationClosed } lockedAgg.aggregation.Add(timestamp, value, annotation) + lockedAgg.dirty = true lockedAgg.Unlock() return nil } @@ -269,6 +267,7 @@ func (e *GenericElem) AddUnique( lockedAgg.aggregation.Add(timestamp, v, metric.Annotation) } } + lockedAgg.dirty = true lockedAgg.Unlock() return nil } @@ -276,9 +275,8 @@ func (e *GenericElem) AddUnique( // remove expired aggregations from the values map. func (e *GenericElem) expireValuesWithLock( targetNanos int64, - timestampNanosFn timestampNanosFn, isEarlierThanFn isEarlierThanFn) { - e.toExpire = e.toExpire[:0] + e.flushStateToExpire = e.flushStateToExpire[:0] if len(e.values) == 0 { return } @@ -296,22 +294,54 @@ func (e *GenericElem) expireValuesWithLock( break } } - // if this current value is eligible to be expired it will no longer be written to. this means it's safe - // to remove the _previous_ value since it will no longer be needed for binary transformations. when the + + // close the agg to prevent any more writes. + dirty := false + currV.lockedAgg.Lock() + currV.lockedAgg.closed = true + dirty = currV.lockedAgg.dirty + currV.lockedAgg.Unlock() + if dirty { + // a race occurred and a write happened before we could close the aggregation. will expire next time. + break + } + + // if this current value is closed and clean it will no longer be flushed. this means it's safe + // to remove the previous value since it will no longer be needed for binary transformations. when the // next value is eligible to be expired, this current value will actually be removed. if prevV, ok := e.values[e.minStartTime]; ok { - // Previous times are used to key into consumedValues, which are non-start-aligned. And so - // we convert from startAligned here when setting previous. - prevV.previousTimeNanos = xtime.UnixNano(timestampNanosFn(int64(e.minStartTime), resolution)) - e.toExpire = append(e.toExpire, prevV) + // can't expire flush state until after the flushing, so we save the time to expire later. + e.flushStateToExpire = append(e.flushStateToExpire, e.minStartTime) delete(e.values, e.minStartTime) e.minStartTime = currStart + + // it's safe to access this outside the agg lock since it was closed in a previous iteration. + // This is to make sure there aren't too many cached source sets taking up + // too much space. + if prevV.lockedAgg.sourcesSeen != nil && len(e.cachedSourceSets) < e.opts.MaxNumCachedSourceSets() { + e.cachedSourceSets = append(e.cachedSourceSets, prevV.lockedAgg.sourcesSeen) + } + prevV.close() } } currStart = currStart.Add(resolution) } } +func (e *GenericElem) expireFlushState() { + for _, t := range e.flushStateToExpire { + prevFlushState, ok := e.flushState[t] + if !ok { + ts := t.ToTime() + instrument.EmitAndLogInvariantViolation(e.opts.InstrumentOptions(), func(l *zap.Logger) { + l.Error("expire time not in state map", zap.Time("ts", ts)) + }) + } + prevFlushState.close() + delete(e.flushState, t) + } +} + // return the timestamp in the values map that is before the provided time. returns false if the provided time is the // smallest time or the map is empty. func (e *GenericElem) previousStartAlignedWithLock(timestamp xtime.UnixNano) (xtime.UnixNano, bool) { @@ -319,6 +349,7 @@ func (e *GenericElem) previousStartAlignedWithLock(timestamp xtime.UnixNano) (xt return 0, false } resolution := e.sp.Resolution().Window + // ensure the input is start aligned and then calculate the previous start time. startAligned := timestamp.Truncate(resolution).Add(-resolution) for !startAligned.Before(e.minStartTime) { _, ok := e.values[startAligned] @@ -372,80 +403,55 @@ func (e *GenericElem) Consume( return false } e.toConsume = e.toConsume[:0] - // Evaluate and GC expired items. dirtyTimes := e.dirty e.dirty = e.dirty[:0] - for _, dirtyTime := range dirtyTimes { - agg, ok := e.values[dirtyTime] - if !ok { - dirtyTime := dirtyTime - instrument.EmitAndLogInvariantViolation(e.opts.InstrumentOptions(), func(l *zap.Logger) { - l.Error("dirty timestamp not in map", zap.Time("ts", dirtyTime.ToTime())) - }) - } - - if !isEarlierThanFn(int64(dirtyTime), resolution, targetNanos) { + for i, dirtyTime := range dirtyTimes { + flushState, ready := e.updateFlushStateWithLock(dirtyTime, targetNanos, isEarlierThanFn) + if !ready || !flushState.dirty { + // there is a race where the value was added to the dirty set, but the writer didn't actually update the + // value yet (by marking dirty). add back to the dirty set so it can be processed in the next round once + // the value has been updated. e.dirty = append(e.dirty, dirtyTime) continue } + val := e.values[dirtyTime] + val.inDirtySet = false + e.values[dirtyTime] = val + e.toConsume = append(e.toConsume, flushState) - agg.lockedAgg.dirty = false - previousStartAligned, ok := e.previousStartAlignedWithLock(dirtyTime) - if ok { - agg.previousTimeNanos = xtime.UnixNano(timestampNanosFn(int64(previousStartAligned), resolution)) - } - e.toConsume = append(e.toConsume, agg) - - // add the nextAgg to the dirty set as well in case we need to cascade the value. + // potentially consume the nextAgg as well in case we need to cascade an update due to a resend. nextAgg, ok := e.nextAggWithLock(dirtyTime, targetNanos, isEarlierThanFn) - // only add nextAgg if not already in the dirty set - if ok && !nextAgg.lockedAgg.dirty { - nextAgg.previousTimeNanos = xtime.UnixNano(timestampNanosFn(int64(dirtyTime), resolution)) - e.toConsume = append(e.toConsume, nextAgg) + if ok && (i == len(dirtyTimes)-1 || dirtyTimes[i+1] != nextAgg.startAtNanos) { + // if not already in the dirty set. + flushState, ready := e.updateFlushStateWithLock(nextAgg.startAtNanos, targetNanos, isEarlierThanFn) + if ready { + e.toConsume = append(e.toConsume, flushState) + } } } - e.expireValuesWithLock(targetNanos, timestampNanosFn, isEarlierThanFn) + // expire the values and aggregations while we still hold the lock. + e.expireValuesWithLock(targetNanos, isEarlierThanFn) canCollect := len(e.dirty) == 0 && e.tombstoned e.Unlock() // Process the aggregations that are ready for consumption. - for i := range e.toConsume { - timeNanos := xtime.UnixNano(timestampNanosFn(int64(e.toConsume[i].startAtNanos), resolution)) - e.toConsume[i].lockedAgg.Lock() - _ = e.processValueWithAggregationLock( - timeNanos, - e.toConsume[i].previousTimeNanos, - e.toConsume[i].resendEnabled, - e.toConsume[i].lockedAgg, + for _, flushState := range e.toConsume { + flushState.timestamp = xtime.UnixNano(timestampNanosFn(int64(flushState.startAt), resolution)) + flushState = e.processValueWithAggregation( + flushState, flushLocalFn, flushForwardedFn, resolution, latenessAllowed, ) - e.toConsume[i].lockedAgg.flushed = true - e.toConsume[i].lockedAgg.Unlock() - } - - // Cleanup expired item after consuming since consuming still has a ref to the locked aggregation. - for i := range e.toExpire { - e.toExpire[i].lockedAgg.closed = true - e.toExpire[i].lockedAgg.aggregation.Close() - if e.toExpire[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.toExpire[i].lockedAgg.sourcesSeen) - } - e.cachedSourceSetsLock.Unlock() - e.toExpire[i].lockedAgg.sourcesSeen = nil - } - e.toExpire[i].Release() - - delete(e.consumedValues, e.toExpire[i].previousTimeNanos) + flushState.flushed = true + e.flushState[flushState.startAt] = flushState } + // expire the flush state after processing since it's needed in the processing. + e.expireFlushState() + if e.parsedPipeline.HasRollup { forwardedAggregationKey, _ := e.ForwardedAggregationKey() onForwardedFlushedFn(e.onForwardedAggregationWrittenFn, forwardedAggregationKey) @@ -454,6 +460,46 @@ func (e *GenericElem) Consume( return canCollect } +func (e *GenericElem) updateFlushStateWithLock(dirtyTime xtime.UnixNano, targetNanos int64, + isEarlierThanFn isEarlierThanFn) (aggFlushState, bool) { + resolution := e.sp.Resolution().Window + agg, ok := e.values[dirtyTime] + if !ok { + ts := dirtyTime.ToTime() + instrument.EmitAndLogInvariantViolation(e.opts.InstrumentOptions(), func(l *zap.Logger) { + l.Error("dirty timestamp not in values map", zap.Time("ts", ts)) + }) + } + if !isEarlierThanFn(int64(dirtyTime), resolution, targetNanos) { + return aggFlushState{}, false + } + + // note: flushState might be empty for the first flush + flushState := e.flushState[dirtyTime] + // copy the lockedAgg data to the flushState while holding the lock. + agg.lockedAgg.Lock() + flushState.dirty = agg.lockedAgg.dirty + flushState.values = flushState.values[:0] + for _, aggType := range e.aggTypes { + flushState.values = append(flushState.values, agg.lockedAgg.aggregation.ValueOf(aggType)) + } + flushState.annotation = raggregation.MaybeReplaceAnnotation( + flushState.annotation, agg.lockedAgg.aggregation.Annotation()) + agg.lockedAgg.dirty = false + agg.lockedAgg.Unlock() + + // update the flushState with everything else. + previousStartAligned, ok := e.previousStartAlignedWithLock(dirtyTime) + if ok { + flushState.prevStartTime = previousStartAligned + } else { + flushState.prevStartTime = 0 + } + flushState.resendEnabled = agg.resendEnabled + flushState.startAt = dirtyTime + return flushState, true +} + // Close closes the element. func (e *GenericElem) Close() { e.Lock() @@ -471,39 +517,44 @@ func (e *GenericElem) Close() { } e.cachedSourceSets = nil - resolution := e.sp.Resolution().Window // note: this is not in the hot path so it's ok to iterate over the map. // this allows to catch any bugs with unexpected entries still in the map. - for k := range e.values { - if k < e.minStartTime { + minStartTime := e.minStartTime + for k, v := range e.values { + if k < minStartTime { k := k + ts := e.minStartTime.ToTime() instrument.EmitAndLogInvariantViolation(e.opts.InstrumentOptions(), func(l *zap.Logger) { - l.Error("aggregate timestamp is less than min", + l.Error("value timestamp is less than min start time", zap.Time("ts", k.ToTime()), - zap.Time("min", e.minStartTime.ToTime())) + zap.Time("min", ts)) }) } + v.close() delete(e.values, k) - // Close the underlying aggregation objects. - if v, ok := e.values[e.minStartTime]; ok { - v.lockedAgg.sourcesSeen = nil - v.lockedAgg.aggregation.Close() - v.Release() - delete(e.values, e.minStartTime) + flushState, ok := e.flushState[k] + if ok { + flushState.close() } - e.minStartTime = e.minStartTime.Add(resolution) + delete(e.flushState, k) + } + // clean up any dangling flush state that should never exist. + for k, v := range e.flushState { + ts := k.ToTime() + instrument.EmitAndLogInvariantViolation(e.opts.InstrumentOptions(), func(l *zap.Logger) { + l.Error("dangling state timestamp", zap.Time("ts", ts)) + }) + v.close() + delete(e.flushState, k) } e.typeSpecificElemBase.Close() aggTypesPool := e.aggTypesOpts.TypesPool() pool := e.ElemPool(e.opts) - e.Unlock() - - // internal consumption state that doesn't need to be synchronized. - e.toConsume = e.toConsume[:0] e.dirty = e.dirty[:0] - e.toExpire = e.toExpire[:0] - e.consumedValues = nil + e.toConsume = e.toConsume[:0] + e.flushStateToExpire = e.flushStateToExpire[:0] e.minStartTime = 0 + e.Unlock() if !e.useDefaultAggregation { aggTypesPool.Put(e.aggTypes) @@ -543,20 +594,19 @@ func (e *GenericElem) insertDirty(alignedStart xtime.UnixNano) { // find finds the aggregation for a given time, or returns nil. //nolint: dupl -func (e *GenericElem) find(alignedStartNanos xtime.UnixNano) (timedAggregation, bool, error) { +func (e *GenericElem) find(alignedStartNanos xtime.UnixNano) (timedAggregation, error) { e.RLock() if e.closed { e.RUnlock() - return timedAggregation{}, false, errElemClosed + return timedAggregation{}, errElemClosed } timedAgg, ok := e.values[alignedStartNanos] if ok { - dirty := timedAgg.lockedAgg.dirty e.RUnlock() - return timedAgg, dirty, nil + return timedAgg, nil } e.RUnlock() - return timedAggregation{}, false, nil + return timedAggregation{}, nil } // findOrCreate finds the aggregation for a given time, or creates one @@ -567,12 +617,12 @@ func (e *GenericElem) findOrCreate( createOpts createAggregationOptions, ) (*lockedAggregation, error) { alignedStart := xtime.UnixNano(alignedStartNanos) - found, isDirty, err := e.find(alignedStart) + found, err := e.find(alignedStart) if err != nil { return nil, err } // if the aggregation is found and does not need to be updated, return as is. - if found.lockedAgg != nil && isDirty && found.resendEnabled == createOpts.resendEnabled { + if found.lockedAgg != nil && found.inDirtySet && found.resendEnabled == createOpts.resendEnabled { return found.lockedAgg, err } @@ -584,9 +634,9 @@ func (e *GenericElem) findOrCreate( timedAgg, ok := e.values[alignedStart] if ok { - // if the agg is not dirty, mark it dirty so it will be flushed. - if !timedAgg.lockedAgg.dirty { - timedAgg.lockedAgg.dirty = true + // add to dirty set so it will be flushed. + if !timedAgg.inDirtySet { + timedAgg.inDirtySet = true e.insertDirty(alignedStart) } // ensure the resendEnabled state is the latest. @@ -598,7 +648,6 @@ func (e *GenericElem) findOrCreate( var sourcesSeen map[uint32]*bitset.BitSet if createOpts.initSourceSet { - e.cachedSourceSetsLock.Lock() if numCachedSourceSets := len(e.cachedSourceSets); numCachedSourceSets > 0 { sourcesSeen = e.cachedSourceSets[numCachedSourceSets-1] e.cachedSourceSets[numCachedSourceSets-1] = nil @@ -609,17 +658,15 @@ func (e *GenericElem) findOrCreate( } else { sourcesSeen = make(map[uint32]*bitset.BitSet) } - e.cachedSourceSetsLock.Unlock() } timedAgg = timedAggregation{ startAtNanos: alignedStart, lockedAgg: &lockedAggregation{ sourcesSeen: sourcesSeen, aggregation: e.NewAggregation(e.opts, e.aggOpts), - prevValues: make([]float64, len(e.aggTypes)), - dirty: true, }, resendEnabled: createOpts.resendEnabled, + inDirtySet: true, } e.values[alignedStart] = timedAgg e.insertDirty(alignedStart) @@ -631,23 +678,19 @@ func (e *GenericElem) findOrCreate( } // returns true if a datapoint is emitted. -func (e *GenericElem) processValueWithAggregationLock( - timeNanos xtime.UnixNano, - prevTimeNanos xtime.UnixNano, - resendEnabled bool, - lockedAgg *lockedAggregation, +func (e *GenericElem) processValueWithAggregation( + flushState aggFlushState, flushLocalFn flushLocalMetricFn, flushForwardedFn flushForwardedMetricFn, resolution time.Duration, - latenessAllowed time.Duration) bool { + latenessAllowed time.Duration) aggFlushState { var ( transformations = e.parsedPipeline.Transformations discardNaNValues = e.opts.DiscardNaNAggregatedValues() - emitted bool ) for aggTypeIdx, aggType := range e.aggTypes { var extraDp transformation.Datapoint - value := lockedAgg.aggregation.ValueOf(aggType) + value := flushState.values[aggTypeIdx] for _, transformOp := range transformations { unaryOp, isUnaryOp := transformOp.UnaryTransform() binaryOp, isBinaryOp := transformOp.BinaryTransform() @@ -655,7 +698,7 @@ func (e *GenericElem) processValueWithAggregationLock( switch { case isUnaryOp: curr := transformation.Datapoint{ - TimeNanos: int64(timeNanos), + TimeNanos: int64(flushState.timestamp), Value: value, } @@ -667,15 +710,21 @@ func (e *GenericElem) processValueWithAggregationLock( prev := transformation.Datapoint{ Value: nan, } - // lazily construct consumedValues since they are only needed by binary transforms. - if e.consumedValues == nil { - e.consumedValues = make(valuesByTime) - } - if _, ok := e.consumedValues[prevTimeNanos]; ok { - prev = e.consumedValues[prevTimeNanos][aggTypeIdx] + if flushState.prevStartTime > 0 { + prevFlushState, ok := e.flushState[flushState.prevStartTime] + if !ok { + ts := flushState.prevStartTime.ToTime() + instrument.EmitAndLogInvariantViolation(e.opts.InstrumentOptions(), func(l *zap.Logger) { + l.Error("previous start time not in state map", + zap.Time("ts", ts)) + }) + } else { + prev.Value = prevFlushState.consumedValues[aggTypeIdx] + prev.TimeNanos = int64(prevFlushState.timestamp) + } } curr := transformation.Datapoint{ - TimeNanos: int64(timeNanos), + TimeNanos: int64(flushState.timestamp), Value: value, } res := binaryOp.Evaluate(prev, curr, transformation.FeatureFlags{}) @@ -685,16 +734,15 @@ func (e *GenericElem) processValueWithAggregationLock( // need to keep one value. In the future if we need to support higher-order // derivative transformations, we need to store an array of values here. if !math.IsNaN(curr.Value) { - if e.consumedValues[timeNanos] == nil { - e.consumedValues[timeNanos] = make([]transformation.Datapoint, len(e.aggTypes)) + if flushState.consumedValues == nil { + flushState.consumedValues = make([]float64, len(e.aggTypes)) } - e.consumedValues[timeNanos][aggTypeIdx] = curr + flushState.consumedValues[aggTypeIdx] = curr.Value } - value = res.Value case isUnaryMultiOp: curr := transformation.Datapoint{ - TimeNanos: int64(timeNanos), + TimeNanos: int64(flushState.timestamp), Value: value, } @@ -710,20 +758,24 @@ func (e *GenericElem) processValueWithAggregationLock( // It's ok to send a 0 prevValue on the first forward because it's not used in AddUnique unless it's a // resend (version > 0) - prevValue := lockedAgg.prevValues[aggTypeIdx] - lockedAgg.prevValues[aggTypeIdx] = value - if lockedAgg.flushed { + var prevValue float64 + if flushState.emittedValues == nil { + flushState.emittedValues = make([]float64, len(e.aggTypes)) + } else { + prevValue = flushState.emittedValues[aggTypeIdx] + } + flushState.emittedValues[aggTypeIdx] = value + if flushState.flushed { // no need to resend a value that hasn't changed. if (math.IsNaN(prevValue) && math.IsNaN(value)) || (prevValue == value) { continue } } - emitted = true if !e.parsedPipeline.HasRollup { toFlush := make([]transformation.Datapoint, 0, 2) toFlush = append(toFlush, transformation.Datapoint{ - TimeNanos: int64(timeNanos), + TimeNanos: int64(flushState.timestamp), Value: value, }) if extraDp.TimeNanos != 0 { @@ -732,22 +784,24 @@ func (e *GenericElem) processValueWithAggregationLock( for _, point := range toFlush { switch e.idPrefixSuffixType { case NoPrefixNoSuffix: - flushLocalFn(nil, e.id, nil, point.TimeNanos, point.Value, lockedAgg.aggregation.Annotation(), e.sp) + flushLocalFn(nil, e.id, nil, point.TimeNanos, point.Value, flushState.annotation, + e.sp) case WithPrefixWithSuffix: flushLocalFn(e.FullPrefix(e.opts), e.id, e.TypeStringFor(e.aggTypesOpts, aggType), - point.TimeNanos, point.Value, lockedAgg.aggregation.Annotation(), e.sp) + point.TimeNanos, point.Value, flushState.annotation, e.sp) } } } else { forwardedAggregationKey, _ := e.ForwardedAggregationKey() // only record lag for the initial flush (not resends) - if !lockedAgg.flushed { + if !flushState.flushed { // latenessAllowed is not due to processing delay, so it remove it from lag calc. - e.forwardLagMetric(resolution).RecordDuration(time.Since(timeNanos.ToTime().Add(-latenessAllowed))) + e.forwardLagMetric(resolution).RecordDuration( + time.Since(flushState.timestamp.ToTime().Add(-latenessAllowed))) } flushForwardedFn(e.writeForwardedMetricFn, forwardedAggregationKey, - int64(timeNanos), value, prevValue, lockedAgg.aggregation.Annotation(), resendEnabled) + int64(flushState.timestamp), value, prevValue, flushState.annotation, flushState.resendEnabled) } } - return emitted + return flushState } diff --git a/src/aggregator/aggregator/timer_elem_gen.go b/src/aggregator/aggregator/timer_elem_gen.go index 8be9fb2b44..b3d0e40565 100644 --- a/src/aggregator/aggregator/timer_elem_gen.go +++ b/src/aggregator/aggregator/timer_elem_gen.go @@ -30,6 +30,7 @@ import ( "sync" "time" + raggregation "github.com/m3db/m3/src/aggregator/aggregation" "github.com/m3db/m3/src/metrics/metadata" "github.com/m3db/m3/src/metrics/metric/aggregated" "github.com/m3db/m3/src/metrics/metric/unaggregated" @@ -45,26 +46,21 @@ type lockedTimerAggregation struct { sync.Mutex dirty bool - flushed bool closed bool sourcesSeen map[uint32]*bitset.BitSet aggregation timerAggregation - prevValues []float64 // the previously emitted values (one per aggregation type). } type timedTimer struct { startAtNanos xtime.UnixNano // start time of an aggregation window lockedAgg *lockedTimerAggregation resendEnabled bool - - // this is mutable data for specifying on each Consume which previous value the - // current agg can reference (i.e. for binary ops). it must be mutable since the - // set of vals within the buffer past can change and so on each consume a given agg's - // previous depends on the state of values preceding the current at that point in time. - previousTimeNanos xtime.UnixNano + inDirtySet bool } -func (ta *timedTimer) Release() { +// close is called when the aggregation has been expired or the element is being closed. +func (ta *timedTimer) close() { + ta.lockedAgg.aggregation.Close() ta.lockedAgg = nil } @@ -75,25 +71,26 @@ type TimerElem struct { // startTime -> agg (new one per every resolution) values map[xtime.UnixNano]timedTimer + // startTime -> state. this is local state to the flusher and does not need to guarded with a lock. + // values and flushState should always have the exact same key set. + flushState map[xtime.UnixNano]aggFlushState // sorted start aligned times that have been written to since the last flush dirty []xtime.UnixNano // min time in the values map. allow for iterating through map. minStartTime xtime.UnixNano // internal consume state that does not need to be synchronized. - toConsume []timedTimer // small buffer to avoid memory allocations during consumption - toExpire []timedTimer // small buffer to avoid memory allocations during consumption - // map of the previous consumed values for each timestamp in the buffer. needed to support binary transforms that - // need the value from the previous timestamp. - consumedValues valuesByTime + toConsume []aggFlushState // small buffer to avoid memory allocations during consumption + flushStateToExpire []xtime.UnixNano // small buffer to avoid memory allocations during consumption } // NewTimerElem returns a new TimerElem. func NewTimerElem(data ElemData, opts ElemOptions) (*TimerElem, error) { e := &TimerElem{ - elemBase: newElemBase(opts), - dirty: make([]xtime.UnixNano, 0, defaultNumAggregations), // in most cases values will have two entries - values: make(map[xtime.UnixNano]timedTimer), + elemBase: newElemBase(opts), + dirty: make([]xtime.UnixNano, 0, defaultNumAggregations), // in most cases values will have two entries + values: make(map[xtime.UnixNano]timedTimer), + flushState: make(map[xtime.UnixNano]aggFlushState), } if err := e.ResetSetData(data); err != nil { return nil, err @@ -137,6 +134,7 @@ func (e *TimerElem) AddUnion(timestamp time.Time, mu unaggregated.MetricUnion, r return errAggregationClosed } lockedAgg.aggregation.AddUnion(timestamp, mu) + lockedAgg.dirty = true lockedAgg.Unlock() return nil } @@ -154,6 +152,7 @@ func (e *TimerElem) AddValue(timestamp time.Time, value float64, annotation []by return errAggregationClosed } lockedAgg.aggregation.Add(timestamp, value, annotation) + lockedAgg.dirty = true lockedAgg.Unlock() return nil } @@ -205,6 +204,7 @@ func (e *TimerElem) AddUnique( lockedAgg.aggregation.Add(timestamp, v, metric.Annotation) } } + lockedAgg.dirty = true lockedAgg.Unlock() return nil } @@ -212,9 +212,8 @@ func (e *TimerElem) AddUnique( // remove expired aggregations from the values map. func (e *TimerElem) expireValuesWithLock( targetNanos int64, - timestampNanosFn timestampNanosFn, isEarlierThanFn isEarlierThanFn) { - e.toExpire = e.toExpire[:0] + e.flushStateToExpire = e.flushStateToExpire[:0] if len(e.values) == 0 { return } @@ -232,22 +231,54 @@ func (e *TimerElem) expireValuesWithLock( break } } - // if this current value is eligible to be expired it will no longer be written to. this means it's safe - // to remove the _previous_ value since it will no longer be needed for binary transformations. when the + + // close the agg to prevent any more writes. + dirty := false + currV.lockedAgg.Lock() + currV.lockedAgg.closed = true + dirty = currV.lockedAgg.dirty + currV.lockedAgg.Unlock() + if dirty { + // a race occurred and a write happened before we could close the aggregation. will expire next time. + break + } + + // if this current value is closed and clean it will no longer be flushed. this means it's safe + // to remove the previous value since it will no longer be needed for binary transformations. when the // next value is eligible to be expired, this current value will actually be removed. if prevV, ok := e.values[e.minStartTime]; ok { - // Previous times are used to key into consumedValues, which are non-start-aligned. And so - // we convert from startAligned here when setting previous. - prevV.previousTimeNanos = xtime.UnixNano(timestampNanosFn(int64(e.minStartTime), resolution)) - e.toExpire = append(e.toExpire, prevV) + // can't expire flush state until after the flushing, so we save the time to expire later. + e.flushStateToExpire = append(e.flushStateToExpire, e.minStartTime) delete(e.values, e.minStartTime) e.minStartTime = currStart + + // it's safe to access this outside the agg lock since it was closed in a previous iteration. + // This is to make sure there aren't too many cached source sets taking up + // too much space. + if prevV.lockedAgg.sourcesSeen != nil && len(e.cachedSourceSets) < e.opts.MaxNumCachedSourceSets() { + e.cachedSourceSets = append(e.cachedSourceSets, prevV.lockedAgg.sourcesSeen) + } + prevV.close() } } currStart = currStart.Add(resolution) } } +func (e *TimerElem) expireFlushState() { + for _, t := range e.flushStateToExpire { + prevFlushState, ok := e.flushState[t] + if !ok { + ts := t.ToTime() + instrument.EmitAndLogInvariantViolation(e.opts.InstrumentOptions(), func(l *zap.Logger) { + l.Error("expire time not in state map", zap.Time("ts", ts)) + }) + } + prevFlushState.close() + delete(e.flushState, t) + } +} + // return the timestamp in the values map that is before the provided time. returns false if the provided time is the // smallest time or the map is empty. func (e *TimerElem) previousStartAlignedWithLock(timestamp xtime.UnixNano) (xtime.UnixNano, bool) { @@ -255,6 +286,7 @@ func (e *TimerElem) previousStartAlignedWithLock(timestamp xtime.UnixNano) (xtim return 0, false } resolution := e.sp.Resolution().Window + // ensure the input is start aligned and then calculate the previous start time. startAligned := timestamp.Truncate(resolution).Add(-resolution) for !startAligned.Before(e.minStartTime) { _, ok := e.values[startAligned] @@ -308,80 +340,55 @@ func (e *TimerElem) Consume( return false } e.toConsume = e.toConsume[:0] - // Evaluate and GC expired items. dirtyTimes := e.dirty e.dirty = e.dirty[:0] - for _, dirtyTime := range dirtyTimes { - agg, ok := e.values[dirtyTime] - if !ok { - dirtyTime := dirtyTime - instrument.EmitAndLogInvariantViolation(e.opts.InstrumentOptions(), func(l *zap.Logger) { - l.Error("dirty timestamp not in map", zap.Time("ts", dirtyTime.ToTime())) - }) - } - - if !isEarlierThanFn(int64(dirtyTime), resolution, targetNanos) { + for i, dirtyTime := range dirtyTimes { + flushState, ready := e.updateFlushStateWithLock(dirtyTime, targetNanos, isEarlierThanFn) + if !ready || !flushState.dirty { + // there is a race where the value was added to the dirty set, but the writer didn't actually update the + // value yet (by marking dirty). add back to the dirty set so it can be processed in the next round once + // the value has been updated. e.dirty = append(e.dirty, dirtyTime) continue } + val := e.values[dirtyTime] + val.inDirtySet = false + e.values[dirtyTime] = val + e.toConsume = append(e.toConsume, flushState) - agg.lockedAgg.dirty = false - previousStartAligned, ok := e.previousStartAlignedWithLock(dirtyTime) - if ok { - agg.previousTimeNanos = xtime.UnixNano(timestampNanosFn(int64(previousStartAligned), resolution)) - } - e.toConsume = append(e.toConsume, agg) - - // add the nextAgg to the dirty set as well in case we need to cascade the value. + // potentially consume the nextAgg as well in case we need to cascade an update due to a resend. nextAgg, ok := e.nextAggWithLock(dirtyTime, targetNanos, isEarlierThanFn) - // only add nextAgg if not already in the dirty set - if ok && !nextAgg.lockedAgg.dirty { - nextAgg.previousTimeNanos = xtime.UnixNano(timestampNanosFn(int64(dirtyTime), resolution)) - e.toConsume = append(e.toConsume, nextAgg) + if ok && (i == len(dirtyTimes)-1 || dirtyTimes[i+1] != nextAgg.startAtNanos) { + // if not already in the dirty set. + flushState, ready := e.updateFlushStateWithLock(nextAgg.startAtNanos, targetNanos, isEarlierThanFn) + if ready { + e.toConsume = append(e.toConsume, flushState) + } } } - e.expireValuesWithLock(targetNanos, timestampNanosFn, isEarlierThanFn) + // expire the values and aggregations while we still hold the lock. + e.expireValuesWithLock(targetNanos, isEarlierThanFn) canCollect := len(e.dirty) == 0 && e.tombstoned e.Unlock() // Process the aggregations that are ready for consumption. - for i := range e.toConsume { - timeNanos := xtime.UnixNano(timestampNanosFn(int64(e.toConsume[i].startAtNanos), resolution)) - e.toConsume[i].lockedAgg.Lock() - _ = e.processValueWithAggregationLock( - timeNanos, - e.toConsume[i].previousTimeNanos, - e.toConsume[i].resendEnabled, - e.toConsume[i].lockedAgg, + for _, flushState := range e.toConsume { + flushState.timestamp = xtime.UnixNano(timestampNanosFn(int64(flushState.startAt), resolution)) + flushState = e.processValueWithAggregation( + flushState, flushLocalFn, flushForwardedFn, resolution, latenessAllowed, ) - e.toConsume[i].lockedAgg.flushed = true - e.toConsume[i].lockedAgg.Unlock() - } - - // Cleanup expired item after consuming since consuming still has a ref to the locked aggregation. - for i := range e.toExpire { - e.toExpire[i].lockedAgg.closed = true - e.toExpire[i].lockedAgg.aggregation.Close() - if e.toExpire[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.toExpire[i].lockedAgg.sourcesSeen) - } - e.cachedSourceSetsLock.Unlock() - e.toExpire[i].lockedAgg.sourcesSeen = nil - } - e.toExpire[i].Release() - - delete(e.consumedValues, e.toExpire[i].previousTimeNanos) + flushState.flushed = true + e.flushState[flushState.startAt] = flushState } + // expire the flush state after processing since it's needed in the processing. + e.expireFlushState() + if e.parsedPipeline.HasRollup { forwardedAggregationKey, _ := e.ForwardedAggregationKey() onForwardedFlushedFn(e.onForwardedAggregationWrittenFn, forwardedAggregationKey) @@ -390,6 +397,46 @@ func (e *TimerElem) Consume( return canCollect } +func (e *TimerElem) updateFlushStateWithLock(dirtyTime xtime.UnixNano, targetNanos int64, + isEarlierThanFn isEarlierThanFn) (aggFlushState, bool) { + resolution := e.sp.Resolution().Window + agg, ok := e.values[dirtyTime] + if !ok { + ts := dirtyTime.ToTime() + instrument.EmitAndLogInvariantViolation(e.opts.InstrumentOptions(), func(l *zap.Logger) { + l.Error("dirty timestamp not in values map", zap.Time("ts", ts)) + }) + } + if !isEarlierThanFn(int64(dirtyTime), resolution, targetNanos) { + return aggFlushState{}, false + } + + // note: flushState might be empty for the first flush + flushState := e.flushState[dirtyTime] + // copy the lockedAgg data to the flushState while holding the lock. + agg.lockedAgg.Lock() + flushState.dirty = agg.lockedAgg.dirty + flushState.values = flushState.values[:0] + for _, aggType := range e.aggTypes { + flushState.values = append(flushState.values, agg.lockedAgg.aggregation.ValueOf(aggType)) + } + flushState.annotation = raggregation.MaybeReplaceAnnotation( + flushState.annotation, agg.lockedAgg.aggregation.Annotation()) + agg.lockedAgg.dirty = false + agg.lockedAgg.Unlock() + + // update the flushState with everything else. + previousStartAligned, ok := e.previousStartAlignedWithLock(dirtyTime) + if ok { + flushState.prevStartTime = previousStartAligned + } else { + flushState.prevStartTime = 0 + } + flushState.resendEnabled = agg.resendEnabled + flushState.startAt = dirtyTime + return flushState, true +} + // Close closes the element. func (e *TimerElem) Close() { e.Lock() @@ -407,39 +454,44 @@ func (e *TimerElem) Close() { } e.cachedSourceSets = nil - resolution := e.sp.Resolution().Window // note: this is not in the hot path so it's ok to iterate over the map. // this allows to catch any bugs with unexpected entries still in the map. - for k := range e.values { - if k < e.minStartTime { + minStartTime := e.minStartTime + for k, v := range e.values { + if k < minStartTime { k := k + ts := e.minStartTime.ToTime() instrument.EmitAndLogInvariantViolation(e.opts.InstrumentOptions(), func(l *zap.Logger) { - l.Error("aggregate timestamp is less than min", + l.Error("value timestamp is less than min start time", zap.Time("ts", k.ToTime()), - zap.Time("min", e.minStartTime.ToTime())) + zap.Time("min", ts)) }) } + v.close() delete(e.values, k) - // Close the underlying aggregation objects. - if v, ok := e.values[e.minStartTime]; ok { - v.lockedAgg.sourcesSeen = nil - v.lockedAgg.aggregation.Close() - v.Release() - delete(e.values, e.minStartTime) + flushState, ok := e.flushState[k] + if ok { + flushState.close() } - e.minStartTime = e.minStartTime.Add(resolution) + delete(e.flushState, k) + } + // clean up any dangling flush state that should never exist. + for k, v := range e.flushState { + ts := k.ToTime() + instrument.EmitAndLogInvariantViolation(e.opts.InstrumentOptions(), func(l *zap.Logger) { + l.Error("dangling state timestamp", zap.Time("ts", ts)) + }) + v.close() + delete(e.flushState, k) } e.timerElemBase.Close() aggTypesPool := e.aggTypesOpts.TypesPool() pool := e.ElemPool(e.opts) - e.Unlock() - - // internal consumption state that doesn't need to be synchronized. - e.toConsume = e.toConsume[:0] e.dirty = e.dirty[:0] - e.toExpire = e.toExpire[:0] - e.consumedValues = nil + e.toConsume = e.toConsume[:0] + e.flushStateToExpire = e.flushStateToExpire[:0] e.minStartTime = 0 + e.Unlock() if !e.useDefaultAggregation { aggTypesPool.Put(e.aggTypes) @@ -479,20 +531,19 @@ func (e *TimerElem) insertDirty(alignedStart xtime.UnixNano) { // find finds the aggregation for a given time, or returns nil. //nolint: dupl -func (e *TimerElem) find(alignedStartNanos xtime.UnixNano) (timedTimer, bool, error) { +func (e *TimerElem) find(alignedStartNanos xtime.UnixNano) (timedTimer, error) { e.RLock() if e.closed { e.RUnlock() - return timedTimer{}, false, errElemClosed + return timedTimer{}, errElemClosed } timedAgg, ok := e.values[alignedStartNanos] if ok { - dirty := timedAgg.lockedAgg.dirty e.RUnlock() - return timedAgg, dirty, nil + return timedAgg, nil } e.RUnlock() - return timedTimer{}, false, nil + return timedTimer{}, nil } // findOrCreate finds the aggregation for a given time, or creates one @@ -503,12 +554,12 @@ func (e *TimerElem) findOrCreate( createOpts createAggregationOptions, ) (*lockedTimerAggregation, error) { alignedStart := xtime.UnixNano(alignedStartNanos) - found, isDirty, err := e.find(alignedStart) + found, err := e.find(alignedStart) if err != nil { return nil, err } // if the aggregation is found and does not need to be updated, return as is. - if found.lockedAgg != nil && isDirty && found.resendEnabled == createOpts.resendEnabled { + if found.lockedAgg != nil && found.inDirtySet && found.resendEnabled == createOpts.resendEnabled { return found.lockedAgg, err } @@ -520,9 +571,9 @@ func (e *TimerElem) findOrCreate( timedAgg, ok := e.values[alignedStart] if ok { - // if the agg is not dirty, mark it dirty so it will be flushed. - if !timedAgg.lockedAgg.dirty { - timedAgg.lockedAgg.dirty = true + // add to dirty set so it will be flushed. + if !timedAgg.inDirtySet { + timedAgg.inDirtySet = true e.insertDirty(alignedStart) } // ensure the resendEnabled state is the latest. @@ -534,7 +585,6 @@ func (e *TimerElem) findOrCreate( var sourcesSeen map[uint32]*bitset.BitSet if createOpts.initSourceSet { - e.cachedSourceSetsLock.Lock() if numCachedSourceSets := len(e.cachedSourceSets); numCachedSourceSets > 0 { sourcesSeen = e.cachedSourceSets[numCachedSourceSets-1] e.cachedSourceSets[numCachedSourceSets-1] = nil @@ -545,17 +595,15 @@ func (e *TimerElem) findOrCreate( } else { sourcesSeen = make(map[uint32]*bitset.BitSet) } - e.cachedSourceSetsLock.Unlock() } timedAgg = timedTimer{ startAtNanos: alignedStart, lockedAgg: &lockedTimerAggregation{ sourcesSeen: sourcesSeen, aggregation: e.NewAggregation(e.opts, e.aggOpts), - prevValues: make([]float64, len(e.aggTypes)), - dirty: true, }, resendEnabled: createOpts.resendEnabled, + inDirtySet: true, } e.values[alignedStart] = timedAgg e.insertDirty(alignedStart) @@ -567,23 +615,19 @@ func (e *TimerElem) findOrCreate( } // returns true if a datapoint is emitted. -func (e *TimerElem) processValueWithAggregationLock( - timeNanos xtime.UnixNano, - prevTimeNanos xtime.UnixNano, - resendEnabled bool, - lockedAgg *lockedTimerAggregation, +func (e *TimerElem) processValueWithAggregation( + flushState aggFlushState, flushLocalFn flushLocalMetricFn, flushForwardedFn flushForwardedMetricFn, resolution time.Duration, - latenessAllowed time.Duration) bool { + latenessAllowed time.Duration) aggFlushState { var ( transformations = e.parsedPipeline.Transformations discardNaNValues = e.opts.DiscardNaNAggregatedValues() - emitted bool ) for aggTypeIdx, aggType := range e.aggTypes { var extraDp transformation.Datapoint - value := lockedAgg.aggregation.ValueOf(aggType) + value := flushState.values[aggTypeIdx] for _, transformOp := range transformations { unaryOp, isUnaryOp := transformOp.UnaryTransform() binaryOp, isBinaryOp := transformOp.BinaryTransform() @@ -591,7 +635,7 @@ func (e *TimerElem) processValueWithAggregationLock( switch { case isUnaryOp: curr := transformation.Datapoint{ - TimeNanos: int64(timeNanos), + TimeNanos: int64(flushState.timestamp), Value: value, } @@ -603,15 +647,21 @@ func (e *TimerElem) processValueWithAggregationLock( prev := transformation.Datapoint{ Value: nan, } - // lazily construct consumedValues since they are only needed by binary transforms. - if e.consumedValues == nil { - e.consumedValues = make(valuesByTime) - } - if _, ok := e.consumedValues[prevTimeNanos]; ok { - prev = e.consumedValues[prevTimeNanos][aggTypeIdx] + if flushState.prevStartTime > 0 { + prevFlushState, ok := e.flushState[flushState.prevStartTime] + if !ok { + ts := flushState.prevStartTime.ToTime() + instrument.EmitAndLogInvariantViolation(e.opts.InstrumentOptions(), func(l *zap.Logger) { + l.Error("previous start time not in state map", + zap.Time("ts", ts)) + }) + } else { + prev.Value = prevFlushState.consumedValues[aggTypeIdx] + prev.TimeNanos = int64(prevFlushState.timestamp) + } } curr := transformation.Datapoint{ - TimeNanos: int64(timeNanos), + TimeNanos: int64(flushState.timestamp), Value: value, } res := binaryOp.Evaluate(prev, curr, transformation.FeatureFlags{}) @@ -621,16 +671,15 @@ func (e *TimerElem) processValueWithAggregationLock( // need to keep one value. In the future if we need to support higher-order // derivative transformations, we need to store an array of values here. if !math.IsNaN(curr.Value) { - if e.consumedValues[timeNanos] == nil { - e.consumedValues[timeNanos] = make([]transformation.Datapoint, len(e.aggTypes)) + if flushState.consumedValues == nil { + flushState.consumedValues = make([]float64, len(e.aggTypes)) } - e.consumedValues[timeNanos][aggTypeIdx] = curr + flushState.consumedValues[aggTypeIdx] = curr.Value } - value = res.Value case isUnaryMultiOp: curr := transformation.Datapoint{ - TimeNanos: int64(timeNanos), + TimeNanos: int64(flushState.timestamp), Value: value, } @@ -646,20 +695,24 @@ func (e *TimerElem) processValueWithAggregationLock( // It's ok to send a 0 prevValue on the first forward because it's not used in AddUnique unless it's a // resend (version > 0) - prevValue := lockedAgg.prevValues[aggTypeIdx] - lockedAgg.prevValues[aggTypeIdx] = value - if lockedAgg.flushed { + var prevValue float64 + if flushState.emittedValues == nil { + flushState.emittedValues = make([]float64, len(e.aggTypes)) + } else { + prevValue = flushState.emittedValues[aggTypeIdx] + } + flushState.emittedValues[aggTypeIdx] = value + if flushState.flushed { // no need to resend a value that hasn't changed. if (math.IsNaN(prevValue) && math.IsNaN(value)) || (prevValue == value) { continue } } - emitted = true if !e.parsedPipeline.HasRollup { toFlush := make([]transformation.Datapoint, 0, 2) toFlush = append(toFlush, transformation.Datapoint{ - TimeNanos: int64(timeNanos), + TimeNanos: int64(flushState.timestamp), Value: value, }) if extraDp.TimeNanos != 0 { @@ -668,22 +721,24 @@ func (e *TimerElem) processValueWithAggregationLock( for _, point := range toFlush { switch e.idPrefixSuffixType { case NoPrefixNoSuffix: - flushLocalFn(nil, e.id, nil, point.TimeNanos, point.Value, lockedAgg.aggregation.Annotation(), e.sp) + flushLocalFn(nil, e.id, nil, point.TimeNanos, point.Value, flushState.annotation, + e.sp) case WithPrefixWithSuffix: flushLocalFn(e.FullPrefix(e.opts), e.id, e.TypeStringFor(e.aggTypesOpts, aggType), - point.TimeNanos, point.Value, lockedAgg.aggregation.Annotation(), e.sp) + point.TimeNanos, point.Value, flushState.annotation, e.sp) } } } else { forwardedAggregationKey, _ := e.ForwardedAggregationKey() // only record lag for the initial flush (not resends) - if !lockedAgg.flushed { + if !flushState.flushed { // latenessAllowed is not due to processing delay, so it remove it from lag calc. - e.forwardLagMetric(resolution).RecordDuration(time.Since(timeNanos.ToTime().Add(-latenessAllowed))) + e.forwardLagMetric(resolution).RecordDuration( + time.Since(flushState.timestamp.ToTime().Add(-latenessAllowed))) } flushForwardedFn(e.writeForwardedMetricFn, forwardedAggregationKey, - int64(timeNanos), value, prevValue, lockedAgg.aggregation.Annotation(), resendEnabled) + int64(flushState.timestamp), value, prevValue, flushState.annotation, flushState.resendEnabled) } } - return emitted + return flushState } From 3f86f04a7dbc28ecde48d889f31d1dbbbd1880fe Mon Sep 17 00:00:00 2001 From: Ryan Hall Date: Fri, 29 Oct 2021 11:37:51 -0700 Subject: [PATCH 2/6] review comments --- src/aggregator/aggregator/counter_elem_gen.go | 1 + src/aggregator/aggregator/gauge_elem_gen.go | 1 + src/aggregator/aggregator/generic_elem.go | 1 + src/aggregator/aggregator/timer_elem_gen.go | 1 + 4 files changed, 4 insertions(+) diff --git a/src/aggregator/aggregator/counter_elem_gen.go b/src/aggregator/aggregator/counter_elem_gen.go index a739a16d82..14bf6b3ea9 100644 --- a/src/aggregator/aggregator/counter_elem_gen.go +++ b/src/aggregator/aggregator/counter_elem_gen.go @@ -406,6 +406,7 @@ func (e *CounterElem) updateFlushStateWithLock(dirtyTime xtime.UnixNano, targetN instrument.EmitAndLogInvariantViolation(e.opts.InstrumentOptions(), func(l *zap.Logger) { l.Error("dirty timestamp not in values map", zap.Time("ts", ts)) }) + return aggFlushState{}, false } if !isEarlierThanFn(int64(dirtyTime), resolution, targetNanos) { return aggFlushState{}, false diff --git a/src/aggregator/aggregator/gauge_elem_gen.go b/src/aggregator/aggregator/gauge_elem_gen.go index de49954d6d..f6a50b66a8 100644 --- a/src/aggregator/aggregator/gauge_elem_gen.go +++ b/src/aggregator/aggregator/gauge_elem_gen.go @@ -406,6 +406,7 @@ func (e *GaugeElem) updateFlushStateWithLock(dirtyTime xtime.UnixNano, targetNan instrument.EmitAndLogInvariantViolation(e.opts.InstrumentOptions(), func(l *zap.Logger) { l.Error("dirty timestamp not in values map", zap.Time("ts", ts)) }) + return aggFlushState{}, false } if !isEarlierThanFn(int64(dirtyTime), resolution, targetNanos) { return aggFlushState{}, false diff --git a/src/aggregator/aggregator/generic_elem.go b/src/aggregator/aggregator/generic_elem.go index feb324da1a..d8dc68adae 100644 --- a/src/aggregator/aggregator/generic_elem.go +++ b/src/aggregator/aggregator/generic_elem.go @@ -469,6 +469,7 @@ func (e *GenericElem) updateFlushStateWithLock(dirtyTime xtime.UnixNano, targetN instrument.EmitAndLogInvariantViolation(e.opts.InstrumentOptions(), func(l *zap.Logger) { l.Error("dirty timestamp not in values map", zap.Time("ts", ts)) }) + return aggFlushState{}, false } if !isEarlierThanFn(int64(dirtyTime), resolution, targetNanos) { return aggFlushState{}, false diff --git a/src/aggregator/aggregator/timer_elem_gen.go b/src/aggregator/aggregator/timer_elem_gen.go index b3d0e40565..a558f20f96 100644 --- a/src/aggregator/aggregator/timer_elem_gen.go +++ b/src/aggregator/aggregator/timer_elem_gen.go @@ -406,6 +406,7 @@ func (e *TimerElem) updateFlushStateWithLock(dirtyTime xtime.UnixNano, targetNan instrument.EmitAndLogInvariantViolation(e.opts.InstrumentOptions(), func(l *zap.Logger) { l.Error("dirty timestamp not in values map", zap.Time("ts", ts)) }) + return aggFlushState{}, false } if !isEarlierThanFn(int64(dirtyTime), resolution, targetNanos) { return aggFlushState{}, false From b570104c98b0b98814b0b515c09402a3c6bb004a Mon Sep 17 00:00:00 2001 From: Ryan Hall Date: Fri, 29 Oct 2021 16:25:29 -0700 Subject: [PATCH 3/6] fix bugs --- src/aggregator/aggregator/counter_elem_gen.go | 22 ++++++++++++++----- src/aggregator/aggregator/gauge_elem_gen.go | 22 ++++++++++++++----- src/aggregator/aggregator/generic_elem.go | 22 ++++++++++++++----- src/aggregator/aggregator/timer_elem_gen.go | 22 ++++++++++++++----- 4 files changed, 68 insertions(+), 20 deletions(-) diff --git a/src/aggregator/aggregator/counter_elem_gen.go b/src/aggregator/aggregator/counter_elem_gen.go index 14bf6b3ea9..2f1a7fd1a9 100644 --- a/src/aggregator/aggregator/counter_elem_gen.go +++ b/src/aggregator/aggregator/counter_elem_gen.go @@ -38,6 +38,7 @@ import ( "github.com/m3db/m3/src/x/instrument" xtime "github.com/m3db/m3/src/x/time" + "github.com/scaleway/scaleway-sdk-go/logger" "github.com/willf/bitset" "go.uber.org/zap" ) @@ -121,8 +122,11 @@ func (e *CounterElem) ResetSetData(data ElemData) error { // AddUnion adds a metric value union at a given timestamp. func (e *CounterElem) AddUnion(timestamp time.Time, mu unaggregated.MetricUnion, resendEnabled bool) error { - alignedStart := timestamp.Truncate(e.sp.Resolution().Window).UnixNano() - lockedAgg, err := e.findOrCreate(alignedStart, createAggregationOptions{ + return e.doAddUnion(timestamp, mu, resendEnabled, false, timestamp) +} +func (e *CounterElem) doAddUnion(timestamp time.Time, mu unaggregated.MetricUnion, resendEnabled bool, retry bool, initialTimestamp time.Time) error { + alignedStart := timestamp.Truncate(e.sp.Resolution().Window) + lockedAgg, err := e.findOrCreate(alignedStart.UnixNano(), createAggregationOptions{ resendEnabled: resendEnabled, }) if err != nil { @@ -131,6 +135,13 @@ func (e *CounterElem) AddUnion(timestamp time.Time, mu unaggregated.MetricUnion, lockedAgg.Lock() if lockedAgg.closed { lockedAgg.Unlock() + if !resendEnabled && !retry { + return e.doAddUnion(alignedStart.Add(e.sp.Resolution().Window), mu, false, true, alignedStart) + } + logger.Errorf("aggregation already closed", + zap.Time("timestamp", timestamp), + zap.Time("initialTimestamp", initialTimestamp), + zap.Bool("retry", retry)) return errAggregationClosed } lockedAgg.aggregation.AddUnion(timestamp, mu) @@ -375,7 +386,7 @@ func (e *CounterElem) Consume( // Process the aggregations that are ready for consumption. for _, flushState := range e.toConsume { flushState.timestamp = xtime.UnixNano(timestampNanosFn(int64(flushState.startAt), resolution)) - flushState = e.processValueWithAggregation( + flushState = e.processValue( flushState, flushLocalFn, flushForwardedFn, @@ -616,7 +627,7 @@ func (e *CounterElem) findOrCreate( } // returns true if a datapoint is emitted. -func (e *CounterElem) processValueWithAggregation( +func (e *CounterElem) processValue( flushState aggFlushState, flushLocalFn flushLocalMetricFn, flushForwardedFn flushForwardedMetricFn, @@ -656,7 +667,8 @@ func (e *CounterElem) processValueWithAggregation( l.Error("previous start time not in state map", zap.Time("ts", ts)) }) - } else { + } else if prevFlushState.consumedValues != nil { + // prev consumedValues may be null if the result was NaN. prev.Value = prevFlushState.consumedValues[aggTypeIdx] prev.TimeNanos = int64(prevFlushState.timestamp) } diff --git a/src/aggregator/aggregator/gauge_elem_gen.go b/src/aggregator/aggregator/gauge_elem_gen.go index f6a50b66a8..1d04821903 100644 --- a/src/aggregator/aggregator/gauge_elem_gen.go +++ b/src/aggregator/aggregator/gauge_elem_gen.go @@ -38,6 +38,7 @@ import ( "github.com/m3db/m3/src/x/instrument" xtime "github.com/m3db/m3/src/x/time" + "github.com/scaleway/scaleway-sdk-go/logger" "github.com/willf/bitset" "go.uber.org/zap" ) @@ -121,8 +122,11 @@ func (e *GaugeElem) ResetSetData(data ElemData) error { // AddUnion adds a metric value union at a given timestamp. func (e *GaugeElem) AddUnion(timestamp time.Time, mu unaggregated.MetricUnion, resendEnabled bool) error { - alignedStart := timestamp.Truncate(e.sp.Resolution().Window).UnixNano() - lockedAgg, err := e.findOrCreate(alignedStart, createAggregationOptions{ + return e.doAddUnion(timestamp, mu, resendEnabled, false, timestamp) +} +func (e *GaugeElem) doAddUnion(timestamp time.Time, mu unaggregated.MetricUnion, resendEnabled bool, retry bool, initialTimestamp time.Time) error { + alignedStart := timestamp.Truncate(e.sp.Resolution().Window) + lockedAgg, err := e.findOrCreate(alignedStart.UnixNano(), createAggregationOptions{ resendEnabled: resendEnabled, }) if err != nil { @@ -131,6 +135,13 @@ func (e *GaugeElem) AddUnion(timestamp time.Time, mu unaggregated.MetricUnion, r lockedAgg.Lock() if lockedAgg.closed { lockedAgg.Unlock() + if !resendEnabled && !retry { + return e.doAddUnion(alignedStart.Add(e.sp.Resolution().Window), mu, false, true, alignedStart) + } + logger.Errorf("aggregation already closed", + zap.Time("timestamp", timestamp), + zap.Time("initialTimestamp", initialTimestamp), + zap.Bool("retry", retry)) return errAggregationClosed } lockedAgg.aggregation.AddUnion(timestamp, mu) @@ -375,7 +386,7 @@ func (e *GaugeElem) Consume( // Process the aggregations that are ready for consumption. for _, flushState := range e.toConsume { flushState.timestamp = xtime.UnixNano(timestampNanosFn(int64(flushState.startAt), resolution)) - flushState = e.processValueWithAggregation( + flushState = e.processValue( flushState, flushLocalFn, flushForwardedFn, @@ -616,7 +627,7 @@ func (e *GaugeElem) findOrCreate( } // returns true if a datapoint is emitted. -func (e *GaugeElem) processValueWithAggregation( +func (e *GaugeElem) processValue( flushState aggFlushState, flushLocalFn flushLocalMetricFn, flushForwardedFn flushForwardedMetricFn, @@ -656,7 +667,8 @@ func (e *GaugeElem) processValueWithAggregation( l.Error("previous start time not in state map", zap.Time("ts", ts)) }) - } else { + } else if prevFlushState.consumedValues != nil { + // prev consumedValues may be null if the result was NaN. prev.Value = prevFlushState.consumedValues[aggTypeIdx] prev.TimeNanos = int64(prevFlushState.timestamp) } diff --git a/src/aggregator/aggregator/generic_elem.go b/src/aggregator/aggregator/generic_elem.go index d8dc68adae..150c976d9c 100644 --- a/src/aggregator/aggregator/generic_elem.go +++ b/src/aggregator/aggregator/generic_elem.go @@ -27,6 +27,7 @@ import ( "time" "github.com/mauricelam/genny/generic" + "github.com/scaleway/scaleway-sdk-go/logger" "github.com/willf/bitset" "go.uber.org/zap" @@ -184,8 +185,11 @@ func (e *GenericElem) ResetSetData(data ElemData) error { // AddUnion adds a metric value union at a given timestamp. func (e *GenericElem) AddUnion(timestamp time.Time, mu unaggregated.MetricUnion, resendEnabled bool) error { - alignedStart := timestamp.Truncate(e.sp.Resolution().Window).UnixNano() - lockedAgg, err := e.findOrCreate(alignedStart, createAggregationOptions{ + return e.doAddUnion(timestamp, mu, resendEnabled, false, timestamp) +} +func (e *GenericElem) doAddUnion(timestamp time.Time, mu unaggregated.MetricUnion, resendEnabled bool, retry bool, initialTimestamp time.Time) error { + alignedStart := timestamp.Truncate(e.sp.Resolution().Window) + lockedAgg, err := e.findOrCreate(alignedStart.UnixNano(), createAggregationOptions{ resendEnabled: resendEnabled, }) if err != nil { @@ -194,6 +198,13 @@ func (e *GenericElem) AddUnion(timestamp time.Time, mu unaggregated.MetricUnion, lockedAgg.Lock() if lockedAgg.closed { lockedAgg.Unlock() + if !resendEnabled && !retry { + return e.doAddUnion(alignedStart.Add(e.sp.Resolution().Window), mu, false, true, alignedStart) + } + logger.Errorf("aggregation already closed", + zap.Time("timestamp", timestamp), + zap.Time("initialTimestamp", initialTimestamp), + zap.Bool("retry", retry)) return errAggregationClosed } lockedAgg.aggregation.AddUnion(timestamp, mu) @@ -438,7 +449,7 @@ func (e *GenericElem) Consume( // Process the aggregations that are ready for consumption. for _, flushState := range e.toConsume { flushState.timestamp = xtime.UnixNano(timestampNanosFn(int64(flushState.startAt), resolution)) - flushState = e.processValueWithAggregation( + flushState = e.processValue( flushState, flushLocalFn, flushForwardedFn, @@ -679,7 +690,7 @@ func (e *GenericElem) findOrCreate( } // returns true if a datapoint is emitted. -func (e *GenericElem) processValueWithAggregation( +func (e *GenericElem) processValue( flushState aggFlushState, flushLocalFn flushLocalMetricFn, flushForwardedFn flushForwardedMetricFn, @@ -719,7 +730,8 @@ func (e *GenericElem) processValueWithAggregation( l.Error("previous start time not in state map", zap.Time("ts", ts)) }) - } else { + } else if prevFlushState.consumedValues != nil { + // prev consumedValues may be null if the result was NaN. prev.Value = prevFlushState.consumedValues[aggTypeIdx] prev.TimeNanos = int64(prevFlushState.timestamp) } diff --git a/src/aggregator/aggregator/timer_elem_gen.go b/src/aggregator/aggregator/timer_elem_gen.go index a558f20f96..9e0e578c8e 100644 --- a/src/aggregator/aggregator/timer_elem_gen.go +++ b/src/aggregator/aggregator/timer_elem_gen.go @@ -38,6 +38,7 @@ import ( "github.com/m3db/m3/src/x/instrument" xtime "github.com/m3db/m3/src/x/time" + "github.com/scaleway/scaleway-sdk-go/logger" "github.com/willf/bitset" "go.uber.org/zap" ) @@ -121,8 +122,11 @@ func (e *TimerElem) ResetSetData(data ElemData) error { // AddUnion adds a metric value union at a given timestamp. func (e *TimerElem) AddUnion(timestamp time.Time, mu unaggregated.MetricUnion, resendEnabled bool) error { - alignedStart := timestamp.Truncate(e.sp.Resolution().Window).UnixNano() - lockedAgg, err := e.findOrCreate(alignedStart, createAggregationOptions{ + return e.doAddUnion(timestamp, mu, resendEnabled, false, timestamp) +} +func (e *TimerElem) doAddUnion(timestamp time.Time, mu unaggregated.MetricUnion, resendEnabled bool, retry bool, initialTimestamp time.Time) error { + alignedStart := timestamp.Truncate(e.sp.Resolution().Window) + lockedAgg, err := e.findOrCreate(alignedStart.UnixNano(), createAggregationOptions{ resendEnabled: resendEnabled, }) if err != nil { @@ -131,6 +135,13 @@ func (e *TimerElem) AddUnion(timestamp time.Time, mu unaggregated.MetricUnion, r lockedAgg.Lock() if lockedAgg.closed { lockedAgg.Unlock() + if !resendEnabled && !retry { + return e.doAddUnion(alignedStart.Add(e.sp.Resolution().Window), mu, false, true, alignedStart) + } + logger.Errorf("aggregation already closed", + zap.Time("timestamp", timestamp), + zap.Time("initialTimestamp", initialTimestamp), + zap.Bool("retry", retry)) return errAggregationClosed } lockedAgg.aggregation.AddUnion(timestamp, mu) @@ -375,7 +386,7 @@ func (e *TimerElem) Consume( // Process the aggregations that are ready for consumption. for _, flushState := range e.toConsume { flushState.timestamp = xtime.UnixNano(timestampNanosFn(int64(flushState.startAt), resolution)) - flushState = e.processValueWithAggregation( + flushState = e.processValue( flushState, flushLocalFn, flushForwardedFn, @@ -616,7 +627,7 @@ func (e *TimerElem) findOrCreate( } // returns true if a datapoint is emitted. -func (e *TimerElem) processValueWithAggregation( +func (e *TimerElem) processValue( flushState aggFlushState, flushLocalFn flushLocalMetricFn, flushForwardedFn flushForwardedMetricFn, @@ -656,7 +667,8 @@ func (e *TimerElem) processValueWithAggregation( l.Error("previous start time not in state map", zap.Time("ts", ts)) }) - } else { + } else if prevFlushState.consumedValues != nil { + // prev consumedValues may be null if the result was NaN. prev.Value = prevFlushState.consumedValues[aggTypeIdx] prev.TimeNanos = int64(prevFlushState.timestamp) } From 30003f1194d63e46f6c20856999cd358de465768 Mon Sep 17 00:00:00 2001 From: Ryan Hall Date: Tue, 2 Nov 2021 08:29:36 -0700 Subject: [PATCH 4/6] fix race condition with dirty set --- src/aggregator/aggregator/counter_elem_gen.go | 102 +++++++++--------- src/aggregator/aggregator/elem_test.go | 1 - src/aggregator/aggregator/entry_test.go | 76 +++++++++++++ src/aggregator/aggregator/gauge_elem_gen.go | 102 +++++++++--------- src/aggregator/aggregator/generic_elem.go | 102 +++++++++--------- src/aggregator/aggregator/timer_elem_gen.go | 102 +++++++++--------- 6 files changed, 288 insertions(+), 197 deletions(-) diff --git a/src/aggregator/aggregator/counter_elem_gen.go b/src/aggregator/aggregator/counter_elem_gen.go index 178d8100ca..82bfe169b3 100644 --- a/src/aggregator/aggregator/counter_elem_gen.go +++ b/src/aggregator/aggregator/counter_elem_gen.go @@ -38,7 +38,6 @@ import ( "github.com/m3db/m3/src/x/instrument" xtime "github.com/m3db/m3/src/x/time" - "github.com/scaleway/scaleway-sdk-go/logger" "github.com/willf/bitset" "go.uber.org/zap" ) @@ -122,9 +121,11 @@ func (e *CounterElem) ResetSetData(data ElemData) error { // AddUnion adds a metric value union at a given timestamp. func (e *CounterElem) AddUnion(timestamp time.Time, mu unaggregated.MetricUnion, resendEnabled bool) error { - return e.doAddUnion(timestamp, mu, resendEnabled, false, timestamp) + return e.doAddUnion(timestamp, mu, resendEnabled, false) } -func (e *CounterElem) doAddUnion(timestamp time.Time, mu unaggregated.MetricUnion, resendEnabled bool, retry bool, initialTimestamp time.Time) error { + +func (e *CounterElem) doAddUnion(timestamp time.Time, mu unaggregated.MetricUnion, resendEnabled bool, retry bool, +) error { alignedStart := timestamp.Truncate(e.sp.Resolution().Window) lockedAgg, err := e.findOrCreate(alignedStart.UnixNano(), createAggregationOptions{ resendEnabled: resendEnabled, @@ -134,14 +135,14 @@ func (e *CounterElem) doAddUnion(timestamp time.Time, mu unaggregated.MetricUnio } lockedAgg.Lock() if lockedAgg.closed { + // Note: this might have created an entry in the dirty set for lockedAgg when calling findOrCreate, even though + // it's already closed. The Consume loop will detect this and clean it up. lockedAgg.Unlock() if !resendEnabled && !retry { - return e.doAddUnion(alignedStart.Add(e.sp.Resolution().Window), mu, false, true, alignedStart) + // handle the edge case where the aggregation was already flushed/closed because the current time is right + // at the boundary. just roll the untimed metric into the next aggregation. + return e.doAddUnion(alignedStart.Add(e.sp.Resolution().Window), mu, false, true) } - logger.Errorf("aggregation already closed", - zap.Time("timestamp", timestamp), - zap.Time("initialTimestamp", initialTimestamp), - zap.Bool("retry", retry)) return errAggregationClosed } lockedAgg.aggregation.AddUnion(timestamp, mu) @@ -230,10 +231,8 @@ func (e *CounterElem) expireValuesWithLock( } resolution := e.sp.Resolution().Window - // start after the minimum to ensure we always keep at least one value in the map for binary transformations. - currStart := e.minStartTime.Add(resolution) + currStart := e.minStartTime resendExpire := targetNanos - int64(e.bufferForPastTimedMetricFn(resolution)) - for isEarlierThanFn(int64(currStart), resolution, targetNanos) { if currV, ok := e.values[currStart]; ok { if currV.resendEnabled { @@ -257,7 +256,9 @@ func (e *CounterElem) expireValuesWithLock( // if this current value is closed and clean it will no longer be flushed. this means it's safe // to remove the previous value since it will no longer be needed for binary transformations. when the // next value is eligible to be expired, this current value will actually be removed. - if prevV, ok := e.values[e.minStartTime]; ok { + // if we're currently pointing at the start skip this there is no previous for the start. this ensures + // we always keep at least one value in the map for binary transformations. + if prevV, ok := e.values[e.minStartTime]; ok && currStart != e.minStartTime { // can't expire flush state until after the flushing, so we save the time to expire later. e.flushStateToExpire = append(e.flushStateToExpire, e.minStartTime) delete(e.values, e.minStartTime) @@ -357,8 +358,26 @@ func (e *CounterElem) Consume( dirtyTimes := e.dirty e.dirty = e.dirty[:0] for i, dirtyTime := range dirtyTimes { - flushState, ready := e.updateFlushStateWithLock(dirtyTime, targetNanos, isEarlierThanFn) - if !ready || !flushState.dirty { + if !isEarlierThanFn(int64(dirtyTime), resolution, targetNanos) { + // not ready yet + e.dirty = append(e.dirty, dirtyTime) + continue + } + agg, ok := e.values[dirtyTime] + if !ok { + // there is a race where a writer adds a closed aggregation to the dirty set. eventually the closed + // aggregation is expired and removed from the values map. ok to skip. + continue + } + + flushState := e.newFlushStateWithLock(agg) + if flushState.dirty && flushState.flushed && !flushState.resendEnabled { + instrument.EmitAndLogInvariantViolation(e.opts.InstrumentOptions(), func(l *zap.Logger) { + l.Error("reflushing aggregation without resendEnabled", zap.Any("flushState", flushState)) + }) + } + + if !flushState.dirty { // there is a race where the value was added to the dirty set, but the writer didn't actually update the // value yet (by marking dirty). add back to the dirty set so it can be processed in the next round once // the value has been updated. @@ -370,13 +389,16 @@ func (e *CounterElem) Consume( e.values[dirtyTime] = val e.toConsume = append(e.toConsume, flushState) - // potentially consume the nextAgg as well in case we need to cascade an update due to a resend. - nextAgg, ok := e.nextAggWithLock(dirtyTime, targetNanos, isEarlierThanFn) - if ok && (i == len(dirtyTimes)-1 || dirtyTimes[i+1] != nextAgg.startAtNanos) { - // if not already in the dirty set. - flushState, ready := e.updateFlushStateWithLock(nextAgg.startAtNanos, targetNanos, isEarlierThanFn) - if ready { - e.toConsume = append(e.toConsume, flushState) + // potentially consume the nextAgg as well in case we need to cascade an update from a previously flushed + // value. + if flushState.flushed { + nextAgg, ok := e.nextAggWithLock(dirtyTime, targetNanos, isEarlierThanFn) + // add if not already in the dirty set. + if ok && + isEarlierThanFn(int64(nextAgg.startAtNanos), resolution, targetNanos) && + // at the end of the dirty times OR the next dirty time does not match. + (i == len(dirtyTimes)-1 || dirtyTimes[i+1] != nextAgg.startAtNanos) { + e.toConsume = append(e.toConsume, e.newFlushStateWithLock(nextAgg)) } } } @@ -412,23 +434,9 @@ func (e *CounterElem) Consume( return canCollect } -func (e *CounterElem) updateFlushStateWithLock(dirtyTime xtime.UnixNano, targetNanos int64, - isEarlierThanFn isEarlierThanFn) (aggFlushState, bool) { - resolution := e.sp.Resolution().Window - agg, ok := e.values[dirtyTime] - if !ok { - ts := dirtyTime.ToTime() - instrument.EmitAndLogInvariantViolation(e.opts.InstrumentOptions(), func(l *zap.Logger) { - l.Error("dirty timestamp not in values map", zap.Time("ts", ts)) - }) - return aggFlushState{}, false - } - if !isEarlierThanFn(int64(dirtyTime), resolution, targetNanos) { - return aggFlushState{}, false - } - +func (e *CounterElem) newFlushStateWithLock(agg timedCounter) aggFlushState { // note: flushState might be empty for the first flush - flushState := e.flushState[dirtyTime] + flushState := e.flushState[agg.startAtNanos] // copy the lockedAgg data to the flushState while holding the lock. agg.lockedAgg.Lock() flushState.dirty = agg.lockedAgg.dirty @@ -442,15 +450,15 @@ func (e *CounterElem) updateFlushStateWithLock(dirtyTime xtime.UnixNano, targetN agg.lockedAgg.Unlock() // update the flushState with everything else. - previousStartAligned, ok := e.previousStartAlignedWithLock(dirtyTime) + previousStartAligned, ok := e.previousStartAlignedWithLock(agg.startAtNanos) if ok { flushState.prevStartTime = previousStartAligned } else { flushState.prevStartTime = 0 } flushState.resendEnabled = agg.resendEnabled - flushState.startAt = dirtyTime - return flushState, true + flushState.startAt = agg.startAtNanos + return flushState } // Close closes the element. @@ -674,11 +682,9 @@ func (e *CounterElem) processValue( l.Error("previous start time not in state map", zap.Time("ts", ts)) }) - } else if prevFlushState.consumedValues != nil { - // prev consumedValues may be null if the result was NaN. - prev.Value = prevFlushState.consumedValues[aggTypeIdx] - prev.TimeNanos = int64(prevFlushState.timestamp) } + prev.Value = prevFlushState.consumedValues[aggTypeIdx] + prev.TimeNanos = int64(prevFlushState.timestamp) } curr := transformation.Datapoint{ TimeNanos: int64(flushState.timestamp), @@ -690,12 +696,10 @@ func (e *CounterElem) processValue( // We currently only support first-order derivative transformations so we only // need to keep one value. In the future if we need to support higher-order // derivative transformations, we need to store an array of values here. - if !math.IsNaN(curr.Value) { - if flushState.consumedValues == nil { - flushState.consumedValues = make([]float64, len(e.aggTypes)) - } - flushState.consumedValues[aggTypeIdx] = curr.Value + if flushState.consumedValues == nil { + flushState.consumedValues = make([]float64, len(e.aggTypes)) } + flushState.consumedValues[aggTypeIdx] = curr.Value value = res.Value case isUnaryMultiOp: curr := transformation.Datapoint{ diff --git a/src/aggregator/aggregator/elem_test.go b/src/aggregator/aggregator/elem_test.go index 522f078d6f..39e45b7cee 100644 --- a/src/aggregator/aggregator/elem_test.go +++ b/src/aggregator/aggregator/elem_test.go @@ -2270,7 +2270,6 @@ func TestGaugeElemReset(t *testing.T) { require.Equal(t, 0, len(f.consumedValues)) } - // Consume all values. localFn, localRes = testFlushLocalMetricFn() forwardFn, forwardRes = testFlushForwardedMetricFn() diff --git a/src/aggregator/aggregator/entry_test.go b/src/aggregator/aggregator/entry_test.go index 42d3b4a53f..ba28bb1ad4 100644 --- a/src/aggregator/aggregator/entry_test.go +++ b/src/aggregator/aggregator/entry_test.go @@ -28,6 +28,8 @@ import ( "testing" "time" + "github.com/uber-go/tally" + "github.com/m3db/m3/src/aggregator/runtime" "github.com/m3db/m3/src/metrics/aggregation" "github.com/m3db/m3/src/metrics/metadata" @@ -1071,6 +1073,80 @@ func TestAddUntimed_ResendEnabled(t *testing.T) { require.True(t, ok) } +func TestAddUntimed_ClosedAggregation(t *testing.T) { + ctrl := gomock.NewController(t) + defer ctrl.Finish() + + e, _, _ := testEntry(ctrl, testEntryOptions{}) + + metadatas := metadata.StagedMetadatas{ + { + Metadata: metadata.Metadata{ + Pipelines: []metadata.PipelineMetadata{ + { + ResendEnabled: false, + StoragePolicies: policy.StoragePolicies{ + testStoragePolicy, + }, + }, + }, + }, + }, + } + resolution := testStoragePolicy.Resolution().Window + + // add an aggregation + require.NoError(t, e.addUntimed(testGauge, metadatas)) + require.Len(t, e.aggregations, 1) + agg := e.aggregations[0] + require.False(t, agg.resendEnabled) + elem := agg.elem.Value.(*GaugeElem) + vals := elem.values + require.Len(t, vals, 1) + t1 := xtime.ToUnixNano(e.nowFn().Truncate(resolution)) + require.NotNil(t, vals[t1].lockedAgg) + + // consume the aggregation. + t2 := t1.Add(resolution) + require.False(t, consume(elem, t2)) + // the agg is now closed but still exists. + require.True(t, vals[t1].lockedAgg.closed) + require.Empty(t, elem.dirty) + + // adding to a closed aggregation rolls forward to the next aggregation. + require.NoError(t, e.addUntimed(testGauge, metadatas)) + require.Len(t, e.aggregations, 1) + require.Len(t, vals, 2) + require.NotNil(t, vals[t2].lockedAgg) + + // consume next aggregation + t3 := t2.Add(resolution) + require.False(t, consume(elem, t3)) + // the second agg is now closed but still exists. + require.True(t, vals[t2].lockedAgg.closed) + // the first agg is GC'd + require.Nil(t, vals[t1].lockedAgg) + // attempting to add to a closed aggregation leaks a dirty entry + require.Len(t, elem.dirty, 1) + require.Equal(t, t1, elem.dirty[0]) + + // consume again to clear the leaked dirty entry + require.False(t, consume(elem, t3.Add(resolution))) + require.True(t, vals[t2].lockedAgg.closed) + require.Empty(t, elem.dirty) +} + +func consume(elem metricElem, targetTime xtime.UnixNano) bool { + l := &baseMetricList{ + metrics: newMetricListMetrics(tally.NewTestScope("", nil)), + } + return elem.Consume( + int64(targetTime), + isStandardMetricEarlierThan, standardMetricTimestampNanos, standardMetricTargetNanos, + l.discardLocalMetric, l.discardForwardedMetric, l.onForwardingElemDiscarded, + 0, discardType) +} + func TestShouldUpdateStagedMetadataWithLock(t *testing.T) { ctrl := gomock.NewController(t) defer ctrl.Finish() diff --git a/src/aggregator/aggregator/gauge_elem_gen.go b/src/aggregator/aggregator/gauge_elem_gen.go index 14d183c188..794b89080e 100644 --- a/src/aggregator/aggregator/gauge_elem_gen.go +++ b/src/aggregator/aggregator/gauge_elem_gen.go @@ -38,7 +38,6 @@ import ( "github.com/m3db/m3/src/x/instrument" xtime "github.com/m3db/m3/src/x/time" - "github.com/scaleway/scaleway-sdk-go/logger" "github.com/willf/bitset" "go.uber.org/zap" ) @@ -122,9 +121,11 @@ func (e *GaugeElem) ResetSetData(data ElemData) error { // AddUnion adds a metric value union at a given timestamp. func (e *GaugeElem) AddUnion(timestamp time.Time, mu unaggregated.MetricUnion, resendEnabled bool) error { - return e.doAddUnion(timestamp, mu, resendEnabled, false, timestamp) + return e.doAddUnion(timestamp, mu, resendEnabled, false) } -func (e *GaugeElem) doAddUnion(timestamp time.Time, mu unaggregated.MetricUnion, resendEnabled bool, retry bool, initialTimestamp time.Time) error { + +func (e *GaugeElem) doAddUnion(timestamp time.Time, mu unaggregated.MetricUnion, resendEnabled bool, retry bool, +) error { alignedStart := timestamp.Truncate(e.sp.Resolution().Window) lockedAgg, err := e.findOrCreate(alignedStart.UnixNano(), createAggregationOptions{ resendEnabled: resendEnabled, @@ -134,14 +135,14 @@ func (e *GaugeElem) doAddUnion(timestamp time.Time, mu unaggregated.MetricUnion, } lockedAgg.Lock() if lockedAgg.closed { + // Note: this might have created an entry in the dirty set for lockedAgg when calling findOrCreate, even though + // it's already closed. The Consume loop will detect this and clean it up. lockedAgg.Unlock() if !resendEnabled && !retry { - return e.doAddUnion(alignedStart.Add(e.sp.Resolution().Window), mu, false, true, alignedStart) + // handle the edge case where the aggregation was already flushed/closed because the current time is right + // at the boundary. just roll the untimed metric into the next aggregation. + return e.doAddUnion(alignedStart.Add(e.sp.Resolution().Window), mu, false, true) } - logger.Errorf("aggregation already closed", - zap.Time("timestamp", timestamp), - zap.Time("initialTimestamp", initialTimestamp), - zap.Bool("retry", retry)) return errAggregationClosed } lockedAgg.aggregation.AddUnion(timestamp, mu) @@ -230,10 +231,8 @@ func (e *GaugeElem) expireValuesWithLock( } resolution := e.sp.Resolution().Window - // start after the minimum to ensure we always keep at least one value in the map for binary transformations. - currStart := e.minStartTime.Add(resolution) + currStart := e.minStartTime resendExpire := targetNanos - int64(e.bufferForPastTimedMetricFn(resolution)) - for isEarlierThanFn(int64(currStart), resolution, targetNanos) { if currV, ok := e.values[currStart]; ok { if currV.resendEnabled { @@ -257,7 +256,9 @@ func (e *GaugeElem) expireValuesWithLock( // if this current value is closed and clean it will no longer be flushed. this means it's safe // to remove the previous value since it will no longer be needed for binary transformations. when the // next value is eligible to be expired, this current value will actually be removed. - if prevV, ok := e.values[e.minStartTime]; ok { + // if we're currently pointing at the start skip this there is no previous for the start. this ensures + // we always keep at least one value in the map for binary transformations. + if prevV, ok := e.values[e.minStartTime]; ok && currStart != e.minStartTime { // can't expire flush state until after the flushing, so we save the time to expire later. e.flushStateToExpire = append(e.flushStateToExpire, e.minStartTime) delete(e.values, e.minStartTime) @@ -357,8 +358,26 @@ func (e *GaugeElem) Consume( dirtyTimes := e.dirty e.dirty = e.dirty[:0] for i, dirtyTime := range dirtyTimes { - flushState, ready := e.updateFlushStateWithLock(dirtyTime, targetNanos, isEarlierThanFn) - if !ready || !flushState.dirty { + if !isEarlierThanFn(int64(dirtyTime), resolution, targetNanos) { + // not ready yet + e.dirty = append(e.dirty, dirtyTime) + continue + } + agg, ok := e.values[dirtyTime] + if !ok { + // there is a race where a writer adds a closed aggregation to the dirty set. eventually the closed + // aggregation is expired and removed from the values map. ok to skip. + continue + } + + flushState := e.newFlushStateWithLock(agg) + if flushState.dirty && flushState.flushed && !flushState.resendEnabled { + instrument.EmitAndLogInvariantViolation(e.opts.InstrumentOptions(), func(l *zap.Logger) { + l.Error("reflushing aggregation without resendEnabled", zap.Any("flushState", flushState)) + }) + } + + if !flushState.dirty { // there is a race where the value was added to the dirty set, but the writer didn't actually update the // value yet (by marking dirty). add back to the dirty set so it can be processed in the next round once // the value has been updated. @@ -370,13 +389,16 @@ func (e *GaugeElem) Consume( e.values[dirtyTime] = val e.toConsume = append(e.toConsume, flushState) - // potentially consume the nextAgg as well in case we need to cascade an update due to a resend. - nextAgg, ok := e.nextAggWithLock(dirtyTime, targetNanos, isEarlierThanFn) - if ok && (i == len(dirtyTimes)-1 || dirtyTimes[i+1] != nextAgg.startAtNanos) { - // if not already in the dirty set. - flushState, ready := e.updateFlushStateWithLock(nextAgg.startAtNanos, targetNanos, isEarlierThanFn) - if ready { - e.toConsume = append(e.toConsume, flushState) + // potentially consume the nextAgg as well in case we need to cascade an update from a previously flushed + // value. + if flushState.flushed { + nextAgg, ok := e.nextAggWithLock(dirtyTime, targetNanos, isEarlierThanFn) + // add if not already in the dirty set. + if ok && + isEarlierThanFn(int64(nextAgg.startAtNanos), resolution, targetNanos) && + // at the end of the dirty times OR the next dirty time does not match. + (i == len(dirtyTimes)-1 || dirtyTimes[i+1] != nextAgg.startAtNanos) { + e.toConsume = append(e.toConsume, e.newFlushStateWithLock(nextAgg)) } } } @@ -412,23 +434,9 @@ func (e *GaugeElem) Consume( return canCollect } -func (e *GaugeElem) updateFlushStateWithLock(dirtyTime xtime.UnixNano, targetNanos int64, - isEarlierThanFn isEarlierThanFn) (aggFlushState, bool) { - resolution := e.sp.Resolution().Window - agg, ok := e.values[dirtyTime] - if !ok { - ts := dirtyTime.ToTime() - instrument.EmitAndLogInvariantViolation(e.opts.InstrumentOptions(), func(l *zap.Logger) { - l.Error("dirty timestamp not in values map", zap.Time("ts", ts)) - }) - return aggFlushState{}, false - } - if !isEarlierThanFn(int64(dirtyTime), resolution, targetNanos) { - return aggFlushState{}, false - } - +func (e *GaugeElem) newFlushStateWithLock(agg timedGauge) aggFlushState { // note: flushState might be empty for the first flush - flushState := e.flushState[dirtyTime] + flushState := e.flushState[agg.startAtNanos] // copy the lockedAgg data to the flushState while holding the lock. agg.lockedAgg.Lock() flushState.dirty = agg.lockedAgg.dirty @@ -442,15 +450,15 @@ func (e *GaugeElem) updateFlushStateWithLock(dirtyTime xtime.UnixNano, targetNan agg.lockedAgg.Unlock() // update the flushState with everything else. - previousStartAligned, ok := e.previousStartAlignedWithLock(dirtyTime) + previousStartAligned, ok := e.previousStartAlignedWithLock(agg.startAtNanos) if ok { flushState.prevStartTime = previousStartAligned } else { flushState.prevStartTime = 0 } flushState.resendEnabled = agg.resendEnabled - flushState.startAt = dirtyTime - return flushState, true + flushState.startAt = agg.startAtNanos + return flushState } // Close closes the element. @@ -674,11 +682,9 @@ func (e *GaugeElem) processValue( l.Error("previous start time not in state map", zap.Time("ts", ts)) }) - } else if prevFlushState.consumedValues != nil { - // prev consumedValues may be null if the result was NaN. - prev.Value = prevFlushState.consumedValues[aggTypeIdx] - prev.TimeNanos = int64(prevFlushState.timestamp) } + prev.Value = prevFlushState.consumedValues[aggTypeIdx] + prev.TimeNanos = int64(prevFlushState.timestamp) } curr := transformation.Datapoint{ TimeNanos: int64(flushState.timestamp), @@ -690,12 +696,10 @@ func (e *GaugeElem) processValue( // We currently only support first-order derivative transformations so we only // need to keep one value. In the future if we need to support higher-order // derivative transformations, we need to store an array of values here. - if !math.IsNaN(curr.Value) { - if flushState.consumedValues == nil { - flushState.consumedValues = make([]float64, len(e.aggTypes)) - } - flushState.consumedValues[aggTypeIdx] = curr.Value + if flushState.consumedValues == nil { + flushState.consumedValues = make([]float64, len(e.aggTypes)) } + flushState.consumedValues[aggTypeIdx] = curr.Value value = res.Value case isUnaryMultiOp: curr := transformation.Datapoint{ diff --git a/src/aggregator/aggregator/generic_elem.go b/src/aggregator/aggregator/generic_elem.go index f36394e3a0..4f41e0d85c 100644 --- a/src/aggregator/aggregator/generic_elem.go +++ b/src/aggregator/aggregator/generic_elem.go @@ -27,7 +27,6 @@ import ( "time" "github.com/mauricelam/genny/generic" - "github.com/scaleway/scaleway-sdk-go/logger" "github.com/willf/bitset" "go.uber.org/zap" @@ -185,9 +184,11 @@ func (e *GenericElem) ResetSetData(data ElemData) error { // AddUnion adds a metric value union at a given timestamp. func (e *GenericElem) AddUnion(timestamp time.Time, mu unaggregated.MetricUnion, resendEnabled bool) error { - return e.doAddUnion(timestamp, mu, resendEnabled, false, timestamp) + return e.doAddUnion(timestamp, mu, resendEnabled, false) } -func (e *GenericElem) doAddUnion(timestamp time.Time, mu unaggregated.MetricUnion, resendEnabled bool, retry bool, initialTimestamp time.Time) error { + +func (e *GenericElem) doAddUnion(timestamp time.Time, mu unaggregated.MetricUnion, resendEnabled bool, retry bool, +) error { alignedStart := timestamp.Truncate(e.sp.Resolution().Window) lockedAgg, err := e.findOrCreate(alignedStart.UnixNano(), createAggregationOptions{ resendEnabled: resendEnabled, @@ -197,14 +198,14 @@ func (e *GenericElem) doAddUnion(timestamp time.Time, mu unaggregated.MetricUnio } lockedAgg.Lock() if lockedAgg.closed { + // Note: this might have created an entry in the dirty set for lockedAgg when calling findOrCreate, even though + // it's already closed. The Consume loop will detect this and clean it up. lockedAgg.Unlock() if !resendEnabled && !retry { - return e.doAddUnion(alignedStart.Add(e.sp.Resolution().Window), mu, false, true, alignedStart) + // handle the edge case where the aggregation was already flushed/closed because the current time is right + // at the boundary. just roll the untimed metric into the next aggregation. + return e.doAddUnion(alignedStart.Add(e.sp.Resolution().Window), mu, false, true) } - logger.Errorf("aggregation already closed", - zap.Time("timestamp", timestamp), - zap.Time("initialTimestamp", initialTimestamp), - zap.Bool("retry", retry)) return errAggregationClosed } lockedAgg.aggregation.AddUnion(timestamp, mu) @@ -293,10 +294,8 @@ func (e *GenericElem) expireValuesWithLock( } resolution := e.sp.Resolution().Window - // start after the minimum to ensure we always keep at least one value in the map for binary transformations. - currStart := e.minStartTime.Add(resolution) + currStart := e.minStartTime resendExpire := targetNanos - int64(e.bufferForPastTimedMetricFn(resolution)) - for isEarlierThanFn(int64(currStart), resolution, targetNanos) { if currV, ok := e.values[currStart]; ok { if currV.resendEnabled { @@ -320,7 +319,9 @@ func (e *GenericElem) expireValuesWithLock( // if this current value is closed and clean it will no longer be flushed. this means it's safe // to remove the previous value since it will no longer be needed for binary transformations. when the // next value is eligible to be expired, this current value will actually be removed. - if prevV, ok := e.values[e.minStartTime]; ok { + // if we're currently pointing at the start skip this there is no previous for the start. this ensures + // we always keep at least one value in the map for binary transformations. + if prevV, ok := e.values[e.minStartTime]; ok && currStart != e.minStartTime { // can't expire flush state until after the flushing, so we save the time to expire later. e.flushStateToExpire = append(e.flushStateToExpire, e.minStartTime) delete(e.values, e.minStartTime) @@ -420,8 +421,26 @@ func (e *GenericElem) Consume( dirtyTimes := e.dirty e.dirty = e.dirty[:0] for i, dirtyTime := range dirtyTimes { - flushState, ready := e.updateFlushStateWithLock(dirtyTime, targetNanos, isEarlierThanFn) - if !ready || !flushState.dirty { + if !isEarlierThanFn(int64(dirtyTime), resolution, targetNanos) { + // not ready yet + e.dirty = append(e.dirty, dirtyTime) + continue + } + agg, ok := e.values[dirtyTime] + if !ok { + // there is a race where a writer adds a closed aggregation to the dirty set. eventually the closed + // aggregation is expired and removed from the values map. ok to skip. + continue + } + + flushState := e.newFlushStateWithLock(agg) + if flushState.dirty && flushState.flushed && !flushState.resendEnabled { + instrument.EmitAndLogInvariantViolation(e.opts.InstrumentOptions(), func(l *zap.Logger) { + l.Error("reflushing aggregation without resendEnabled", zap.Any("flushState", flushState)) + }) + } + + if !flushState.dirty { // there is a race where the value was added to the dirty set, but the writer didn't actually update the // value yet (by marking dirty). add back to the dirty set so it can be processed in the next round once // the value has been updated. @@ -433,13 +452,16 @@ func (e *GenericElem) Consume( e.values[dirtyTime] = val e.toConsume = append(e.toConsume, flushState) - // potentially consume the nextAgg as well in case we need to cascade an update due to a resend. - nextAgg, ok := e.nextAggWithLock(dirtyTime, targetNanos, isEarlierThanFn) - if ok && (i == len(dirtyTimes)-1 || dirtyTimes[i+1] != nextAgg.startAtNanos) { - // if not already in the dirty set. - flushState, ready := e.updateFlushStateWithLock(nextAgg.startAtNanos, targetNanos, isEarlierThanFn) - if ready { - e.toConsume = append(e.toConsume, flushState) + // potentially consume the nextAgg as well in case we need to cascade an update from a previously flushed + // value. + if flushState.flushed { + nextAgg, ok := e.nextAggWithLock(dirtyTime, targetNanos, isEarlierThanFn) + // add if not already in the dirty set. + if ok && + isEarlierThanFn(int64(nextAgg.startAtNanos), resolution, targetNanos) && + // at the end of the dirty times OR the next dirty time does not match. + (i == len(dirtyTimes)-1 || dirtyTimes[i+1] != nextAgg.startAtNanos) { + e.toConsume = append(e.toConsume, e.newFlushStateWithLock(nextAgg)) } } } @@ -475,23 +497,9 @@ func (e *GenericElem) Consume( return canCollect } -func (e *GenericElem) updateFlushStateWithLock(dirtyTime xtime.UnixNano, targetNanos int64, - isEarlierThanFn isEarlierThanFn) (aggFlushState, bool) { - resolution := e.sp.Resolution().Window - agg, ok := e.values[dirtyTime] - if !ok { - ts := dirtyTime.ToTime() - instrument.EmitAndLogInvariantViolation(e.opts.InstrumentOptions(), func(l *zap.Logger) { - l.Error("dirty timestamp not in values map", zap.Time("ts", ts)) - }) - return aggFlushState{}, false - } - if !isEarlierThanFn(int64(dirtyTime), resolution, targetNanos) { - return aggFlushState{}, false - } - +func (e *GenericElem) newFlushStateWithLock(agg timedAggregation) aggFlushState { // note: flushState might be empty for the first flush - flushState := e.flushState[dirtyTime] + flushState := e.flushState[agg.startAtNanos] // copy the lockedAgg data to the flushState while holding the lock. agg.lockedAgg.Lock() flushState.dirty = agg.lockedAgg.dirty @@ -505,15 +513,15 @@ func (e *GenericElem) updateFlushStateWithLock(dirtyTime xtime.UnixNano, targetN agg.lockedAgg.Unlock() // update the flushState with everything else. - previousStartAligned, ok := e.previousStartAlignedWithLock(dirtyTime) + previousStartAligned, ok := e.previousStartAlignedWithLock(agg.startAtNanos) if ok { flushState.prevStartTime = previousStartAligned } else { flushState.prevStartTime = 0 } flushState.resendEnabled = agg.resendEnabled - flushState.startAt = dirtyTime - return flushState, true + flushState.startAt = agg.startAtNanos + return flushState } // Close closes the element. @@ -737,11 +745,9 @@ func (e *GenericElem) processValue( l.Error("previous start time not in state map", zap.Time("ts", ts)) }) - } else if prevFlushState.consumedValues != nil { - // prev consumedValues may be null if the result was NaN. - prev.Value = prevFlushState.consumedValues[aggTypeIdx] - prev.TimeNanos = int64(prevFlushState.timestamp) } + prev.Value = prevFlushState.consumedValues[aggTypeIdx] + prev.TimeNanos = int64(prevFlushState.timestamp) } curr := transformation.Datapoint{ TimeNanos: int64(flushState.timestamp), @@ -753,12 +759,10 @@ func (e *GenericElem) processValue( // We currently only support first-order derivative transformations so we only // need to keep one value. In the future if we need to support higher-order // derivative transformations, we need to store an array of values here. - if !math.IsNaN(curr.Value) { - if flushState.consumedValues == nil { - flushState.consumedValues = make([]float64, len(e.aggTypes)) - } - flushState.consumedValues[aggTypeIdx] = curr.Value + if flushState.consumedValues == nil { + flushState.consumedValues = make([]float64, len(e.aggTypes)) } + flushState.consumedValues[aggTypeIdx] = curr.Value value = res.Value case isUnaryMultiOp: curr := transformation.Datapoint{ diff --git a/src/aggregator/aggregator/timer_elem_gen.go b/src/aggregator/aggregator/timer_elem_gen.go index c59d05efd4..84fc68715e 100644 --- a/src/aggregator/aggregator/timer_elem_gen.go +++ b/src/aggregator/aggregator/timer_elem_gen.go @@ -38,7 +38,6 @@ import ( "github.com/m3db/m3/src/x/instrument" xtime "github.com/m3db/m3/src/x/time" - "github.com/scaleway/scaleway-sdk-go/logger" "github.com/willf/bitset" "go.uber.org/zap" ) @@ -122,9 +121,11 @@ func (e *TimerElem) ResetSetData(data ElemData) error { // AddUnion adds a metric value union at a given timestamp. func (e *TimerElem) AddUnion(timestamp time.Time, mu unaggregated.MetricUnion, resendEnabled bool) error { - return e.doAddUnion(timestamp, mu, resendEnabled, false, timestamp) + return e.doAddUnion(timestamp, mu, resendEnabled, false) } -func (e *TimerElem) doAddUnion(timestamp time.Time, mu unaggregated.MetricUnion, resendEnabled bool, retry bool, initialTimestamp time.Time) error { + +func (e *TimerElem) doAddUnion(timestamp time.Time, mu unaggregated.MetricUnion, resendEnabled bool, retry bool, +) error { alignedStart := timestamp.Truncate(e.sp.Resolution().Window) lockedAgg, err := e.findOrCreate(alignedStart.UnixNano(), createAggregationOptions{ resendEnabled: resendEnabled, @@ -134,14 +135,14 @@ func (e *TimerElem) doAddUnion(timestamp time.Time, mu unaggregated.MetricUnion, } lockedAgg.Lock() if lockedAgg.closed { + // Note: this might have created an entry in the dirty set for lockedAgg when calling findOrCreate, even though + // it's already closed. The Consume loop will detect this and clean it up. lockedAgg.Unlock() if !resendEnabled && !retry { - return e.doAddUnion(alignedStart.Add(e.sp.Resolution().Window), mu, false, true, alignedStart) + // handle the edge case where the aggregation was already flushed/closed because the current time is right + // at the boundary. just roll the untimed metric into the next aggregation. + return e.doAddUnion(alignedStart.Add(e.sp.Resolution().Window), mu, false, true) } - logger.Errorf("aggregation already closed", - zap.Time("timestamp", timestamp), - zap.Time("initialTimestamp", initialTimestamp), - zap.Bool("retry", retry)) return errAggregationClosed } lockedAgg.aggregation.AddUnion(timestamp, mu) @@ -230,10 +231,8 @@ func (e *TimerElem) expireValuesWithLock( } resolution := e.sp.Resolution().Window - // start after the minimum to ensure we always keep at least one value in the map for binary transformations. - currStart := e.minStartTime.Add(resolution) + currStart := e.minStartTime resendExpire := targetNanos - int64(e.bufferForPastTimedMetricFn(resolution)) - for isEarlierThanFn(int64(currStart), resolution, targetNanos) { if currV, ok := e.values[currStart]; ok { if currV.resendEnabled { @@ -257,7 +256,9 @@ func (e *TimerElem) expireValuesWithLock( // if this current value is closed and clean it will no longer be flushed. this means it's safe // to remove the previous value since it will no longer be needed for binary transformations. when the // next value is eligible to be expired, this current value will actually be removed. - if prevV, ok := e.values[e.minStartTime]; ok { + // if we're currently pointing at the start skip this there is no previous for the start. this ensures + // we always keep at least one value in the map for binary transformations. + if prevV, ok := e.values[e.minStartTime]; ok && currStart != e.minStartTime { // can't expire flush state until after the flushing, so we save the time to expire later. e.flushStateToExpire = append(e.flushStateToExpire, e.minStartTime) delete(e.values, e.minStartTime) @@ -357,8 +358,26 @@ func (e *TimerElem) Consume( dirtyTimes := e.dirty e.dirty = e.dirty[:0] for i, dirtyTime := range dirtyTimes { - flushState, ready := e.updateFlushStateWithLock(dirtyTime, targetNanos, isEarlierThanFn) - if !ready || !flushState.dirty { + if !isEarlierThanFn(int64(dirtyTime), resolution, targetNanos) { + // not ready yet + e.dirty = append(e.dirty, dirtyTime) + continue + } + agg, ok := e.values[dirtyTime] + if !ok { + // there is a race where a writer adds a closed aggregation to the dirty set. eventually the closed + // aggregation is expired and removed from the values map. ok to skip. + continue + } + + flushState := e.newFlushStateWithLock(agg) + if flushState.dirty && flushState.flushed && !flushState.resendEnabled { + instrument.EmitAndLogInvariantViolation(e.opts.InstrumentOptions(), func(l *zap.Logger) { + l.Error("reflushing aggregation without resendEnabled", zap.Any("flushState", flushState)) + }) + } + + if !flushState.dirty { // there is a race where the value was added to the dirty set, but the writer didn't actually update the // value yet (by marking dirty). add back to the dirty set so it can be processed in the next round once // the value has been updated. @@ -370,13 +389,16 @@ func (e *TimerElem) Consume( e.values[dirtyTime] = val e.toConsume = append(e.toConsume, flushState) - // potentially consume the nextAgg as well in case we need to cascade an update due to a resend. - nextAgg, ok := e.nextAggWithLock(dirtyTime, targetNanos, isEarlierThanFn) - if ok && (i == len(dirtyTimes)-1 || dirtyTimes[i+1] != nextAgg.startAtNanos) { - // if not already in the dirty set. - flushState, ready := e.updateFlushStateWithLock(nextAgg.startAtNanos, targetNanos, isEarlierThanFn) - if ready { - e.toConsume = append(e.toConsume, flushState) + // potentially consume the nextAgg as well in case we need to cascade an update from a previously flushed + // value. + if flushState.flushed { + nextAgg, ok := e.nextAggWithLock(dirtyTime, targetNanos, isEarlierThanFn) + // add if not already in the dirty set. + if ok && + isEarlierThanFn(int64(nextAgg.startAtNanos), resolution, targetNanos) && + // at the end of the dirty times OR the next dirty time does not match. + (i == len(dirtyTimes)-1 || dirtyTimes[i+1] != nextAgg.startAtNanos) { + e.toConsume = append(e.toConsume, e.newFlushStateWithLock(nextAgg)) } } } @@ -412,23 +434,9 @@ func (e *TimerElem) Consume( return canCollect } -func (e *TimerElem) updateFlushStateWithLock(dirtyTime xtime.UnixNano, targetNanos int64, - isEarlierThanFn isEarlierThanFn) (aggFlushState, bool) { - resolution := e.sp.Resolution().Window - agg, ok := e.values[dirtyTime] - if !ok { - ts := dirtyTime.ToTime() - instrument.EmitAndLogInvariantViolation(e.opts.InstrumentOptions(), func(l *zap.Logger) { - l.Error("dirty timestamp not in values map", zap.Time("ts", ts)) - }) - return aggFlushState{}, false - } - if !isEarlierThanFn(int64(dirtyTime), resolution, targetNanos) { - return aggFlushState{}, false - } - +func (e *TimerElem) newFlushStateWithLock(agg timedTimer) aggFlushState { // note: flushState might be empty for the first flush - flushState := e.flushState[dirtyTime] + flushState := e.flushState[agg.startAtNanos] // copy the lockedAgg data to the flushState while holding the lock. agg.lockedAgg.Lock() flushState.dirty = agg.lockedAgg.dirty @@ -442,15 +450,15 @@ func (e *TimerElem) updateFlushStateWithLock(dirtyTime xtime.UnixNano, targetNan agg.lockedAgg.Unlock() // update the flushState with everything else. - previousStartAligned, ok := e.previousStartAlignedWithLock(dirtyTime) + previousStartAligned, ok := e.previousStartAlignedWithLock(agg.startAtNanos) if ok { flushState.prevStartTime = previousStartAligned } else { flushState.prevStartTime = 0 } flushState.resendEnabled = agg.resendEnabled - flushState.startAt = dirtyTime - return flushState, true + flushState.startAt = agg.startAtNanos + return flushState } // Close closes the element. @@ -674,11 +682,9 @@ func (e *TimerElem) processValue( l.Error("previous start time not in state map", zap.Time("ts", ts)) }) - } else if prevFlushState.consumedValues != nil { - // prev consumedValues may be null if the result was NaN. - prev.Value = prevFlushState.consumedValues[aggTypeIdx] - prev.TimeNanos = int64(prevFlushState.timestamp) } + prev.Value = prevFlushState.consumedValues[aggTypeIdx] + prev.TimeNanos = int64(prevFlushState.timestamp) } curr := transformation.Datapoint{ TimeNanos: int64(flushState.timestamp), @@ -690,12 +696,10 @@ func (e *TimerElem) processValue( // We currently only support first-order derivative transformations so we only // need to keep one value. In the future if we need to support higher-order // derivative transformations, we need to store an array of values here. - if !math.IsNaN(curr.Value) { - if flushState.consumedValues == nil { - flushState.consumedValues = make([]float64, len(e.aggTypes)) - } - flushState.consumedValues[aggTypeIdx] = curr.Value + if flushState.consumedValues == nil { + flushState.consumedValues = make([]float64, len(e.aggTypes)) } + flushState.consumedValues[aggTypeIdx] = curr.Value value = res.Value case isUnaryMultiOp: curr := transformation.Datapoint{ From 4bbee8abc45ca0bc907f1e39e3043b2be24e815f Mon Sep 17 00:00:00 2001 From: Ryan Hall Date: Wed, 3 Nov 2021 09:41:53 -0700 Subject: [PATCH 5/6] review comments --- src/aggregator/aggregator/counter_elem_gen.go | 36 +++++++++++-------- src/aggregator/aggregator/elem_base.go | 2 ++ src/aggregator/aggregator/elem_test.go | 13 +++++-- src/aggregator/aggregator/gauge_elem_gen.go | 36 +++++++++++-------- src/aggregator/aggregator/generic_elem.go | 36 +++++++++++-------- src/aggregator/aggregator/timer_elem_gen.go | 36 +++++++++++-------- 6 files changed, 100 insertions(+), 59 deletions(-) diff --git a/src/aggregator/aggregator/counter_elem_gen.go b/src/aggregator/aggregator/counter_elem_gen.go index 82bfe169b3..002979904c 100644 --- a/src/aggregator/aggregator/counter_elem_gen.go +++ b/src/aggregator/aggregator/counter_elem_gen.go @@ -148,6 +148,9 @@ func (e *CounterElem) doAddUnion(timestamp time.Time, mu unaggregated.MetricUnio lockedAgg.aggregation.AddUnion(timestamp, mu) lockedAgg.dirty = true lockedAgg.Unlock() + if retry { + e.metrics.retriedValues.Inc(1) + } return nil } @@ -256,8 +259,8 @@ func (e *CounterElem) expireValuesWithLock( // if this current value is closed and clean it will no longer be flushed. this means it's safe // to remove the previous value since it will no longer be needed for binary transformations. when the // next value is eligible to be expired, this current value will actually be removed. - // if we're currently pointing at the start skip this there is no previous for the start. this ensures - // we always keep at least one value in the map for binary transformations. + // if we're currently pointing at the start skip this because there is no previous for the start. this + // ensures we always keep at least one value in the map for binary transformations. if prevV, ok := e.values[e.minStartTime]; ok && currStart != e.minStartTime { // can't expire flush state until after the flushing, so we save the time to expire later. e.flushStateToExpire = append(e.flushStateToExpire, e.minStartTime) @@ -371,12 +374,6 @@ func (e *CounterElem) Consume( } flushState := e.newFlushStateWithLock(agg) - if flushState.dirty && flushState.flushed && !flushState.resendEnabled { - instrument.EmitAndLogInvariantViolation(e.opts.InstrumentOptions(), func(l *zap.Logger) { - l.Error("reflushing aggregation without resendEnabled", zap.Any("flushState", flushState)) - }) - } - if !flushState.dirty { // there is a race where the value was added to the dirty set, but the writer didn't actually update the // value yet (by marking dirty). add back to the dirty set so it can be processed in the next round once @@ -389,16 +386,21 @@ func (e *CounterElem) Consume( e.values[dirtyTime] = val e.toConsume = append(e.toConsume, flushState) - // potentially consume the nextAgg as well in case we need to cascade an update from a previously flushed - // value. - if flushState.flushed { + // potentially consume the nextAgg as well in case we need to cascade an update to the nextAgg. + // this is necessary for binary transformations that rely on the previous aggregation value for calculating the + // current aggregation value. if the nextAgg was already flushed, it used an outdated value for the previous + // value (this agg). this can only happen when we allow updating previously flushed data (i.e resendEnabled). + if flushState.resendEnabled { nextAgg, ok := e.nextAggWithLock(dirtyTime, targetNanos, isEarlierThanFn) - // add if not already in the dirty set. + // only need to add if not already in the dirty set (since it will be added in a subsequent iteration). if ok && - isEarlierThanFn(int64(nextAgg.startAtNanos), resolution, targetNanos) && // at the end of the dirty times OR the next dirty time does not match. (i == len(dirtyTimes)-1 || dirtyTimes[i+1] != nextAgg.startAtNanos) { - e.toConsume = append(e.toConsume, e.newFlushStateWithLock(nextAgg)) + nextAggFlush := e.newFlushStateWithLock(nextAgg) + // only need to add if it was previously flushed. + if nextAggFlush.flushed { + e.toConsume = append(e.toConsume, nextAggFlush) + } } } } @@ -409,6 +411,12 @@ func (e *CounterElem) Consume( // Process the aggregations that are ready for consumption. for _, flushState := range e.toConsume { + if flushState.dirty && flushState.flushed && !flushState.resendEnabled { + flushState := flushState + instrument.EmitAndLogInvariantViolation(e.opts.InstrumentOptions(), func(l *zap.Logger) { + l.Error("reflushing aggregation without resendEnabled", zap.Any("flushState", flushState)) + }) + } flushState.timestamp = xtime.UnixNano(timestampNanosFn(int64(flushState.startAt), resolution)) flushState = e.processValue( flushState, diff --git a/src/aggregator/aggregator/elem_base.go b/src/aggregator/aggregator/elem_base.go index 0037bd4593..6b786fdf9d 100644 --- a/src/aggregator/aggregator/elem_base.go +++ b/src/aggregator/aggregator/elem_base.go @@ -223,6 +223,7 @@ type elemMetrics struct { sync.RWMutex scope tally.Scope updatedValues tally.Counter + retriedValues tally.Counter forwardLags map[forwardLagKey]tally.Histogram } @@ -298,6 +299,7 @@ func NewElemOptions(aggregatorOpts Options) ElemOptions { aggregationOpts: raggregation.NewOptions(aggregatorOpts.InstrumentOptions()), elemMetrics: &elemMetrics{ updatedValues: scope.Counter("updated-values"), + retriedValues: scope.Counter("retried-values"), scope: scope, forwardLags: make(map[forwardLagKey]tally.Histogram), }, diff --git a/src/aggregator/aggregator/elem_test.go b/src/aggregator/aggregator/elem_test.go index 39e45b7cee..988858c631 100644 --- a/src/aggregator/aggregator/elem_test.go +++ b/src/aggregator/aggregator/elem_test.go @@ -1765,7 +1765,10 @@ func TestGaugeElemConsumeResendBuffer(t *testing.T) { // Update the first value after flushing updatedVal := testGaugeVals[0] - 1.0 - require.NoError(t, e.AddValue(time.Unix(0, testAlignedStarts[0]), updatedVal, nil)) + mu := unaggregated.MetricUnion{ + GaugeVal: updatedVal, + } + require.NoError(t, e.AddUnion(time.Unix(0, testAlignedStarts[0]), mu, true)) localFn, localRes = testFlushLocalMetricFn() forwardFn, forwardRes = testFlushForwardedMetricFn() onForwardedFlushedFn, onForwardedFlushedRes = testOnForwardedFlushedFn() @@ -2057,7 +2060,8 @@ func TestGaugeElemResendSumReset(t *testing.T) { PrevValues: []float64{123}, Version: 1, }, metadata.ForwardMetadata{ - SourceID: 1, + ResendEnabled: true, + SourceID: 1, })) localFn, localRes = testFlushLocalMetricFn() forwardFn, forwardRes = testFlushForwardedMetricFn() @@ -2191,7 +2195,10 @@ func TestGaugeElemResendBufferForwarding(t *testing.T) { require.Equal(t, 589.0, e.flushState[xtime.ToUnixNano(time.Unix(230, 0))].consumedValues[0]) // Update a previous value - require.NoError(t, e.AddValue(time.Unix(210, 0), 124.0, nil)) + mu := unaggregated.MetricUnion{ + GaugeVal: 124.0, + } + require.NoError(t, e.AddUnion(time.Unix(210, 0), mu, true)) expectedForwardedRes = []testForwardedMetricWithMetadata{ { aggregationKey: aggKey, diff --git a/src/aggregator/aggregator/gauge_elem_gen.go b/src/aggregator/aggregator/gauge_elem_gen.go index 794b89080e..e56e76737e 100644 --- a/src/aggregator/aggregator/gauge_elem_gen.go +++ b/src/aggregator/aggregator/gauge_elem_gen.go @@ -148,6 +148,9 @@ func (e *GaugeElem) doAddUnion(timestamp time.Time, mu unaggregated.MetricUnion, lockedAgg.aggregation.AddUnion(timestamp, mu) lockedAgg.dirty = true lockedAgg.Unlock() + if retry { + e.metrics.retriedValues.Inc(1) + } return nil } @@ -256,8 +259,8 @@ func (e *GaugeElem) expireValuesWithLock( // if this current value is closed and clean it will no longer be flushed. this means it's safe // to remove the previous value since it will no longer be needed for binary transformations. when the // next value is eligible to be expired, this current value will actually be removed. - // if we're currently pointing at the start skip this there is no previous for the start. this ensures - // we always keep at least one value in the map for binary transformations. + // if we're currently pointing at the start skip this because there is no previous for the start. this + // ensures we always keep at least one value in the map for binary transformations. if prevV, ok := e.values[e.minStartTime]; ok && currStart != e.minStartTime { // can't expire flush state until after the flushing, so we save the time to expire later. e.flushStateToExpire = append(e.flushStateToExpire, e.minStartTime) @@ -371,12 +374,6 @@ func (e *GaugeElem) Consume( } flushState := e.newFlushStateWithLock(agg) - if flushState.dirty && flushState.flushed && !flushState.resendEnabled { - instrument.EmitAndLogInvariantViolation(e.opts.InstrumentOptions(), func(l *zap.Logger) { - l.Error("reflushing aggregation without resendEnabled", zap.Any("flushState", flushState)) - }) - } - if !flushState.dirty { // there is a race where the value was added to the dirty set, but the writer didn't actually update the // value yet (by marking dirty). add back to the dirty set so it can be processed in the next round once @@ -389,16 +386,21 @@ func (e *GaugeElem) Consume( e.values[dirtyTime] = val e.toConsume = append(e.toConsume, flushState) - // potentially consume the nextAgg as well in case we need to cascade an update from a previously flushed - // value. - if flushState.flushed { + // potentially consume the nextAgg as well in case we need to cascade an update to the nextAgg. + // this is necessary for binary transformations that rely on the previous aggregation value for calculating the + // current aggregation value. if the nextAgg was already flushed, it used an outdated value for the previous + // value (this agg). this can only happen when we allow updating previously flushed data (i.e resendEnabled). + if flushState.resendEnabled { nextAgg, ok := e.nextAggWithLock(dirtyTime, targetNanos, isEarlierThanFn) - // add if not already in the dirty set. + // only need to add if not already in the dirty set (since it will be added in a subsequent iteration). if ok && - isEarlierThanFn(int64(nextAgg.startAtNanos), resolution, targetNanos) && // at the end of the dirty times OR the next dirty time does not match. (i == len(dirtyTimes)-1 || dirtyTimes[i+1] != nextAgg.startAtNanos) { - e.toConsume = append(e.toConsume, e.newFlushStateWithLock(nextAgg)) + nextAggFlush := e.newFlushStateWithLock(nextAgg) + // only need to add if it was previously flushed. + if nextAggFlush.flushed { + e.toConsume = append(e.toConsume, nextAggFlush) + } } } } @@ -409,6 +411,12 @@ func (e *GaugeElem) Consume( // Process the aggregations that are ready for consumption. for _, flushState := range e.toConsume { + if flushState.dirty && flushState.flushed && !flushState.resendEnabled { + flushState := flushState + instrument.EmitAndLogInvariantViolation(e.opts.InstrumentOptions(), func(l *zap.Logger) { + l.Error("reflushing aggregation without resendEnabled", zap.Any("flushState", flushState)) + }) + } flushState.timestamp = xtime.UnixNano(timestampNanosFn(int64(flushState.startAt), resolution)) flushState = e.processValue( flushState, diff --git a/src/aggregator/aggregator/generic_elem.go b/src/aggregator/aggregator/generic_elem.go index 4f41e0d85c..4a4d73ca85 100644 --- a/src/aggregator/aggregator/generic_elem.go +++ b/src/aggregator/aggregator/generic_elem.go @@ -211,6 +211,9 @@ func (e *GenericElem) doAddUnion(timestamp time.Time, mu unaggregated.MetricUnio lockedAgg.aggregation.AddUnion(timestamp, mu) lockedAgg.dirty = true lockedAgg.Unlock() + if retry { + e.metrics.retriedValues.Inc(1) + } return nil } @@ -319,8 +322,8 @@ func (e *GenericElem) expireValuesWithLock( // if this current value is closed and clean it will no longer be flushed. this means it's safe // to remove the previous value since it will no longer be needed for binary transformations. when the // next value is eligible to be expired, this current value will actually be removed. - // if we're currently pointing at the start skip this there is no previous for the start. this ensures - // we always keep at least one value in the map for binary transformations. + // if we're currently pointing at the start skip this because there is no previous for the start. this + // ensures we always keep at least one value in the map for binary transformations. if prevV, ok := e.values[e.minStartTime]; ok && currStart != e.minStartTime { // can't expire flush state until after the flushing, so we save the time to expire later. e.flushStateToExpire = append(e.flushStateToExpire, e.minStartTime) @@ -434,12 +437,6 @@ func (e *GenericElem) Consume( } flushState := e.newFlushStateWithLock(agg) - if flushState.dirty && flushState.flushed && !flushState.resendEnabled { - instrument.EmitAndLogInvariantViolation(e.opts.InstrumentOptions(), func(l *zap.Logger) { - l.Error("reflushing aggregation without resendEnabled", zap.Any("flushState", flushState)) - }) - } - if !flushState.dirty { // there is a race where the value was added to the dirty set, but the writer didn't actually update the // value yet (by marking dirty). add back to the dirty set so it can be processed in the next round once @@ -452,16 +449,21 @@ func (e *GenericElem) Consume( e.values[dirtyTime] = val e.toConsume = append(e.toConsume, flushState) - // potentially consume the nextAgg as well in case we need to cascade an update from a previously flushed - // value. - if flushState.flushed { + // potentially consume the nextAgg as well in case we need to cascade an update to the nextAgg. + // this is necessary for binary transformations that rely on the previous aggregation value for calculating the + // current aggregation value. if the nextAgg was already flushed, it used an outdated value for the previous + // value (this agg). this can only happen when we allow updating previously flushed data (i.e resendEnabled). + if flushState.resendEnabled { nextAgg, ok := e.nextAggWithLock(dirtyTime, targetNanos, isEarlierThanFn) - // add if not already in the dirty set. + // only need to add if not already in the dirty set (since it will be added in a subsequent iteration). if ok && - isEarlierThanFn(int64(nextAgg.startAtNanos), resolution, targetNanos) && // at the end of the dirty times OR the next dirty time does not match. (i == len(dirtyTimes)-1 || dirtyTimes[i+1] != nextAgg.startAtNanos) { - e.toConsume = append(e.toConsume, e.newFlushStateWithLock(nextAgg)) + nextAggFlush := e.newFlushStateWithLock(nextAgg) + // only need to add if it was previously flushed. + if nextAggFlush.flushed { + e.toConsume = append(e.toConsume, nextAggFlush) + } } } } @@ -472,6 +474,12 @@ func (e *GenericElem) Consume( // Process the aggregations that are ready for consumption. for _, flushState := range e.toConsume { + if flushState.dirty && flushState.flushed && !flushState.resendEnabled { + flushState := flushState + instrument.EmitAndLogInvariantViolation(e.opts.InstrumentOptions(), func(l *zap.Logger) { + l.Error("reflushing aggregation without resendEnabled", zap.Any("flushState", flushState)) + }) + } flushState.timestamp = xtime.UnixNano(timestampNanosFn(int64(flushState.startAt), resolution)) flushState = e.processValue( flushState, diff --git a/src/aggregator/aggregator/timer_elem_gen.go b/src/aggregator/aggregator/timer_elem_gen.go index 84fc68715e..95fee6c783 100644 --- a/src/aggregator/aggregator/timer_elem_gen.go +++ b/src/aggregator/aggregator/timer_elem_gen.go @@ -148,6 +148,9 @@ func (e *TimerElem) doAddUnion(timestamp time.Time, mu unaggregated.MetricUnion, lockedAgg.aggregation.AddUnion(timestamp, mu) lockedAgg.dirty = true lockedAgg.Unlock() + if retry { + e.metrics.retriedValues.Inc(1) + } return nil } @@ -256,8 +259,8 @@ func (e *TimerElem) expireValuesWithLock( // if this current value is closed and clean it will no longer be flushed. this means it's safe // to remove the previous value since it will no longer be needed for binary transformations. when the // next value is eligible to be expired, this current value will actually be removed. - // if we're currently pointing at the start skip this there is no previous for the start. this ensures - // we always keep at least one value in the map for binary transformations. + // if we're currently pointing at the start skip this because there is no previous for the start. this + // ensures we always keep at least one value in the map for binary transformations. if prevV, ok := e.values[e.minStartTime]; ok && currStart != e.minStartTime { // can't expire flush state until after the flushing, so we save the time to expire later. e.flushStateToExpire = append(e.flushStateToExpire, e.minStartTime) @@ -371,12 +374,6 @@ func (e *TimerElem) Consume( } flushState := e.newFlushStateWithLock(agg) - if flushState.dirty && flushState.flushed && !flushState.resendEnabled { - instrument.EmitAndLogInvariantViolation(e.opts.InstrumentOptions(), func(l *zap.Logger) { - l.Error("reflushing aggregation without resendEnabled", zap.Any("flushState", flushState)) - }) - } - if !flushState.dirty { // there is a race where the value was added to the dirty set, but the writer didn't actually update the // value yet (by marking dirty). add back to the dirty set so it can be processed in the next round once @@ -389,16 +386,21 @@ func (e *TimerElem) Consume( e.values[dirtyTime] = val e.toConsume = append(e.toConsume, flushState) - // potentially consume the nextAgg as well in case we need to cascade an update from a previously flushed - // value. - if flushState.flushed { + // potentially consume the nextAgg as well in case we need to cascade an update to the nextAgg. + // this is necessary for binary transformations that rely on the previous aggregation value for calculating the + // current aggregation value. if the nextAgg was already flushed, it used an outdated value for the previous + // value (this agg). this can only happen when we allow updating previously flushed data (i.e resendEnabled). + if flushState.resendEnabled { nextAgg, ok := e.nextAggWithLock(dirtyTime, targetNanos, isEarlierThanFn) - // add if not already in the dirty set. + // only need to add if not already in the dirty set (since it will be added in a subsequent iteration). if ok && - isEarlierThanFn(int64(nextAgg.startAtNanos), resolution, targetNanos) && // at the end of the dirty times OR the next dirty time does not match. (i == len(dirtyTimes)-1 || dirtyTimes[i+1] != nextAgg.startAtNanos) { - e.toConsume = append(e.toConsume, e.newFlushStateWithLock(nextAgg)) + nextAggFlush := e.newFlushStateWithLock(nextAgg) + // only need to add if it was previously flushed. + if nextAggFlush.flushed { + e.toConsume = append(e.toConsume, nextAggFlush) + } } } } @@ -409,6 +411,12 @@ func (e *TimerElem) Consume( // Process the aggregations that are ready for consumption. for _, flushState := range e.toConsume { + if flushState.dirty && flushState.flushed && !flushState.resendEnabled { + flushState := flushState + instrument.EmitAndLogInvariantViolation(e.opts.InstrumentOptions(), func(l *zap.Logger) { + l.Error("reflushing aggregation without resendEnabled", zap.Any("flushState", flushState)) + }) + } flushState.timestamp = xtime.UnixNano(timestampNanosFn(int64(flushState.startAt), resolution)) flushState = e.processValue( flushState, From 15e41cf09e146a018c1459589d223b16b656d9ee Mon Sep 17 00:00:00 2001 From: Ryan Hall Date: Wed, 3 Nov 2021 15:45:55 -0700 Subject: [PATCH 6/6] fix test from merge conflict --- src/aggregator/aggregator/elem_test.go | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/src/aggregator/aggregator/elem_test.go b/src/aggregator/aggregator/elem_test.go index 004ae309b2..fbbaf46f11 100644 --- a/src/aggregator/aggregator/elem_test.go +++ b/src/aggregator/aggregator/elem_test.go @@ -2709,10 +2709,10 @@ func TestExpireValues(t *testing.T) { t.Run(test.name, func(t *testing.T) { e, err := NewCounterElem(testCounterElemData, NewElemOptions(opts)) require.NoError(t, err) - require.Equal(t, 0, len(e.toExpire)) + require.Equal(t, 0, len(e.flushStateToExpire)) // Add initial toExpire since expireValues should reset this. - e.toExpire = make([]timedCounter, 10) + e.flushStateToExpire = make([]xtime.UnixNano, 10) // Add test values. for _, v := range test.values { @@ -2723,12 +2723,12 @@ func TestExpireValues(t *testing.T) { } // Expire up to target. - e.expireValuesWithLock(int64(test.targetNanos), standardMetricTimestampNanos, isStandardMetricEarlierThan) + e.expireValuesWithLock(int64(test.targetNanos), isStandardMetricEarlierThan) // Validate toExpire and remaining values. - require.Equal(t, len(test.expectedToExpire), len(e.toExpire)) + require.Equal(t, len(test.expectedToExpire), len(e.flushStateToExpire)) for i, toExpire := range test.expectedToExpire { - require.Equal(t, toExpire, e.toExpire[i].startAtNanos, "missing expire") + require.Equal(t, toExpire, e.flushStateToExpire[i], "missing expire") } require.Equal(t, len(test.expectedValues), len(e.values)) for _, value := range test.expectedValues {