Alerting: refactor state tracker (#33292)

* set processing time

* merge labels and set on response

* use state cache for adding alerts to rules

* minor cleanup

* add support for NoData and Error results

* rename test

* bring in changes from other PRs tha have been merged

* pr feedback

* add integration test

* close state tracker cleanup on context.Done

* fixup test

* rename state tracker

* set EvaluationDuration on Result

* default labels set as constants

* separate cache and state from manager

* use RWMutex in cache
This commit is contained in:
David Parrott 2021-04-23 12:32:25 -07:00 committed by GitHub
parent ca79206498
commit 788bc2a793
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
13 changed files with 564 additions and 475 deletions

View File

@ -50,7 +50,7 @@ type API struct {
AlertingStore store.AlertingStore
DataProxy *datasourceproxy.DatasourceProxyService
Alertmanager Alertmanager
StateTracker *state.StateTracker
StateManager *state.Manager
}
// RegisterAPIEndpoints registers API handlers
@ -69,7 +69,7 @@ func (api *API) RegisterAPIEndpoints() {
api.RegisterPrometheusApiEndpoints(NewForkedProm(
api.DatasourceCache,
NewLotexProm(proxy, logger),
PrometheusSrv{log: logger, stateTracker: api.StateTracker, store: api.RuleStore},
PrometheusSrv{log: logger, manager: api.StateManager, store: api.RuleStore},
))
// Register endpoints for proxing to Cortex Ruler-compatible backends.
api.RegisterRulerApiEndpoints(NewForkedRuler(

View File

@ -19,9 +19,9 @@ import (
)
type PrometheusSrv struct {
log log.Logger
stateTracker *state.StateTracker
store store.RuleStore
log log.Logger
manager *state.Manager
store store.RuleStore
}
func (srv PrometheusSrv) RouteGetAlertStatuses(c *models.ReqContext) response.Response {
@ -33,7 +33,7 @@ func (srv PrometheusSrv) RouteGetAlertStatuses(c *models.ReqContext) response.Re
Alerts: []*apimodels.Alert{},
},
}
for _, alertState := range srv.stateTracker.GetAll() {
for _, alertState := range srv.manager.GetAll() {
startsAt := alertState.StartsAt
alertResponse.Data.Alerts = append(alertResponse.Data.Alerts, &apimodels.Alert{
Labels: map[string]string(alertState.Labels),
@ -88,7 +88,7 @@ func (srv PrometheusSrv) RouteGetRuleStatuses(c *models.ReqContext) response.Res
EvaluationTime: 0, // TODO: see if we are able to pass this along with evaluation results
}
stateMap := srv.stateTracker.GetStatesByRuleUID()
stateMap := srv.manager.GetStatesByRuleUID()
for _, rule := range alertRuleQuery.Result {
alertingRule := apimodels.AlertingRule{
State: "inactive",

View File

@ -59,9 +59,10 @@ type Results []Result
// Result contains the evaluated State of an alert instance
// identified by its labels.
type Result struct {
Instance data.Labels
State State // Enum
EvaluatedAt time.Time
Instance data.Labels
State State // Enum
EvaluatedAt time.Time
EvaluationDuration time.Duration
}
// State is an enum of the evaluation State for an alert instance.
@ -213,8 +214,9 @@ func evaluateExecutionResult(results *ExecutionResults, ts time.Time) (Results,
}
r := Result{
Instance: f.Fields[0].Labels,
EvaluatedAt: ts,
Instance: f.Fields[0].Labels,
EvaluatedAt: ts,
EvaluationDuration: time.Since(ts),
}
switch {

View File

@ -43,6 +43,11 @@ const (
KeepLastStateErrState ExecutionErrorState = "KeepLastState"
)
const (
UIDLabel = "__alert_rule_uid__"
NamespaceUIDLabel = "__alert_rule_namespace_uid__"
)
// AlertRule is the model for alert rules in unified alerting.
type AlertRule struct {
ID int64 `xorm:"pk autoincr 'id'"`

View File

@ -47,7 +47,7 @@ type AlertNG struct {
DataProxy *datasourceproxy.DatasourceProxyService `inject:""`
Log log.Logger
schedule schedule.ScheduleService
stateTracker *state.StateTracker
stateManager *state.Manager
}
func init() {
@ -57,7 +57,7 @@ func init() {
// Init initializes the AlertingService.
func (ng *AlertNG) Init() error {
ng.Log = log.New("ngalert")
ng.stateTracker = state.NewStateTracker(ng.Log)
ng.stateManager = state.NewManager(ng.Log)
baseInterval := baseIntervalSeconds * time.Second
store := store.DBstore{BaseInterval: baseInterval, DefaultIntervalSeconds: defaultIntervalSeconds, SQLStore: ng.SQLStore}
@ -85,7 +85,7 @@ func (ng *AlertNG) Init() error {
RuleStore: store,
AlertingStore: store,
Alertmanager: ng.Alertmanager,
StateTracker: ng.stateTracker,
StateManager: ng.stateManager,
}
api.RegisterAPIEndpoints()
@ -95,8 +95,8 @@ func (ng *AlertNG) Init() error {
// Run starts the scheduler
func (ng *AlertNG) Run(ctx context.Context) error {
ng.Log.Debug("ngalert starting")
ng.schedule.WarmStateCache(ng.stateTracker)
return ng.schedule.Ticker(ctx, ng.stateTracker)
ng.schedule.WarmStateCache(ng.stateManager)
return ng.schedule.Ticker(ctx, ng.stateManager)
}
// IsDisabled returns true if the alerting service is disable for this instance.

View File

@ -9,7 +9,7 @@ import (
"github.com/grafana/grafana/pkg/services/ngalert/state"
)
func FromAlertStateToPostableAlerts(firingStates []state.AlertState) apimodels.PostableAlerts {
func FromAlertStateToPostableAlerts(firingStates []*state.State) apimodels.PostableAlerts {
alerts := apimodels.PostableAlerts{PostableAlerts: make([]models.PostableAlert, 0, len(firingStates))}
for _, alertState := range firingStates {

View File

@ -24,10 +24,10 @@ var timeNow = time.Now
// ScheduleService handles scheduling
type ScheduleService interface {
Ticker(context.Context, *state.StateTracker) error
Ticker(context.Context, *state.Manager) error
Pause() error
Unpause() error
WarmStateCache(*state.StateTracker)
WarmStateCache(*state.Manager)
// the following are used by tests only used for tests
evalApplied(models.AlertRuleKey, time.Time)
@ -35,7 +35,7 @@ type ScheduleService interface {
overrideCfg(cfg SchedulerCfg)
}
func (sch *schedule) ruleRoutine(grafanaCtx context.Context, key models.AlertRuleKey, evalCh <-chan *evalContext, stopCh <-chan struct{}, stateTracker *state.StateTracker) error {
func (sch *schedule) ruleRoutine(grafanaCtx context.Context, key models.AlertRuleKey, evalCh <-chan *evalContext, stopCh <-chan struct{}, stateManager *state.Manager) error {
sch.log.Debug("alert rule routine started", "key", key)
evalRunning := false
@ -78,7 +78,7 @@ func (sch *schedule) ruleRoutine(grafanaCtx context.Context, key models.AlertRul
return err
}
processedStates := stateTracker.ProcessEvalResults(alertRule, results, end.Sub(start))
processedStates := stateManager.ProcessEvalResults(alertRule, results)
sch.saveAlertStates(processedStates)
alerts := FromAlertStateToPostableAlerts(processedStates)
sch.log.Debug("sending alerts to notifier", "count", len(alerts.PostableAlerts), "alerts", alerts.PostableAlerts)
@ -232,7 +232,7 @@ func (sch *schedule) Unpause() error {
return nil
}
func (sch *schedule) Ticker(grafanaCtx context.Context, stateTracker *state.StateTracker) error {
func (sch *schedule) Ticker(grafanaCtx context.Context, stateManager *state.Manager) error {
dispatcherGroup, ctx := errgroup.WithContext(grafanaCtx)
for {
select {
@ -261,7 +261,7 @@ func (sch *schedule) Ticker(grafanaCtx context.Context, stateTracker *state.Stat
if newRoutine && !invalidInterval {
dispatcherGroup.Go(func() error {
return sch.ruleRoutine(ctx, key, ruleInfo.evalCh, ruleInfo.stopCh, stateTracker)
return sch.ruleRoutine(ctx, key, ruleInfo.evalCh, ruleInfo.stopCh, stateManager)
})
}
@ -306,8 +306,8 @@ func (sch *schedule) Ticker(grafanaCtx context.Context, stateTracker *state.Stat
}
case <-grafanaCtx.Done():
err := dispatcherGroup.Wait()
sch.saveAlertStates(stateTracker.GetAll())
stateTracker.Close()
sch.saveAlertStates(stateManager.GetAll())
stateManager.Close()
return err
}
}
@ -317,7 +317,7 @@ func (sch *schedule) sendAlerts(alerts apimodels.PostableAlerts) error {
return sch.notifier.PutAlerts(alerts)
}
func (sch *schedule) saveAlertStates(states []state.AlertState) {
func (sch *schedule) saveAlertStates(states []*state.State) {
sch.log.Debug("saving alert states", "count", len(states))
for _, s := range states {
cmd := models.SaveAlertInstanceCommand{
@ -336,7 +336,7 @@ func (sch *schedule) saveAlertStates(states []state.AlertState) {
}
}
func (sch *schedule) WarmStateCache(st *state.StateTracker) {
func (sch *schedule) WarmStateCache(st *state.Manager) {
sch.log.Info("warming cache for startup")
st.ResetCache()
@ -345,7 +345,7 @@ func (sch *schedule) WarmStateCache(st *state.StateTracker) {
sch.log.Error("unable to fetch orgIds", "msg", err.Error())
}
var states []state.AlertState
var states []*state.State
for _, orgIdResult := range orgIdsCmd.Result {
cmd := models.ListAlertInstancesQuery{
DefinitionOrgID: orgIdResult.DefinitionOrgID,
@ -355,13 +355,13 @@ func (sch *schedule) WarmStateCache(st *state.StateTracker) {
}
for _, entry := range cmd.Result {
lbs := map[string]string(entry.Labels)
stateForEntry := state.AlertState{
stateForEntry := &state.State{
AlertRuleUID: entry.DefinitionUID,
OrgID: entry.DefinitionOrgID,
CacheId: fmt.Sprintf("%s %s", entry.DefinitionUID, lbs),
Labels: lbs,
State: translateInstanceState(entry.CurrentState),
Results: []state.StateEvaluation{},
Results: []state.Evaluation{},
StartsAt: entry.CurrentStateSince,
EndsAt: entry.CurrentStateEnd,
LastEvaluationTime: entry.LastEvalTime,

View File

@ -0,0 +1,135 @@
package state
import (
"fmt"
"sync"
"github.com/grafana/grafana-plugin-sdk-go/data"
"github.com/grafana/grafana/pkg/services/ngalert/eval"
ngModels "github.com/grafana/grafana/pkg/services/ngalert/models"
prometheusModel "github.com/prometheus/common/model"
)
type cache struct {
states map[string]*State
mtxStates sync.RWMutex
}
func newCache() *cache {
return &cache{
states: make(map[string]*State),
}
}
func (c *cache) getOrCreate(alertRule *ngModels.AlertRule, result eval.Result) *State {
c.mtxStates.Lock()
defer c.mtxStates.Unlock()
// if duplicate labels exist, alertRule label will take precedence
lbs := mergeLabels(alertRule.Labels, result.Instance)
lbs[ngModels.UIDLabel] = alertRule.UID
lbs[ngModels.NamespaceUIDLabel] = alertRule.NamespaceUID
lbs[prometheusModel.AlertNameLabel] = alertRule.Title
id := fmt.Sprintf("%s", map[string]string(lbs))
if state, ok := c.states[id]; ok {
return state
}
annotations := map[string]string{}
if len(alertRule.Annotations) > 0 {
annotations = alertRule.Annotations
}
// If the first result we get is alerting, set StartsAt to EvaluatedAt because we
// do not have data for determining StartsAt otherwise
newState := &State{
AlertRuleUID: alertRule.UID,
OrgID: alertRule.OrgID,
CacheId: id,
Labels: lbs,
State: result.State,
Annotations: annotations,
EvaluationDuration: result.EvaluationDuration,
}
if result.State == eval.Alerting {
newState.StartsAt = result.EvaluatedAt
}
c.states[id] = newState
return newState
}
func (c *cache) set(entry *State) {
c.mtxStates.Lock()
defer c.mtxStates.Unlock()
c.states[entry.CacheId] = entry
}
func (c *cache) get(id string) (*State, error) {
c.mtxStates.Lock()
defer c.mtxStates.Unlock()
if state, ok := c.states[id]; ok {
return state, nil
}
return nil, fmt.Errorf("no entry for id: %s", id)
}
func (c *cache) getAll() []*State {
var states []*State
c.mtxStates.Lock()
defer c.mtxStates.Unlock()
for _, v := range c.states {
states = append(states, v)
}
return states
}
func (c *cache) getStatesByRuleUID() map[string][]*State {
ruleMap := make(map[string][]*State)
c.mtxStates.Lock()
defer c.mtxStates.Unlock()
for _, state := range c.states {
if ruleStates, ok := ruleMap[state.AlertRuleUID]; ok {
ruleStates = append(ruleStates, state)
ruleMap[state.AlertRuleUID] = ruleStates
} else {
ruleStates := []*State{state}
ruleMap[state.AlertRuleUID] = ruleStates
}
}
return ruleMap
}
func (c *cache) reset() {
c.mtxStates.Lock()
defer c.mtxStates.Unlock()
c.states = make(map[string]*State)
}
func (c *cache) trim() {
c.mtxStates.Lock()
defer c.mtxStates.Unlock()
for _, v := range c.states {
if len(v.Results) > 100 {
newResults := make([]Evaluation, 100)
copy(newResults, v.Results[100:])
v.Results = newResults
c.set(v)
}
}
}
// if duplicate labels exist, keep the value from the first set
func mergeLabels(a, b data.Labels) data.Labels {
newLbs := data.Labels{}
for k, v := range a {
newLbs[k] = v
}
for k, v := range b {
if _, ok := newLbs[k]; !ok {
newLbs[k] = v
}
}
return newLbs
}

View File

@ -0,0 +1,117 @@
package state
import (
"time"
"github.com/grafana/grafana/pkg/infra/log"
"github.com/grafana/grafana/pkg/services/ngalert/eval"
ngModels "github.com/grafana/grafana/pkg/services/ngalert/models"
)
type Manager struct {
cache *cache
quit chan struct{}
Log log.Logger
}
func NewManager(logger log.Logger) *Manager {
manager := &Manager{
cache: newCache(),
quit: make(chan struct{}),
Log: logger,
}
go manager.cleanUp()
return manager
}
func (st *Manager) Close() {
st.quit <- struct{}{}
}
func (st *Manager) getOrCreate(alertRule *ngModels.AlertRule, result eval.Result) *State {
return st.cache.getOrCreate(alertRule, result)
}
func (st *Manager) set(entry *State) {
st.cache.set(entry)
}
func (st *Manager) Get(id string) (*State, error) {
return st.cache.get(id)
}
//Used to ensure a clean cache on startup
func (st *Manager) ResetCache() {
st.cache.reset()
}
func (st *Manager) ProcessEvalResults(alertRule *ngModels.AlertRule, results eval.Results) []*State {
st.Log.Debug("state manager processing evaluation results", "uid", alertRule.UID, "resultCount", len(results))
var states []*State
for _, result := range results {
s := st.setNextState(alertRule, result)
states = append(states, s)
}
st.Log.Debug("returning changed states to scheduler", "count", len(states))
return states
}
//TODO: When calculating if an alert should not be firing anymore, we should take into account the re-send delay if any. We don't want to send every firing alert every time, we should have a fixed delay across all alerts to avoid saturating the notification system
//Set the current state based on evaluation results
func (st *Manager) setNextState(alertRule *ngModels.AlertRule, result eval.Result) *State {
currentState := st.getOrCreate(alertRule, result)
currentState.LastEvaluationTime = result.EvaluatedAt
currentState.EvaluationDuration = result.EvaluationDuration
currentState.Results = append(currentState.Results, Evaluation{
EvaluationTime: result.EvaluatedAt,
EvaluationState: result.State,
})
st.Log.Debug("setting alert state", "uid", alertRule.UID)
switch result.State {
case eval.Normal:
currentState = resultNormal(currentState, result)
case eval.Alerting:
currentState = currentState.resultAlerting(alertRule, result)
case eval.Error:
currentState = currentState.resultError(alertRule, result)
case eval.NoData:
currentState = currentState.resultNoData(alertRule, result)
case eval.Pending: // we do not emit results with this state
}
st.set(currentState)
return currentState
}
func (st *Manager) GetAll() []*State {
return st.cache.getAll()
}
func (st *Manager) GetStatesByRuleUID() map[string][]*State {
return st.cache.getStatesByRuleUID()
}
func (st *Manager) cleanUp() {
ticker := time.NewTicker(time.Duration(60) * time.Minute)
st.Log.Debug("starting cleanup process", "intervalMinutes", 60)
for {
select {
case <-ticker.C:
st.Log.Info("trimming alert state cache", "now", time.Now())
st.cache.trim()
case <-st.quit:
st.Log.Debug("stopping cleanup process", "now", time.Now())
ticker.Stop()
return
}
}
}
func (st *Manager) Put(states []*State) {
for _, s := range states {
st.set(s)
}
}

View File

@ -0,0 +1,130 @@
package state
import (
"time"
"github.com/grafana/grafana-plugin-sdk-go/data"
"github.com/grafana/grafana/pkg/services/ngalert/eval"
ngModels "github.com/grafana/grafana/pkg/services/ngalert/models"
)
type State struct {
AlertRuleUID string
OrgID int64
CacheId string
Labels data.Labels
State eval.State
Results []Evaluation
StartsAt time.Time
EndsAt time.Time
LastEvaluationTime time.Time
EvaluationDuration time.Duration
Annotations map[string]string
}
type Evaluation struct {
EvaluationTime time.Time
EvaluationState eval.State
}
func resultNormal(alertState *State, result eval.Result) *State {
newState := alertState
if alertState.State != eval.Normal {
newState.EndsAt = result.EvaluatedAt
}
newState.State = eval.Normal
return newState
}
func (a *State) resultAlerting(alertRule *ngModels.AlertRule, result eval.Result) *State {
switch a.State {
case eval.Alerting:
if !(alertRule.For > 0) {
// If there is not For set, we will set EndsAt to be twice the evaluation interval
// to avoid flapping with every evaluation
a.EndsAt = result.EvaluatedAt.Add(time.Duration(alertRule.IntervalSeconds*2) * time.Second)
return a
}
a.EndsAt = result.EvaluatedAt.Add(alertRule.For)
case eval.Pending:
if result.EvaluatedAt.Sub(a.StartsAt) > alertRule.For {
a.State = eval.Alerting
a.StartsAt = result.EvaluatedAt
a.EndsAt = result.EvaluatedAt.Add(alertRule.For)
a.Annotations["alerting_at"] = result.EvaluatedAt.String()
}
default:
a.StartsAt = result.EvaluatedAt
if !(alertRule.For > 0) {
a.EndsAt = result.EvaluatedAt.Add(time.Duration(alertRule.IntervalSeconds*2) * time.Second)
a.State = eval.Alerting
a.Annotations["alerting_at"] = result.EvaluatedAt.String()
} else {
a.EndsAt = result.EvaluatedAt.Add(alertRule.For)
if result.EvaluatedAt.Sub(a.StartsAt) > alertRule.For {
a.State = eval.Alerting
a.Annotations["alerting_at"] = result.EvaluatedAt.String()
} else {
a.State = eval.Pending
}
}
}
return a
}
func (a *State) resultError(alertRule *ngModels.AlertRule, result eval.Result) *State {
if a.StartsAt.IsZero() {
a.StartsAt = result.EvaluatedAt
}
if !(alertRule.For > 0) {
a.EndsAt = result.EvaluatedAt.Add(time.Duration(alertRule.IntervalSeconds*2) * time.Second)
} else {
a.EndsAt = result.EvaluatedAt.Add(alertRule.For)
}
if a.State != eval.Error {
a.Annotations["last_error"] = result.EvaluatedAt.String()
}
switch alertRule.ExecErrState {
case ngModels.AlertingErrState:
a.State = eval.Alerting
case ngModels.KeepLastStateErrState:
}
return a
}
func (a *State) resultNoData(alertRule *ngModels.AlertRule, result eval.Result) *State {
if a.StartsAt.IsZero() {
a.StartsAt = result.EvaluatedAt
}
if !(alertRule.For > 0) {
a.EndsAt = result.EvaluatedAt.Add(time.Duration(alertRule.IntervalSeconds*2) * time.Second)
} else {
a.EndsAt = result.EvaluatedAt.Add(alertRule.For)
}
if a.State != eval.NoData {
a.Annotations["no_data"] = result.EvaluatedAt.String()
}
switch alertRule.NoDataState {
case ngModels.Alerting:
a.State = eval.Alerting
case ngModels.NoData:
a.State = eval.NoData
case ngModels.KeepLastState:
case ngModels.OK:
a.State = eval.Normal
}
return a
}
func (a *State) Equals(b *State) bool {
return a.AlertRuleUID == b.AlertRuleUID &&
a.OrgID == b.OrgID &&
a.CacheId == b.CacheId &&
a.Labels.String() == b.Labels.String() &&
a.State.String() == b.State.String() &&
a.StartsAt == b.StartsAt &&
a.EndsAt == b.EndsAt &&
a.LastEvaluationTime == b.LastEvaluationTime
}

View File

@ -1,329 +0,0 @@
package state
import (
"fmt"
"sync"
"time"
prometheusModel "github.com/prometheus/common/model"
"github.com/grafana/grafana/pkg/infra/log"
"github.com/grafana/grafana-plugin-sdk-go/data"
"github.com/grafana/grafana/pkg/services/ngalert/eval"
ngModels "github.com/grafana/grafana/pkg/services/ngalert/models"
)
type AlertState struct {
AlertRuleUID string
OrgID int64
CacheId string
Labels data.Labels
State eval.State
Results []StateEvaluation
StartsAt time.Time
EndsAt time.Time
LastEvaluationTime time.Time
EvaluationDuration time.Duration
Annotations map[string]string
}
type StateEvaluation struct {
EvaluationTime time.Time
EvaluationState eval.State
}
type cache struct {
states map[string]AlertState
mtxStates sync.Mutex
}
type StateTracker struct {
cache cache
quit chan struct{}
Log log.Logger
}
func NewStateTracker(logger log.Logger) *StateTracker {
tracker := &StateTracker{
cache: cache{
states: make(map[string]AlertState),
},
quit: make(chan struct{}),
Log: logger,
}
go tracker.cleanUp()
return tracker
}
func (st *StateTracker) Close() {
st.quit <- struct{}{}
}
func (st *StateTracker) getOrCreate(alertRule *ngModels.AlertRule, result eval.Result, evaluationDuration time.Duration) AlertState {
st.cache.mtxStates.Lock()
defer st.cache.mtxStates.Unlock()
// if duplicate labels exist, alertRule label will take precedence
lbs := mergeLabels(alertRule.Labels, result.Instance)
lbs["__alert_rule_uid__"] = alertRule.UID
lbs["__alert_rule_namespace_uid__"] = alertRule.NamespaceUID
lbs[prometheusModel.AlertNameLabel] = alertRule.Title
id := fmt.Sprintf("%s", map[string]string(lbs))
if state, ok := st.cache.states[id]; ok {
return state
}
annotations := map[string]string{}
if len(alertRule.Annotations) > 0 {
annotations = alertRule.Annotations
}
// If the first result we get is alerting, set StartsAt to EvaluatedAt because we
// do not have data for determining StartsAt otherwise
st.Log.Debug("adding new alert state cache entry", "cacheId", id, "state", result.State.String(), "evaluatedAt", result.EvaluatedAt.String())
newState := AlertState{
AlertRuleUID: alertRule.UID,
OrgID: alertRule.OrgID,
CacheId: id,
Labels: lbs,
State: result.State,
Annotations: annotations,
EvaluationDuration: evaluationDuration,
}
if result.State == eval.Alerting {
newState.StartsAt = result.EvaluatedAt
}
st.cache.states[id] = newState
return newState
}
func (st *StateTracker) set(entry AlertState) {
st.cache.mtxStates.Lock()
defer st.cache.mtxStates.Unlock()
st.cache.states[entry.CacheId] = entry
}
func (st *StateTracker) Get(id string) AlertState {
st.cache.mtxStates.Lock()
defer st.cache.mtxStates.Unlock()
return st.cache.states[id]
}
//Used to ensure a clean cache on startup
func (st *StateTracker) ResetCache() {
st.cache.mtxStates.Lock()
defer st.cache.mtxStates.Unlock()
st.cache.states = make(map[string]AlertState)
}
func (st *StateTracker) ProcessEvalResults(alertRule *ngModels.AlertRule, results eval.Results, evaluationDuration time.Duration) []AlertState {
st.Log.Info("state tracker processing evaluation results", "uid", alertRule.UID, "resultCount", len(results))
var states []AlertState
for _, result := range results {
s := st.setNextState(alertRule, result, evaluationDuration)
states = append(states, s)
}
st.Log.Debug("returning changed states to scheduler", "count", len(states))
return states
}
//TODO: When calculating if an alert should not be firing anymore, we should take three things into account:
// 1. The re-send the delay if any, we don't want to send every firing alert every time, we should have a fixed delay across all alerts to avoid saturating the notification system
//Set the current state based on evaluation results
func (st *StateTracker) setNextState(alertRule *ngModels.AlertRule, result eval.Result, evaluationDuration time.Duration) AlertState {
currentState := st.getOrCreate(alertRule, result, evaluationDuration)
currentState.LastEvaluationTime = result.EvaluatedAt
currentState.EvaluationDuration = evaluationDuration
currentState.Results = append(currentState.Results, StateEvaluation{
EvaluationTime: result.EvaluatedAt,
EvaluationState: result.State,
})
st.Log.Debug("setting alert state", "uid", alertRule.UID)
switch result.State {
case eval.Normal:
currentState = resultNormal(currentState, result)
case eval.Alerting:
currentState = currentState.resultAlerting(alertRule, result)
case eval.Error:
currentState = currentState.resultError(alertRule, result)
case eval.NoData:
currentState = currentState.resultNoData(alertRule, result)
case eval.Pending: // we do not emit results with this state
}
st.set(currentState)
return currentState
}
func resultNormal(alertState AlertState, result eval.Result) AlertState {
newState := alertState
if alertState.State != eval.Normal {
newState.EndsAt = result.EvaluatedAt
}
newState.State = eval.Normal
return newState
}
func (a AlertState) resultAlerting(alertRule *ngModels.AlertRule, result eval.Result) AlertState {
switch a.State {
case eval.Alerting:
if !(alertRule.For > 0) {
// If there is not For set, we will set EndsAt to be twice the evaluation interval
// to avoid flapping with every evaluation
a.EndsAt = result.EvaluatedAt.Add(time.Duration(alertRule.IntervalSeconds*2) * time.Second)
return a
}
a.EndsAt = result.EvaluatedAt.Add(alertRule.For)
case eval.Pending:
if result.EvaluatedAt.Sub(a.StartsAt) > alertRule.For {
a.State = eval.Alerting
a.StartsAt = result.EvaluatedAt
a.EndsAt = result.EvaluatedAt.Add(alertRule.For)
}
default:
a.StartsAt = result.EvaluatedAt
if !(alertRule.For > 0) {
a.EndsAt = result.EvaluatedAt.Add(time.Duration(alertRule.IntervalSeconds*2) * time.Second)
a.State = eval.Alerting
} else {
a.EndsAt = result.EvaluatedAt.Add(alertRule.For)
if result.EvaluatedAt.Sub(a.StartsAt) > alertRule.For {
a.State = eval.Alerting
} else {
a.State = eval.Pending
}
}
}
return a
}
func (a AlertState) resultError(alertRule *ngModels.AlertRule, result eval.Result) AlertState {
if a.StartsAt.IsZero() {
a.StartsAt = result.EvaluatedAt
}
if !(alertRule.For > 0) {
a.EndsAt = result.EvaluatedAt.Add(time.Duration(alertRule.IntervalSeconds*2) * time.Second)
} else {
a.EndsAt = result.EvaluatedAt.Add(alertRule.For)
}
switch alertRule.ExecErrState {
case ngModels.AlertingErrState:
a.State = eval.Alerting
case ngModels.KeepLastStateErrState:
}
return a
}
func (a AlertState) resultNoData(alertRule *ngModels.AlertRule, result eval.Result) AlertState {
if a.StartsAt.IsZero() {
a.StartsAt = result.EvaluatedAt
}
if !(alertRule.For > 0) {
a.EndsAt = result.EvaluatedAt.Add(time.Duration(alertRule.IntervalSeconds*2) * time.Second)
} else {
a.EndsAt = result.EvaluatedAt.Add(alertRule.For)
}
switch alertRule.NoDataState {
case ngModels.Alerting:
a.State = eval.Alerting
case ngModels.NoData:
a.State = eval.NoData
case ngModels.KeepLastState:
case ngModels.OK:
a.State = eval.Normal
}
return a
}
func (st *StateTracker) GetAll() []AlertState {
var states []AlertState
st.cache.mtxStates.Lock()
defer st.cache.mtxStates.Unlock()
for _, v := range st.cache.states {
states = append(states, v)
}
return states
}
func (st *StateTracker) GetStatesByRuleUID() map[string][]AlertState {
ruleMap := make(map[string][]AlertState)
st.cache.mtxStates.Lock()
defer st.cache.mtxStates.Unlock()
for _, state := range st.cache.states {
if ruleStates, ok := ruleMap[state.AlertRuleUID]; ok {
ruleStates = append(ruleStates, state)
ruleMap[state.AlertRuleUID] = ruleStates
} else {
ruleStates := []AlertState{state}
ruleMap[state.AlertRuleUID] = ruleStates
}
}
return ruleMap
}
func (st *StateTracker) cleanUp() {
ticker := time.NewTicker(time.Duration(60) * time.Minute)
st.Log.Debug("starting cleanup process", "intervalMinutes", 60)
for {
select {
case <-ticker.C:
st.trim()
case <-st.quit:
st.Log.Debug("stopping cleanup process", "now", time.Now())
ticker.Stop()
return
}
}
}
func (st *StateTracker) trim() {
st.Log.Info("trimming alert state cache", "now", time.Now())
st.cache.mtxStates.Lock()
defer st.cache.mtxStates.Unlock()
for _, v := range st.cache.states {
if len(v.Results) > 100 {
st.Log.Debug("trimming result set", "cacheId", v.CacheId, "count", len(v.Results)-100)
newResults := make([]StateEvaluation, 100)
copy(newResults, v.Results[100:])
v.Results = newResults
st.set(v)
}
}
}
func (a AlertState) Equals(b AlertState) bool {
return a.AlertRuleUID == b.AlertRuleUID &&
a.OrgID == b.OrgID &&
a.CacheId == b.CacheId &&
a.Labels.String() == b.Labels.String() &&
a.State.String() == b.State.String() &&
a.StartsAt == b.StartsAt &&
a.EndsAt == b.EndsAt &&
a.LastEvaluationTime == b.LastEvaluationTime
}
func (st *StateTracker) Put(states []AlertState) {
for _, s := range states {
st.set(s)
}
}
// if duplicate labels exist, keep the value from the first set
func mergeLabels(a, b data.Labels) data.Labels {
newLbs := data.Labels{}
for k, v := range a {
newLbs[k] = v
}
for k, v := range b {
if _, ok := newLbs[k]; !ok {
newLbs[k] = v
}
}
return newLbs
}

View File

@ -4,6 +4,8 @@ import (
"testing"
"time"
"github.com/stretchr/testify/require"
"github.com/grafana/grafana/pkg/services/ngalert/state"
"github.com/grafana/grafana/pkg/infra/log"
@ -25,7 +27,7 @@ func TestProcessEvalResults(t *testing.T) {
desc string
alertRule *models.AlertRule
evalResults []eval.Results
expectedStates map[string]state.AlertState
expectedStates map[string]*state.State
}{
{
desc: "a cache entry is correctly created",
@ -41,13 +43,14 @@ func TestProcessEvalResults(t *testing.T) {
evalResults: []eval.Results{
{
eval.Result{
Instance: data.Labels{"instance_label": "test"},
State: eval.Normal,
EvaluatedAt: evaluationTime,
Instance: data.Labels{"instance_label": "test"},
State: eval.Normal,
EvaluatedAt: evaluationTime,
EvaluationDuration: evaluationDuration,
},
},
},
expectedStates: map[string]state.AlertState{
expectedStates: map[string]*state.State{
"map[__alert_rule_namespace_uid__:test_namespace_uid __alert_rule_uid__:test_alert_rule_uid alertname:test_title instance_label:test label:test]": {
AlertRuleUID: "test_alert_rule_uid",
OrgID: 1,
@ -60,7 +63,7 @@ func TestProcessEvalResults(t *testing.T) {
"instance_label": "test",
},
State: eval.Normal,
Results: []state.StateEvaluation{
Results: []state.Evaluation{
{
EvaluationTime: evaluationTime,
EvaluationState: eval.Normal,
@ -86,18 +89,20 @@ func TestProcessEvalResults(t *testing.T) {
evalResults: []eval.Results{
{
eval.Result{
Instance: data.Labels{"instance_label_1": "test"},
State: eval.Normal,
EvaluatedAt: evaluationTime,
Instance: data.Labels{"instance_label_1": "test"},
State: eval.Normal,
EvaluatedAt: evaluationTime,
EvaluationDuration: evaluationDuration,
},
eval.Result{
Instance: data.Labels{"instance_label_2": "test"},
State: eval.Alerting,
EvaluatedAt: evaluationTime,
Instance: data.Labels{"instance_label_2": "test"},
State: eval.Alerting,
EvaluatedAt: evaluationTime,
EvaluationDuration: evaluationDuration,
},
},
},
expectedStates: map[string]state.AlertState{
expectedStates: map[string]*state.State{
"map[__alert_rule_namespace_uid__:test_namespace_uid __alert_rule_uid__:test_alert_rule_uid alertname:test_title instance_label_1:test label:test]": {
AlertRuleUID: "test_alert_rule_uid",
OrgID: 1,
@ -110,7 +115,7 @@ func TestProcessEvalResults(t *testing.T) {
"instance_label_1": "test",
},
State: eval.Normal,
Results: []state.StateEvaluation{
Results: []state.Evaluation{
{
EvaluationTime: evaluationTime,
EvaluationState: eval.Normal,
@ -132,7 +137,7 @@ func TestProcessEvalResults(t *testing.T) {
"instance_label_2": "test",
},
State: eval.Alerting,
Results: []state.StateEvaluation{
Results: []state.Evaluation{
{
EvaluationTime: evaluationTime,
EvaluationState: eval.Alerting,
@ -160,20 +165,22 @@ func TestProcessEvalResults(t *testing.T) {
evalResults: []eval.Results{
{
eval.Result{
Instance: data.Labels{"instance_label": "test"},
State: eval.Normal,
EvaluatedAt: evaluationTime,
Instance: data.Labels{"instance_label": "test"},
State: eval.Normal,
EvaluatedAt: evaluationTime,
EvaluationDuration: evaluationDuration,
},
},
{
eval.Result{
Instance: data.Labels{"instance_label": "test"},
State: eval.Normal,
EvaluatedAt: evaluationTime.Add(1 * time.Minute),
Instance: data.Labels{"instance_label": "test"},
State: eval.Normal,
EvaluatedAt: evaluationTime.Add(1 * time.Minute),
EvaluationDuration: evaluationDuration,
},
},
},
expectedStates: map[string]state.AlertState{
expectedStates: map[string]*state.State{
"map[__alert_rule_namespace_uid__:test_namespace_uid __alert_rule_uid__:test_alert_rule_uid_1 alertname:test_title instance_label:test label:test]": {
AlertRuleUID: "test_alert_rule_uid_1",
OrgID: 1,
@ -186,7 +193,7 @@ func TestProcessEvalResults(t *testing.T) {
"instance_label": "test",
},
State: eval.Normal,
Results: []state.StateEvaluation{
Results: []state.Evaluation{
{
EvaluationTime: evaluationTime,
EvaluationState: eval.Normal,
@ -216,20 +223,22 @@ func TestProcessEvalResults(t *testing.T) {
evalResults: []eval.Results{
{
eval.Result{
Instance: data.Labels{"instance_label": "test"},
State: eval.Normal,
EvaluatedAt: evaluationTime,
Instance: data.Labels{"instance_label": "test"},
State: eval.Normal,
EvaluatedAt: evaluationTime,
EvaluationDuration: evaluationDuration,
},
},
{
eval.Result{
Instance: data.Labels{"instance_label": "test"},
State: eval.Alerting,
EvaluatedAt: evaluationTime.Add(1 * time.Minute),
Instance: data.Labels{"instance_label": "test"},
State: eval.Alerting,
EvaluatedAt: evaluationTime.Add(1 * time.Minute),
EvaluationDuration: evaluationDuration,
},
},
},
expectedStates: map[string]state.AlertState{
expectedStates: map[string]*state.State{
"map[__alert_rule_namespace_uid__:test_namespace_uid __alert_rule_uid__:test_alert_rule_uid_2 alertname:test_title instance_label:test label:test]": {
AlertRuleUID: "test_alert_rule_uid_2",
OrgID: 1,
@ -242,7 +251,7 @@ func TestProcessEvalResults(t *testing.T) {
"instance_label": "test",
},
State: eval.Alerting,
Results: []state.StateEvaluation{
Results: []state.Evaluation{
{
EvaluationTime: evaluationTime,
EvaluationState: eval.Normal,
@ -256,7 +265,7 @@ func TestProcessEvalResults(t *testing.T) {
EndsAt: evaluationTime.Add(1 * time.Minute).Add(time.Duration(20) * time.Second),
LastEvaluationTime: evaluationTime.Add(1 * time.Minute),
EvaluationDuration: evaluationDuration,
Annotations: map[string]string{"annotation": "test"},
Annotations: map[string]string{"annotation": "test", "alerting_at": "2021-03-25 00:01:00 +0000 UTC"},
},
},
},
@ -275,27 +284,30 @@ func TestProcessEvalResults(t *testing.T) {
evalResults: []eval.Results{
{
eval.Result{
Instance: data.Labels{"instance_label": "test"},
State: eval.Normal,
EvaluatedAt: evaluationTime,
Instance: data.Labels{"instance_label": "test"},
State: eval.Normal,
EvaluatedAt: evaluationTime,
EvaluationDuration: evaluationDuration,
},
},
{
eval.Result{
Instance: data.Labels{"instance_label": "test"},
State: eval.Alerting,
EvaluatedAt: evaluationTime.Add(10 * time.Second),
Instance: data.Labels{"instance_label": "test"},
State: eval.Alerting,
EvaluatedAt: evaluationTime.Add(10 * time.Second),
EvaluationDuration: evaluationDuration,
},
},
{
eval.Result{
Instance: data.Labels{"instance_label": "test"},
State: eval.Alerting,
EvaluatedAt: evaluationTime.Add(80 * time.Second),
Instance: data.Labels{"instance_label": "test"},
State: eval.Alerting,
EvaluatedAt: evaluationTime.Add(80 * time.Second),
EvaluationDuration: evaluationDuration,
},
},
},
expectedStates: map[string]state.AlertState{
expectedStates: map[string]*state.State{
"map[__alert_rule_namespace_uid__:test_namespace_uid __alert_rule_uid__:test_alert_rule_uid_2 alertname:test_title instance_label:test label:test]": {
AlertRuleUID: "test_alert_rule_uid_2",
OrgID: 1,
@ -308,7 +320,7 @@ func TestProcessEvalResults(t *testing.T) {
"instance_label": "test",
},
State: eval.Alerting,
Results: []state.StateEvaluation{
Results: []state.Evaluation{
{
EvaluationTime: evaluationTime,
EvaluationState: eval.Normal,
@ -326,7 +338,7 @@ func TestProcessEvalResults(t *testing.T) {
EndsAt: evaluationTime.Add(80 * time.Second).Add(1 * time.Minute),
LastEvaluationTime: evaluationTime.Add(80 * time.Second),
EvaluationDuration: evaluationDuration,
Annotations: map[string]string{"annotation": "test"},
Annotations: map[string]string{"annotation": "test", "alerting_at": "2021-03-25 00:01:20 +0000 UTC"},
},
},
},
@ -345,20 +357,22 @@ func TestProcessEvalResults(t *testing.T) {
evalResults: []eval.Results{
{
eval.Result{
Instance: data.Labels{"instance_label": "test"},
State: eval.Normal,
EvaluatedAt: evaluationTime,
Instance: data.Labels{"instance_label": "test"},
State: eval.Normal,
EvaluatedAt: evaluationTime,
EvaluationDuration: evaluationDuration,
},
},
{
eval.Result{
Instance: data.Labels{"instance_label": "test"},
State: eval.Alerting,
EvaluatedAt: evaluationTime.Add(10 * time.Second),
Instance: data.Labels{"instance_label": "test"},
State: eval.Alerting,
EvaluatedAt: evaluationTime.Add(10 * time.Second),
EvaluationDuration: evaluationDuration,
},
},
},
expectedStates: map[string]state.AlertState{
expectedStates: map[string]*state.State{
"map[__alert_rule_namespace_uid__:test_namespace_uid __alert_rule_uid__:test_alert_rule_uid_2 alertname:test_title instance_label:test label:test]": {
AlertRuleUID: "test_alert_rule_uid_2",
OrgID: 1,
@ -371,7 +385,7 @@ func TestProcessEvalResults(t *testing.T) {
"instance_label": "test",
},
State: eval.Pending,
Results: []state.StateEvaluation{
Results: []state.Evaluation{
{
EvaluationTime: evaluationTime,
EvaluationState: eval.Normal,
@ -404,20 +418,22 @@ func TestProcessEvalResults(t *testing.T) {
evalResults: []eval.Results{
{
eval.Result{
Instance: data.Labels{"instance_label": "test"},
State: eval.Normal,
EvaluatedAt: evaluationTime,
Instance: data.Labels{"instance_label": "test"},
State: eval.Normal,
EvaluatedAt: evaluationTime,
EvaluationDuration: evaluationDuration,
},
},
{
eval.Result{
Instance: data.Labels{"instance_label": "test"},
State: eval.NoData,
EvaluatedAt: evaluationTime.Add(10 * time.Second),
Instance: data.Labels{"instance_label": "test"},
State: eval.NoData,
EvaluatedAt: evaluationTime.Add(10 * time.Second),
EvaluationDuration: evaluationDuration,
},
},
},
expectedStates: map[string]state.AlertState{
expectedStates: map[string]*state.State{
"map[__alert_rule_namespace_uid__:test_namespace_uid __alert_rule_uid__:test_alert_rule_uid_2 alertname:test_title instance_label:test label:test]": {
AlertRuleUID: "test_alert_rule_uid_2",
OrgID: 1,
@ -430,7 +446,7 @@ func TestProcessEvalResults(t *testing.T) {
"instance_label": "test",
},
State: eval.Alerting,
Results: []state.StateEvaluation{
Results: []state.Evaluation{
{
EvaluationTime: evaluationTime,
EvaluationState: eval.Normal,
@ -444,7 +460,7 @@ func TestProcessEvalResults(t *testing.T) {
EndsAt: evaluationTime.Add(10 * time.Second).Add(20 * time.Second),
LastEvaluationTime: evaluationTime.Add(10 * time.Second),
EvaluationDuration: evaluationDuration,
Annotations: map[string]string{"annotation": "test"},
Annotations: map[string]string{"annotation": "test", "no_data": "2021-03-25 00:00:10 +0000 UTC"},
},
},
},
@ -463,20 +479,22 @@ func TestProcessEvalResults(t *testing.T) {
evalResults: []eval.Results{
{
eval.Result{
Instance: data.Labels{"instance_label": "test"},
State: eval.Normal,
EvaluatedAt: evaluationTime,
Instance: data.Labels{"instance_label": "test"},
State: eval.Normal,
EvaluatedAt: evaluationTime,
EvaluationDuration: evaluationDuration,
},
},
{
eval.Result{
Instance: data.Labels{"instance_label": "test"},
State: eval.NoData,
EvaluatedAt: evaluationTime.Add(10 * time.Second),
Instance: data.Labels{"instance_label": "test"},
State: eval.NoData,
EvaluatedAt: evaluationTime.Add(10 * time.Second),
EvaluationDuration: evaluationDuration,
},
},
},
expectedStates: map[string]state.AlertState{
expectedStates: map[string]*state.State{
"map[__alert_rule_namespace_uid__:test_namespace_uid __alert_rule_uid__:test_alert_rule_uid_2 alertname:test_title instance_label:test label:test]": {
AlertRuleUID: "test_alert_rule_uid_2",
OrgID: 1,
@ -489,7 +507,7 @@ func TestProcessEvalResults(t *testing.T) {
"instance_label": "test",
},
State: eval.NoData,
Results: []state.StateEvaluation{
Results: []state.Evaluation{
{
EvaluationTime: evaluationTime,
EvaluationState: eval.Normal,
@ -503,7 +521,7 @@ func TestProcessEvalResults(t *testing.T) {
EndsAt: evaluationTime.Add(10 * time.Second).Add(20 * time.Second),
LastEvaluationTime: evaluationTime.Add(10 * time.Second),
EvaluationDuration: evaluationDuration,
Annotations: map[string]string{"annotation": "test"},
Annotations: map[string]string{"annotation": "test", "no_data": "2021-03-25 00:00:10 +0000 UTC"},
},
},
},
@ -522,20 +540,22 @@ func TestProcessEvalResults(t *testing.T) {
evalResults: []eval.Results{
{
eval.Result{
Instance: data.Labels{"instance_label": "test"},
State: eval.Normal,
EvaluatedAt: evaluationTime,
Instance: data.Labels{"instance_label": "test"},
State: eval.Normal,
EvaluatedAt: evaluationTime,
EvaluationDuration: evaluationDuration,
},
},
{
eval.Result{
Instance: data.Labels{"instance_label": "test"},
State: eval.NoData,
EvaluatedAt: evaluationTime.Add(10 * time.Second),
Instance: data.Labels{"instance_label": "test"},
State: eval.NoData,
EvaluatedAt: evaluationTime.Add(10 * time.Second),
EvaluationDuration: evaluationDuration,
},
},
},
expectedStates: map[string]state.AlertState{
expectedStates: map[string]*state.State{
"map[__alert_rule_namespace_uid__:test_namespace_uid __alert_rule_uid__:test_alert_rule_uid_2 alertname:test_title instance_label:test label:test]": {
AlertRuleUID: "test_alert_rule_uid_2",
OrgID: 1,
@ -548,7 +568,7 @@ func TestProcessEvalResults(t *testing.T) {
"instance_label": "test",
},
State: eval.Normal,
Results: []state.StateEvaluation{
Results: []state.Evaluation{
{
EvaluationTime: evaluationTime,
EvaluationState: eval.Normal,
@ -562,7 +582,7 @@ func TestProcessEvalResults(t *testing.T) {
EndsAt: evaluationTime.Add(10 * time.Second).Add(20 * time.Second),
LastEvaluationTime: evaluationTime.Add(10 * time.Second),
EvaluationDuration: evaluationDuration,
Annotations: map[string]string{"annotation": "test"},
Annotations: map[string]string{"annotation": "test", "no_data": "2021-03-25 00:00:10 +0000 UTC"},
},
},
},
@ -582,20 +602,22 @@ func TestProcessEvalResults(t *testing.T) {
evalResults: []eval.Results{
{
eval.Result{
Instance: data.Labels{"instance_label": "test"},
State: eval.Normal,
EvaluatedAt: evaluationTime,
Instance: data.Labels{"instance_label": "test"},
State: eval.Normal,
EvaluatedAt: evaluationTime,
EvaluationDuration: evaluationDuration,
},
},
{
eval.Result{
Instance: data.Labels{"instance_label": "test"},
State: eval.NoData,
EvaluatedAt: evaluationTime.Add(10 * time.Second),
Instance: data.Labels{"instance_label": "test"},
State: eval.NoData,
EvaluatedAt: evaluationTime.Add(10 * time.Second),
EvaluationDuration: evaluationDuration,
},
},
},
expectedStates: map[string]state.AlertState{
expectedStates: map[string]*state.State{
"map[__alert_rule_namespace_uid__:test_namespace_uid __alert_rule_uid__:test_alert_rule_uid_2 alertname:test_title instance_label:test label:test]": {
AlertRuleUID: "test_alert_rule_uid_2",
OrgID: 1,
@ -608,7 +630,7 @@ func TestProcessEvalResults(t *testing.T) {
"instance_label": "test",
},
State: eval.Alerting,
Results: []state.StateEvaluation{
Results: []state.Evaluation{
{
EvaluationTime: evaluationTime,
EvaluationState: eval.Normal,
@ -622,7 +644,7 @@ func TestProcessEvalResults(t *testing.T) {
EndsAt: evaluationTime.Add(10 * time.Second).Add(1 * time.Minute),
LastEvaluationTime: evaluationTime.Add(10 * time.Second),
EvaluationDuration: evaluationDuration,
Annotations: map[string]string{"annotation": "test"},
Annotations: map[string]string{"annotation": "test", "no_data": "2021-03-25 00:00:10 +0000 UTC"},
},
},
},
@ -642,20 +664,22 @@ func TestProcessEvalResults(t *testing.T) {
evalResults: []eval.Results{
{
eval.Result{
Instance: data.Labels{"instance_label": "test"},
State: eval.Normal,
EvaluatedAt: evaluationTime,
Instance: data.Labels{"instance_label": "test"},
State: eval.Normal,
EvaluatedAt: evaluationTime,
EvaluationDuration: evaluationDuration,
},
},
{
eval.Result{
Instance: data.Labels{"instance_label": "test"},
State: eval.NoData,
EvaluatedAt: evaluationTime.Add(10 * time.Second),
Instance: data.Labels{"instance_label": "test"},
State: eval.NoData,
EvaluatedAt: evaluationTime.Add(10 * time.Second),
EvaluationDuration: evaluationDuration,
},
},
},
expectedStates: map[string]state.AlertState{
expectedStates: map[string]*state.State{
"map[__alert_rule_namespace_uid__:test_namespace_uid __alert_rule_uid__:test_alert_rule_uid_2 alertname:test_title instance_label:test label:test]": {
AlertRuleUID: "test_alert_rule_uid_2",
OrgID: 1,
@ -668,7 +692,7 @@ func TestProcessEvalResults(t *testing.T) {
"instance_label": "test",
},
State: eval.Normal,
Results: []state.StateEvaluation{
Results: []state.Evaluation{
{
EvaluationTime: evaluationTime,
EvaluationState: eval.Normal,
@ -682,7 +706,7 @@ func TestProcessEvalResults(t *testing.T) {
EndsAt: evaluationTime.Add(10 * time.Second).Add(1 * time.Minute),
LastEvaluationTime: evaluationTime.Add(10 * time.Second),
EvaluationDuration: evaluationDuration,
Annotations: map[string]string{"annotation": "test"},
Annotations: map[string]string{"annotation": "test", "no_data": "2021-03-25 00:00:10 +0000 UTC"},
},
},
},
@ -702,20 +726,22 @@ func TestProcessEvalResults(t *testing.T) {
evalResults: []eval.Results{
{
eval.Result{
Instance: data.Labels{"instance_label": "test"},
State: eval.Normal,
EvaluatedAt: evaluationTime,
Instance: data.Labels{"instance_label": "test"},
State: eval.Normal,
EvaluatedAt: evaluationTime,
EvaluationDuration: evaluationDuration,
},
},
{
eval.Result{
Instance: data.Labels{"instance_label": "test"},
State: eval.NoData,
EvaluatedAt: evaluationTime.Add(10 * time.Second),
Instance: data.Labels{"instance_label": "test"},
State: eval.NoData,
EvaluatedAt: evaluationTime.Add(10 * time.Second),
EvaluationDuration: evaluationDuration,
},
},
},
expectedStates: map[string]state.AlertState{
expectedStates: map[string]*state.State{
"map[__alert_rule_namespace_uid__:test_namespace_uid __alert_rule_uid__:test_alert_rule_uid_2 alertname:test_title instance_label:test label:test]": {
AlertRuleUID: "test_alert_rule_uid_2",
OrgID: 1,
@ -728,7 +754,7 @@ func TestProcessEvalResults(t *testing.T) {
"instance_label": "test",
},
State: eval.Normal,
Results: []state.StateEvaluation{
Results: []state.Evaluation{
{
EvaluationTime: evaluationTime,
EvaluationState: eval.Normal,
@ -742,20 +768,22 @@ func TestProcessEvalResults(t *testing.T) {
EndsAt: evaluationTime.Add(10 * time.Second).Add(1 * time.Minute),
LastEvaluationTime: evaluationTime.Add(10 * time.Second),
EvaluationDuration: evaluationDuration,
Annotations: map[string]string{"annotation": "test"},
Annotations: map[string]string{"annotation": "test", "no_data": "2021-03-25 00:00:10 +0000 UTC"},
},
},
},
}
for _, tc := range testCases {
st := state.NewStateTracker(log.New("test_state_tracker"))
st := state.NewManager(log.New("test_state_manager"))
t.Run(tc.desc, func(t *testing.T) {
for _, res := range tc.evalResults {
_ = st.ProcessEvalResults(tc.alertRule, res, evaluationDuration)
_ = st.ProcessEvalResults(tc.alertRule, res)
}
for id, s := range tc.expectedStates {
assert.Equal(t, s, st.Get(id))
cachedState, err := st.Get(id)
require.NoError(t, err)
assert.Equal(t, s, cachedState)
}
})
}

View File

@ -33,14 +33,14 @@ type evalAppliedInfo struct {
func TestWarmStateCache(t *testing.T) {
evaluationTime, _ := time.Parse("2006-01-02", "2021-03-25")
expectedEntries := []state.AlertState{
expectedEntries := []*state.State{
{
AlertRuleUID: "test_uid",
OrgID: 123,
CacheId: "test_uid map[test1:testValue1]",
Labels: data.Labels{"test1": "testValue1"},
State: eval.Normal,
Results: []state.StateEvaluation{
Results: []state.Evaluation{
{EvaluationTime: evaluationTime, EvaluationState: eval.Normal},
},
StartsAt: evaluationTime.Add(-1 * time.Minute),
@ -52,7 +52,7 @@ func TestWarmStateCache(t *testing.T) {
CacheId: "test_uid map[test2:testValue2]",
Labels: data.Labels{"test2": "testValue2"},
State: eval.Alerting,
Results: []state.StateEvaluation{
Results: []state.Evaluation{
{EvaluationTime: evaluationTime, EvaluationState: eval.Alerting},
},
StartsAt: evaluationTime.Add(-1 * time.Minute),
@ -94,12 +94,13 @@ func TestWarmStateCache(t *testing.T) {
Store: dbstore,
}
sched := schedule.NewScheduler(schedCfg, nil)
st := state.NewStateTracker(schedCfg.Logger)
st := state.NewManager(schedCfg.Logger)
sched.WarmStateCache(st)
t.Run("instance cache has expected entries", func(t *testing.T) {
for _, entry := range expectedEntries {
cacheEntry := st.Get(entry.CacheId)
cacheEntry, err := st.Get(entry.CacheId)
require.NoError(t, err)
assert.True(t, entry.Equals(cacheEntry))
}
})
@ -139,7 +140,7 @@ func TestAlertingTicker(t *testing.T) {
ctx := context.Background()
st := state.NewStateTracker(schedCfg.Logger)
st := state.NewManager(schedCfg.Logger)
go func() {
err := sched.Ticker(ctx, st)
require.NoError(t, err)