alertmanager/dispatch/dispatch.go

546 lines
14 KiB
Go
Raw Permalink Normal View History

// Copyright 2018 Prometheus Team
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
package dispatch
import (
"context"
"errors"
2015-09-26 09:12:47 +00:00
"fmt"
"log/slog"
"sort"
"sync"
"time"
"github.com/prometheus/client_golang/prometheus"
"github.com/prometheus/common/model"
2015-09-25 11:12:51 +00:00
2015-09-29 13:12:31 +00:00
"github.com/prometheus/alertmanager/notify"
2015-09-25 11:12:51 +00:00
"github.com/prometheus/alertmanager/provider"
"github.com/prometheus/alertmanager/store"
2015-09-25 12:38:57 +00:00
"github.com/prometheus/alertmanager/types"
)
// DispatcherMetrics represents metrics associated to a dispatcher.
type DispatcherMetrics struct {
aggrGroups prometheus.Gauge
processingDuration prometheus.Summary
aggrGroupLimitReached prometheus.Counter
}
// NewDispatcherMetrics returns a new registered DispatchMetrics.
func NewDispatcherMetrics(registerLimitMetrics bool, r prometheus.Registerer) *DispatcherMetrics {
m := DispatcherMetrics{
aggrGroups: prometheus.NewGauge(
prometheus.GaugeOpts{
Name: "alertmanager_dispatcher_aggregation_groups",
Help: "Number of active aggregation groups",
},
),
processingDuration: prometheus.NewSummary(
prometheus.SummaryOpts{
Name: "alertmanager_dispatcher_alert_processing_duration_seconds",
Help: "Summary of latencies for the processing of alerts.",
},
),
aggrGroupLimitReached: prometheus.NewCounter(
prometheus.CounterOpts{
Name: "alertmanager_dispatcher_aggregation_group_limit_reached_total",
Help: "Number of times when dispatcher failed to create new aggregation group due to limit.",
},
),
}
if r != nil {
r.MustRegister(m.aggrGroups, m.processingDuration)
if registerLimitMetrics {
r.MustRegister(m.aggrGroupLimitReached)
}
}
return &m
}
// Dispatcher sorts incoming alerts into aggregation groups and
// assigns the correct notifiers to each.
type Dispatcher struct {
route *Route
alerts provider.Alerts
stage notify.Stage
marker types.GroupMarker
metrics *DispatcherMetrics
limits Limits
timeout func(time.Duration) time.Duration
mtx sync.RWMutex
aggrGroupsPerRoute map[*Route]map[model.Fingerprint]*aggrGroup
aggrGroupsNum int
done chan struct{}
ctx context.Context
cancel func()
2015-09-29 09:58:30 +00:00
logger *slog.Logger
}
// Limits describes limits used by Dispatcher.
type Limits interface {
// MaxNumberOfAggregationGroups returns max number of aggregation groups that dispatcher can have.
// 0 or negative value = unlimited.
// If dispatcher hits this limit, it will not create additional groups, but will log an error instead.
MaxNumberOfAggregationGroups() int
}
// NewDispatcher returns a new Dispatcher.
func NewDispatcher(
ap provider.Alerts,
r *Route,
s notify.Stage,
mk types.GroupMarker,
to func(time.Duration) time.Duration,
lim Limits,
l *slog.Logger,
m *DispatcherMetrics,
) *Dispatcher {
if lim == nil {
lim = nilLimits{}
}
disp := &Dispatcher{
alerts: ap,
stage: s,
route: r,
marker: mk,
timeout: to,
logger: l.With("component", "dispatcher"),
metrics: m,
limits: lim,
}
return disp
}
// Run starts dispatching alerts incoming via the updates channel.
func (d *Dispatcher) Run() {
d.done = make(chan struct{})
d.mtx.Lock()
d.aggrGroupsPerRoute = map[*Route]map[model.Fingerprint]*aggrGroup{}
d.aggrGroupsNum = 0
d.metrics.aggrGroups.Set(0)
d.ctx, d.cancel = context.WithCancel(context.Background())
d.mtx.Unlock()
d.run(d.alerts.Subscribe())
close(d.done)
}
func (d *Dispatcher) run(it provider.AlertIterator) {
maintenance := time.NewTicker(30 * time.Second)
defer maintenance.Stop()
defer it.Close()
for {
select {
case alert, ok := <-it.Next():
if !ok {
// Iterator exhausted for some reason.
if err := it.Err(); err != nil {
d.logger.Error("Error on alert update", "err", err)
}
return
}
d.logger.Debug("Received alert", "alert", alert)
2015-09-29 09:58:30 +00:00
// Log errors but keep trying.
if err := it.Err(); err != nil {
d.logger.Error("Error on alert update", "err", err)
continue
}
now := time.Now()
2015-10-19 14:17:15 +00:00
for _, r := range d.route.Match(alert.Labels) {
d.processAlert(alert, r)
}
d.metrics.processingDuration.Observe(time.Since(now).Seconds())
case <-maintenance.C:
d.doMaintenance()
case <-d.ctx.Done():
return
}
}
}
func (d *Dispatcher) doMaintenance() {
d.mtx.Lock()
defer d.mtx.Unlock()
for _, groups := range d.aggrGroupsPerRoute {
for _, ag := range groups {
if ag.empty() {
ag.stop()
d.marker.DeleteByGroupKey(ag.routeID, ag.GroupKey())
delete(groups, ag.fingerprint())
d.aggrGroupsNum--
d.metrics.aggrGroups.Dec()
}
}
}
}
// AlertGroup represents how alerts exist within an aggrGroup.
type AlertGroup struct {
Alerts types.AlertSlice
Labels model.LabelSet
Receiver string
GroupKey string
RouteID string
}
type AlertGroups []*AlertGroup
func (ag AlertGroups) Swap(i, j int) { ag[i], ag[j] = ag[j], ag[i] }
func (ag AlertGroups) Less(i, j int) bool {
if ag[i].Labels.Equal(ag[j].Labels) {
return ag[i].Receiver < ag[j].Receiver
}
return ag[i].Labels.Before(ag[j].Labels)
}
func (ag AlertGroups) Len() int { return len(ag) }
// Groups returns a slice of AlertGroups from the dispatcher's internal state.
func (d *Dispatcher) Groups(routeFilter func(*Route) bool, alertFilter func(*types.Alert, time.Time) bool) (AlertGroups, map[model.Fingerprint][]string) {
groups := AlertGroups{}
d.mtx.RLock()
defer d.mtx.RUnlock()
// Keep a list of receivers for an alert to prevent checking each alert
// again against all routes. The alert has already matched against this
// route on ingestion.
receivers := map[model.Fingerprint][]string{}
now := time.Now()
for route, ags := range d.aggrGroupsPerRoute {
if !routeFilter(route) {
continue
}
for _, ag := range ags {
receiver := route.RouteOpts.Receiver
alertGroup := &AlertGroup{
Labels: ag.labels,
Receiver: receiver,
GroupKey: ag.GroupKey(),
RouteID: ag.routeID,
}
alerts := ag.alerts.List()
filteredAlerts := make([]*types.Alert, 0, len(alerts))
for _, a := range alerts {
if !alertFilter(a, now) {
continue
}
fp := a.Fingerprint()
if r, ok := receivers[fp]; ok {
// Receivers slice already exists. Add
// the current receiver to the slice.
receivers[fp] = append(r, receiver)
} else {
// First time we've seen this alert fingerprint.
// Initialize a new receivers slice.
receivers[fp] = []string{receiver}
}
filteredAlerts = append(filteredAlerts, a)
}
if len(filteredAlerts) == 0 {
continue
}
alertGroup.Alerts = filteredAlerts
groups = append(groups, alertGroup)
}
}
sort.Sort(groups)
for i := range groups {
sort.Sort(groups[i].Alerts)
}
for i := range receivers {
sort.Strings(receivers[i])
}
return groups, receivers
}
// Stop the dispatcher.
func (d *Dispatcher) Stop() {
if d == nil {
return
}
d.mtx.Lock()
if d.cancel == nil {
d.mtx.Unlock()
return
}
d.cancel()
2015-09-25 16:14:46 +00:00
d.cancel = nil
d.mtx.Unlock()
2015-09-26 09:12:47 +00:00
<-d.done
}
// notifyFunc is a function that performs notification for the alert
// with the given fingerprint. It aborts on context cancelation.
2015-09-26 16:12:56 +00:00
// Returns false iff notifying failed.
type notifyFunc func(context.Context, ...*types.Alert) bool
2016-06-06 11:18:55 +00:00
// processAlert determines in which aggregation group the alert falls
// and inserts it.
func (d *Dispatcher) processAlert(alert *types.Alert, route *Route) {
groupLabels := getGroupLabels(alert, route)
fp := groupLabels.Fingerprint()
d.mtx.Lock()
defer d.mtx.Unlock()
routeGroups, ok := d.aggrGroupsPerRoute[route]
if !ok {
routeGroups = map[model.Fingerprint]*aggrGroup{}
d.aggrGroupsPerRoute[route] = routeGroups
}
ag, ok := routeGroups[fp]
if ok {
ag.insert(alert)
return
}
// If the group does not exist, create it. But check the limit first.
if limit := d.limits.MaxNumberOfAggregationGroups(); limit > 0 && d.aggrGroupsNum >= limit {
d.metrics.aggrGroupLimitReached.Inc()
d.logger.Error("Too many aggregation groups, cannot create new group for alert", "groups", d.aggrGroupsNum, "limit", limit, "alert", alert.Name())
return
}
ag = newAggrGroup(d.ctx, groupLabels, route, d.timeout, d.logger)
routeGroups[fp] = ag
d.aggrGroupsNum++
d.metrics.aggrGroups.Inc()
// Insert the 1st alert in the group before starting the group's run()
// function, to make sure that when the run() will be executed the 1st
// alert is already there.
ag.insert(alert)
go ag.run(func(ctx context.Context, alerts ...*types.Alert) bool {
_, _, err := d.stage.Exec(ctx, d.logger, alerts...)
if err != nil {
logger := d.logger.With("num_alerts", len(alerts), "err", err)
if errors.Is(ctx.Err(), context.Canceled) {
// It is expected for the context to be canceled on
// configuration reload or shutdown. In this case, the
// message should only be logged at the debug level.
logger.Debug("Notify for alerts failed")
} else {
logger.Error("Notify for alerts failed")
}
}
return err == nil
})
}
func getGroupLabels(alert *types.Alert, route *Route) model.LabelSet {
groupLabels := model.LabelSet{}
for ln, lv := range alert.Labels {
if _, ok := route.RouteOpts.GroupBy[ln]; ok || route.RouteOpts.GroupByAll {
groupLabels[ln] = lv
}
}
return groupLabels
}
// aggrGroup aggregates alert fingerprints into groups to which a
// common set of routing options applies.
// It emits notifications in the specified intervals.
type aggrGroup struct {
labels model.LabelSet
opts *RouteOpts
logger *slog.Logger
routeID string
routeKey string
alerts *store.Alerts
ctx context.Context
cancel func()
done chan struct{}
next *time.Timer
timeout func(time.Duration) time.Duration
mtx sync.RWMutex
hasFlushed bool
}
// newAggrGroup returns a new aggregation group.
func newAggrGroup(ctx context.Context, labels model.LabelSet, r *Route, to func(time.Duration) time.Duration, logger *slog.Logger) *aggrGroup {
if to == nil {
to = func(d time.Duration) time.Duration { return d }
}
ag := &aggrGroup{
labels: labels,
routeID: r.ID(),
routeKey: r.Key(),
opts: &r.RouteOpts,
timeout: to,
alerts: store.NewAlerts(),
done: make(chan struct{}),
}
ag.ctx, ag.cancel = context.WithCancel(ctx)
ag.logger = logger.With("aggrGroup", ag)
2015-10-27 17:24:09 +00:00
2015-10-07 14:18:55 +00:00
// Set an initial one-time wait before flushing
// the first batch of notifications.
ag.next = time.NewTimer(ag.opts.GroupWait)
return ag
}
func (ag *aggrGroup) fingerprint() model.Fingerprint {
return ag.labels.Fingerprint()
}
func (ag *aggrGroup) GroupKey() string {
return fmt.Sprintf("%s:%s", ag.routeKey, ag.labels)
}
func (ag *aggrGroup) String() string {
return ag.GroupKey()
}
2015-09-29 13:12:31 +00:00
func (ag *aggrGroup) run(nf notifyFunc) {
defer close(ag.done)
defer ag.next.Stop()
for {
select {
case now := <-ag.next.C:
// Give the notifications time until the next flush to
// finish before terminating them.
ctx, cancel := context.WithTimeout(ag.ctx, ag.timeout(ag.opts.GroupInterval))
// The now time we retrieve from the ticker is the only reliable
// point of time reference for the subsequent notification pipeline.
// Calculating the current time directly is prone to flaky behavior,
// which usually only becomes apparent in tests.
ctx = notify.WithNow(ctx, now)
// Populate context with information needed along the pipeline.
ctx = notify.WithGroupKey(ctx, ag.GroupKey())
ctx = notify.WithGroupLabels(ctx, ag.labels)
ctx = notify.WithReceiverName(ctx, ag.opts.Receiver)
ctx = notify.WithRepeatInterval(ctx, ag.opts.RepeatInterval)
ctx = notify.WithMuteTimeIntervals(ctx, ag.opts.MuteTimeIntervals)
Adds: Active time interval (#2779) * add active time interval Signed-off-by: Sinuhe Tellez <dubyte@gmail.com> * fix active time interval Signed-off-by: Sinuhe Tellez <dubyte@gmail.com> * fix unittests for active time interval Signed-off-by: Sinuhe Tellez <dubyte@gmail.com> * Update notify/notify.go Co-authored-by: Simon Pasquier <spasquie@redhat.com> Signed-off-by: Sinuhe Tellez <dubyte@gmail.com> * Update dispatch/route.go Co-authored-by: Simon Pasquier <spasquie@redhat.com> Signed-off-by: Sinuhe Tellez <dubyte@gmail.com> * split the stage for active and mute intervals Signed-off-by: Sinuhe Tellez <dubyte@gmail.com> * Update notify/notify.go Adds doc for a helper function Co-authored-by: Simon Pasquier <spasquie@redhat.com> Signed-off-by: Sinuhe Tellez <dubyte@gmail.com> * Update notify/notify.go Co-authored-by: Simon Pasquier <spasquie@redhat.com> Signed-off-by: Sinuhe Tellez <dubyte@gmail.com> * Update notify/notify.go Co-authored-by: Simon Pasquier <spasquie@redhat.com> Signed-off-by: Sinuhe Tellez <dubyte@gmail.com> * Update notify/notify.go Co-authored-by: Simon Pasquier <spasquie@redhat.com> Signed-off-by: Sinuhe Tellez <dubyte@gmail.com> * fix code after commit suggestions Signed-off-by: Sinuhe Tellez <dubyte@gmail.com> * Making mute_time_interval and time_intervals can coexist in the config Signed-off-by: Sinuhe Tellez <dubyte@gmail.com> * docs: configuration's doc has been updated about time intervals Signed-off-by: Sinuhe Tellez <dubyte@gmail.com> * Update config/config.go Co-authored-by: Simon Pasquier <spasquie@redhat.com> Signed-off-by: Sinuhe Tellez <dubyte@gmail.com> * Update docs/configuration.md Co-authored-by: Simon Pasquier <spasquie@redhat.com> Signed-off-by: Sinuhe Tellez <dubyte@gmail.com> * Update docs/configuration.md Co-authored-by: Simon Pasquier <spasquie@redhat.com> Signed-off-by: Sinuhe Tellez <dubyte@gmail.com> * Update docs/configuration.md Co-authored-by: Simon Pasquier <spasquie@redhat.com> Signed-off-by: Sinuhe Tellez <dubyte@gmail.com> * Update docs/configuration.md Co-authored-by: Simon Pasquier <spasquie@redhat.com> Signed-off-by: Sinuhe Tellez <dubyte@gmail.com> * updates configuration readme to improve active time description Signed-off-by: Sinuhe Tellez <dubyte@gmail.com> * merge deprecated mute_time_intervals and time_intervals Signed-off-by: Sinuhe Tellez <dubyte@gmail.com> * Update docs/configuration.md Co-authored-by: Simon Pasquier <spasquie@redhat.com> Signed-off-by: Sinuhe Tellez <dubyte@gmail.com> * Update docs/configuration.md Co-authored-by: Simon Pasquier <spasquie@redhat.com> Signed-off-by: Sinuhe Tellez <dubyte@gmail.com> * Update docs/configuration.md Co-authored-by: Simon Pasquier <spasquie@redhat.com> Signed-off-by: Sinuhe Tellez <dubyte@gmail.com> * Update docs/configuration.md Co-authored-by: Simon Pasquier <spasquie@redhat.com> Signed-off-by: Sinuhe Tellez <dubyte@gmail.com> * Update docs/configuration.md Co-authored-by: Simon Pasquier <spasquie@redhat.com> Signed-off-by: Sinuhe Tellez <dubyte@gmail.com> * Update docs/configuration.md Co-authored-by: Simon Pasquier <spasquie@redhat.com> Signed-off-by: Sinuhe Tellez <dubyte@gmail.com> * Update cmd/alertmanager/main.go Co-authored-by: Simon Pasquier <spasquie@redhat.com> Signed-off-by: Sinuhe Tellez <dubyte@gmail.com> * Update cmd/alertmanager/main.go Co-authored-by: Simon Pasquier <spasquie@redhat.com> Signed-off-by: Sinuhe Tellez <dubyte@gmail.com> * fmt main.go Signed-off-by: Sinuhe Tellez <dubyte@gmail.com> * Update docs/configuration.md Co-authored-by: Simon Pasquier <spasquie@redhat.com> Signed-off-by: Sinuhe Tellez <dubyte@gmail.com> * Update docs/configuration.md Co-authored-by: Simon Pasquier <spasquie@redhat.com> Signed-off-by: Sinuhe Tellez <dubyte@gmail.com> * fix lint error Signed-off-by: clyang82 <chuyang@redhat.com> Signed-off-by: Sinuhe Tellez <dubyte@gmail.com> * Document that matchers are ANDed together Signed-off-by: Mac Chaffee <me@macchaffee.com> Signed-off-by: Sinuhe Tellez <dubyte@gmail.com> * Remove extra parentheticals Signed-off-by: Mac Chaffee <me@macchaffee.com> Signed-off-by: Sinuhe Tellez <dubyte@gmail.com> * config: root route should have empty matchers Unmarshal should validate that the root route does not contain any matchers. Prior to this change, only the deprecated match structures were checked. Signed-off-by: Philip Gough <philip.p.gough@gmail.com> Signed-off-by: Sinuhe Tellez <dubyte@gmail.com> * chore: Let git ignore temporary files for ui/app Signed-off-by: nekketsuuu <nekketsuuu@users.noreply.github.com> Signed-off-by: Sinuhe Tellez <dubyte@gmail.com> * adding max_alerts parameter to slack webhook config correcting the logic to trucate fields instead of dropping alerts in the slack integration Signed-off-by: Prashant Balachandran <pnair@redhat.com> Signed-off-by: Sinuhe Tellez <dubyte@gmail.com> * *: bump to Go 1.17 (#2792) * *: bump to Go 1.17 Signed-off-by: Simon Pasquier <spasquie@redhat.com> * *: fix yamllint errors Signed-off-by: Simon Pasquier <spasquie@redhat.com> Signed-off-by: Sinuhe Tellez <dubyte@gmail.com> * Automate CSS-inlining for default HTML email template (#2798) * Automate CSS-inlining for default HTML email template The original HTML email template was added in `template/email.html`. It looks like the CSS was manually inlined. Most likely using the premailer.dialect.ca web form, which is mentioned in the README for the Mailgun transactional-email-templates project. The resulting HTML with inlined CSS was then copied into `template/default.tmpl`. This has resulted in `email.html` and `default.tmpl` diverging at times. This commit adds build automation to inline the CSS automatically using [juice][1]. The Go template containing the resulting HTML has been moved into its own file to avoid the script that performs the CSS inlining having to parse the `default.tmpl` file to insert it there. Fixes #1939. [1]: https://www.npmjs.com/package/juice Signed-off-by: Brad Ison <bison@xvdf.io> * Update asset/assets_vfsdata.go Signed-off-by: Brad Ison <bison@xvdf.io> Signed-off-by: Sinuhe Tellez <dubyte@gmail.com> * go.{mod,sum}: update Go dependencies Signed-off-by: Simon Pasquier <spasquie@redhat.com> Signed-off-by: Sinuhe Tellez <dubyte@gmail.com> * amtool to support http_config to access alertmanager (#2764) * Support http_config for amtool Co-authored-by: Julien Pivotto <roidelapluie@gmail.com> Co-authored-by: Simon Pasquier <spasquie@redhat.com> Signed-off-by: clyang82 <chuyang@redhat.com> Signed-off-by: Sinuhe Tellez <dubyte@gmail.com> * notify/sns: detect FIFO topic based on the rendered value Since the TopicARN field is a template string, it's safer to check for the ".fifo" suffix in the rendered string. Signed-off-by: Simon Pasquier <spasquie@redhat.com> Signed-off-by: Sinuhe Tellez <dubyte@gmail.com> * config: delegate Sigv4 validation to the inner type This change also adds unit tests for SNS configuration. Signed-off-by: Simon Pasquier <spasquie@redhat.com> Signed-off-by: Sinuhe Tellez <dubyte@gmail.com> * fix unittests Signed-off-by: Sinuhe Tellez <dubyte@gmail.com> * fix comment about active time interval Signed-off-by: Sinuhe Tellez <dubyte@gmail.com> * fix another comment about active time interval Signed-off-by: Sinuhe Tellez <dubyte@gmail.com> * Update docs/configuration.md Fix typo in documentation Co-authored-by: Simon Pasquier <spasquie@redhat.com> Signed-off-by: Sinuhe Tellez <dubyte@gmail.com> * Update docs/configuration.md Co-authored-by: Simon Pasquier <spasquie@redhat.com> Signed-off-by: Sinuhe Tellez <dubyte@gmail.com> Co-authored-by: Simon Pasquier <spasquie@redhat.com> Co-authored-by: clyang82 <chuyang@redhat.com> Co-authored-by: Mac Chaffee <me@macchaffee.com> Co-authored-by: Philip Gough <philip.p.gough@gmail.com> Co-authored-by: nekketsuuu <nekketsuuu@users.noreply.github.com> Co-authored-by: Prashant Balachandran <pnair@redhat.com> Co-authored-by: Simon Pasquier <pasquier.simon@gmail.com> Co-authored-by: Brad Ison <brad.ison@redhat.com> Co-authored-by: Julien Pivotto <roidelapluie@gmail.com>
2022-03-04 14:24:29 +00:00
ctx = notify.WithActiveTimeIntervals(ctx, ag.opts.ActiveTimeIntervals)
ctx = notify.WithRouteID(ctx, ag.routeID)
// Wait the configured interval before calling flush again.
ag.mtx.Lock()
ag.next.Reset(ag.opts.GroupInterval)
ag.hasFlushed = true
ag.mtx.Unlock()
ag.flush(func(alerts ...*types.Alert) bool {
2015-09-29 13:12:31 +00:00
return nf(ctx, alerts...)
})
cancel()
case <-ag.ctx.Done():
return
}
}
}
func (ag *aggrGroup) stop() {
// Calling cancel will terminate all in-process notifications
// and the run() loop.
ag.cancel()
<-ag.done
}
// insert inserts the alert into the aggregation group.
2015-09-25 16:14:46 +00:00
func (ag *aggrGroup) insert(alert *types.Alert) {
if err := ag.alerts.Set(alert); err != nil {
ag.logger.Error("error on set alert", "err", err)
}
// Immediately trigger a flush if the wait duration for this
// alert is already over.
ag.mtx.Lock()
defer ag.mtx.Unlock()
if !ag.hasFlushed && alert.StartsAt.Add(ag.opts.GroupWait).Before(time.Now()) {
ag.next.Reset(0)
}
}
func (ag *aggrGroup) empty() bool {
return ag.alerts.Empty()
}
// flush sends notifications for all new alerts.
func (ag *aggrGroup) flush(notify func(...*types.Alert) bool) {
2015-09-27 17:50:41 +00:00
if ag.empty() {
return
}
var (
alerts = ag.alerts.List()
alertsSlice = make(types.AlertSlice, 0, len(alerts))
resolvedSlice = make(types.AlertSlice, 0, len(alerts))
now = time.Now()
)
for _, alert := range alerts {
a := *alert
// Ensure that alerts don't resolve as time move forwards.
if a.ResolvedAt(now) {
resolvedSlice = append(resolvedSlice, &a)
} else {
a.EndsAt = time.Time{}
}
alertsSlice = append(alertsSlice, &a)
}
sort.Stable(alertsSlice)
ag.logger.Debug("flushing", "alerts", fmt.Sprintf("%v", alertsSlice))
if notify(alertsSlice...) {
// Delete all resolved alerts as we just sent a notification for them,
// and we don't want to send another one. However, we need to make sure
// that each resolved alert has not fired again during the flush as then
// we would delete an active alert thinking it was resolved.
if err := ag.alerts.DeleteIfNotModified(resolvedSlice); err != nil {
ag.logger.Error("error on delete alerts", "err", err)
}
2015-07-10 17:25:56 +00:00
}
}
type nilLimits struct{}
func (n nilLimits) MaxNumberOfAggregationGroups() int { return 0 }