mirror of
https://github.com/grafana/grafana.git
synced 2025-02-25 18:55:37 -06:00
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:
parent
ca79206498
commit
788bc2a793
@ -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(
|
||||
|
@ -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",
|
||||
|
@ -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 {
|
||||
|
@ -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'"`
|
||||
|
@ -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.
|
||||
|
@ -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 {
|
||||
|
@ -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,
|
||||
|
135
pkg/services/ngalert/state/cache.go
Normal file
135
pkg/services/ngalert/state/cache.go
Normal 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
|
||||
}
|
117
pkg/services/ngalert/state/manager.go
Normal file
117
pkg/services/ngalert/state/manager.go
Normal 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)
|
||||
}
|
||||
}
|
130
pkg/services/ngalert/state/state.go
Normal file
130
pkg/services/ngalert/state/state.go
Normal 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
|
||||
}
|
@ -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
|
||||
}
|
@ -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)
|
||||
}
|
||||
})
|
||||
}
|
@ -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)
|
||||
|
Loading…
Reference in New Issue
Block a user