diff --git a/alerting/alertmanager.go b/alerting/alertmanager.go
new file mode 100644
index 00000000..7243a9ac
--- /dev/null
+++ b/alerting/alertmanager.go
@@ -0,0 +1,704 @@
+package notifier
+
+import (
+ "context"
+ "crypto/md5"
+ "encoding/base64"
+ "encoding/json"
+ "errors"
+ "fmt"
+ "net/url"
+ "path/filepath"
+ "regexp"
+ "strconv"
+ "sync"
+ "time"
+ "unicode/utf8"
+
+ amv2 "github.com/prometheus/alertmanager/api/v2/models"
+ "github.com/prometheus/alertmanager/cluster"
+ "github.com/prometheus/alertmanager/config"
+ "github.com/prometheus/alertmanager/dispatch"
+ "github.com/prometheus/alertmanager/inhibit"
+ "github.com/prometheus/alertmanager/nflog"
+ "github.com/prometheus/alertmanager/nflog/nflogpb"
+ "github.com/prometheus/alertmanager/notify"
+ "github.com/prometheus/alertmanager/provider/mem"
+ "github.com/prometheus/alertmanager/silence"
+ "github.com/prometheus/alertmanager/template"
+ "github.com/prometheus/alertmanager/timeinterval"
+ "github.com/prometheus/alertmanager/types"
+ "github.com/prometheus/client_golang/prometheus"
+ "github.com/prometheus/common/model"
+
+ pb "github.com/prometheus/alertmanager/silence/silencepb"
+
+ "github.com/grafana/grafana/pkg/infra/kvstore"
+ "github.com/grafana/grafana/pkg/infra/log"
+ apimodels "github.com/grafana/grafana/pkg/services/ngalert/api/tooling/definitions"
+ "github.com/grafana/grafana/pkg/services/ngalert/metrics"
+ ngmodels "github.com/grafana/grafana/pkg/services/ngalert/models"
+ "github.com/grafana/grafana/pkg/services/ngalert/notifier/channels"
+ "github.com/grafana/grafana/pkg/services/ngalert/store"
+ "github.com/grafana/grafana/pkg/services/notifications"
+ "github.com/grafana/grafana/pkg/setting"
+)
+
+const (
+ notificationLogFilename = "notifications"
+ silencesFilename = "silences"
+
+ workingDir = "alerting"
+ // maintenanceNotificationAndSilences how often should we flush and gargabe collect notifications and silences
+ maintenanceNotificationAndSilences = 15 * time.Minute
+ // defaultResolveTimeout is the default timeout used for resolving an alert
+ // if the end time is not specified.
+ defaultResolveTimeout = 5 * time.Minute
+ // memoryAlertsGCInterval is the interval at which we'll remove resolved alerts from memory.
+ memoryAlertsGCInterval = 30 * time.Minute
+)
+
+// How long should we keep silences and notification entries on-disk after they've served their purpose.
+var retentionNotificationsAndSilences = 5 * 24 * time.Hour
+var silenceMaintenanceInterval = 15 * time.Minute
+
+func init() {
+ silence.ValidateMatcher = func(m *pb.Matcher) error {
+ switch m.Type {
+ case pb.Matcher_EQUAL, pb.Matcher_NOT_EQUAL:
+ if !model.LabelValue(m.Pattern).IsValid() {
+ return fmt.Errorf("invalid label value %q", m.Pattern)
+ }
+ case pb.Matcher_REGEXP, pb.Matcher_NOT_REGEXP:
+ if _, err := regexp.Compile(m.Pattern); err != nil {
+ return fmt.Errorf("invalid regular expression %q: %s", m.Pattern, err)
+ }
+ default:
+ return fmt.Errorf("unknown matcher type %q", m.Type)
+ }
+ return nil
+ }
+}
+
+type ClusterPeer interface {
+ AddState(string, cluster.State, prometheus.Registerer) cluster.ClusterChannel
+ Position() int
+ WaitReady(context.Context) error
+}
+
+type AlertingStore interface {
+ store.AlertingStore
+ store.ImageStore
+}
+
+type Alertmanager struct {
+ logger log.Logger
+
+ Settings *setting.Cfg
+ Store AlertingStore
+ fileStore *FileStore
+ Metrics *metrics.Alertmanager
+ NotificationService notifications.Service
+
+ notificationLog *nflog.Log
+ marker types.Marker
+ alerts *mem.Alerts
+ route *dispatch.Route
+ peer ClusterPeer
+ peerTimeout time.Duration
+
+ dispatcher *dispatch.Dispatcher
+ inhibitor *inhibit.Inhibitor
+ // wg is for dispatcher, inhibitor, silences and notifications
+ // Across configuration changes dispatcher and inhibitor are completely replaced, however, silences, notification log and alerts remain the same.
+ // stopc is used to let silences and notifications know we are done.
+ wg sync.WaitGroup
+ stopc chan struct{}
+
+ silencer *silence.Silencer
+ silences *silence.Silences
+
+ // muteTimes is a map where the key is the name of the mute_time_interval
+ // and the value represents all configured time_interval(s)
+ muteTimes map[string][]timeinterval.TimeInterval
+
+ stageMetrics *notify.Metrics
+ dispatcherMetrics *dispatch.DispatcherMetrics
+
+ reloadConfigMtx sync.RWMutex
+ config *apimodels.PostableUserConfig
+ configHash [16]byte
+ orgID int64
+
+ decryptFn channels.GetDecryptedValueFn
+}
+
+func newAlertmanager(ctx context.Context, orgID int64, cfg *setting.Cfg, store AlertingStore, kvStore kvstore.KVStore,
+ peer ClusterPeer, decryptFn channels.GetDecryptedValueFn, ns notifications.Service, m *metrics.Alertmanager) (*Alertmanager, error) {
+ am := &Alertmanager{
+ Settings: cfg,
+ stopc: make(chan struct{}),
+ logger: log.New("alertmanager", "org", orgID),
+ marker: types.NewMarker(m.Registerer),
+ stageMetrics: notify.NewMetrics(m.Registerer),
+ dispatcherMetrics: dispatch.NewDispatcherMetrics(false, m.Registerer),
+ Store: store,
+ peer: peer,
+ peerTimeout: cfg.UnifiedAlerting.HAPeerTimeout,
+ Metrics: m,
+ NotificationService: ns,
+ orgID: orgID,
+ decryptFn: decryptFn,
+ }
+
+ am.fileStore = NewFileStore(am.orgID, kvStore, am.WorkingDirPath())
+
+ nflogFilepath, err := am.fileStore.FilepathFor(ctx, notificationLogFilename)
+ if err != nil {
+ return nil, err
+ }
+ silencesFilePath, err := am.fileStore.FilepathFor(ctx, silencesFilename)
+ if err != nil {
+ return nil, err
+ }
+
+ // Initialize the notification log
+ am.wg.Add(1)
+ am.notificationLog, err = nflog.New(
+ nflog.WithRetention(retentionNotificationsAndSilences),
+ nflog.WithSnapshot(nflogFilepath),
+ nflog.WithMaintenance(maintenanceNotificationAndSilences, am.stopc, am.wg.Done, func() (int64, error) {
+ return am.fileStore.Persist(ctx, notificationLogFilename, am.notificationLog)
+ }),
+ )
+ if err != nil {
+ return nil, fmt.Errorf("unable to initialize the notification log component of alerting: %w", err)
+ }
+ c := am.peer.AddState(fmt.Sprintf("notificationlog:%d", am.orgID), am.notificationLog, m.Registerer)
+ am.notificationLog.SetBroadcast(c.Broadcast)
+
+ // Initialize silences
+ am.silences, err = silence.New(silence.Options{
+ Metrics: m.Registerer,
+ SnapshotFile: silencesFilePath,
+ Retention: retentionNotificationsAndSilences,
+ })
+ if err != nil {
+ return nil, fmt.Errorf("unable to initialize the silencing component of alerting: %w", err)
+ }
+
+ c = am.peer.AddState(fmt.Sprintf("silences:%d", am.orgID), am.silences, m.Registerer)
+ am.silences.SetBroadcast(c.Broadcast)
+
+ am.wg.Add(1)
+ go func() {
+ am.silences.Maintenance(silenceMaintenanceInterval, silencesFilePath, am.stopc, func() (int64, error) {
+ // Delete silences older than the retention period.
+ if _, err := am.silences.GC(); err != nil {
+ am.logger.Error("silence garbage collection", "err", err)
+ // Don't return here - we need to snapshot our state first.
+ }
+
+ // Snapshot our silences to the Grafana KV store
+ return am.fileStore.Persist(ctx, silencesFilename, am.silences)
+ })
+ am.wg.Done()
+ }()
+
+ // Initialize in-memory alerts
+ am.alerts, err = mem.NewAlerts(context.Background(), am.marker, memoryAlertsGCInterval, nil, am.logger)
+ if err != nil {
+ return nil, fmt.Errorf("unable to initialize the alert provider component of alerting: %w", err)
+ }
+
+ return am, nil
+}
+
+func (am *Alertmanager) Ready() bool {
+ // We consider AM as ready only when the config has been
+ // applied at least once successfully. Until then, some objects
+ // can still be nil.
+ am.reloadConfigMtx.RLock()
+ defer am.reloadConfigMtx.RUnlock()
+
+ return am.ready()
+}
+
+func (am *Alertmanager) ready() bool {
+ return am.config != nil
+}
+
+func (am *Alertmanager) StopAndWait() {
+ if am.dispatcher != nil {
+ am.dispatcher.Stop()
+ }
+
+ if am.inhibitor != nil {
+ am.inhibitor.Stop()
+ }
+
+ am.alerts.Close()
+
+ close(am.stopc)
+
+ am.wg.Wait()
+}
+
+// SaveAndApplyDefaultConfig saves the default configuration the database and applies the configuration to the Alertmanager.
+// It rollbacks the save if we fail to apply the configuration.
+func (am *Alertmanager) SaveAndApplyDefaultConfig(ctx context.Context) error {
+ am.reloadConfigMtx.Lock()
+ defer am.reloadConfigMtx.Unlock()
+
+ cmd := &ngmodels.SaveAlertmanagerConfigurationCmd{
+ AlertmanagerConfiguration: am.Settings.UnifiedAlerting.DefaultConfiguration,
+ Default: true,
+ ConfigurationVersion: fmt.Sprintf("v%d", ngmodels.AlertConfigurationVersion),
+ OrgID: am.orgID,
+ }
+
+ cfg, err := Load([]byte(am.Settings.UnifiedAlerting.DefaultConfiguration))
+ if err != nil {
+ return err
+ }
+
+ err = am.Store.SaveAlertmanagerConfigurationWithCallback(ctx, cmd, func() error {
+ if err := am.applyConfig(cfg, []byte(am.Settings.UnifiedAlerting.DefaultConfiguration)); err != nil {
+ return err
+ }
+ return nil
+ })
+ if err != nil {
+ return err
+ }
+
+ return nil
+}
+
+// SaveAndApplyConfig saves the configuration the database and applies the configuration to the Alertmanager.
+// It rollbacks the save if we fail to apply the configuration.
+func (am *Alertmanager) SaveAndApplyConfig(ctx context.Context, cfg *apimodels.PostableUserConfig) error {
+ rawConfig, err := json.Marshal(&cfg)
+ if err != nil {
+ return fmt.Errorf("failed to serialize to the Alertmanager configuration: %w", err)
+ }
+
+ am.reloadConfigMtx.Lock()
+ defer am.reloadConfigMtx.Unlock()
+
+ cmd := &ngmodels.SaveAlertmanagerConfigurationCmd{
+ AlertmanagerConfiguration: string(rawConfig),
+ ConfigurationVersion: fmt.Sprintf("v%d", ngmodels.AlertConfigurationVersion),
+ OrgID: am.orgID,
+ }
+
+ err = am.Store.SaveAlertmanagerConfigurationWithCallback(ctx, cmd, func() error {
+ if err := am.applyConfig(cfg, rawConfig); err != nil {
+ return err
+ }
+ return nil
+ })
+ if err != nil {
+ return err
+ }
+
+ return nil
+}
+
+// ApplyConfig applies the configuration to the Alertmanager.
+func (am *Alertmanager) ApplyConfig(dbCfg *ngmodels.AlertConfiguration) error {
+ var err error
+ cfg, err := Load([]byte(dbCfg.AlertmanagerConfiguration))
+ if err != nil {
+ return fmt.Errorf("failed to parse Alertmanager config: %w", err)
+ }
+
+ am.reloadConfigMtx.Lock()
+ defer am.reloadConfigMtx.Unlock()
+
+ if err = am.applyConfig(cfg, nil); err != nil {
+ return fmt.Errorf("unable to apply configuration: %w", err)
+ }
+ return nil
+}
+
+func (am *Alertmanager) getTemplate() (*template.Template, error) {
+ am.reloadConfigMtx.RLock()
+ defer am.reloadConfigMtx.RUnlock()
+ if !am.ready() {
+ return nil, errors.New("alertmanager is not initialized")
+ }
+ paths := make([]string, 0, len(am.config.TemplateFiles))
+ for name := range am.config.TemplateFiles {
+ paths = append(paths, filepath.Join(am.WorkingDirPath(), name))
+ }
+ return am.templateFromPaths(paths...)
+}
+
+func (am *Alertmanager) templateFromPaths(paths ...string) (*template.Template, error) {
+ tmpl, err := template.FromGlobs(paths...)
+ if err != nil {
+ return nil, err
+ }
+ externalURL, err := url.Parse(am.Settings.AppURL)
+ if err != nil {
+ return nil, err
+ }
+ tmpl.ExternalURL = externalURL
+ return tmpl, nil
+}
+
+func (am *Alertmanager) buildMuteTimesMap(muteTimeIntervals []config.MuteTimeInterval) map[string][]timeinterval.TimeInterval {
+ muteTimes := make(map[string][]timeinterval.TimeInterval, len(muteTimeIntervals))
+ for _, ti := range muteTimeIntervals {
+ muteTimes[ti.Name] = ti.TimeIntervals
+ }
+ return muteTimes
+}
+
+// applyConfig applies a new configuration by re-initializing all components using the configuration provided.
+// It is not safe to call concurrently.
+func (am *Alertmanager) applyConfig(cfg *apimodels.PostableUserConfig, rawConfig []byte) (err error) {
+ // First, let's make sure this config is not already loaded
+ var configChanged bool
+ if rawConfig == nil {
+ enc, err := json.Marshal(cfg.AlertmanagerConfig)
+ if err != nil {
+ // In theory, this should never happen.
+ return err
+ }
+ rawConfig = enc
+ }
+
+ if am.configHash != md5.Sum(rawConfig) {
+ configChanged = true
+ }
+
+ if cfg.TemplateFiles == nil {
+ cfg.TemplateFiles = map[string]string{}
+ }
+ cfg.TemplateFiles["__default__.tmpl"] = channels.DefaultTemplateString
+
+ // next, we need to make sure we persist the templates to disk.
+ paths, templatesChanged, err := PersistTemplates(cfg, am.WorkingDirPath())
+ if err != nil {
+ return err
+ }
+
+ // If neither the configuration nor templates have changed, we've got nothing to do.
+ if !configChanged && !templatesChanged {
+ am.logger.Debug("neither config nor template have changed, skipping configuration sync.")
+ return nil
+ }
+
+ // With the templates persisted, create the template list using the paths.
+ tmpl, err := am.templateFromPaths(paths...)
+ if err != nil {
+ return err
+ }
+
+ // Finally, build the integrations map using the receiver configuration and templates.
+ integrationsMap, err := am.buildIntegrationsMap(cfg.AlertmanagerConfig.Receivers, tmpl)
+ if err != nil {
+ return fmt.Errorf("failed to build integration map: %w", err)
+ }
+
+ // Now, let's put together our notification pipeline
+ routingStage := make(notify.RoutingStage, len(integrationsMap))
+
+ if am.inhibitor != nil {
+ am.inhibitor.Stop()
+ }
+ if am.dispatcher != nil {
+ am.dispatcher.Stop()
+ }
+
+ am.inhibitor = inhibit.NewInhibitor(am.alerts, cfg.AlertmanagerConfig.InhibitRules, am.marker, am.logger)
+ am.muteTimes = am.buildMuteTimesMap(cfg.AlertmanagerConfig.MuteTimeIntervals)
+ am.silencer = silence.NewSilencer(am.silences, am.marker, am.logger)
+
+ meshStage := notify.NewGossipSettleStage(am.peer)
+ inhibitionStage := notify.NewMuteStage(am.inhibitor)
+ timeMuteStage := notify.NewTimeMuteStage(am.muteTimes)
+ silencingStage := notify.NewMuteStage(am.silencer)
+ for name := range integrationsMap {
+ stage := am.createReceiverStage(name, integrationsMap[name], am.waitFunc, am.notificationLog)
+ routingStage[name] = notify.MultiStage{meshStage, silencingStage, timeMuteStage, inhibitionStage, stage}
+ }
+
+ am.route = dispatch.NewRoute(cfg.AlertmanagerConfig.Route.AsAMRoute(), nil)
+ am.dispatcher = dispatch.NewDispatcher(am.alerts, am.route, routingStage, am.marker, am.timeoutFunc, &nilLimits{}, am.logger, am.dispatcherMetrics)
+
+ am.wg.Add(1)
+ go func() {
+ defer am.wg.Done()
+ am.dispatcher.Run()
+ }()
+
+ am.wg.Add(1)
+ go func() {
+ defer am.wg.Done()
+ am.inhibitor.Run()
+ }()
+
+ am.config = cfg
+ am.configHash = md5.Sum(rawConfig)
+
+ return nil
+}
+
+func (am *Alertmanager) WorkingDirPath() string {
+ return filepath.Join(am.Settings.DataPath, workingDir, strconv.Itoa(int(am.orgID)))
+}
+
+// buildIntegrationsMap builds a map of name to the list of Grafana integration notifiers off of a list of receiver config.
+func (am *Alertmanager) buildIntegrationsMap(receivers []*apimodels.PostableApiReceiver, templates *template.Template) (map[string][]notify.Integration, error) {
+ integrationsMap := make(map[string][]notify.Integration, len(receivers))
+ for _, receiver := range receivers {
+ integrations, err := am.buildReceiverIntegrations(receiver, templates)
+ if err != nil {
+ return nil, err
+ }
+ integrationsMap[receiver.Name] = integrations
+ }
+
+ return integrationsMap, nil
+}
+
+// buildReceiverIntegrations builds a list of integration notifiers off of a receiver config.
+func (am *Alertmanager) buildReceiverIntegrations(receiver *apimodels.PostableApiReceiver, tmpl *template.Template) ([]notify.Integration, error) {
+ var integrations []notify.Integration
+ for i, r := range receiver.GrafanaManagedReceivers {
+ n, err := am.buildReceiverIntegration(r, tmpl)
+ if err != nil {
+ return nil, err
+ }
+ integrations = append(integrations, notify.NewIntegration(n, n, r.Type, i))
+ }
+ return integrations, nil
+}
+
+func (am *Alertmanager) buildReceiverIntegration(r *apimodels.PostableGrafanaReceiver, tmpl *template.Template) (channels.NotificationChannel, error) {
+ // secure settings are already encrypted at this point
+ secureSettings := make(map[string][]byte, len(r.SecureSettings))
+
+ for k, v := range r.SecureSettings {
+ d, err := base64.StdEncoding.DecodeString(v)
+ if err != nil {
+ return nil, InvalidReceiverError{
+ Receiver: r,
+ Err: errors.New("failed to decode secure setting"),
+ }
+ }
+ secureSettings[k] = d
+ }
+
+ var (
+ cfg = &channels.NotificationChannelConfig{
+ UID: r.UID,
+ OrgID: am.orgID,
+ Name: r.Name,
+ Type: r.Type,
+ DisableResolveMessage: r.DisableResolveMessage,
+ Settings: r.Settings,
+ SecureSettings: secureSettings,
+ }
+ )
+ factoryConfig, err := channels.NewFactoryConfig(cfg, am.NotificationService, am.decryptFn, tmpl, am.Store)
+ if err != nil {
+ return nil, InvalidReceiverError{
+ Receiver: r,
+ Err: err,
+ }
+ }
+ receiverFactory, exists := channels.Factory(r.Type)
+ if !exists {
+ return nil, InvalidReceiverError{
+ Receiver: r,
+ Err: fmt.Errorf("notifier %s is not supported", r.Type),
+ }
+ }
+ n, err := receiverFactory(factoryConfig)
+ if err != nil {
+ return nil, InvalidReceiverError{
+ Receiver: r,
+ Err: err,
+ }
+ }
+ return n, nil
+}
+
+// PutAlerts receives the alerts and then sends them through the corresponding route based on whenever the alert has a receiver embedded or not
+func (am *Alertmanager) PutAlerts(postableAlerts apimodels.PostableAlerts) error {
+ now := time.Now()
+ alerts := make([]*types.Alert, 0, len(postableAlerts.PostableAlerts))
+ var validationErr *AlertValidationError
+ for _, a := range postableAlerts.PostableAlerts {
+ alert := &types.Alert{
+ Alert: model.Alert{
+ Labels: model.LabelSet{},
+ Annotations: model.LabelSet{},
+ StartsAt: time.Time(a.StartsAt),
+ EndsAt: time.Time(a.EndsAt),
+ GeneratorURL: a.GeneratorURL.String(),
+ },
+ UpdatedAt: now,
+ }
+
+ for k, v := range a.Labels {
+ if len(v) == 0 || k == ngmodels.NamespaceUIDLabel { // Skip empty and namespace UID labels.
+ continue
+ }
+ alert.Alert.Labels[model.LabelName(k)] = model.LabelValue(v)
+ }
+ for k, v := range a.Annotations {
+ if len(v) == 0 { // Skip empty annotation.
+ continue
+ }
+ alert.Alert.Annotations[model.LabelName(k)] = model.LabelValue(v)
+ }
+
+ // Ensure StartsAt is set.
+ if alert.StartsAt.IsZero() {
+ if alert.EndsAt.IsZero() {
+ alert.StartsAt = now
+ } else {
+ alert.StartsAt = alert.EndsAt
+ }
+ }
+ // If no end time is defined, set a timeout after which an alert
+ // is marked resolved if it is not updated.
+ if alert.EndsAt.IsZero() {
+ alert.Timeout = true
+ alert.EndsAt = now.Add(defaultResolveTimeout)
+ }
+
+ if alert.EndsAt.After(now) {
+ am.Metrics.Firing().Inc()
+ } else {
+ am.Metrics.Resolved().Inc()
+ }
+
+ if err := validateAlert(alert); err != nil {
+ if validationErr == nil {
+ validationErr = &AlertValidationError{}
+ }
+ validationErr.Alerts = append(validationErr.Alerts, a)
+ validationErr.Errors = append(validationErr.Errors, err)
+ am.Metrics.Invalid().Inc()
+ continue
+ }
+
+ alerts = append(alerts, alert)
+ }
+
+ if err := am.alerts.Put(alerts...); err != nil {
+ // Notification sending alert takes precedence over validation errors.
+ return err
+ }
+ if validationErr != nil {
+ // Even if validationErr is nil, the require.NoError fails on it.
+ return validationErr
+ }
+ return nil
+}
+
+// validateAlert is a.Validate() while additionally allowing
+// space for label and annotation names.
+func validateAlert(a *types.Alert) error {
+ if a.StartsAt.IsZero() {
+ return fmt.Errorf("start time missing")
+ }
+ if !a.EndsAt.IsZero() && a.EndsAt.Before(a.StartsAt) {
+ return fmt.Errorf("start time must be before end time")
+ }
+ if err := validateLabelSet(a.Labels); err != nil {
+ return fmt.Errorf("invalid label set: %s", err)
+ }
+ if len(a.Labels) == 0 {
+ return fmt.Errorf("at least one label pair required")
+ }
+ if err := validateLabelSet(a.Annotations); err != nil {
+ return fmt.Errorf("invalid annotations: %s", err)
+ }
+ return nil
+}
+
+// validateLabelSet is ls.Validate() while additionally allowing
+// space for label names.
+func validateLabelSet(ls model.LabelSet) error {
+ for ln, lv := range ls {
+ if !isValidLabelName(ln) {
+ return fmt.Errorf("invalid name %q", ln)
+ }
+ if !lv.IsValid() {
+ return fmt.Errorf("invalid value %q", lv)
+ }
+ }
+ return nil
+}
+
+// isValidLabelName is ln.IsValid() without restrictions other than it can not be empty.
+// The regex for Prometheus data model is ^[a-zA-Z_][a-zA-Z0-9_]*$.
+func isValidLabelName(ln model.LabelName) bool {
+ if len(ln) == 0 {
+ return false
+ }
+
+ return utf8.ValidString(string(ln))
+}
+
+// AlertValidationError is the error capturing the validation errors
+// faced on the alerts.
+type AlertValidationError struct {
+ Alerts []amv2.PostableAlert
+ Errors []error // Errors[i] refers to Alerts[i].
+}
+
+func (e AlertValidationError) Error() string {
+ errMsg := ""
+ if len(e.Errors) != 0 {
+ errMsg = e.Errors[0].Error()
+ for _, e := range e.Errors[1:] {
+ errMsg += ";" + e.Error()
+ }
+ }
+ return errMsg
+}
+
+// createReceiverStage creates a pipeline of stages for a receiver.
+func (am *Alertmanager) createReceiverStage(name string, integrations []notify.Integration, wait func() time.Duration, notificationLog notify.NotificationLog) notify.Stage {
+ var fs notify.FanoutStage
+ for i := range integrations {
+ recv := &nflogpb.Receiver{
+ GroupName: name,
+ Integration: integrations[i].Name(),
+ Idx: uint32(integrations[i].Index()),
+ }
+ var s notify.MultiStage
+ s = append(s, notify.NewWaitStage(wait))
+ s = append(s, notify.NewDedupStage(&integrations[i], notificationLog, recv))
+ s = append(s, notify.NewRetryStage(integrations[i], name, am.stageMetrics))
+ s = append(s, notify.NewSetNotifiesStage(notificationLog, recv))
+
+ fs = append(fs, s)
+ }
+ return fs
+}
+
+func (am *Alertmanager) waitFunc() time.Duration {
+ return time.Duration(am.peer.Position()) * am.peerTimeout
+}
+
+func (am *Alertmanager) timeoutFunc(d time.Duration) time.Duration {
+ // time.Duration d relates to the receiver's group_interval. Even with a group interval of 1s,
+ // we need to make sure (non-position-0) peers in the cluster wait before flushing the notifications.
+ if d < notify.MinTimeout {
+ d = notify.MinTimeout
+ }
+ return d + am.waitFunc()
+}
+
+type nilLimits struct{}
+
+func (n nilLimits) MaxNumberOfAggregationGroups() int { return 0 }
diff --git a/alerting/alertmanager_config.go b/alerting/alertmanager_config.go
new file mode 100644
index 00000000..e564813f
--- /dev/null
+++ b/alerting/alertmanager_config.go
@@ -0,0 +1,163 @@
+package notifier
+
+import (
+ "context"
+ "errors"
+ "fmt"
+
+ "github.com/grafana/grafana/pkg/services/ngalert/api/tooling/definitions"
+ "github.com/grafana/grafana/pkg/services/ngalert/models"
+ "github.com/grafana/grafana/pkg/services/ngalert/store"
+)
+
+type UnknownReceiverError struct {
+ UID string
+}
+
+func (e UnknownReceiverError) Error() string {
+ return fmt.Sprintf("unknown receiver: %s", e.UID)
+}
+
+type AlertmanagerConfigRejectedError struct {
+ Inner error
+}
+
+func (e AlertmanagerConfigRejectedError) Error() string {
+ return fmt.Sprintf("failed to save and apply Alertmanager configuration: %s", e.Inner.Error())
+}
+
+type configurationStore interface {
+ GetLatestAlertmanagerConfiguration(ctx context.Context, query *models.GetLatestAlertmanagerConfigurationQuery) error
+}
+
+func (moa *MultiOrgAlertmanager) GetAlertmanagerConfiguration(ctx context.Context, org int64) (definitions.GettableUserConfig, error) {
+ query := models.GetLatestAlertmanagerConfigurationQuery{OrgID: org}
+ err := moa.configStore.GetLatestAlertmanagerConfiguration(ctx, &query)
+ if err != nil {
+ return definitions.GettableUserConfig{}, fmt.Errorf("failed to get latest configuration: %w", err)
+ }
+ cfg, err := Load([]byte(query.Result.AlertmanagerConfiguration))
+ if err != nil {
+ return definitions.GettableUserConfig{}, fmt.Errorf("failed to unmarshal alertmanager configuration: %w", err)
+ }
+
+ result := definitions.GettableUserConfig{
+ TemplateFiles: cfg.TemplateFiles,
+ AlertmanagerConfig: definitions.GettableApiAlertingConfig{
+ Config: cfg.AlertmanagerConfig.Config,
+ },
+ }
+
+ for _, recv := range cfg.AlertmanagerConfig.Receivers {
+ receivers := make([]*definitions.GettableGrafanaReceiver, 0, len(recv.PostableGrafanaReceivers.GrafanaManagedReceivers))
+ for _, pr := range recv.PostableGrafanaReceivers.GrafanaManagedReceivers {
+ secureFields := make(map[string]bool, len(pr.SecureSettings))
+ for k := range pr.SecureSettings {
+ decryptedValue, err := moa.Crypto.getDecryptedSecret(pr, k)
+ if err != nil {
+ return definitions.GettableUserConfig{}, fmt.Errorf("failed to decrypt stored secure setting: %w", err)
+ }
+ if decryptedValue == "" {
+ continue
+ }
+ secureFields[k] = true
+ }
+ gr := definitions.GettableGrafanaReceiver{
+ UID: pr.UID,
+ Name: pr.Name,
+ Type: pr.Type,
+ DisableResolveMessage: pr.DisableResolveMessage,
+ Settings: pr.Settings,
+ SecureFields: secureFields,
+ }
+ receivers = append(receivers, &gr)
+ }
+ gettableApiReceiver := definitions.GettableApiReceiver{
+ GettableGrafanaReceivers: definitions.GettableGrafanaReceivers{
+ GrafanaManagedReceivers: receivers,
+ },
+ }
+ gettableApiReceiver.Name = recv.Name
+ result.AlertmanagerConfig.Receivers = append(result.AlertmanagerConfig.Receivers, &gettableApiReceiver)
+ }
+
+ result, err = moa.mergeProvenance(ctx, result, org)
+ if err != nil {
+ return definitions.GettableUserConfig{}, err
+ }
+
+ return result, nil
+}
+
+func (moa *MultiOrgAlertmanager) ApplyAlertmanagerConfiguration(ctx context.Context, org int64, config definitions.PostableUserConfig) error {
+ // Get the last known working configuration
+ query := models.GetLatestAlertmanagerConfigurationQuery{OrgID: org}
+ if err := moa.configStore.GetLatestAlertmanagerConfiguration(ctx, &query); err != nil {
+ // If we don't have a configuration there's nothing for us to know and we should just continue saving the new one
+ if !errors.Is(err, store.ErrNoAlertmanagerConfiguration) {
+ return fmt.Errorf("failed to get latest configuration %w", err)
+ }
+ }
+
+ if err := moa.Crypto.LoadSecureSettings(ctx, org, config.AlertmanagerConfig.Receivers); err != nil {
+ return err
+ }
+
+ if err := config.ProcessConfig(moa.Crypto.Encrypt); err != nil {
+ return fmt.Errorf("failed to post process Alertmanager configuration: %w", err)
+ }
+
+ am, err := moa.AlertmanagerFor(org)
+ if err != nil {
+ // It's okay if the alertmanager isn't ready yet, we're changing its config anyway.
+ if !errors.Is(err, ErrAlertmanagerNotReady) {
+ return err
+ }
+ }
+
+ if err := am.SaveAndApplyConfig(ctx, &config); err != nil {
+ moa.logger.Error("unable to save and apply alertmanager configuration", "err", err)
+ return AlertmanagerConfigRejectedError{err}
+ }
+
+ return nil
+}
+
+func (moa *MultiOrgAlertmanager) mergeProvenance(ctx context.Context, config definitions.GettableUserConfig, org int64) (definitions.GettableUserConfig, error) {
+ if config.AlertmanagerConfig.Route != nil {
+ provenance, err := moa.ProvStore.GetProvenance(ctx, config.AlertmanagerConfig.Route, org)
+ if err != nil {
+ return definitions.GettableUserConfig{}, err
+ }
+ config.AlertmanagerConfig.Route.Provenance = provenance
+ }
+
+ cp := definitions.EmbeddedContactPoint{}
+ cpProvs, err := moa.ProvStore.GetProvenances(ctx, org, cp.ResourceType())
+ if err != nil {
+ return definitions.GettableUserConfig{}, err
+ }
+ for _, receiver := range config.AlertmanagerConfig.Receivers {
+ for _, contactPoint := range receiver.GrafanaManagedReceivers {
+ if provenance, exists := cpProvs[contactPoint.UID]; exists {
+ contactPoint.Provenance = provenance
+ }
+ }
+ }
+
+ tmpl := definitions.MessageTemplate{}
+ tmplProvs, err := moa.ProvStore.GetProvenances(ctx, org, tmpl.ResourceType())
+ if err != nil {
+ return definitions.GettableUserConfig{}, nil
+ }
+ config.TemplateFileProvenances = tmplProvs
+
+ mt := definitions.MuteTimeInterval{}
+ mtProvs, err := moa.ProvStore.GetProvenances(ctx, org, mt.ResourceType())
+ if err != nil {
+ return definitions.GettableUserConfig{}, nil
+ }
+ config.AlertmanagerConfig.MuteTimeProvenances = mtProvs
+
+ return config, nil
+}
diff --git a/alerting/alertmanager_test.go b/alerting/alertmanager_test.go
new file mode 100644
index 00000000..c308e800
--- /dev/null
+++ b/alerting/alertmanager_test.go
@@ -0,0 +1,416 @@
+package notifier
+
+import (
+ "context"
+ "errors"
+ "sort"
+ "testing"
+ "time"
+
+ "github.com/grafana/grafana/pkg/services/dashboards"
+ "github.com/grafana/grafana/pkg/services/secrets/database"
+
+ "github.com/go-openapi/strfmt"
+ "github.com/prometheus/alertmanager/api/v2/models"
+ "github.com/prometheus/alertmanager/provider/mem"
+ "github.com/prometheus/alertmanager/types"
+ "github.com/prometheus/client_golang/prometheus"
+ "github.com/prometheus/common/model"
+ "github.com/stretchr/testify/require"
+
+ "github.com/grafana/grafana/pkg/infra/log"
+ apimodels "github.com/grafana/grafana/pkg/services/ngalert/api/tooling/definitions"
+ "github.com/grafana/grafana/pkg/services/ngalert/metrics"
+ "github.com/grafana/grafana/pkg/services/ngalert/store"
+ secretsManager "github.com/grafana/grafana/pkg/services/secrets/manager"
+ "github.com/grafana/grafana/pkg/services/sqlstore"
+ "github.com/grafana/grafana/pkg/setting"
+)
+
+func setupAMTest(t *testing.T) *Alertmanager {
+ dir := t.TempDir()
+ cfg := &setting.Cfg{
+ DataPath: dir,
+ }
+
+ m := metrics.NewAlertmanagerMetrics(prometheus.NewRegistry())
+ sqlStore := sqlstore.InitTestDB(t)
+ s := &store.DBstore{
+ BaseInterval: 10 * time.Second,
+ DefaultInterval: 60 * time.Second,
+ SQLStore: sqlStore,
+ Logger: log.New("alertmanager-test"),
+ DashboardService: dashboards.NewFakeDashboardService(t),
+ }
+
+ kvStore := NewFakeKVStore(t)
+ secretsService := secretsManager.SetupTestService(t, database.ProvideSecretsStore(sqlStore))
+ decryptFn := secretsService.GetDecryptedValue
+ am, err := newAlertmanager(context.Background(), 1, cfg, s, kvStore, &NilPeer{}, decryptFn, nil, m)
+ require.NoError(t, err)
+ return am
+}
+
+func TestPutAlert(t *testing.T) {
+ am := setupAMTest(t)
+
+ startTime := time.Now()
+ endTime := startTime.Add(2 * time.Hour)
+
+ cases := []struct {
+ title string
+ postableAlerts apimodels.PostableAlerts
+ expAlerts func(now time.Time) []*types.Alert
+ expError *AlertValidationError
+ }{
+ {
+ title: "Valid alerts with different start/end set",
+ postableAlerts: apimodels.PostableAlerts{
+ PostableAlerts: []models.PostableAlert{
+ { // Start and end set.
+ Annotations: models.LabelSet{"msg": "Alert1 annotation"},
+ Alert: models.Alert{
+ Labels: models.LabelSet{"alertname": "Alert1"},
+ GeneratorURL: "http://localhost/url1",
+ },
+ StartsAt: strfmt.DateTime(startTime),
+ EndsAt: strfmt.DateTime(endTime),
+ }, { // Only end is set.
+ Annotations: models.LabelSet{"msg": "Alert2 annotation"},
+ Alert: models.Alert{
+ Labels: models.LabelSet{"alertname": "Alert2"},
+ GeneratorURL: "http://localhost/url2",
+ },
+ StartsAt: strfmt.DateTime{},
+ EndsAt: strfmt.DateTime(endTime),
+ }, { // Only start is set.
+ Annotations: models.LabelSet{"msg": "Alert3 annotation"},
+ Alert: models.Alert{
+ Labels: models.LabelSet{"alertname": "Alert3"},
+ GeneratorURL: "http://localhost/url3",
+ },
+ StartsAt: strfmt.DateTime(startTime),
+ EndsAt: strfmt.DateTime{},
+ }, { // Both start and end are not set.
+ Annotations: models.LabelSet{"msg": "Alert4 annotation"},
+ Alert: models.Alert{
+ Labels: models.LabelSet{"alertname": "Alert4"},
+ GeneratorURL: "http://localhost/url4",
+ },
+ StartsAt: strfmt.DateTime{},
+ EndsAt: strfmt.DateTime{},
+ },
+ },
+ },
+ expAlerts: func(now time.Time) []*types.Alert {
+ return []*types.Alert{
+ {
+ Alert: model.Alert{
+ Annotations: model.LabelSet{"msg": "Alert1 annotation"},
+ Labels: model.LabelSet{"alertname": "Alert1"},
+ StartsAt: startTime,
+ EndsAt: endTime,
+ GeneratorURL: "http://localhost/url1",
+ },
+ UpdatedAt: now,
+ }, {
+ Alert: model.Alert{
+ Annotations: model.LabelSet{"msg": "Alert2 annotation"},
+ Labels: model.LabelSet{"alertname": "Alert2"},
+ StartsAt: endTime,
+ EndsAt: endTime,
+ GeneratorURL: "http://localhost/url2",
+ },
+ UpdatedAt: now,
+ }, {
+ Alert: model.Alert{
+ Annotations: model.LabelSet{"msg": "Alert3 annotation"},
+ Labels: model.LabelSet{"alertname": "Alert3"},
+ StartsAt: startTime,
+ EndsAt: now.Add(defaultResolveTimeout),
+ GeneratorURL: "http://localhost/url3",
+ },
+ UpdatedAt: now,
+ Timeout: true,
+ }, {
+ Alert: model.Alert{
+ Annotations: model.LabelSet{"msg": "Alert4 annotation"},
+ Labels: model.LabelSet{"alertname": "Alert4"},
+ StartsAt: now,
+ EndsAt: now.Add(defaultResolveTimeout),
+ GeneratorURL: "http://localhost/url4",
+ },
+ UpdatedAt: now,
+ Timeout: true,
+ },
+ }
+ },
+ }, {
+ title: "Removing empty labels and annotations",
+ postableAlerts: apimodels.PostableAlerts{
+ PostableAlerts: []models.PostableAlert{
+ {
+ Annotations: models.LabelSet{"msg": "Alert4 annotation", "empty": ""},
+ Alert: models.Alert{
+ Labels: models.LabelSet{"alertname": "Alert4", "emptylabel": ""},
+ GeneratorURL: "http://localhost/url1",
+ },
+ StartsAt: strfmt.DateTime{},
+ EndsAt: strfmt.DateTime{},
+ },
+ },
+ },
+ expAlerts: func(now time.Time) []*types.Alert {
+ return []*types.Alert{
+ {
+ Alert: model.Alert{
+ Annotations: model.LabelSet{"msg": "Alert4 annotation"},
+ Labels: model.LabelSet{"alertname": "Alert4"},
+ StartsAt: now,
+ EndsAt: now.Add(defaultResolveTimeout),
+ GeneratorURL: "http://localhost/url1",
+ },
+ UpdatedAt: now,
+ Timeout: true,
+ },
+ }
+ },
+ }, {
+ title: "Allow spaces in label and annotation name",
+ postableAlerts: apimodels.PostableAlerts{
+ PostableAlerts: []models.PostableAlert{
+ {
+ Annotations: models.LabelSet{"Dashboard URL": "http://localhost:3000"},
+ Alert: models.Alert{
+ Labels: models.LabelSet{"alertname": "Alert4", "Spaced Label": "works"},
+ GeneratorURL: "http://localhost/url1",
+ },
+ StartsAt: strfmt.DateTime{},
+ EndsAt: strfmt.DateTime{},
+ },
+ },
+ },
+ expAlerts: func(now time.Time) []*types.Alert {
+ return []*types.Alert{
+ {
+ Alert: model.Alert{
+ Annotations: model.LabelSet{"Dashboard URL": "http://localhost:3000"},
+ Labels: model.LabelSet{"alertname": "Alert4", "Spaced Label": "works"},
+ StartsAt: now,
+ EndsAt: now.Add(defaultResolveTimeout),
+ GeneratorURL: "http://localhost/url1",
+ },
+ UpdatedAt: now,
+ Timeout: true,
+ },
+ }
+ },
+ }, {
+ title: "Special characters in labels",
+ postableAlerts: apimodels.PostableAlerts{
+ PostableAlerts: []models.PostableAlert{
+ {
+ Alert: models.Alert{
+ Labels: models.LabelSet{"alertname$": "Alert1", "az3-- __...++!!!£@@312312": "1"},
+ },
+ },
+ },
+ },
+ expAlerts: func(now time.Time) []*types.Alert {
+ return []*types.Alert{
+ {
+ Alert: model.Alert{
+ Labels: model.LabelSet{"alertname$": "Alert1", "az3-- __...++!!!£@@312312": "1"},
+ Annotations: model.LabelSet{},
+ StartsAt: now,
+ EndsAt: now.Add(defaultResolveTimeout),
+ GeneratorURL: "",
+ },
+ UpdatedAt: now,
+ Timeout: true,
+ },
+ }
+ },
+ }, {
+ title: "Special characters in annotations",
+ postableAlerts: apimodels.PostableAlerts{
+ PostableAlerts: []models.PostableAlert{
+ {
+ Annotations: models.LabelSet{"az3-- __...++!!!£@@312312": "Alert4 annotation"},
+ Alert: models.Alert{
+ Labels: models.LabelSet{"alertname": "Alert4"},
+ },
+ },
+ },
+ },
+ expAlerts: func(now time.Time) []*types.Alert {
+ return []*types.Alert{
+ {
+ Alert: model.Alert{
+ Labels: model.LabelSet{"alertname": "Alert4"},
+ Annotations: model.LabelSet{"az3-- __...++!!!£@@312312": "Alert4 annotation"},
+ StartsAt: now,
+ EndsAt: now.Add(defaultResolveTimeout),
+ GeneratorURL: "",
+ },
+ UpdatedAt: now,
+ Timeout: true,
+ },
+ }
+ },
+ }, {
+ title: "No labels after removing empty",
+ postableAlerts: apimodels.PostableAlerts{
+ PostableAlerts: []models.PostableAlert{
+ {
+ Alert: models.Alert{
+ Labels: models.LabelSet{"alertname": ""},
+ },
+ },
+ },
+ },
+ expError: &AlertValidationError{
+ Alerts: []models.PostableAlert{
+ {
+ Alert: models.Alert{
+ Labels: models.LabelSet{"alertname": ""},
+ },
+ },
+ },
+ Errors: []error{errors.New("at least one label pair required")},
+ },
+ }, {
+ title: "Start should be before end",
+ postableAlerts: apimodels.PostableAlerts{
+ PostableAlerts: []models.PostableAlert{
+ {
+ Alert: models.Alert{
+ Labels: models.LabelSet{"alertname": ""},
+ },
+ StartsAt: strfmt.DateTime(endTime),
+ EndsAt: strfmt.DateTime(startTime),
+ },
+ },
+ },
+ expError: &AlertValidationError{
+ Alerts: []models.PostableAlert{
+ {
+ Alert: models.Alert{
+ Labels: models.LabelSet{"alertname": ""},
+ },
+ StartsAt: strfmt.DateTime(endTime),
+ EndsAt: strfmt.DateTime(startTime),
+ },
+ },
+ Errors: []error{errors.New("start time must be before end time")},
+ },
+ },
+ }
+
+ for _, c := range cases {
+ var err error
+ t.Run(c.title, func(t *testing.T) {
+ r := prometheus.NewRegistry()
+ am.marker = types.NewMarker(r)
+ am.alerts, err = mem.NewAlerts(context.Background(), am.marker, 15*time.Minute, nil, am.logger)
+ require.NoError(t, err)
+
+ alerts := []*types.Alert{}
+ err := am.PutAlerts(c.postableAlerts)
+ if c.expError != nil {
+ require.Error(t, err)
+ require.Equal(t, c.expError, err)
+ require.Equal(t, 0, len(alerts))
+ return
+ }
+ require.NoError(t, err)
+
+ iter := am.alerts.GetPending()
+ defer iter.Close()
+ for a := range iter.Next() {
+ alerts = append(alerts, a)
+ }
+
+ // We take the "now" time from one of the UpdatedAt.
+ now := alerts[0].UpdatedAt
+ expAlerts := c.expAlerts(now)
+
+ sort.Sort(types.AlertSlice(expAlerts))
+ sort.Sort(types.AlertSlice(alerts))
+
+ require.Equal(t, expAlerts, alerts)
+ })
+ }
+}
+
+// Tests cleanup of expired Silences. We rely on prometheus/alertmanager for
+// our alert silencing functionality, so we rely on its tests. However, we
+// implement a custom maintenance function for silences, because we snapshot
+// our data differently, so we test that functionality.
+func TestSilenceCleanup(t *testing.T) {
+ require := require.New(t)
+
+ oldRetention := retentionNotificationsAndSilences
+ retentionNotificationsAndSilences = 30 * time.Millisecond
+ oldMaintenance := silenceMaintenanceInterval
+ silenceMaintenanceInterval = 15 * time.Millisecond
+ t.Cleanup(
+ func() {
+ retentionNotificationsAndSilences = oldRetention
+ silenceMaintenanceInterval = oldMaintenance
+ })
+
+ am := setupAMTest(t)
+ now := time.Now()
+ dt := func(t time.Time) strfmt.DateTime { return strfmt.DateTime(t) }
+
+ makeSilence := func(comment string, createdBy string,
+ startsAt, endsAt strfmt.DateTime, matchers models.Matchers) *apimodels.PostableSilence {
+ return &apimodels.PostableSilence{
+ ID: "",
+ Silence: models.Silence{
+ Comment: &comment,
+ CreatedBy: &createdBy,
+ StartsAt: &startsAt,
+ EndsAt: &endsAt,
+ Matchers: matchers,
+ },
+ }
+ }
+
+ tru := true
+ testString := "testName"
+ matchers := models.Matchers{&models.Matcher{Name: &testString, IsEqual: &tru, IsRegex: &tru, Value: &testString}}
+ // Create silences - one in the future, one currently active, one expired but
+ // retained, one expired and not retained.
+ silences := []*apimodels.PostableSilence{
+ // Active in future
+ makeSilence("", "tests", dt(now.Add(5*time.Hour)), dt(now.Add(6*time.Hour)), matchers),
+ // Active now
+ makeSilence("", "tests", dt(now.Add(-5*time.Hour)), dt(now.Add(6*time.Hour)), matchers),
+ // Expiring soon.
+ makeSilence("", "tests", dt(now.Add(-5*time.Hour)), dt(now.Add(5*time.Second)), matchers),
+ // Expiring *very* soon
+ makeSilence("", "tests", dt(now.Add(-5*time.Hour)), dt(now.Add(2*time.Second)), matchers),
+ }
+
+ for _, s := range silences {
+ _, err := am.CreateSilence(s)
+ require.NoError(err)
+ }
+
+ // Let enough time pass for the maintenance window to run.
+ require.Eventually(func() bool {
+ // So, what silences do we have now?
+ found, err := am.ListSilences(nil)
+ require.NoError(err)
+ return len(found) == 3
+ }, 3*time.Second, 150*time.Millisecond)
+
+ // Wait again for another silence to expire.
+ require.Eventually(func() bool {
+ found, err := am.ListSilences(nil)
+ require.NoError(err)
+ return len(found) == 2
+ }, 6*time.Second, 150*time.Millisecond)
+}
diff --git a/alerting/alerts.go b/alerting/alerts.go
new file mode 100644
index 00000000..0bc795d9
--- /dev/null
+++ b/alerting/alerts.go
@@ -0,0 +1,228 @@
+package notifier
+
+import (
+ "fmt"
+ "regexp"
+ "sort"
+ "time"
+
+ apimodels "github.com/grafana/grafana/pkg/services/ngalert/api/tooling/definitions"
+ v2 "github.com/prometheus/alertmanager/api/v2"
+ "github.com/prometheus/alertmanager/dispatch"
+ "github.com/prometheus/alertmanager/pkg/labels"
+ "github.com/prometheus/alertmanager/types"
+ prometheus_model "github.com/prometheus/common/model"
+)
+
+var (
+ ErrGetAlertsInternal = fmt.Errorf("unable to retrieve alerts(s) due to an internal error")
+ ErrGetAlertsUnavailable = fmt.Errorf("unable to retrieve alerts(s) as alertmanager is not initialised yet")
+ ErrGetAlertsBadPayload = fmt.Errorf("unable to retrieve alerts")
+ ErrGetAlertGroupsBadPayload = fmt.Errorf("unable to retrieve alerts groups")
+)
+
+func (am *Alertmanager) GetAlerts(active, silenced, inhibited bool, filter []string, receivers string) (apimodels.GettableAlerts, error) {
+ var (
+ // Initialize result slice to prevent api returning `null` when there
+ // are no alerts present
+ res = apimodels.GettableAlerts{}
+ )
+
+ if !am.Ready() {
+ return res, ErrGetAlertsUnavailable
+ }
+
+ matchers, err := parseFilter(filter)
+ if err != nil {
+ am.logger.Error("failed to parse matchers", "err", err)
+ return nil, fmt.Errorf("%s: %w", err.Error(), ErrGetAlertsBadPayload)
+ }
+
+ receiverFilter, err := parseReceivers(receivers)
+ if err != nil {
+ am.logger.Error("failed to parse receiver regex", "err", err)
+ return nil, fmt.Errorf("%s: %w", err.Error(), ErrGetAlertsBadPayload)
+ }
+
+ alerts := am.alerts.GetPending()
+ defer alerts.Close()
+
+ alertFilter := am.alertFilter(matchers, silenced, inhibited, active)
+ now := time.Now()
+
+ am.reloadConfigMtx.RLock()
+ for a := range alerts.Next() {
+ if err = alerts.Err(); err != nil {
+ break
+ }
+
+ routes := am.route.Match(a.Labels)
+ receivers := make([]string, 0, len(routes))
+ for _, r := range routes {
+ receivers = append(receivers, r.RouteOpts.Receiver)
+ }
+
+ if receiverFilter != nil && !receiversMatchFilter(receivers, receiverFilter) {
+ continue
+ }
+
+ if !alertFilter(a, now) {
+ continue
+ }
+
+ alert := v2.AlertToOpenAPIAlert(a, am.marker.Status(a.Fingerprint()), receivers)
+
+ res = append(res, alert)
+ }
+ am.reloadConfigMtx.RUnlock()
+
+ if err != nil {
+ am.logger.Error("failed to iterate through the alerts", "err", err)
+ return nil, fmt.Errorf("%s: %w", err.Error(), ErrGetAlertsInternal)
+ }
+ sort.Slice(res, func(i, j int) bool {
+ return *res[i].Fingerprint < *res[j].Fingerprint
+ })
+
+ return res, nil
+}
+
+func (am *Alertmanager) GetAlertGroups(active, silenced, inhibited bool, filter []string, receivers string) (apimodels.AlertGroups, error) {
+ matchers, err := parseFilter(filter)
+ if err != nil {
+ am.logger.Error("msg", "failed to parse matchers", "err", err)
+ return nil, fmt.Errorf("%s: %w", err.Error(), ErrGetAlertGroupsBadPayload)
+ }
+
+ receiverFilter, err := parseReceivers(receivers)
+ if err != nil {
+ am.logger.Error("msg", "failed to compile receiver regex", "err", err)
+ return nil, fmt.Errorf("%s: %w", err.Error(), ErrGetAlertGroupsBadPayload)
+ }
+
+ rf := func(receiverFilter *regexp.Regexp) func(r *dispatch.Route) bool {
+ return func(r *dispatch.Route) bool {
+ receiver := r.RouteOpts.Receiver
+ if receiverFilter != nil && !receiverFilter.MatchString(receiver) {
+ return false
+ }
+ return true
+ }
+ }(receiverFilter)
+
+ af := am.alertFilter(matchers, silenced, inhibited, active)
+ alertGroups, allReceivers := am.dispatcher.Groups(rf, af)
+
+ res := make(apimodels.AlertGroups, 0, len(alertGroups))
+
+ for _, alertGroup := range alertGroups {
+ ag := &apimodels.AlertGroup{
+ Receiver: &apimodels.Receiver{Name: &alertGroup.Receiver},
+ Labels: v2.ModelLabelSetToAPILabelSet(alertGroup.Labels),
+ Alerts: make([]*apimodels.GettableAlert, 0, len(alertGroup.Alerts)),
+ }
+
+ for _, alert := range alertGroup.Alerts {
+ fp := alert.Fingerprint()
+ receivers := allReceivers[fp]
+ status := am.marker.Status(fp)
+ apiAlert := v2.AlertToOpenAPIAlert(alert, status, receivers)
+ ag.Alerts = append(ag.Alerts, apiAlert)
+ }
+ res = append(res, ag)
+ }
+
+ return res, nil
+}
+
+func (am *Alertmanager) alertFilter(matchers []*labels.Matcher, silenced, inhibited, active bool) func(a *types.Alert, now time.Time) bool {
+ return func(a *types.Alert, now time.Time) bool {
+ if !a.EndsAt.IsZero() && a.EndsAt.Before(now) {
+ return false
+ }
+
+ // Set alert's current status based on its label set.
+ am.silencer.Mutes(a.Labels)
+ am.inhibitor.Mutes(a.Labels)
+
+ // Get alert's current status after seeing if it is suppressed.
+ status := am.marker.Status(a.Fingerprint())
+
+ if !active && status.State == types.AlertStateActive {
+ return false
+ }
+
+ if !silenced && len(status.SilencedBy) != 0 {
+ return false
+ }
+
+ if !inhibited && len(status.InhibitedBy) != 0 {
+ return false
+ }
+
+ return alertMatchesFilterLabels(&a.Alert, matchers)
+ }
+}
+
+func alertMatchesFilterLabels(a *prometheus_model.Alert, matchers []*labels.Matcher) bool {
+ sms := make(map[string]string)
+ for name, value := range a.Labels {
+ sms[string(name)] = string(value)
+ }
+ return matchFilterLabels(matchers, sms)
+}
+
+func matchFilterLabels(matchers []*labels.Matcher, sms map[string]string) bool {
+ for _, m := range matchers {
+ v, prs := sms[m.Name]
+ switch m.Type {
+ case labels.MatchNotRegexp, labels.MatchNotEqual:
+ if m.Value == "" && prs {
+ continue
+ }
+ if !m.Matches(v) {
+ return false
+ }
+ default:
+ if m.Value == "" && !prs {
+ continue
+ }
+ if !m.Matches(v) {
+ return false
+ }
+ }
+ }
+
+ return true
+}
+
+func parseReceivers(receivers string) (*regexp.Regexp, error) {
+ if receivers == "" {
+ return nil, nil
+ }
+
+ return regexp.Compile("^(?:" + receivers + ")$")
+}
+
+func parseFilter(filter []string) ([]*labels.Matcher, error) {
+ matchers := make([]*labels.Matcher, 0, len(filter))
+ for _, matcherString := range filter {
+ matcher, err := labels.ParseMatcher(matcherString)
+ if err != nil {
+ return nil, err
+ }
+
+ matchers = append(matchers, matcher)
+ }
+ return matchers, nil
+}
+
+func receiversMatchFilter(receivers []string, filter *regexp.Regexp) bool {
+ for _, r := range receivers {
+ if filter.MatchString(r) {
+ return true
+ }
+ }
+
+ return false
+}
diff --git a/alerting/available_channels.go b/alerting/available_channels.go
new file mode 100644
index 00000000..48c82c7b
--- /dev/null
+++ b/alerting/available_channels.go
@@ -0,0 +1,922 @@
+package notifier
+
+import (
+ "github.com/grafana/grafana/pkg/services/alerting"
+ "github.com/grafana/grafana/pkg/services/ngalert/notifier/channels"
+)
+
+// GetAvailableNotifiers returns the metadata of all the notification channels that can be configured.
+func GetAvailableNotifiers() []*alerting.NotifierPlugin {
+ pushoverSoundOptions := []alerting.SelectOption{
+ {
+ Value: "default",
+ Label: "Default",
+ },
+ {
+ Value: "pushover",
+ Label: "Pushover",
+ }, {
+ Value: "bike",
+ Label: "Bike",
+ }, {
+ Value: "bugle",
+ Label: "Bugle",
+ }, {
+ Value: "cashregister",
+ Label: "Cashregister",
+ }, {
+ Value: "classical",
+ Label: "Classical",
+ }, {
+ Value: "cosmic",
+ Label: "Cosmic",
+ }, {
+ Value: "falling",
+ Label: "Falling",
+ }, {
+ Value: "gamelan",
+ Label: "Gamelan",
+ }, {
+ Value: "incoming",
+ Label: "Incoming",
+ }, {
+ Value: "intermission",
+ Label: "Intermission",
+ }, {
+ Value: "magic",
+ Label: "Magic",
+ }, {
+ Value: "mechanical",
+ Label: "Mechanical",
+ }, {
+ Value: "pianobar",
+ Label: "Pianobar",
+ }, {
+ Value: "siren",
+ Label: "Siren",
+ }, {
+ Value: "spacealarm",
+ Label: "Spacealarm",
+ }, {
+ Value: "tugboat",
+ Label: "Tugboat",
+ }, {
+ Value: "alien",
+ Label: "Alien",
+ }, {
+ Value: "climb",
+ Label: "Climb",
+ }, {
+ Value: "persistent",
+ Label: "Persistent",
+ }, {
+ Value: "echo",
+ Label: "Echo",
+ }, {
+ Value: "updown",
+ Label: "Updown",
+ }, {
+ Value: "none",
+ Label: "None",
+ },
+ }
+
+ pushoverPriorityOptions := []alerting.SelectOption{
+ {
+ Value: "2",
+ Label: "Emergency",
+ },
+ {
+ Value: "1",
+ Label: "High",
+ },
+ {
+ Value: "0",
+ Label: "Normal",
+ },
+ {
+ Value: "-1",
+ Label: "Low",
+ },
+ {
+ Value: "-2",
+ Label: "Lowest",
+ },
+ }
+
+ return []*alerting.NotifierPlugin{
+ {
+ Type: "dingding",
+ Name: "DingDing",
+ Description: "Sends HTTP POST request to DingDing",
+ Heading: "DingDing settings",
+ Options: []alerting.NotifierOption{
+ {
+ Label: "Url",
+ Element: alerting.ElementTypeInput,
+ InputType: alerting.InputTypeText,
+ Placeholder: "https://oapi.dingtalk.com/robot/send?access_token=xxxxxxxxx",
+ PropertyName: "url",
+ Required: true,
+ },
+ {
+ Label: "Message Type",
+ Element: alerting.ElementTypeSelect,
+ PropertyName: "msgType",
+ SelectOptions: []alerting.SelectOption{
+ {
+ Value: "link",
+ Label: "Link"},
+ {
+ Value: "actionCard",
+ Label: "ActionCard",
+ },
+ },
+ },
+ { // New in 8.0.
+ Label: "Message",
+ Element: alerting.ElementTypeTextArea,
+ Placeholder: `{{ template "default.message" . }}`,
+ PropertyName: "message",
+ },
+ },
+ },
+ {
+ Type: "kafka",
+ Name: "Kafka REST Proxy",
+ Description: "Sends notifications to Kafka Rest Proxy",
+ Heading: "Kafka settings",
+ Options: []alerting.NotifierOption{
+ {
+ Label: "Kafka REST Proxy",
+ Element: alerting.ElementTypeInput,
+ InputType: alerting.InputTypeText,
+ Placeholder: "http://localhost:8082",
+ PropertyName: "kafkaRestProxy",
+ Required: true,
+ },
+ {
+ Label: "Topic",
+ Element: alerting.ElementTypeInput,
+ InputType: alerting.InputTypeText,
+ Placeholder: "topic1",
+ PropertyName: "kafkaTopic",
+ Required: true,
+ },
+ },
+ },
+ {
+ Type: "email",
+ Name: "Email",
+ Description: "Sends notifications using Grafana server configured SMTP settings",
+ Heading: "Email settings",
+ Options: []alerting.NotifierOption{
+ {
+ Label: "Single email",
+ Description: "Send a single email to all recipients",
+ Element: alerting.ElementTypeCheckbox,
+ PropertyName: "singleEmail",
+ },
+ {
+ Label: "Addresses",
+ Description: "You can enter multiple email addresses using a \";\" separator",
+ Element: alerting.ElementTypeTextArea,
+ PropertyName: "addresses",
+ Required: true,
+ },
+ { // New in 8.0.
+ Label: "Message",
+ Description: "Optional message to include with the email. You can use template variables",
+ Element: alerting.ElementTypeTextArea,
+ PropertyName: "message",
+ },
+ { // New in 9.0.
+ Label: "Subject",
+ Element: alerting.ElementTypeInput,
+ InputType: alerting.InputTypeText,
+ Description: "Templated subject of the email",
+ PropertyName: "subject",
+ Placeholder: `{{ template "default.title" . }}`,
+ },
+ },
+ },
+ {
+ Type: "pagerduty",
+ Name: "PagerDuty",
+ Description: "Sends notifications to PagerDuty",
+ Heading: "PagerDuty settings",
+ Options: []alerting.NotifierOption{
+ {
+ Label: "Integration Key",
+ Element: alerting.ElementTypeInput,
+ InputType: alerting.InputTypeText,
+ Placeholder: "Pagerduty Integration Key",
+ PropertyName: "integrationKey",
+ Required: true,
+ Secure: true,
+ },
+ {
+ Label: "Severity",
+ Element: alerting.ElementTypeSelect,
+ SelectOptions: []alerting.SelectOption{
+ {
+ Value: "critical",
+ Label: "Critical",
+ },
+ {
+ Value: "error",
+ Label: "Error",
+ },
+ {
+ Value: "warning",
+ Label: "Warning",
+ },
+ {
+ Value: "info",
+ Label: "Info",
+ },
+ },
+ PropertyName: "severity",
+ },
+ { // New in 8.0.
+ Label: "Class",
+ Description: "The class/type of the event, for example 'ping failure' or 'cpu load'",
+ Element: alerting.ElementTypeInput,
+ InputType: alerting.InputTypeText,
+ PropertyName: "class",
+ },
+ { // New in 8.0.
+ Label: "Component",
+ Description: "Component of the source machine that is responsible for the event, for example mysql or eth0",
+ Element: alerting.ElementTypeInput,
+ InputType: alerting.InputTypeText,
+ Placeholder: "Grafana",
+ PropertyName: "component",
+ },
+ { // New in 8.0.
+ Label: "Group",
+ Description: "Logical grouping of components of a service, for example 'app-stack'",
+ Element: alerting.ElementTypeInput,
+ InputType: alerting.InputTypeText,
+ PropertyName: "group",
+ },
+ { // New in 8.0.
+ Label: "Summary",
+ Description: "You can use templates for summary",
+ Element: alerting.ElementTypeTextArea,
+ Placeholder: `{{ template "default.message" . }}`,
+ PropertyName: "summary",
+ },
+ },
+ },
+ {
+ Type: "victorops",
+ Name: "VictorOps",
+ Description: "Sends notifications to VictorOps",
+ Heading: "VictorOps settings",
+ Options: []alerting.NotifierOption{
+ {
+ Label: "Url",
+ Element: alerting.ElementTypeInput,
+ InputType: alerting.InputTypeText,
+ Placeholder: "VictorOps url",
+ PropertyName: "url",
+ Required: true,
+ },
+ { // New in 8.0.
+ Label: "Message Type",
+ Element: alerting.ElementTypeSelect,
+ PropertyName: "messageType",
+ SelectOptions: []alerting.SelectOption{
+ {
+ Value: "CRITICAL",
+ Label: "CRITICAL"},
+ {
+ Value: "WARNING",
+ Label: "WARNING",
+ },
+ },
+ },
+ },
+ },
+ {
+ Type: "pushover",
+ Name: "Pushover",
+ Description: "Sends HTTP POST request to the Pushover API",
+ Heading: "Pushover settings",
+ Options: []alerting.NotifierOption{
+ {
+ Label: "API Token",
+ Element: alerting.ElementTypeInput,
+ InputType: alerting.InputTypeText,
+ Placeholder: "Application token",
+ PropertyName: "apiToken",
+ Required: true,
+ Secure: true,
+ },
+ {
+ Label: "User key(s)",
+ Element: alerting.ElementTypeInput,
+ InputType: alerting.InputTypeText,
+ Placeholder: "comma-separated list",
+ PropertyName: "userKey",
+ Required: true,
+ Secure: true,
+ },
+ {
+ Label: "Device(s) (optional)",
+ Element: alerting.ElementTypeInput,
+ InputType: alerting.InputTypeText,
+ Placeholder: "comma-separated list; leave empty to send to all devices",
+ PropertyName: "device",
+ },
+ {
+ Label: "Alerting priority",
+ Element: alerting.ElementTypeSelect,
+ SelectOptions: pushoverPriorityOptions,
+ PropertyName: "priority",
+ },
+ {
+ Label: "OK priority",
+ Element: alerting.ElementTypeSelect,
+ SelectOptions: pushoverPriorityOptions,
+ PropertyName: "okPriority",
+ },
+ {
+ Description: "How often (in seconds) the Pushover servers will send the same alerting or OK notification to the user.",
+ Label: "Retry (Only used for Emergency Priority)",
+ Element: alerting.ElementTypeInput,
+ InputType: alerting.InputTypeText,
+ Placeholder: "minimum 30 seconds",
+ PropertyName: "retry",
+ },
+ {
+ Description: "How many seconds the alerting or OK notification will continue to be retried.",
+ Label: "Expire (Only used for Emergency Priority)",
+ Element: alerting.ElementTypeInput,
+ InputType: alerting.InputTypeText,
+ Placeholder: "maximum 86400 seconds",
+ PropertyName: "expire",
+ },
+ {
+ Label: "Alerting sound",
+ Element: alerting.ElementTypeSelect,
+ SelectOptions: pushoverSoundOptions,
+ PropertyName: "sound",
+ },
+ {
+ Label: "OK sound",
+ Element: alerting.ElementTypeSelect,
+ SelectOptions: pushoverSoundOptions,
+ PropertyName: "okSound",
+ },
+ { // New in 8.0.
+ Label: "Message",
+ Element: alerting.ElementTypeTextArea,
+ Placeholder: `{{ template "default.message" . }}`,
+ PropertyName: "message",
+ },
+ },
+ },
+ {
+ Type: "slack",
+ Name: "Slack",
+ Description: "Sends notifications to Slack",
+ Heading: "Slack settings",
+ Options: []alerting.NotifierOption{
+ {
+ Label: "Recipient",
+ Element: alerting.ElementTypeInput,
+ InputType: alerting.InputTypeText,
+ Description: "Specify channel, private group, or IM channel (can be an encoded ID or a name) - required unless you provide a webhook",
+ PropertyName: "recipient",
+ Required: true,
+ DependsOn: "url",
+ },
+ // Logically, this field should be required when not using a webhook, since the Slack API needs a token.
+ // However, since the UI doesn't allow to say that a field is required or not depending on another field,
+ // we've gone with the compromise of making this field optional and instead return a validation error
+ // if it's necessary and missing.
+ {
+ Label: "Token",
+ Element: alerting.ElementTypeInput,
+ InputType: alerting.InputTypeText,
+ Description: "Provide a Slack API token (starts with \"xoxb\") - required unless you provide a webhook",
+ PropertyName: "token",
+ Secure: true,
+ Required: true,
+ DependsOn: "url",
+ },
+ {
+ Label: "Username",
+ Element: alerting.ElementTypeInput,
+ InputType: alerting.InputTypeText,
+ Description: "Set the username for the bot's message",
+ PropertyName: "username",
+ },
+ {
+ Label: "Icon emoji",
+ Element: alerting.ElementTypeInput,
+ InputType: alerting.InputTypeText,
+ Description: "Provide an emoji to use as the icon for the bot's message. Overrides the icon URL.",
+ PropertyName: "icon_emoji",
+ },
+ {
+ Label: "Icon URL",
+ Element: alerting.ElementTypeInput,
+ InputType: alerting.InputTypeText,
+ Description: "Provide a URL to an image to use as the icon for the bot's message",
+ PropertyName: "icon_url",
+ },
+ {
+ Label: "Mention Users",
+ Element: alerting.ElementTypeInput,
+ InputType: alerting.InputTypeText,
+ Description: "Mention one or more users (comma separated) when notifying in a channel, by ID (you can copy this from the user's Slack profile)",
+ PropertyName: "mentionUsers",
+ },
+ {
+ Label: "Mention Groups",
+ Element: alerting.ElementTypeInput,
+ InputType: alerting.InputTypeText,
+ Description: "Mention one or more groups (comma separated) when notifying in a channel (you can copy this from the group's Slack profile URL)",
+ PropertyName: "mentionGroups",
+ },
+ {
+ Label: "Mention Channel",
+ Element: alerting.ElementTypeSelect,
+ SelectOptions: []alerting.SelectOption{
+ {
+ Value: "",
+ Label: "Disabled",
+ },
+ {
+ Value: "here",
+ Label: "Every active channel member",
+ },
+ {
+ Value: "channel",
+ Label: "Every channel member",
+ },
+ },
+ Description: "Mention whole channel or just active members when notifying",
+ PropertyName: "mentionChannel",
+ },
+ {
+ Label: "Webhook URL",
+ Element: alerting.ElementTypeInput,
+ InputType: alerting.InputTypeText,
+ Description: "Optionally provide a Slack incoming webhook URL for sending messages, in this case the token isn't necessary",
+ Placeholder: "Slack incoming webhook URL",
+ PropertyName: "url",
+ Secure: true,
+ Required: true,
+ DependsOn: "token",
+ },
+ { // New in 8.4.
+ Label: "Endpoint URL",
+ Element: alerting.ElementTypeInput,
+ InputType: alerting.InputTypeText,
+ Description: "Optionally provide a custom Slack message API endpoint for non-webhook requests, default is https://slack.com/api/chat.postMessage",
+ Placeholder: "Slack endpoint url",
+ PropertyName: "endpointUrl",
+ },
+ { // New in 8.0.
+ Label: "Title",
+ Element: alerting.ElementTypeInput,
+ InputType: alerting.InputTypeText,
+ Description: "Templated title of the slack message",
+ PropertyName: "title",
+ Placeholder: `{{ template "slack.default.title" . }}`,
+ },
+ { // New in 8.0.
+ Label: "Text Body",
+ Element: alerting.ElementTypeTextArea,
+ Description: "Body of the slack message",
+ PropertyName: "text",
+ Placeholder: `{{ template "slack.default.text" . }}`,
+ },
+ },
+ },
+ {
+ Type: "sensugo",
+ Name: "Sensu Go",
+ Description: "Sends HTTP POST request to a Sensu Go API",
+ Heading: "Sensu Go Settings",
+ Options: []alerting.NotifierOption{
+ {
+ Label: "Backend URL",
+ Element: alerting.ElementTypeInput,
+ InputType: alerting.InputTypeText,
+ Placeholder: "http://sensu-api.local:8080",
+ PropertyName: "url",
+ Required: true,
+ },
+ {
+ Label: "API Key",
+ Element: alerting.ElementTypeInput,
+ InputType: alerting.InputTypePassword,
+ Description: "API key to auth to Sensu Go backend",
+ PropertyName: "apikey",
+ Required: true,
+ Secure: true,
+ },
+ {
+ Label: "Proxy entity name",
+ Element: alerting.ElementTypeInput,
+ InputType: alerting.InputTypeText,
+ Placeholder: "default",
+ PropertyName: "entity",
+ },
+ {
+ Label: "Check name",
+ Element: alerting.ElementTypeInput,
+ InputType: alerting.InputTypeText,
+ Placeholder: "default",
+ PropertyName: "check",
+ },
+ {
+ Label: "Handler",
+ Element: alerting.ElementTypeInput,
+ InputType: alerting.InputTypeText,
+ PropertyName: "handler",
+ },
+ {
+ Label: "Namespace",
+ Element: alerting.ElementTypeInput,
+ InputType: alerting.InputTypeText,
+ Placeholder: "default",
+ PropertyName: "namespace",
+ },
+ { // New in 8.0.
+ Label: "Message",
+ Element: alerting.ElementTypeTextArea,
+ Placeholder: `{{ template "default.message" . }}`,
+ PropertyName: "message",
+ },
+ },
+ },
+ {
+ Type: "teams",
+ Name: "Microsoft Teams",
+ Description: "Sends notifications using Incoming Webhook connector to Microsoft Teams",
+ Heading: "Teams settings",
+ Options: []alerting.NotifierOption{
+ {
+ Label: "URL",
+ Element: alerting.ElementTypeInput,
+ InputType: alerting.InputTypeText,
+ Placeholder: "Teams incoming webhook url",
+ PropertyName: "url",
+ Required: true,
+ },
+ {
+ Label: "Title",
+ Element: alerting.ElementTypeInput,
+ InputType: alerting.InputTypeText,
+ Description: "Templated title of the Teams message.",
+ PropertyName: "title",
+ Placeholder: `{{ template "default.title" . }}`,
+ },
+ {
+ Label: "Section Title",
+ Element: alerting.ElementTypeInput,
+ InputType: alerting.InputTypeText,
+ Description: "Section title for the Teams message. Leave blank for none.",
+ PropertyName: "sectiontitle",
+ },
+ { // New in 8.0.
+ Label: "Message",
+ Element: alerting.ElementTypeTextArea,
+ Placeholder: `{{ template "default.message" . }}`,
+ PropertyName: "message",
+ },
+ },
+ },
+ {
+ Type: "telegram",
+ Name: "Telegram",
+ Description: "Sends notifications to Telegram",
+ Heading: "Telegram API settings",
+ Options: []alerting.NotifierOption{
+ {
+ Label: "BOT API Token",
+ Element: alerting.ElementTypeInput,
+ InputType: alerting.InputTypeText,
+ Placeholder: "Telegram BOT API Token",
+ PropertyName: "bottoken",
+ Required: true,
+ Secure: true,
+ },
+ {
+ Label: "Chat ID",
+ Element: alerting.ElementTypeInput,
+ InputType: alerting.InputTypeText,
+ Description: "Integer Telegram Chat Identifier",
+ PropertyName: "chatid",
+ Required: true,
+ },
+ { // New in 8.0.
+ Label: "Message",
+ Element: alerting.ElementTypeTextArea,
+ Placeholder: `{{ template "default.message" . }}`,
+ PropertyName: "message",
+ },
+ },
+ },
+ {
+ Type: "webhook",
+ Name: "Webhook",
+ Description: "Sends HTTP POST request to a URL",
+ Heading: "Webhook settings",
+ Options: []alerting.NotifierOption{
+ {
+ Label: "Url",
+ Element: alerting.ElementTypeInput,
+ InputType: alerting.InputTypeText,
+ PropertyName: "url",
+ Required: true,
+ },
+ {
+ Label: "Http Method",
+ Element: alerting.ElementTypeSelect,
+ SelectOptions: []alerting.SelectOption{
+ {
+ Value: "POST",
+ Label: "POST",
+ },
+ {
+ Value: "PUT",
+ Label: "PUT",
+ },
+ },
+ PropertyName: "httpMethod",
+ },
+ {
+ Label: "Username",
+ Element: alerting.ElementTypeInput,
+ InputType: alerting.InputTypeText,
+ PropertyName: "username",
+ },
+ {
+ Label: "Password",
+ Element: alerting.ElementTypeInput,
+ InputType: alerting.InputTypePassword,
+ PropertyName: "password",
+ Secure: true,
+ },
+ { // New in 8.0. TODO: How to enforce only numbers?
+ Label: "Max Alerts",
+ Description: "Max alerts to include in a notification. Remaining alerts in the same batch will be ignored above this number. 0 means no limit.",
+ Element: alerting.ElementTypeInput,
+ InputType: alerting.InputTypeText,
+ PropertyName: "maxAlerts",
+ },
+ },
+ },
+ {
+ Type: "wecom",
+ Name: "WeCom",
+ Description: "Send alerts generated by Grafana to WeCom",
+ Heading: "WeCom settings",
+ Options: []alerting.NotifierOption{
+ {
+ Label: "Url",
+ Element: alerting.ElementTypeInput,
+ InputType: alerting.InputTypeText,
+ Placeholder: "https://qyapi.weixin.qq.com/cgi-bin/webhook/send?key=xxxxxxxx",
+ PropertyName: "url",
+ Required: true,
+ Secure: true,
+ },
+ {
+ Label: "Message",
+ Description: "Custom WeCom message. You can use template variables.",
+ Element: alerting.ElementTypeTextArea,
+ Placeholder: `{{ template "default.message" . }}`,
+ PropertyName: "message",
+ },
+ { // New in 9.1.
+ Label: "Title",
+ Element: alerting.ElementTypeInput,
+ InputType: alerting.InputTypeText,
+ Description: "Templated title of the message",
+ PropertyName: "title",
+ Placeholder: `{{ template "default.title" . }}`,
+ },
+ },
+ },
+ {
+ Type: "prometheus-alertmanager",
+ Name: "Alertmanager",
+ Description: "Sends notifications to Alertmanager",
+ Heading: "Alertmanager Settings",
+ Options: []alerting.NotifierOption{
+ {
+ Label: "URL",
+ Element: alerting.ElementTypeInput,
+ InputType: alerting.InputTypeText,
+ Placeholder: "http://localhost:9093",
+ PropertyName: "url",
+ Required: true,
+ },
+ {
+ Label: "Basic Auth User",
+ Element: alerting.ElementTypeInput,
+ InputType: alerting.InputTypeText,
+ PropertyName: "basicAuthUser",
+ },
+ {
+ Label: "Basic Auth Password",
+ Element: alerting.ElementTypeInput,
+ InputType: alerting.InputTypePassword,
+ PropertyName: "basicAuthPassword",
+ Secure: true,
+ },
+ },
+ },
+ {
+ Type: "discord",
+ Name: "Discord",
+ Heading: "Discord settings",
+ Description: "Sends notifications to Discord",
+ Options: []alerting.NotifierOption{
+ {
+ Label: "Message Content",
+ Description: "Mention a group using @ or a user using <@ID> when notifying in a channel",
+ Element: alerting.ElementTypeInput,
+ InputType: alerting.InputTypeText,
+ Placeholder: `{{ template "default.message" . }}`,
+ PropertyName: "message",
+ },
+ {
+ Label: "Webhook URL",
+ Element: alerting.ElementTypeInput,
+ InputType: alerting.InputTypeText,
+ Placeholder: "Discord webhook URL",
+ PropertyName: "url",
+ Required: true,
+ },
+ {
+ Label: "Avatar URL",
+ Element: alerting.ElementTypeInput,
+ InputType: alerting.InputTypeText,
+ PropertyName: "avatar_url",
+ },
+ {
+ Label: "Use Discord's Webhook Username",
+ Description: "Use the username configured in Discord's webhook settings. Otherwise, the username will be 'Grafana'",
+ Element: alerting.ElementTypeCheckbox,
+ PropertyName: "use_discord_username",
+ },
+ },
+ },
+ {
+ Type: "googlechat",
+ Name: "Google Hangouts Chat",
+ Description: "Sends notifications to Google Hangouts Chat via webhooks based on the official JSON message format",
+ Heading: "Google Hangouts Chat settings",
+ Options: []alerting.NotifierOption{
+ {
+ Label: "Url",
+ Element: alerting.ElementTypeInput,
+ InputType: alerting.InputTypeText,
+ Placeholder: "Google Hangouts Chat incoming webhook url",
+ PropertyName: "url",
+ Required: true,
+ },
+ {
+ Label: "Message",
+ Element: alerting.ElementTypeTextArea,
+ Placeholder: `{{ template "default.message" . }}`,
+ PropertyName: "message",
+ },
+ },
+ },
+ {
+ Type: "LINE",
+ Name: "LINE",
+ Description: "Send notifications to LINE notify",
+ Heading: "LINE notify settings",
+ Options: []alerting.NotifierOption{
+ {
+ Label: "Token",
+ Element: alerting.ElementTypeInput,
+ InputType: alerting.InputTypeText,
+ Placeholder: "LINE notify token key",
+ PropertyName: "token",
+ Required: true,
+ Secure: true,
+ }},
+ },
+ {
+ Type: "threema",
+ Name: "Threema Gateway",
+ Description: "Sends notifications to Threema using Threema Gateway (Basic IDs)",
+ Heading: "Threema Gateway settings",
+ Info: "Notifications can be configured for any Threema Gateway ID of type \"Basic\". End-to-End IDs are not currently supported." +
+ "The Threema Gateway ID can be set up at https://gateway.threema.ch/.",
+ Options: []alerting.NotifierOption{
+ {
+ Label: "Gateway ID",
+ Element: alerting.ElementTypeInput,
+ InputType: alerting.InputTypeText,
+ Placeholder: "*3MAGWID",
+ Description: "Your 8 character Threema Gateway Basic ID (starting with a *).",
+ PropertyName: "gateway_id",
+ Required: true,
+ ValidationRule: "\\*[0-9A-Z]{7}",
+ },
+ {
+ Label: "Recipient ID",
+ Element: alerting.ElementTypeInput,
+ InputType: alerting.InputTypeText,
+ Placeholder: "YOUR3MID",
+ Description: "The 8 character Threema ID that should receive the alerts.",
+ PropertyName: "recipient_id",
+ Required: true,
+ ValidationRule: "[0-9A-Z]{8}",
+ },
+ {
+ Label: "API Secret",
+ Element: alerting.ElementTypeInput,
+ InputType: alerting.InputTypeText,
+ Description: "Your Threema Gateway API secret.",
+ PropertyName: "api_secret",
+ Required: true,
+ Secure: true,
+ },
+ },
+ },
+ {
+ Type: "opsgenie",
+ Name: "OpsGenie",
+ Description: "Sends notifications to OpsGenie",
+ Heading: "OpsGenie settings",
+ Options: []alerting.NotifierOption{
+ {
+ Label: "API Key",
+ Element: alerting.ElementTypeInput,
+ InputType: alerting.InputTypeText,
+ Placeholder: "OpsGenie API Key",
+ PropertyName: "apiKey",
+ Required: true,
+ Secure: true,
+ },
+ {
+ Label: "Alert API Url",
+ Element: alerting.ElementTypeInput,
+ InputType: alerting.InputTypeText,
+ Placeholder: "https://api.opsgenie.com/v2/alerts",
+ PropertyName: "apiUrl",
+ Required: true,
+ },
+ {
+ Label: "Message",
+ Description: "Alert text limited to 130 characters.",
+ Element: alerting.ElementTypeInput,
+ InputType: alerting.InputTypeText,
+ Placeholder: `{{ template "default.title" . }}`,
+ PropertyName: "message",
+ },
+ {
+ Label: "Description",
+ Description: "A description of the incident.",
+ Element: alerting.ElementTypeTextArea,
+ PropertyName: "description",
+ },
+ {
+ Label: "Auto close incidents",
+ Element: alerting.ElementTypeCheckbox,
+ Description: "Automatically close alerts in OpsGenie once the alert goes back to ok.",
+ PropertyName: "autoClose",
+ }, {
+ Label: "Override priority",
+ Element: alerting.ElementTypeCheckbox,
+ Description: "Allow the alert priority to be set using the og_priority annotation",
+ PropertyName: "overridePriority",
+ },
+ {
+ Label: "Send notification tags as",
+ Element: alerting.ElementTypeSelect,
+ SelectOptions: []alerting.SelectOption{
+ {
+ Value: channels.OpsgenieSendTags,
+ Label: "Tags",
+ },
+ {
+ Value: channels.OpsgenieSendDetails,
+ Label: "Extra Properties",
+ },
+ {
+ Value: channels.OpsgenieSendBoth,
+ Label: "Tags & Extra Properties",
+ },
+ },
+ Description: "Send the common annotations to Opsgenie as either Extra Properties, Tags or both",
+ PropertyName: "sendTagsAs",
+ },
+ },
+ },
+ }
+}
diff --git a/alerting/channels/alertmanager.go b/alerting/channels/alertmanager.go
new file mode 100644
index 00000000..9357b14a
--- /dev/null
+++ b/alerting/channels/alertmanager.go
@@ -0,0 +1,146 @@
+package channels
+
+import (
+ "context"
+ "encoding/json"
+ "errors"
+ "fmt"
+
+ "net/url"
+ "strings"
+
+ "github.com/prometheus/alertmanager/template"
+ "github.com/prometheus/alertmanager/types"
+ "github.com/prometheus/common/model"
+
+ "github.com/grafana/grafana/pkg/infra/log"
+ "github.com/grafana/grafana/pkg/models"
+ ngmodels "github.com/grafana/grafana/pkg/services/ngalert/models"
+)
+
+// GetDecryptedValueFn is a function that returns the decrypted value of
+// the given key. If the key is not present, then it returns the fallback value.
+type GetDecryptedValueFn func(ctx context.Context, sjd map[string][]byte, key string, fallback string) string
+
+type AlertmanagerConfig struct {
+ *NotificationChannelConfig
+ URLs []*url.URL
+ BasicAuthUser string
+ BasicAuthPassword string
+}
+
+func NewAlertmanagerConfig(config *NotificationChannelConfig, fn GetDecryptedValueFn) (*AlertmanagerConfig, error) {
+ urlStr := config.Settings.Get("url").MustString()
+ if urlStr == "" {
+ return nil, errors.New("could not find url property in settings")
+ }
+ var urls []*url.URL
+ for _, uS := range strings.Split(urlStr, ",") {
+ uS = strings.TrimSpace(uS)
+ if uS == "" {
+ continue
+ }
+ uS = strings.TrimSuffix(uS, "/") + "/api/v1/alerts"
+ url, err := url.Parse(uS)
+ if err != nil {
+ return nil, fmt.Errorf("invalid url property in settings: %w", err)
+ }
+ urls = append(urls, url)
+ }
+ return &AlertmanagerConfig{
+ NotificationChannelConfig: config,
+ URLs: urls,
+ BasicAuthUser: config.Settings.Get("basicAuthUser").MustString(),
+ BasicAuthPassword: fn(context.Background(), config.SecureSettings, "basicAuthPassword", config.Settings.Get("basicAuthPassword").MustString()),
+ }, nil
+}
+
+func AlertmanagerFactory(fc FactoryConfig) (NotificationChannel, error) {
+ config, err := NewAlertmanagerConfig(fc.Config, fc.DecryptFunc)
+ if err != nil {
+ return nil, receiverInitError{
+ Reason: err.Error(),
+ Cfg: *fc.Config,
+ }
+ }
+ return NewAlertmanagerNotifier(config, fc.ImageStore, nil, fc.DecryptFunc), nil
+}
+
+// NewAlertmanagerNotifier returns a new Alertmanager notifier.
+func NewAlertmanagerNotifier(config *AlertmanagerConfig, images ImageStore, _ *template.Template, fn GetDecryptedValueFn) *AlertmanagerNotifier {
+ return &AlertmanagerNotifier{
+ Base: NewBase(&models.AlertNotification{
+ Uid: config.UID,
+ Name: config.Name,
+ DisableResolveMessage: config.DisableResolveMessage,
+ Settings: config.Settings,
+ }),
+ images: images,
+ urls: config.URLs,
+ basicAuthUser: config.BasicAuthUser,
+ basicAuthPassword: config.BasicAuthPassword,
+ logger: log.New("alerting.notifier.prometheus-alertmanager"),
+ }
+}
+
+// AlertmanagerNotifier sends alert notifications to the alert manager
+type AlertmanagerNotifier struct {
+ *Base
+ images ImageStore
+
+ urls []*url.URL
+ basicAuthUser string
+ basicAuthPassword string
+ logger log.Logger
+}
+
+// Notify sends alert notifications to Alertmanager.
+func (n *AlertmanagerNotifier) Notify(ctx context.Context, as ...*types.Alert) (bool, error) {
+ n.logger.Debug("sending Alertmanager alert", "alertmanager", n.Name)
+ if len(as) == 0 {
+ return true, nil
+ }
+
+ _ = withStoredImages(ctx, n.logger, n.images,
+ func(index int, image ngmodels.Image) error {
+ // If there is an image for this alert and the image has been uploaded
+ // to a public URL then include it as an annotation
+ if image.URL != "" {
+ as[index].Annotations["image"] = model.LabelValue(image.URL)
+ }
+ return nil
+ }, as...)
+
+ body, err := json.Marshal(as)
+ if err != nil {
+ return false, err
+ }
+
+ var (
+ lastErr error
+ numErrs int
+ )
+ for _, u := range n.urls {
+ if _, err := sendHTTPRequest(ctx, u, httpCfg{
+ user: n.basicAuthUser,
+ password: n.basicAuthPassword,
+ body: body,
+ }, n.logger); err != nil {
+ n.logger.Warn("failed to send to Alertmanager", "err", err, "alertmanager", n.Name, "url", u.String())
+ lastErr = err
+ numErrs++
+ }
+ }
+
+ if numErrs == len(n.urls) {
+ // All attempts to send alerts have failed
+ n.logger.Warn("all attempts to send to Alertmanager failed", "alertmanager", n.Name)
+ return false, fmt.Errorf("failed to send alert to Alertmanager: %w", lastErr)
+ }
+
+ return true, nil
+}
+
+func (n *AlertmanagerNotifier) SendResolved() bool {
+ return !n.GetDisableResolveMessage()
+}
diff --git a/alerting/channels/alertmanager_test.go b/alerting/channels/alertmanager_test.go
new file mode 100644
index 00000000..17b9f59b
--- /dev/null
+++ b/alerting/channels/alertmanager_test.go
@@ -0,0 +1,189 @@
+package channels
+
+import (
+ "context"
+ "encoding/json"
+ "errors"
+ "net/url"
+ "testing"
+
+ "github.com/grafana/grafana/pkg/components/simplejson"
+ "github.com/grafana/grafana/pkg/infra/log"
+ "github.com/grafana/grafana/pkg/services/secrets/fakes"
+ secretsManager "github.com/grafana/grafana/pkg/services/secrets/manager"
+
+ "github.com/prometheus/alertmanager/notify"
+ "github.com/prometheus/alertmanager/types"
+ "github.com/prometheus/common/model"
+ "github.com/stretchr/testify/require"
+)
+
+func TestNewAlertmanagerNotifier(t *testing.T) {
+ tmpl := templateForTests(t)
+
+ externalURL, err := url.Parse("http://localhost")
+ require.NoError(t, err)
+ tmpl.ExternalURL = externalURL
+
+ cases := []struct {
+ name string
+ settings string
+ alerts []*types.Alert
+ expectedInitError string
+ receiverName string
+ }{
+ {
+ name: "Error in initing: missing URL",
+ settings: `{}`,
+ expectedInitError: `could not find url property in settings`,
+ }, {
+ name: "Error in initing: invalid URL",
+ settings: `{
+ "url": "://alertmanager.com"
+ }`,
+ expectedInitError: `invalid url property in settings: parse "://alertmanager.com/api/v1/alerts": missing protocol scheme`,
+ receiverName: "Alertmanager",
+ },
+ }
+ for _, c := range cases {
+ t.Run(c.name, func(t *testing.T) {
+ settingsJSON, err := simplejson.NewJson([]byte(c.settings))
+ require.NoError(t, err)
+ secureSettings := make(map[string][]byte)
+
+ m := &NotificationChannelConfig{
+ Name: c.receiverName,
+ Type: "prometheus-alertmanager",
+ Settings: settingsJSON,
+ SecureSettings: secureSettings,
+ }
+
+ secretsService := secretsManager.SetupTestService(t, fakes.NewFakeSecretsStore())
+ decryptFn := secretsService.GetDecryptedValue
+ cfg, err := NewAlertmanagerConfig(m, decryptFn)
+ if c.expectedInitError != "" {
+ require.Equal(t, c.expectedInitError, err.Error())
+ return
+ }
+ require.NoError(t, err)
+ sn := NewAlertmanagerNotifier(cfg, &UnavailableImageStore{}, tmpl, decryptFn)
+ require.NotNil(t, sn)
+ })
+ }
+}
+
+func TestAlertmanagerNotifier_Notify(t *testing.T) {
+ tmpl := templateForTests(t)
+
+ images := newFakeImageStore(1)
+
+ externalURL, err := url.Parse("http://localhost")
+ require.NoError(t, err)
+ tmpl.ExternalURL = externalURL
+
+ cases := []struct {
+ name string
+ settings string
+ alerts []*types.Alert
+ expectedError string
+ sendHTTPRequestError error
+ receiverName string
+ }{
+ {
+ name: "Default config with one alert",
+ settings: `{"url": "https://alertmanager.com"}`,
+ alerts: []*types.Alert{
+ {
+ Alert: model.Alert{
+ Labels: model.LabelSet{"__alert_rule_uid__": "rule uid", "alertname": "alert1", "lbl1": "val1"},
+ Annotations: model.LabelSet{"ann1": "annv1"},
+ },
+ },
+ },
+ receiverName: "Alertmanager",
+ }, {
+ name: "Default config with one alert with image URL",
+ settings: `{"url": "https://alertmanager.com"}`,
+ alerts: []*types.Alert{
+ {
+ Alert: model.Alert{
+ Labels: model.LabelSet{"__alert_rule_uid__": "rule uid", "alertname": "alert1"},
+ Annotations: model.LabelSet{"__alertImageToken__": "test-image-1"},
+ },
+ },
+ },
+ receiverName: "Alertmanager",
+ }, {
+ name: "Default config with one alert with empty receiver name",
+ settings: `{"url": "https://alertmanager.com"}`,
+ alerts: []*types.Alert{
+ {
+ Alert: model.Alert{
+ Labels: model.LabelSet{"__alert_rule_uid__": "rule uid", "alertname": "alert1", "lbl1": "val1"},
+ Annotations: model.LabelSet{"ann1": "annv1"},
+ },
+ },
+ },
+ }, {
+ name: "Error sending to Alertmanager",
+ settings: `{
+ "url": "https://alertmanager.com"
+ }`,
+ alerts: []*types.Alert{
+ {
+ Alert: model.Alert{
+ Labels: model.LabelSet{"__alert_rule_uid__": "rule uid", "alertname": "alert1", "lbl1": "val1"},
+ Annotations: model.LabelSet{"ann1": "annv1"},
+ },
+ },
+ },
+ expectedError: "failed to send alert to Alertmanager: expected error",
+ sendHTTPRequestError: errors.New("expected error"),
+ receiverName: "Alertmanager",
+ },
+ }
+ for _, c := range cases {
+ t.Run(c.name, func(t *testing.T) {
+ settingsJSON, err := simplejson.NewJson([]byte(c.settings))
+ require.NoError(t, err)
+ secureSettings := make(map[string][]byte)
+
+ m := &NotificationChannelConfig{
+ Name: c.receiverName,
+ Type: "prometheus-alertmanager",
+ Settings: settingsJSON,
+ SecureSettings: secureSettings,
+ }
+
+ secretsService := secretsManager.SetupTestService(t, fakes.NewFakeSecretsStore())
+ decryptFn := secretsService.GetDecryptedValue
+ cfg, err := NewAlertmanagerConfig(m, decryptFn)
+ require.NoError(t, err)
+ sn := NewAlertmanagerNotifier(cfg, images, tmpl, decryptFn)
+ var body []byte
+ origSendHTTPRequest := sendHTTPRequest
+ t.Cleanup(func() {
+ sendHTTPRequest = origSendHTTPRequest
+ })
+ sendHTTPRequest = func(ctx context.Context, url *url.URL, cfg httpCfg, logger log.Logger) ([]byte, error) {
+ body = cfg.body
+ return nil, c.sendHTTPRequestError
+ }
+
+ ctx := notify.WithGroupKey(context.Background(), "alertname")
+ ctx = notify.WithGroupLabels(ctx, model.LabelSet{"alertname": ""})
+ ok, err := sn.Notify(ctx, c.alerts...)
+
+ if c.sendHTTPRequestError != nil {
+ require.EqualError(t, err, c.expectedError)
+ require.False(t, ok)
+ } else {
+ require.NoError(t, err)
+ require.True(t, ok)
+ expBody, err := json.Marshal(c.alerts)
+ require.NoError(t, err)
+ require.JSONEq(t, string(expBody), string(body))
+ }
+ })
+ }
+}
diff --git a/alerting/channels/base.go b/alerting/channels/base.go
new file mode 100644
index 00000000..6f261bc7
--- /dev/null
+++ b/alerting/channels/base.go
@@ -0,0 +1,32 @@
+package channels
+
+import (
+ "github.com/grafana/grafana/pkg/infra/log"
+ "github.com/grafana/grafana/pkg/models"
+)
+
+// Base is the base implementation of a notifier. It contains the common fields across all notifier types.
+type Base struct {
+ Name string
+ Type string
+ UID string
+ IsDefault bool
+ DisableResolveMessage bool
+
+ log log.Logger
+}
+
+func (n *Base) GetDisableResolveMessage() bool {
+ return n.DisableResolveMessage
+}
+
+func NewBase(model *models.AlertNotification) *Base {
+ return &Base{
+ UID: model.Uid,
+ Name: model.Name,
+ IsDefault: model.IsDefault,
+ Type: model.Type,
+ DisableResolveMessage: model.DisableResolveMessage,
+ log: log.New("alerting.notifier." + model.Name),
+ }
+}
diff --git a/alerting/channels/default_template.go b/alerting/channels/default_template.go
new file mode 100644
index 00000000..7798b2fd
--- /dev/null
+++ b/alerting/channels/default_template.go
@@ -0,0 +1,112 @@
+package channels
+
+import (
+ "io/ioutil"
+ "os"
+ "testing"
+
+ "github.com/prometheus/alertmanager/template"
+ "github.com/stretchr/testify/require"
+)
+
+const DefaultMessageTitleEmbed = `{{ template "default.title" . }}`
+
+var DefaultTemplateString = `
+{{ define "__subject" }}[{{ .Status | toUpper }}{{ if eq .Status "firing" }}:{{ .Alerts.Firing | len }}{{ if gt (.Alerts.Resolved | len) 0 }}, RESOLVED:{{ .Alerts.Resolved | len }}{{ end }}{{ end }}] {{ .GroupLabels.SortedPairs.Values | join " " }} {{ if gt (len .CommonLabels) (len .GroupLabels) }}({{ with .CommonLabels.Remove .GroupLabels.Names }}{{ .Values | join " " }}{{ end }}){{ end }}{{ end }}
+
+{{ define "__text_alert_list" }}{{ range . }}
+Value: {{ or .ValueString "[no value]" }}
+Labels:
+{{ range .Labels.SortedPairs }} - {{ .Name }} = {{ .Value }}
+{{ end }}Annotations:
+{{ range .Annotations.SortedPairs }} - {{ .Name }} = {{ .Value }}
+{{ end }}{{ if gt (len .GeneratorURL) 0 }}Source: {{ .GeneratorURL }}
+{{ end }}{{ if gt (len .SilenceURL) 0 }}Silence: {{ .SilenceURL }}
+{{ end }}{{ if gt (len .DashboardURL) 0 }}Dashboard: {{ .DashboardURL }}
+{{ end }}{{ if gt (len .PanelURL) 0 }}Panel: {{ .PanelURL }}
+{{ end }}{{ end }}{{ end }}
+
+{{ define "default.title" }}{{ template "__subject" . }}{{ end }}
+
+{{ define "default.message" }}{{ if gt (len .Alerts.Firing) 0 }}**Firing**
+{{ template "__text_alert_list" .Alerts.Firing }}{{ if gt (len .Alerts.Resolved) 0 }}
+
+{{ end }}{{ end }}{{ if gt (len .Alerts.Resolved) 0 }}**Resolved**
+{{ template "__text_alert_list" .Alerts.Resolved }}{{ end }}{{ end }}
+
+
+{{ define "__teams_text_alert_list" }}{{ range . }}
+Value: {{ or .ValueString "[no value]" }}
+Labels:
+{{ range .Labels.SortedPairs }} - {{ .Name }} = {{ .Value }}
+{{ end }}
+Annotations:
+{{ range .Annotations.SortedPairs }} - {{ .Name }} = {{ .Value }}
+{{ end }}
+{{ if gt (len .GeneratorURL) 0 }}Source: {{ .GeneratorURL }}
+
+{{ end }}{{ if gt (len .SilenceURL) 0 }}Silence: {{ .SilenceURL }}
+
+{{ end }}{{ if gt (len .DashboardURL) 0 }}Dashboard: {{ .DashboardURL }}
+
+{{ end }}{{ if gt (len .PanelURL) 0 }}Panel: {{ .PanelURL }}
+
+{{ end }}
+{{ end }}{{ end }}
+
+
+{{ define "teams.default.message" }}{{ if gt (len .Alerts.Firing) 0 }}**Firing**
+{{ template "__teams_text_alert_list" .Alerts.Firing }}{{ if gt (len .Alerts.Resolved) 0 }}
+
+{{ end }}{{ end }}{{ if gt (len .Alerts.Resolved) 0 }}**Resolved**
+{{ template "__teams_text_alert_list" .Alerts.Resolved }}{{ end }}{{ end }}
+`
+
+// TemplateForTestsString is the template used for unit tests and integration tests.
+// We have it separate from above default template because any tiny change in the template
+// will require updating almost all channel tests (15+ files) and it's very time consuming.
+const TemplateForTestsString = `
+{{ define "__subject" }}[{{ .Status | toUpper }}{{ if eq .Status "firing" }}:{{ .Alerts.Firing | len }}{{ end }}] {{ .GroupLabels.SortedPairs.Values | join " " }} {{ if gt (len .CommonLabels) (len .GroupLabels) }}({{ with .CommonLabels.Remove .GroupLabels.Names }}{{ .Values | join " " }}{{ end }}){{ end }}{{ end }}
+
+{{ define "__text_alert_list" }}{{ range . }}
+Value: {{ or .ValueString "[no value]" }}
+Labels:
+{{ range .Labels.SortedPairs }} - {{ .Name }} = {{ .Value }}
+{{ end }}Annotations:
+{{ range .Annotations.SortedPairs }} - {{ .Name }} = {{ .Value }}
+{{ end }}{{ if gt (len .GeneratorURL) 0 }}Source: {{ .GeneratorURL }}
+{{ end }}{{ if gt (len .SilenceURL) 0 }}Silence: {{ .SilenceURL }}
+{{ end }}{{ if gt (len .DashboardURL) 0 }}Dashboard: {{ .DashboardURL }}
+{{ end }}{{ if gt (len .PanelURL) 0 }}Panel: {{ .PanelURL }}
+{{ end }}{{ end }}{{ end }}
+
+{{ define "default.title" }}{{ template "__subject" . }}{{ end }}
+
+{{ define "default.message" }}{{ if gt (len .Alerts.Firing) 0 }}**Firing**
+{{ template "__text_alert_list" .Alerts.Firing }}{{ if gt (len .Alerts.Resolved) 0 }}
+
+{{ end }}{{ end }}{{ if gt (len .Alerts.Resolved) 0 }}**Resolved**
+{{ template "__text_alert_list" .Alerts.Resolved }}{{ end }}{{ end }}
+
+{{ define "teams.default.message" }}{{ template "default.message" . }}{{ end }}
+`
+
+func templateForTests(t *testing.T) *template.Template {
+ f, err := ioutil.TempFile("/tmp", "template")
+ require.NoError(t, err)
+ defer func(f *os.File) {
+ _ = f.Close()
+ }(f)
+
+ t.Cleanup(func() {
+ require.NoError(t, os.RemoveAll(f.Name()))
+ })
+
+ _, err = f.WriteString(TemplateForTestsString)
+ require.NoError(t, err)
+
+ tmpl, err := template.FromGlobs(f.Name())
+ require.NoError(t, err)
+
+ return tmpl
+}
diff --git a/alerting/channels/default_template_test.go b/alerting/channels/default_template_test.go
new file mode 100644
index 00000000..2647a187
--- /dev/null
+++ b/alerting/channels/default_template_test.go
@@ -0,0 +1,221 @@
+package channels
+
+import (
+ "context"
+ "io/ioutil"
+ "net/url"
+ "os"
+ "testing"
+ "time"
+
+ "github.com/grafana/grafana/pkg/infra/log"
+ "github.com/prometheus/alertmanager/template"
+ "github.com/prometheus/alertmanager/types"
+ "github.com/prometheus/common/model"
+ "github.com/stretchr/testify/require"
+)
+
+func TestDefaultTemplateString(t *testing.T) {
+ alerts := []*types.Alert{
+ { // Firing with dashboard and panel ID.
+ Alert: model.Alert{
+ Labels: model.LabelSet{"alertname": "alert1", "lbl1": "val1"},
+ Annotations: model.LabelSet{
+ "ann1": "annv1", "__dashboardUid__": "dbuid123", "__panelId__": "puid123", "__value_string__": "1234",
+ },
+ StartsAt: time.Now(),
+ EndsAt: time.Now().Add(1 * time.Hour),
+ GeneratorURL: "http://localhost/alert1",
+ },
+ }, { // Firing without dashboard and panel ID.
+ Alert: model.Alert{
+ Labels: model.LabelSet{"alertname": "alert1", "lbl1": "val2"},
+ Annotations: model.LabelSet{"ann1": "annv2", "__value_string__": "1234"},
+ StartsAt: time.Now(),
+ EndsAt: time.Now().Add(2 * time.Hour),
+ GeneratorURL: "http://localhost/alert2",
+ },
+ }, { // Resolved with dashboard and panel ID.
+ Alert: model.Alert{
+ Labels: model.LabelSet{"alertname": "alert1", "lbl1": "val3"},
+ Annotations: model.LabelSet{
+ "ann1": "annv3", "__dashboardUid__": "dbuid456", "__panelId__": "puid456", "__value_string__": "1234",
+ },
+ StartsAt: time.Now().Add(-1 * time.Hour),
+ EndsAt: time.Now().Add(-30 * time.Minute),
+ GeneratorURL: "http://localhost/alert3",
+ },
+ }, { // Resolved without dashboard and panel ID.
+ Alert: model.Alert{
+ Labels: model.LabelSet{"alertname": "alert1", "lbl1": "val4"},
+ Annotations: model.LabelSet{"ann1": "annv4", "__value_string__": "1234"},
+ StartsAt: time.Now().Add(-2 * time.Hour),
+ EndsAt: time.Now().Add(-3 * time.Hour),
+ GeneratorURL: "http://localhost/alert4",
+ },
+ },
+ }
+
+ f, err := ioutil.TempFile("/tmp", "template")
+ require.NoError(t, err)
+ defer func(f *os.File) {
+ _ = f.Close()
+ }(f)
+
+ t.Cleanup(func() {
+ require.NoError(t, os.RemoveAll(f.Name()))
+ })
+
+ _, err = f.WriteString(DefaultTemplateString)
+ require.NoError(t, err)
+
+ tmpl, err := template.FromGlobs(f.Name())
+ require.NoError(t, err)
+
+ externalURL, err := url.Parse("http://localhost/grafana")
+ require.NoError(t, err)
+ tmpl.ExternalURL = externalURL
+
+ var tmplErr error
+ l := log.New("default-template-test")
+ expand, _ := TmplText(context.Background(), tmpl, alerts, l, &tmplErr)
+
+ cases := []struct {
+ templateString string
+ expected string
+ }{
+ {
+ templateString: DefaultMessageTitleEmbed,
+ expected: `[FIRING:2, RESOLVED:2] (alert1)`,
+ },
+ {
+ templateString: `{{ template "default.message" .}}`,
+ expected: `**Firing**
+
+Value: 1234
+Labels:
+ - alertname = alert1
+ - lbl1 = val1
+Annotations:
+ - ann1 = annv1
+Source: http://localhost/alert1
+Silence: http://localhost/grafana/alerting/silence/new?alertmanager=grafana&matcher=alertname%3Dalert1&matcher=lbl1%3Dval1
+Dashboard: http://localhost/grafana/d/dbuid123
+Panel: http://localhost/grafana/d/dbuid123?viewPanel=puid123
+
+Value: 1234
+Labels:
+ - alertname = alert1
+ - lbl1 = val2
+Annotations:
+ - ann1 = annv2
+Source: http://localhost/alert2
+Silence: http://localhost/grafana/alerting/silence/new?alertmanager=grafana&matcher=alertname%3Dalert1&matcher=lbl1%3Dval2
+
+
+**Resolved**
+
+Value: 1234
+Labels:
+ - alertname = alert1
+ - lbl1 = val3
+Annotations:
+ - ann1 = annv3
+Source: http://localhost/alert3
+Silence: http://localhost/grafana/alerting/silence/new?alertmanager=grafana&matcher=alertname%3Dalert1&matcher=lbl1%3Dval3
+Dashboard: http://localhost/grafana/d/dbuid456
+Panel: http://localhost/grafana/d/dbuid456?viewPanel=puid456
+
+Value: 1234
+Labels:
+ - alertname = alert1
+ - lbl1 = val4
+Annotations:
+ - ann1 = annv4
+Source: http://localhost/alert4
+Silence: http://localhost/grafana/alerting/silence/new?alertmanager=grafana&matcher=alertname%3Dalert1&matcher=lbl1%3Dval4
+`,
+ },
+ {
+ templateString: `{{ template "teams.default.message" .}}`,
+ expected: `**Firing**
+
+Value: 1234
+Labels:
+ - alertname = alert1
+ - lbl1 = val1
+
+Annotations:
+ - ann1 = annv1
+
+Source: http://localhost/alert1
+
+Silence: http://localhost/grafana/alerting/silence/new?alertmanager=grafana&matcher=alertname%3Dalert1&matcher=lbl1%3Dval1
+
+Dashboard: http://localhost/grafana/d/dbuid123
+
+Panel: http://localhost/grafana/d/dbuid123?viewPanel=puid123
+
+
+
+Value: 1234
+Labels:
+ - alertname = alert1
+ - lbl1 = val2
+
+Annotations:
+ - ann1 = annv2
+
+Source: http://localhost/alert2
+
+Silence: http://localhost/grafana/alerting/silence/new?alertmanager=grafana&matcher=alertname%3Dalert1&matcher=lbl1%3Dval2
+
+
+
+
+**Resolved**
+
+Value: 1234
+Labels:
+ - alertname = alert1
+ - lbl1 = val3
+
+Annotations:
+ - ann1 = annv3
+
+Source: http://localhost/alert3
+
+Silence: http://localhost/grafana/alerting/silence/new?alertmanager=grafana&matcher=alertname%3Dalert1&matcher=lbl1%3Dval3
+
+Dashboard: http://localhost/grafana/d/dbuid456
+
+Panel: http://localhost/grafana/d/dbuid456?viewPanel=puid456
+
+
+
+Value: 1234
+Labels:
+ - alertname = alert1
+ - lbl1 = val4
+
+Annotations:
+ - ann1 = annv4
+
+Source: http://localhost/alert4
+
+Silence: http://localhost/grafana/alerting/silence/new?alertmanager=grafana&matcher=alertname%3Dalert1&matcher=lbl1%3Dval4
+
+
+`,
+ },
+ }
+
+ for _, c := range cases {
+ t.Run(c.templateString, func(t *testing.T) {
+ act := expand(c.templateString)
+ require.NoError(t, tmplErr)
+ require.Equal(t, c.expected, act)
+ })
+ }
+ require.NoError(t, tmplErr)
+}
diff --git a/alerting/channels/dingding.go b/alerting/channels/dingding.go
new file mode 100644
index 00000000..db4c21e3
--- /dev/null
+++ b/alerting/channels/dingding.go
@@ -0,0 +1,155 @@
+package channels
+
+import (
+ "context"
+ "encoding/json"
+ "errors"
+ "fmt"
+ "net/url"
+
+ "github.com/prometheus/alertmanager/template"
+ "github.com/prometheus/alertmanager/types"
+
+ "github.com/grafana/grafana/pkg/infra/log"
+ "github.com/grafana/grafana/pkg/models"
+ "github.com/grafana/grafana/pkg/services/notifications"
+)
+
+const defaultDingdingMsgType = "link"
+
+type DingDingConfig struct {
+ *NotificationChannelConfig
+ MsgType string
+ Message string
+ URL string
+}
+
+func NewDingDingConfig(config *NotificationChannelConfig) (*DingDingConfig, error) {
+ url := config.Settings.Get("url").MustString()
+ if url == "" {
+ return nil, errors.New("could not find url property in settings")
+ }
+ return &DingDingConfig{
+ NotificationChannelConfig: config,
+ MsgType: config.Settings.Get("msgType").MustString(defaultDingdingMsgType),
+ Message: config.Settings.Get("message").MustString(`{{ template "default.message" .}}`),
+ URL: config.Settings.Get("url").MustString(),
+ }, nil
+}
+func DingDingFactory(fc FactoryConfig) (NotificationChannel, error) {
+ cfg, err := NewDingDingConfig(fc.Config)
+ if err != nil {
+ return nil, receiverInitError{
+ Reason: err.Error(),
+ Cfg: *fc.Config,
+ }
+ }
+ return NewDingDingNotifier(cfg, fc.NotificationService, fc.Template), nil
+}
+
+// NewDingDingNotifier is the constructor for the Dingding notifier
+func NewDingDingNotifier(config *DingDingConfig, ns notifications.WebhookSender, t *template.Template) *DingDingNotifier {
+ return &DingDingNotifier{
+ Base: NewBase(&models.AlertNotification{
+ Uid: config.UID,
+ Name: config.Name,
+ Type: config.Type,
+ DisableResolveMessage: config.DisableResolveMessage,
+ Settings: config.Settings,
+ }),
+ MsgType: config.MsgType,
+ Message: config.Message,
+ URL: config.URL,
+ log: log.New("alerting.notifier.dingding"),
+ tmpl: t,
+ ns: ns,
+ }
+}
+
+// DingDingNotifier is responsible for sending alert notifications to ding ding.
+type DingDingNotifier struct {
+ *Base
+ MsgType string
+ URL string
+ Message string
+ tmpl *template.Template
+ ns notifications.WebhookSender
+ log log.Logger
+}
+
+// Notify sends the alert notification to dingding.
+func (dd *DingDingNotifier) Notify(ctx context.Context, as ...*types.Alert) (bool, error) {
+ dd.log.Info("sending dingding")
+
+ ruleURL := joinUrlPath(dd.tmpl.ExternalURL.String(), "/alerting/list", dd.log)
+
+ q := url.Values{
+ "pc_slide": {"false"},
+ "url": {ruleURL},
+ }
+
+ // Use special link to auto open the message url outside of Dingding
+ // Refer: https://open-doc.dingtalk.com/docs/doc.htm?treeId=385&articleId=104972&docType=1#s9
+ messageURL := "dingtalk://dingtalkclient/page/link?" + q.Encode()
+
+ var tmplErr error
+ tmpl, _ := TmplText(ctx, dd.tmpl, as, dd.log, &tmplErr)
+
+ message := tmpl(dd.Message)
+ title := tmpl(DefaultMessageTitleEmbed)
+
+ var bodyMsg map[string]interface{}
+ if tmpl(dd.MsgType) == "actionCard" {
+ bodyMsg = map[string]interface{}{
+ "msgtype": "actionCard",
+ "actionCard": map[string]string{
+ "text": message,
+ "title": title,
+ "singleTitle": "More",
+ "singleURL": messageURL,
+ },
+ }
+ } else {
+ link := map[string]string{
+ "text": message,
+ "title": title,
+ "messageUrl": messageURL,
+ }
+
+ bodyMsg = map[string]interface{}{
+ "msgtype": "link",
+ "link": link,
+ }
+ }
+
+ if tmplErr != nil {
+ dd.log.Warn("failed to template DingDing message", "err", tmplErr.Error())
+ tmplErr = nil
+ }
+
+ u := tmpl(dd.URL)
+ if tmplErr != nil {
+ dd.log.Warn("failed to template DingDing URL", "err", tmplErr.Error(), "fallback", dd.URL)
+ u = dd.URL
+ }
+
+ body, err := json.Marshal(bodyMsg)
+ if err != nil {
+ return false, err
+ }
+
+ cmd := &models.SendWebhookSync{
+ Url: u,
+ Body: string(body),
+ }
+
+ if err := dd.ns.SendWebhookSync(ctx, cmd); err != nil {
+ return false, fmt.Errorf("send notification to dingding: %w", err)
+ }
+
+ return true, nil
+}
+
+func (dd *DingDingNotifier) SendResolved() bool {
+ return !dd.GetDisableResolveMessage()
+}
diff --git a/alerting/channels/dingding_test.go b/alerting/channels/dingding_test.go
new file mode 100644
index 00000000..db1a927a
--- /dev/null
+++ b/alerting/channels/dingding_test.go
@@ -0,0 +1,187 @@
+package channels
+
+import (
+ "context"
+ "encoding/json"
+ "net/url"
+ "testing"
+
+ "github.com/prometheus/alertmanager/notify"
+ "github.com/prometheus/alertmanager/types"
+ "github.com/prometheus/common/model"
+ "github.com/stretchr/testify/require"
+
+ "github.com/grafana/grafana/pkg/components/simplejson"
+)
+
+func TestDingdingNotifier(t *testing.T) {
+ tmpl := templateForTests(t)
+
+ externalURL, err := url.Parse("http://localhost")
+ require.NoError(t, err)
+ tmpl.ExternalURL = externalURL
+
+ cases := []struct {
+ name string
+ settings string
+ alerts []*types.Alert
+ expMsg map[string]interface{}
+ expInitError string
+ expMsgError error
+ }{
+ {
+ name: "Default config with one alert",
+ settings: `{"url": "http://localhost"}`,
+ alerts: []*types.Alert{
+ {
+ Alert: model.Alert{
+ Labels: model.LabelSet{"alertname": "alert1", "lbl1": "val1"},
+ Annotations: model.LabelSet{"ann1": "annv1", "__dashboardUid__": "abcd", "__panelId__": "efgh", "__value_string__": "1234"},
+ },
+ },
+ },
+ expMsg: map[string]interface{}{
+ "msgtype": "link",
+ "link": map[string]interface{}{
+ "messageUrl": "dingtalk://dingtalkclient/page/link?pc_slide=false&url=http%3A%2F%2Flocalhost%2Falerting%2Flist",
+ "text": "**Firing**\n\nValue: 1234\nLabels:\n - alertname = alert1\n - lbl1 = val1\nAnnotations:\n - ann1 = annv1\nSilence: http://localhost/alerting/silence/new?alertmanager=grafana&matcher=alertname%3Dalert1&matcher=lbl1%3Dval1\nDashboard: http://localhost/d/abcd\nPanel: http://localhost/d/abcd?viewPanel=efgh\n",
+ "title": "[FIRING:1] (val1)",
+ },
+ },
+ expMsgError: nil,
+ }, {
+ name: "Custom config with multiple alerts",
+ settings: `{
+ "url": "http://localhost",
+ "message": "{{ len .Alerts.Firing }} alerts are firing, {{ len .Alerts.Resolved }} are resolved",
+ "msgType": "actionCard"
+ }`,
+ alerts: []*types.Alert{
+ {
+ Alert: model.Alert{
+ Labels: model.LabelSet{"alertname": "alert1", "lbl1": "val1"},
+ Annotations: model.LabelSet{"ann1": "annv1"},
+ },
+ }, {
+ Alert: model.Alert{
+ Labels: model.LabelSet{"alertname": "alert1", "lbl1": "val2"},
+ Annotations: model.LabelSet{"ann1": "annv2"},
+ },
+ },
+ },
+ expMsg: map[string]interface{}{
+ "actionCard": map[string]interface{}{
+ "singleTitle": "More",
+ "singleURL": "dingtalk://dingtalkclient/page/link?pc_slide=false&url=http%3A%2F%2Flocalhost%2Falerting%2Flist",
+ "text": "2 alerts are firing, 0 are resolved",
+ "title": "[FIRING:2] ",
+ },
+ "msgtype": "actionCard",
+ },
+ expMsgError: nil,
+ }, {
+ name: "Missing field in template",
+ settings: `{
+ "url": "http://localhost",
+ "message": "I'm a custom template {{ .NotAField }} bad template",
+ "msgType": "actionCard"
+ }`,
+ alerts: []*types.Alert{
+ {
+ Alert: model.Alert{
+ Labels: model.LabelSet{"alertname": "alert1", "lbl1": "val1"},
+ Annotations: model.LabelSet{"ann1": "annv1"},
+ },
+ }, {
+ Alert: model.Alert{
+ Labels: model.LabelSet{"alertname": "alert1", "lbl1": "val2"},
+ Annotations: model.LabelSet{"ann1": "annv2"},
+ },
+ },
+ },
+ expMsg: map[string]interface{}{
+ "link": map[string]interface{}{
+ "messageUrl": "dingtalk://dingtalkclient/page/link?pc_slide=false&url=http%3A%2F%2Flocalhost%2Falerting%2Flist",
+ "text": "I'm a custom template ",
+ "title": "",
+ },
+ "msgtype": "link",
+ },
+ expMsgError: nil,
+ }, {
+ name: "Invalid template",
+ settings: `{
+ "url": "http://localhost",
+ "message": "I'm a custom template {{ {.NotAField }} bad template",
+ "msgType": "actionCard"
+ }`,
+ alerts: []*types.Alert{
+ {
+ Alert: model.Alert{
+ Labels: model.LabelSet{"alertname": "alert1", "lbl1": "val1"},
+ Annotations: model.LabelSet{"ann1": "annv1"},
+ },
+ }, {
+ Alert: model.Alert{
+ Labels: model.LabelSet{"alertname": "alert1", "lbl1": "val2"},
+ Annotations: model.LabelSet{"ann1": "annv2"},
+ },
+ },
+ },
+ expMsg: map[string]interface{}{
+ "link": map[string]interface{}{
+ "messageUrl": "dingtalk://dingtalkclient/page/link?pc_slide=false&url=http%3A%2F%2Flocalhost%2Falerting%2Flist",
+ "text": "",
+ "title": "",
+ },
+ "msgtype": "link",
+ },
+ expMsgError: nil,
+ }, {
+ name: "Error in initing",
+ settings: `{}`,
+ expInitError: `could not find url property in settings`,
+ },
+ }
+
+ for _, c := range cases {
+ t.Run(c.name, func(t *testing.T) {
+ settingsJSON, err := simplejson.NewJson([]byte(c.settings))
+ require.NoError(t, err)
+
+ m := &NotificationChannelConfig{
+ Name: "dingding_testing",
+ Type: "dingding",
+ Settings: settingsJSON,
+ }
+
+ webhookSender := mockNotificationService()
+ cfg, err := NewDingDingConfig(m)
+ if c.expInitError != "" {
+ require.Equal(t, c.expInitError, err.Error())
+ return
+ }
+ require.NoError(t, err)
+
+ ctx := notify.WithGroupKey(context.Background(), "alertname")
+ ctx = notify.WithGroupLabels(ctx, model.LabelSet{"alertname": ""})
+ pn := NewDingDingNotifier(cfg, webhookSender, tmpl)
+ ok, err := pn.Notify(ctx, c.alerts...)
+ if c.expMsgError != nil {
+ require.False(t, ok)
+ require.Error(t, err)
+ require.Equal(t, c.expMsgError.Error(), err.Error())
+ return
+ }
+ require.NoError(t, err)
+ require.True(t, ok)
+
+ require.NotEmpty(t, webhookSender.Webhook.Url)
+
+ expBody, err := json.Marshal(c.expMsg)
+ require.NoError(t, err)
+
+ require.JSONEq(t, string(expBody), webhookSender.Webhook.Body)
+ })
+ }
+}
diff --git a/alerting/channels/discord.go b/alerting/channels/discord.go
new file mode 100644
index 00000000..a6296b5a
--- /dev/null
+++ b/alerting/channels/discord.go
@@ -0,0 +1,297 @@
+package channels
+
+import (
+ "bytes"
+ "context"
+ "encoding/json"
+ "errors"
+ "fmt"
+ "io"
+ "mime/multipart"
+ "path/filepath"
+ "strconv"
+ "strings"
+
+ "github.com/prometheus/alertmanager/template"
+ "github.com/prometheus/alertmanager/types"
+ "github.com/prometheus/common/model"
+
+ "github.com/grafana/grafana/pkg/components/simplejson"
+ "github.com/grafana/grafana/pkg/infra/log"
+ "github.com/grafana/grafana/pkg/models"
+ ngmodels "github.com/grafana/grafana/pkg/services/ngalert/models"
+ "github.com/grafana/grafana/pkg/services/notifications"
+ "github.com/grafana/grafana/pkg/setting"
+)
+
+type DiscordNotifier struct {
+ *Base
+ log log.Logger
+ ns notifications.WebhookSender
+ images ImageStore
+ tmpl *template.Template
+ Content string
+ AvatarURL string
+ WebhookURL string
+ UseDiscordUsername bool
+}
+
+type DiscordConfig struct {
+ *NotificationChannelConfig
+ Content string
+ AvatarURL string
+ WebhookURL string
+ UseDiscordUsername bool
+}
+
+type discordAttachment struct {
+ url string
+ reader io.ReadCloser
+ name string
+ alertName string
+ state model.AlertStatus
+}
+
+const DiscordMaxEmbeds = 10
+
+func NewDiscordConfig(config *NotificationChannelConfig) (*DiscordConfig, error) {
+ discordURL := config.Settings.Get("url").MustString()
+ if discordURL == "" {
+ return nil, errors.New("could not find webhook url property in settings")
+ }
+ return &DiscordConfig{
+ NotificationChannelConfig: config,
+ Content: config.Settings.Get("message").MustString(`{{ template "default.message" . }}`),
+ AvatarURL: config.Settings.Get("avatar_url").MustString(),
+ WebhookURL: discordURL,
+ UseDiscordUsername: config.Settings.Get("use_discord_username").MustBool(false),
+ }, nil
+}
+
+func DiscordFactory(fc FactoryConfig) (NotificationChannel, error) {
+ cfg, err := NewDiscordConfig(fc.Config)
+ if err != nil {
+ return nil, receiverInitError{
+ Reason: err.Error(),
+ Cfg: *fc.Config,
+ }
+ }
+ return NewDiscordNotifier(cfg, fc.NotificationService, fc.ImageStore, fc.Template), nil
+}
+
+func NewDiscordNotifier(config *DiscordConfig, ns notifications.WebhookSender, images ImageStore, t *template.Template) *DiscordNotifier {
+ return &DiscordNotifier{
+ Base: NewBase(&models.AlertNotification{
+ Uid: config.UID,
+ Name: config.Name,
+ Type: config.Type,
+ DisableResolveMessage: config.DisableResolveMessage,
+ Settings: config.Settings,
+ SecureSettings: config.SecureSettings,
+ }),
+ Content: config.Content,
+ AvatarURL: config.AvatarURL,
+ WebhookURL: config.WebhookURL,
+ log: log.New("alerting.notifier.discord"),
+ ns: ns,
+ images: images,
+ tmpl: t,
+ UseDiscordUsername: config.UseDiscordUsername,
+ }
+}
+
+func (d DiscordNotifier) Notify(ctx context.Context, as ...*types.Alert) (bool, error) {
+ alerts := types.Alerts(as...)
+
+ bodyJSON := simplejson.New()
+
+ if !d.UseDiscordUsername {
+ bodyJSON.Set("username", "Grafana")
+ }
+
+ var tmplErr error
+ tmpl, _ := TmplText(ctx, d.tmpl, as, d.log, &tmplErr)
+
+ if d.Content != "" {
+ bodyJSON.Set("content", tmpl(d.Content))
+ if tmplErr != nil {
+ d.log.Warn("failed to template Discord notification content", "err", tmplErr.Error())
+ // Reset tmplErr for templating other fields.
+ tmplErr = nil
+ }
+ }
+
+ if d.AvatarURL != "" {
+ bodyJSON.Set("avatar_url", tmpl(d.AvatarURL))
+ if tmplErr != nil {
+ d.log.Warn("failed to template Discord Avatar URL", "err", tmplErr.Error(), "fallback", d.AvatarURL)
+ bodyJSON.Set("avatar_url", d.AvatarURL)
+ tmplErr = nil
+ }
+ }
+
+ footer := map[string]interface{}{
+ "text": "Grafana v" + setting.BuildVersion,
+ "icon_url": "https://grafana.com/assets/img/fav32.png",
+ }
+
+ linkEmbed := simplejson.New()
+ linkEmbed.Set("title", tmpl(DefaultMessageTitleEmbed))
+ linkEmbed.Set("footer", footer)
+ linkEmbed.Set("type", "rich")
+
+ color, _ := strconv.ParseInt(strings.TrimLeft(getAlertStatusColor(alerts.Status()), "#"), 16, 0)
+ linkEmbed.Set("color", color)
+
+ ruleURL := joinUrlPath(d.tmpl.ExternalURL.String(), "/alerting/list", d.log)
+ linkEmbed.Set("url", ruleURL)
+
+ embeds := []interface{}{linkEmbed}
+
+ attachments := d.constructAttachments(ctx, as, DiscordMaxEmbeds-1)
+ for _, a := range attachments {
+ color, _ := strconv.ParseInt(strings.TrimLeft(getAlertStatusColor(alerts.Status()), "#"), 16, 0)
+ embed := map[string]interface{}{
+ "image": map[string]interface{}{
+ "url": a.url,
+ },
+ "color": color,
+ "title": a.alertName,
+ }
+ embeds = append(embeds, embed)
+ }
+
+ bodyJSON.Set("embeds", embeds)
+
+ if tmplErr != nil {
+ d.log.Warn("failed to template Discord message", "err", tmplErr.Error())
+ tmplErr = nil
+ }
+
+ u := tmpl(d.WebhookURL)
+ if tmplErr != nil {
+ d.log.Warn("failed to template Discord URL", "err", tmplErr.Error(), "fallback", d.WebhookURL)
+ u = d.WebhookURL
+ }
+
+ body, err := json.Marshal(bodyJSON)
+ if err != nil {
+ return false, err
+ }
+
+ cmd, err := d.buildRequest(ctx, u, body, attachments)
+ if err != nil {
+ return false, err
+ }
+
+ if err := d.ns.SendWebhookSync(ctx, cmd); err != nil {
+ d.log.Error("failed to send notification to Discord", "err", err)
+ return false, err
+ }
+ return true, nil
+}
+
+func (d DiscordNotifier) SendResolved() bool {
+ return !d.GetDisableResolveMessage()
+}
+
+func (d DiscordNotifier) constructAttachments(ctx context.Context, as []*types.Alert, embedQuota int) []discordAttachment {
+ attachments := make([]discordAttachment, 0)
+
+ _ = withStoredImages(ctx, d.log, d.images,
+ func(index int, image ngmodels.Image) error {
+ if embedQuota < 1 {
+ return ErrImagesDone
+ }
+
+ if len(image.URL) > 0 {
+ attachments = append(attachments, discordAttachment{
+ url: image.URL,
+ state: as[index].Status(),
+ alertName: as[index].Name(),
+ })
+ embedQuota--
+ return nil
+ }
+
+ // If we have a local file, but no public URL, upload the image as an attachment.
+ if len(image.Path) > 0 {
+ base := filepath.Base(image.Path)
+ url := fmt.Sprintf("attachment://%s", base)
+ reader, err := openImage(image.Path)
+ if err != nil && !errors.Is(err, ngmodels.ErrImageNotFound) {
+ d.log.Warn("failed to retrieve image data from store", "err", err)
+ return nil
+ }
+
+ attachments = append(attachments, discordAttachment{
+ url: url,
+ name: base,
+ reader: reader,
+ state: as[index].Status(),
+ alertName: as[index].Name(),
+ })
+ embedQuota--
+ }
+ return nil
+ },
+ as...,
+ )
+
+ return attachments
+}
+
+func (d DiscordNotifier) buildRequest(ctx context.Context, url string, body []byte, attachments []discordAttachment) (*models.SendWebhookSync, error) {
+ cmd := &models.SendWebhookSync{
+ Url: url,
+ HttpMethod: "POST",
+ }
+ if len(attachments) == 0 {
+ cmd.ContentType = "application/json"
+ cmd.Body = string(body)
+ return cmd, nil
+ }
+
+ var b bytes.Buffer
+ w := multipart.NewWriter(&b)
+ defer func() {
+ if err := w.Close(); err != nil {
+ // Shouldn't matter since we already close w explicitly on the non-error path
+ d.log.Warn("failed to close multipart writer", "err", err)
+ }
+ }()
+
+ payload, err := w.CreateFormField("payload_json")
+ if err != nil {
+ return nil, err
+ }
+
+ if _, err := payload.Write(body); err != nil {
+ return nil, err
+ }
+
+ for _, a := range attachments {
+ if a.reader != nil { // We have an image to upload.
+ err = func() error {
+ defer func() { _ = a.reader.Close() }()
+ part, err := w.CreateFormFile("", a.name)
+ if err != nil {
+ return err
+ }
+ _, err = io.Copy(part, a.reader)
+ return err
+ }()
+ if err != nil {
+ return nil, err
+ }
+ }
+ }
+
+ if err := w.Close(); err != nil {
+ return nil, fmt.Errorf("failed to close multipart writer: %w", err)
+ }
+
+ cmd.ContentType = w.FormDataContentType()
+ cmd.Body = b.String()
+ return cmd, nil
+}
diff --git a/alerting/channels/discord_test.go b/alerting/channels/discord_test.go
new file mode 100644
index 00000000..e2216a60
--- /dev/null
+++ b/alerting/channels/discord_test.go
@@ -0,0 +1,300 @@
+package channels
+
+import (
+ "context"
+ "encoding/json"
+ "net/url"
+ "testing"
+
+ "github.com/prometheus/alertmanager/notify"
+ "github.com/prometheus/alertmanager/types"
+ "github.com/prometheus/common/model"
+ "github.com/stretchr/testify/require"
+
+ "github.com/grafana/grafana/pkg/components/simplejson"
+ "github.com/grafana/grafana/pkg/setting"
+)
+
+func TestDiscordNotifier(t *testing.T) {
+ tmpl := templateForTests(t)
+
+ externalURL, err := url.Parse("http://localhost")
+ require.NoError(t, err)
+ tmpl.ExternalURL = externalURL
+
+ cases := []struct {
+ name string
+ settings string
+ alerts []*types.Alert
+ expMsg map[string]interface{}
+ expInitError string
+ expMsgError error
+ }{
+ {
+ name: "Default config with one alert",
+ settings: `{"url": "http://localhost"}`,
+ alerts: []*types.Alert{
+ {
+ Alert: model.Alert{
+ Labels: model.LabelSet{"alertname": "alert1", "lbl1": "val1"},
+ Annotations: model.LabelSet{"ann1": "annv1", "__dashboardUid__": "abcd", "__panelId__": "efgh"},
+ },
+ },
+ },
+ expMsg: map[string]interface{}{
+ "content": "**Firing**\n\nValue: [no value]\nLabels:\n - alertname = alert1\n - lbl1 = val1\nAnnotations:\n - ann1 = annv1\nSilence: http://localhost/alerting/silence/new?alertmanager=grafana&matcher=alertname%3Dalert1&matcher=lbl1%3Dval1\nDashboard: http://localhost/d/abcd\nPanel: http://localhost/d/abcd?viewPanel=efgh\n",
+ "embeds": []interface{}{map[string]interface{}{
+ "color": 1.4037554e+07,
+ "footer": map[string]interface{}{
+ "icon_url": "https://grafana.com/assets/img/fav32.png",
+ "text": "Grafana v" + setting.BuildVersion,
+ },
+ "title": "[FIRING:1] (val1)",
+ "url": "http://localhost/alerting/list",
+ "type": "rich",
+ }},
+ "username": "Grafana",
+ },
+ expMsgError: nil,
+ },
+ {
+ name: "Missing field in template",
+ settings: `{
+ "avatar_url": "https://grafana.com/assets/img/fav32.png",
+ "url": "http://localhost",
+ "message": "I'm a custom template {{ .NotAField }} bad template"
+ }`,
+ alerts: []*types.Alert{
+ {
+ Alert: model.Alert{
+ Labels: model.LabelSet{"alertname": "alert1", "lbl1": "val1"},
+ Annotations: model.LabelSet{"ann1": "annv1"},
+ },
+ },
+ },
+ expMsg: map[string]interface{}{
+ "avatar_url": "https://grafana.com/assets/img/fav32.png",
+ "content": "I'm a custom template ",
+ "embeds": []interface{}{map[string]interface{}{
+ "color": 1.4037554e+07,
+ "footer": map[string]interface{}{
+ "icon_url": "https://grafana.com/assets/img/fav32.png",
+ "text": "Grafana v" + setting.BuildVersion,
+ },
+ "title": "[FIRING:1] (val1)",
+ "url": "http://localhost/alerting/list",
+ "type": "rich",
+ }},
+ "username": "Grafana",
+ },
+ expMsgError: nil,
+ },
+ {
+ name: "Invalid message template",
+ settings: `{
+ "avatar_url": "https://grafana.com/assets/img/fav32.png",
+ "url": "http://localhost",
+ "message": "{{ template \"invalid.template\" }}"
+ }`,
+ alerts: []*types.Alert{
+ {
+ Alert: model.Alert{
+ Labels: model.LabelSet{"alertname": "alert1", "lbl1": "val1"},
+ Annotations: model.LabelSet{"ann1": "annv1"},
+ },
+ },
+ },
+ expMsg: map[string]interface{}{
+ "avatar_url": "https://grafana.com/assets/img/fav32.png",
+ "content": "",
+ "embeds": []interface{}{map[string]interface{}{
+ "color": 1.4037554e+07,
+ "footer": map[string]interface{}{
+ "icon_url": "https://grafana.com/assets/img/fav32.png",
+ "text": "Grafana v" + setting.BuildVersion,
+ },
+ "title": "[FIRING:1] (val1)",
+ "url": "http://localhost/alerting/list",
+ "type": "rich",
+ }},
+ "username": "Grafana",
+ },
+ expMsgError: nil,
+ },
+ {
+ name: "Invalid avatar URL template",
+ settings: `{
+ "avatar_url": "{{ invalid } }}",
+ "url": "http://localhost",
+ "message": "valid message"
+ }`,
+ alerts: []*types.Alert{
+ {
+ Alert: model.Alert{
+ Labels: model.LabelSet{"alertname": "alert1", "lbl1": "val1"},
+ Annotations: model.LabelSet{"ann1": "annv1"},
+ },
+ },
+ },
+ expMsg: map[string]interface{}{
+ "avatar_url": "{{ invalid } }}",
+ "content": "valid message",
+ "embeds": []interface{}{map[string]interface{}{
+ "color": 1.4037554e+07,
+ "footer": map[string]interface{}{
+ "icon_url": "https://grafana.com/assets/img/fav32.png",
+ "text": "Grafana v" + setting.BuildVersion,
+ },
+ "title": "[FIRING:1] (val1)",
+ "url": "http://localhost/alerting/list",
+ "type": "rich",
+ }},
+ "username": "Grafana",
+ },
+ expMsgError: nil,
+ },
+ {
+ name: "Invalid URL template",
+ settings: `{
+ "avatar_url": "https://grafana.com/assets/img/fav32.png",
+ "url": "http://localhost?q={{invalid }}}",
+ "message": "valid message"
+ }`,
+ alerts: []*types.Alert{
+ {
+ Alert: model.Alert{
+ Labels: model.LabelSet{"alertname": "alert1", "lbl1": "val1"},
+ Annotations: model.LabelSet{"ann1": "annv1"},
+ },
+ },
+ },
+ expMsg: map[string]interface{}{
+ "avatar_url": "https://grafana.com/assets/img/fav32.png",
+ "content": "valid message",
+ "embeds": []interface{}{map[string]interface{}{
+ "color": 1.4037554e+07,
+ "footer": map[string]interface{}{
+ "icon_url": "https://grafana.com/assets/img/fav32.png",
+ "text": "Grafana v" + setting.BuildVersion,
+ },
+ "title": "[FIRING:1] (val1)",
+ "url": "http://localhost/alerting/list",
+ "type": "rich",
+ }},
+ "username": "Grafana",
+ },
+ expMsgError: nil,
+ },
+ {
+ name: "Custom config with multiple alerts",
+ settings: `{
+ "avatar_url": "https://grafana.com/assets/img/fav32.png",
+ "url": "http://localhost",
+ "message": "{{ len .Alerts.Firing }} alerts are firing, {{ len .Alerts.Resolved }} are resolved"
+ }`,
+ alerts: []*types.Alert{
+ {
+ Alert: model.Alert{
+ Labels: model.LabelSet{"alertname": "alert1", "lbl1": "val1"},
+ Annotations: model.LabelSet{"ann1": "annv1"},
+ },
+ }, {
+ Alert: model.Alert{
+ Labels: model.LabelSet{"alertname": "alert1", "lbl1": "val2"},
+ Annotations: model.LabelSet{"ann1": "annv2"},
+ },
+ },
+ },
+ expMsg: map[string]interface{}{
+ "avatar_url": "https://grafana.com/assets/img/fav32.png",
+ "content": "2 alerts are firing, 0 are resolved",
+ "embeds": []interface{}{map[string]interface{}{
+ "color": 1.4037554e+07,
+ "footer": map[string]interface{}{
+ "icon_url": "https://grafana.com/assets/img/fav32.png",
+ "text": "Grafana v" + setting.BuildVersion,
+ },
+ "title": "[FIRING:2] ",
+ "url": "http://localhost/alerting/list",
+ "type": "rich",
+ }},
+ "username": "Grafana",
+ },
+ expMsgError: nil,
+ },
+ {
+ name: "Error in initialization",
+ settings: `{}`,
+ expInitError: `could not find webhook url property in settings`,
+ },
+ {
+ name: "Default config with one alert, use default discord username",
+ settings: `{
+ "url": "http://localhost",
+ "use_discord_username": true
+ }`,
+ alerts: []*types.Alert{
+ {
+ Alert: model.Alert{
+ Labels: model.LabelSet{"alertname": "alert1", "lbl1": "val1"},
+ Annotations: model.LabelSet{"ann1": "annv1", "__dashboardUid__": "abcd", "__panelId__": "efgh"},
+ },
+ },
+ },
+ expMsg: map[string]interface{}{
+ "content": "**Firing**\n\nValue: [no value]\nLabels:\n - alertname = alert1\n - lbl1 = val1\nAnnotations:\n - ann1 = annv1\nSilence: http://localhost/alerting/silence/new?alertmanager=grafana&matcher=alertname%3Dalert1&matcher=lbl1%3Dval1\nDashboard: http://localhost/d/abcd\nPanel: http://localhost/d/abcd?viewPanel=efgh\n",
+ "embeds": []interface{}{map[string]interface{}{
+ "color": 1.4037554e+07,
+ "footer": map[string]interface{}{
+ "icon_url": "https://grafana.com/assets/img/fav32.png",
+ "text": "Grafana v" + setting.BuildVersion,
+ },
+ "title": "[FIRING:1] (val1)",
+ "url": "http://localhost/alerting/list",
+ "type": "rich",
+ }},
+ },
+ expMsgError: nil,
+ },
+ }
+
+ for _, c := range cases {
+ t.Run(c.name, func(t *testing.T) {
+ settingsJson, err := simplejson.NewJson([]byte(c.settings))
+ require.NoError(t, err)
+
+ m := &NotificationChannelConfig{
+ Name: "discord_testing",
+ Type: "discord",
+ Settings: settingsJson,
+ }
+
+ webhookSender := mockNotificationService()
+ cfg, err := NewDiscordConfig(m)
+ if c.expInitError != "" {
+ require.Equal(t, c.expInitError, err.Error())
+ return
+ }
+ require.NoError(t, err)
+ imageStore := &UnavailableImageStore{}
+
+ ctx := notify.WithGroupKey(context.Background(), "alertname")
+ ctx = notify.WithGroupLabels(ctx, model.LabelSet{"alertname": ""})
+ dn := NewDiscordNotifier(cfg, webhookSender, imageStore, tmpl)
+ ok, err := dn.Notify(ctx, c.alerts...)
+ if c.expMsgError != nil {
+ require.False(t, ok)
+ require.Error(t, err)
+ require.Equal(t, c.expMsgError.Error(), err.Error())
+ return
+ }
+ require.NoError(t, err)
+ require.True(t, ok)
+
+ expBody, err := json.Marshal(c.expMsg)
+ require.NoError(t, err)
+
+ require.JSONEq(t, string(expBody), webhookSender.Webhook.Body)
+ })
+ }
+}
diff --git a/alerting/channels/email.go b/alerting/channels/email.go
new file mode 100644
index 00000000..b33ab8df
--- /dev/null
+++ b/alerting/channels/email.go
@@ -0,0 +1,163 @@
+package channels
+
+import (
+ "context"
+ "errors"
+ "net/url"
+ "os"
+ "path"
+
+ "github.com/prometheus/alertmanager/template"
+ "github.com/prometheus/alertmanager/types"
+
+ "github.com/grafana/grafana/pkg/infra/log"
+ "github.com/grafana/grafana/pkg/models"
+ ngmodels "github.com/grafana/grafana/pkg/services/ngalert/models"
+ "github.com/grafana/grafana/pkg/services/notifications"
+ "github.com/grafana/grafana/pkg/util"
+)
+
+// EmailNotifier is responsible for sending
+// alert notifications over email.
+type EmailNotifier struct {
+ *Base
+ Addresses []string
+ SingleEmail bool
+ Message string
+ Subject string
+ log log.Logger
+ ns notifications.EmailSender
+ images ImageStore
+ tmpl *template.Template
+}
+
+type EmailConfig struct {
+ *NotificationChannelConfig
+ SingleEmail bool
+ Addresses []string
+ Message string
+ Subject string
+}
+
+func EmailFactory(fc FactoryConfig) (NotificationChannel, error) {
+ cfg, err := NewEmailConfig(fc.Config)
+ if err != nil {
+ return nil, receiverInitError{
+ Reason: err.Error(),
+ Cfg: *fc.Config,
+ }
+ }
+ return NewEmailNotifier(cfg, fc.NotificationService, fc.ImageStore, fc.Template), nil
+}
+
+func NewEmailConfig(config *NotificationChannelConfig) (*EmailConfig, error) {
+ addressesString := config.Settings.Get("addresses").MustString()
+ if addressesString == "" {
+ return nil, errors.New("could not find addresses in settings")
+ }
+ // split addresses with a few different ways
+ addresses := util.SplitEmails(addressesString)
+ return &EmailConfig{
+ NotificationChannelConfig: config,
+ SingleEmail: config.Settings.Get("singleEmail").MustBool(false),
+ Message: config.Settings.Get("message").MustString(),
+ Subject: config.Settings.Get("subject").MustString(DefaultMessageTitleEmbed),
+ Addresses: addresses,
+ }, nil
+}
+
+// NewEmailNotifier is the constructor function
+// for the EmailNotifier.
+func NewEmailNotifier(config *EmailConfig, ns notifications.EmailSender, images ImageStore, t *template.Template) *EmailNotifier {
+ return &EmailNotifier{
+ Base: NewBase(&models.AlertNotification{
+ Uid: config.UID,
+ Name: config.Name,
+ Type: config.Type,
+ DisableResolveMessage: config.DisableResolveMessage,
+ Settings: config.Settings,
+ }),
+ Addresses: config.Addresses,
+ SingleEmail: config.SingleEmail,
+ Message: config.Message,
+ Subject: config.Subject,
+ log: log.New("alerting.notifier.email"),
+ ns: ns,
+ images: images,
+ tmpl: t,
+ }
+}
+
+// Notify sends the alert notification.
+func (en *EmailNotifier) Notify(ctx context.Context, alerts ...*types.Alert) (bool, error) {
+ var tmplErr error
+ tmpl, data := TmplText(ctx, en.tmpl, alerts, en.log, &tmplErr)
+
+ subject := tmpl(en.Subject)
+ alertPageURL := en.tmpl.ExternalURL.String()
+ ruleURL := en.tmpl.ExternalURL.String()
+ u, err := url.Parse(en.tmpl.ExternalURL.String())
+ if err == nil {
+ basePath := u.Path
+ u.Path = path.Join(basePath, "/alerting/list")
+ ruleURL = u.String()
+ u.RawQuery = "alertState=firing&view=state"
+ alertPageURL = u.String()
+ } else {
+ en.log.Debug("failed to parse external URL", "url", en.tmpl.ExternalURL.String(), "err", err.Error())
+ }
+
+ // Extend alerts data with images, if available.
+ var embeddedFiles []string
+ _ = withStoredImages(ctx, en.log, en.images,
+ func(index int, image ngmodels.Image) error {
+ if len(image.URL) != 0 {
+ data.Alerts[index].ImageURL = image.URL
+ } else if len(image.Path) != 0 {
+ _, err := os.Stat(image.Path)
+ if err == nil {
+ data.Alerts[index].EmbeddedImage = path.Base(image.Path)
+ embeddedFiles = append(embeddedFiles, image.Path)
+ } else {
+ en.log.Warn("failed to get image file for email attachment", "file", image.Path, "err", err)
+ }
+ }
+ return nil
+ }, alerts...)
+
+ cmd := &models.SendEmailCommandSync{
+ SendEmailCommand: models.SendEmailCommand{
+ Subject: subject,
+ Data: map[string]interface{}{
+ "Title": subject,
+ "Message": tmpl(en.Message),
+ "Status": data.Status,
+ "Alerts": data.Alerts,
+ "GroupLabels": data.GroupLabels,
+ "CommonLabels": data.CommonLabels,
+ "CommonAnnotations": data.CommonAnnotations,
+ "ExternalURL": data.ExternalURL,
+ "RuleUrl": ruleURL,
+ "AlertPageUrl": alertPageURL,
+ },
+ EmbeddedFiles: embeddedFiles,
+ To: en.Addresses,
+ SingleEmail: en.SingleEmail,
+ Template: "ng_alert_notification",
+ },
+ }
+
+ if tmplErr != nil {
+ en.log.Warn("failed to template email message", "err", tmplErr.Error())
+ }
+
+ if err := en.ns.SendEmailCommandHandlerSync(ctx, cmd); err != nil {
+ return false, err
+ }
+
+ return true, nil
+}
+
+func (en *EmailNotifier) SendResolved() bool {
+ return !en.GetDisableResolveMessage()
+}
diff --git a/alerting/channels/email_test.go b/alerting/channels/email_test.go
new file mode 100644
index 00000000..8bfb69de
--- /dev/null
+++ b/alerting/channels/email_test.go
@@ -0,0 +1,328 @@
+package channels
+
+import (
+ "context"
+ "net/url"
+ "testing"
+
+ "github.com/prometheus/alertmanager/template"
+ "github.com/prometheus/alertmanager/types"
+ "github.com/prometheus/common/model"
+ "github.com/stretchr/testify/require"
+
+ "github.com/grafana/grafana/pkg/bus"
+ "github.com/grafana/grafana/pkg/components/simplejson"
+ "github.com/grafana/grafana/pkg/infra/tracing"
+ "github.com/grafana/grafana/pkg/services/notifications"
+ "github.com/grafana/grafana/pkg/setting"
+)
+
+func TestEmailNotifier(t *testing.T) {
+ tmpl := templateForTests(t)
+
+ externalURL, err := url.Parse("http://localhost/base")
+ require.NoError(t, err)
+ tmpl.ExternalURL = externalURL
+
+ t.Run("empty settings should return error", func(t *testing.T) {
+ json := `{ }`
+
+ settingsJSON, _ := simplejson.NewJson([]byte(json))
+ model := &NotificationChannelConfig{
+ Name: "ops",
+ Type: "email",
+ Settings: settingsJSON,
+ }
+
+ _, err := NewEmailConfig(model)
+ require.Error(t, err)
+ })
+
+ t.Run("with the correct settings it should not fail and produce the expected command", func(t *testing.T) {
+ json := `{
+ "addresses": "someops@example.com;somedev@example.com",
+ "message": "{{ template \"default.title\" . }}"
+ }`
+ settingsJSON, err := simplejson.NewJson([]byte(json))
+ require.NoError(t, err)
+
+ emailSender := mockNotificationService()
+ cfg, err := NewEmailConfig(&NotificationChannelConfig{
+ Name: "ops",
+ Type: "email",
+ Settings: settingsJSON,
+ })
+ require.NoError(t, err)
+ emailNotifier := NewEmailNotifier(cfg, emailSender, &UnavailableImageStore{}, tmpl)
+
+ alerts := []*types.Alert{
+ {
+ Alert: model.Alert{
+ Labels: model.LabelSet{"alertname": "AlwaysFiring", "severity": "warning"},
+ Annotations: model.LabelSet{"runbook_url": "http://fix.me", "__dashboardUid__": "abc", "__panelId__": "5"},
+ },
+ },
+ }
+
+ ok, err := emailNotifier.Notify(context.Background(), alerts...)
+ require.NoError(t, err)
+ require.True(t, ok)
+
+ expected := map[string]interface{}{
+ "subject": emailSender.EmailSync.Subject,
+ "to": emailSender.EmailSync.To,
+ "single_email": emailSender.EmailSync.SingleEmail,
+ "template": emailSender.EmailSync.Template,
+ "data": emailSender.EmailSync.Data,
+ }
+ require.Equal(t, map[string]interface{}{
+ "subject": "[FIRING:1] (AlwaysFiring warning)",
+ "to": []string{"someops@example.com", "somedev@example.com"},
+ "single_email": false,
+ "template": "ng_alert_notification",
+ "data": map[string]interface{}{
+ "Title": "[FIRING:1] (AlwaysFiring warning)",
+ "Message": "[FIRING:1] (AlwaysFiring warning)",
+ "Status": "firing",
+ "Alerts": ExtendedAlerts{
+ ExtendedAlert{
+ Status: "firing",
+ Labels: template.KV{"alertname": "AlwaysFiring", "severity": "warning"},
+ Annotations: template.KV{"runbook_url": "http://fix.me"},
+ Fingerprint: "15a37193dce72bab",
+ SilenceURL: "http://localhost/base/alerting/silence/new?alertmanager=grafana&matcher=alertname%3DAlwaysFiring&matcher=severity%3Dwarning",
+ DashboardURL: "http://localhost/base/d/abc",
+ PanelURL: "http://localhost/base/d/abc?viewPanel=5",
+ },
+ },
+ "GroupLabels": template.KV{},
+ "CommonLabels": template.KV{"alertname": "AlwaysFiring", "severity": "warning"},
+ "CommonAnnotations": template.KV{"runbook_url": "http://fix.me"},
+ "ExternalURL": "http://localhost/base",
+ "RuleUrl": "http://localhost/base/alerting/list",
+ "AlertPageUrl": "http://localhost/base/alerting/list?alertState=firing&view=state",
+ },
+ }, expected)
+ })
+}
+
+func TestEmailNotifierIntegration(t *testing.T) {
+ ns := createCoreEmailService(t)
+
+ emailTmpl := templateForTests(t)
+ externalURL, err := url.Parse("http://localhost/base")
+ require.NoError(t, err)
+ emailTmpl.ExternalURL = externalURL
+
+ cases := []struct {
+ name string
+ alerts []*types.Alert
+ messageTmpl string
+ subjectTmpl string
+ expSubject string
+ expSnippets []string
+ }{
+ {
+ name: "single alert with templated message",
+ alerts: []*types.Alert{
+ {
+ Alert: model.Alert{
+ Labels: model.LabelSet{"alertname": "AlwaysFiring", "severity": "warning"},
+ Annotations: model.LabelSet{"runbook_url": "http://fix.me", "__dashboardUid__": "abc", "__panelId__": "5"},
+ },
+ },
+ },
+ messageTmpl: `Hi, this is a custom template.
+ {{ if gt (len .Alerts.Firing) 0 }}
+ You have {{ len .Alerts.Firing }} alerts firing.
+ {{ range .Alerts.Firing }} Firing: {{ .Labels.alertname }} at {{ .Labels.severity }} {{ end }}
+ {{ end }}`,
+ expSubject: "[FIRING:1] (AlwaysFiring warning)",
+ expSnippets: []string{
+ "Hi, this is a custom template.",
+ "You have 1 alerts firing.",
+ "Firing: AlwaysFiring at warning",
+ },
+ },
+ {
+ name: "multiple alerts with templated message",
+ alerts: []*types.Alert{
+ {
+ Alert: model.Alert{
+ Labels: model.LabelSet{"alertname": "FiringOne", "severity": "warning"},
+ Annotations: model.LabelSet{"runbook_url": "http://fix.me", "__dashboardUid__": "abc", "__panelId__": "5"},
+ },
+ },
+ {
+ Alert: model.Alert{
+ Labels: model.LabelSet{"alertname": "FiringTwo", "severity": "critical"},
+ Annotations: model.LabelSet{"runbook_url": "http://fix.me", "__dashboardUid__": "abc", "__panelId__": "5"},
+ },
+ },
+ },
+ messageTmpl: `Hi, this is a custom template.
+ {{ if gt (len .Alerts.Firing) 0 }}
+ You have {{ len .Alerts.Firing }} alerts firing.
+ {{ range .Alerts.Firing }} Firing: {{ .Labels.alertname }} at {{ .Labels.severity }} {{ end }}
+ {{ end }}`,
+ expSubject: "[FIRING:2] ",
+ expSnippets: []string{
+ "Hi, this is a custom template.",
+ "You have 2 alerts firing.",
+ "Firing: FiringOne at warning",
+ "Firing: FiringTwo at critical",
+ },
+ },
+ {
+ name: "empty message with alerts uses default template content",
+ alerts: []*types.Alert{
+ {
+ Alert: model.Alert{
+ Labels: model.LabelSet{"alertname": "FiringOne", "severity": "warning"},
+ Annotations: model.LabelSet{"runbook_url": "http://fix.me", "__dashboardUid__": "abc", "__panelId__": "5"},
+ },
+ },
+ {
+ Alert: model.Alert{
+ Labels: model.LabelSet{"alertname": "FiringTwo", "severity": "critical"},
+ Annotations: model.LabelSet{"runbook_url": "http://fix.me", "__dashboardUid__": "abc", "__panelId__": "5"},
+ },
+ },
+ },
+ messageTmpl: "",
+ expSubject: "[FIRING:2] ",
+ expSnippets: []string{
+ "Firing: 2 alerts",
+ "
alertname: FiringOneseverity: warning",
+ "alertname: FiringTwoseverity: critical",
+ "Hi, this is a custom template.
+ {{ if gt (len .Alerts.Firing) 0 }}
+
+ {{range .Alerts.Firing }}- Firing: {{ .Labels.alertname }} at {{ .Labels.severity }}
{{ end }}
+
+ {{ end }}`,
+ expSubject: "[FIRING:1] (AlwaysFiring warning)",
+ expSnippets: []string{
+ "<marquee>Hi, this is a custom template.</marquee>",
+ "<li>Firing: AlwaysFiring at warning </li>",
+ },
+ },
+ {
+ name: "single alert with templated subject",
+ alerts: []*types.Alert{
+ {
+ Alert: model.Alert{
+ Labels: model.LabelSet{"alertname": "AlwaysFiring", "severity": "warning"},
+ Annotations: model.LabelSet{"runbook_url": "http://fix.me", "__dashboardUid__": "abc", "__panelId__": "5"},
+ },
+ },
+ },
+ subjectTmpl: `This notification is {{ .Status }}!`,
+ expSubject: "This notification is firing!",
+ expSnippets: []string{},
+ },
+ }
+
+ for _, c := range cases {
+ t.Run(c.name, func(t *testing.T) {
+ emailNotifier := createSut(t, c.messageTmpl, c.subjectTmpl, emailTmpl, ns)
+
+ ok, err := emailNotifier.Notify(context.Background(), c.alerts...)
+ require.NoError(t, err)
+ require.True(t, ok)
+
+ sentMsg := getSingleSentMessage(t, ns)
+
+ require.NotNil(t, sentMsg)
+
+ require.Equal(t, "\"Grafana Admin\" ", sentMsg.From)
+ require.Equal(t, sentMsg.To[0], "someops@example.com")
+
+ require.Equal(t, c.expSubject, sentMsg.Subject)
+
+ require.Contains(t, sentMsg.Body, "text/html")
+ html := sentMsg.Body["text/html"]
+ require.NotNil(t, html)
+
+ for _, s := range c.expSnippets {
+ require.Contains(t, html, s)
+ }
+ })
+ }
+}
+
+func createCoreEmailService(t *testing.T) *notifications.NotificationService {
+ t.Helper()
+
+ tracer := tracing.InitializeTracerForTest()
+ bus := bus.ProvideBus(tracer)
+
+ cfg := setting.NewCfg()
+ cfg.StaticRootPath = "../../../../../public/"
+ cfg.BuildVersion = "4.0.0"
+ cfg.Smtp.Enabled = true
+ cfg.Smtp.TemplatesPatterns = []string{"emails/*.html", "emails/*.txt"}
+ cfg.Smtp.FromAddress = "from@address.com"
+ cfg.Smtp.FromName = "Grafana Admin"
+ cfg.Smtp.ContentTypes = []string{"text/html", "text/plain"}
+ cfg.Smtp.Host = "localhost:1234"
+ mailer := notifications.NewFakeMailer()
+
+ ns, err := notifications.ProvideService(bus, cfg, mailer, nil)
+ require.NoError(t, err)
+
+ return ns
+}
+
+func createSut(t *testing.T, messageTmpl string, subjectTmpl string, emailTmpl *template.Template, ns notifications.EmailSender) *EmailNotifier {
+ t.Helper()
+
+ json := `{
+ "addresses": "someops@example.com;somedev@example.com",
+ "singleEmail": true
+ }`
+ settingsJSON, err := simplejson.NewJson([]byte(json))
+ if messageTmpl != "" {
+ settingsJSON.Set("message", messageTmpl)
+ }
+
+ if subjectTmpl != "" {
+ settingsJSON.Set("subject", subjectTmpl)
+ }
+
+ require.NoError(t, err)
+ cfg, err := NewEmailConfig(&NotificationChannelConfig{
+ Name: "ops",
+ Type: "email",
+ Settings: settingsJSON,
+ })
+ require.NoError(t, err)
+ emailNotifier := NewEmailNotifier(cfg, ns, &UnavailableImageStore{}, emailTmpl)
+
+ return emailNotifier
+}
+
+func getSingleSentMessage(t *testing.T, ns *notifications.NotificationService) *notifications.Message {
+ t.Helper()
+
+ mailer := ns.GetMailer().(*notifications.FakeMailer)
+ require.Len(t, mailer.Sent, 1)
+ sent := mailer.Sent[0]
+ mailer.Sent = []*notifications.Message{}
+ return sent
+}
diff --git a/alerting/channels/factory.go b/alerting/channels/factory.go
new file mode 100644
index 00000000..0a8d4c6a
--- /dev/null
+++ b/alerting/channels/factory.go
@@ -0,0 +1,74 @@
+package channels
+
+import (
+ "context"
+ "errors"
+ "strings"
+
+ "github.com/grafana/grafana/pkg/services/ngalert/models"
+ "github.com/grafana/grafana/pkg/services/notifications"
+ "github.com/prometheus/alertmanager/template"
+)
+
+type FactoryConfig struct {
+ Config *NotificationChannelConfig
+ NotificationService notifications.Service
+ DecryptFunc GetDecryptedValueFn
+ ImageStore ImageStore
+ // Used to retrieve image URLs for messages, or data for uploads.
+ Template *template.Template
+}
+
+type ImageStore interface {
+ GetImage(ctx context.Context, token string) (*models.Image, error)
+}
+
+func NewFactoryConfig(config *NotificationChannelConfig, notificationService notifications.Service,
+ decryptFunc GetDecryptedValueFn, template *template.Template, imageStore ImageStore) (FactoryConfig, error) {
+ if config.Settings == nil {
+ return FactoryConfig{}, errors.New("no settings supplied")
+ }
+ // not all receivers do need secure settings, we still might interact with
+ // them, so we make sure they are never nil
+ if config.SecureSettings == nil {
+ config.SecureSettings = map[string][]byte{}
+ }
+
+ if imageStore == nil {
+ imageStore = &UnavailableImageStore{}
+ }
+ return FactoryConfig{
+ Config: config,
+ NotificationService: notificationService,
+ DecryptFunc: decryptFunc,
+ Template: template,
+ ImageStore: imageStore,
+ }, nil
+}
+
+var receiverFactories = map[string]func(FactoryConfig) (NotificationChannel, error){
+ "prometheus-alertmanager": AlertmanagerFactory,
+ "dingding": DingDingFactory,
+ "discord": DiscordFactory,
+ "email": EmailFactory,
+ "googlechat": GoogleChatFactory,
+ "kafka": KafkaFactory,
+ "line": LineFactory,
+ "opsgenie": OpsgenieFactory,
+ "pagerduty": PagerdutyFactory,
+ "pushover": PushoverFactory,
+ "sensugo": SensuGoFactory,
+ "slack": SlackFactory,
+ "teams": TeamsFactory,
+ "telegram": TelegramFactory,
+ "threema": ThreemaFactory,
+ "victorops": VictorOpsFactory,
+ "webhook": WebHookFactory,
+ "wecom": WeComFactory,
+}
+
+func Factory(receiverType string) (func(FactoryConfig) (NotificationChannel, error), bool) {
+ receiverType = strings.ToLower(receiverType)
+ factory, exists := receiverFactories[receiverType]
+ return factory, exists
+}
diff --git a/alerting/channels/googlechat.go b/alerting/channels/googlechat.go
new file mode 100644
index 00000000..b213f9c4
--- /dev/null
+++ b/alerting/channels/googlechat.go
@@ -0,0 +1,298 @@
+package channels
+
+import (
+ "context"
+ "encoding/json"
+ "errors"
+ "fmt"
+ "net/url"
+ "time"
+
+ "github.com/prometheus/alertmanager/template"
+ "github.com/prometheus/alertmanager/types"
+
+ "github.com/grafana/grafana/pkg/infra/log"
+ "github.com/grafana/grafana/pkg/models"
+ ngmodels "github.com/grafana/grafana/pkg/services/ngalert/models"
+ "github.com/grafana/grafana/pkg/services/notifications"
+ "github.com/grafana/grafana/pkg/setting"
+)
+
+// GoogleChatNotifier is responsible for sending
+// alert notifications to Google chat.
+type GoogleChatNotifier struct {
+ *Base
+ URL string
+ log log.Logger
+ ns notifications.WebhookSender
+ images ImageStore
+ tmpl *template.Template
+ content string
+}
+
+type GoogleChatConfig struct {
+ *NotificationChannelConfig
+ URL string
+ Content string
+}
+
+func GoogleChatFactory(fc FactoryConfig) (NotificationChannel, error) {
+ cfg, err := NewGoogleChatConfig(fc.Config)
+ if err != nil {
+ return nil, receiverInitError{
+ Reason: err.Error(),
+ Cfg: *fc.Config,
+ }
+ }
+ return NewGoogleChatNotifier(cfg, fc.ImageStore, fc.NotificationService, fc.Template), nil
+}
+
+func NewGoogleChatConfig(config *NotificationChannelConfig) (*GoogleChatConfig, error) {
+ url := config.Settings.Get("url").MustString()
+ if url == "" {
+ return nil, errors.New("could not find url property in settings")
+ }
+ return &GoogleChatConfig{
+ NotificationChannelConfig: config,
+ URL: url,
+ Content: config.Settings.Get("message").MustString(`{{ template "default.message" . }}`),
+ }, nil
+}
+
+func NewGoogleChatNotifier(config *GoogleChatConfig, images ImageStore, ns notifications.WebhookSender, t *template.Template) *GoogleChatNotifier {
+ return &GoogleChatNotifier{
+ Base: NewBase(&models.AlertNotification{
+ Uid: config.UID,
+ Name: config.Name,
+ Type: config.Type,
+ DisableResolveMessage: config.DisableResolveMessage,
+ Settings: config.Settings,
+ }),
+ content: config.Content,
+ URL: config.URL,
+ log: log.New("alerting.notifier.googlechat"),
+ ns: ns,
+ images: images,
+ tmpl: t,
+ }
+}
+
+// Notify send an alert notification to Google Chat.
+func (gcn *GoogleChatNotifier) Notify(ctx context.Context, as ...*types.Alert) (bool, error) {
+ gcn.log.Debug("executing Google Chat notification")
+
+ var tmplErr error
+ tmpl, _ := TmplText(ctx, gcn.tmpl, as, gcn.log, &tmplErr)
+
+ widgets := []widget{}
+
+ if msg := tmpl(gcn.content); msg != "" {
+ // Add a text paragraph widget for the message if there is a message.
+ // Google Chat API doesn't accept an empty text property.
+ widgets = append(widgets, textParagraphWidget{
+ Text: text{
+ Text: msg,
+ },
+ })
+ }
+
+ if tmplErr != nil {
+ gcn.log.Warn("failed to template Google Chat message", "err", tmplErr.Error())
+ tmplErr = nil
+ }
+
+ ruleURL := joinUrlPath(gcn.tmpl.ExternalURL.String(), "/alerting/list", gcn.log)
+ if gcn.isUrlAbsolute(ruleURL) {
+ // Add a button widget (link to Grafana).
+ widgets = append(widgets, buttonWidget{
+ Buttons: []button{
+ {
+ TextButton: textButton{
+ Text: "OPEN IN GRAFANA",
+ OnClick: onClick{
+ OpenLink: openLink{
+ URL: ruleURL,
+ },
+ },
+ },
+ },
+ },
+ })
+ } else {
+ gcn.log.Warn("Grafana external URL setting is missing or invalid. Skipping 'open in grafana' button to prevent Google from displaying empty alerts.", "ruleURL", ruleURL)
+ }
+
+ // Add text paragraph widget for the build version and timestamp.
+ widgets = append(widgets, textParagraphWidget{
+ Text: text{
+ Text: "Grafana v" + setting.BuildVersion + " | " + (timeNow()).Format(time.RFC822),
+ },
+ })
+
+ // Nest the required structs.
+ res := &outerStruct{
+ PreviewText: tmpl(DefaultMessageTitleEmbed),
+ FallbackText: tmpl(DefaultMessageTitleEmbed),
+ Cards: []card{
+ {
+ Header: header{
+ Title: tmpl(DefaultMessageTitleEmbed),
+ },
+ Sections: []section{
+ {
+ Widgets: widgets,
+ },
+ },
+ },
+ },
+ }
+ if screenshots := gcn.buildScreenshotCard(ctx, as); screenshots != nil {
+ res.Cards = append(res.Cards, *screenshots)
+ }
+
+ if tmplErr != nil {
+ gcn.log.Warn("failed to template GoogleChat message", "err", tmplErr.Error())
+ tmplErr = nil
+ }
+
+ u := tmpl(gcn.URL)
+ if tmplErr != nil {
+ gcn.log.Warn("failed to template GoogleChat URL", "err", tmplErr.Error(), "fallback", gcn.URL)
+ u = gcn.URL
+ }
+
+ body, err := json.Marshal(res)
+ if err != nil {
+ return false, fmt.Errorf("marshal json: %w", err)
+ }
+
+ cmd := &models.SendWebhookSync{
+ Url: u,
+ HttpMethod: "POST",
+ HttpHeader: map[string]string{
+ "Content-Type": "application/json; charset=UTF-8",
+ },
+ Body: string(body),
+ }
+
+ if err := gcn.ns.SendWebhookSync(ctx, cmd); err != nil {
+ gcn.log.Error("Failed to send Google Hangouts Chat alert", "error", err, "webhook", gcn.Name)
+ return false, err
+ }
+
+ return true, nil
+}
+
+func (gcn *GoogleChatNotifier) SendResolved() bool {
+ return !gcn.GetDisableResolveMessage()
+}
+
+func (gcn *GoogleChatNotifier) isUrlAbsolute(urlToCheck string) bool {
+ parsed, err := url.Parse(urlToCheck)
+ if err != nil {
+ gcn.log.Warn("could not parse URL", "urlToCheck", urlToCheck)
+ return false
+ }
+
+ return parsed.IsAbs()
+}
+
+func (gcn *GoogleChatNotifier) buildScreenshotCard(ctx context.Context, alerts []*types.Alert) *card {
+ card := card{
+ Header: header{
+ Title: "Screenshots",
+ },
+ Sections: []section{},
+ }
+
+ _ = withStoredImages(ctx, gcn.log, gcn.images,
+ func(index int, image ngmodels.Image) error {
+ if len(image.URL) == 0 {
+ return nil
+ }
+
+ section := section{
+ Widgets: []widget{
+ textParagraphWidget{
+ Text: text{
+ Text: fmt.Sprintf("%s: %s", alerts[index].Status(), alerts[index].Name()),
+ },
+ },
+ imageWidget{
+ Image: imageData{
+ ImageURL: image.URL,
+ },
+ },
+ },
+ }
+ card.Sections = append(card.Sections, section)
+
+ return nil
+ }, alerts...)
+
+ if len(card.Sections) == 0 {
+ return nil
+ }
+ return &card
+}
+
+// Structs used to build a custom Google Hangouts Chat message card.
+// See: https://developers.google.com/hangouts/chat/reference/message-formats/cards
+type outerStruct struct {
+ PreviewText string `json:"previewText"`
+ FallbackText string `json:"fallbackText"`
+ Cards []card `json:"cards"`
+}
+
+type card struct {
+ Header header `json:"header"`
+ Sections []section `json:"sections"`
+}
+
+type header struct {
+ Title string `json:"title"`
+}
+
+type section struct {
+ Widgets []widget `json:"widgets"`
+}
+
+// "generic" widget used to add different types of widgets (buttonWidget, textParagraphWidget, imageWidget)
+type widget interface{}
+
+type buttonWidget struct {
+ Buttons []button `json:"buttons"`
+}
+
+type textParagraphWidget struct {
+ Text text `json:"textParagraph"`
+}
+
+type imageWidget struct {
+ Image imageData `json:"image"`
+}
+
+type imageData struct {
+ ImageURL string `json:"imageUrl"`
+}
+
+type text struct {
+ Text string `json:"text"`
+}
+
+type button struct {
+ TextButton textButton `json:"textButton"`
+}
+
+type textButton struct {
+ Text string `json:"text"`
+ OnClick onClick `json:"onClick"`
+}
+
+type onClick struct {
+ OpenLink openLink `json:"openLink"`
+}
+
+type openLink struct {
+ URL string `json:"url"`
+}
diff --git a/alerting/channels/googlechat_test.go b/alerting/channels/googlechat_test.go
new file mode 100644
index 00000000..ddf68a8c
--- /dev/null
+++ b/alerting/channels/googlechat_test.go
@@ -0,0 +1,450 @@
+package channels
+
+import (
+ "context"
+ "encoding/json"
+ "net/url"
+ "testing"
+ "time"
+
+ "github.com/prometheus/alertmanager/notify"
+ "github.com/prometheus/alertmanager/types"
+ "github.com/prometheus/common/model"
+ "github.com/stretchr/testify/require"
+
+ "github.com/grafana/grafana/pkg/components/simplejson"
+ "github.com/grafana/grafana/pkg/setting"
+)
+
+func TestGoogleChatNotifier(t *testing.T) {
+ constNow := time.Now()
+ defer mockTimeNow(constNow)()
+
+ cases := []struct {
+ name string
+ settings string
+ alerts []*types.Alert
+ expMsg *outerStruct
+ expInitError string
+ expMsgError error
+ externalURL string
+ }{
+ {
+ name: "One alert",
+ settings: `{"url": "http://localhost"}`,
+ externalURL: "http://localhost",
+ alerts: []*types.Alert{
+ {
+ Alert: model.Alert{
+ Labels: model.LabelSet{"alertname": "alert1", "lbl1": "val1"},
+ Annotations: model.LabelSet{"ann1": "annv1", "__dashboardUid__": "abcd", "__panelId__": "efgh"},
+ },
+ },
+ },
+ expMsg: &outerStruct{
+ PreviewText: "[FIRING:1] (val1)",
+ FallbackText: "[FIRING:1] (val1)",
+ Cards: []card{
+ {
+ Header: header{
+ Title: "[FIRING:1] (val1)",
+ },
+ Sections: []section{
+ {
+ Widgets: []widget{
+ textParagraphWidget{
+ Text: text{
+ Text: "**Firing**\n\nValue: [no value]\nLabels:\n - alertname = alert1\n - lbl1 = val1\nAnnotations:\n - ann1 = annv1\nSilence: http://localhost/alerting/silence/new?alertmanager=grafana&matcher=alertname%3Dalert1&matcher=lbl1%3Dval1\nDashboard: http://localhost/d/abcd\nPanel: http://localhost/d/abcd?viewPanel=efgh\n",
+ },
+ },
+ buttonWidget{
+ Buttons: []button{
+ {
+ TextButton: textButton{
+ Text: "OPEN IN GRAFANA",
+ OnClick: onClick{
+ OpenLink: openLink{
+ URL: "http://localhost/alerting/list",
+ },
+ },
+ },
+ },
+ },
+ },
+ textParagraphWidget{
+ Text: text{
+ // RFC822 only has the minute, hence it works in most cases.
+ Text: "Grafana v" + setting.BuildVersion + " | " + constNow.Format(time.RFC822),
+ },
+ },
+ },
+ },
+ },
+ },
+ },
+ },
+ expMsgError: nil,
+ }, {
+ name: "Multiple alerts",
+ settings: `{"url": "http://localhost"}`,
+ externalURL: "http://localhost",
+ alerts: []*types.Alert{
+ {
+ Alert: model.Alert{
+ Labels: model.LabelSet{"alertname": "alert1", "lbl1": "val1"},
+ Annotations: model.LabelSet{"ann1": "annv1"},
+ },
+ }, {
+ Alert: model.Alert{
+ Labels: model.LabelSet{"alertname": "alert1", "lbl1": "val2"},
+ Annotations: model.LabelSet{"ann1": "annv2"},
+ },
+ },
+ },
+ expMsg: &outerStruct{
+ PreviewText: "[FIRING:2] ",
+ FallbackText: "[FIRING:2] ",
+ Cards: []card{
+ {
+ Header: header{
+ Title: "[FIRING:2] ",
+ },
+ Sections: []section{
+ {
+ Widgets: []widget{
+ textParagraphWidget{
+ Text: text{
+ Text: "**Firing**\n\nValue: [no value]\nLabels:\n - alertname = alert1\n - lbl1 = val1\nAnnotations:\n - ann1 = annv1\nSilence: http://localhost/alerting/silence/new?alertmanager=grafana&matcher=alertname%3Dalert1&matcher=lbl1%3Dval1\n\nValue: [no value]\nLabels:\n - alertname = alert1\n - lbl1 = val2\nAnnotations:\n - ann1 = annv2\nSilence: http://localhost/alerting/silence/new?alertmanager=grafana&matcher=alertname%3Dalert1&matcher=lbl1%3Dval2\n",
+ },
+ },
+ buttonWidget{
+ Buttons: []button{
+ {
+ TextButton: textButton{
+ Text: "OPEN IN GRAFANA",
+ OnClick: onClick{
+ OpenLink: openLink{
+ URL: "http://localhost/alerting/list",
+ },
+ },
+ },
+ },
+ },
+ },
+ textParagraphWidget{
+ Text: text{
+ Text: "Grafana v" + setting.BuildVersion + " | " + constNow.Format(time.RFC822),
+ },
+ },
+ },
+ },
+ },
+ },
+ },
+ },
+ expMsgError: nil,
+ }, {
+ name: "Error in initing",
+ settings: `{}`,
+ externalURL: "http://localhost",
+ expInitError: `could not find url property in settings`,
+ }, {
+ name: "Customized message",
+ settings: `{"url": "http://localhost", "message": "I'm a custom template and you have {{ len .Alerts.Firing }} firing alert."}`,
+ externalURL: "http://localhost",
+ alerts: []*types.Alert{
+ {
+ Alert: model.Alert{
+ Labels: model.LabelSet{"alertname": "alert1", "lbl1": "val1"},
+ Annotations: model.LabelSet{"ann1": "annv1", "__dashboardUid__": "abcd", "__panelId__": "efgh"},
+ },
+ },
+ },
+ expMsg: &outerStruct{
+ PreviewText: "[FIRING:1] (val1)",
+ FallbackText: "[FIRING:1] (val1)",
+ Cards: []card{
+ {
+ Header: header{
+ Title: "[FIRING:1] (val1)",
+ },
+ Sections: []section{
+ {
+ Widgets: []widget{
+ textParagraphWidget{
+ Text: text{
+ Text: "I'm a custom template and you have 1 firing alert.",
+ },
+ },
+ buttonWidget{
+ Buttons: []button{
+ {
+ TextButton: textButton{
+ Text: "OPEN IN GRAFANA",
+ OnClick: onClick{
+ OpenLink: openLink{
+ URL: "http://localhost/alerting/list",
+ },
+ },
+ },
+ },
+ },
+ },
+ textParagraphWidget{
+ Text: text{
+ // RFC822 only has the minute, hence it works in most cases.
+ Text: "Grafana v" + setting.BuildVersion + " | " + constNow.Format(time.RFC822),
+ },
+ },
+ },
+ },
+ },
+ },
+ },
+ },
+ expMsgError: nil,
+ }, {
+ name: "Missing field in template",
+ settings: `{"url": "http://localhost", "message": "I'm a custom template {{ .NotAField }} bad template"}`,
+ externalURL: "http://localhost",
+ alerts: []*types.Alert{
+ {
+ Alert: model.Alert{
+ Labels: model.LabelSet{"alertname": "alert1", "lbl1": "val1"},
+ Annotations: model.LabelSet{"ann1": "annv1", "__dashboardUid__": "abcd", "__panelId__": "efgh"},
+ },
+ },
+ },
+ expMsg: &outerStruct{
+ PreviewText: "[FIRING:1] (val1)",
+ FallbackText: "[FIRING:1] (val1)",
+ Cards: []card{
+ {
+ Header: header{
+ Title: "[FIRING:1] (val1)",
+ },
+ Sections: []section{
+ {
+ Widgets: []widget{
+ textParagraphWidget{
+ Text: text{
+ Text: "I'm a custom template ",
+ },
+ },
+ buttonWidget{
+ Buttons: []button{
+ {
+ TextButton: textButton{
+ Text: "OPEN IN GRAFANA",
+ OnClick: onClick{
+ OpenLink: openLink{
+ URL: "http://localhost/alerting/list",
+ },
+ },
+ },
+ },
+ },
+ },
+ textParagraphWidget{
+ Text: text{
+ // RFC822 only has the minute, hence it works in most cases.
+ Text: "Grafana v" + setting.BuildVersion + " | " + constNow.Format(time.RFC822),
+ },
+ },
+ },
+ },
+ },
+ },
+ },
+ },
+ expMsgError: nil,
+ }, {
+ name: "Invalid template",
+ settings: `{"url": "http://localhost", "message": "I'm a custom template {{ {.NotAField }} bad template"}`,
+ externalURL: "http://localhost",
+ alerts: []*types.Alert{
+ {
+ Alert: model.Alert{
+ Labels: model.LabelSet{"alertname": "alert1", "lbl1": "val1"},
+ Annotations: model.LabelSet{"ann1": "annv1", "__dashboardUid__": "abcd", "__panelId__": "efgh"},
+ },
+ },
+ },
+ expMsg: &outerStruct{
+ PreviewText: "[FIRING:1] (val1)",
+ FallbackText: "[FIRING:1] (val1)",
+ Cards: []card{
+ {
+ Header: header{
+ Title: "[FIRING:1] (val1)",
+ },
+ Sections: []section{
+ {
+ Widgets: []widget{
+ buttonWidget{
+ Buttons: []button{
+ {
+ TextButton: textButton{
+ Text: "OPEN IN GRAFANA",
+ OnClick: onClick{
+ OpenLink: openLink{
+ URL: "http://localhost/alerting/list",
+ },
+ },
+ },
+ },
+ },
+ },
+ textParagraphWidget{
+ Text: text{
+ // RFC822 only has the minute, hence it works in most cases.
+ Text: "Grafana v" + setting.BuildVersion + " | " + constNow.Format(time.RFC822),
+ },
+ },
+ },
+ },
+ },
+ },
+ },
+ },
+ expMsgError: nil,
+ },
+ {
+ name: "Empty external URL",
+ settings: `{ "url": "http://localhost" }`, // URL in settings = googlechat url
+ externalURL: "", // external URL = URL of grafana from configuration
+ alerts: []*types.Alert{
+ {
+ Alert: model.Alert{
+ Labels: model.LabelSet{"alertname": "alert1", "lbl1": "val1"},
+ Annotations: model.LabelSet{"ann1": "annv1", "__dashboardUid__": "abcd", "__panelId__": "efgh"},
+ },
+ },
+ },
+ expMsg: &outerStruct{
+ PreviewText: "[FIRING:1] (val1)",
+ FallbackText: "[FIRING:1] (val1)",
+ Cards: []card{
+ {
+ Header: header{
+ Title: "[FIRING:1] (val1)",
+ },
+ Sections: []section{
+ {
+ Widgets: []widget{
+ textParagraphWidget{
+ Text: text{
+ Text: "**Firing**\n\nValue: [no value]\nLabels:\n - alertname = alert1\n - lbl1 = val1\nAnnotations:\n - ann1 = annv1\n",
+ },
+ },
+
+ // No button widget here since the external URL is not absolute
+
+ textParagraphWidget{
+ Text: text{
+ // RFC822 only has the minute, hence it works in most cases.
+ Text: "Grafana v" + setting.BuildVersion + " | " + constNow.Format(time.RFC822),
+ },
+ },
+ },
+ },
+ },
+ },
+ },
+ },
+ },
+ {
+ name: "Relative external URL",
+ settings: `{ "url": "http://localhost" }`, // URL in settings = googlechat url
+ externalURL: "/grafana", // external URL = URL of grafana from configuration
+ alerts: []*types.Alert{
+ {
+ Alert: model.Alert{
+ Labels: model.LabelSet{"alertname": "alert1", "lbl1": "val1"},
+ Annotations: model.LabelSet{"ann1": "annv1", "__dashboardUid__": "abcd", "__panelId__": "efgh"},
+ },
+ },
+ },
+ expMsg: &outerStruct{
+ PreviewText: "[FIRING:1] (val1)",
+ FallbackText: "[FIRING:1] (val1)",
+ Cards: []card{
+ {
+ Header: header{
+ Title: "[FIRING:1] (val1)",
+ },
+ Sections: []section{
+ {
+ Widgets: []widget{
+ textParagraphWidget{
+ Text: text{
+ Text: "**Firing**\n\nValue: [no value]\nLabels:\n - alertname = alert1\n - lbl1 = val1\nAnnotations:\n - ann1 = annv1\nSilence: /grafana/alerting/silence/new?alertmanager=grafana&matcher=alertname%3Dalert1&matcher=lbl1%3Dval1\nDashboard: /grafana/d/abcd\nPanel: /grafana/d/abcd?viewPanel=efgh\n",
+ },
+ },
+
+ // No button widget here since the external URL is not absolute
+
+ textParagraphWidget{
+ Text: text{
+ // RFC822 only has the minute, hence it works in most cases.
+ Text: "Grafana v" + setting.BuildVersion + " | " + constNow.Format(time.RFC822),
+ },
+ },
+ },
+ },
+ },
+ },
+ },
+ },
+ },
+ }
+
+ for _, c := range cases {
+ t.Run(c.name, func(t *testing.T) {
+ tmpl := templateForTests(t)
+
+ externalURL, err := url.Parse(c.externalURL)
+ require.NoError(t, err)
+ tmpl.ExternalURL = externalURL
+
+ settingsJSON, err := simplejson.NewJson([]byte(c.settings))
+ require.NoError(t, err)
+
+ m := &NotificationChannelConfig{
+ Name: "googlechat_testing",
+ Type: "googlechat",
+ Settings: settingsJSON,
+ }
+
+ webhookSender := mockNotificationService()
+ cfg, err := NewGoogleChatConfig(m)
+ if c.expInitError != "" {
+ require.Error(t, err)
+ require.Equal(t, c.expInitError, err.Error())
+ return
+ }
+ require.NoError(t, err)
+ imageStore := &UnavailableImageStore{}
+
+ ctx := notify.WithGroupKey(context.Background(), "alertname")
+ ctx = notify.WithGroupLabels(ctx, model.LabelSet{"alertname": ""})
+ pn := NewGoogleChatNotifier(cfg, imageStore, webhookSender, tmpl)
+ ok, err := pn.Notify(ctx, c.alerts...)
+ if c.expMsgError != nil {
+ require.False(t, ok)
+ require.Error(t, err)
+ require.Equal(t, c.expMsgError.Error(), err.Error())
+ return
+ }
+ require.NoError(t, err)
+ require.True(t, ok)
+
+ require.NotEmpty(t, webhookSender.Webhook.Url)
+
+ expBody, err := json.Marshal(c.expMsg)
+ require.NoError(t, err)
+
+ require.JSONEq(t, string(expBody), webhookSender.Webhook.Body)
+ })
+ }
+}
diff --git a/alerting/channels/kafka.go b/alerting/channels/kafka.go
new file mode 100644
index 00000000..e97c0173
--- /dev/null
+++ b/alerting/channels/kafka.go
@@ -0,0 +1,166 @@
+package channels
+
+import (
+ "context"
+ "errors"
+ "strings"
+
+ "github.com/prometheus/alertmanager/notify"
+ "github.com/prometheus/alertmanager/template"
+ "github.com/prometheus/alertmanager/types"
+ "github.com/prometheus/common/model"
+
+ "github.com/grafana/grafana/pkg/components/simplejson"
+ "github.com/grafana/grafana/pkg/infra/log"
+ "github.com/grafana/grafana/pkg/models"
+ ngmodels "github.com/grafana/grafana/pkg/services/ngalert/models"
+ "github.com/grafana/grafana/pkg/services/notifications"
+)
+
+// KafkaNotifier is responsible for sending
+// alert notifications to Kafka.
+type KafkaNotifier struct {
+ *Base
+ Endpoint string
+ Topic string
+ log log.Logger
+ images ImageStore
+ ns notifications.WebhookSender
+ tmpl *template.Template
+}
+
+type KafkaConfig struct {
+ *NotificationChannelConfig
+ Endpoint string
+ Topic string
+}
+
+func KafkaFactory(fc FactoryConfig) (NotificationChannel, error) {
+ cfg, err := NewKafkaConfig(fc.Config)
+ if err != nil {
+ return nil, receiverInitError{
+ Reason: err.Error(),
+ Cfg: *fc.Config,
+ }
+ }
+ return NewKafkaNotifier(cfg, fc.ImageStore, fc.NotificationService, fc.Template), nil
+}
+
+func NewKafkaConfig(config *NotificationChannelConfig) (*KafkaConfig, error) {
+ endpoint := config.Settings.Get("kafkaRestProxy").MustString()
+ if endpoint == "" {
+ return nil, errors.New("could not find kafka rest proxy endpoint property in settings")
+ }
+ topic := config.Settings.Get("kafkaTopic").MustString()
+ if topic == "" {
+ return nil, errors.New("could not find kafka topic property in settings")
+ }
+ return &KafkaConfig{
+ NotificationChannelConfig: config,
+ Endpoint: endpoint,
+ Topic: topic,
+ }, nil
+}
+
+// NewKafkaNotifier is the constructor function for the Kafka notifier.
+func NewKafkaNotifier(config *KafkaConfig, images ImageStore, ns notifications.WebhookSender, t *template.Template) *KafkaNotifier {
+ return &KafkaNotifier{
+ Base: NewBase(&models.AlertNotification{
+ Uid: config.UID,
+ Name: config.Name,
+ Type: config.Type,
+ DisableResolveMessage: config.DisableResolveMessage,
+ Settings: config.Settings,
+ }),
+ Endpoint: config.Endpoint,
+ Topic: config.Topic,
+ log: log.New("alerting.notifier.kafka"),
+ images: images,
+ ns: ns,
+ tmpl: t,
+ }
+}
+
+// Notify sends the alert notification.
+func (kn *KafkaNotifier) Notify(ctx context.Context, as ...*types.Alert) (bool, error) {
+ // We are using the state from 7.x to not break kafka.
+ // TODO: should we switch to the new ones?
+ alerts := types.Alerts(as...)
+ state := models.AlertStateAlerting
+ if alerts.Status() == model.AlertResolved {
+ state = models.AlertStateOK
+ }
+
+ kn.log.Debug("notifying Kafka", "alert_state", state)
+
+ var tmplErr error
+ tmpl, _ := TmplText(ctx, kn.tmpl, as, kn.log, &tmplErr)
+
+ bodyJSON := simplejson.New()
+ bodyJSON.Set("alert_state", state)
+ bodyJSON.Set("description", tmpl(DefaultMessageTitleEmbed))
+ bodyJSON.Set("client", "Grafana")
+ bodyJSON.Set("details", tmpl(`{{ template "default.message" . }}`))
+
+ ruleURL := joinUrlPath(kn.tmpl.ExternalURL.String(), "/alerting/list", kn.log)
+ bodyJSON.Set("client_url", ruleURL)
+
+ var contexts []interface{}
+ _ = withStoredImages(ctx, kn.log, kn.images,
+ func(_ int, image ngmodels.Image) error {
+ if image.URL != "" {
+ imageJSON := simplejson.New()
+ imageJSON.Set("type", "image")
+ imageJSON.Set("src", image.URL)
+ contexts = append(contexts, imageJSON)
+ }
+ return nil
+ }, as...)
+ if len(contexts) > 0 {
+ bodyJSON.Set("contexts", contexts)
+ }
+
+ groupKey, err := notify.ExtractGroupKey(ctx)
+ if err != nil {
+ return false, err
+ }
+ bodyJSON.Set("incident_key", groupKey.Hash())
+
+ valueJSON := simplejson.New()
+ valueJSON.Set("value", bodyJSON)
+
+ recordJSON := simplejson.New()
+ recordJSON.Set("records", []interface{}{valueJSON})
+
+ body, err := recordJSON.MarshalJSON()
+ if err != nil {
+ return false, err
+ }
+
+ topicURL := strings.TrimRight(kn.Endpoint, "/") + "/topics/" + tmpl(kn.Topic)
+
+ if tmplErr != nil {
+ kn.log.Warn("failed to template Kafka message", "err", tmplErr.Error())
+ }
+
+ cmd := &models.SendWebhookSync{
+ Url: topicURL,
+ Body: string(body),
+ HttpMethod: "POST",
+ HttpHeader: map[string]string{
+ "Content-Type": "application/vnd.kafka.json.v2+json",
+ "Accept": "application/vnd.kafka.v2+json",
+ },
+ }
+
+ if err := kn.ns.SendWebhookSync(ctx, cmd); err != nil {
+ kn.log.Error("Failed to send notification to Kafka", "error", err, "body", string(body))
+ return false, err
+ }
+
+ return true, nil
+}
+
+func (kn *KafkaNotifier) SendResolved() bool {
+ return !kn.GetDisableResolveMessage()
+}
diff --git a/alerting/channels/kafka_test.go b/alerting/channels/kafka_test.go
new file mode 100644
index 00000000..5dd9b218
--- /dev/null
+++ b/alerting/channels/kafka_test.go
@@ -0,0 +1,149 @@
+package channels
+
+import (
+ "context"
+ "net/url"
+ "testing"
+
+ "github.com/prometheus/alertmanager/notify"
+ "github.com/prometheus/alertmanager/types"
+ "github.com/prometheus/common/model"
+ "github.com/stretchr/testify/require"
+
+ "github.com/grafana/grafana/pkg/components/simplejson"
+)
+
+func TestKafkaNotifier(t *testing.T) {
+ tmpl := templateForTests(t)
+
+ images := newFakeImageStore(2)
+
+ externalURL, err := url.Parse("http://localhost")
+ require.NoError(t, err)
+ tmpl.ExternalURL = externalURL
+
+ cases := []struct {
+ name string
+ settings string
+ alerts []*types.Alert
+ expUrl, expMsg string
+ expInitError string
+ expMsgError error
+ }{
+ {
+ name: "A single alert with image",
+ settings: `{
+ "kafkaRestProxy": "http://localhost",
+ "kafkaTopic": "sometopic"
+ }`,
+ alerts: []*types.Alert{
+ {
+ Alert: model.Alert{
+ Labels: model.LabelSet{"alertname": "alert1", "lbl1": "val1"},
+ Annotations: model.LabelSet{"ann1": "annv1", "__dashboardUid__": "abcd", "__panelId__": "efgh", "__alertImageToken__": "test-image-1"},
+ },
+ },
+ },
+ expUrl: "http://localhost/topics/sometopic",
+ expMsg: `{
+ "records": [
+ {
+ "value": {
+ "alert_state": "alerting",
+ "client": "Grafana",
+ "client_url": "http://localhost/alerting/list",
+ "contexts": [{"type": "image", "src": "https://www.example.com/test-image-1.jpg"}],
+ "description": "[FIRING:1] (val1)",
+ "details": "**Firing**\n\nValue: [no value]\nLabels:\n - alertname = alert1\n - lbl1 = val1\nAnnotations:\n - ann1 = annv1\nSilence: http://localhost/alerting/silence/new?alertmanager=grafana&matcher=alertname%3Dalert1&matcher=lbl1%3Dval1\nDashboard: http://localhost/d/abcd\nPanel: http://localhost/d/abcd?viewPanel=efgh\n",
+ "incident_key": "6e3538104c14b583da237e9693b76debbc17f0f8058ef20492e5853096cf8733"
+ }
+ }
+ ]
+ }`,
+ expMsgError: nil,
+ }, {
+ name: "Multiple alerts with images",
+ settings: `{
+ "kafkaRestProxy": "http://localhost",
+ "kafkaTopic": "sometopic"
+ }`,
+ alerts: []*types.Alert{
+ {
+ Alert: model.Alert{
+ Labels: model.LabelSet{"alertname": "alert1", "lbl1": "val1"},
+ Annotations: model.LabelSet{"ann1": "annv1", "__alertImageToken__": "test-image-1"},
+ },
+ }, {
+ Alert: model.Alert{
+ Labels: model.LabelSet{"alertname": "alert1", "lbl1": "val2"},
+ Annotations: model.LabelSet{"ann1": "annv2", "__alertImageToken__": "test-image-2"},
+ },
+ },
+ },
+ expUrl: "http://localhost/topics/sometopic",
+ expMsg: `{
+ "records": [
+ {
+ "value": {
+ "alert_state": "alerting",
+ "client": "Grafana",
+ "client_url": "http://localhost/alerting/list",
+ "contexts": [{"type": "image", "src": "https://www.example.com/test-image-1.jpg"}, {"type": "image", "src": "https://www.example.com/test-image-2.jpg"}],
+ "description": "[FIRING:2] ",
+ "details": "**Firing**\n\nValue: [no value]\nLabels:\n - alertname = alert1\n - lbl1 = val1\nAnnotations:\n - ann1 = annv1\nSilence: http://localhost/alerting/silence/new?alertmanager=grafana&matcher=alertname%3Dalert1&matcher=lbl1%3Dval1\n\nValue: [no value]\nLabels:\n - alertname = alert1\n - lbl1 = val2\nAnnotations:\n - ann1 = annv2\nSilence: http://localhost/alerting/silence/new?alertmanager=grafana&matcher=alertname%3Dalert1&matcher=lbl1%3Dval2\n",
+ "incident_key": "6e3538104c14b583da237e9693b76debbc17f0f8058ef20492e5853096cf8733"
+ }
+ }
+ ]
+ }`,
+ expMsgError: nil,
+ }, {
+ name: "Endpoint missing",
+ settings: `{"kafkaTopic": "sometopic"}`,
+ expInitError: `could not find kafka rest proxy endpoint property in settings`,
+ }, {
+ name: "Topic missing",
+ settings: `{"kafkaRestProxy": "http://localhost"}`,
+ expInitError: `could not find kafka topic property in settings`,
+ },
+ }
+
+ for _, c := range cases {
+ t.Run(c.name, func(t *testing.T) {
+ settingsJSON, err := simplejson.NewJson([]byte(c.settings))
+ require.NoError(t, err)
+
+ m := &NotificationChannelConfig{
+ Name: "kafka_testing",
+ Type: "kafka",
+ Settings: settingsJSON,
+ }
+
+ webhookSender := mockNotificationService()
+ cfg, err := NewKafkaConfig(m)
+ if c.expInitError != "" {
+ require.Error(t, err)
+ require.Equal(t, c.expInitError, err.Error())
+ return
+ }
+ require.NoError(t, err)
+
+ ctx := notify.WithGroupKey(context.Background(), "alertname")
+ ctx = notify.WithGroupLabels(ctx, model.LabelSet{"alertname": ""})
+
+ pn := NewKafkaNotifier(cfg, images, webhookSender, tmpl)
+ ok, err := pn.Notify(ctx, c.alerts...)
+ if c.expMsgError != nil {
+ require.False(t, ok)
+ require.Error(t, err)
+ require.Equal(t, c.expMsgError.Error(), err.Error())
+ return
+ }
+ require.NoError(t, err)
+ require.True(t, ok)
+
+ require.Equal(t, c.expUrl, webhookSender.Webhook.Url)
+ require.JSONEq(t, c.expMsg, webhookSender.Webhook.Body)
+ })
+ }
+}
diff --git a/alerting/channels/line.go b/alerting/channels/line.go
new file mode 100644
index 00000000..0ec73c9f
--- /dev/null
+++ b/alerting/channels/line.go
@@ -0,0 +1,118 @@
+package channels
+
+import (
+ "context"
+ "errors"
+ "fmt"
+ "net/url"
+ "path"
+
+ "github.com/prometheus/alertmanager/template"
+ "github.com/prometheus/alertmanager/types"
+
+ "github.com/grafana/grafana/pkg/infra/log"
+ "github.com/grafana/grafana/pkg/models"
+ "github.com/grafana/grafana/pkg/services/notifications"
+)
+
+var (
+ LineNotifyURL string = "https://notify-api.line.me/api/notify"
+)
+
+type LineConfig struct {
+ *NotificationChannelConfig
+ Token string
+}
+
+func LineFactory(fc FactoryConfig) (NotificationChannel, error) {
+ cfg, err := NewLineConfig(fc.Config, fc.DecryptFunc)
+ if err != nil {
+ return nil, receiverInitError{
+ Reason: err.Error(),
+ Cfg: *fc.Config,
+ }
+ }
+ return NewLineNotifier(cfg, fc.NotificationService, fc.Template), nil
+}
+
+func NewLineConfig(config *NotificationChannelConfig, decryptFunc GetDecryptedValueFn) (*LineConfig, error) {
+ token := decryptFunc(context.Background(), config.SecureSettings, "token", config.Settings.Get("token").MustString())
+ if token == "" {
+ return nil, errors.New("could not find token in settings")
+ }
+ return &LineConfig{
+ NotificationChannelConfig: config,
+ Token: token,
+ }, nil
+}
+
+// NewLineNotifier is the constructor for the LINE notifier
+func NewLineNotifier(config *LineConfig, ns notifications.WebhookSender, t *template.Template) *LineNotifier {
+ return &LineNotifier{
+ Base: NewBase(&models.AlertNotification{
+ Uid: config.UID,
+ Name: config.Name,
+ Type: config.Type,
+ DisableResolveMessage: config.DisableResolveMessage,
+ Settings: config.Settings,
+ }),
+ Token: config.Token,
+ log: log.New("alerting.notifier.line"),
+ ns: ns,
+ tmpl: t,
+ }
+}
+
+// LineNotifier is responsible for sending
+// alert notifications to LINE.
+type LineNotifier struct {
+ *Base
+ Token string
+ log log.Logger
+ ns notifications.WebhookSender
+ tmpl *template.Template
+}
+
+// Notify send an alert notification to LINE
+func (ln *LineNotifier) Notify(ctx context.Context, as ...*types.Alert) (bool, error) {
+ ln.log.Debug("executing line notification", "notification", ln.Name)
+
+ ruleURL := path.Join(ln.tmpl.ExternalURL.String(), "/alerting/list")
+
+ var tmplErr error
+ tmpl, _ := TmplText(ctx, ln.tmpl, as, ln.log, &tmplErr)
+
+ body := fmt.Sprintf(
+ "%s\n%s\n\n%s",
+ tmpl(DefaultMessageTitleEmbed),
+ ruleURL,
+ tmpl(`{{ template "default.message" . }}`),
+ )
+ if tmplErr != nil {
+ ln.log.Warn("failed to template Line message", "err", tmplErr.Error())
+ }
+
+ form := url.Values{}
+ form.Add("message", body)
+
+ cmd := &models.SendWebhookSync{
+ Url: LineNotifyURL,
+ HttpMethod: "POST",
+ HttpHeader: map[string]string{
+ "Authorization": fmt.Sprintf("Bearer %s", ln.Token),
+ "Content-Type": "application/x-www-form-urlencoded;charset=UTF-8",
+ },
+ Body: form.Encode(),
+ }
+
+ if err := ln.ns.SendWebhookSync(ctx, cmd); err != nil {
+ ln.log.Error("failed to send notification to LINE", "err", err, "body", body)
+ return false, err
+ }
+
+ return true, nil
+}
+
+func (ln *LineNotifier) SendResolved() bool {
+ return !ln.GetDisableResolveMessage()
+}
diff --git a/alerting/channels/line_test.go b/alerting/channels/line_test.go
new file mode 100644
index 00000000..259045d3
--- /dev/null
+++ b/alerting/channels/line_test.go
@@ -0,0 +1,121 @@
+package channels
+
+import (
+ "context"
+ "net/url"
+ "testing"
+
+ "github.com/prometheus/alertmanager/notify"
+ "github.com/prometheus/alertmanager/types"
+ "github.com/prometheus/common/model"
+ "github.com/stretchr/testify/require"
+
+ "github.com/grafana/grafana/pkg/components/simplejson"
+ "github.com/grafana/grafana/pkg/services/secrets/fakes"
+ secretsManager "github.com/grafana/grafana/pkg/services/secrets/manager"
+)
+
+func TestLineNotifier(t *testing.T) {
+ tmpl := templateForTests(t)
+
+ externalURL, err := url.Parse("http://localhost")
+ require.NoError(t, err)
+ tmpl.ExternalURL = externalURL
+
+ cases := []struct {
+ name string
+ settings string
+ alerts []*types.Alert
+ expHeaders map[string]string
+ expMsg string
+ expInitError string
+ expMsgError error
+ }{
+ {
+ name: "One alert",
+ settings: `{"token": "sometoken"}`,
+ alerts: []*types.Alert{
+ {
+ Alert: model.Alert{
+ Labels: model.LabelSet{"alertname": "alert1", "lbl1": "val1"},
+ Annotations: model.LabelSet{"ann1": "annv1", "__dashboardUid__": "abcd", "__panelId__": "efgh"},
+ },
+ },
+ },
+ expHeaders: map[string]string{
+ "Authorization": "Bearer sometoken",
+ "Content-Type": "application/x-www-form-urlencoded;charset=UTF-8",
+ },
+ expMsg: "message=%5BFIRING%3A1%5D++%28val1%29%0Ahttp%3A%2Flocalhost%2Falerting%2Flist%0A%0A%2A%2AFiring%2A%2A%0A%0AValue%3A+%5Bno+value%5D%0ALabels%3A%0A+-+alertname+%3D+alert1%0A+-+lbl1+%3D+val1%0AAnnotations%3A%0A+-+ann1+%3D+annv1%0ASilence%3A+http%3A%2F%2Flocalhost%2Falerting%2Fsilence%2Fnew%3Falertmanager%3Dgrafana%26matcher%3Dalertname%253Dalert1%26matcher%3Dlbl1%253Dval1%0ADashboard%3A+http%3A%2F%2Flocalhost%2Fd%2Fabcd%0APanel%3A+http%3A%2F%2Flocalhost%2Fd%2Fabcd%3FviewPanel%3Defgh%0A",
+ expMsgError: nil,
+ }, {
+ name: "Multiple alerts",
+ settings: `{"token": "sometoken"}`,
+ alerts: []*types.Alert{
+ {
+ Alert: model.Alert{
+ Labels: model.LabelSet{"alertname": "alert1", "lbl1": "val1"},
+ Annotations: model.LabelSet{"ann1": "annv1"},
+ },
+ }, {
+ Alert: model.Alert{
+ Labels: model.LabelSet{"alertname": "alert1", "lbl1": "val2"},
+ Annotations: model.LabelSet{"ann1": "annv2"},
+ },
+ },
+ },
+ expHeaders: map[string]string{
+ "Authorization": "Bearer sometoken",
+ "Content-Type": "application/x-www-form-urlencoded;charset=UTF-8",
+ },
+ expMsg: "message=%5BFIRING%3A2%5D++%0Ahttp%3A%2Flocalhost%2Falerting%2Flist%0A%0A%2A%2AFiring%2A%2A%0A%0AValue%3A+%5Bno+value%5D%0ALabels%3A%0A+-+alertname+%3D+alert1%0A+-+lbl1+%3D+val1%0AAnnotations%3A%0A+-+ann1+%3D+annv1%0ASilence%3A+http%3A%2F%2Flocalhost%2Falerting%2Fsilence%2Fnew%3Falertmanager%3Dgrafana%26matcher%3Dalertname%253Dalert1%26matcher%3Dlbl1%253Dval1%0A%0AValue%3A+%5Bno+value%5D%0ALabels%3A%0A+-+alertname+%3D+alert1%0A+-+lbl1+%3D+val2%0AAnnotations%3A%0A+-+ann1+%3D+annv2%0ASilence%3A+http%3A%2F%2Flocalhost%2Falerting%2Fsilence%2Fnew%3Falertmanager%3Dgrafana%26matcher%3Dalertname%253Dalert1%26matcher%3Dlbl1%253Dval2%0A",
+ expMsgError: nil,
+ }, {
+ name: "Token missing",
+ settings: `{}`,
+ expInitError: `could not find token in settings`,
+ },
+ }
+
+ for _, c := range cases {
+ t.Run(c.name, func(t *testing.T) {
+ settingsJSON, err := simplejson.NewJson([]byte(c.settings))
+ require.NoError(t, err)
+ secureSettings := make(map[string][]byte)
+
+ m := &NotificationChannelConfig{
+ Name: "line_testing",
+ Type: "line",
+ Settings: settingsJSON,
+ SecureSettings: secureSettings,
+ }
+
+ webhookSender := mockNotificationService()
+ secretsService := secretsManager.SetupTestService(t, fakes.NewFakeSecretsStore())
+ decryptFn := secretsService.GetDecryptedValue
+ cfg, err := NewLineConfig(m, decryptFn)
+ if c.expInitError != "" {
+ require.Error(t, err)
+ require.Equal(t, c.expInitError, err.Error())
+ return
+ }
+ require.NoError(t, err)
+
+ ctx := notify.WithGroupKey(context.Background(), "alertname")
+ ctx = notify.WithGroupLabels(ctx, model.LabelSet{"alertname": ""})
+ pn := NewLineNotifier(cfg, webhookSender, tmpl)
+ ok, err := pn.Notify(ctx, c.alerts...)
+ if c.expMsgError != nil {
+ require.False(t, ok)
+ require.Error(t, err)
+ require.Equal(t, c.expMsgError.Error(), err.Error())
+ return
+ }
+ require.NoError(t, err)
+ require.True(t, ok)
+
+ require.Equal(t, c.expHeaders, webhookSender.Webhook.HttpHeader)
+ require.Equal(t, c.expMsg, webhookSender.Webhook.Body)
+ })
+ }
+}
diff --git a/alerting/channels/opsgenie.go b/alerting/channels/opsgenie.go
new file mode 100644
index 00000000..1e55b66b
--- /dev/null
+++ b/alerting/channels/opsgenie.go
@@ -0,0 +1,292 @@
+package channels
+
+import (
+ "context"
+ "encoding/json"
+ "errors"
+ "fmt"
+ "net/http"
+ "sort"
+ "strings"
+
+ "github.com/prometheus/alertmanager/notify"
+ "github.com/prometheus/alertmanager/template"
+ "github.com/prometheus/alertmanager/types"
+ "github.com/prometheus/common/model"
+
+ "github.com/grafana/grafana/pkg/components/simplejson"
+ "github.com/grafana/grafana/pkg/infra/log"
+ "github.com/grafana/grafana/pkg/models"
+ ngmodels "github.com/grafana/grafana/pkg/services/ngalert/models"
+ "github.com/grafana/grafana/pkg/services/notifications"
+)
+
+const (
+ OpsgenieSendTags = "tags"
+ OpsgenieSendDetails = "details"
+ OpsgenieSendBoth = "both"
+)
+
+var (
+ OpsgenieAlertURL = "https://api.opsgenie.com/v2/alerts"
+ ValidPriorities = map[string]bool{"P1": true, "P2": true, "P3": true, "P4": true, "P5": true}
+)
+
+// OpsgenieNotifier is responsible for sending alert notifications to Opsgenie.
+type OpsgenieNotifier struct {
+ *Base
+ APIKey string
+ APIUrl string
+ Message string
+ Description string
+ AutoClose bool
+ OverridePriority bool
+ SendTagsAs string
+ tmpl *template.Template
+ log log.Logger
+ ns notifications.WebhookSender
+ images ImageStore
+}
+
+type OpsgenieConfig struct {
+ *NotificationChannelConfig
+ APIKey string
+ APIUrl string
+ Message string
+ Description string
+ AutoClose bool
+ OverridePriority bool
+ SendTagsAs string
+}
+
+func OpsgenieFactory(fc FactoryConfig) (NotificationChannel, error) {
+ cfg, err := NewOpsgenieConfig(fc.Config, fc.DecryptFunc)
+ if err != nil {
+ return nil, receiverInitError{
+ Reason: err.Error(),
+ Cfg: *fc.Config,
+ }
+ }
+ return NewOpsgenieNotifier(cfg, fc.NotificationService, fc.ImageStore, fc.Template, fc.DecryptFunc), nil
+}
+
+func NewOpsgenieConfig(config *NotificationChannelConfig, decryptFunc GetDecryptedValueFn) (*OpsgenieConfig, error) {
+ apiKey := decryptFunc(context.Background(), config.SecureSettings, "apiKey", config.Settings.Get("apiKey").MustString())
+ if apiKey == "" {
+ return nil, errors.New("could not find api key property in settings")
+ }
+ sendTagsAs := config.Settings.Get("sendTagsAs").MustString(OpsgenieSendTags)
+ if sendTagsAs != OpsgenieSendTags &&
+ sendTagsAs != OpsgenieSendDetails &&
+ sendTagsAs != OpsgenieSendBoth {
+ return nil, fmt.Errorf("invalid value for sendTagsAs: %q", sendTagsAs)
+ }
+ return &OpsgenieConfig{
+ NotificationChannelConfig: config,
+ APIKey: apiKey,
+ APIUrl: config.Settings.Get("apiUrl").MustString(OpsgenieAlertURL),
+ AutoClose: config.Settings.Get("autoClose").MustBool(true),
+ OverridePriority: config.Settings.Get("overridePriority").MustBool(true),
+ Message: config.Settings.Get("message").MustString(`{{ template "default.title" . }}`),
+ Description: config.Settings.Get("description").MustString(""),
+ SendTagsAs: sendTagsAs,
+ }, nil
+}
+
+// NewOpsgenieNotifier is the constructor for the Opsgenie notifier
+func NewOpsgenieNotifier(config *OpsgenieConfig, ns notifications.WebhookSender, images ImageStore, t *template.Template, fn GetDecryptedValueFn) *OpsgenieNotifier {
+ return &OpsgenieNotifier{
+ Base: NewBase(&models.AlertNotification{
+ Uid: config.UID,
+ Name: config.Name,
+ Type: config.Type,
+ DisableResolveMessage: config.DisableResolveMessage,
+ Settings: config.Settings,
+ }),
+ APIKey: config.APIKey,
+ APIUrl: config.APIUrl,
+ Description: config.Description,
+ Message: config.Message,
+ AutoClose: config.AutoClose,
+ OverridePriority: config.OverridePriority,
+ SendTagsAs: config.SendTagsAs,
+ tmpl: t,
+ log: log.New("alerting.notifier." + config.Name),
+ ns: ns,
+ images: images,
+ }
+}
+
+// Notify sends an alert notification to Opsgenie
+func (on *OpsgenieNotifier) Notify(ctx context.Context, as ...*types.Alert) (bool, error) {
+ on.log.Debug("executing Opsgenie notification", "notification", on.Name)
+
+ alerts := types.Alerts(as...)
+ if alerts.Status() == model.AlertResolved && !on.SendResolved() {
+ on.log.Debug("not sending a trigger to Opsgenie", "status", alerts.Status(), "auto resolve", on.SendResolved())
+ return true, nil
+ }
+
+ bodyJSON, url, err := on.buildOpsgenieMessage(ctx, alerts, as)
+ if err != nil {
+ return false, fmt.Errorf("build Opsgenie message: %w", err)
+ }
+
+ if url == "" {
+ // Resolved alert with no auto close.
+ // Hence skip sending anything.
+ return true, nil
+ }
+
+ body, err := json.Marshal(bodyJSON)
+ if err != nil {
+ return false, fmt.Errorf("marshal json: %w", err)
+ }
+
+ cmd := &models.SendWebhookSync{
+ Url: url,
+ Body: string(body),
+ HttpMethod: http.MethodPost,
+ HttpHeader: map[string]string{
+ "Content-Type": "application/json",
+ "Authorization": fmt.Sprintf("GenieKey %s", on.APIKey),
+ },
+ }
+
+ if err := on.ns.SendWebhookSync(ctx, cmd); err != nil {
+ return false, fmt.Errorf("send notification to Opsgenie: %w", err)
+ }
+
+ return true, nil
+}
+
+func (on *OpsgenieNotifier) buildOpsgenieMessage(ctx context.Context, alerts model.Alerts, as []*types.Alert) (payload *simplejson.Json, apiURL string, err error) {
+ key, err := notify.ExtractGroupKey(ctx)
+ if err != nil {
+ return nil, "", err
+ }
+
+ var (
+ alias = key.Hash()
+ bodyJSON = simplejson.New()
+ details = simplejson.New()
+ )
+
+ if alerts.Status() == model.AlertResolved {
+ // For resolved notification, we only need the source.
+ // Don't need to run other templates.
+ if on.AutoClose {
+ bodyJSON := simplejson.New()
+ bodyJSON.Set("source", "Grafana")
+ apiURL = fmt.Sprintf("%s/%s/close?identifierType=alias", on.APIUrl, alias)
+ return bodyJSON, apiURL, nil
+ }
+ return nil, "", nil
+ }
+
+ ruleURL := joinUrlPath(on.tmpl.ExternalURL.String(), "/alerting/list", on.log)
+
+ var tmplErr error
+ tmpl, data := TmplText(ctx, on.tmpl, as, on.log, &tmplErr)
+
+ titleTmpl := on.Message
+ if strings.TrimSpace(titleTmpl) == "" {
+ titleTmpl = `{{ template "default.title" . }}`
+ }
+
+ title := tmpl(titleTmpl)
+ if len(title) > 130 {
+ title = title[:127] + "..."
+ }
+
+ description := tmpl(on.Description)
+ if strings.TrimSpace(description) == "" {
+ description = fmt.Sprintf(
+ "%s\n%s\n\n%s",
+ tmpl(DefaultMessageTitleEmbed),
+ ruleURL,
+ tmpl(`{{ template "default.message" . }}`),
+ )
+ }
+
+ var priority string
+
+ // In the new alerting system we've moved away from the grafana-tags. Instead, annotations on the rule itself should be used.
+ lbls := make(map[string]string, len(data.CommonLabels))
+ for k, v := range data.CommonLabels {
+ lbls[k] = tmpl(v)
+
+ if k == "og_priority" {
+ if ValidPriorities[v] {
+ priority = v
+ }
+ }
+ }
+
+ // Check for templating errors
+ if tmplErr != nil {
+ on.log.Warn("failed to template Opsgenie message", "err", tmplErr.Error())
+ tmplErr = nil
+ }
+
+ bodyJSON.Set("message", title)
+ bodyJSON.Set("source", "Grafana")
+ bodyJSON.Set("alias", alias)
+ bodyJSON.Set("description", description)
+ details.Set("url", ruleURL)
+
+ if on.sendDetails() {
+ for k, v := range lbls {
+ details.Set(k, v)
+ }
+
+ images := []string{}
+ _ = withStoredImages(ctx, on.log, on.images,
+ func(_ int, image ngmodels.Image) error {
+ if len(image.URL) == 0 {
+ return nil
+ }
+ images = append(images, image.URL)
+ return nil
+ },
+ as...)
+
+ if len(images) != 0 {
+ details.Set("image_urls", images)
+ }
+ }
+
+ tags := make([]string, 0, len(lbls))
+ if on.sendTags() {
+ for k, v := range lbls {
+ tags = append(tags, fmt.Sprintf("%s:%s", k, v))
+ }
+ }
+ sort.Strings(tags)
+
+ if priority != "" && on.OverridePriority {
+ bodyJSON.Set("priority", priority)
+ }
+
+ bodyJSON.Set("tags", tags)
+ bodyJSON.Set("details", details)
+ apiURL = tmpl(on.APIUrl)
+ if tmplErr != nil {
+ on.log.Warn("failed to template Opsgenie URL", "err", tmplErr.Error(), "fallback", on.APIUrl)
+ apiURL = on.APIUrl
+ }
+
+ return bodyJSON, apiURL, nil
+}
+
+func (on *OpsgenieNotifier) SendResolved() bool {
+ return !on.GetDisableResolveMessage()
+}
+
+func (on *OpsgenieNotifier) sendDetails() bool {
+ return on.SendTagsAs == OpsgenieSendDetails || on.SendTagsAs == OpsgenieSendBoth
+}
+
+func (on *OpsgenieNotifier) sendTags() bool {
+ return on.SendTagsAs == OpsgenieSendTags || on.SendTagsAs == OpsgenieSendBoth
+}
diff --git a/alerting/channels/opsgenie_test.go b/alerting/channels/opsgenie_test.go
new file mode 100644
index 00000000..04b50751
--- /dev/null
+++ b/alerting/channels/opsgenie_test.go
@@ -0,0 +1,277 @@
+package channels
+
+import (
+ "context"
+ "net/url"
+ "testing"
+ "time"
+
+ "github.com/grafana/grafana/pkg/components/simplejson"
+ "github.com/grafana/grafana/pkg/services/secrets/fakes"
+ secretsManager "github.com/grafana/grafana/pkg/services/secrets/manager"
+
+ "github.com/prometheus/alertmanager/notify"
+ "github.com/prometheus/alertmanager/types"
+ "github.com/prometheus/common/model"
+ "github.com/stretchr/testify/require"
+)
+
+func TestOpsgenieNotifier(t *testing.T) {
+ tmpl := templateForTests(t)
+
+ externalURL, err := url.Parse("http://localhost")
+ require.NoError(t, err)
+ tmpl.ExternalURL = externalURL
+
+ cases := []struct {
+ name string
+ settings string
+ alerts []*types.Alert
+ expMsg string
+ expInitError string
+ expMsgError error
+ }{
+ {
+ name: "Default config with one alert",
+ settings: `{"apiKey": "abcdefgh0123456789"}`,
+ alerts: []*types.Alert{
+ {
+ Alert: model.Alert{
+ Labels: model.LabelSet{"alertname": "alert1", "lbl1": "val1"},
+ Annotations: model.LabelSet{"ann1": "annv1", "__dashboardUid__": "abcd", "__panelId__": "efgh"},
+ },
+ },
+ },
+ expMsg: `{
+ "alias": "6e3538104c14b583da237e9693b76debbc17f0f8058ef20492e5853096cf8733",
+ "description": "[FIRING:1] (val1)\nhttp://localhost/alerting/list\n\n**Firing**\n\nValue: [no value]\nLabels:\n - alertname = alert1\n - lbl1 = val1\nAnnotations:\n - ann1 = annv1\nSilence: http://localhost/alerting/silence/new?alertmanager=grafana&matcher=alertname%3Dalert1&matcher=lbl1%3Dval1\nDashboard: http://localhost/d/abcd\nPanel: http://localhost/d/abcd?viewPanel=efgh\n",
+ "details": {
+ "url": "http://localhost/alerting/list"
+ },
+ "message": "[FIRING:1] (val1)",
+ "source": "Grafana",
+ "tags": ["alertname:alert1", "lbl1:val1"]
+ }`,
+ },
+ {
+ name: "Default config with one alert, custom message and description",
+ settings: `{"apiKey": "abcdefgh0123456789", "message": "test message", "description": "test description"}`,
+ alerts: []*types.Alert{
+ {
+ Alert: model.Alert{
+ Labels: model.LabelSet{"alertname": "alert1", "lbl1": "val1"},
+ Annotations: model.LabelSet{"ann1": "annv1", "__dashboardUid__": "abcd", "__panelId__": "efgh"},
+ },
+ },
+ },
+ expMsg: `{
+ "alias": "6e3538104c14b583da237e9693b76debbc17f0f8058ef20492e5853096cf8733",
+ "description": "test description",
+ "details": {
+ "url": "http://localhost/alerting/list"
+ },
+ "message": "test message",
+ "source": "Grafana",
+ "tags": ["alertname:alert1", "lbl1:val1"]
+ }`,
+ },
+ {
+ name: "Default config with one alert, message length > 130",
+ settings: `{
+ "apiKey": "abcdefgh0123456789",
+ "message": "IyJnsW78xQoiBJ7L7NqASv31JCFf0At3r9KUykqBVxSiC6qkDhvDLDW9VImiFcq0Iw2XwFy5fX4FcbTmlkaZzUzjVwx9VUuokhzqQlJVhWDYFqhj3a5wX0LjyvNQjsqT9WaWJAWOJanwOAWon"
+ }`,
+ alerts: []*types.Alert{
+ {
+ Alert: model.Alert{
+ Labels: model.LabelSet{"alertname": "alert1", "lbl1": "val1"},
+ Annotations: model.LabelSet{"ann1": "annv1", "__dashboardUid__": "abcd", "__panelId__": "efgh"},
+ },
+ },
+ },
+ expMsg: `{
+ "alias": "6e3538104c14b583da237e9693b76debbc17f0f8058ef20492e5853096cf8733",
+ "description": "[FIRING:1] (val1)\nhttp://localhost/alerting/list\n\n**Firing**\n\nValue: [no value]\nLabels:\n - alertname = alert1\n - lbl1 = val1\nAnnotations:\n - ann1 = annv1\nSilence: http://localhost/alerting/silence/new?alertmanager=grafana&matcher=alertname%3Dalert1&matcher=lbl1%3Dval1\nDashboard: http://localhost/d/abcd\nPanel: http://localhost/d/abcd?viewPanel=efgh\n",
+ "details": {
+ "url": "http://localhost/alerting/list"
+ },
+ "message": "IyJnsW78xQoiBJ7L7NqASv31JCFf0At3r9KUykqBVxSiC6qkDhvDLDW9VImiFcq0Iw2XwFy5fX4FcbTmlkaZzUzjVwx9VUuokhzqQlJVhWDYFqhj3a5wX0LjyvNQjsq...",
+ "source": "Grafana",
+ "tags": ["alertname:alert1", "lbl1:val1"]
+ }`,
+ },
+ {
+ name: "Default config with one alert, templated message and description",
+ settings: `{"apiKey": "abcdefgh0123456789", "message": "Firing: {{ len .Alerts.Firing }}", "description": "{{ len .Alerts.Firing }} firing, {{ len .Alerts.Resolved }} resolved."}`,
+ alerts: []*types.Alert{
+ {
+ Alert: model.Alert{
+ Labels: model.LabelSet{"alertname": "alert1", "lbl1": "val1"},
+ Annotations: model.LabelSet{"ann1": "annv1", "__dashboardUid__": "abcd", "__panelId__": "efgh"},
+ },
+ },
+ },
+ expMsg: `{
+ "alias": "6e3538104c14b583da237e9693b76debbc17f0f8058ef20492e5853096cf8733",
+ "description": "1 firing, 0 resolved.",
+ "details": {
+ "url": "http://localhost/alerting/list"
+ },
+ "message": "Firing: 1",
+ "source": "Grafana",
+ "tags": ["alertname:alert1", "lbl1:val1"]
+ }`,
+ },
+ {
+ name: "Default config with one alert and send tags as tags, empty description and message",
+ settings: `{
+ "apiKey": "abcdefgh0123456789",
+ "sendTagsAs": "tags",
+ "message": " ",
+ "description": " "
+ }`,
+ alerts: []*types.Alert{
+ {
+ Alert: model.Alert{
+ Labels: model.LabelSet{"alertname": "alert1", "lbl1": "val1"},
+ Annotations: model.LabelSet{"ann1": "annv1"},
+ },
+ },
+ },
+ expMsg: `{
+ "alias": "6e3538104c14b583da237e9693b76debbc17f0f8058ef20492e5853096cf8733",
+ "description": "[FIRING:1] (val1)\nhttp://localhost/alerting/list\n\n**Firing**\n\nValue: [no value]\nLabels:\n - alertname = alert1\n - lbl1 = val1\nAnnotations:\n - ann1 = annv1\nSilence: http://localhost/alerting/silence/new?alertmanager=grafana&matcher=alertname%3Dalert1&matcher=lbl1%3Dval1\n",
+ "details": {
+ "url": "http://localhost/alerting/list"
+ },
+ "message": "[FIRING:1] (val1)",
+ "source": "Grafana",
+ "tags": ["alertname:alert1", "lbl1:val1"]
+ }`,
+ },
+ {
+ name: "Default config with one alert and send tags as details",
+ settings: `{
+ "apiKey": "abcdefgh0123456789",
+ "sendTagsAs": "details"
+ }`,
+ alerts: []*types.Alert{
+ {
+ Alert: model.Alert{
+ Labels: model.LabelSet{"alertname": "alert1", "lbl1": "val1"},
+ Annotations: model.LabelSet{"ann1": "annv1"},
+ },
+ },
+ },
+ expMsg: `{
+ "alias": "6e3538104c14b583da237e9693b76debbc17f0f8058ef20492e5853096cf8733",
+ "description": "[FIRING:1] (val1)\nhttp://localhost/alerting/list\n\n**Firing**\n\nValue: [no value]\nLabels:\n - alertname = alert1\n - lbl1 = val1\nAnnotations:\n - ann1 = annv1\nSilence: http://localhost/alerting/silence/new?alertmanager=grafana&matcher=alertname%3Dalert1&matcher=lbl1%3Dval1\n",
+ "details": {
+ "alertname": "alert1",
+ "lbl1": "val1",
+ "url": "http://localhost/alerting/list"
+ },
+ "message": "[FIRING:1] (val1)",
+ "source": "Grafana",
+ "tags": []
+ }`,
+ },
+ {
+ name: "Custom config with multiple alerts and send tags as both details and tag",
+ settings: `{
+ "apiKey": "abcdefgh0123456789",
+ "sendTagsAs": "both"
+ }`,
+ alerts: []*types.Alert{
+ {
+ Alert: model.Alert{
+ Labels: model.LabelSet{"alertname": "alert1", "lbl1": "val1"},
+ Annotations: model.LabelSet{"ann1": "annv1"},
+ },
+ }, {
+ Alert: model.Alert{
+ Labels: model.LabelSet{"alertname": "alert1", "lbl1": "val2"},
+ Annotations: model.LabelSet{"ann1": "annv1"},
+ },
+ },
+ },
+ expMsg: `{
+ "alias": "6e3538104c14b583da237e9693b76debbc17f0f8058ef20492e5853096cf8733",
+ "description": "[FIRING:2] \nhttp://localhost/alerting/list\n\n**Firing**\n\nValue: [no value]\nLabels:\n - alertname = alert1\n - lbl1 = val1\nAnnotations:\n - ann1 = annv1\nSilence: http://localhost/alerting/silence/new?alertmanager=grafana&matcher=alertname%3Dalert1&matcher=lbl1%3Dval1\n\nValue: [no value]\nLabels:\n - alertname = alert1\n - lbl1 = val2\nAnnotations:\n - ann1 = annv1\nSilence: http://localhost/alerting/silence/new?alertmanager=grafana&matcher=alertname%3Dalert1&matcher=lbl1%3Dval2\n",
+ "details": {
+ "alertname": "alert1",
+ "url": "http://localhost/alerting/list"
+ },
+ "message": "[FIRING:2] ",
+ "source": "Grafana",
+ "tags": ["alertname:alert1"]
+ }`,
+ expMsgError: nil,
+ },
+ {
+ name: "Resolved is not sent when auto close is false",
+ settings: `{"apiKey": "abcdefgh0123456789", "autoClose": false}`,
+ alerts: []*types.Alert{
+ {
+ Alert: model.Alert{
+ Labels: model.LabelSet{"alertname": "alert1", "lbl1": "val1"},
+ Annotations: model.LabelSet{"ann1": "annv1"},
+ EndsAt: time.Now().Add(-1 * time.Minute),
+ },
+ },
+ },
+ },
+ {
+ name: "Error when incorrect settings",
+ settings: `{}`,
+ expInitError: `could not find api key property in settings`,
+ },
+ }
+
+ for _, c := range cases {
+ t.Run(c.name, func(t *testing.T) {
+ settingsJSON, err := simplejson.NewJson([]byte(c.settings))
+ require.NoError(t, err)
+ secureSettings := make(map[string][]byte)
+
+ m := &NotificationChannelConfig{
+ Name: "opsgenie_testing",
+ Type: "opsgenie",
+ Settings: settingsJSON,
+ SecureSettings: secureSettings,
+ }
+
+ webhookSender := mockNotificationService()
+ webhookSender.Webhook.Body = ""
+ secretsService := secretsManager.SetupTestService(t, fakes.NewFakeSecretsStore())
+ decryptFn := secretsService.GetDecryptedValue
+ cfg, err := NewOpsgenieConfig(m, decryptFn)
+ if c.expInitError != "" {
+ require.Error(t, err)
+ require.Equal(t, c.expInitError, err.Error())
+ return
+ }
+ require.NoError(t, err)
+
+ ctx := notify.WithGroupKey(context.Background(), "alertname")
+ ctx = notify.WithGroupLabels(ctx, model.LabelSet{"alertname": ""})
+ pn := NewOpsgenieNotifier(cfg, webhookSender, &UnavailableImageStore{}, tmpl, decryptFn)
+ ok, err := pn.Notify(ctx, c.alerts...)
+ if c.expMsgError != nil {
+ require.False(t, ok)
+ require.Error(t, err)
+ require.Equal(t, c.expMsgError.Error(), err.Error())
+ return
+ }
+ require.True(t, ok)
+ require.NoError(t, err)
+
+ if c.expMsg == "" {
+ // No notification was expected.
+ require.Equal(t, "", webhookSender.Webhook.Body)
+ } else {
+ require.JSONEq(t, c.expMsg, webhookSender.Webhook.Body)
+ }
+ })
+ }
+}
diff --git a/alerting/channels/pagerduty.go b/alerting/channels/pagerduty.go
new file mode 100644
index 00000000..5e949140
--- /dev/null
+++ b/alerting/channels/pagerduty.go
@@ -0,0 +1,252 @@
+package channels
+
+import (
+ "context"
+ "encoding/json"
+ "errors"
+ "fmt"
+ "os"
+
+ "github.com/prometheus/alertmanager/notify"
+ "github.com/prometheus/alertmanager/template"
+ "github.com/prometheus/alertmanager/types"
+ "github.com/prometheus/common/model"
+
+ "github.com/grafana/grafana/pkg/infra/log"
+ "github.com/grafana/grafana/pkg/models"
+ ngmodels "github.com/grafana/grafana/pkg/services/ngalert/models"
+ "github.com/grafana/grafana/pkg/services/notifications"
+)
+
+const (
+ pagerDutyEventTrigger = "trigger"
+ pagerDutyEventResolve = "resolve"
+)
+
+var (
+ PagerdutyEventAPIURL = "https://events.pagerduty.com/v2/enqueue"
+)
+
+// PagerdutyNotifier is responsible for sending
+// alert notifications to pagerduty
+type PagerdutyNotifier struct {
+ *Base
+ Key string
+ Severity string
+ CustomDetails map[string]string
+ Class string
+ Component string
+ Group string
+ Summary string
+ tmpl *template.Template
+ log log.Logger
+ ns notifications.WebhookSender
+ images ImageStore
+}
+
+type PagerdutyConfig struct {
+ *NotificationChannelConfig
+ Key string
+ Severity string
+ Class string
+ Component string
+ Group string
+ Summary string
+}
+
+func PagerdutyFactory(fc FactoryConfig) (NotificationChannel, error) {
+ cfg, err := NewPagerdutyConfig(fc.Config, fc.DecryptFunc)
+ if err != nil {
+ return nil, receiverInitError{
+ Reason: err.Error(),
+ Cfg: *fc.Config,
+ }
+ }
+ return NewPagerdutyNotifier(cfg, fc.NotificationService, fc.ImageStore, fc.Template), nil
+}
+
+func NewPagerdutyConfig(config *NotificationChannelConfig, decryptFunc GetDecryptedValueFn) (*PagerdutyConfig, error) {
+ key := decryptFunc(context.Background(), config.SecureSettings, "integrationKey", config.Settings.Get("integrationKey").MustString())
+ if key == "" {
+ return nil, errors.New("could not find integration key property in settings")
+ }
+ return &PagerdutyConfig{
+ NotificationChannelConfig: config,
+ Key: key,
+ Severity: config.Settings.Get("severity").MustString("critical"),
+ Class: config.Settings.Get("class").MustString("default"),
+ Component: config.Settings.Get("component").MustString("Grafana"),
+ Group: config.Settings.Get("group").MustString("default"),
+ Summary: config.Settings.Get("summary").MustString(DefaultMessageTitleEmbed),
+ }, nil
+}
+
+// NewPagerdutyNotifier is the constructor for the PagerDuty notifier
+func NewPagerdutyNotifier(config *PagerdutyConfig, ns notifications.WebhookSender, images ImageStore, t *template.Template) *PagerdutyNotifier {
+ return &PagerdutyNotifier{
+ Base: NewBase(&models.AlertNotification{
+ Uid: config.UID,
+ Name: config.Name,
+ Type: config.Type,
+ DisableResolveMessage: config.DisableResolveMessage,
+ Settings: config.Settings,
+ }),
+ Key: config.Key,
+ CustomDetails: map[string]string{
+ "firing": `{{ template "__text_alert_list" .Alerts.Firing }}`,
+ "resolved": `{{ template "__text_alert_list" .Alerts.Resolved }}`,
+ "num_firing": `{{ .Alerts.Firing | len }}`,
+ "num_resolved": `{{ .Alerts.Resolved | len }}`,
+ },
+ Severity: config.Severity,
+ Class: config.Class,
+ Component: config.Component,
+ Group: config.Group,
+ Summary: config.Summary,
+ tmpl: t,
+ log: log.New("alerting.notifier." + config.Name),
+ ns: ns,
+ images: images,
+ }
+}
+
+// Notify sends an alert notification to PagerDuty
+func (pn *PagerdutyNotifier) Notify(ctx context.Context, as ...*types.Alert) (bool, error) {
+ alerts := types.Alerts(as...)
+ if alerts.Status() == model.AlertResolved && !pn.SendResolved() {
+ pn.log.Debug("not sending a trigger to Pagerduty", "status", alerts.Status(), "auto resolve", pn.SendResolved())
+ return true, nil
+ }
+
+ msg, eventType, err := pn.buildPagerdutyMessage(ctx, alerts, as)
+ if err != nil {
+ return false, fmt.Errorf("build pagerduty message: %w", err)
+ }
+
+ body, err := json.Marshal(msg)
+ if err != nil {
+ return false, fmt.Errorf("marshal json: %w", err)
+ }
+
+ pn.log.Info("notifying Pagerduty", "event_type", eventType)
+ cmd := &models.SendWebhookSync{
+ Url: PagerdutyEventAPIURL,
+ Body: string(body),
+ HttpMethod: "POST",
+ HttpHeader: map[string]string{
+ "Content-Type": "application/json",
+ },
+ }
+ if err := pn.ns.SendWebhookSync(ctx, cmd); err != nil {
+ return false, fmt.Errorf("send notification to Pagerduty: %w", err)
+ }
+
+ return true, nil
+}
+
+func (pn *PagerdutyNotifier) buildPagerdutyMessage(ctx context.Context, alerts model.Alerts, as []*types.Alert) (*pagerDutyMessage, string, error) {
+ key, err := notify.ExtractGroupKey(ctx)
+ if err != nil {
+ return nil, "", err
+ }
+
+ eventType := pagerDutyEventTrigger
+ if alerts.Status() == model.AlertResolved {
+ eventType = pagerDutyEventResolve
+ }
+
+ var tmplErr error
+ tmpl, data := TmplText(ctx, pn.tmpl, as, pn.log, &tmplErr)
+
+ details := make(map[string]string, len(pn.CustomDetails))
+ for k, v := range pn.CustomDetails {
+ detail, err := pn.tmpl.ExecuteTextString(v, data)
+ if err != nil {
+ return nil, "", fmt.Errorf("%q: failed to template %q: %w", k, v, err)
+ }
+ details[k] = detail
+ }
+
+ msg := &pagerDutyMessage{
+ Client: "Grafana",
+ ClientURL: pn.tmpl.ExternalURL.String(),
+ RoutingKey: pn.Key,
+ EventAction: eventType,
+ DedupKey: key.Hash(),
+ Links: []pagerDutyLink{{
+ HRef: pn.tmpl.ExternalURL.String(),
+ Text: "External URL",
+ }},
+ Description: tmpl(DefaultMessageTitleEmbed), // TODO: this can be configurable template.
+ Payload: pagerDutyPayload{
+ Component: tmpl(pn.Component),
+ Summary: tmpl(pn.Summary),
+ Severity: tmpl(pn.Severity),
+ CustomDetails: details,
+ Class: tmpl(pn.Class),
+ Group: tmpl(pn.Group),
+ },
+ }
+
+ _ = withStoredImages(ctx, pn.log, pn.images,
+ func(_ int, image ngmodels.Image) error {
+ if len(image.URL) != 0 {
+ msg.Images = append(msg.Images, pagerDutyImage{Src: image.URL})
+ }
+
+ return nil
+ },
+ as...)
+
+ if len(msg.Payload.Summary) > 1024 {
+ // This is the Pagerduty limit.
+ msg.Payload.Summary = msg.Payload.Summary[:1021] + "..."
+ }
+
+ if hostname, err := os.Hostname(); err == nil {
+ // TODO: should this be configured like in Prometheus AM?
+ msg.Payload.Source = hostname
+ }
+
+ if tmplErr != nil {
+ pn.log.Warn("failed to template PagerDuty message", "err", tmplErr.Error())
+ }
+
+ return msg, eventType, nil
+}
+
+func (pn *PagerdutyNotifier) SendResolved() bool {
+ return !pn.GetDisableResolveMessage()
+}
+
+type pagerDutyMessage struct {
+ RoutingKey string `json:"routing_key,omitempty"`
+ ServiceKey string `json:"service_key,omitempty"`
+ DedupKey string `json:"dedup_key,omitempty"`
+ Description string `json:"description,omitempty"`
+ EventAction string `json:"event_action"`
+ Payload pagerDutyPayload `json:"payload"`
+ Client string `json:"client,omitempty"`
+ ClientURL string `json:"client_url,omitempty"`
+ Links []pagerDutyLink `json:"links,omitempty"`
+ Images []pagerDutyImage `json:"images,omitempty"`
+}
+
+type pagerDutyLink struct {
+ HRef string `json:"href"`
+ Text string `json:"text"`
+}
+
+type pagerDutyImage struct {
+ Src string `json:"src"`
+}
+
+type pagerDutyPayload struct {
+ Summary string `json:"summary"`
+ Source string `json:"source"`
+ Severity string `json:"severity"`
+ Class string `json:"class,omitempty"`
+ Component string `json:"component,omitempty"`
+ Group string `json:"group,omitempty"`
+ CustomDetails map[string]string `json:"custom_details,omitempty"`
+}
diff --git a/alerting/channels/pagerduty_test.go b/alerting/channels/pagerduty_test.go
new file mode 100644
index 00000000..560fef13
--- /dev/null
+++ b/alerting/channels/pagerduty_test.go
@@ -0,0 +1,169 @@
+package channels
+
+import (
+ "context"
+ "encoding/json"
+ "net/url"
+ "os"
+ "testing"
+
+ "github.com/grafana/grafana/pkg/components/simplejson"
+ "github.com/grafana/grafana/pkg/services/secrets/fakes"
+ secretsManager "github.com/grafana/grafana/pkg/services/secrets/manager"
+
+ "github.com/prometheus/alertmanager/notify"
+ "github.com/prometheus/alertmanager/types"
+ "github.com/prometheus/common/model"
+ "github.com/stretchr/testify/require"
+)
+
+func TestPagerdutyNotifier(t *testing.T) {
+ tmpl := templateForTests(t)
+
+ externalURL, err := url.Parse("http://localhost")
+ require.NoError(t, err)
+ tmpl.ExternalURL = externalURL
+
+ hostname, err := os.Hostname()
+ require.NoError(t, err)
+
+ cases := []struct {
+ name string
+ settings string
+ alerts []*types.Alert
+ expMsg *pagerDutyMessage
+ expInitError string
+ expMsgError error
+ }{
+ {
+ name: "Default config with one alert",
+ settings: `{"integrationKey": "abcdefgh0123456789"}`,
+ alerts: []*types.Alert{
+ {
+ Alert: model.Alert{
+ Labels: model.LabelSet{"alertname": "alert1", "lbl1": "val1"},
+ Annotations: model.LabelSet{"ann1": "annv1", "__dashboardUid__": "abcd", "__panelId__": "efgh"},
+ },
+ },
+ },
+ expMsg: &pagerDutyMessage{
+ RoutingKey: "abcdefgh0123456789",
+ DedupKey: "6e3538104c14b583da237e9693b76debbc17f0f8058ef20492e5853096cf8733",
+ Description: "[FIRING:1] (val1)",
+ EventAction: "trigger",
+ Payload: pagerDutyPayload{
+ Summary: "[FIRING:1] (val1)",
+ Source: hostname,
+ Severity: "critical",
+ Class: "default",
+ Component: "Grafana",
+ Group: "default",
+ CustomDetails: map[string]string{
+ "firing": "\nValue: [no value]\nLabels:\n - alertname = alert1\n - lbl1 = val1\nAnnotations:\n - ann1 = annv1\nSilence: http://localhost/alerting/silence/new?alertmanager=grafana&matcher=alertname%3Dalert1&matcher=lbl1%3Dval1\nDashboard: http://localhost/d/abcd\nPanel: http://localhost/d/abcd?viewPanel=efgh\n",
+ "num_firing": "1",
+ "num_resolved": "0",
+ "resolved": "",
+ },
+ },
+ Client: "Grafana",
+ ClientURL: "http://localhost",
+ Links: []pagerDutyLink{{HRef: "http://localhost", Text: "External URL"}},
+ },
+ expMsgError: nil,
+ }, {
+ name: "Custom config with multiple alerts",
+ settings: `{
+ "integrationKey": "abcdefgh0123456789",
+ "severity": "warning",
+ "class": "{{ .Status }}",
+ "component": "My Grafana",
+ "group": "my_group"
+ }`,
+ alerts: []*types.Alert{
+ {
+ Alert: model.Alert{
+ Labels: model.LabelSet{"alertname": "alert1", "lbl1": "val1"},
+ Annotations: model.LabelSet{"ann1": "annv1"},
+ },
+ }, {
+ Alert: model.Alert{
+ Labels: model.LabelSet{"alertname": "alert1", "lbl1": "val2"},
+ Annotations: model.LabelSet{"ann1": "annv2"},
+ },
+ },
+ },
+ expMsg: &pagerDutyMessage{
+ RoutingKey: "abcdefgh0123456789",
+ DedupKey: "6e3538104c14b583da237e9693b76debbc17f0f8058ef20492e5853096cf8733",
+ Description: "[FIRING:2] ",
+ EventAction: "trigger",
+ Payload: pagerDutyPayload{
+ Summary: "[FIRING:2] ",
+ Source: hostname,
+ Severity: "warning",
+ Class: "firing",
+ Component: "My Grafana",
+ Group: "my_group",
+ CustomDetails: map[string]string{
+ "firing": "\nValue: [no value]\nLabels:\n - alertname = alert1\n - lbl1 = val1\nAnnotations:\n - ann1 = annv1\nSilence: http://localhost/alerting/silence/new?alertmanager=grafana&matcher=alertname%3Dalert1&matcher=lbl1%3Dval1\n\nValue: [no value]\nLabels:\n - alertname = alert1\n - lbl1 = val2\nAnnotations:\n - ann1 = annv2\nSilence: http://localhost/alerting/silence/new?alertmanager=grafana&matcher=alertname%3Dalert1&matcher=lbl1%3Dval2\n",
+ "num_firing": "2",
+ "num_resolved": "0",
+ "resolved": "",
+ },
+ },
+ Client: "Grafana",
+ ClientURL: "http://localhost",
+ Links: []pagerDutyLink{{HRef: "http://localhost", Text: "External URL"}},
+ },
+ expMsgError: nil,
+ }, {
+ name: "Error in initing",
+ settings: `{}`,
+ expInitError: `could not find integration key property in settings`,
+ },
+ }
+
+ for _, c := range cases {
+ t.Run(c.name, func(t *testing.T) {
+ settingsJSON, err := simplejson.NewJson([]byte(c.settings))
+ require.NoError(t, err)
+ secureSettings := make(map[string][]byte)
+
+ m := &NotificationChannelConfig{
+ Name: "pageduty_testing",
+ Type: "pagerduty",
+ Settings: settingsJSON,
+ SecureSettings: secureSettings,
+ }
+
+ webhookSender := mockNotificationService()
+ secretsService := secretsManager.SetupTestService(t, fakes.NewFakeSecretsStore())
+ decryptFn := secretsService.GetDecryptedValue
+ cfg, err := NewPagerdutyConfig(m, decryptFn)
+ if c.expInitError != "" {
+ require.Error(t, err)
+ require.Equal(t, c.expInitError, err.Error())
+ return
+ }
+ require.NoError(t, err)
+
+ ctx := notify.WithGroupKey(context.Background(), "alertname")
+ ctx = notify.WithGroupLabels(ctx, model.LabelSet{"alertname": ""})
+ pn := NewPagerdutyNotifier(cfg, webhookSender, &UnavailableImageStore{}, tmpl)
+ ok, err := pn.Notify(ctx, c.alerts...)
+ if c.expMsgError != nil {
+ require.False(t, ok)
+ require.Error(t, err)
+ require.Equal(t, c.expMsgError.Error(), err.Error())
+ return
+ }
+ require.True(t, ok)
+ require.NoError(t, err)
+
+ expBody, err := json.Marshal(c.expMsg)
+ require.NoError(t, err)
+
+ require.JSONEq(t, string(expBody), webhookSender.Webhook.Body)
+ })
+ }
+}
diff --git a/alerting/channels/pushover.go b/alerting/channels/pushover.go
new file mode 100644
index 00000000..43c107e8
--- /dev/null
+++ b/alerting/channels/pushover.go
@@ -0,0 +1,297 @@
+package channels
+
+import (
+ "bytes"
+ "context"
+ "errors"
+ "fmt"
+ "io"
+ "mime/multipart"
+ "os"
+ "strconv"
+
+ "github.com/prometheus/alertmanager/template"
+ "github.com/prometheus/alertmanager/types"
+ "github.com/prometheus/common/model"
+
+ "github.com/grafana/grafana/pkg/infra/log"
+ "github.com/grafana/grafana/pkg/models"
+ ngmodels "github.com/grafana/grafana/pkg/services/ngalert/models"
+ "github.com/grafana/grafana/pkg/services/notifications"
+)
+
+const (
+ pushoverMaxFileSize = 1 << 21 // 2MB
+)
+
+var (
+ PushoverEndpoint = "https://api.pushover.net/1/messages.json"
+)
+
+// PushoverNotifier is responsible for sending
+// alert notifications to Pushover
+type PushoverNotifier struct {
+ *Base
+ UserKey string
+ APIToken string
+ AlertingPriority int
+ OKPriority int
+ Retry int
+ Expire int
+ Device string
+ AlertingSound string
+ OKSound string
+ Upload bool
+ Message string
+ tmpl *template.Template
+ log log.Logger
+ images ImageStore
+ ns notifications.WebhookSender
+}
+
+type PushoverConfig struct {
+ *NotificationChannelConfig
+ UserKey string
+ APIToken string
+ AlertingPriority int
+ OKPriority int
+ Retry int
+ Expire int
+ Device string
+ AlertingSound string
+ OKSound string
+ Upload bool
+ Message string
+}
+
+func PushoverFactory(fc FactoryConfig) (NotificationChannel, error) {
+ cfg, err := NewPushoverConfig(fc.Config, fc.DecryptFunc)
+ if err != nil {
+ return nil, receiverInitError{
+ Reason: err.Error(),
+ Cfg: *fc.Config,
+ }
+ }
+ return NewPushoverNotifier(cfg, fc.ImageStore, fc.NotificationService, fc.Template), nil
+}
+
+func NewPushoverConfig(config *NotificationChannelConfig, decryptFunc GetDecryptedValueFn) (*PushoverConfig, error) {
+ userKey := decryptFunc(context.Background(), config.SecureSettings, "userKey", config.Settings.Get("userKey").MustString())
+ if userKey == "" {
+ return nil, errors.New("user key not found")
+ }
+ APIToken := decryptFunc(context.Background(), config.SecureSettings, "apiToken", config.Settings.Get("apiToken").MustString())
+ if APIToken == "" {
+ return nil, errors.New("API token not found")
+ }
+ alertingPriority, err := strconv.Atoi(config.Settings.Get("priority").MustString("0")) // default Normal
+ if err != nil {
+ return nil, fmt.Errorf("failed to convert alerting priority to integer: %w", err)
+ }
+ okPriority, err := strconv.Atoi(config.Settings.Get("okPriority").MustString("0")) // default Normal
+ if err != nil {
+ return nil, fmt.Errorf("failed to convert OK priority to integer: %w", err)
+ }
+ retry, _ := strconv.Atoi(config.Settings.Get("retry").MustString())
+ expire, _ := strconv.Atoi(config.Settings.Get("expire").MustString())
+ return &PushoverConfig{
+ NotificationChannelConfig: config,
+ APIToken: APIToken,
+ UserKey: userKey,
+ AlertingPriority: alertingPriority,
+ OKPriority: okPriority,
+ Retry: retry,
+ Expire: expire,
+ Device: config.Settings.Get("device").MustString(),
+ AlertingSound: config.Settings.Get("sound").MustString(),
+ OKSound: config.Settings.Get("okSound").MustString(),
+ Upload: config.Settings.Get("uploadImage").MustBool(true),
+ Message: config.Settings.Get("message").MustString(`{{ template "default.message" .}}`),
+ }, nil
+}
+
+// NewSlackNotifier is the constructor for the Slack notifier
+func NewPushoverNotifier(config *PushoverConfig, images ImageStore,
+ ns notifications.WebhookSender, t *template.Template) *PushoverNotifier {
+ return &PushoverNotifier{
+ Base: NewBase(&models.AlertNotification{
+ Uid: config.UID,
+ Name: config.Name,
+ Type: config.Type,
+ DisableResolveMessage: config.DisableResolveMessage,
+ Settings: config.Settings,
+ SecureSettings: config.SecureSettings,
+ }),
+ UserKey: config.UserKey,
+ APIToken: config.APIToken,
+ AlertingPriority: config.AlertingPriority,
+ OKPriority: config.OKPriority,
+ Retry: config.Retry,
+ Expire: config.Expire,
+ Device: config.Device,
+ AlertingSound: config.AlertingSound,
+ OKSound: config.OKSound,
+ Upload: config.Upload,
+ Message: config.Message,
+ tmpl: t,
+ log: log.New("alerting.notifier.pushover"),
+ images: images,
+ ns: ns,
+ }
+}
+
+// Notify sends an alert notification to Slack.
+func (pn *PushoverNotifier) Notify(ctx context.Context, as ...*types.Alert) (bool, error) {
+ headers, uploadBody, err := pn.genPushoverBody(ctx, as...)
+ if err != nil {
+ pn.log.Error("Failed to generate body for pushover", "error", err)
+ return false, err
+ }
+
+ cmd := &models.SendWebhookSync{
+ Url: PushoverEndpoint,
+ HttpMethod: "POST",
+ HttpHeader: headers,
+ Body: uploadBody.String(),
+ }
+
+ if err := pn.ns.SendWebhookSync(ctx, cmd); err != nil {
+ pn.log.Error("failed to send pushover notification", "err", err, "webhook", pn.Name)
+ return false, err
+ }
+
+ return true, nil
+}
+func (pn *PushoverNotifier) SendResolved() bool {
+ return !pn.GetDisableResolveMessage()
+}
+
+func (pn *PushoverNotifier) genPushoverBody(ctx context.Context, as ...*types.Alert) (map[string]string, bytes.Buffer, error) {
+ b := bytes.Buffer{}
+ w := multipart.NewWriter(&b)
+
+ // tests use a non-random boundary separator
+ if boundary := GetBoundary(); boundary != "" {
+ err := w.SetBoundary(boundary)
+ if err != nil {
+ return nil, b, err
+ }
+ }
+
+ var tmplErr error
+ tmpl, _ := TmplText(ctx, pn.tmpl, as, pn.log, &tmplErr)
+
+ if err := w.WriteField("user", tmpl(pn.UserKey)); err != nil {
+ return nil, b, fmt.Errorf("failed to write the user: %w", err)
+ }
+
+ if err := w.WriteField("token", pn.APIToken); err != nil {
+ return nil, b, fmt.Errorf("failed to write the token: %w", err)
+ }
+
+ status := types.Alerts(as...).Status()
+ priority := pn.AlertingPriority
+ if status == model.AlertResolved {
+ priority = pn.OKPriority
+ }
+ if err := w.WriteField("priority", strconv.Itoa(priority)); err != nil {
+ return nil, b, fmt.Errorf("failed to write the priority: %w", err)
+ }
+
+ if priority == 2 {
+ if err := w.WriteField("retry", strconv.Itoa(pn.Retry)); err != nil {
+ return nil, b, fmt.Errorf("failed to write retry: %w", err)
+ }
+
+ if err := w.WriteField("expire", strconv.Itoa(pn.Expire)); err != nil {
+ return nil, b, fmt.Errorf("failed to write expire: %w", err)
+ }
+ }
+
+ if pn.Device != "" {
+ if err := w.WriteField("device", tmpl(pn.Device)); err != nil {
+ return nil, b, fmt.Errorf("failed to write the device: %w", err)
+ }
+ }
+
+ if err := w.WriteField("title", tmpl(DefaultMessageTitleEmbed)); err != nil {
+ return nil, b, fmt.Errorf("failed to write the title: %w", err)
+ }
+
+ ruleURL := joinUrlPath(pn.tmpl.ExternalURL.String(), "/alerting/list", pn.log)
+ if err := w.WriteField("url", ruleURL); err != nil {
+ return nil, b, fmt.Errorf("failed to write the URL: %w", err)
+ }
+
+ if err := w.WriteField("url_title", "Show alert rule"); err != nil {
+ return nil, b, fmt.Errorf("failed to write the URL title: %w", err)
+ }
+
+ if err := w.WriteField("message", tmpl(pn.Message)); err != nil {
+ return nil, b, fmt.Errorf("failed write the message: %w", err)
+ }
+
+ // Pushover supports at most one image attachment with a maximum size of pushoverMaxFileSize.
+ // If the image is larger than pushoverMaxFileSize then return an error.
+ _ = withStoredImages(ctx, pn.log, pn.images, func(index int, image ngmodels.Image) error {
+ f, err := os.Open(image.Path)
+ if err != nil {
+ return fmt.Errorf("failed to open the image: %w", err)
+ }
+ defer func() {
+ if err := f.Close(); err != nil {
+ pn.log.Error("failed to close the image", "file", image.Path)
+ }
+ }()
+
+ fileInfo, err := f.Stat()
+ if err != nil {
+ return fmt.Errorf("failed to stat the image: %w", err)
+ }
+
+ if fileInfo.Size() > pushoverMaxFileSize {
+ return fmt.Errorf("image would exceeded maximum file size: %d", fileInfo.Size())
+ }
+
+ fw, err := w.CreateFormFile("attachment", image.Path)
+ if err != nil {
+ return fmt.Errorf("failed to create form file for the image: %w", err)
+ }
+
+ if _, err = io.Copy(fw, f); err != nil {
+ return fmt.Errorf("failed to copy the image to the form file: %w", err)
+ }
+
+ return ErrImagesDone
+ }, as...)
+
+ var sound string
+ if status == model.AlertResolved {
+ sound = tmpl(pn.OKSound)
+ } else {
+ sound = tmpl(pn.AlertingSound)
+ }
+ if sound != "default" {
+ if err := w.WriteField("sound", sound); err != nil {
+ return nil, b, fmt.Errorf("failed to write the sound: %w", err)
+ }
+ }
+
+ // Mark the message as HTML
+ if err := w.WriteField("html", "1"); err != nil {
+ return nil, b, fmt.Errorf("failed to mark the message as HTML: %w", err)
+ }
+ if err := w.Close(); err != nil {
+ return nil, b, fmt.Errorf("failed to close the multipart request: %w", err)
+ }
+
+ if tmplErr != nil {
+ pn.log.Warn("failed to template pushover message", "err", tmplErr.Error())
+ }
+
+ headers := map[string]string{
+ "Content-Type": w.FormDataContentType(),
+ }
+
+ return headers, b, nil
+}
diff --git a/alerting/channels/pushover_test.go b/alerting/channels/pushover_test.go
new file mode 100644
index 00000000..ae8cd654
--- /dev/null
+++ b/alerting/channels/pushover_test.go
@@ -0,0 +1,197 @@
+package channels
+
+import (
+ "bytes"
+ "context"
+ "errors"
+ "fmt"
+ "io"
+ "mime/multipart"
+ "net/url"
+ "strings"
+ "testing"
+
+ "github.com/grafana/grafana/pkg/components/simplejson"
+ "github.com/grafana/grafana/pkg/services/secrets/fakes"
+ secretsManager "github.com/grafana/grafana/pkg/services/secrets/manager"
+
+ "github.com/prometheus/alertmanager/notify"
+ "github.com/prometheus/alertmanager/types"
+ "github.com/prometheus/common/model"
+ "github.com/stretchr/testify/assert"
+ "github.com/stretchr/testify/require"
+)
+
+func TestPushoverNotifier(t *testing.T) {
+ tmpl := templateForTests(t)
+
+ images := newFakeImageStoreWithFile(t, 2)
+
+ externalURL, err := url.Parse("http://localhost")
+ require.NoError(t, err)
+ tmpl.ExternalURL = externalURL
+
+ cases := []struct {
+ name string
+ settings string
+ alerts []*types.Alert
+ expMsg map[string]string
+ expInitError string
+ expMsgError error
+ }{
+ {
+ name: "Correct config with single alert",
+ settings: `{
+ "userKey": "",
+ "apiToken": ""
+ }`,
+ alerts: []*types.Alert{
+ {
+ Alert: model.Alert{
+ Labels: model.LabelSet{"__alert_rule_uid__": "rule uid", "alertname": "alert1", "lbl1": "val1"},
+ Annotations: model.LabelSet{"ann1": "annv1", "__dashboardUid__": "abcd", "__panelId__": "efgh", "__alertImageToken__": "test-image-1"},
+ },
+ },
+ },
+ expMsg: map[string]string{
+ "user": "",
+ "token": "",
+ "priority": "0",
+ "sound": "",
+ "title": "[FIRING:1] (val1)",
+ "url": "http://localhost/alerting/list",
+ "url_title": "Show alert rule",
+ "message": "**Firing**\n\nValue: [no value]\nLabels:\n - alertname = alert1\n - lbl1 = val1\nAnnotations:\n - ann1 = annv1\nSilence: http://localhost/alerting/silence/new?alertmanager=grafana&matcher=alertname%3Dalert1&matcher=lbl1%3Dval1\nDashboard: http://localhost/d/abcd\nPanel: http://localhost/d/abcd?viewPanel=efgh\n",
+ "attachment": "\x89PNG\r\n\x1a\n\x00\x00\x00\rIHDR\x00\x00\x00\x01\x00\x00\x00\x01\b\x04\x00\x00\x00\xb5\x1c\f\x02\x00\x00\x00\vIDATx\xdacd`\x00\x00\x00\x06\x00\x020\x81\xd0/\x00\x00\x00\x00IEND\xaeB`\x82",
+ "html": "1",
+ },
+ expMsgError: nil,
+ },
+ {
+ name: "Custom config with multiple alerts",
+ settings: `{
+ "userKey": "",
+ "apiToken": "",
+ "device": "device",
+ "priority": "2",
+ "okpriority": "0",
+ "retry": "30",
+ "expire": "86400",
+ "sound": "echo",
+ "oksound": "magic",
+ "message": "{{ len .Alerts.Firing }} alerts are firing, {{ len .Alerts.Resolved }} are resolved"
+ }`,
+ alerts: []*types.Alert{
+ {
+ Alert: model.Alert{
+ Labels: model.LabelSet{"__alert_rule_uid__": "rule uid", "alertname": "alert1", "lbl1": "val1"},
+ Annotations: model.LabelSet{"ann1": "annv1", "__alertImageToken__": "test-image-1"},
+ },
+ }, {
+ Alert: model.Alert{
+ Labels: model.LabelSet{"alertname": "alert1", "lbl1": "val2"},
+ Annotations: model.LabelSet{"ann1": "annv2", "__alertImageToken__": "test-image-2"},
+ },
+ },
+ },
+ expMsg: map[string]string{
+ "user": "",
+ "token": "",
+ "priority": "2",
+ "sound": "echo",
+ "title": "[FIRING:2] ",
+ "url": "http://localhost/alerting/list",
+ "url_title": "Show alert rule",
+ "message": "2 alerts are firing, 0 are resolved",
+ "attachment": "\x89PNG\r\n\x1a\n\x00\x00\x00\rIHDR\x00\x00\x00\x01\x00\x00\x00\x01\b\x04\x00\x00\x00\xb5\x1c\f\x02\x00\x00\x00\vIDATx\xdacd`\x00\x00\x00\x06\x00\x020\x81\xd0/\x00\x00\x00\x00IEND\xaeB`\x82",
+ "html": "1",
+ "retry": "30",
+ "expire": "86400",
+ "device": "device",
+ },
+ expMsgError: nil,
+ },
+ {
+ name: "Missing user key",
+ settings: `{
+ "apiToken": ""
+ }`,
+ expInitError: `user key not found`,
+ }, {
+ name: "Missing api key",
+ settings: `{
+ "userKey": ""
+ }`,
+ expInitError: `API token not found`,
+ },
+ }
+
+ for _, c := range cases {
+ origGetBoundary := GetBoundary
+ boundary := "abcd"
+ GetBoundary = func() string {
+ return boundary
+ }
+ t.Cleanup(func() {
+ GetBoundary = origGetBoundary
+ })
+
+ t.Run(c.name, func(t *testing.T) {
+ settingsJSON, err := simplejson.NewJson([]byte(c.settings))
+ require.NoError(t, err)
+ secureSettings := make(map[string][]byte)
+
+ m := &NotificationChannelConfig{
+ Name: "pushover_testing",
+ Type: "pushover",
+ Settings: settingsJSON,
+ SecureSettings: secureSettings,
+ }
+
+ webhookSender := mockNotificationService()
+ secretsService := secretsManager.SetupTestService(t, fakes.NewFakeSecretsStore())
+ decryptFn := secretsService.GetDecryptedValue
+ cfg, err := NewPushoverConfig(m, decryptFn)
+ if c.expInitError != "" {
+ require.Error(t, err)
+ require.Equal(t, c.expInitError, err.Error())
+ return
+ }
+ require.NoError(t, err)
+
+ ctx := notify.WithGroupKey(context.Background(), "alertname")
+ ctx = notify.WithGroupLabels(ctx, model.LabelSet{"alertname": ""})
+ pn := NewPushoverNotifier(cfg, images, webhookSender, tmpl)
+ ok, err := pn.Notify(ctx, c.alerts...)
+ if c.expMsgError != nil {
+ require.Error(t, err)
+ require.False(t, ok)
+ require.Equal(t, c.expMsgError.Error(), err.Error())
+ return
+ }
+ require.NoError(t, err)
+ require.True(t, ok)
+
+ bodyReader := multipart.NewReader(strings.NewReader(webhookSender.Webhook.Body), boundary)
+ for {
+ part, err := bodyReader.NextPart()
+ if part == nil || errors.Is(err, io.EOF) {
+ assert.Empty(t, c.expMsg, fmt.Sprintf("expected fields %v", c.expMsg))
+ break
+ }
+ formField := part.FormName()
+ expected, ok := c.expMsg[formField]
+ assert.True(t, ok, fmt.Sprintf("unexpected field %s", formField))
+ actual := []byte("")
+ if expected != "" {
+ buf := new(bytes.Buffer)
+ _, err := buf.ReadFrom(part)
+ require.NoError(t, err)
+ actual = buf.Bytes()
+ }
+ assert.Equal(t, expected, string(actual))
+ delete(c.expMsg, formField)
+ }
+ })
+ }
+}
diff --git a/alerting/channels/sensugo.go b/alerting/channels/sensugo.go
new file mode 100644
index 00000000..2407e340
--- /dev/null
+++ b/alerting/channels/sensugo.go
@@ -0,0 +1,206 @@
+package channels
+
+import (
+ "context"
+ "encoding/json"
+ "errors"
+ "fmt"
+ "strings"
+
+ "github.com/prometheus/alertmanager/template"
+ "github.com/prometheus/alertmanager/types"
+ "github.com/prometheus/common/model"
+
+ "github.com/grafana/grafana/pkg/infra/log"
+ "github.com/grafana/grafana/pkg/models"
+ ngmodels "github.com/grafana/grafana/pkg/services/ngalert/models"
+ "github.com/grafana/grafana/pkg/services/notifications"
+)
+
+type SensuGoNotifier struct {
+ *Base
+ log log.Logger
+ images ImageStore
+ ns notifications.WebhookSender
+ tmpl *template.Template
+
+ URL string
+ Entity string
+ Check string
+ Namespace string
+ Handler string
+ APIKey string
+ Message string
+}
+
+type SensuGoConfig struct {
+ *NotificationChannelConfig
+ URL string
+ Entity string
+ Check string
+ Namespace string
+ Handler string
+ APIKey string
+ Message string
+}
+
+func SensuGoFactory(fc FactoryConfig) (NotificationChannel, error) {
+ cfg, err := NewSensuGoConfig(fc.Config, fc.DecryptFunc)
+ if err != nil {
+ return nil, receiverInitError{
+ Reason: err.Error(),
+ Cfg: *fc.Config,
+ }
+ }
+ return NewSensuGoNotifier(cfg, fc.ImageStore, fc.NotificationService, fc.Template), nil
+}
+
+func NewSensuGoConfig(config *NotificationChannelConfig, decryptFunc GetDecryptedValueFn) (*SensuGoConfig, error) {
+ url := config.Settings.Get("url").MustString()
+ if url == "" {
+ return nil, errors.New("could not find URL property in settings")
+ }
+ apikey := decryptFunc(context.Background(), config.SecureSettings, "apikey", config.Settings.Get("apikey").MustString())
+ if apikey == "" {
+ return nil, errors.New("could not find the API key property in settings")
+ }
+ return &SensuGoConfig{
+ NotificationChannelConfig: config,
+ URL: url,
+ Entity: config.Settings.Get("entity").MustString(),
+ Check: config.Settings.Get("check").MustString(),
+ Namespace: config.Settings.Get("namespace").MustString(),
+ Handler: config.Settings.Get("handler").MustString(),
+ APIKey: apikey,
+ Message: config.Settings.Get("message").MustString(`{{ template "default.message" .}}`),
+ }, nil
+}
+
+// NewSensuGoNotifier is the constructor for the SensuGo notifier
+func NewSensuGoNotifier(config *SensuGoConfig, images ImageStore, ns notifications.WebhookSender, t *template.Template) *SensuGoNotifier {
+ return &SensuGoNotifier{
+ Base: NewBase(&models.AlertNotification{
+ Uid: config.UID,
+ Name: config.Name,
+ Type: config.Type,
+ DisableResolveMessage: config.DisableResolveMessage,
+ Settings: config.Settings,
+ SecureSettings: config.SecureSettings,
+ }),
+ URL: config.URL,
+ Entity: config.Entity,
+ Check: config.Check,
+ Namespace: config.Namespace,
+ Handler: config.Handler,
+ APIKey: config.APIKey,
+ Message: config.Message,
+ log: log.New("alerting.notifier.sensugo"),
+ images: images,
+ ns: ns,
+ tmpl: t,
+ }
+}
+
+// Notify sends an alert notification to Sensu Go
+func (sn *SensuGoNotifier) Notify(ctx context.Context, as ...*types.Alert) (bool, error) {
+ sn.log.Debug("sending Sensu Go result")
+
+ var tmplErr error
+ tmpl, _ := TmplText(ctx, sn.tmpl, as, sn.log, &tmplErr)
+
+ // Sensu Go alerts require an entity and a check. We set it to the user-specified
+ // value (optional), else we fallback and use the grafana rule anme and ruleID.
+ entity := tmpl(sn.Entity)
+ if entity == "" {
+ entity = "default"
+ }
+
+ check := tmpl(sn.Check)
+ if check == "" {
+ check = "default"
+ }
+
+ alerts := types.Alerts(as...)
+ status := 0
+ if alerts.Status() == model.AlertFiring {
+ // TODO figure out about NoData old state (we used to send status 1 in that case)
+ status = 2
+ }
+
+ namespace := tmpl(sn.Namespace)
+ if namespace == "" {
+ namespace = "default"
+ }
+
+ var handlers []string
+ if sn.Handler != "" {
+ handlers = []string{tmpl(sn.Handler)}
+ }
+
+ labels := make(map[string]string)
+
+ _ = withStoredImages(ctx, sn.log, sn.images,
+ func(_ int, image ngmodels.Image) error {
+ // If there is an image for this alert and the image has been uploaded
+ // to a public URL then add it to the request. We cannot add more than
+ // one image per request.
+ if image.URL != "" {
+ labels["imageURL"] = image.URL
+ return ErrImagesDone
+ }
+ return nil
+ }, as...)
+
+ ruleURL := joinUrlPath(sn.tmpl.ExternalURL.String(), "/alerting/list", sn.log)
+ labels["ruleURL"] = ruleURL
+
+ bodyMsgType := map[string]interface{}{
+ "entity": map[string]interface{}{
+ "metadata": map[string]interface{}{
+ "name": entity,
+ "namespace": namespace,
+ },
+ },
+ "check": map[string]interface{}{
+ "metadata": map[string]interface{}{
+ "name": check,
+ "labels": labels,
+ },
+ "output": tmpl(sn.Message),
+ "issued": timeNow().Unix(),
+ "interval": 86400,
+ "status": status,
+ "handlers": handlers,
+ },
+ "ruleUrl": ruleURL,
+ }
+
+ if tmplErr != nil {
+ sn.log.Warn("failed to template sensugo message", "err", tmplErr.Error())
+ }
+
+ body, err := json.Marshal(bodyMsgType)
+ if err != nil {
+ return false, err
+ }
+
+ cmd := &models.SendWebhookSync{
+ Url: fmt.Sprintf("%s/api/core/v2/namespaces/%s/events", strings.TrimSuffix(sn.URL, "/"), namespace),
+ Body: string(body),
+ HttpMethod: "POST",
+ HttpHeader: map[string]string{
+ "Content-Type": "application/json",
+ "Authorization": fmt.Sprintf("Key %s", sn.APIKey),
+ },
+ }
+ if err := sn.ns.SendWebhookSync(ctx, cmd); err != nil {
+ sn.log.Error("failed to send Sensu Go event", "err", err, "sensugo", sn.Name)
+ return false, err
+ }
+
+ return true, nil
+}
+
+func (sn *SensuGoNotifier) SendResolved() bool {
+ return !sn.GetDisableResolveMessage()
+}
diff --git a/alerting/channels/sensugo_test.go b/alerting/channels/sensugo_test.go
new file mode 100644
index 00000000..ea0a8a9a
--- /dev/null
+++ b/alerting/channels/sensugo_test.go
@@ -0,0 +1,181 @@
+package channels
+
+import (
+ "context"
+ "encoding/json"
+ "net/url"
+ "testing"
+ "time"
+
+ "github.com/grafana/grafana/pkg/components/simplejson"
+ "github.com/grafana/grafana/pkg/services/secrets/fakes"
+ secretsManager "github.com/grafana/grafana/pkg/services/secrets/manager"
+
+ "github.com/prometheus/alertmanager/notify"
+ "github.com/prometheus/alertmanager/types"
+ "github.com/prometheus/common/model"
+ "github.com/stretchr/testify/require"
+)
+
+func TestSensuGoNotifier(t *testing.T) {
+ constNow := time.Now()
+ defer mockTimeNow(constNow)()
+
+ tmpl := templateForTests(t)
+
+ externalURL, err := url.Parse("http://localhost")
+ require.NoError(t, err)
+ tmpl.ExternalURL = externalURL
+
+ images := newFakeImageStore(2)
+
+ cases := []struct {
+ name string
+ settings string
+ alerts []*types.Alert
+ expMsg map[string]interface{}
+ expInitError string
+ expMsgError error
+ }{
+ {
+ name: "Default config with one alert",
+ settings: `{"url": "http://sensu-api.local:8080", "apikey": ""}`,
+ alerts: []*types.Alert{
+ {
+ Alert: model.Alert{
+ Labels: model.LabelSet{"__alert_rule_uid__": "rule uid", "alertname": "alert1", "lbl1": "val1"},
+ Annotations: model.LabelSet{"ann1": "annv1", "__dashboardUid__": "abcd", "__panelId__": "efgh", "__alertImageToken__": "test-image-1"},
+ },
+ },
+ },
+ expMsg: map[string]interface{}{
+ "entity": map[string]interface{}{
+ "metadata": map[string]interface{}{
+ "name": "default",
+ "namespace": "default",
+ },
+ },
+ "check": map[string]interface{}{
+ "metadata": map[string]interface{}{
+ "name": "default",
+ "labels": map[string]string{
+ "imageURL": "https://www.example.com/test-image-1.jpg",
+ "ruleURL": "http://localhost/alerting/list",
+ },
+ },
+ "output": "**Firing**\n\nValue: [no value]\nLabels:\n - alertname = alert1\n - lbl1 = val1\nAnnotations:\n - ann1 = annv1\nSilence: http://localhost/alerting/silence/new?alertmanager=grafana&matcher=alertname%3Dalert1&matcher=lbl1%3Dval1\nDashboard: http://localhost/d/abcd\nPanel: http://localhost/d/abcd?viewPanel=efgh\n",
+ "issued": timeNow().Unix(),
+ "interval": 86400,
+ "status": 2,
+ "handlers": nil,
+ },
+ "ruleUrl": "http://localhost/alerting/list",
+ },
+ expMsgError: nil,
+ }, {
+ name: "Custom config with multiple alerts",
+ settings: `{
+ "url": "http://sensu-api.local:8080",
+ "entity": "grafana_instance_01",
+ "check": "grafana_rule_0",
+ "namespace": "namespace",
+ "handler": "myhandler",
+ "apikey": "",
+ "message": "{{ len .Alerts.Firing }} alerts are firing, {{ len .Alerts.Resolved }} are resolved"
+ }`,
+ alerts: []*types.Alert{
+ {
+ Alert: model.Alert{
+ Labels: model.LabelSet{"__alert_rule_uid__": "rule uid", "alertname": "alert1", "lbl1": "val1"},
+ Annotations: model.LabelSet{"ann1": "annv1", "__alertImageToken__": "test-image-1"},
+ },
+ }, {
+ Alert: model.Alert{
+ Labels: model.LabelSet{"alertname": "alert1", "lbl1": "val2"},
+ Annotations: model.LabelSet{"ann1": "annv2", "__alertImageToken__": "test-image-2"},
+ },
+ },
+ },
+ expMsg: map[string]interface{}{
+ "entity": map[string]interface{}{
+ "metadata": map[string]interface{}{
+ "name": "grafana_instance_01",
+ "namespace": "namespace",
+ },
+ },
+ "check": map[string]interface{}{
+ "metadata": map[string]interface{}{
+ "name": "grafana_rule_0",
+ "labels": map[string]string{
+ "imageURL": "https://www.example.com/test-image-1.jpg",
+ "ruleURL": "http://localhost/alerting/list",
+ },
+ },
+ "output": "2 alerts are firing, 0 are resolved",
+ "issued": timeNow().Unix(),
+ "interval": 86400,
+ "status": 2,
+ "handlers": []string{"myhandler"},
+ },
+ "ruleUrl": "http://localhost/alerting/list",
+ },
+ expMsgError: nil,
+ }, {
+ name: "Error in initing: missing URL",
+ settings: `{
+ "apikey": ""
+ }`,
+ expInitError: `could not find URL property in settings`,
+ }, {
+ name: "Error in initing: missing API key",
+ settings: `{
+ "url": "http://sensu-api.local:8080"
+ }`,
+ expInitError: `could not find the API key property in settings`,
+ },
+ }
+
+ for _, c := range cases {
+ t.Run(c.name, func(t *testing.T) {
+ settingsJSON, err := simplejson.NewJson([]byte(c.settings))
+ require.NoError(t, err)
+ secureSettings := make(map[string][]byte)
+
+ m := &NotificationChannelConfig{
+ Name: "Sensu Go",
+ Type: "sensugo",
+ Settings: settingsJSON,
+ SecureSettings: secureSettings,
+ }
+
+ webhookSender := mockNotificationService()
+ secretsService := secretsManager.SetupTestService(t, fakes.NewFakeSecretsStore())
+ decryptFn := secretsService.GetDecryptedValue
+ cfg, err := NewSensuGoConfig(m, decryptFn)
+ if c.expInitError != "" {
+ require.Error(t, err)
+ require.Equal(t, c.expInitError, err.Error())
+ return
+ }
+ require.NoError(t, err)
+
+ ctx := notify.WithGroupKey(context.Background(), "alertname")
+ ctx = notify.WithGroupLabels(ctx, model.LabelSet{"alertname": ""})
+ sn := NewSensuGoNotifier(cfg, images, webhookSender, tmpl)
+ ok, err := sn.Notify(ctx, c.alerts...)
+ if c.expMsgError != nil {
+ require.False(t, ok)
+ require.Error(t, err)
+ require.Equal(t, c.expMsgError.Error(), err.Error())
+ return
+ }
+ require.NoError(t, err)
+ require.True(t, ok)
+
+ expBody, err := json.Marshal(c.expMsg)
+ require.NoError(t, err)
+
+ require.JSONEq(t, string(expBody), webhookSender.Webhook.Body)
+ })
+ }
+}
diff --git a/alerting/channels/slack.go b/alerting/channels/slack.go
new file mode 100644
index 00000000..741fde20
--- /dev/null
+++ b/alerting/channels/slack.go
@@ -0,0 +1,371 @@
+package channels
+
+import (
+ "bytes"
+ "context"
+ "crypto/tls"
+ "encoding/json"
+ "errors"
+ "fmt"
+ "io"
+ "net"
+ "net/http"
+ "net/url"
+ "strings"
+ "time"
+
+ "github.com/prometheus/alertmanager/config"
+ "github.com/prometheus/alertmanager/template"
+ "github.com/prometheus/alertmanager/types"
+
+ "github.com/grafana/grafana/pkg/infra/log"
+ "github.com/grafana/grafana/pkg/models"
+ ngmodels "github.com/grafana/grafana/pkg/services/ngalert/models"
+ "github.com/grafana/grafana/pkg/services/notifications"
+ "github.com/grafana/grafana/pkg/setting"
+)
+
+var SlackAPIEndpoint = "https://slack.com/api/chat.postMessage"
+
+// SlackNotifier is responsible for sending
+// alert notification to Slack.
+type SlackNotifier struct {
+ *Base
+ log log.Logger
+ tmpl *template.Template
+ images ImageStore
+ webhookSender notifications.WebhookSender
+
+ URL *url.URL
+ Username string
+ IconEmoji string
+ IconURL string
+ Recipient string
+ Text string
+ Title string
+ MentionUsers []string
+ MentionGroups []string
+ MentionChannel string
+ Token string
+}
+
+type SlackConfig struct {
+ *NotificationChannelConfig
+ URL *url.URL
+ Username string
+ IconEmoji string
+ IconURL string
+ Recipient string
+ Text string
+ Title string
+ MentionUsers []string
+ MentionGroups []string
+ MentionChannel string
+ Token string
+}
+
+func SlackFactory(fc FactoryConfig) (NotificationChannel, error) {
+ cfg, err := NewSlackConfig(fc)
+ if err != nil {
+ return nil, receiverInitError{
+ Reason: err.Error(),
+ Cfg: *fc.Config,
+ }
+ }
+ return NewSlackNotifier(cfg, fc.ImageStore, fc.NotificationService, fc.Template), nil
+}
+
+func NewSlackConfig(factoryConfig FactoryConfig) (*SlackConfig, error) {
+ channelConfig := factoryConfig.Config
+ decryptFunc := factoryConfig.DecryptFunc
+ endpointURL := channelConfig.Settings.Get("endpointUrl").MustString(SlackAPIEndpoint)
+ slackURL := decryptFunc(context.Background(), channelConfig.SecureSettings, "url", channelConfig.Settings.Get("url").MustString())
+ if slackURL == "" {
+ slackURL = endpointURL
+ }
+ apiURL, err := url.Parse(slackURL)
+ if err != nil {
+ return nil, fmt.Errorf("invalid URL %q", slackURL)
+ }
+ recipient := strings.TrimSpace(channelConfig.Settings.Get("recipient").MustString())
+ if recipient == "" && apiURL.String() == SlackAPIEndpoint {
+ return nil, errors.New("recipient must be specified when using the Slack chat API")
+ }
+ mentionChannel := channelConfig.Settings.Get("mentionChannel").MustString()
+ if mentionChannel != "" && mentionChannel != "here" && mentionChannel != "channel" {
+ return nil, fmt.Errorf("invalid value for mentionChannel: %q", mentionChannel)
+ }
+ token := decryptFunc(context.Background(), channelConfig.SecureSettings, "token", channelConfig.Settings.Get("token").MustString())
+ if token == "" && apiURL.String() == SlackAPIEndpoint {
+ return nil, errors.New("token must be specified when using the Slack chat API")
+ }
+ mentionUsersStr := channelConfig.Settings.Get("mentionUsers").MustString()
+ mentionUsers := []string{}
+ for _, u := range strings.Split(mentionUsersStr, ",") {
+ u = strings.TrimSpace(u)
+ if u != "" {
+ mentionUsers = append(mentionUsers, u)
+ }
+ }
+ mentionGroupsStr := channelConfig.Settings.Get("mentionGroups").MustString()
+ mentionGroups := []string{}
+ for _, g := range strings.Split(mentionGroupsStr, ",") {
+ g = strings.TrimSpace(g)
+ if g != "" {
+ mentionGroups = append(mentionGroups, g)
+ }
+ }
+ return &SlackConfig{
+ NotificationChannelConfig: channelConfig,
+ Recipient: strings.TrimSpace(channelConfig.Settings.Get("recipient").MustString()),
+ MentionChannel: channelConfig.Settings.Get("mentionChannel").MustString(),
+ MentionUsers: mentionUsers,
+ MentionGroups: mentionGroups,
+ URL: apiURL,
+ Username: channelConfig.Settings.Get("username").MustString("Grafana"),
+ IconEmoji: channelConfig.Settings.Get("icon_emoji").MustString(),
+ IconURL: channelConfig.Settings.Get("icon_url").MustString(),
+ Token: token,
+ Text: channelConfig.Settings.Get("text").MustString(`{{ template "default.message" . }}`),
+ Title: channelConfig.Settings.Get("title").MustString(DefaultMessageTitleEmbed),
+ }, nil
+}
+
+// NewSlackNotifier is the constructor for the Slack notifier
+func NewSlackNotifier(config *SlackConfig,
+ images ImageStore,
+ webhookSender notifications.WebhookSender,
+ t *template.Template,
+) *SlackNotifier {
+ return &SlackNotifier{
+ Base: NewBase(&models.AlertNotification{
+ Uid: config.UID,
+ Name: config.Name,
+ Type: config.Type,
+ DisableResolveMessage: config.DisableResolveMessage,
+ Settings: config.Settings,
+ }),
+ URL: config.URL,
+ Recipient: config.Recipient,
+ MentionUsers: config.MentionUsers,
+ MentionGroups: config.MentionGroups,
+ MentionChannel: config.MentionChannel,
+ Username: config.Username,
+ IconEmoji: config.IconEmoji,
+ IconURL: config.IconURL,
+ Token: config.Token,
+ Text: config.Text,
+ Title: config.Title,
+ images: images,
+ webhookSender: webhookSender,
+ log: log.New("alerting.notifier.slack"),
+ tmpl: t,
+ }
+}
+
+// slackMessage is the slackMessage for sending a slack notification.
+type slackMessage struct {
+ Channel string `json:"channel,omitempty"`
+ Username string `json:"username,omitempty"`
+ IconEmoji string `json:"icon_emoji,omitempty"`
+ IconURL string `json:"icon_url,omitempty"`
+ Attachments []attachment `json:"attachments"`
+ Blocks []map[string]interface{} `json:"blocks"`
+}
+
+// attachment is used to display a richly-formatted message block.
+type attachment struct {
+ Title string `json:"title,omitempty"`
+ TitleLink string `json:"title_link,omitempty"`
+ Text string `json:"text"`
+ ImageURL string `json:"image_url,omitempty"`
+ Fallback string `json:"fallback"`
+ Fields []config.SlackField `json:"fields,omitempty"`
+ Footer string `json:"footer"`
+ FooterIcon string `json:"footer_icon"`
+ Color string `json:"color,omitempty"`
+ Ts int64 `json:"ts,omitempty"`
+}
+
+// Notify sends an alert notification to Slack.
+func (sn *SlackNotifier) Notify(ctx context.Context, alerts ...*types.Alert) (bool, error) {
+ sn.log.Debug("building slack message", "alerts", len(alerts))
+ msg, err := sn.buildSlackMessage(ctx, alerts)
+ if err != nil {
+ return false, fmt.Errorf("build slack message: %w", err)
+ }
+
+ b, err := json.Marshal(msg)
+ if err != nil {
+ return false, fmt.Errorf("marshal json: %w", err)
+ }
+
+ sn.log.Debug("sending Slack API request", "url", sn.URL.String(), "data", string(b))
+ request, err := http.NewRequestWithContext(ctx, http.MethodPost, sn.URL.String(), bytes.NewReader(b))
+ if err != nil {
+ return false, fmt.Errorf("failed to create HTTP request: %w", err)
+ }
+
+ request.Header.Set("Content-Type", "application/json")
+ request.Header.Set("User-Agent", "Grafana")
+ if sn.Token == "" {
+ if sn.URL.String() == SlackAPIEndpoint {
+ panic("Token should be set when using the Slack chat API")
+ }
+ } else {
+ sn.log.Debug("adding authorization header to HTTP request")
+ request.Header.Set("Authorization", fmt.Sprintf("Bearer %s", sn.Token))
+ }
+
+ if err := sendSlackRequest(request, sn.log); err != nil {
+ return false, err
+ }
+
+ return true, nil
+}
+
+// sendSlackRequest sends a request to the Slack API.
+// Stubbable by tests.
+var sendSlackRequest = func(request *http.Request, logger log.Logger) (retErr error) {
+ defer func() {
+ if retErr != nil {
+ logger.Warn("failed to send slack request", "err", retErr)
+ }
+ }()
+
+ netTransport := &http.Transport{
+ TLSClientConfig: &tls.Config{
+ Renegotiation: tls.RenegotiateFreelyAsClient,
+ },
+ Proxy: http.ProxyFromEnvironment,
+ DialContext: (&net.Dialer{
+ Timeout: 30 * time.Second,
+ }).DialContext,
+ TLSHandshakeTimeout: 5 * time.Second,
+ }
+ netClient := &http.Client{
+ Timeout: time.Second * 30,
+ Transport: netTransport,
+ }
+ resp, err := netClient.Do(request)
+ if err != nil {
+ return err
+ }
+ defer func() {
+ if err := resp.Body.Close(); err != nil {
+ logger.Warn("failed to close response body", "err", err)
+ }
+ }()
+
+ body, err := io.ReadAll(resp.Body)
+ if err != nil {
+ return fmt.Errorf("failed to read response body: %w", err)
+ }
+
+ if resp.StatusCode < 200 || resp.StatusCode >= 300 {
+ logger.Error("Slack API request failed", "url", request.URL.String(), "statusCode", resp.Status, "body", string(body))
+ return fmt.Errorf("request to Slack API failed with status code %d", resp.StatusCode)
+ }
+
+ // Slack responds to some requests with a JSON document, that might contain an error.
+ rslt := struct {
+ Ok bool `json:"ok"`
+ Err string `json:"error"`
+ }{}
+
+ // Marshaling can fail if Slack's response body is plain text (e.g. "ok").
+ if err := json.Unmarshal(body, &rslt); err != nil && json.Valid(body) {
+ logger.Error("Failed to unmarshal Slack API response", "url", request.URL.String(), "statusCode", resp.Status,
+ "body", string(body))
+ return fmt.Errorf("failed to unmarshal Slack API response: %s", err)
+ }
+
+ if !rslt.Ok && rslt.Err != "" {
+ logger.Error("Sending Slack API request failed", "url", request.URL.String(), "statusCode", resp.Status,
+ "err", rslt.Err)
+ return fmt.Errorf("failed to make Slack API request: %s", rslt.Err)
+ }
+
+ logger.Debug("sending Slack API request succeeded", "url", request.URL.String(), "statusCode", resp.Status)
+ return nil
+}
+
+func (sn *SlackNotifier) buildSlackMessage(ctx context.Context, alrts []*types.Alert) (*slackMessage, error) {
+ alerts := types.Alerts(alrts...)
+ var tmplErr error
+ tmpl, _ := TmplText(ctx, sn.tmpl, alrts, sn.log, &tmplErr)
+
+ ruleURL := joinUrlPath(sn.tmpl.ExternalURL.String(), "/alerting/list", sn.log)
+
+ req := &slackMessage{
+ Channel: tmpl(sn.Recipient),
+ Username: tmpl(sn.Username),
+ IconEmoji: tmpl(sn.IconEmoji),
+ IconURL: tmpl(sn.IconURL),
+ // TODO: We should use the Block Kit API instead:
+ // https://api.slack.com/messaging/composing/layouts#when-to-use-attachments
+ Attachments: []attachment{
+ {
+ Color: getAlertStatusColor(alerts.Status()),
+ Title: tmpl(sn.Title),
+ Fallback: tmpl(sn.Title),
+ Footer: "Grafana v" + setting.BuildVersion,
+ FooterIcon: FooterIconURL,
+ Ts: time.Now().Unix(),
+ TitleLink: ruleURL,
+ Text: tmpl(sn.Text),
+ Fields: nil, // TODO. Should be a config.
+ },
+ },
+ }
+
+ _ = withStoredImages(ctx, sn.log, sn.images, func(index int, image ngmodels.Image) error {
+ req.Attachments[0].ImageURL = image.URL
+ return ErrImagesDone
+ }, alrts...)
+
+ if tmplErr != nil {
+ sn.log.Warn("failed to template Slack message", "err", tmplErr.Error())
+ }
+
+ mentionsBuilder := strings.Builder{}
+ appendSpace := func() {
+ if mentionsBuilder.Len() > 0 {
+ mentionsBuilder.WriteString(" ")
+ }
+ }
+ mentionChannel := strings.TrimSpace(sn.MentionChannel)
+ if mentionChannel != "" {
+ mentionsBuilder.WriteString(fmt.Sprintf("", mentionChannel, mentionChannel))
+ }
+ if len(sn.MentionGroups) > 0 {
+ appendSpace()
+ for _, g := range sn.MentionGroups {
+ mentionsBuilder.WriteString(fmt.Sprintf("", tmpl(g)))
+ }
+ }
+ if len(sn.MentionUsers) > 0 {
+ appendSpace()
+ for _, u := range sn.MentionUsers {
+ mentionsBuilder.WriteString(fmt.Sprintf("<@%s>", tmpl(u)))
+ }
+ }
+
+ if mentionsBuilder.Len() > 0 {
+ req.Blocks = []map[string]interface{}{
+ {
+ "type": "section",
+ "text": map[string]interface{}{
+ "type": "mrkdwn",
+ "text": mentionsBuilder.String(),
+ },
+ },
+ }
+ }
+
+ return req, nil
+}
+
+func (sn *SlackNotifier) SendResolved() bool {
+ return !sn.GetDisableResolveMessage()
+}
diff --git a/alerting/channels/slack_test.go b/alerting/channels/slack_test.go
new file mode 100644
index 00000000..b95467f7
--- /dev/null
+++ b/alerting/channels/slack_test.go
@@ -0,0 +1,421 @@
+package channels
+
+import (
+ "context"
+ "encoding/json"
+ "io"
+ "net/http"
+ "net/http/httptest"
+ "net/url"
+ "testing"
+
+ "github.com/grafana/grafana/pkg/components/simplejson"
+ "github.com/grafana/grafana/pkg/infra/log"
+ "github.com/grafana/grafana/pkg/services/ngalert/models"
+ "github.com/grafana/grafana/pkg/services/secrets/fakes"
+ secretsManager "github.com/grafana/grafana/pkg/services/secrets/manager"
+ "github.com/grafana/grafana/pkg/setting"
+
+ "github.com/prometheus/alertmanager/notify"
+ "github.com/prometheus/alertmanager/types"
+ "github.com/prometheus/common/model"
+ "github.com/stretchr/testify/require"
+)
+
+func TestSlackNotifier(t *testing.T) {
+ tmpl := templateForTests(t)
+
+ fakeImageStore := &fakeImageStore{
+ Images: []*models.Image{
+ {
+ Token: "test-with-url",
+ URL: "https://www.example.com/image.jpg",
+ },
+ },
+ }
+
+ externalURL, err := url.Parse("http://localhost")
+ require.NoError(t, err)
+ tmpl.ExternalURL = externalURL
+
+ cases := []struct {
+ name string
+ settings string
+ alerts []*types.Alert
+ expMsg *slackMessage
+ expInitError string
+ expMsgError error
+ expWebhookURL string
+ }{
+ {
+ name: "Correct config with one alert",
+ settings: `{
+ "token": "1234",
+ "recipient": "#testchannel",
+ "icon_emoji": ":emoji:"
+ }`,
+ alerts: []*types.Alert{
+ {
+ Alert: model.Alert{
+ Labels: model.LabelSet{"alertname": "alert1", "lbl1": "val1"},
+ Annotations: model.LabelSet{"ann1": "annv1", "__dashboardUid__": "abcd", "__panelId__": "efgh"},
+ },
+ },
+ },
+ expMsg: &slackMessage{
+ Channel: "#testchannel",
+ Username: "Grafana",
+ IconEmoji: ":emoji:",
+ Attachments: []attachment{
+ {
+ Title: "[FIRING:1] (val1)",
+ TitleLink: "http://localhost/alerting/list",
+ Text: "**Firing**\n\nValue: [no value]\nLabels:\n - alertname = alert1\n - lbl1 = val1\nAnnotations:\n - ann1 = annv1\nSilence: http://localhost/alerting/silence/new?alertmanager=grafana&matcher=alertname%3Dalert1&matcher=lbl1%3Dval1\nDashboard: http://localhost/d/abcd\nPanel: http://localhost/d/abcd?viewPanel=efgh\n",
+ Fallback: "[FIRING:1] (val1)",
+ Fields: nil,
+ Footer: "Grafana v" + setting.BuildVersion,
+ FooterIcon: "https://grafana.com/assets/img/fav32.png",
+ Color: "#D63232",
+ Ts: 0,
+ },
+ },
+ },
+ expMsgError: nil,
+ },
+ {
+ name: "Correct config with webhook",
+ settings: `{
+ "url": "https://webhook.com",
+ "recipient": "#testchannel",
+ "icon_emoji": ":emoji:"
+ }`,
+ alerts: []*types.Alert{
+ {
+ Alert: model.Alert{
+ Labels: model.LabelSet{"alertname": "alert1", "lbl1": "val1"},
+ Annotations: model.LabelSet{"ann1": "annv1"},
+ },
+ },
+ },
+ expMsg: &slackMessage{
+ Channel: "#testchannel",
+ Username: "Grafana",
+ IconEmoji: ":emoji:",
+ Attachments: []attachment{
+ {
+ Title: "[FIRING:1] (val1)",
+ TitleLink: "http://localhost/alerting/list",
+ Text: "**Firing**\n\nValue: [no value]\nLabels:\n - alertname = alert1\n - lbl1 = val1\nAnnotations:\n - ann1 = annv1\nSilence: http://localhost/alerting/silence/new?alertmanager=grafana&matcher=alertname%3Dalert1&matcher=lbl1%3Dval1\n",
+ Fallback: "[FIRING:1] (val1)",
+ Fields: nil,
+ Footer: "Grafana v" + setting.BuildVersion,
+ FooterIcon: "https://grafana.com/assets/img/fav32.png",
+ Color: "#D63232",
+ Ts: 0,
+ },
+ },
+ },
+ expMsgError: nil,
+ },
+ {
+ name: "Image URL in alert appears in slack message",
+ settings: `{
+ "token": "1234",
+ "recipient": "#testchannel",
+ "icon_emoji": ":emoji:"
+ }`,
+ alerts: []*types.Alert{
+ {
+ Alert: model.Alert{
+ Labels: model.LabelSet{"alertname": "alert1", "lbl1": "val1"},
+ Annotations: model.LabelSet{"ann1": "annv1", "__dashboardUid__": "abcd", "__panelId__": "efgh", "__alertImageToken__": "test-with-url"},
+ },
+ },
+ },
+ expMsg: &slackMessage{
+ Channel: "#testchannel",
+ Username: "Grafana",
+ IconEmoji: ":emoji:",
+ Attachments: []attachment{
+ {
+ Title: "[FIRING:1] (val1)",
+ TitleLink: "http://localhost/alerting/list",
+ Text: "**Firing**\n\nValue: [no value]\nLabels:\n - alertname = alert1\n - lbl1 = val1\nAnnotations:\n - ann1 = annv1\nSilence: http://localhost/alerting/silence/new?alertmanager=grafana&matcher=alertname%3Dalert1&matcher=lbl1%3Dval1\nDashboard: http://localhost/d/abcd\nPanel: http://localhost/d/abcd?viewPanel=efgh\n",
+ Fallback: "[FIRING:1] (val1)",
+ Fields: nil,
+ Footer: "Grafana v" + setting.BuildVersion,
+ FooterIcon: "https://grafana.com/assets/img/fav32.png",
+ Color: "#D63232",
+ Ts: 0,
+ ImageURL: "https://www.example.com/image.jpg",
+ },
+ },
+ },
+ expMsgError: nil,
+ },
+ {
+ name: "Correct config with multiple alerts and template",
+ settings: `{
+ "token": "1234",
+ "recipient": "#testchannel",
+ "icon_emoji": ":emoji:",
+ "title": "{{ .Alerts.Firing | len }} firing, {{ .Alerts.Resolved | len }} resolved"
+ }`,
+ alerts: []*types.Alert{
+ {
+ Alert: model.Alert{
+ Labels: model.LabelSet{"alertname": "alert1", "lbl1": "val1"},
+ Annotations: model.LabelSet{"ann1": "annv1"},
+ },
+ },
+ {
+ Alert: model.Alert{
+ Labels: model.LabelSet{"alertname": "alert1", "lbl1": "val2"},
+ Annotations: model.LabelSet{"ann1": "annv2"},
+ },
+ },
+ },
+ expMsg: &slackMessage{
+ Channel: "#testchannel",
+ Username: "Grafana",
+ IconEmoji: ":emoji:",
+ Attachments: []attachment{
+ {
+ Title: "2 firing, 0 resolved",
+ TitleLink: "http://localhost/alerting/list",
+ Text: "**Firing**\n\nValue: [no value]\nLabels:\n - alertname = alert1\n - lbl1 = val1\nAnnotations:\n - ann1 = annv1\nSilence: http://localhost/alerting/silence/new?alertmanager=grafana&matcher=alertname%3Dalert1&matcher=lbl1%3Dval1\n\nValue: [no value]\nLabels:\n - alertname = alert1\n - lbl1 = val2\nAnnotations:\n - ann1 = annv2\nSilence: http://localhost/alerting/silence/new?alertmanager=grafana&matcher=alertname%3Dalert1&matcher=lbl1%3Dval2\n",
+ Fallback: "2 firing, 0 resolved",
+ Fields: nil,
+ Footer: "Grafana v" + setting.BuildVersion,
+ FooterIcon: "https://grafana.com/assets/img/fav32.png",
+ Color: "#D63232",
+ Ts: 0,
+ },
+ },
+ },
+ expMsgError: nil,
+ }, {
+ name: "Missing token",
+ settings: `{
+ "recipient": "#testchannel"
+ }`,
+ expInitError: `token must be specified when using the Slack chat API`,
+ }, {
+ name: "Missing recipient",
+ settings: `{
+ "token": "1234"
+ }`,
+ expInitError: `recipient must be specified when using the Slack chat API`,
+ },
+ {
+ name: "Custom endpoint url",
+ settings: `{
+ "token": "1234",
+ "recipient": "#testchannel",
+ "endpointUrl": "https://slack-custom.com/api/",
+ "icon_emoji": ":emoji:"
+ }`,
+ alerts: []*types.Alert{
+ {
+ Alert: model.Alert{
+ Labels: model.LabelSet{"alertname": "alert1", "lbl1": "val1"},
+ Annotations: model.LabelSet{"ann1": "annv1"},
+ },
+ },
+ },
+ expMsg: &slackMessage{
+ Channel: "#testchannel",
+ Username: "Grafana",
+ IconEmoji: ":emoji:",
+ Attachments: []attachment{
+ {
+ Title: "[FIRING:1] (val1)",
+ TitleLink: "http://localhost/alerting/list",
+ Text: "**Firing**\n\nValue: [no value]\nLabels:\n - alertname = alert1\n - lbl1 = val1\nAnnotations:\n - ann1 = annv1\nSilence: http://localhost/alerting/silence/new?alertmanager=grafana&matcher=alertname%3Dalert1&matcher=lbl1%3Dval1\n",
+ Fallback: "[FIRING:1] (val1)",
+ Fields: nil,
+ Footer: "Grafana v" + setting.BuildVersion,
+ FooterIcon: "https://grafana.com/assets/img/fav32.png",
+ Color: "#D63232",
+ Ts: 0,
+ },
+ },
+ },
+ expMsgError: nil,
+ },
+ }
+
+ for _, c := range cases {
+ t.Run(c.name, func(t *testing.T) {
+ settingsJSON, err := simplejson.NewJson([]byte(c.settings))
+ require.NoError(t, err)
+ secureSettings := make(map[string][]byte)
+
+ secretsService := secretsManager.SetupTestService(t, fakes.NewFakeSecretsStore())
+ decryptFn := secretsService.GetDecryptedValue
+ notificationService := mockNotificationService()
+
+ fc := FactoryConfig{
+ Config: &NotificationChannelConfig{
+ Name: "slack_testing",
+ Type: "slack",
+ Settings: settingsJSON,
+ SecureSettings: secureSettings,
+ },
+ ImageStore: fakeImageStore,
+ // TODO: allow changing the associated values for different tests.
+ NotificationService: notificationService,
+ DecryptFunc: decryptFn,
+ }
+
+ cfg, err := NewSlackConfig(fc)
+ if c.expInitError != "" {
+ require.Error(t, err)
+ require.Equal(t, c.expInitError, err.Error())
+ return
+ }
+ require.NoError(t, err)
+
+ body := ""
+ origSendSlackRequest := sendSlackRequest
+ t.Cleanup(func() {
+ sendSlackRequest = origSendSlackRequest
+ })
+ sendSlackRequest = func(request *http.Request, log log.Logger) error {
+ t.Helper()
+ defer func() {
+ _ = request.Body.Close()
+ }()
+
+ url := settingsJSON.Get("url").MustString()
+ if len(url) == 0 {
+ endpointUrl := settingsJSON.Get("endpointUrl").MustString(SlackAPIEndpoint)
+ require.Equal(t, endpointUrl, request.URL.String())
+ }
+
+ b, err := io.ReadAll(request.Body)
+ require.NoError(t, err)
+ body = string(b)
+ return nil
+ }
+
+ ctx := notify.WithGroupKey(context.Background(), "alertname")
+ ctx = notify.WithGroupLabels(ctx, model.LabelSet{"alertname": ""})
+ pn := NewSlackNotifier(cfg, fc.ImageStore, fc.NotificationService, tmpl)
+ ok, err := pn.Notify(ctx, c.alerts...)
+ if c.expMsgError != nil {
+ require.Error(t, err)
+ require.False(t, ok)
+ require.Equal(t, c.expMsgError.Error(), err.Error())
+ return
+ }
+ require.True(t, ok)
+ require.NoError(t, err)
+
+ // Getting Ts from actual since that can't be predicted.
+ var obj slackMessage
+ require.NoError(t, json.Unmarshal([]byte(body), &obj))
+ c.expMsg.Attachments[0].Ts = obj.Attachments[0].Ts
+
+ expBody, err := json.Marshal(c.expMsg)
+ require.NoError(t, err)
+
+ require.JSONEq(t, string(expBody), body)
+
+ // If we should have sent to the webhook, the mock notification service
+ // will have a record of it.
+ require.Equal(t, c.expWebhookURL, notificationService.Webhook.Url)
+ })
+ }
+}
+
+func TestSendSlackRequest(t *testing.T) {
+ tests := []struct {
+ name string
+ slackResponse string
+ statusCode int
+ expectError bool
+ }{
+ {
+ name: "Example error",
+ slackResponse: `{
+ "ok": false,
+ "error": "too_many_attachments"
+ }`,
+ statusCode: http.StatusBadRequest,
+ expectError: true,
+ },
+ {
+ name: "Non 200 status code, no response body",
+ statusCode: http.StatusMovedPermanently,
+ expectError: true,
+ },
+ {
+ name: "Success case, normal response body",
+ slackResponse: `{
+ "ok": true,
+ "channel": "C1H9RESGL",
+ "ts": "1503435956.000247",
+ "message": {
+ "text": "Here's a message for you",
+ "username": "ecto1",
+ "bot_id": "B19LU7CSY",
+ "attachments": [
+ {
+ "text": "This is an attachment",
+ "id": 1,
+ "fallback": "This is an attachment's fallback"
+ }
+ ],
+ "type": "message",
+ "subtype": "bot_message",
+ "ts": "1503435956.000247"
+ }
+ }`,
+ statusCode: http.StatusOK,
+ expectError: false,
+ },
+ {
+ name: "No response body",
+ statusCode: http.StatusOK,
+ },
+ {
+ name: "Success case, unexpected response body",
+ statusCode: http.StatusOK,
+ slackResponse: `{"test": true}`,
+ expectError: false,
+ },
+ {
+ name: "Success case, ok: true",
+ statusCode: http.StatusOK,
+ slackResponse: `{"ok": true}`,
+ expectError: false,
+ },
+ {
+ name: "200 status code, error in body",
+ statusCode: http.StatusOK,
+ slackResponse: `{"ok": false, "error": "test error"}`,
+ expectError: true,
+ },
+ }
+
+ for _, test := range tests {
+ t.Run(test.name, func(tt *testing.T) {
+ server := httptest.NewServer(http.HandlerFunc(func(w http.ResponseWriter, r *http.Request) {
+ w.WriteHeader(test.statusCode)
+ _, err := w.Write([]byte(test.slackResponse))
+ require.NoError(tt, err)
+ }))
+ defer server.Close()
+ req, err := http.NewRequest(http.MethodGet, server.URL, nil)
+ require.NoError(tt, err)
+
+ err = sendSlackRequest(req, log.New("test"))
+ if !test.expectError {
+ require.NoError(tt, err)
+ } else {
+ require.Error(tt, err)
+ }
+ })
+ }
+}
diff --git a/alerting/channels/teams.go b/alerting/channels/teams.go
new file mode 100644
index 00000000..fd45764a
--- /dev/null
+++ b/alerting/channels/teams.go
@@ -0,0 +1,169 @@
+package channels
+
+import (
+ "context"
+ "encoding/json"
+
+ "github.com/pkg/errors"
+ "github.com/prometheus/alertmanager/template"
+ "github.com/prometheus/alertmanager/types"
+
+ "github.com/grafana/grafana/pkg/infra/log"
+ "github.com/grafana/grafana/pkg/models"
+ ngmodels "github.com/grafana/grafana/pkg/services/ngalert/models"
+ "github.com/grafana/grafana/pkg/services/notifications"
+)
+
+// TeamsNotifier is responsible for sending
+// alert notifications to Microsoft teams.
+type TeamsNotifier struct {
+ *Base
+ URL string
+ Message string
+ Title string
+ SectionTitle string
+ tmpl *template.Template
+ log log.Logger
+ ns notifications.WebhookSender
+ images ImageStore
+}
+
+type TeamsConfig struct {
+ *NotificationChannelConfig
+ URL string
+ Message string
+ Title string
+ SectionTitle string
+}
+
+func TeamsFactory(fc FactoryConfig) (NotificationChannel, error) {
+ cfg, err := NewTeamsConfig(fc.Config)
+ if err != nil {
+ return nil, receiverInitError{
+ Reason: err.Error(),
+ Cfg: *fc.Config,
+ }
+ }
+ return NewTeamsNotifier(cfg, fc.NotificationService, fc.ImageStore, fc.Template), nil
+}
+
+func NewTeamsConfig(config *NotificationChannelConfig) (*TeamsConfig, error) {
+ URL := config.Settings.Get("url").MustString()
+ if URL == "" {
+ return nil, errors.New("could not find url property in settings")
+ }
+ return &TeamsConfig{
+ NotificationChannelConfig: config,
+ URL: URL,
+ Message: config.Settings.Get("message").MustString(`{{ template "teams.default.message" .}}`),
+ Title: config.Settings.Get("title").MustString(DefaultMessageTitleEmbed),
+ SectionTitle: config.Settings.Get("sectiontitle").MustString(""),
+ }, nil
+}
+
+type teamsImage struct {
+ Image string `json:"image"`
+}
+
+// NewTeamsNotifier is the constructor for Teams notifier.
+func NewTeamsNotifier(config *TeamsConfig, ns notifications.WebhookSender, images ImageStore, t *template.Template) *TeamsNotifier {
+ return &TeamsNotifier{
+ Base: NewBase(&models.AlertNotification{
+ Uid: config.UID,
+ Name: config.Name,
+ Type: config.Type,
+ DisableResolveMessage: config.DisableResolveMessage,
+ Settings: config.Settings,
+ }),
+ URL: config.URL,
+ Message: config.Message,
+ Title: config.Title,
+ SectionTitle: config.SectionTitle,
+ log: log.New("alerting.notifier.teams"),
+ ns: ns,
+ images: images,
+ tmpl: t,
+ }
+}
+
+// Notify send an alert notification to Microsoft teams.
+func (tn *TeamsNotifier) Notify(ctx context.Context, as ...*types.Alert) (bool, error) {
+ var tmplErr error
+ tmpl, _ := TmplText(ctx, tn.tmpl, as, tn.log, &tmplErr)
+
+ ruleURL := joinUrlPath(tn.tmpl.ExternalURL.String(), "/alerting/list", tn.log)
+
+ images := []teamsImage{}
+ _ = withStoredImages(ctx, tn.log, tn.images,
+ func(_ int, image ngmodels.Image) error {
+ if len(image.URL) != 0 {
+ images = append(images, teamsImage{Image: image.URL})
+ }
+ return nil
+ },
+ as...)
+
+ // Note: these template calls must remain in this order
+ title := tmpl(tn.Title)
+ sections := []map[string]interface{}{
+ {
+ "title": tmpl(tn.SectionTitle),
+ "text": tmpl(tn.Message),
+ },
+ }
+
+ if len(images) != 0 {
+ sections[0]["images"] = images
+ }
+
+ body := map[string]interface{}{
+ "@type": "MessageCard",
+ "@context": "http://schema.org/extensions",
+ // summary MUST not be empty or the webhook request fails
+ // summary SHOULD contain some meaningful information, since it is used for mobile notifications
+ "summary": title,
+ "title": title,
+ "themeColor": getAlertStatusColor(types.Alerts(as...).Status()),
+ "sections": sections,
+ "potentialAction": []map[string]interface{}{
+ {
+ "@context": "http://schema.org",
+ "@type": "OpenUri",
+ "name": "View Rule",
+ "targets": []map[string]interface{}{
+ {
+ "os": "default",
+ "uri": ruleURL,
+ },
+ },
+ },
+ },
+ }
+
+ if tmplErr != nil {
+ tn.log.Warn("failed to template Teams message", "err", tmplErr.Error())
+ tmplErr = nil
+ }
+
+ u := tmpl(tn.URL)
+ if tmplErr != nil {
+ tn.log.Warn("failed to template Teams URL", "err", tmplErr.Error(), "fallback", tn.URL)
+ u = tn.URL
+ }
+
+ b, err := json.Marshal(&body)
+ if err != nil {
+ return false, errors.Wrap(err, "marshal json")
+ }
+ cmd := &models.SendWebhookSync{Url: u, Body: string(b)}
+
+ if err := tn.ns.SendWebhookSync(ctx, cmd); err != nil {
+ return false, errors.Wrap(err, "send notification to Teams")
+ }
+
+ return true, nil
+}
+
+func (tn *TeamsNotifier) SendResolved() bool {
+ return !tn.GetDisableResolveMessage()
+}
diff --git a/alerting/channels/teams_test.go b/alerting/channels/teams_test.go
new file mode 100644
index 00000000..68834b88
--- /dev/null
+++ b/alerting/channels/teams_test.go
@@ -0,0 +1,242 @@
+package channels
+
+import (
+ "context"
+ "encoding/json"
+ "net/url"
+ "testing"
+
+ "github.com/prometheus/alertmanager/notify"
+ "github.com/prometheus/alertmanager/types"
+ "github.com/prometheus/common/model"
+ "github.com/stretchr/testify/require"
+
+ "github.com/grafana/grafana/pkg/components/simplejson"
+)
+
+func TestTeamsNotifier(t *testing.T) {
+ tmpl := templateForTests(t)
+
+ externalURL, err := url.Parse("http://localhost")
+ require.NoError(t, err)
+ tmpl.ExternalURL = externalURL
+
+ cases := []struct {
+ name string
+ settings string
+ alerts []*types.Alert
+ expMsg map[string]interface{}
+ expInitError string
+ expMsgError error
+ }{
+ {
+ name: "Default config with one alert",
+ settings: `{"url": "http://localhost"}`,
+ alerts: []*types.Alert{
+ {
+ Alert: model.Alert{
+ Labels: model.LabelSet{"alertname": "alert1", "lbl1": "val1"},
+ Annotations: model.LabelSet{"ann1": "annv1", "__dashboardUid__": "abcd", "__panelId__": "efgh"},
+ },
+ },
+ },
+ expMsg: map[string]interface{}{
+ "@type": "MessageCard",
+ "@context": "http://schema.org/extensions",
+ "summary": "[FIRING:1] (val1)",
+ "title": "[FIRING:1] (val1)",
+ "themeColor": "#D63232",
+ "sections": []map[string]interface{}{
+ {
+ "title": "",
+ "text": "**Firing**\n\nValue: [no value]\nLabels:\n - alertname = alert1\n - lbl1 = val1\nAnnotations:\n - ann1 = annv1\nSilence: http://localhost/alerting/silence/new?alertmanager=grafana&matcher=alertname%3Dalert1&matcher=lbl1%3Dval1\nDashboard: http://localhost/d/abcd\nPanel: http://localhost/d/abcd?viewPanel=efgh\n",
+ },
+ },
+ "potentialAction": []map[string]interface{}{
+ {
+ "@context": "http://schema.org",
+ "@type": "OpenUri",
+ "name": "View Rule",
+ "targets": []map[string]interface{}{{"os": "default", "uri": "http://localhost/alerting/list"}},
+ },
+ },
+ },
+ expMsgError: nil,
+ }, {
+ name: "Custom config with multiple alerts",
+ settings: `{
+ "url": "http://localhost",
+ "title": "{{ .CommonLabels.alertname }}",
+ "sectiontitle": "Details",
+ "message": "{{ len .Alerts.Firing }} alerts are firing, {{ len .Alerts.Resolved }} are resolved"
+ }`,
+ alerts: []*types.Alert{
+ {
+ Alert: model.Alert{
+ Labels: model.LabelSet{"alertname": "alert1", "lbl1": "val1"},
+ Annotations: model.LabelSet{"ann1": "annv1"},
+ },
+ }, {
+ Alert: model.Alert{
+ Labels: model.LabelSet{"alertname": "alert1", "lbl1": "val2"},
+ Annotations: model.LabelSet{"ann1": "annv2"},
+ },
+ },
+ },
+ expMsg: map[string]interface{}{
+ "@type": "MessageCard",
+ "@context": "http://schema.org/extensions",
+ "summary": "alert1",
+ "title": "alert1",
+ "themeColor": "#D63232",
+ "sections": []map[string]interface{}{
+ {
+ "title": "Details",
+ "text": "2 alerts are firing, 0 are resolved",
+ },
+ },
+ "potentialAction": []map[string]interface{}{
+ {
+ "@context": "http://schema.org",
+ "@type": "OpenUri",
+ "name": "View Rule",
+ "targets": []map[string]interface{}{{"os": "default", "uri": "http://localhost/alerting/list"}},
+ },
+ },
+ },
+ expMsgError: nil,
+ }, {
+ name: "Missing field in template",
+ settings: `{
+ "url": "http://localhost",
+ "title": "{{ .CommonLabels.alertname }}",
+ "sectiontitle": "Details",
+ "message": "I'm a custom template {{ .NotAField }} bad template"
+ }`,
+ alerts: []*types.Alert{
+ {
+ Alert: model.Alert{
+ Labels: model.LabelSet{"alertname": "alert1", "lbl1": "val1"},
+ Annotations: model.LabelSet{"ann1": "annv1"},
+ },
+ }, {
+ Alert: model.Alert{
+ Labels: model.LabelSet{"alertname": "alert1", "lbl1": "val2"},
+ Annotations: model.LabelSet{"ann1": "annv2"},
+ },
+ },
+ },
+ expMsg: map[string]interface{}{
+ "@type": "MessageCard",
+ "@context": "http://schema.org/extensions",
+ "summary": "alert1",
+ "title": "alert1",
+ "themeColor": "#D63232",
+ "sections": []map[string]interface{}{
+ {
+ "title": "Details",
+ "text": "I'm a custom template ",
+ },
+ },
+ "potentialAction": []map[string]interface{}{
+ {
+ "@context": "http://schema.org",
+ "@type": "OpenUri",
+ "name": "View Rule",
+ "targets": []map[string]interface{}{{"os": "default", "uri": "http://localhost/alerting/list"}},
+ },
+ },
+ },
+ expMsgError: nil,
+ }, {
+ name: "Invalid template",
+ settings: `{
+ "url": "http://localhost",
+ "title": "{{ .CommonLabels.alertname }}",
+ "sectiontitle": "Details",
+ "message": "I'm a custom template {{ {.NotAField }} bad template"
+ }`,
+ alerts: []*types.Alert{
+ {
+ Alert: model.Alert{
+ Labels: model.LabelSet{"alertname": "alert1", "lbl1": "val1"},
+ Annotations: model.LabelSet{"ann1": "annv1"},
+ },
+ }, {
+ Alert: model.Alert{
+ Labels: model.LabelSet{"alertname": "alert1", "lbl1": "val2"},
+ Annotations: model.LabelSet{"ann1": "annv2"},
+ },
+ },
+ },
+ expMsg: map[string]interface{}{
+ "@type": "MessageCard",
+ "@context": "http://schema.org/extensions",
+ "summary": "alert1",
+ "title": "alert1",
+ "themeColor": "#D63232",
+ "sections": []map[string]interface{}{
+ {
+ "title": "Details",
+ "text": "",
+ },
+ },
+ "potentialAction": []map[string]interface{}{
+ {
+ "@context": "http://schema.org",
+ "@type": "OpenUri",
+ "name": "View Rule",
+ "targets": []map[string]interface{}{{"os": "default", "uri": "http://localhost/alerting/list"}},
+ },
+ },
+ },
+ expMsgError: nil,
+ }, {
+ name: "Error in initing",
+ settings: `{}`,
+ expInitError: `could not find url property in settings`,
+ },
+ }
+
+ for _, c := range cases {
+ t.Run(c.name, func(t *testing.T) {
+ settingsJSON, err := simplejson.NewJson([]byte(c.settings))
+ require.NoError(t, err)
+
+ m := &NotificationChannelConfig{
+ Name: "teams_testing",
+ Type: "teams",
+ Settings: settingsJSON,
+ }
+
+ webhookSender := mockNotificationService()
+ cfg, err := NewTeamsConfig(m)
+ if c.expInitError != "" {
+ require.Error(t, err)
+ require.Equal(t, c.expInitError, err.Error())
+ return
+ }
+ require.NoError(t, err)
+
+ ctx := notify.WithGroupKey(context.Background(), "alertname")
+ ctx = notify.WithGroupLabels(ctx, model.LabelSet{"alertname": ""})
+ pn := NewTeamsNotifier(cfg, webhookSender, &UnavailableImageStore{}, tmpl)
+ ok, err := pn.Notify(ctx, c.alerts...)
+ if c.expMsgError != nil {
+ require.False(t, ok)
+ require.Error(t, err)
+ require.Equal(t, c.expMsgError.Error(), err.Error())
+ return
+ }
+ require.True(t, ok)
+ require.NoError(t, err)
+
+ require.NotEmpty(t, webhookSender.Webhook.Url)
+
+ expBody, err := json.Marshal(c.expMsg)
+ require.NoError(t, err)
+
+ require.JSONEq(t, string(expBody), webhookSender.Webhook.Body)
+ })
+ }
+}
diff --git a/alerting/channels/telegram.go b/alerting/channels/telegram.go
new file mode 100644
index 00000000..650a390c
--- /dev/null
+++ b/alerting/channels/telegram.go
@@ -0,0 +1,207 @@
+package channels
+
+import (
+ "bytes"
+ "context"
+ "errors"
+ "fmt"
+ "io"
+ "mime/multipart"
+ "os"
+
+ "github.com/prometheus/alertmanager/template"
+ "github.com/prometheus/alertmanager/types"
+
+ "github.com/grafana/grafana/pkg/infra/log"
+ "github.com/grafana/grafana/pkg/models"
+ ngmodels "github.com/grafana/grafana/pkg/services/ngalert/models"
+ "github.com/grafana/grafana/pkg/services/notifications"
+)
+
+var (
+ TelegramAPIURL = "https://api.telegram.org/bot%s/%s"
+)
+
+// TelegramNotifier is responsible for sending
+// alert notifications to Telegram.
+type TelegramNotifier struct {
+ *Base
+ BotToken string
+ ChatID string
+ Message string
+ log log.Logger
+ images ImageStore
+ ns notifications.WebhookSender
+ tmpl *template.Template
+}
+
+type TelegramConfig struct {
+ *NotificationChannelConfig
+ BotToken string
+ ChatID string
+ Message string
+}
+
+func TelegramFactory(fc FactoryConfig) (NotificationChannel, error) {
+ config, err := NewTelegramConfig(fc.Config, fc.DecryptFunc)
+ if err != nil {
+ return nil, receiverInitError{
+ Reason: err.Error(),
+ Cfg: *fc.Config,
+ }
+ }
+ return NewTelegramNotifier(config, fc.ImageStore, fc.NotificationService, fc.Template), nil
+}
+
+func NewTelegramConfig(config *NotificationChannelConfig, fn GetDecryptedValueFn) (*TelegramConfig, error) {
+ botToken := fn(context.Background(), config.SecureSettings, "bottoken", config.Settings.Get("bottoken").MustString())
+ if botToken == "" {
+ return &TelegramConfig{}, errors.New("could not find Bot Token in settings")
+ }
+ chatID := config.Settings.Get("chatid").MustString()
+ if chatID == "" {
+ return &TelegramConfig{}, errors.New("could not find Chat Id in settings")
+ }
+ return &TelegramConfig{
+ NotificationChannelConfig: config,
+ BotToken: botToken,
+ ChatID: chatID,
+ Message: config.Settings.Get("message").MustString(`{{ template "default.message" . }}`),
+ }, nil
+}
+
+// NewTelegramNotifier is the constructor for the Telegram notifier
+func NewTelegramNotifier(config *TelegramConfig, images ImageStore, ns notifications.WebhookSender, t *template.Template) *TelegramNotifier {
+ return &TelegramNotifier{
+ Base: NewBase(&models.AlertNotification{
+ Uid: config.UID,
+ Name: config.Name,
+ Type: config.Type,
+ DisableResolveMessage: config.DisableResolveMessage,
+ Settings: config.Settings,
+ }),
+ BotToken: config.BotToken,
+ ChatID: config.ChatID,
+ Message: config.Message,
+ tmpl: t,
+ log: log.New("alerting.notifier.telegram"),
+ images: images,
+ ns: ns,
+ }
+}
+
+// Notify send an alert notification to Telegram.
+func (tn *TelegramNotifier) Notify(ctx context.Context, as ...*types.Alert) (bool, error) {
+ // Create the cmd for sendMessage
+ cmd, err := tn.newWebhookSyncCmd("sendMessage", func(w *multipart.Writer) error {
+ msg, err := tn.buildTelegramMessage(ctx, as)
+ if err != nil {
+ return fmt.Errorf("failed to build message: %w", err)
+ }
+ for k, v := range msg {
+ fw, err := w.CreateFormField(k)
+ if err != nil {
+ return fmt.Errorf("failed to create form field: %w", err)
+ }
+ if _, err := fw.Write([]byte(v)); err != nil {
+ return fmt.Errorf("failed to write value: %w", err)
+ }
+ }
+ return nil
+ })
+ if err != nil {
+ return false, fmt.Errorf("failed to create telegram message: %w", err)
+ }
+ if err := tn.ns.SendWebhookSync(ctx, cmd); err != nil {
+ return false, fmt.Errorf("failed to send telegram message: %w", err)
+ }
+
+ // Create the cmd to upload each image
+ _ = withStoredImages(ctx, tn.log, tn.images, func(index int, image ngmodels.Image) error {
+ cmd, err = tn.newWebhookSyncCmd("sendPhoto", func(w *multipart.Writer) error {
+ f, err := os.Open(image.Path)
+ if err != nil {
+ return fmt.Errorf("failed to open image: %w", err)
+ }
+ defer func() {
+ if err := f.Close(); err != nil {
+ tn.log.Warn("failed to close image", "err", err)
+ }
+ }()
+ fw, err := w.CreateFormFile("photo", image.Path)
+ if err != nil {
+ return fmt.Errorf("failed to create form file: %w", err)
+ }
+ if _, err := io.Copy(fw, f); err != nil {
+ return fmt.Errorf("failed to write to form file: %w", err)
+ }
+ return nil
+ })
+ if err != nil {
+ return fmt.Errorf("failed to create image: %w", err)
+ }
+ if err := tn.ns.SendWebhookSync(ctx, cmd); err != nil {
+ return fmt.Errorf("failed to upload image to telegram: %w", err)
+ }
+ return nil
+ }, as...)
+
+ return true, nil
+}
+
+func (tn *TelegramNotifier) buildTelegramMessage(ctx context.Context, as []*types.Alert) (map[string]string, error) {
+ var tmplErr error
+ defer func() {
+ if tmplErr != nil {
+ tn.log.Warn("failed to template Telegram message", "err", tmplErr)
+ }
+ }()
+
+ tmpl, _ := TmplText(ctx, tn.tmpl, as, tn.log, &tmplErr)
+ m := make(map[string]string)
+ m["text"] = tmpl(tn.Message)
+ m["parse_mode"] = "html"
+ return m, nil
+}
+
+func (tn *TelegramNotifier) newWebhookSyncCmd(action string, fn func(writer *multipart.Writer) error) (*models.SendWebhookSync, error) {
+ b := bytes.Buffer{}
+ w := multipart.NewWriter(&b)
+
+ boundary := GetBoundary()
+ if boundary != "" {
+ if err := w.SetBoundary(boundary); err != nil {
+ return nil, err
+ }
+ }
+
+ fw, err := w.CreateFormField("chat_id")
+ if err != nil {
+ return nil, err
+ }
+ if _, err := fw.Write([]byte(tn.ChatID)); err != nil {
+ return nil, err
+ }
+
+ if err := fn(w); err != nil {
+ return nil, err
+ }
+
+ if err := w.Close(); err != nil {
+ return nil, fmt.Errorf("failed to close multipart: %w", err)
+ }
+
+ cmd := &models.SendWebhookSync{
+ Url: fmt.Sprintf(TelegramAPIURL, tn.BotToken, action),
+ Body: b.String(),
+ HttpMethod: "POST",
+ HttpHeader: map[string]string{
+ "Content-Type": w.FormDataContentType(),
+ },
+ }
+ return cmd, nil
+}
+
+func (tn *TelegramNotifier) SendResolved() bool {
+ return !tn.GetDisableResolveMessage()
+}
diff --git a/alerting/channels/telegram_test.go b/alerting/channels/telegram_test.go
new file mode 100644
index 00000000..7d92125d
--- /dev/null
+++ b/alerting/channels/telegram_test.go
@@ -0,0 +1,134 @@
+package channels
+
+import (
+ "context"
+ "net/url"
+ "testing"
+
+ "github.com/grafana/grafana/pkg/components/simplejson"
+ "github.com/grafana/grafana/pkg/services/secrets/fakes"
+ secretsManager "github.com/grafana/grafana/pkg/services/secrets/manager"
+
+ "github.com/prometheus/alertmanager/notify"
+ "github.com/prometheus/alertmanager/types"
+ "github.com/prometheus/common/model"
+ "github.com/stretchr/testify/require"
+)
+
+func TestTelegramNotifier(t *testing.T) {
+ tmpl := templateForTests(t)
+ images := newFakeImageStoreWithFile(t, 2)
+ externalURL, err := url.Parse("http://localhost")
+ require.NoError(t, err)
+ tmpl.ExternalURL = externalURL
+
+ cases := []struct {
+ name string
+ settings string
+ alerts []*types.Alert
+ expMsg map[string]string
+ expInitError string
+ expMsgError error
+ }{
+ {
+ name: "A single alert with default template",
+ settings: `{
+ "bottoken": "abcdefgh0123456789",
+ "chatid": "someid"
+ }`,
+ alerts: []*types.Alert{
+ {
+ Alert: model.Alert{
+ Labels: model.LabelSet{"alertname": "alert1", "lbl1": "val1"},
+ Annotations: model.LabelSet{"ann1": "annv1", "__dashboardUid__": "abcd", "__panelId__": "efgh", "__alertImageToken__": "test-image-1"},
+ GeneratorURL: "a URL",
+ },
+ },
+ },
+ expMsg: map[string]string{
+ "parse_mode": "html",
+ "text": "**Firing**\n\nValue: [no value]\nLabels:\n - alertname = alert1\n - lbl1 = val1\nAnnotations:\n - ann1 = annv1\nSource: a URL\nSilence: http://localhost/alerting/silence/new?alertmanager=grafana&matcher=alertname%3Dalert1&matcher=lbl1%3Dval1\nDashboard: http://localhost/d/abcd\nPanel: http://localhost/d/abcd?viewPanel=efgh\n",
+ },
+ expMsgError: nil,
+ }, {
+ name: "Multiple alerts with custom template",
+ settings: `{
+ "bottoken": "abcdefgh0123456789",
+ "chatid": "someid",
+ "message": "__Custom Firing__\n{{len .Alerts.Firing}} Firing\n{{ template \"__text_alert_list\" .Alerts.Firing }}"
+ }`,
+ alerts: []*types.Alert{
+ {
+ Alert: model.Alert{
+ Labels: model.LabelSet{"alertname": "alert1", "lbl1": "val1"},
+ Annotations: model.LabelSet{"ann1": "annv1", "__alertImageToken__": "test-image-1"},
+ GeneratorURL: "a URL",
+ },
+ }, {
+ Alert: model.Alert{
+ Labels: model.LabelSet{"alertname": "alert1", "lbl1": "val2"},
+ Annotations: model.LabelSet{"ann1": "annv2", "__alertImageToken__": "test-image-2"},
+ },
+ },
+ },
+ expMsg: map[string]string{
+ "parse_mode": "html",
+ "text": "__Custom Firing__\n2 Firing\n\nValue: [no value]\nLabels:\n - alertname = alert1\n - lbl1 = val1\nAnnotations:\n - ann1 = annv1\nSource: a URL\nSilence: http://localhost/alerting/silence/new?alertmanager=grafana&matcher=alertname%3Dalert1&matcher=lbl1%3Dval1\n\nValue: [no value]\nLabels:\n - alertname = alert1\n - lbl1 = val2\nAnnotations:\n - ann1 = annv2\nSilence: http://localhost/alerting/silence/new?alertmanager=grafana&matcher=alertname%3Dalert1&matcher=lbl1%3Dval2\n",
+ },
+ expMsgError: nil,
+ }, {
+ name: "Error in initing",
+ settings: `{}`,
+ expInitError: `could not find Bot Token in settings`,
+ },
+ }
+
+ for _, c := range cases {
+ t.Run(c.name, func(t *testing.T) {
+ settingsJSON, err := simplejson.NewJson([]byte(c.settings))
+ require.NoError(t, err)
+ secureSettings := make(map[string][]byte)
+
+ secretsService := secretsManager.SetupTestService(t, fakes.NewFakeSecretsStore())
+ decryptFn := secretsService.GetDecryptedValue
+ notificationService := mockNotificationService()
+
+ fc := FactoryConfig{
+ Config: &NotificationChannelConfig{
+ Name: "telegram_tests",
+ Type: "telegram",
+ Settings: settingsJSON,
+ SecureSettings: secureSettings,
+ },
+ ImageStore: images,
+ NotificationService: notificationService,
+ DecryptFunc: decryptFn,
+ }
+
+ cfg, err := NewTelegramConfig(fc.Config, decryptFn)
+ if c.expInitError != "" {
+ require.Error(t, err)
+ require.Equal(t, c.expInitError, err.Error())
+ return
+ }
+ require.NoError(t, err)
+
+ n := NewTelegramNotifier(cfg, images, notificationService, tmpl)
+
+ ctx := notify.WithGroupKey(context.Background(), "alertname")
+ ctx = notify.WithGroupLabels(ctx, model.LabelSet{"alertname": ""})
+ ok, err := n.Notify(ctx, c.alerts...)
+ require.NoError(t, err)
+ require.True(t, ok)
+
+ msg, err := n.buildTelegramMessage(ctx, c.alerts)
+ if c.expMsgError != nil {
+ require.Error(t, err)
+ require.Equal(t, c.expMsgError.Error(), err.Error())
+ return
+ }
+ require.NoError(t, err)
+ require.Equal(t, c.expMsg, msg)
+ })
+ }
+}
diff --git a/alerting/channels/template_data.go b/alerting/channels/template_data.go
new file mode 100644
index 00000000..c021d8ec
--- /dev/null
+++ b/alerting/channels/template_data.go
@@ -0,0 +1,172 @@
+package channels
+
+import (
+ "context"
+ "net/url"
+ "path"
+ "sort"
+ "strings"
+ "time"
+
+ "github.com/prometheus/alertmanager/notify"
+ "github.com/prometheus/alertmanager/template"
+ "github.com/prometheus/alertmanager/types"
+ "github.com/prometheus/common/model"
+
+ "github.com/grafana/grafana/pkg/infra/log"
+ ngmodels "github.com/grafana/grafana/pkg/services/ngalert/models"
+)
+
+type ExtendedAlert struct {
+ Status string `json:"status"`
+ Labels template.KV `json:"labels"`
+ Annotations template.KV `json:"annotations"`
+ StartsAt time.Time `json:"startsAt"`
+ EndsAt time.Time `json:"endsAt"`
+ GeneratorURL string `json:"generatorURL"`
+ Fingerprint string `json:"fingerprint"`
+ SilenceURL string `json:"silenceURL"`
+ DashboardURL string `json:"dashboardURL"`
+ PanelURL string `json:"panelURL"`
+ ValueString string `json:"valueString"`
+ ImageURL string `json:"imageURL,omitempty"`
+ EmbeddedImage string `json:"embeddedImage,omitempty"`
+}
+
+type ExtendedAlerts []ExtendedAlert
+
+type ExtendedData struct {
+ Receiver string `json:"receiver"`
+ Status string `json:"status"`
+ Alerts ExtendedAlerts `json:"alerts"`
+
+ GroupLabels template.KV `json:"groupLabels"`
+ CommonLabels template.KV `json:"commonLabels"`
+ CommonAnnotations template.KV `json:"commonAnnotations"`
+
+ ExternalURL string `json:"externalURL"`
+}
+
+func removePrivateItems(kv template.KV) template.KV {
+ for key := range kv {
+ if strings.HasPrefix(key, "__") && strings.HasSuffix(key, "__") {
+ kv = kv.Remove([]string{key})
+ }
+ }
+ return kv
+}
+
+func extendAlert(alert template.Alert, externalURL string, logger log.Logger) *ExtendedAlert {
+ // remove "private" annotations & labels so they don't show up in the template
+ extended := &ExtendedAlert{
+ Status: alert.Status,
+ Labels: removePrivateItems(alert.Labels),
+ Annotations: removePrivateItems(alert.Annotations),
+ StartsAt: alert.StartsAt,
+ EndsAt: alert.EndsAt,
+ GeneratorURL: alert.GeneratorURL,
+ Fingerprint: alert.Fingerprint,
+ }
+
+ // fill in some grafana-specific urls
+ if len(externalURL) == 0 {
+ return extended
+ }
+ u, err := url.Parse(externalURL)
+ if err != nil {
+ logger.Debug("failed to parse external URL while extending template data", "url", externalURL, "err", err.Error())
+ return extended
+ }
+ externalPath := u.Path
+ dashboardUid := alert.Annotations[ngmodels.DashboardUIDAnnotation]
+ if len(dashboardUid) > 0 {
+ u.Path = path.Join(externalPath, "/d/", dashboardUid)
+ extended.DashboardURL = u.String()
+ panelId := alert.Annotations[ngmodels.PanelIDAnnotation]
+ if len(panelId) > 0 {
+ u.RawQuery = "viewPanel=" + panelId
+ extended.PanelURL = u.String()
+ }
+ }
+
+ if alert.Annotations != nil {
+ extended.ValueString = alert.Annotations[`__value_string__`]
+ }
+
+ matchers := make([]string, 0)
+ for key, value := range alert.Labels {
+ if !(strings.HasPrefix(key, "__") && strings.HasSuffix(key, "__")) {
+ matchers = append(matchers, key+"="+value)
+ }
+ }
+ sort.Strings(matchers)
+ u.Path = path.Join(externalPath, "/alerting/silence/new")
+
+ query := make(url.Values)
+ query.Add("alertmanager", "grafana")
+ for _, matcher := range matchers {
+ query.Add("matcher", matcher)
+ }
+
+ u.RawQuery = query.Encode()
+
+ extended.SilenceURL = u.String()
+
+ return extended
+}
+
+func ExtendData(data *template.Data, logger log.Logger) *ExtendedData {
+ alerts := []ExtendedAlert{}
+
+ for _, alert := range data.Alerts {
+ extendedAlert := extendAlert(alert, data.ExternalURL, logger)
+ alerts = append(alerts, *extendedAlert)
+ }
+
+ extended := &ExtendedData{
+ Receiver: data.Receiver,
+ Status: data.Status,
+ Alerts: alerts,
+ GroupLabels: data.GroupLabels,
+ CommonLabels: removePrivateItems(data.CommonLabels),
+ CommonAnnotations: removePrivateItems(data.CommonAnnotations),
+
+ ExternalURL: data.ExternalURL,
+ }
+ return extended
+}
+
+func TmplText(ctx context.Context, tmpl *template.Template, alerts []*types.Alert, l log.Logger, tmplErr *error) (func(string) string, *ExtendedData) {
+ promTmplData := notify.GetTemplateData(ctx, tmpl, alerts, l)
+ data := ExtendData(promTmplData, l)
+
+ return func(name string) (s string) {
+ if *tmplErr != nil {
+ return
+ }
+ s, *tmplErr = tmpl.ExecuteTextString(name, data)
+ return s
+ }, data
+}
+
+// Firing returns the subset of alerts that are firing.
+func (as ExtendedAlerts) Firing() []ExtendedAlert {
+ res := []ExtendedAlert{}
+ for _, a := range as {
+ if a.Status == string(model.AlertFiring) {
+ res = append(res, a)
+ }
+ }
+ return res
+}
+
+// Resolved returns the subset of alerts that are resolved.
+func (as ExtendedAlerts) Resolved() []ExtendedAlert {
+ res := []ExtendedAlert{}
+ for _, a := range as {
+ if a.Status == string(model.AlertResolved) {
+ res = append(res, a)
+ }
+ }
+ return res
+}
diff --git a/alerting/channels/testing.go b/alerting/channels/testing.go
new file mode 100644
index 00000000..af7467cc
--- /dev/null
+++ b/alerting/channels/testing.go
@@ -0,0 +1,144 @@
+package channels
+
+import (
+ "context"
+ "encoding/base64"
+ "fmt"
+ "os"
+ "testing"
+ "time"
+
+ "github.com/grafana/grafana/pkg/models"
+ ngmodels "github.com/grafana/grafana/pkg/services/ngalert/models"
+)
+
+type fakeImageStore struct {
+ Images []*ngmodels.Image
+}
+
+// getImage returns an image with the same token.
+func (f *fakeImageStore) GetImage(_ context.Context, token string) (*ngmodels.Image, error) {
+ for _, img := range f.Images {
+ if img.Token == token {
+ return img, nil
+ }
+ }
+ return nil, ngmodels.ErrImageNotFound
+}
+
+// newFakeImageStore returns an image store with N test images.
+// Each image has a token and a URL, but does not have a file on disk.
+func newFakeImageStore(n int) ImageStore {
+ s := fakeImageStore{}
+ for i := 1; i <= n; i++ {
+ s.Images = append(s.Images, &ngmodels.Image{
+ Token: fmt.Sprintf("test-image-%d", i),
+ URL: fmt.Sprintf("https://www.example.com/test-image-%d.jpg", i),
+ CreatedAt: time.Now().UTC(),
+ })
+ }
+ return &s
+}
+
+// newFakeImageStoreWithFile returns an image store with N test images.
+// Each image has a token, path and a URL, where the path is 1x1 transparent
+// PNG on disk. The test should call deleteFunc to delete the images from disk
+// at the end of the test.
+// nolint:deadcode,unused
+func newFakeImageStoreWithFile(t *testing.T, n int) ImageStore {
+ var (
+ files []string
+ s fakeImageStore
+ )
+
+ t.Cleanup(func() {
+ // remove all files from disk
+ for _, f := range files {
+ if err := os.Remove(f); err != nil {
+ t.Logf("failed to delete file: %s", err)
+ }
+ }
+ })
+
+ for i := 1; i <= n; i++ {
+ file, err := newTestImage()
+ if err != nil {
+ t.Fatalf("failed to create test image: %s", err)
+ }
+ files = append(files, file)
+ s.Images = append(s.Images, &ngmodels.Image{
+ Token: fmt.Sprintf("test-image-%d", i),
+ Path: file,
+ URL: fmt.Sprintf("https://www.example.com/test-image-%d", i),
+ CreatedAt: time.Now().UTC(),
+ })
+ }
+
+ return &s
+}
+
+// nolint:deadcode,unused
+func newTestImage() (string, error) {
+ f, err := os.CreateTemp("", "test-image-*.png")
+ if err != nil {
+ return "", fmt.Errorf("failed to create temp image: %s", err)
+ }
+
+ // 1x1 transparent PNG
+ b, err := base64.StdEncoding.DecodeString("iVBORw0KGgoAAAANSUhEUgAAAAEAAAABCAQAAAC1HAwCAAAAC0lEQVR42mNkYAAAAAYAAjCB0C8AAAAASUVORK5CYII=")
+ if err != nil {
+ return f.Name(), fmt.Errorf("failed to decode PNG data: %s", err)
+ }
+
+ if _, err := f.Write(b); err != nil {
+ return f.Name(), fmt.Errorf("failed to write to file: %s", err)
+ }
+
+ if err := f.Close(); err != nil {
+ return f.Name(), fmt.Errorf("failed to close file: %s", err)
+ }
+
+ return f.Name(), nil
+}
+
+// mockTimeNow replaces function timeNow to return constant time.
+// It returns a function that resets the variable back to its original value.
+// This allows usage of this function with defer:
+// func Test (t *testing.T) {
+// now := time.Now()
+// defer mockTimeNow(now)()
+// ...
+// }
+func mockTimeNow(constTime time.Time) func() {
+ timeNow = func() time.Time {
+ return constTime
+ }
+ return resetTimeNow
+}
+
+// resetTimeNow resets the global variable timeNow to the default value, which is time.Now
+func resetTimeNow() {
+ timeNow = time.Now
+}
+
+type notificationServiceMock struct {
+ Webhook models.SendWebhookSync
+ EmailSync models.SendEmailCommandSync
+ Emailx models.SendEmailCommand
+ ShouldError error
+}
+
+func (ns *notificationServiceMock) SendWebhookSync(ctx context.Context, cmd *models.SendWebhookSync) error {
+ ns.Webhook = *cmd
+ return ns.ShouldError
+}
+func (ns *notificationServiceMock) SendEmailCommandHandlerSync(ctx context.Context, cmd *models.SendEmailCommandSync) error {
+ ns.EmailSync = *cmd
+ return ns.ShouldError
+}
+func (ns *notificationServiceMock) SendEmailCommandHandler(ctx context.Context, cmd *models.SendEmailCommand) error {
+ ns.Emailx = *cmd
+ return ns.ShouldError
+}
+
+func mockNotificationService() *notificationServiceMock { return ¬ificationServiceMock{} }
diff --git a/alerting/channels/threema.go b/alerting/channels/threema.go
new file mode 100644
index 00000000..81f53290
--- /dev/null
+++ b/alerting/channels/threema.go
@@ -0,0 +1,166 @@
+package channels
+
+import (
+ "context"
+ "errors"
+ "fmt"
+ "net/url"
+ "path"
+ "strings"
+
+ "github.com/prometheus/alertmanager/template"
+ "github.com/prometheus/alertmanager/types"
+ "github.com/prometheus/common/model"
+
+ "github.com/grafana/grafana/pkg/infra/log"
+ "github.com/grafana/grafana/pkg/models"
+ ngmodels "github.com/grafana/grafana/pkg/services/ngalert/models"
+ "github.com/grafana/grafana/pkg/services/notifications"
+)
+
+var (
+ ThreemaGwBaseURL = "https://msgapi.threema.ch/send_simple"
+)
+
+// ThreemaNotifier is responsible for sending
+// alert notifications to Threema.
+type ThreemaNotifier struct {
+ *Base
+ GatewayID string
+ RecipientID string
+ APISecret string
+ log log.Logger
+ images ImageStore
+ ns notifications.WebhookSender
+ tmpl *template.Template
+}
+
+type ThreemaConfig struct {
+ *NotificationChannelConfig
+ GatewayID string
+ RecipientID string
+ APISecret string
+}
+
+func ThreemaFactory(fc FactoryConfig) (NotificationChannel, error) {
+ cfg, err := NewThreemaConfig(fc.Config, fc.DecryptFunc)
+ if err != nil {
+ return nil, receiverInitError{
+ Reason: err.Error(),
+ Cfg: *fc.Config,
+ }
+ }
+ return NewThreemaNotifier(cfg, fc.ImageStore, fc.NotificationService, fc.Template), nil
+}
+
+func NewThreemaConfig(config *NotificationChannelConfig, decryptFunc GetDecryptedValueFn) (*ThreemaConfig, error) {
+ gatewayID := config.Settings.Get("gateway_id").MustString()
+ if gatewayID == "" {
+ return nil, errors.New("could not find Threema Gateway ID in settings")
+ }
+ if !strings.HasPrefix(gatewayID, "*") {
+ return nil, errors.New("invalid Threema Gateway ID: Must start with a *")
+ }
+ if len(gatewayID) != 8 {
+ return nil, errors.New("invalid Threema Gateway ID: Must be 8 characters long")
+ }
+ recipientID := config.Settings.Get("recipient_id").MustString()
+ if recipientID == "" {
+ return nil, errors.New("could not find Threema Recipient ID in settings")
+ }
+ if len(recipientID) != 8 {
+ return nil, errors.New("invalid Threema Recipient ID: Must be 8 characters long")
+ }
+ apiSecret := decryptFunc(context.Background(), config.SecureSettings, "api_secret", config.Settings.Get("api_secret").MustString())
+ if apiSecret == "" {
+ return nil, errors.New("could not find Threema API secret in settings")
+ }
+ return &ThreemaConfig{
+ NotificationChannelConfig: config,
+ GatewayID: gatewayID,
+ RecipientID: recipientID,
+ APISecret: apiSecret,
+ }, nil
+}
+
+// NewThreemaNotifier is the constructor for the Threema notifier
+func NewThreemaNotifier(config *ThreemaConfig, images ImageStore, ns notifications.WebhookSender, t *template.Template) *ThreemaNotifier {
+ return &ThreemaNotifier{
+ Base: NewBase(&models.AlertNotification{
+ Uid: config.UID,
+ Name: config.Name,
+ Type: config.Type,
+ DisableResolveMessage: config.DisableResolveMessage,
+ Settings: config.Settings,
+ }),
+ GatewayID: config.GatewayID,
+ RecipientID: config.RecipientID,
+ APISecret: config.APISecret,
+ log: log.New("alerting.notifier.threema"),
+ images: images,
+ ns: ns,
+ tmpl: t,
+ }
+}
+
+// Notify send an alert notification to Threema
+func (tn *ThreemaNotifier) Notify(ctx context.Context, as ...*types.Alert) (bool, error) {
+ tn.log.Debug("sending threema alert notification", "from", tn.GatewayID, "to", tn.RecipientID)
+
+ var tmplErr error
+ tmpl, _ := TmplText(ctx, tn.tmpl, as, tn.log, &tmplErr)
+
+ // Set up basic API request data
+ data := url.Values{}
+ data.Set("from", tn.GatewayID)
+ data.Set("to", tn.RecipientID)
+ data.Set("secret", tn.APISecret)
+
+ // Determine emoji
+ stateEmoji := "\u26A0\uFE0F " // Warning sign
+ alerts := types.Alerts(as...)
+ if alerts.Status() == model.AlertResolved {
+ stateEmoji = "\u2705 " // Check Mark Button
+ }
+
+ // Build message
+ message := fmt.Sprintf("%s%s\n\n*Message:*\n%s\n*URL:* %s\n",
+ stateEmoji,
+ tmpl(DefaultMessageTitleEmbed),
+ tmpl(`{{ template "default.message" . }}`),
+ path.Join(tn.tmpl.ExternalURL.String(), "/alerting/list"),
+ )
+
+ _ = withStoredImages(ctx, tn.log, tn.images,
+ func(_ int, image ngmodels.Image) error {
+ if image.URL != "" {
+ message += fmt.Sprintf("*Image:* %s\n", image.URL)
+ }
+ return nil
+ }, as...)
+
+ data.Set("text", message)
+
+ if tmplErr != nil {
+ tn.log.Warn("failed to template Threema message", "err", tmplErr.Error())
+ }
+
+ cmd := &models.SendWebhookSync{
+ Url: ThreemaGwBaseURL,
+ Body: data.Encode(),
+ HttpMethod: "POST",
+ HttpHeader: map[string]string{
+ "Content-Type": "application/x-www-form-urlencoded",
+ },
+ }
+ if err := tn.ns.SendWebhookSync(ctx, cmd); err != nil {
+ tn.log.Error("Failed to send threema notification", "err", err, "webhook", tn.Name)
+ return false, err
+ }
+
+ return true, nil
+}
+
+func (tn *ThreemaNotifier) SendResolved() bool {
+ return !tn.GetDisableResolveMessage()
+}
diff --git a/alerting/channels/threema_test.go b/alerting/channels/threema_test.go
new file mode 100644
index 00000000..7c1e1cf7
--- /dev/null
+++ b/alerting/channels/threema_test.go
@@ -0,0 +1,140 @@
+package channels
+
+import (
+ "context"
+ "net/url"
+ "testing"
+
+ "github.com/grafana/grafana/pkg/components/simplejson"
+ "github.com/grafana/grafana/pkg/services/secrets/fakes"
+ secretsManager "github.com/grafana/grafana/pkg/services/secrets/manager"
+
+ "github.com/prometheus/alertmanager/notify"
+ "github.com/prometheus/alertmanager/types"
+ "github.com/prometheus/common/model"
+ "github.com/stretchr/testify/require"
+)
+
+func TestThreemaNotifier(t *testing.T) {
+ tmpl := templateForTests(t)
+
+ images := newFakeImageStore(2)
+
+ externalURL, err := url.Parse("http://localhost")
+ require.NoError(t, err)
+ tmpl.ExternalURL = externalURL
+
+ cases := []struct {
+ name string
+ settings string
+ alerts []*types.Alert
+ expMsg string
+ expInitError string
+ expMsgError error
+ }{
+ {
+ name: "A single alert with an image",
+ settings: `{
+ "gateway_id": "*1234567",
+ "recipient_id": "87654321",
+ "api_secret": "supersecret"
+ }`,
+ alerts: []*types.Alert{
+ {
+ Alert: model.Alert{
+ Labels: model.LabelSet{"alertname": "alert1", "lbl1": "val1"},
+ Annotations: model.LabelSet{"ann1": "annv1", "__dashboardUid__": "abcd", "__panelId__": "efgh", "__alertImageToken__": "test-image-1"},
+ },
+ },
+ },
+ expMsg: "from=%2A1234567&secret=supersecret&text=%E2%9A%A0%EF%B8%8F+%5BFIRING%3A1%5D++%28val1%29%0A%0A%2AMessage%3A%2A%0A%2A%2AFiring%2A%2A%0A%0AValue%3A+%5Bno+value%5D%0ALabels%3A%0A+-+alertname+%3D+alert1%0A+-+lbl1+%3D+val1%0AAnnotations%3A%0A+-+ann1+%3D+annv1%0ASilence%3A+http%3A%2F%2Flocalhost%2Falerting%2Fsilence%2Fnew%3Falertmanager%3Dgrafana%26matcher%3Dalertname%253Dalert1%26matcher%3Dlbl1%253Dval1%0ADashboard%3A+http%3A%2F%2Flocalhost%2Fd%2Fabcd%0APanel%3A+http%3A%2F%2Flocalhost%2Fd%2Fabcd%3FviewPanel%3Defgh%0A%0A%2AURL%3A%2A+http%3A%2Flocalhost%2Falerting%2Flist%0A%2AImage%3A%2A+https%3A%2F%2Fwww.example.com%2Ftest-image-1.jpg%0A&to=87654321",
+ expMsgError: nil,
+ }, {
+ name: "Multiple alerts with images",
+ settings: `{
+ "gateway_id": "*1234567",
+ "recipient_id": "87654321",
+ "api_secret": "supersecret"
+ }`,
+ alerts: []*types.Alert{
+ {
+ Alert: model.Alert{
+ Labels: model.LabelSet{"alertname": "alert1", "lbl1": "val1"},
+ Annotations: model.LabelSet{"ann1": "annv1", "__alertImageToken__": "test-image-1"},
+ },
+ }, {
+ Alert: model.Alert{
+ Labels: model.LabelSet{"alertname": "alert1", "lbl1": "val2"},
+ Annotations: model.LabelSet{"ann1": "annv2", "__alertImageToken__": "test-image-2"},
+ },
+ },
+ },
+ expMsg: "from=%2A1234567&secret=supersecret&text=%E2%9A%A0%EF%B8%8F+%5BFIRING%3A2%5D++%0A%0A%2AMessage%3A%2A%0A%2A%2AFiring%2A%2A%0A%0AValue%3A+%5Bno+value%5D%0ALabels%3A%0A+-+alertname+%3D+alert1%0A+-+lbl1+%3D+val1%0AAnnotations%3A%0A+-+ann1+%3D+annv1%0ASilence%3A+http%3A%2F%2Flocalhost%2Falerting%2Fsilence%2Fnew%3Falertmanager%3Dgrafana%26matcher%3Dalertname%253Dalert1%26matcher%3Dlbl1%253Dval1%0A%0AValue%3A+%5Bno+value%5D%0ALabels%3A%0A+-+alertname+%3D+alert1%0A+-+lbl1+%3D+val2%0AAnnotations%3A%0A+-+ann1+%3D+annv2%0ASilence%3A+http%3A%2F%2Flocalhost%2Falerting%2Fsilence%2Fnew%3Falertmanager%3Dgrafana%26matcher%3Dalertname%253Dalert1%26matcher%3Dlbl1%253Dval2%0A%0A%2AURL%3A%2A+http%3A%2Flocalhost%2Falerting%2Flist%0A%2AImage%3A%2A+https%3A%2F%2Fwww.example.com%2Ftest-image-1.jpg%0A%2AImage%3A%2A+https%3A%2F%2Fwww.example.com%2Ftest-image-2.jpg%0A&to=87654321",
+ expMsgError: nil,
+ }, {
+ name: "Invalid gateway id",
+ settings: `{
+ "gateway_id": "12345678",
+ "recipient_id": "87654321",
+ "api_secret": "supersecret"
+ }`,
+ expInitError: `invalid Threema Gateway ID: Must start with a *`,
+ }, {
+ name: "Invalid receipent id",
+ settings: `{
+ "gateway_id": "*1234567",
+ "recipient_id": "8765432",
+ "api_secret": "supersecret"
+ }`,
+ expInitError: `invalid Threema Recipient ID: Must be 8 characters long`,
+ }, {
+ name: "No API secret",
+ settings: `{
+ "gateway_id": "*1234567",
+ "recipient_id": "87654321"
+ }`,
+ expInitError: `could not find Threema API secret in settings`,
+ },
+ }
+
+ for _, c := range cases {
+ t.Run(c.name, func(t *testing.T) {
+ settingsJSON, err := simplejson.NewJson([]byte(c.settings))
+ require.NoError(t, err)
+ secureSettings := make(map[string][]byte)
+
+ m := &NotificationChannelConfig{
+ Name: "threema_testing",
+ Type: "threema",
+ Settings: settingsJSON,
+ SecureSettings: secureSettings,
+ }
+
+ webhookSender := mockNotificationService()
+ secretsService := secretsManager.SetupTestService(t, fakes.NewFakeSecretsStore())
+ decryptFn := secretsService.GetDecryptedValue
+ cfg, err := NewThreemaConfig(m, decryptFn)
+ if c.expInitError != "" {
+ require.Error(t, err)
+ require.Equal(t, c.expInitError, err.Error())
+ return
+ }
+ require.NoError(t, err)
+
+ ctx := notify.WithGroupKey(context.Background(), "alertname")
+ ctx = notify.WithGroupLabels(ctx, model.LabelSet{"alertname": ""})
+ pn := NewThreemaNotifier(cfg, images, webhookSender, tmpl)
+ ok, err := pn.Notify(ctx, c.alerts...)
+ if c.expMsgError != nil {
+ require.False(t, ok)
+ require.Error(t, err)
+ require.Equal(t, c.expMsgError.Error(), err.Error())
+ return
+ }
+ require.NoError(t, err)
+ require.True(t, ok)
+
+ require.Equal(t, c.expMsg, webhookSender.Webhook.Body)
+ })
+ }
+}
diff --git a/alerting/channels/util.go b/alerting/channels/util.go
new file mode 100644
index 00000000..226e540f
--- /dev/null
+++ b/alerting/channels/util.go
@@ -0,0 +1,250 @@
+package channels
+
+import (
+ "bytes"
+ "context"
+ "crypto/tls"
+ "errors"
+ "fmt"
+ "io"
+ "net"
+ "net/http"
+ "net/url"
+ "os"
+ "path"
+ "path/filepath"
+ "time"
+
+ "github.com/prometheus/alertmanager/notify"
+ "github.com/prometheus/alertmanager/types"
+ "github.com/prometheus/common/model"
+
+ "github.com/grafana/grafana/pkg/infra/log"
+ "github.com/grafana/grafana/pkg/services/ngalert/models"
+ "github.com/grafana/grafana/pkg/util"
+
+ "github.com/grafana/grafana/pkg/components/simplejson"
+)
+
+const (
+ FooterIconURL = "https://grafana.com/assets/img/fav32.png"
+ ColorAlertFiring = "#D63232"
+ ColorAlertResolved = "#36a64f"
+
+ // ImageStoreTimeout should be used by all callers for calles to `Images`
+ ImageStoreTimeout time.Duration = 500 * time.Millisecond
+)
+
+var (
+ // Provides current time. Can be overwritten in tests.
+ timeNow = time.Now
+
+ // ErrImagesDone is used to stop iteration of subsequent images. It should be
+ // returned from forEachFunc when either the intended image has been found or
+ // the maximum number of images has been iterated.
+ ErrImagesDone = errors.New("images done")
+ ErrImagesUnavailable = errors.New("alert screenshots are unavailable")
+)
+
+type forEachImageFunc func(index int, image models.Image) error
+
+// getImage returns the image for the alert or an error. It returns a nil
+// image if the alert does not have an image token or the image does not exist.
+func getImage(ctx context.Context, l log.Logger, imageStore ImageStore, alert types.Alert) (*models.Image, error) {
+ token := getTokenFromAnnotations(alert.Annotations)
+ if token == "" {
+ return nil, nil
+ }
+
+ ctx, cancelFunc := context.WithTimeout(ctx, ImageStoreTimeout)
+ defer cancelFunc()
+
+ img, err := imageStore.GetImage(ctx, token)
+ if errors.Is(err, models.ErrImageNotFound) || errors.Is(err, ErrImagesUnavailable) {
+ return nil, nil
+ } else if err != nil {
+ l.Warn("failed to get image with token", "token", token, "err", err)
+ return nil, err
+ } else {
+ return img, nil
+ }
+}
+
+// withStoredImages retrieves the image for each alert and then calls forEachFunc
+// with the index of the alert and the retrieved image struct. If the alert does
+// not have an image token, or the image does not exist then forEachFunc will not be
+// called for that alert. If forEachFunc returns an error, withStoredImages will return
+// the error and not iterate the remaining alerts. A forEachFunc can return ErrImagesDone
+// to stop the iteration of remaining alerts if the intended image or maximum number of
+// images have been found.
+func withStoredImages(ctx context.Context, l log.Logger, imageStore ImageStore, forEachFunc forEachImageFunc, alerts ...*types.Alert) error {
+ for index, alert := range alerts {
+ img, err := getImage(ctx, l, imageStore, *alert)
+ if err != nil {
+ return err
+ } else if img != nil {
+ if err := forEachFunc(index, *img); err != nil {
+ if errors.Is(err, ErrImagesDone) {
+ return nil
+ }
+ return err
+ }
+ }
+ }
+ return nil
+}
+
+// The path argument here comes from reading internal image storage, not user
+// input, so we ignore the security check here.
+//nolint:gosec
+func openImage(path string) (io.ReadCloser, error) {
+ fp := filepath.Clean(path)
+ _, err := os.Stat(fp)
+ if os.IsNotExist(err) || os.IsPermission(err) {
+ return nil, models.ErrImageNotFound
+ }
+
+ f, err := os.Open(fp)
+ if err != nil {
+ return nil, err
+ }
+
+ return f, nil
+}
+
+func getTokenFromAnnotations(annotations model.LabelSet) string {
+ if value, ok := annotations[models.ImageTokenAnnotation]; ok {
+ return string(value)
+ }
+ return ""
+}
+
+type UnavailableImageStore struct{}
+
+// Get returns the image with the corresponding token, or ErrImageNotFound.
+func (u *UnavailableImageStore) GetImage(ctx context.Context, token string) (*models.Image, error) {
+ return nil, ErrImagesUnavailable
+}
+
+type receiverInitError struct {
+ Reason string
+ Err error
+ Cfg NotificationChannelConfig
+}
+
+func (e receiverInitError) Error() string {
+ name := ""
+ if e.Cfg.Name != "" {
+ name = fmt.Sprintf("%q ", e.Cfg.Name)
+ }
+
+ s := fmt.Sprintf("failed to validate receiver %sof type %q: %s", name, e.Cfg.Type, e.Reason)
+ if e.Err != nil {
+ return fmt.Sprintf("%s: %s", s, e.Err.Error())
+ }
+
+ return s
+}
+
+func (e receiverInitError) Unwrap() error { return e.Err }
+
+func getAlertStatusColor(status model.AlertStatus) string {
+ if status == model.AlertFiring {
+ return ColorAlertFiring
+ }
+ return ColorAlertResolved
+}
+
+type NotificationChannel interface {
+ notify.Notifier
+ notify.ResolvedSender
+}
+type NotificationChannelConfig struct {
+ OrgID int64 // only used internally
+ UID string `json:"uid"`
+ Name string `json:"name"`
+ Type string `json:"type"`
+ DisableResolveMessage bool `json:"disableResolveMessage"`
+ Settings *simplejson.Json `json:"settings"`
+ SecureSettings map[string][]byte `json:"secureSettings"`
+}
+
+type httpCfg struct {
+ body []byte
+ user string
+ password string
+}
+
+// sendHTTPRequest sends an HTTP request.
+// Stubbable by tests.
+var sendHTTPRequest = func(ctx context.Context, url *url.URL, cfg httpCfg, logger log.Logger) ([]byte, error) {
+ var reader io.Reader
+ if len(cfg.body) > 0 {
+ reader = bytes.NewReader(cfg.body)
+ }
+ request, err := http.NewRequestWithContext(ctx, http.MethodPost, url.String(), reader)
+ if err != nil {
+ return nil, fmt.Errorf("failed to create HTTP request: %w", err)
+ }
+ if cfg.user != "" && cfg.password != "" {
+ request.Header.Set("Authorization", util.GetBasicAuthHeader(cfg.user, cfg.password))
+ }
+
+ request.Header.Set("Content-Type", "application/json")
+ request.Header.Set("User-Agent", "Grafana")
+ netTransport := &http.Transport{
+ TLSClientConfig: &tls.Config{
+ Renegotiation: tls.RenegotiateFreelyAsClient,
+ },
+ Proxy: http.ProxyFromEnvironment,
+ DialContext: (&net.Dialer{
+ Timeout: 30 * time.Second,
+ }).DialContext,
+ TLSHandshakeTimeout: 5 * time.Second,
+ }
+ netClient := &http.Client{
+ Timeout: time.Second * 30,
+ Transport: netTransport,
+ }
+ resp, err := netClient.Do(request)
+ if err != nil {
+ return nil, err
+ }
+ defer func() {
+ if err := resp.Body.Close(); err != nil {
+ logger.Warn("failed to close response body", "err", err)
+ }
+ }()
+
+ respBody, err := io.ReadAll(resp.Body)
+ if err != nil {
+ return nil, fmt.Errorf("failed to read response body: %w", err)
+ }
+
+ if resp.StatusCode/100 != 2 {
+ logger.Warn("HTTP request failed", "url", request.URL.String(), "statusCode", resp.Status, "body",
+ string(respBody))
+ return nil, fmt.Errorf("failed to send HTTP request - status code %d", resp.StatusCode)
+ }
+
+ logger.Debug("sending HTTP request succeeded", "url", request.URL.String(), "statusCode", resp.Status)
+ return respBody, nil
+}
+
+func joinUrlPath(base, additionalPath string, logger log.Logger) string {
+ u, err := url.Parse(base)
+ if err != nil {
+ logger.Debug("failed to parse URL while joining URL", "url", base, "err", err.Error())
+ return base
+ }
+
+ u.Path = path.Join(u.Path, additionalPath)
+
+ return u.String()
+}
+
+// GetBoundary is used for overriding the behaviour for tests
+// and set a boundary for multipart body. DO NOT set this outside tests.
+var GetBoundary = func() string {
+ return ""
+}
diff --git a/alerting/channels/util_test.go b/alerting/channels/util_test.go
new file mode 100644
index 00000000..08ae8c19
--- /dev/null
+++ b/alerting/channels/util_test.go
@@ -0,0 +1,63 @@
+package channels
+
+import (
+ "context"
+ "testing"
+ "time"
+
+ "github.com/prometheus/alertmanager/types"
+ "github.com/prometheus/common/model"
+ "github.com/stretchr/testify/assert"
+ "github.com/stretchr/testify/require"
+
+ "github.com/grafana/grafana/pkg/infra/log"
+ "github.com/grafana/grafana/pkg/services/ngalert/models"
+)
+
+func TestWithStoredImages(t *testing.T) {
+ ctx := context.Background()
+ alerts := []*types.Alert{{
+ Alert: model.Alert{
+ Annotations: model.LabelSet{
+ models.ImageTokenAnnotation: "test-image-1",
+ },
+ },
+ }, {
+ Alert: model.Alert{
+ Annotations: model.LabelSet{
+ models.ImageTokenAnnotation: "test-image-2",
+ },
+ },
+ }}
+ imageStore := &fakeImageStore{Images: []*models.Image{{
+ Token: "test-image-1",
+ URL: "https://www.example.com/test-image-1.jpg",
+ CreatedAt: time.Now().UTC(),
+ }, {
+ Token: "test-image-2",
+ URL: "https://www.example.com/test-image-2.jpg",
+ CreatedAt: time.Now().UTC(),
+ }}}
+
+ var (
+ err error
+ i int
+ )
+
+ // should iterate all images
+ err = withStoredImages(ctx, log.New(ctx), imageStore, func(index int, image models.Image) error {
+ i += 1
+ return nil
+ }, alerts...)
+ require.NoError(t, err)
+ assert.Equal(t, 2, i)
+
+ // should iterate just the first image
+ i = 0
+ err = withStoredImages(ctx, log.New(ctx), imageStore, func(index int, image models.Image) error {
+ i += 1
+ return ErrImagesDone
+ }, alerts...)
+ require.NoError(t, err)
+ assert.Equal(t, 1, i)
+}
diff --git a/alerting/channels/victorops.go b/alerting/channels/victorops.go
new file mode 100644
index 00000000..cfd66db7
--- /dev/null
+++ b/alerting/channels/victorops.go
@@ -0,0 +1,163 @@
+package channels
+
+import (
+ "context"
+ "errors"
+ "strings"
+ "time"
+
+ "github.com/prometheus/alertmanager/notify"
+ "github.com/prometheus/alertmanager/template"
+ "github.com/prometheus/alertmanager/types"
+ "github.com/prometheus/common/model"
+
+ "github.com/grafana/grafana/pkg/components/simplejson"
+ "github.com/grafana/grafana/pkg/infra/log"
+ "github.com/grafana/grafana/pkg/models"
+ ngmodels "github.com/grafana/grafana/pkg/services/ngalert/models"
+ "github.com/grafana/grafana/pkg/services/notifications"
+ "github.com/grafana/grafana/pkg/setting"
+)
+
+const (
+ // victoropsAlertStateCritical - Victorops uses "CRITICAL" string to indicate "Alerting" state
+ victoropsAlertStateCritical = "CRITICAL"
+
+ // victoropsAlertStateRecovery - VictorOps "RECOVERY" message type
+ victoropsAlertStateRecovery = "RECOVERY"
+)
+
+type VictorOpsConfig struct {
+ *NotificationChannelConfig
+ URL string
+ MessageType string
+}
+
+func VictorOpsFactory(fc FactoryConfig) (NotificationChannel, error) {
+ cfg, err := NewVictorOpsConfig(fc.Config)
+ if err != nil {
+ return nil, receiverInitError{
+ Reason: err.Error(),
+ Cfg: *fc.Config,
+ }
+ }
+ return NewVictoropsNotifier(cfg, fc.ImageStore, fc.NotificationService, fc.Template), nil
+}
+
+func NewVictorOpsConfig(config *NotificationChannelConfig) (*VictorOpsConfig, error) {
+ url := config.Settings.Get("url").MustString()
+ if url == "" {
+ return nil, errors.New("could not find victorops url property in settings")
+ }
+ return &VictorOpsConfig{
+ NotificationChannelConfig: config,
+ URL: url,
+ MessageType: config.Settings.Get("messageType").MustString(),
+ }, nil
+}
+
+// NewVictoropsNotifier creates an instance of VictoropsNotifier that
+// handles posting notifications to Victorops REST API
+func NewVictoropsNotifier(config *VictorOpsConfig, images ImageStore, ns notifications.WebhookSender, t *template.Template) *VictoropsNotifier {
+ return &VictoropsNotifier{
+ Base: NewBase(&models.AlertNotification{
+ Uid: config.UID,
+ Name: config.Name,
+ Type: config.Type,
+ DisableResolveMessage: config.DisableResolveMessage,
+ Settings: config.Settings,
+ }),
+ URL: config.URL,
+ MessageType: config.MessageType,
+ log: log.New("alerting.notifier.victorops"),
+ images: images,
+ ns: ns,
+ tmpl: t,
+ }
+}
+
+// VictoropsNotifier defines URL property for Victorops REST API
+// and handles notification process by formatting POST body according to
+// Victorops specifications (http://victorops.force.com/knowledgebase/articles/Integration/Alert-Ingestion-API-Documentation/)
+type VictoropsNotifier struct {
+ *Base
+ URL string
+ MessageType string
+ log log.Logger
+ images ImageStore
+ ns notifications.WebhookSender
+ tmpl *template.Template
+}
+
+// Notify sends notification to Victorops via POST to URL endpoint
+func (vn *VictoropsNotifier) Notify(ctx context.Context, as ...*types.Alert) (bool, error) {
+ vn.log.Debug("executing victorops notification", "notification", vn.Name)
+
+ var tmplErr error
+ tmpl, _ := TmplText(ctx, vn.tmpl, as, vn.log, &tmplErr)
+
+ messageType := strings.ToUpper(tmpl(vn.MessageType))
+ if messageType == "" {
+ messageType = victoropsAlertStateCritical
+ }
+ alerts := types.Alerts(as...)
+ if alerts.Status() == model.AlertResolved {
+ messageType = victoropsAlertStateRecovery
+ }
+
+ groupKey, err := notify.ExtractGroupKey(ctx)
+ if err != nil {
+ return false, err
+ }
+
+ bodyJSON := simplejson.New()
+ bodyJSON.Set("message_type", messageType)
+ bodyJSON.Set("entity_id", groupKey.Hash())
+ bodyJSON.Set("entity_display_name", tmpl(DefaultMessageTitleEmbed))
+ bodyJSON.Set("timestamp", time.Now().Unix())
+ bodyJSON.Set("state_message", tmpl(`{{ template "default.message" . }}`))
+ bodyJSON.Set("monitoring_tool", "Grafana v"+setting.BuildVersion)
+
+ _ = withStoredImages(ctx, vn.log, vn.images,
+ func(index int, image ngmodels.Image) error {
+ if image.URL != "" {
+ bodyJSON.Set("image_url", image.URL)
+ return ErrImagesDone
+ }
+ return nil
+ }, as...)
+
+ ruleURL := joinUrlPath(vn.tmpl.ExternalURL.String(), "/alerting/list", vn.log)
+ bodyJSON.Set("alert_url", ruleURL)
+
+ if tmplErr != nil {
+ vn.log.Warn("failed to template VictorOps message", "err", tmplErr.Error())
+ tmplErr = nil
+ }
+
+ u := tmpl(vn.URL)
+ if tmplErr != nil {
+ vn.log.Info("failed to template VictorOps URL", "err", tmplErr.Error(), "fallback", vn.URL)
+ u = vn.URL
+ }
+
+ b, err := bodyJSON.MarshalJSON()
+ if err != nil {
+ return false, err
+ }
+ cmd := &models.SendWebhookSync{
+ Url: u,
+ Body: string(b),
+ }
+
+ if err := vn.ns.SendWebhookSync(ctx, cmd); err != nil {
+ vn.log.Error("Failed to send Victorops notification", "err", err, "webhook", vn.Name)
+ return false, err
+ }
+
+ return true, nil
+}
+
+func (vn *VictoropsNotifier) SendResolved() bool {
+ return !vn.GetDisableResolveMessage()
+}
diff --git a/alerting/channels/victorops_test.go b/alerting/channels/victorops_test.go
new file mode 100644
index 00000000..251618ba
--- /dev/null
+++ b/alerting/channels/victorops_test.go
@@ -0,0 +1,212 @@
+package channels
+
+import (
+ "context"
+ "encoding/json"
+ "net/url"
+ "testing"
+
+ "github.com/prometheus/alertmanager/notify"
+ "github.com/prometheus/alertmanager/types"
+ "github.com/prometheus/common/model"
+ "github.com/stretchr/testify/require"
+
+ "github.com/grafana/grafana/pkg/components/simplejson"
+ "github.com/grafana/grafana/pkg/setting"
+)
+
+func TestVictoropsNotifier(t *testing.T) {
+ tmpl := templateForTests(t)
+
+ images := newFakeImageStore(2)
+
+ externalURL, err := url.Parse("http://localhost")
+ require.NoError(t, err)
+ tmpl.ExternalURL = externalURL
+
+ cases := []struct {
+ name string
+ settings string
+ alerts []*types.Alert
+ expMsg map[string]interface{}
+ expInitError string
+ expMsgError error
+ }{
+ {
+ name: "A single alert with image",
+ settings: `{"url": "http://localhost"}`,
+ alerts: []*types.Alert{
+ {
+ Alert: model.Alert{
+ Labels: model.LabelSet{"alertname": "alert1", "lbl1": "val1"},
+ Annotations: model.LabelSet{"ann1": "annv1", "__dashboardUid__": "abcd", "__panelId__": "efgh", "__alertImageToken__": "test-image-1"},
+ },
+ },
+ },
+ expMsg: map[string]interface{}{
+ "alert_url": "http://localhost/alerting/list",
+ "entity_display_name": "[FIRING:1] (val1)",
+ "entity_id": "6e3538104c14b583da237e9693b76debbc17f0f8058ef20492e5853096cf8733",
+ "image_url": "https://www.example.com/test-image-1.jpg",
+ "message_type": "CRITICAL",
+ "monitoring_tool": "Grafana v" + setting.BuildVersion,
+ "state_message": "**Firing**\n\nValue: [no value]\nLabels:\n - alertname = alert1\n - lbl1 = val1\nAnnotations:\n - ann1 = annv1\nSilence: http://localhost/alerting/silence/new?alertmanager=grafana&matcher=alertname%3Dalert1&matcher=lbl1%3Dval1\nDashboard: http://localhost/d/abcd\nPanel: http://localhost/d/abcd?viewPanel=efgh\n",
+ },
+ expMsgError: nil,
+ }, {
+ name: "Multiple alerts with images",
+ settings: `{"url": "http://localhost"}`,
+ alerts: []*types.Alert{
+ {
+ Alert: model.Alert{
+ Labels: model.LabelSet{"alertname": "alert1", "lbl1": "val1"},
+ Annotations: model.LabelSet{"ann1": "annv1", "__alertImageToken__": "test-image-1"},
+ },
+ }, {
+ Alert: model.Alert{
+ Labels: model.LabelSet{"alertname": "alert1", "lbl1": "val2"},
+ Annotations: model.LabelSet{"ann1": "annv2", "__alertImageToken__": "test-image-2"},
+ },
+ },
+ },
+ expMsg: map[string]interface{}{
+ "alert_url": "http://localhost/alerting/list",
+ "entity_display_name": "[FIRING:2] ",
+ "entity_id": "6e3538104c14b583da237e9693b76debbc17f0f8058ef20492e5853096cf8733",
+ "image_url": "https://www.example.com/test-image-1.jpg",
+ "message_type": "CRITICAL",
+ "monitoring_tool": "Grafana v" + setting.BuildVersion,
+ "state_message": "**Firing**\n\nValue: [no value]\nLabels:\n - alertname = alert1\n - lbl1 = val1\nAnnotations:\n - ann1 = annv1\nSilence: http://localhost/alerting/silence/new?alertmanager=grafana&matcher=alertname%3Dalert1&matcher=lbl1%3Dval1\n\nValue: [no value]\nLabels:\n - alertname = alert1\n - lbl1 = val2\nAnnotations:\n - ann1 = annv2\nSilence: http://localhost/alerting/silence/new?alertmanager=grafana&matcher=alertname%3Dalert1&matcher=lbl1%3Dval2\n",
+ },
+ expMsgError: nil,
+ }, {
+ name: "Custom message",
+ settings: `{"url": "http://localhost", "messageType": "Alerts firing: {{ len .Alerts.Firing }}"}`,
+ alerts: []*types.Alert{
+ {
+ Alert: model.Alert{
+ Labels: model.LabelSet{"alertname": "alert1", "lbl1": "val1"},
+ Annotations: model.LabelSet{"ann1": "annv1"},
+ },
+ }, {
+ Alert: model.Alert{
+ Labels: model.LabelSet{"alertname": "alert1", "lbl1": "val2"},
+ Annotations: model.LabelSet{"ann1": "annv2"},
+ },
+ },
+ },
+ expMsg: map[string]interface{}{
+ "alert_url": "http://localhost/alerting/list",
+ "entity_display_name": "[FIRING:2] ",
+ "entity_id": "6e3538104c14b583da237e9693b76debbc17f0f8058ef20492e5853096cf8733",
+ "message_type": "ALERTS FIRING: 2",
+ "monitoring_tool": "Grafana v" + setting.BuildVersion,
+ "state_message": "**Firing**\n\nValue: [no value]\nLabels:\n - alertname = alert1\n - lbl1 = val1\nAnnotations:\n - ann1 = annv1\nSilence: http://localhost/alerting/silence/new?alertmanager=grafana&matcher=alertname%3Dalert1&matcher=lbl1%3Dval1\n\nValue: [no value]\nLabels:\n - alertname = alert1\n - lbl1 = val2\nAnnotations:\n - ann1 = annv2\nSilence: http://localhost/alerting/silence/new?alertmanager=grafana&matcher=alertname%3Dalert1&matcher=lbl1%3Dval2\n",
+ },
+ expMsgError: nil,
+ }, {
+ name: "Missing field in template",
+ settings: `{"url": "http://localhost", "messageType": "custom template {{ .NotAField }} bad template"}`,
+ alerts: []*types.Alert{
+ {
+ Alert: model.Alert{
+ Labels: model.LabelSet{"alertname": "alert1", "lbl1": "val1"},
+ Annotations: model.LabelSet{"ann1": "annv1"},
+ },
+ }, {
+ Alert: model.Alert{
+ Labels: model.LabelSet{"alertname": "alert1", "lbl1": "val2"},
+ Annotations: model.LabelSet{"ann1": "annv2"},
+ },
+ },
+ },
+ expMsg: map[string]interface{}{
+ "alert_url": "http://localhost/alerting/list",
+ "entity_display_name": "",
+ "entity_id": "6e3538104c14b583da237e9693b76debbc17f0f8058ef20492e5853096cf8733",
+ "message_type": "CUSTOM TEMPLATE ",
+ "monitoring_tool": "Grafana v" + setting.BuildVersion,
+ "state_message": "",
+ },
+ expMsgError: nil,
+ }, {
+ name: "Invalid template",
+ settings: `{"url": "http://localhost", "messageType": "custom template {{ {.NotAField }} bad template"}`,
+ alerts: []*types.Alert{
+ {
+ Alert: model.Alert{
+ Labels: model.LabelSet{"alertname": "alert1", "lbl1": "val1"},
+ Annotations: model.LabelSet{"ann1": "annv1"},
+ },
+ }, {
+ Alert: model.Alert{
+ Labels: model.LabelSet{"alertname": "alert1", "lbl1": "val2"},
+ Annotations: model.LabelSet{"ann1": "annv2"},
+ },
+ },
+ },
+ expMsg: map[string]interface{}{
+ "alert_url": "http://localhost/alerting/list",
+ "entity_display_name": "",
+ "entity_id": "6e3538104c14b583da237e9693b76debbc17f0f8058ef20492e5853096cf8733",
+ "message_type": "CRITICAL",
+ "monitoring_tool": "Grafana v" + setting.BuildVersion,
+ "state_message": "",
+ },
+ expMsgError: nil,
+ }, {
+ name: "Error in initing, no URL",
+ settings: `{}`,
+ expInitError: `could not find victorops url property in settings`,
+ },
+ }
+
+ for _, c := range cases {
+ t.Run(c.name, func(t *testing.T) {
+ settingsJSON, err := simplejson.NewJson([]byte(c.settings))
+ require.NoError(t, err)
+
+ m := &NotificationChannelConfig{
+ Name: "victorops_testing",
+ Type: "victorops",
+ Settings: settingsJSON,
+ }
+
+ webhookSender := mockNotificationService()
+ cfg, err := NewVictorOpsConfig(m)
+ if c.expInitError != "" {
+ require.Error(t, err)
+ require.Equal(t, c.expInitError, err.Error())
+ return
+ }
+ require.NoError(t, err)
+
+ ctx := notify.WithGroupKey(context.Background(), "alertname")
+ ctx = notify.WithGroupLabels(ctx, model.LabelSet{"alertname": ""})
+ pn := NewVictoropsNotifier(cfg, images, webhookSender, tmpl)
+ ok, err := pn.Notify(ctx, c.alerts...)
+ if c.expMsgError != nil {
+ require.False(t, ok)
+ require.Error(t, err)
+ require.Equal(t, c.expMsgError.Error(), err.Error())
+ return
+ }
+ require.NoError(t, err)
+ require.True(t, ok)
+
+ require.NotEmpty(t, webhookSender.Webhook.Url)
+
+ // Remove the non-constant timestamp
+ j, err := simplejson.NewJson([]byte(webhookSender.Webhook.Body))
+ require.NoError(t, err)
+ j.Del("timestamp")
+ b, err := j.MarshalJSON()
+ require.NoError(t, err)
+ body := string(b)
+
+ expJson, err := json.Marshal(c.expMsg)
+ require.NoError(t, err)
+ require.JSONEq(t, string(expJson), body)
+ })
+ }
+}
diff --git a/alerting/channels/webhook.go b/alerting/channels/webhook.go
new file mode 100644
index 00000000..4e92444c
--- /dev/null
+++ b/alerting/channels/webhook.go
@@ -0,0 +1,180 @@
+package channels
+
+import (
+ "context"
+ "encoding/json"
+ "errors"
+
+ "github.com/grafana/grafana/pkg/infra/log"
+ "github.com/grafana/grafana/pkg/models"
+ ngmodels "github.com/grafana/grafana/pkg/services/ngalert/models"
+ "github.com/grafana/grafana/pkg/services/notifications"
+ "github.com/prometheus/alertmanager/notify"
+ "github.com/prometheus/alertmanager/template"
+ "github.com/prometheus/alertmanager/types"
+ "github.com/prometheus/common/model"
+)
+
+// WebhookNotifier is responsible for sending
+// alert notifications as webhooks.
+type WebhookNotifier struct {
+ *Base
+ URL string
+ User string
+ Password string
+ HTTPMethod string
+ MaxAlerts int
+ log log.Logger
+ ns notifications.WebhookSender
+ images ImageStore
+ tmpl *template.Template
+ orgID int64
+}
+
+type WebhookConfig struct {
+ *NotificationChannelConfig
+ URL string
+ User string
+ Password string
+ HTTPMethod string
+ MaxAlerts int
+}
+
+func WebHookFactory(fc FactoryConfig) (NotificationChannel, error) {
+ cfg, err := NewWebHookConfig(fc.Config, fc.DecryptFunc)
+ if err != nil {
+ return nil, receiverInitError{
+ Reason: err.Error(),
+ Cfg: *fc.Config,
+ }
+ }
+ return NewWebHookNotifier(cfg, fc.NotificationService, fc.ImageStore, fc.Template), nil
+}
+
+func NewWebHookConfig(config *NotificationChannelConfig, decryptFunc GetDecryptedValueFn) (*WebhookConfig, error) {
+ url := config.Settings.Get("url").MustString()
+ if url == "" {
+ return nil, errors.New("could not find url property in settings")
+ }
+ return &WebhookConfig{
+ NotificationChannelConfig: config,
+ URL: url,
+ User: config.Settings.Get("username").MustString(),
+ Password: decryptFunc(context.Background(), config.SecureSettings, "password", config.Settings.Get("password").MustString()),
+ HTTPMethod: config.Settings.Get("httpMethod").MustString("POST"),
+ MaxAlerts: config.Settings.Get("maxAlerts").MustInt(0),
+ }, nil
+}
+
+// NewWebHookNotifier is the constructor for
+// the WebHook notifier.
+func NewWebHookNotifier(config *WebhookConfig, ns notifications.WebhookSender, images ImageStore, t *template.Template) *WebhookNotifier {
+ return &WebhookNotifier{
+ Base: NewBase(&models.AlertNotification{
+ Uid: config.UID,
+ Name: config.Name,
+ Type: config.Type,
+ DisableResolveMessage: config.DisableResolveMessage,
+ Settings: config.Settings,
+ }),
+ orgID: config.OrgID,
+ URL: config.URL,
+ User: config.User,
+ Password: config.Password,
+ HTTPMethod: config.HTTPMethod,
+ MaxAlerts: config.MaxAlerts,
+ log: log.New("alerting.notifier.webhook"),
+ ns: ns,
+ images: images,
+ tmpl: t,
+ }
+}
+
+// webhookMessage defines the JSON object send to webhook endpoints.
+type webhookMessage struct {
+ *ExtendedData
+
+ // The protocol version.
+ Version string `json:"version"`
+ GroupKey string `json:"groupKey"`
+ TruncatedAlerts int `json:"truncatedAlerts"`
+ OrgID int64 `json:"orgId"`
+
+ // Deprecated, to be removed in 8.1.
+ // These are present to make migration a little less disruptive.
+ Title string `json:"title"`
+ State string `json:"state"`
+ Message string `json:"message"`
+}
+
+// Notify implements the Notifier interface.
+func (wn *WebhookNotifier) Notify(ctx context.Context, as ...*types.Alert) (bool, error) {
+ groupKey, err := notify.ExtractGroupKey(ctx)
+ if err != nil {
+ return false, err
+ }
+
+ as, numTruncated := truncateAlerts(wn.MaxAlerts, as)
+ var tmplErr error
+ tmpl, data := TmplText(ctx, wn.tmpl, as, wn.log, &tmplErr)
+
+ // Augment our Alert data with ImageURLs if available.
+ _ = withStoredImages(ctx, wn.log, wn.images,
+ func(index int, image ngmodels.Image) error {
+ if len(image.URL) != 0 {
+ data.Alerts[index].ImageURL = image.URL
+ }
+ return nil
+ },
+ as...)
+
+ msg := &webhookMessage{
+ Version: "1",
+ ExtendedData: data,
+ GroupKey: groupKey.String(),
+ TruncatedAlerts: numTruncated,
+ OrgID: wn.orgID,
+ Title: tmpl(DefaultMessageTitleEmbed),
+ Message: tmpl(`{{ template "default.message" . }}`),
+ }
+ if types.Alerts(as...).Status() == model.AlertFiring {
+ msg.State = string(models.AlertStateAlerting)
+ } else {
+ msg.State = string(models.AlertStateOK)
+ }
+
+ if tmplErr != nil {
+ wn.log.Warn("failed to template webhook message", "err", tmplErr.Error())
+ }
+
+ body, err := json.Marshal(msg)
+ if err != nil {
+ return false, err
+ }
+
+ cmd := &models.SendWebhookSync{
+ Url: wn.URL,
+ User: wn.User,
+ Password: wn.Password,
+ Body: string(body),
+ HttpMethod: wn.HTTPMethod,
+ }
+
+ if err := wn.ns.SendWebhookSync(ctx, cmd); err != nil {
+ return false, err
+ }
+
+ return true, nil
+}
+
+func truncateAlerts(maxAlerts int, alerts []*types.Alert) ([]*types.Alert, int) {
+ if maxAlerts > 0 && len(alerts) > maxAlerts {
+ return alerts[:maxAlerts], len(alerts) - maxAlerts
+ }
+
+ return alerts, 0
+}
+
+func (wn *WebhookNotifier) SendResolved() bool {
+ return !wn.GetDisableResolveMessage()
+}
diff --git a/alerting/channels/webhook_test.go b/alerting/channels/webhook_test.go
new file mode 100644
index 00000000..e8b594b8
--- /dev/null
+++ b/alerting/channels/webhook_test.go
@@ -0,0 +1,228 @@
+package channels
+
+import (
+ "context"
+ "encoding/json"
+ "net/url"
+ "testing"
+
+ "github.com/grafana/grafana/pkg/components/simplejson"
+ "github.com/grafana/grafana/pkg/services/secrets/fakes"
+ secretsManager "github.com/grafana/grafana/pkg/services/secrets/manager"
+
+ "github.com/prometheus/alertmanager/notify"
+ "github.com/prometheus/alertmanager/template"
+ "github.com/prometheus/alertmanager/types"
+ "github.com/prometheus/common/model"
+ "github.com/stretchr/testify/require"
+)
+
+func TestWebhookNotifier(t *testing.T) {
+ tmpl := templateForTests(t)
+
+ externalURL, err := url.Parse("http://localhost")
+ require.NoError(t, err)
+ tmpl.ExternalURL = externalURL
+
+ orgID := int64(1)
+
+ cases := []struct {
+ name string
+ settings string
+ alerts []*types.Alert
+ expMsg *webhookMessage
+ expUrl string
+ expUsername string
+ expPassword string
+ expHttpMethod string
+ expInitError string
+ expMsgError error
+ }{
+ {
+ name: "Default config with one alert",
+ settings: `{"url": "http://localhost/test"}`,
+ alerts: []*types.Alert{
+ {
+ Alert: model.Alert{
+ Labels: model.LabelSet{"alertname": "alert1", "lbl1": "val1"},
+ Annotations: model.LabelSet{"ann1": "annv1", "__dashboardUid__": "abcd", "__panelId__": "efgh"},
+ },
+ },
+ },
+ expUrl: "http://localhost/test",
+ expHttpMethod: "POST",
+ expMsg: &webhookMessage{
+ ExtendedData: &ExtendedData{
+ Receiver: "my_receiver",
+ Status: "firing",
+ Alerts: ExtendedAlerts{
+ {
+ Status: "firing",
+ Labels: template.KV{
+ "alertname": "alert1",
+ "lbl1": "val1",
+ },
+ Annotations: template.KV{
+ "ann1": "annv1",
+ },
+ Fingerprint: "fac0861a85de433a",
+ DashboardURL: "http://localhost/d/abcd",
+ PanelURL: "http://localhost/d/abcd?viewPanel=efgh",
+ SilenceURL: "http://localhost/alerting/silence/new?alertmanager=grafana&matcher=alertname%3Dalert1&matcher=lbl1%3Dval1",
+ },
+ },
+ GroupLabels: template.KV{
+ "alertname": "",
+ },
+ CommonLabels: template.KV{
+ "alertname": "alert1",
+ "lbl1": "val1",
+ },
+ CommonAnnotations: template.KV{
+ "ann1": "annv1",
+ },
+ ExternalURL: "http://localhost",
+ },
+ Version: "1",
+ GroupKey: "alertname",
+ Title: "[FIRING:1] (val1)",
+ State: "alerting",
+ Message: "**Firing**\n\nValue: [no value]\nLabels:\n - alertname = alert1\n - lbl1 = val1\nAnnotations:\n - ann1 = annv1\nSilence: http://localhost/alerting/silence/new?alertmanager=grafana&matcher=alertname%3Dalert1&matcher=lbl1%3Dval1\nDashboard: http://localhost/d/abcd\nPanel: http://localhost/d/abcd?viewPanel=efgh\n",
+ OrgID: orgID,
+ },
+ expMsgError: nil,
+ }, {
+ name: "Custom config with multiple alerts",
+ settings: `{
+ "url": "http://localhost/test1",
+ "username": "user1",
+ "password": "mysecret",
+ "httpMethod": "PUT",
+ "maxAlerts": 2
+ }`,
+ alerts: []*types.Alert{
+ {
+ Alert: model.Alert{
+ Labels: model.LabelSet{"alertname": "alert1", "lbl1": "val1"},
+ Annotations: model.LabelSet{"ann1": "annv1"},
+ },
+ }, {
+ Alert: model.Alert{
+ Labels: model.LabelSet{"alertname": "alert1", "lbl1": "val2"},
+ Annotations: model.LabelSet{"ann1": "annv2"},
+ },
+ }, {
+ Alert: model.Alert{
+ Labels: model.LabelSet{"alertname": "alert1", "lbl1": "val3"},
+ Annotations: model.LabelSet{"ann1": "annv3"},
+ },
+ },
+ },
+ expUrl: "http://localhost/test1",
+ expHttpMethod: "PUT",
+ expUsername: "user1",
+ expPassword: "mysecret",
+ expMsg: &webhookMessage{
+ ExtendedData: &ExtendedData{
+ Receiver: "my_receiver",
+ Status: "firing",
+ Alerts: ExtendedAlerts{
+ {
+ Status: "firing",
+ Labels: template.KV{
+ "alertname": "alert1",
+ "lbl1": "val1",
+ },
+ Annotations: template.KV{
+ "ann1": "annv1",
+ },
+ Fingerprint: "fac0861a85de433a",
+ SilenceURL: "http://localhost/alerting/silence/new?alertmanager=grafana&matcher=alertname%3Dalert1&matcher=lbl1%3Dval1",
+ }, {
+ Status: "firing",
+ Labels: template.KV{
+ "alertname": "alert1",
+ "lbl1": "val2",
+ },
+ Annotations: template.KV{
+ "ann1": "annv2",
+ },
+ Fingerprint: "fab6861a85d5eeb5",
+ SilenceURL: "http://localhost/alerting/silence/new?alertmanager=grafana&matcher=alertname%3Dalert1&matcher=lbl1%3Dval2",
+ },
+ },
+ GroupLabels: template.KV{
+ "alertname": "",
+ },
+ CommonLabels: template.KV{
+ "alertname": "alert1",
+ },
+ CommonAnnotations: template.KV{},
+ ExternalURL: "http://localhost",
+ },
+ Version: "1",
+ GroupKey: "alertname",
+ TruncatedAlerts: 1,
+ Title: "[FIRING:2] ",
+ State: "alerting",
+ Message: "**Firing**\n\nValue: [no value]\nLabels:\n - alertname = alert1\n - lbl1 = val1\nAnnotations:\n - ann1 = annv1\nSilence: http://localhost/alerting/silence/new?alertmanager=grafana&matcher=alertname%3Dalert1&matcher=lbl1%3Dval1\n\nValue: [no value]\nLabels:\n - alertname = alert1\n - lbl1 = val2\nAnnotations:\n - ann1 = annv2\nSilence: http://localhost/alerting/silence/new?alertmanager=grafana&matcher=alertname%3Dalert1&matcher=lbl1%3Dval2\n",
+ OrgID: orgID,
+ },
+ expMsgError: nil,
+ }, {
+ name: "Error in initing",
+ settings: `{}`,
+ expInitError: `could not find url property in settings`,
+ },
+ }
+
+ for _, c := range cases {
+ t.Run(c.name, func(t *testing.T) {
+ settingsJSON, err := simplejson.NewJson([]byte(c.settings))
+ require.NoError(t, err)
+ secureSettings := make(map[string][]byte)
+
+ m := &NotificationChannelConfig{
+ OrgID: orgID,
+ Name: "webhook_testing",
+ Type: "webhook",
+ Settings: settingsJSON,
+ SecureSettings: secureSettings,
+ }
+
+ webhookSender := mockNotificationService()
+ secretsService := secretsManager.SetupTestService(t, fakes.NewFakeSecretsStore())
+ decryptFn := secretsService.GetDecryptedValue
+ cfg, err := NewWebHookConfig(m, decryptFn)
+ if c.expInitError != "" {
+ require.Error(t, err)
+ require.Equal(t, c.expInitError, err.Error())
+ return
+ }
+ require.NoError(t, err)
+
+ ctx := notify.WithGroupKey(context.Background(), "alertname")
+ ctx = notify.WithGroupLabels(ctx, model.LabelSet{"alertname": ""})
+ ctx = notify.WithReceiverName(ctx, "my_receiver")
+ pn := NewWebHookNotifier(cfg, webhookSender, &UnavailableImageStore{}, tmpl)
+ ok, err := pn.Notify(ctx, c.alerts...)
+ if c.expMsgError != nil {
+ require.False(t, ok)
+ require.Error(t, err)
+ require.Equal(t, c.expMsgError.Error(), err.Error())
+ return
+ }
+ require.NoError(t, err)
+ require.True(t, ok)
+
+ expBody, err := json.Marshal(c.expMsg)
+ require.NoError(t, err)
+
+ require.JSONEq(t, string(expBody), webhookSender.Webhook.Body)
+ require.Equal(t, c.expUrl, webhookSender.Webhook.Url)
+ require.Equal(t, c.expUsername, webhookSender.Webhook.User)
+ require.Equal(t, c.expPassword, webhookSender.Webhook.Password)
+ require.Equal(t, c.expHttpMethod, webhookSender.Webhook.HttpMethod)
+ })
+ }
+}
diff --git a/alerting/channels/wecom.go b/alerting/channels/wecom.go
new file mode 100644
index 00000000..4d25c1b8
--- /dev/null
+++ b/alerting/channels/wecom.go
@@ -0,0 +1,117 @@
+package channels
+
+import (
+ "context"
+ "encoding/json"
+ "errors"
+ "fmt"
+
+ "github.com/prometheus/alertmanager/template"
+ "github.com/prometheus/alertmanager/types"
+
+ "github.com/grafana/grafana/pkg/infra/log"
+ "github.com/grafana/grafana/pkg/models"
+ "github.com/grafana/grafana/pkg/services/notifications"
+)
+
+type WeComConfig struct {
+ *NotificationChannelConfig
+ URL string
+ Message string
+ Title string
+}
+
+func WeComFactory(fc FactoryConfig) (NotificationChannel, error) {
+ cfg, err := NewWeComConfig(fc.Config, fc.DecryptFunc)
+ if err != nil {
+ return nil, receiverInitError{
+ Reason: err.Error(),
+ Cfg: *fc.Config,
+ }
+ }
+ return NewWeComNotifier(cfg, fc.NotificationService, fc.Template), nil
+}
+
+func NewWeComConfig(config *NotificationChannelConfig, decryptFunc GetDecryptedValueFn) (*WeComConfig, error) {
+ url := decryptFunc(context.Background(), config.SecureSettings, "url", config.Settings.Get("url").MustString())
+ if url == "" {
+ return nil, errors.New("could not find webhook URL in settings")
+ }
+ return &WeComConfig{
+ NotificationChannelConfig: config,
+ URL: url,
+ Message: config.Settings.Get("message").MustString(`{{ template "default.message" .}}`),
+ Title: config.Settings.Get("title").MustString(DefaultMessageTitleEmbed),
+ }, nil
+}
+
+// NewWeComNotifier is the constructor for WeCom notifier.
+func NewWeComNotifier(config *WeComConfig, ns notifications.WebhookSender, t *template.Template) *WeComNotifier {
+ return &WeComNotifier{
+ Base: NewBase(&models.AlertNotification{
+ Uid: config.UID,
+ Name: config.Name,
+ Type: config.Type,
+ DisableResolveMessage: config.DisableResolveMessage,
+ Settings: config.Settings,
+ }),
+ URL: config.URL,
+ Message: config.Message,
+ Title: config.Title,
+ log: log.New("alerting.notifier.wecom"),
+ ns: ns,
+ tmpl: t,
+ }
+}
+
+// WeComNotifier is responsible for sending alert notifications to WeCom.
+type WeComNotifier struct {
+ *Base
+ URL string
+ Message string
+ Title string
+ tmpl *template.Template
+ log log.Logger
+ ns notifications.WebhookSender
+}
+
+// Notify send an alert notification to WeCom.
+func (w *WeComNotifier) Notify(ctx context.Context, as ...*types.Alert) (bool, error) {
+ w.log.Info("executing WeCom notification", "notification", w.Name)
+
+ var tmplErr error
+ tmpl, _ := TmplText(ctx, w.tmpl, as, w.log, &tmplErr)
+
+ bodyMsg := map[string]interface{}{
+ "msgtype": "markdown",
+ }
+ content := fmt.Sprintf("# %s\n%s\n",
+ tmpl(w.Title),
+ tmpl(w.Message),
+ )
+
+ bodyMsg["markdown"] = map[string]interface{}{
+ "content": content,
+ }
+
+ body, err := json.Marshal(bodyMsg)
+ if err != nil {
+ return false, err
+ }
+
+ cmd := &models.SendWebhookSync{
+ Url: w.URL,
+ Body: string(body),
+ }
+
+ if err := w.ns.SendWebhookSync(ctx, cmd); err != nil {
+ w.log.Error("failed to send WeCom webhook", "err", err, "notification", w.Name)
+ return false, err
+ }
+
+ return true, nil
+}
+
+func (w *WeComNotifier) SendResolved() bool {
+ return !w.GetDisableResolveMessage()
+}
diff --git a/alerting/channels/wecom_test.go b/alerting/channels/wecom_test.go
new file mode 100644
index 00000000..d5343f36
--- /dev/null
+++ b/alerting/channels/wecom_test.go
@@ -0,0 +1,153 @@
+package channels
+
+import (
+ "context"
+ "encoding/json"
+ "net/url"
+ "testing"
+
+ "github.com/grafana/grafana/pkg/services/secrets/fakes"
+ secretsManager "github.com/grafana/grafana/pkg/services/secrets/manager"
+
+ "github.com/prometheus/alertmanager/notify"
+ "github.com/prometheus/alertmanager/types"
+ "github.com/prometheus/common/model"
+ "github.com/stretchr/testify/require"
+
+ "github.com/grafana/grafana/pkg/components/simplejson"
+)
+
+func TestWeComNotifier(t *testing.T) {
+ tmpl := templateForTests(t)
+
+ externalURL, err := url.Parse("http://localhost")
+ require.NoError(t, err)
+ tmpl.ExternalURL = externalURL
+
+ cases := []struct {
+ name string
+ settings string
+ alerts []*types.Alert
+ expMsg map[string]interface{}
+ expInitError string
+ expMsgError error
+ }{
+ {
+ name: "Default config with one alert",
+ settings: `{"url": "http://localhost"}`,
+ alerts: []*types.Alert{
+ {
+ Alert: model.Alert{
+ Labels: model.LabelSet{"alertname": "alert1", "lbl1": "val1"},
+ Annotations: model.LabelSet{"ann1": "annv1", "__dashboardUid__": "abcd", "__panelId__": "efgh"},
+ },
+ },
+ },
+ expMsg: map[string]interface{}{
+ "markdown": map[string]interface{}{
+ "content": "# [FIRING:1] (val1)\n**Firing**\n\nValue: [no value]\nLabels:\n - alertname = alert1\n - lbl1 = val1\nAnnotations:\n - ann1 = annv1\nSilence: http://localhost/alerting/silence/new?alertmanager=grafana&matcher=alertname%3Dalert1&matcher=lbl1%3Dval1\nDashboard: http://localhost/d/abcd\nPanel: http://localhost/d/abcd?viewPanel=efgh\n\n",
+ },
+ "msgtype": "markdown",
+ },
+ expMsgError: nil,
+ }, {
+ name: "Custom config with multiple alerts",
+ settings: `{
+ "url": "http://localhost",
+ "message": "{{ len .Alerts.Firing }} alerts are firing, {{ len .Alerts.Resolved }} are resolved"
+ }`,
+ alerts: []*types.Alert{
+ {
+ Alert: model.Alert{
+ Labels: model.LabelSet{"alertname": "alert1", "lbl1": "val1"},
+ Annotations: model.LabelSet{"ann1": "annv1"},
+ },
+ }, {
+ Alert: model.Alert{
+ Labels: model.LabelSet{"alertname": "alert1", "lbl1": "val2"},
+ Annotations: model.LabelSet{"ann1": "annv2"},
+ },
+ },
+ },
+ expMsg: map[string]interface{}{
+ "markdown": map[string]interface{}{
+ "content": "# [FIRING:2] \n2 alerts are firing, 0 are resolved\n",
+ },
+ "msgtype": "markdown",
+ },
+ expMsgError: nil,
+ }, {
+ name: "Custom title and message with multiple alerts",
+ settings: `{
+ "url": "http://localhost",
+ "message": "{{ len .Alerts.Firing }} alerts are firing, {{ len .Alerts.Resolved }} are resolved",
+ "title": "This notification is {{ .Status }}!"
+ }`,
+ alerts: []*types.Alert{
+ {
+ Alert: model.Alert{
+ Labels: model.LabelSet{"alertname": "alert1", "lbl1": "val1"},
+ Annotations: model.LabelSet{"ann1": "annv1"},
+ },
+ }, {
+ Alert: model.Alert{
+ Labels: model.LabelSet{"alertname": "alert1", "lbl1": "val2"},
+ Annotations: model.LabelSet{"ann1": "annv2"},
+ },
+ },
+ },
+ expMsg: map[string]interface{}{
+ "markdown": map[string]interface{}{
+ "content": "# This notification is firing!\n2 alerts are firing, 0 are resolved\n",
+ },
+ "msgtype": "markdown",
+ },
+ expMsgError: nil,
+ }, {
+ name: "Error in initing",
+ settings: `{}`,
+ expInitError: `could not find webhook URL in settings`,
+ },
+ }
+
+ for _, c := range cases {
+ t.Run(c.name, func(t *testing.T) {
+ settingsJSON, err := simplejson.NewJson([]byte(c.settings))
+ require.NoError(t, err)
+
+ m := &NotificationChannelConfig{
+ Name: "wecom_testing",
+ Type: "wecom",
+ Settings: settingsJSON,
+ }
+
+ webhookSender := mockNotificationService()
+ secretsService := secretsManager.SetupTestService(t, fakes.NewFakeSecretsStore())
+ decryptFn := secretsService.GetDecryptedValue
+ cfg, err := NewWeComConfig(m, decryptFn)
+ if c.expInitError != "" {
+ require.Equal(t, c.expInitError, err.Error())
+ return
+ }
+ require.NoError(t, err)
+
+ ctx := notify.WithGroupKey(context.Background(), "alertname")
+ ctx = notify.WithGroupLabels(ctx, model.LabelSet{"alertname": ""})
+ pn := NewWeComNotifier(cfg, webhookSender, tmpl)
+ ok, err := pn.Notify(ctx, c.alerts...)
+ if c.expMsgError != nil {
+ require.False(t, ok)
+ require.Error(t, err)
+ require.Equal(t, c.expMsgError.Error(), err.Error())
+ return
+ }
+ require.NoError(t, err)
+ require.True(t, ok)
+
+ expBody, err := json.Marshal(c.expMsg)
+ require.NoError(t, err)
+
+ require.JSONEq(t, string(expBody), webhookSender.Webhook.Body)
+ })
+ }
+}
diff --git a/alerting/config.go b/alerting/config.go
new file mode 100644
index 00000000..1879773b
--- /dev/null
+++ b/alerting/config.go
@@ -0,0 +1,87 @@
+package notifier
+
+import (
+ "encoding/json"
+ "fmt"
+ "io/ioutil"
+ "os"
+ "path/filepath"
+
+ "github.com/grafana/grafana/pkg/infra/log"
+ api "github.com/grafana/grafana/pkg/services/ngalert/api/tooling/definitions"
+)
+
+var cfglogger = log.New("notifier.config")
+
+func PersistTemplates(cfg *api.PostableUserConfig, path string) ([]string, bool, error) {
+ if len(cfg.TemplateFiles) < 1 {
+ return nil, false, nil
+ }
+
+ var templatesChanged bool
+ pathSet := map[string]struct{}{}
+ for name, content := range cfg.TemplateFiles {
+ if name != filepath.Base(filepath.Clean(name)) {
+ return nil, false, fmt.Errorf("template file name '%s' is not valid", name)
+ }
+
+ err := os.MkdirAll(path, 0750)
+ if err != nil {
+ return nil, false, fmt.Errorf("unable to create template directory %q: %s", path, err)
+ }
+
+ file := filepath.Join(path, name)
+ pathSet[file] = struct{}{}
+
+ // Check if the template file already exists and if it has changed
+ // We can safely ignore gosec here as we've previously checked the filename is clean
+ // nolint:gosec
+ if tmpl, err := ioutil.ReadFile(file); err == nil && string(tmpl) == content {
+ // Templates file is the same we have, no-op and continue.
+ continue
+ } else if err != nil && !os.IsNotExist(err) {
+ return nil, false, err
+ }
+
+ // We can safely ignore gosec here as we've previously checked the filename is clean
+ // nolint:gosec
+ if err := ioutil.WriteFile(file, []byte(content), 0644); err != nil {
+ return nil, false, fmt.Errorf("unable to create Alertmanager template file %q: %s", file, err)
+ }
+
+ templatesChanged = true
+ }
+
+ // Now that we have the list of _actual_ templates, let's remove the ones that we don't need.
+ existingFiles, err := ioutil.ReadDir(path)
+ if err != nil {
+ cfglogger.Error("unable to read directory for deleting Alertmanager templates", "err", err, "path", path)
+ }
+ for _, existingFile := range existingFiles {
+ p := filepath.Join(path, existingFile.Name())
+ _, ok := pathSet[p]
+ if !ok {
+ templatesChanged = true
+ err := os.Remove(p)
+ if err != nil {
+ cfglogger.Error("unable to delete template", "err", err, "file", p)
+ }
+ }
+ }
+
+ paths := make([]string, 0, len(pathSet))
+ for path := range pathSet {
+ paths = append(paths, path)
+ }
+ return paths, templatesChanged, nil
+}
+
+func Load(rawConfig []byte) (*api.PostableUserConfig, error) {
+ cfg := &api.PostableUserConfig{}
+
+ if err := json.Unmarshal(rawConfig, cfg); err != nil {
+ return nil, fmt.Errorf("unable to parse Alertmanager configuration: %w", err)
+ }
+
+ return cfg, nil
+}
diff --git a/alerting/config_test.go b/alerting/config_test.go
new file mode 100644
index 00000000..a987dfc8
--- /dev/null
+++ b/alerting/config_test.go
@@ -0,0 +1,163 @@
+package notifier
+
+import (
+ "errors"
+ "io/ioutil"
+ "path/filepath"
+ "testing"
+
+ api "github.com/grafana/grafana/pkg/services/ngalert/api/tooling/definitions"
+ "github.com/stretchr/testify/assert"
+ "github.com/stretchr/testify/require"
+)
+
+func TestPersistTemplates(t *testing.T) {
+ tc := []struct {
+ name string
+ templates map[string]string
+ existingTemplates map[string]string
+ expectedPaths []string
+ expectedError error
+ expectedChange bool
+ }{
+ {
+ name: "With valid templates file names, it persists successfully",
+ templates: map[string]string{"email.template": "a perfectly fine template"},
+ expectedChange: true,
+ expectedError: nil,
+ expectedPaths: []string{"email.template"},
+ },
+ {
+ name: "With a invalid filename, it fails",
+ templates: map[string]string{"adirectory/email.template": "a perfectly fine template"},
+ expectedError: errors.New("template file name 'adirectory/email.template' is not valid"),
+ },
+ {
+ name: "with a template that has the same name but different content to an existing one",
+ existingTemplates: map[string]string{"email.template": "a perfectly fine template"},
+ templates: map[string]string{"email.template": "a completely different content"},
+ expectedChange: true,
+ expectedError: nil,
+ expectedPaths: []string{"email.template"},
+ },
+ {
+ name: "with a template that has the same name and the same content as an existing one",
+ existingTemplates: map[string]string{"email.template": "a perfectly fine template"},
+ templates: map[string]string{"email.template": "a perfectly fine template"},
+ expectedChange: false,
+ expectedError: nil,
+ expectedPaths: []string{"email.template"},
+ },
+ {
+ name: "with two new template files, it changes the template tree",
+ existingTemplates: map[string]string{"email.template": "a perfectly fine template"},
+ templates: map[string]string{"slack.template": "a perfectly fine template", "webhook.template": "a webhook template"},
+ expectedChange: true,
+ expectedError: nil,
+ expectedPaths: []string{"slack.template", "webhook.template"},
+ },
+ {
+ name: "when we remove a template file from the list, it changes the template tree",
+ existingTemplates: map[string]string{"slack.template": "a perfectly fine template", "webhook.template": "a webhook template"},
+ templates: map[string]string{"slack.template": "a perfectly fine template"},
+ expectedChange: true,
+ expectedError: nil,
+ expectedPaths: []string{"slack.template"},
+ },
+ }
+
+ for _, tt := range tc {
+ t.Run(tt.name, func(t *testing.T) {
+ dir := t.TempDir()
+ // Write "existing files"
+ for name, content := range tt.existingTemplates {
+ err := ioutil.WriteFile(filepath.Join(dir, name), []byte(content), 0644)
+ require.NoError(t, err)
+ }
+ c := &api.PostableUserConfig{TemplateFiles: tt.templates}
+
+ paths, changed, persistErr := PersistTemplates(c, dir)
+
+ files := map[string]string{}
+ readFiles, err := ioutil.ReadDir(dir)
+ require.NoError(t, err)
+ for _, f := range readFiles {
+ if f.IsDir() || f.Name() == "" {
+ continue
+ }
+ // Safe to disable, this is a test.
+ // nolint:gosec
+ content, err := ioutil.ReadFile(filepath.Join(dir, f.Name()))
+ // nolint:gosec
+ require.NoError(t, err)
+ files[f.Name()] = string(content)
+ }
+
+ // Given we use a temporary directory in tests, we need to prepend the expected paths with it.
+ for i, p := range tt.expectedPaths {
+ tt.expectedPaths[i] = filepath.Join(dir, p)
+ }
+
+ require.Equal(t, tt.expectedError, persistErr)
+ require.ElementsMatch(t, tt.expectedPaths, paths)
+ require.Equal(t, tt.expectedChange, changed)
+ if tt.expectedError == nil {
+ require.Equal(t, tt.templates, files)
+ }
+ })
+ }
+}
+
+func TestLoad(t *testing.T) {
+ tc := []struct {
+ name string
+ rawConfig string
+ expectedTemplates map[string]string
+ expectedError error
+ }{
+ {
+ name: "with a valid config and template",
+ rawConfig: `
+{
+ "alertmanager_config": {
+ "global": {
+ "smtp_from": "noreply@grafana.net"
+ },
+ "route": {
+ "receiver": "email"
+ },
+ "receivers": [
+ {
+ "name": "email"
+ }
+ ]
+ },
+ "template_files": {
+ "email.template": "something with a pretty good content"
+ }
+}
+`,
+ expectedTemplates: map[string]string{"email.template": "something with a pretty good content"},
+ },
+ {
+ name: "with an empty configuration, it is not valid.",
+ rawConfig: "{}",
+ expectedError: errors.New("unable to parse Alertmanager configuration: no route provided in config"),
+ },
+ }
+
+ for _, tt := range tc {
+ t.Run(tt.name, func(t *testing.T) {
+ c, err := Load([]byte(tt.rawConfig))
+
+ if tt.expectedError != nil {
+ assert.Nil(t, c)
+ assert.Equal(t, tt.expectedError.Error(), err.Error())
+ } else {
+ require.NoError(t, err)
+ assert.NotNil(t, c.TemplateFiles)
+ assert.Equal(t, tt.expectedTemplates, c.TemplateFiles)
+ }
+ })
+ }
+}
diff --git a/alerting/crypto.go b/alerting/crypto.go
new file mode 100644
index 00000000..97a97525
--- /dev/null
+++ b/alerting/crypto.go
@@ -0,0 +1,118 @@
+package notifier
+
+import (
+ "context"
+ "encoding/base64"
+ "errors"
+ "fmt"
+
+ "github.com/grafana/grafana/pkg/infra/log"
+ "github.com/grafana/grafana/pkg/services/ngalert/api/tooling/definitions"
+ "github.com/grafana/grafana/pkg/services/ngalert/models"
+ "github.com/grafana/grafana/pkg/services/ngalert/store"
+ "github.com/grafana/grafana/pkg/services/secrets"
+)
+
+// Crypto allows decryption of Alertmanager Configuration and encryption of arbitrary payloads.
+type Crypto interface {
+ LoadSecureSettings(ctx context.Context, orgId int64, receivers []*definitions.PostableApiReceiver) error
+ Encrypt(ctx context.Context, payload []byte, opt secrets.EncryptionOptions) ([]byte, error)
+
+ getDecryptedSecret(r *definitions.PostableGrafanaReceiver, key string) (string, error)
+}
+
+// alertmanagerCrypto implements decryption of Alertmanager configuration and encryption of arbitrary payloads based on Grafana's encryptions.
+type alertmanagerCrypto struct {
+ secrets secrets.Service
+ configs configurationStore
+ log log.Logger
+}
+
+func NewCrypto(secrets secrets.Service, configs configurationStore, log log.Logger) Crypto {
+ return &alertmanagerCrypto{
+ secrets: secrets,
+ configs: configs,
+ log: log,
+ }
+}
+
+// LoadSecureSettings adds the corresponding unencrypted secrets stored to the list of input receivers.
+func (c *alertmanagerCrypto) LoadSecureSettings(ctx context.Context, orgId int64, receivers []*definitions.PostableApiReceiver) error {
+ // Get the last known working configuration.
+ query := models.GetLatestAlertmanagerConfigurationQuery{OrgID: orgId}
+ if err := c.configs.GetLatestAlertmanagerConfiguration(ctx, &query); err != nil {
+ // If we don't have a configuration there's nothing for us to know and we should just continue saving the new one.
+ if !errors.Is(err, store.ErrNoAlertmanagerConfiguration) {
+ return fmt.Errorf("failed to get latest configuration: %w", err)
+ }
+ }
+
+ currentReceiverMap := make(map[string]*definitions.PostableGrafanaReceiver)
+ if query.Result != nil {
+ currentConfig, err := Load([]byte(query.Result.AlertmanagerConfiguration))
+ // If the current config is un-loadable, treat it as if it never existed. Providing a new, valid config should be able to "fix" this state.
+ if err != nil {
+ c.log.Warn("last known alertmanager configuration was invalid. Overwriting...")
+ } else {
+ currentReceiverMap = currentConfig.GetGrafanaReceiverMap()
+ }
+ }
+
+ // Copy the previously known secure settings.
+ for i, r := range receivers {
+ for j, gr := range r.PostableGrafanaReceivers.GrafanaManagedReceivers {
+ if gr.UID == "" { // new receiver
+ continue
+ }
+
+ cgmr, ok := currentReceiverMap[gr.UID]
+ if !ok {
+ // It tries to update a receiver that didn't previously exist
+ return UnknownReceiverError{UID: gr.UID}
+ }
+
+ // Frontend sends only the secure settings that have to be updated
+ // Therefore we have to copy from the last configuration only those secure settings not included in the request
+ for key := range cgmr.SecureSettings {
+ _, ok := gr.SecureSettings[key]
+ if !ok {
+ decryptedValue, err := c.getDecryptedSecret(cgmr, key)
+ if err != nil {
+ return fmt.Errorf("failed to decrypt stored secure setting: %s: %w", key, err)
+ }
+
+ if receivers[i].PostableGrafanaReceivers.GrafanaManagedReceivers[j].SecureSettings == nil {
+ receivers[i].PostableGrafanaReceivers.GrafanaManagedReceivers[j].SecureSettings = make(map[string]string, len(cgmr.SecureSettings))
+ }
+
+ receivers[i].PostableGrafanaReceivers.GrafanaManagedReceivers[j].SecureSettings[key] = decryptedValue
+ }
+ }
+ }
+ }
+ return nil
+}
+
+func (c *alertmanagerCrypto) getDecryptedSecret(r *definitions.PostableGrafanaReceiver, key string) (string, error) {
+ storedValue, ok := r.SecureSettings[key]
+ if !ok {
+ return "", nil
+ }
+
+ decodeValue, err := base64.StdEncoding.DecodeString(storedValue)
+ if err != nil {
+ return "", err
+ }
+
+ decryptedValue, err := c.secrets.Decrypt(context.Background(), decodeValue)
+ if err != nil {
+ return "", err
+ }
+
+ return string(decryptedValue), nil
+}
+
+// Encrypt delegates encryption to secrets.Service.
+func (c *alertmanagerCrypto) Encrypt(ctx context.Context, payload []byte, opt secrets.EncryptionOptions) ([]byte, error) {
+ return c.secrets.Encrypt(ctx, payload, opt)
+}
diff --git a/alerting/file_store.go b/alerting/file_store.go
new file mode 100644
index 00000000..42ec6e20
--- /dev/null
+++ b/alerting/file_store.go
@@ -0,0 +1,116 @@
+package notifier
+
+import (
+ "context"
+ "encoding/base64"
+ "fmt"
+ "os"
+ "path/filepath"
+
+ "github.com/grafana/grafana/pkg/infra/kvstore"
+ "github.com/grafana/grafana/pkg/infra/log"
+)
+
+const KVNamespace = "alertmanager"
+
+// State represents any of the two 'states' of the alertmanager. Notification log or Silences.
+// MarshalBinary returns the binary representation of this internal state based on the protobuf.
+type State interface {
+ MarshalBinary() ([]byte, error)
+}
+
+// FileStore is in charge of persisting the alertmanager files to the database.
+// It uses the KVstore table and encodes the files as a base64 string.
+type FileStore struct {
+ kv *kvstore.NamespacedKVStore
+ orgID int64
+ workingDirPath string
+ logger log.Logger
+}
+
+func NewFileStore(orgID int64, store kvstore.KVStore, workingDirPath string) *FileStore {
+ return &FileStore{
+ workingDirPath: workingDirPath,
+ orgID: orgID,
+ kv: kvstore.WithNamespace(store, orgID, KVNamespace),
+ logger: log.New("filestore", "org", orgID),
+ }
+}
+
+// FilepathFor returns the filepath to an Alertmanager file.
+// If the file is already present on disk it no-ops.
+// If not, it tries to read the database and if there's no file it no-ops.
+// If there is a file in the database, it decodes it and writes to disk for Alertmanager consumption.
+func (fileStore *FileStore) FilepathFor(ctx context.Context, filename string) (string, error) {
+ // Then, let's attempt to read it from the database.
+ content, exists, err := fileStore.kv.Get(ctx, filename)
+ if err != nil {
+ return "", fmt.Errorf("error reading file '%s' from database: %w", filename, err)
+ }
+
+ // if it doesn't exist, let's no-op and let the Alertmanager create one. We'll eventually save it to the database.
+ if !exists {
+ return fileStore.pathFor(filename), nil
+ }
+
+ // If we have a file stored in the database, let's decode it and write it to disk to perform that initial load to memory.
+ bytes, err := decode(content)
+ if err != nil {
+ return "", fmt.Errorf("error decoding file '%s': %w", filename, err)
+ }
+
+ if err := fileStore.WriteFileToDisk(filename, bytes); err != nil {
+ return "", fmt.Errorf("error writing file %s: %w", filename, err)
+ }
+
+ return fileStore.pathFor(filename), err
+}
+
+// Persist takes care of persisting the binary representation of internal state to the database as a base64 encoded string.
+func (fileStore *FileStore) Persist(ctx context.Context, filename string, st State) (int64, error) {
+ var size int64
+
+ bytes, err := st.MarshalBinary()
+ if err != nil {
+ return size, err
+ }
+
+ if err = fileStore.kv.Set(ctx, filename, encode(bytes)); err != nil {
+ return size, err
+ }
+
+ return int64(len(bytes)), err
+}
+
+// WriteFileToDisk writes a file with the provided name and contents to the Alertmanager working directory with the default grafana permission.
+func (fileStore *FileStore) WriteFileToDisk(fn string, content []byte) error {
+ // Ensure the working directory is created
+ err := os.MkdirAll(fileStore.workingDirPath, 0750)
+ if err != nil {
+ return fmt.Errorf("unable to create the working directory %q: %s", fileStore.workingDirPath, err)
+ }
+
+ return os.WriteFile(fileStore.pathFor(fn), content, 0644)
+}
+
+// CleanUp will remove the working directory from disk.
+func (fileStore *FileStore) CleanUp() {
+ if err := os.RemoveAll(fileStore.workingDirPath); err != nil {
+ fileStore.logger.Warn("unable to delete the local working directory", "dir", fileStore.workingDirPath,
+ "err", err)
+ return
+ }
+ fileStore.logger.Info("successfully deleted working directory", "dir", fileStore.workingDirPath)
+}
+
+func (fileStore *FileStore) pathFor(fn string) string {
+ return filepath.Join(fileStore.workingDirPath, fn)
+}
+
+func decode(s string) ([]byte, error) {
+ return base64.StdEncoding.DecodeString(s)
+}
+
+func encode(b []byte) string {
+ return base64.StdEncoding.EncodeToString(b)
+}
diff --git a/alerting/file_store_test.go b/alerting/file_store_test.go
new file mode 100644
index 00000000..b6268a6d
--- /dev/null
+++ b/alerting/file_store_test.go
@@ -0,0 +1,95 @@
+package notifier
+
+import (
+ "context"
+ "io/ioutil"
+ "os"
+ "path/filepath"
+ "testing"
+
+ "github.com/stretchr/testify/require"
+)
+
+func TestFileStore_FilepathFor_DirectoryNotExist(t *testing.T) {
+ store := NewFakeKVStore(t)
+ workingDir := filepath.Join(t.TempDir(), "notexistdir")
+ fs := NewFileStore(1, store, workingDir)
+ filekey := "silences"
+ filePath := filepath.Join(workingDir, filekey)
+
+ // With a file already on the database and the path does not exist yet, it creates the path,
+ // writes the file to disk, then returns the filepath.
+ {
+ require.NoError(t, store.Set(context.Background(), 1, KVNamespace, filekey, encode([]byte("silence1,silence3"))))
+ r, err := fs.FilepathFor(context.Background(), filekey)
+ require.NoError(t, err)
+ require.Equal(t, filePath, r)
+ f, err := ioutil.ReadFile(filepath.Clean(filePath))
+ require.NoError(t, err)
+ require.Equal(t, "silence1,silence3", string(f))
+ require.NoError(t, os.Remove(filePath))
+ require.NoError(t, store.Del(context.Background(), 1, KVNamespace, filekey))
+ }
+}
+func TestFileStore_FilepathFor(t *testing.T) {
+ store := NewFakeKVStore(t)
+ workingDir := t.TempDir()
+ fs := NewFileStore(1, store, workingDir)
+ filekey := "silences"
+ filePath := filepath.Join(workingDir, filekey)
+
+ // With a file already on disk, it returns the existing file's filepath and no modification to the original file.
+ {
+ require.NoError(t, os.WriteFile(filePath, []byte("silence1,silence2"), 0644))
+ r, err := fs.FilepathFor(context.Background(), filekey)
+ require.NoError(t, err)
+ require.Equal(t, filePath, r)
+ f, err := ioutil.ReadFile(filepath.Clean(filePath))
+ require.NoError(t, err)
+ require.Equal(t, "silence1,silence2", string(f))
+ require.NoError(t, os.Remove(filePath))
+ }
+
+ // With a file already on the database, it writes the file to disk and returns the filepath.
+ {
+ require.NoError(t, store.Set(context.Background(), 1, KVNamespace, filekey, encode([]byte("silence1,silence3"))))
+ r, err := fs.FilepathFor(context.Background(), filekey)
+ require.NoError(t, err)
+ require.Equal(t, filePath, r)
+ f, err := ioutil.ReadFile(filepath.Clean(filePath))
+ require.NoError(t, err)
+ require.Equal(t, "silence1,silence3", string(f))
+ require.NoError(t, os.Remove(filePath))
+ require.NoError(t, store.Del(context.Background(), 1, KVNamespace, filekey))
+ }
+
+ // With no file on disk or database, it returns the original filepath.
+ {
+ r, err := fs.FilepathFor(context.Background(), filekey)
+ require.NoError(t, err)
+ require.Equal(t, filePath, r)
+ _, err = ioutil.ReadFile(filepath.Clean(filePath))
+ require.Error(t, err)
+ }
+}
+
+func TestFileStore_Persist(t *testing.T) {
+ store := NewFakeKVStore(t)
+ state := &fakeState{data: "something to marshal"}
+ workingDir := t.TempDir()
+ fs := NewFileStore(1, store, workingDir)
+ filekey := "silences"
+
+ size, err := fs.Persist(context.Background(), filekey, state)
+ require.NoError(t, err)
+ require.Equal(t, int64(20), size)
+ store.mtx.Lock()
+ require.Len(t, store.store, 1)
+ store.mtx.Unlock()
+ v, ok, err := store.Get(context.Background(), 1, KVNamespace, filekey)
+ require.NoError(t, err)
+ require.True(t, ok)
+ b, err := decode(v)
+ require.NoError(t, err)
+ require.Equal(t, "something to marshal", string(b))
+}
diff --git a/alerting/multiorg_alertmanager.go b/alerting/multiorg_alertmanager.go
new file mode 100644
index 00000000..0d5475b0
--- /dev/null
+++ b/alerting/multiorg_alertmanager.go
@@ -0,0 +1,342 @@
+package notifier
+
+import (
+ "context"
+ "fmt"
+ "io/ioutil"
+ "path/filepath"
+ "strconv"
+ "sync"
+ "time"
+
+ "github.com/grafana/grafana/pkg/services/ngalert/notifier/channels"
+ "github.com/grafana/grafana/pkg/services/ngalert/provisioning"
+ "github.com/grafana/grafana/pkg/services/notifications"
+ "github.com/grafana/grafana/pkg/services/secrets"
+
+ "github.com/prometheus/alertmanager/cluster"
+ "github.com/prometheus/client_golang/prometheus"
+
+ "github.com/grafana/grafana/pkg/infra/kvstore"
+ "github.com/grafana/grafana/pkg/infra/log"
+ "github.com/grafana/grafana/pkg/services/ngalert/metrics"
+ "github.com/grafana/grafana/pkg/services/ngalert/models"
+ "github.com/grafana/grafana/pkg/services/ngalert/store"
+ "github.com/grafana/grafana/pkg/setting"
+)
+
+var (
+ ErrNoAlertmanagerForOrg = fmt.Errorf("Alertmanager does not exist for this organization")
+ ErrAlertmanagerNotReady = fmt.Errorf("Alertmanager is not ready yet")
+)
+
+type MultiOrgAlertmanager struct {
+ Crypto Crypto
+ ProvStore provisioning.ProvisioningStore
+
+ alertmanagersMtx sync.RWMutex
+ alertmanagers map[int64]*Alertmanager
+
+ settings *setting.Cfg
+ logger log.Logger
+
+ // clusterPeer represents the clustering peers of Alertmanagers between Grafana instances.
+ peer ClusterPeer
+ settleCancel context.CancelFunc
+
+ configStore AlertingStore
+ orgStore store.OrgStore
+ kvStore kvstore.KVStore
+
+ decryptFn channels.GetDecryptedValueFn
+
+ metrics *metrics.MultiOrgAlertmanager
+ ns notifications.Service
+}
+
+func NewMultiOrgAlertmanager(cfg *setting.Cfg, configStore AlertingStore, orgStore store.OrgStore,
+ kvStore kvstore.KVStore, provStore provisioning.ProvisioningStore, decryptFn channels.GetDecryptedValueFn,
+ m *metrics.MultiOrgAlertmanager, ns notifications.Service, l log.Logger, s secrets.Service,
+) (*MultiOrgAlertmanager, error) {
+ moa := &MultiOrgAlertmanager{
+ Crypto: NewCrypto(s, configStore, l),
+ ProvStore: provStore,
+
+ logger: l,
+ settings: cfg,
+ alertmanagers: map[int64]*Alertmanager{},
+ configStore: configStore,
+ orgStore: orgStore,
+ kvStore: kvStore,
+ decryptFn: decryptFn,
+ metrics: m,
+ ns: ns,
+ }
+
+ clusterLogger := l.New("component", "cluster")
+ moa.peer = &NilPeer{}
+ if len(cfg.UnifiedAlerting.HAPeers) > 0 {
+ peer, err := cluster.Create(
+ clusterLogger,
+ m.Registerer,
+ cfg.UnifiedAlerting.HAListenAddr,
+ cfg.UnifiedAlerting.HAAdvertiseAddr,
+ cfg.UnifiedAlerting.HAPeers, // peers
+ true,
+ cfg.UnifiedAlerting.HAPushPullInterval,
+ cfg.UnifiedAlerting.HAGossipInterval,
+ cluster.DefaultTcpTimeout,
+ cluster.DefaultProbeTimeout,
+ cluster.DefaultProbeInterval,
+ nil,
+ true,
+ )
+
+ if err != nil {
+ return nil, fmt.Errorf("unable to initialize gossip mesh: %w", err)
+ }
+
+ err = peer.Join(cluster.DefaultReconnectInterval, cluster.DefaultReconnectTimeout)
+ if err != nil {
+ l.Error("msg", "unable to join gossip mesh while initializing cluster for high availability mode", "err", err)
+ }
+ // Attempt to verify the number of peers for 30s every 2s. The risk here is what we send a notification "too soon".
+ // Which should _never_ happen given we share the notification log via the database so the risk of double notification is very low.
+ var ctx context.Context
+ ctx, moa.settleCancel = context.WithTimeout(context.Background(), 30*time.Second)
+ go peer.Settle(ctx, cluster.DefaultGossipInterval*10)
+ moa.peer = peer
+ }
+
+ return moa, nil
+}
+
+func (moa *MultiOrgAlertmanager) Run(ctx context.Context) error {
+ moa.logger.Info("starting MultiOrg Alertmanager")
+
+ for {
+ select {
+ case <-ctx.Done():
+ moa.StopAndWait()
+ return nil
+ case <-time.After(moa.settings.UnifiedAlerting.AlertmanagerConfigPollInterval):
+ if err := moa.LoadAndSyncAlertmanagersForOrgs(ctx); err != nil {
+ moa.logger.Error("error while synchronizing Alertmanager orgs", "err", err)
+ }
+ }
+ }
+}
+
+func (moa *MultiOrgAlertmanager) LoadAndSyncAlertmanagersForOrgs(ctx context.Context) error {
+ moa.logger.Debug("synchronizing Alertmanagers for orgs")
+ // First, load all the organizations from the database.
+ orgIDs, err := moa.orgStore.GetOrgs(ctx)
+ if err != nil {
+ return err
+ }
+
+ // Then, sync them by creating or deleting Alertmanagers as necessary.
+ moa.metrics.DiscoveredConfigurations.Set(float64(len(orgIDs)))
+ moa.SyncAlertmanagersForOrgs(ctx, orgIDs)
+
+ moa.logger.Debug("done synchronizing Alertmanagers for orgs")
+
+ return nil
+}
+
+// getLatestConfigs retrieves the latest Alertmanager configuration for every organization. It returns a map where the key is the ID of each organization and the value is the configuration.
+func (moa *MultiOrgAlertmanager) getLatestConfigs(ctx context.Context) (map[int64]*models.AlertConfiguration, error) {
+ configs, err := moa.configStore.GetAllLatestAlertmanagerConfiguration(ctx)
+ if err != nil {
+ return nil, err
+ }
+
+ result := make(map[int64]*models.AlertConfiguration, len(configs))
+ for _, config := range configs {
+ result[config.OrgID] = config
+ }
+
+ return result, nil
+}
+
+// SyncAlertmanagersForOrgs syncs configuration of the Alertmanager required by each organization.
+func (moa *MultiOrgAlertmanager) SyncAlertmanagersForOrgs(ctx context.Context, orgIDs []int64) {
+ orgsFound := make(map[int64]struct{}, len(orgIDs))
+ dbConfigs, err := moa.getLatestConfigs(ctx)
+ if err != nil {
+ moa.logger.Error("failed to load Alertmanager configurations", "err", err)
+ return
+ }
+ moa.alertmanagersMtx.Lock()
+ for _, orgID := range orgIDs {
+ if _, isDisabledOrg := moa.settings.UnifiedAlerting.DisabledOrgs[orgID]; isDisabledOrg {
+ moa.logger.Debug("skipping syncing Alertmanger for disabled org", "org", orgID)
+ continue
+ }
+ orgsFound[orgID] = struct{}{}
+
+ alertmanager, found := moa.alertmanagers[orgID]
+
+ if !found {
+ // These metrics are not exported by Grafana and are mostly a placeholder.
+ // To export them, we need to translate the metrics from each individual registry and,
+ // then aggregate them on the main registry.
+ m := metrics.NewAlertmanagerMetrics(moa.metrics.GetOrCreateOrgRegistry(orgID))
+ am, err := newAlertmanager(ctx, orgID, moa.settings, moa.configStore, moa.kvStore, moa.peer, moa.decryptFn, moa.ns, m)
+ if err != nil {
+ moa.logger.Error("unable to create Alertmanager for org", "org", orgID, "err", err)
+ }
+ moa.alertmanagers[orgID] = am
+ alertmanager = am
+ }
+
+ dbConfig, cfgFound := dbConfigs[orgID]
+ if !cfgFound {
+ if found {
+ // This means that the configuration is gone but the organization, as well as the Alertmanager, exists.
+ moa.logger.Warn("Alertmanager exists for org but the configuration is gone. Applying the default configuration", "org", orgID)
+ }
+ err := alertmanager.SaveAndApplyDefaultConfig(ctx)
+ if err != nil {
+ moa.logger.Error("failed to apply the default Alertmanager configuration", "org", orgID)
+ continue
+ }
+ moa.alertmanagers[orgID] = alertmanager
+ continue
+ }
+
+ err := alertmanager.ApplyConfig(dbConfig)
+ if err != nil {
+ moa.logger.Error("failed to apply Alertmanager config for org", "org", orgID, "id", dbConfig.ID, "err", err)
+ continue
+ }
+ moa.alertmanagers[orgID] = alertmanager
+ }
+
+ amsToStop := map[int64]*Alertmanager{}
+ for orgId, am := range moa.alertmanagers {
+ if _, exists := orgsFound[orgId]; !exists {
+ amsToStop[orgId] = am
+ delete(moa.alertmanagers, orgId)
+ moa.metrics.RemoveOrgRegistry(orgId)
+ }
+ }
+ moa.metrics.ActiveConfigurations.Set(float64(len(moa.alertmanagers)))
+ moa.alertmanagersMtx.Unlock()
+
+ // Now, we can stop the Alertmanagers without having to hold a lock.
+ for orgID, am := range amsToStop {
+ moa.logger.Info("stopping Alertmanager", "org", orgID)
+ am.StopAndWait()
+ moa.logger.Info("stopped Alertmanager", "org", orgID)
+ // Cleanup all the remaining resources from this alertmanager.
+ am.fileStore.CleanUp()
+ }
+
+ // We look for orphan directories and remove them. Orphan directories can
+ // occur when an organization is deleted and the node running Grafana is
+ // shutdown before the next sync is executed.
+ moa.cleanupOrphanLocalOrgState(ctx, orgsFound)
+}
+
+// cleanupOrphanLocalOrgState will check if there is any organization on
+// disk that is not part of the active organizations. If this is the case
+// it will delete the local state from disk.
+func (moa *MultiOrgAlertmanager) cleanupOrphanLocalOrgState(ctx context.Context,
+ activeOrganizations map[int64]struct{}) {
+ dataDir := filepath.Join(moa.settings.DataPath, workingDir)
+ files, err := ioutil.ReadDir(dataDir)
+ if err != nil {
+ moa.logger.Error("failed to list local working directory", "dir", dataDir, "err", err)
+ return
+ }
+ for _, file := range files {
+ if !file.IsDir() {
+ moa.logger.Warn("ignoring unexpected file while scanning local working directory", "filename", filepath.Join(dataDir, file.Name()))
+ continue
+ }
+ orgID, err := strconv.ParseInt(file.Name(), 10, 64)
+ if err != nil {
+ moa.logger.Error("unable to parse orgID from directory name", "name", file.Name(), "err", err)
+ continue
+ }
+ _, exists := activeOrganizations[orgID]
+ if !exists {
+ moa.logger.Info("found orphan organization directory", "orgID", orgID)
+ workingDirPath := filepath.Join(dataDir, strconv.FormatInt(orgID, 10))
+ fileStore := NewFileStore(orgID, moa.kvStore, workingDirPath)
+ // Cleanup all the remaining resources from this alertmanager.
+ fileStore.CleanUp()
+ }
+ }
+ // Remove all orphaned items from kvstore by listing all existing items
+ // in our used namespace and comparing them to the currently active
+ // organizations.
+ storedFiles := []string{notificationLogFilename, silencesFilename}
+ for _, fileName := range storedFiles {
+ keys, err := moa.kvStore.Keys(ctx, kvstore.AllOrganizations, KVNamespace, fileName)
+ if err != nil {
+ moa.logger.Error("failed to fetch items from kvstore", "err", err,
+ "namespace", KVNamespace, "key", fileName)
+ }
+ for _, key := range keys {
+ if _, exists := activeOrganizations[key.OrgId]; exists {
+ continue
+ }
+ err = moa.kvStore.Del(ctx, key.OrgId, key.Namespace, key.Key)
+ if err != nil {
+ moa.logger.Error("failed to delete item from kvstore", "err", err,
+ "orgID", key.OrgId, "namespace", KVNamespace, "key", key.Key)
+ }
+ }
+ }
+}
+
+func (moa *MultiOrgAlertmanager) StopAndWait() {
+ moa.alertmanagersMtx.Lock()
+ defer moa.alertmanagersMtx.Unlock()
+
+ for _, am := range moa.alertmanagers {
+ am.StopAndWait()
+ }
+
+ p, ok := moa.peer.(*cluster.Peer)
+ if ok {
+ moa.settleCancel()
+ if err := p.Leave(10 * time.Second); err != nil {
+ moa.logger.Warn("unable to leave the gossip mesh", "err", err)
+ }
+ }
+}
+
+// AlertmanagerFor returns the Alertmanager instance for the organization provided.
+// When the organization does not have an active Alertmanager, it returns a ErrNoAlertmanagerForOrg.
+// When the Alertmanager of the organization is not ready, it returns a ErrAlertmanagerNotReady.
+func (moa *MultiOrgAlertmanager) AlertmanagerFor(orgID int64) (*Alertmanager, error) {
+ moa.alertmanagersMtx.RLock()
+ defer moa.alertmanagersMtx.RUnlock()
+
+ orgAM, existing := moa.alertmanagers[orgID]
+ if !existing {
+ return nil, ErrNoAlertmanagerForOrg
+ }
+
+ if !orgAM.Ready() {
+ return orgAM, ErrAlertmanagerNotReady
+ }
+
+ return orgAM, nil
+}
+
+// NilPeer and NilChannel implements the Alertmanager clustering interface.
+type NilPeer struct{}
+
+func (p *NilPeer) Position() int { return 0 }
+func (p *NilPeer) WaitReady(context.Context) error { return nil }
+func (p *NilPeer) AddState(string, cluster.State, prometheus.Registerer) cluster.ClusterChannel {
+ return &NilChannel{}
+}
+
+type NilChannel struct{}
+
+func (c *NilChannel) Broadcast([]byte) {}
diff --git a/alerting/multiorg_alertmanager_test.go b/alerting/multiorg_alertmanager_test.go
new file mode 100644
index 00000000..d0f81977
--- /dev/null
+++ b/alerting/multiorg_alertmanager_test.go
@@ -0,0 +1,289 @@
+package notifier
+
+import (
+ "bytes"
+ "context"
+ "errors"
+ "io/fs"
+ "os"
+ "path/filepath"
+ "testing"
+ "time"
+
+ "github.com/grafana/grafana/pkg/infra/log"
+ "github.com/grafana/grafana/pkg/services/ngalert/metrics"
+ "github.com/grafana/grafana/pkg/services/ngalert/models"
+ "github.com/grafana/grafana/pkg/services/ngalert/provisioning"
+ "github.com/grafana/grafana/pkg/services/secrets/fakes"
+ secretsManager "github.com/grafana/grafana/pkg/services/secrets/manager"
+ "github.com/grafana/grafana/pkg/setting"
+
+ "github.com/prometheus/client_golang/prometheus"
+ "github.com/prometheus/client_golang/prometheus/testutil"
+ "github.com/stretchr/testify/require"
+)
+
+func TestMultiOrgAlertmanager_SyncAlertmanagersForOrgs(t *testing.T) {
+ configStore := &FakeConfigStore{
+ configs: map[int64]*models.AlertConfiguration{},
+ }
+ orgStore := &FakeOrgStore{
+ orgs: []int64{1, 2, 3},
+ }
+
+ tmpDir := t.TempDir()
+ kvStore := NewFakeKVStore(t)
+ provStore := provisioning.NewFakeProvisioningStore()
+ secretsService := secretsManager.SetupTestService(t, fakes.NewFakeSecretsStore())
+ decryptFn := secretsService.GetDecryptedValue
+ reg := prometheus.NewPedanticRegistry()
+ m := metrics.NewNGAlert(reg)
+ cfg := &setting.Cfg{
+ DataPath: tmpDir,
+ UnifiedAlerting: setting.UnifiedAlertingSettings{
+ AlertmanagerConfigPollInterval: 3 * time.Minute,
+ DefaultConfiguration: setting.GetAlertmanagerDefaultConfiguration(),
+ DisabledOrgs: map[int64]struct{}{5: {}},
+ }, // do not poll in tests.
+ }
+ mam, err := NewMultiOrgAlertmanager(cfg, configStore, orgStore, kvStore, provStore, decryptFn, m.GetMultiOrgAlertmanagerMetrics(), nil, log.New("testlogger"), secretsService)
+ require.NoError(t, err)
+ ctx := context.Background()
+
+ // Ensure that one Alertmanager is created per org.
+ {
+ require.NoError(t, mam.LoadAndSyncAlertmanagersForOrgs(ctx))
+ require.Len(t, mam.alertmanagers, 3)
+ require.NoError(t, testutil.GatherAndCompare(reg, bytes.NewBufferString(`
+# HELP grafana_alerting_active_configurations The number of active Alertmanager configurations.
+# TYPE grafana_alerting_active_configurations gauge
+grafana_alerting_active_configurations 3
+# HELP grafana_alerting_discovered_configurations The number of organizations we've discovered that require an Alertmanager configuration.
+# TYPE grafana_alerting_discovered_configurations gauge
+grafana_alerting_discovered_configurations 3
+`), "grafana_alerting_discovered_configurations", "grafana_alerting_active_configurations"))
+ }
+ // When an org is removed, it should detect it.
+ {
+ orgStore.orgs = []int64{1, 3}
+ require.NoError(t, mam.LoadAndSyncAlertmanagersForOrgs(ctx))
+ require.Len(t, mam.alertmanagers, 2)
+ require.NoError(t, testutil.GatherAndCompare(reg, bytes.NewBufferString(`
+# HELP grafana_alerting_active_configurations The number of active Alertmanager configurations.
+# TYPE grafana_alerting_active_configurations gauge
+grafana_alerting_active_configurations 2
+# HELP grafana_alerting_discovered_configurations The number of organizations we've discovered that require an Alertmanager configuration.
+# TYPE grafana_alerting_discovered_configurations gauge
+grafana_alerting_discovered_configurations 2
+`), "grafana_alerting_discovered_configurations", "grafana_alerting_active_configurations"))
+ }
+ // if the org comes back, it should detect it.
+ {
+ orgStore.orgs = []int64{1, 2, 3, 4}
+ require.NoError(t, mam.LoadAndSyncAlertmanagersForOrgs(ctx))
+ require.Len(t, mam.alertmanagers, 4)
+ require.NoError(t, testutil.GatherAndCompare(reg, bytes.NewBufferString(`
+# HELP grafana_alerting_active_configurations The number of active Alertmanager configurations.
+# TYPE grafana_alerting_active_configurations gauge
+grafana_alerting_active_configurations 4
+# HELP grafana_alerting_discovered_configurations The number of organizations we've discovered that require an Alertmanager configuration.
+# TYPE grafana_alerting_discovered_configurations gauge
+grafana_alerting_discovered_configurations 4
+`), "grafana_alerting_discovered_configurations", "grafana_alerting_active_configurations"))
+ }
+ // if the disabled org comes back, it should not detect it.
+ {
+ orgStore.orgs = []int64{1, 2, 3, 4, 5}
+ require.NoError(t, mam.LoadAndSyncAlertmanagersForOrgs(ctx))
+ require.Len(t, mam.alertmanagers, 4)
+ }
+
+ // Orphaned state should be removed.
+ {
+ orgID := int64(6)
+ // First we create a directory and two files for an ograniztation that
+ // is not existing in the current state.
+ orphanDir := filepath.Join(tmpDir, "alerting", "6")
+ err := os.Mkdir(orphanDir, 0750)
+ require.NoError(t, err)
+
+ silencesPath := filepath.Join(orphanDir, silencesFilename)
+ err = os.WriteFile(silencesPath, []byte("file_1"), 0644)
+ require.NoError(t, err)
+
+ notificationPath := filepath.Join(orphanDir, notificationLogFilename)
+ err = os.WriteFile(notificationPath, []byte("file_2"), 0644)
+ require.NoError(t, err)
+
+ // We make sure that both files are on disk.
+ info, err := os.Stat(silencesPath)
+ require.NoError(t, err)
+ require.Equal(t, info.Name(), silencesFilename)
+ info, err = os.Stat(notificationPath)
+ require.NoError(t, err)
+ require.Equal(t, info.Name(), notificationLogFilename)
+
+ // We also populate the kvstore with orphaned records.
+ err = kvStore.Set(ctx, orgID, KVNamespace, silencesFilename, "file_1")
+ require.NoError(t, err)
+
+ err = kvStore.Set(ctx, orgID, KVNamespace, notificationLogFilename, "file_1")
+ require.NoError(t, err)
+
+ // Now re run the sync job once.
+ require.NoError(t, mam.LoadAndSyncAlertmanagersForOrgs(ctx))
+
+ // The organization directory should be gone by now.
+ _, err = os.Stat(orphanDir)
+ require.True(t, errors.Is(err, fs.ErrNotExist))
+
+ // The organization kvstore records should be gone by now.
+ _, exists, _ := kvStore.Get(ctx, orgID, KVNamespace, silencesFilename)
+ require.False(t, exists)
+
+ _, exists, _ = kvStore.Get(ctx, orgID, KVNamespace, notificationLogFilename)
+ require.False(t, exists)
+ }
+}
+
+func TestMultiOrgAlertmanager_SyncAlertmanagersForOrgsWithFailures(t *testing.T) {
+ // Include a broken configuration for organization 2.
+ configStore := &FakeConfigStore{
+ configs: map[int64]*models.AlertConfiguration{
+ 2: {AlertmanagerConfiguration: brokenConfig, OrgID: 2},
+ },
+ }
+ orgStore := &FakeOrgStore{
+ orgs: []int64{1, 2, 3},
+ }
+
+ tmpDir := t.TempDir()
+ kvStore := NewFakeKVStore(t)
+ provStore := provisioning.NewFakeProvisioningStore()
+ secretsService := secretsManager.SetupTestService(t, fakes.NewFakeSecretsStore())
+ decryptFn := secretsService.GetDecryptedValue
+ reg := prometheus.NewPedanticRegistry()
+ m := metrics.NewNGAlert(reg)
+ cfg := &setting.Cfg{
+ DataPath: tmpDir,
+ UnifiedAlerting: setting.UnifiedAlertingSettings{
+ AlertmanagerConfigPollInterval: 10 * time.Minute,
+ DefaultConfiguration: setting.GetAlertmanagerDefaultConfiguration(),
+ }, // do not poll in tests.
+ }
+ mam, err := NewMultiOrgAlertmanager(cfg, configStore, orgStore, kvStore, provStore, decryptFn, m.GetMultiOrgAlertmanagerMetrics(), nil, log.New("testlogger"), secretsService)
+ require.NoError(t, err)
+ ctx := context.Background()
+
+ // When you sync the first time, the alertmanager is created but is doesn't become ready until you have a configuration applied.
+ {
+ require.NoError(t, mam.LoadAndSyncAlertmanagersForOrgs(ctx))
+ require.Len(t, mam.alertmanagers, 3)
+ require.True(t, mam.alertmanagers[1].ready())
+ require.False(t, mam.alertmanagers[2].ready())
+ require.True(t, mam.alertmanagers[3].ready())
+ }
+
+ // On the next sync, it never panics and alertmanager is still not ready.
+ {
+ require.NoError(t, mam.LoadAndSyncAlertmanagersForOrgs(ctx))
+ require.Len(t, mam.alertmanagers, 3)
+ require.True(t, mam.alertmanagers[1].ready())
+ require.False(t, mam.alertmanagers[2].ready())
+ require.True(t, mam.alertmanagers[3].ready())
+ }
+
+ // If we fix the configuration, it becomes ready.
+ {
+ configStore.configs = map[int64]*models.AlertConfiguration{} // It'll apply the default config.
+ require.NoError(t, mam.LoadAndSyncAlertmanagersForOrgs(ctx))
+ require.Len(t, mam.alertmanagers, 3)
+ require.True(t, mam.alertmanagers[1].ready())
+ require.True(t, mam.alertmanagers[2].ready())
+ require.True(t, mam.alertmanagers[3].ready())
+ }
+}
+
+func TestMultiOrgAlertmanager_AlertmanagerFor(t *testing.T) {
+ configStore := &FakeConfigStore{
+ configs: map[int64]*models.AlertConfiguration{},
+ }
+ orgStore := &FakeOrgStore{
+ orgs: []int64{1, 2, 3},
+ }
+ tmpDir := t.TempDir()
+ cfg := &setting.Cfg{
+ DataPath: tmpDir,
+ UnifiedAlerting: setting.UnifiedAlertingSettings{AlertmanagerConfigPollInterval: 3 * time.Minute, DefaultConfiguration: setting.GetAlertmanagerDefaultConfiguration()}, // do not poll in tests.
+ }
+ kvStore := NewFakeKVStore(t)
+ provStore := provisioning.NewFakeProvisioningStore()
+ secretsService := secretsManager.SetupTestService(t, fakes.NewFakeSecretsStore())
+ decryptFn := secretsService.GetDecryptedValue
+ reg := prometheus.NewPedanticRegistry()
+ m := metrics.NewNGAlert(reg)
+ mam, err := NewMultiOrgAlertmanager(cfg, configStore, orgStore, kvStore, provStore, decryptFn, m.GetMultiOrgAlertmanagerMetrics(), nil, log.New("testlogger"), secretsService)
+ require.NoError(t, err)
+ ctx := context.Background()
+
+ // Ensure that one Alertmanagers is created per org.
+ {
+ require.NoError(t, mam.LoadAndSyncAlertmanagersForOrgs(ctx))
+ require.Len(t, mam.alertmanagers, 3)
+ }
+
+ // First, let's try to request an Alertmanager from an org that doesn't exist.
+ {
+ _, err := mam.AlertmanagerFor(5)
+ require.EqualError(t, err, ErrNoAlertmanagerForOrg.Error())
+ }
+
+ // Now, let's try to request an Alertmanager that is not ready.
+ {
+ // let's delete its "running config" to make it non-ready
+ mam.alertmanagers[1].config = nil
+ am, err := mam.AlertmanagerFor(1)
+ require.NotNil(t, am)
+ require.False(t, am.Ready())
+ require.EqualError(t, err, ErrAlertmanagerNotReady.Error())
+ }
+
+ // With an Alertmanager that exists, it responds correctly.
+ {
+ am, err := mam.AlertmanagerFor(2)
+ require.NoError(t, err)
+ require.Equal(t, *am.GetStatus().VersionInfo.Version, "N/A")
+ require.Equal(t, am.orgID, int64(2))
+ require.NotNil(t, am.config)
+ }
+
+ // Let's now remove the previous queried organization.
+ orgStore.orgs = []int64{1, 3}
+ require.NoError(t, mam.LoadAndSyncAlertmanagersForOrgs(ctx))
+ {
+ _, err := mam.AlertmanagerFor(2)
+ require.EqualError(t, err, ErrNoAlertmanagerForOrg.Error())
+ }
+}
+
+var brokenConfig = `
+ "alertmanager_config": {
+ "route": {
+ "receiver": "grafana-default-email"
+ },
+ "receivers": [{
+ "name": "grafana-default-email",
+ "grafana_managed_receiver_configs": [{
+ "uid": "",
+ "name": "slack receiver",
+ "type": "slack",
+ "isDefault": true,
+ "settings": {
+ "addresses": ""
+ "url": "�r_��q/b�����p@ⱎȏ =��@ӹtd>Rú�H�� �;�@Uf��0�\k2*jh�}Íu�)"2�F6]�}r��R�b�d�J;��S퓧��$��",
+ "recipient": "#graphana-metrics",
+ }
+ }]
+ }]
+ }
+}`
diff --git a/alerting/receivers.go b/alerting/receivers.go
new file mode 100644
index 00000000..1b60a727
--- /dev/null
+++ b/alerting/receivers.go
@@ -0,0 +1,260 @@
+package notifier
+
+import (
+ "context"
+ "errors"
+ "fmt"
+ "net/url"
+ "sort"
+ "time"
+
+ apimodels "github.com/grafana/grafana/pkg/services/ngalert/api/tooling/definitions"
+ "github.com/prometheus/alertmanager/notify"
+ "github.com/prometheus/alertmanager/types"
+ "github.com/prometheus/common/model"
+ "golang.org/x/sync/errgroup"
+)
+
+const (
+ maxTestReceiversWorkers = 10
+)
+
+var (
+ ErrNoReceivers = errors.New("no receivers")
+)
+
+type TestReceiversResult struct {
+ Alert types.Alert
+ Receivers []TestReceiverResult
+ NotifedAt time.Time
+}
+
+type TestReceiverResult struct {
+ Name string
+ Configs []TestReceiverConfigResult
+}
+
+type TestReceiverConfigResult struct {
+ Name string
+ UID string
+ Status string
+ Error error
+}
+
+type InvalidReceiverError struct {
+ Receiver *apimodels.PostableGrafanaReceiver
+ Err error
+}
+
+func (e InvalidReceiverError) Error() string {
+ return fmt.Sprintf("the receiver is invalid: %s", e.Err)
+}
+
+type ReceiverTimeoutError struct {
+ Receiver *apimodels.PostableGrafanaReceiver
+ Err error
+}
+
+func (e ReceiverTimeoutError) Error() string {
+ return fmt.Sprintf("the receiver timed out: %s", e.Err)
+}
+
+func (am *Alertmanager) TestReceivers(ctx context.Context, c apimodels.TestReceiversConfigBodyParams) (*TestReceiversResult, error) {
+ // now represents the start time of the test
+ now := time.Now()
+ testAlert := newTestAlert(c, now, now)
+
+ // we must set a group key that is unique per test as some receivers use this key to deduplicate alerts
+ ctx = notify.WithGroupKey(ctx, testAlert.Labels.String()+now.String())
+
+ tmpl, err := am.getTemplate()
+ if err != nil {
+ return nil, fmt.Errorf("failed to get template: %w", err)
+ }
+
+ // job contains all metadata required to test a receiver
+ type job struct {
+ Config *apimodels.PostableGrafanaReceiver
+ ReceiverName string
+ Notifier notify.Notifier
+ }
+
+ // result contains the receiver that was tested and an error that is non-nil if the test failed
+ type result struct {
+ Config *apimodels.PostableGrafanaReceiver
+ ReceiverName string
+ Error error
+ }
+
+ newTestReceiversResult := func(alert types.Alert, results []result, notifiedAt time.Time) *TestReceiversResult {
+ m := make(map[string]TestReceiverResult)
+ for _, receiver := range c.Receivers {
+ // set up the result for this receiver
+ m[receiver.Name] = TestReceiverResult{
+ Name: receiver.Name,
+ // A Grafana receiver can have multiple nested receivers
+ Configs: make([]TestReceiverConfigResult, 0, len(receiver.GrafanaManagedReceivers)),
+ }
+ }
+ for _, next := range results {
+ tmp := m[next.ReceiverName]
+ status := "ok"
+ if next.Error != nil {
+ status = "failed"
+ }
+ tmp.Configs = append(tmp.Configs, TestReceiverConfigResult{
+ Name: next.Config.Name,
+ UID: next.Config.UID,
+ Status: status,
+ Error: processNotifierError(next.Config, next.Error),
+ })
+ m[next.ReceiverName] = tmp
+ }
+ v := new(TestReceiversResult)
+ v.Alert = alert
+ v.Receivers = make([]TestReceiverResult, 0, len(c.Receivers))
+ v.NotifedAt = notifiedAt
+ for _, next := range m {
+ v.Receivers = append(v.Receivers, next)
+ }
+
+ // Make sure the return order is deterministic.
+ sort.Slice(v.Receivers, func(i, j int) bool {
+ return v.Receivers[i].Name < v.Receivers[j].Name
+ })
+
+ return v
+ }
+
+ // invalid keeps track of all invalid receiver configurations
+ invalid := make([]result, 0, len(c.Receivers))
+ // jobs keeps track of all receivers that need to be sent test notifications
+ jobs := make([]job, 0, len(c.Receivers))
+
+ for _, receiver := range c.Receivers {
+ for _, next := range receiver.GrafanaManagedReceivers {
+ n, err := am.buildReceiverIntegration(next, tmpl)
+ if err != nil {
+ invalid = append(invalid, result{
+ Config: next,
+ ReceiverName: next.Name,
+ Error: err,
+ })
+ } else {
+ jobs = append(jobs, job{
+ Config: next,
+ ReceiverName: receiver.Name,
+ Notifier: n,
+ })
+ }
+ }
+ }
+
+ if len(invalid)+len(jobs) == 0 {
+ return nil, ErrNoReceivers
+ }
+
+ if len(jobs) == 0 {
+ return newTestReceiversResult(testAlert, invalid, now), nil
+ }
+
+ numWorkers := maxTestReceiversWorkers
+ if numWorkers > len(jobs) {
+ numWorkers = len(jobs)
+ }
+
+ resultCh := make(chan result, len(jobs))
+ workCh := make(chan job, len(jobs))
+ for _, job := range jobs {
+ workCh <- job
+ }
+ close(workCh)
+
+ g, ctx := errgroup.WithContext(ctx)
+ for i := 0; i < numWorkers; i++ {
+ g.Go(func() error {
+ for next := range workCh {
+ v := result{
+ Config: next.Config,
+ ReceiverName: next.ReceiverName,
+ }
+ if _, err := next.Notifier.Notify(ctx, &testAlert); err != nil {
+ v.Error = err
+ }
+ resultCh <- v
+ }
+ return nil
+ })
+ }
+ g.Wait() // nolint
+ close(resultCh)
+
+ results := make([]result, 0, len(jobs))
+ for next := range resultCh {
+ results = append(results, next)
+ }
+
+ return newTestReceiversResult(testAlert, append(invalid, results...), now), nil
+}
+
+func newTestAlert(c apimodels.TestReceiversConfigBodyParams, startsAt, updatedAt time.Time) types.Alert {
+ var (
+ defaultAnnotations = model.LabelSet{
+ "summary": "Notification test",
+ "__value_string__": "[ metric='foo' labels={instance=bar} value=10 ]",
+ }
+ defaultLabels = model.LabelSet{
+ "alertname": "TestAlert",
+ "instance": "Grafana",
+ }
+ )
+
+ alert := types.Alert{
+ Alert: model.Alert{
+ Labels: defaultLabels,
+ Annotations: defaultAnnotations,
+ StartsAt: startsAt,
+ },
+ UpdatedAt: updatedAt,
+ }
+
+ if c.Alert != nil {
+ if c.Alert.Annotations != nil {
+ for k, v := range c.Alert.Annotations {
+ alert.Annotations[k] = v
+ }
+ }
+ if c.Alert.Labels != nil {
+ for k, v := range c.Alert.Labels {
+ alert.Labels[k] = v
+ }
+ }
+ }
+
+ return alert
+}
+
+func processNotifierError(config *apimodels.PostableGrafanaReceiver, err error) error {
+ if err == nil {
+ return nil
+ }
+
+ var urlError *url.Error
+ if errors.As(err, &urlError) {
+ if urlError.Timeout() {
+ return ReceiverTimeoutError{
+ Receiver: config,
+ Err: err,
+ }
+ }
+ }
+
+ if errors.Is(err, context.DeadlineExceeded) {
+ return ReceiverTimeoutError{
+ Receiver: config,
+ Err: err,
+ }
+ }
+
+ return err
+}
diff --git a/alerting/receivers_test.go b/alerting/receivers_test.go
new file mode 100644
index 00000000..136d3eeb
--- /dev/null
+++ b/alerting/receivers_test.go
@@ -0,0 +1,82 @@
+package notifier
+
+import (
+ "context"
+ "errors"
+ "net/url"
+ "testing"
+
+ "github.com/stretchr/testify/require"
+
+ "github.com/grafana/grafana/pkg/services/ngalert/api/tooling/definitions"
+)
+
+func TestInvalidReceiverError_Error(t *testing.T) {
+ e := InvalidReceiverError{
+ Receiver: &definitions.PostableGrafanaReceiver{
+ Name: "test",
+ UID: "uid",
+ },
+ Err: errors.New("this is an error"),
+ }
+ require.Equal(t, "the receiver is invalid: this is an error", e.Error())
+}
+
+func TestReceiverTimeoutError_Error(t *testing.T) {
+ e := ReceiverTimeoutError{
+ Receiver: &definitions.PostableGrafanaReceiver{
+ Name: "test",
+ UID: "uid",
+ },
+ Err: errors.New("context deadline exceeded"),
+ }
+ require.Equal(t, "the receiver timed out: context deadline exceeded", e.Error())
+}
+
+type timeoutError struct{}
+
+func (e timeoutError) Error() string {
+ return "the request timed out"
+}
+
+func (e timeoutError) Timeout() bool {
+ return true
+}
+
+func TestProcessNotifierError(t *testing.T) {
+ t.Run("assert ReceiverTimeoutError is returned for context deadline exceeded", func(t *testing.T) {
+ r := &definitions.PostableGrafanaReceiver{
+ Name: "test",
+ UID: "uid",
+ }
+ require.Equal(t, ReceiverTimeoutError{
+ Receiver: r,
+ Err: context.DeadlineExceeded,
+ }, processNotifierError(r, context.DeadlineExceeded))
+ })
+
+ t.Run("assert ReceiverTimeoutError is returned for *url.Error timeout", func(t *testing.T) {
+ r := &definitions.PostableGrafanaReceiver{
+ Name: "test",
+ UID: "uid",
+ }
+ urlError := &url.Error{
+ Op: "Get",
+ URL: "https://grafana.net",
+ Err: timeoutError{},
+ }
+ require.Equal(t, ReceiverTimeoutError{
+ Receiver: r,
+ Err: urlError,
+ }, processNotifierError(r, urlError))
+ })
+
+ t.Run("assert unknown error is returned unmodified", func(t *testing.T) {
+ r := &definitions.PostableGrafanaReceiver{
+ Name: "test",
+ UID: "uid",
+ }
+ err := errors.New("this is an error")
+ require.Equal(t, err, processNotifierError(r, err))
+ })
+}
diff --git a/alerting/silences.go b/alerting/silences.go
new file mode 100644
index 00000000..27a9cd76
--- /dev/null
+++ b/alerting/silences.go
@@ -0,0 +1,119 @@
+package notifier
+
+import (
+ "errors"
+ "fmt"
+ "time"
+
+ apimodels "github.com/grafana/grafana/pkg/services/ngalert/api/tooling/definitions"
+ v2 "github.com/prometheus/alertmanager/api/v2"
+ "github.com/prometheus/alertmanager/silence"
+)
+
+var (
+ ErrGetSilencesInternal = fmt.Errorf("unable to retrieve silence(s) due to an internal error")
+ ErrDeleteSilenceInternal = fmt.Errorf("unable to delete silence due to an internal error")
+ ErrCreateSilenceBadPayload = fmt.Errorf("unable to create silence")
+ ErrListSilencesBadPayload = fmt.Errorf("unable to list silences")
+ ErrSilenceNotFound = silence.ErrNotFound
+)
+
+// ListSilences retrieves a list of stored silences. It supports a set of labels as filters.
+func (am *Alertmanager) ListSilences(filter []string) (apimodels.GettableSilences, error) {
+ matchers, err := parseFilter(filter)
+ if err != nil {
+ am.logger.Error("failed to parse matchers", "err", err)
+ return nil, fmt.Errorf("%s: %w", ErrListSilencesBadPayload.Error(), err)
+ }
+
+ psils, _, err := am.silences.Query()
+ if err != nil {
+ am.logger.Error(ErrGetSilencesInternal.Error(), "err", err)
+ return nil, fmt.Errorf("%s: %w", ErrGetSilencesInternal.Error(), err)
+ }
+
+ sils := apimodels.GettableSilences{}
+ for _, ps := range psils {
+ if !v2.CheckSilenceMatchesFilterLabels(ps, matchers) {
+ continue
+ }
+ silence, err := v2.GettableSilenceFromProto(ps)
+ if err != nil {
+ am.logger.Error("unmarshaling from protobuf failed", "err", err)
+ return apimodels.GettableSilences{}, fmt.Errorf("%s: failed to convert internal silence to API silence: %w",
+ ErrGetSilencesInternal.Error(), err)
+ }
+ sils = append(sils, &silence)
+ }
+
+ v2.SortSilences(sils)
+
+ return sils, nil
+}
+
+// GetSilence retrieves a silence by the provided silenceID. It returns ErrSilenceNotFound if the silence is not present.
+func (am *Alertmanager) GetSilence(silenceID string) (apimodels.GettableSilence, error) {
+ sils, _, err := am.silences.Query(silence.QIDs(silenceID))
+ if err != nil {
+ return apimodels.GettableSilence{}, fmt.Errorf("%s: %w", ErrGetSilencesInternal.Error(), err)
+ }
+
+ if len(sils) == 0 {
+ am.logger.Error("failed to find silence", "err", err, "id", sils)
+ return apimodels.GettableSilence{}, ErrSilenceNotFound
+ }
+
+ sil, err := v2.GettableSilenceFromProto(sils[0])
+ if err != nil {
+ am.logger.Error("unmarshaling from protobuf failed", "err", err)
+ return apimodels.GettableSilence{}, fmt.Errorf("%s: failed to convert internal silence to API silence: %w",
+ ErrGetSilencesInternal.Error(), err)
+ }
+
+ return sil, nil
+}
+
+// CreateSilence persists the provided silence and returns the silence ID if successful.
+func (am *Alertmanager) CreateSilence(ps *apimodels.PostableSilence) (string, error) {
+ sil, err := v2.PostableSilenceToProto(ps)
+ if err != nil {
+ am.logger.Error("marshaling to protobuf failed", "err", err)
+ return "", fmt.Errorf("%s: failed to convert API silence to internal silence: %w",
+ ErrCreateSilenceBadPayload.Error(), err)
+ }
+
+ if sil.StartsAt.After(sil.EndsAt) || sil.StartsAt.Equal(sil.EndsAt) {
+ msg := "start time must be before end time"
+ am.logger.Error(msg, "err", "starts_at", sil.StartsAt, "ends_at", sil.EndsAt)
+ return "", fmt.Errorf("%s: %w", msg, ErrCreateSilenceBadPayload)
+ }
+
+ if sil.EndsAt.Before(time.Now()) {
+ msg := "end time can't be in the past"
+ am.logger.Error(msg, "ends_at", sil.EndsAt)
+ return "", fmt.Errorf("%s: %w", msg, ErrCreateSilenceBadPayload)
+ }
+
+ silenceID, err := am.silences.Set(sil)
+ if err != nil {
+ am.logger.Error("msg", "unable to save silence", "err", err)
+ if errors.Is(err, silence.ErrNotFound) {
+ return "", ErrSilenceNotFound
+ }
+ return "", fmt.Errorf("unable to save silence: %s: %w", err.Error(), ErrCreateSilenceBadPayload)
+ }
+
+ return silenceID, nil
+}
+
+// DeleteSilence looks for and expires the silence by the provided silenceID. It returns ErrSilenceNotFound if the silence is not present.
+func (am *Alertmanager) DeleteSilence(silenceID string) error {
+ if err := am.silences.Expire(silenceID); err != nil {
+ if errors.Is(err, silence.ErrNotFound) {
+ return ErrSilenceNotFound
+ }
+ return fmt.Errorf("%s: %w", err.Error(), ErrDeleteSilenceInternal)
+ }
+
+ return nil
+}
diff --git a/alerting/status.go b/alerting/status.go
new file mode 100644
index 00000000..eef11ab4
--- /dev/null
+++ b/alerting/status.go
@@ -0,0 +1,16 @@
+package notifier
+
+import (
+ apimodels "github.com/grafana/grafana/pkg/services/ngalert/api/tooling/definitions"
+)
+
+func (am *Alertmanager) GetStatus() apimodels.GettableStatus {
+ am.reloadConfigMtx.RLock()
+ defer am.reloadConfigMtx.RUnlock()
+
+ config := apimodels.PostableApiAlertingConfig{}
+ if am.ready() {
+ config = am.config.AlertmanagerConfig
+ }
+ return *apimodels.NewGettableStatus(&config)
+}
diff --git a/alerting/testing.go b/alerting/testing.go
new file mode 100644
index 00000000..3415569b
--- /dev/null
+++ b/alerting/testing.go
@@ -0,0 +1,214 @@
+package notifier
+
+import (
+ "context"
+ "crypto/md5"
+ "errors"
+ "fmt"
+ "strings"
+ "sync"
+ "testing"
+
+ "github.com/grafana/grafana/pkg/infra/kvstore"
+ "github.com/grafana/grafana/pkg/services/ngalert/models"
+ "github.com/grafana/grafana/pkg/services/ngalert/store"
+)
+
+type FakeConfigStore struct {
+ configs map[int64]*models.AlertConfiguration
+}
+
+// Saves the image or returns an error.
+func (f *FakeConfigStore) SaveImage(ctx context.Context, img *models.Image) error {
+ return models.ErrImageNotFound
+}
+
+func (f *FakeConfigStore) GetImage(ctx context.Context, token string) (*models.Image, error) {
+ return nil, models.ErrImageNotFound
+}
+
+func (f *FakeConfigStore) GetImages(ctx context.Context, tokens []string) ([]models.Image, error) {
+ return nil, models.ErrImageNotFound
+}
+
+func NewFakeConfigStore(t *testing.T, configs map[int64]*models.AlertConfiguration) FakeConfigStore {
+ t.Helper()
+
+ return FakeConfigStore{
+ configs: configs,
+ }
+}
+
+func (f *FakeConfigStore) GetAllLatestAlertmanagerConfiguration(context.Context) ([]*models.AlertConfiguration, error) {
+ result := make([]*models.AlertConfiguration, 0, len(f.configs))
+ for _, configuration := range f.configs {
+ result = append(result, configuration)
+ }
+ return result, nil
+}
+
+func (f *FakeConfigStore) GetLatestAlertmanagerConfiguration(_ context.Context, query *models.GetLatestAlertmanagerConfigurationQuery) error {
+ var ok bool
+ query.Result, ok = f.configs[query.OrgID]
+ if !ok {
+ return store.ErrNoAlertmanagerConfiguration
+ }
+
+ return nil
+}
+
+func (f *FakeConfigStore) SaveAlertmanagerConfiguration(_ context.Context, cmd *models.SaveAlertmanagerConfigurationCmd) error {
+ f.configs[cmd.OrgID] = &models.AlertConfiguration{
+ AlertmanagerConfiguration: cmd.AlertmanagerConfiguration,
+ OrgID: cmd.OrgID,
+ ConfigurationVersion: "v1",
+ Default: cmd.Default,
+ }
+
+ return nil
+}
+
+func (f *FakeConfigStore) SaveAlertmanagerConfigurationWithCallback(_ context.Context, cmd *models.SaveAlertmanagerConfigurationCmd, callback store.SaveCallback) error {
+ f.configs[cmd.OrgID] = &models.AlertConfiguration{
+ AlertmanagerConfiguration: cmd.AlertmanagerConfiguration,
+ OrgID: cmd.OrgID,
+ ConfigurationVersion: "v1",
+ Default: cmd.Default,
+ }
+
+ if err := callback(); err != nil {
+ return err
+ }
+
+ return nil
+}
+
+func (f *FakeConfigStore) UpdateAlertmanagerConfiguration(_ context.Context, cmd *models.SaveAlertmanagerConfigurationCmd) error {
+ if config, exists := f.configs[cmd.OrgID]; exists && config.ConfigurationHash == cmd.FetchedConfigurationHash {
+ f.configs[cmd.OrgID] = &models.AlertConfiguration{
+ AlertmanagerConfiguration: cmd.AlertmanagerConfiguration,
+ OrgID: cmd.OrgID,
+ ConfigurationHash: fmt.Sprintf("%x", md5.Sum([]byte(cmd.AlertmanagerConfiguration))),
+ ConfigurationVersion: "v1",
+ Default: cmd.Default,
+ }
+ return nil
+ }
+ return errors.New("config not found or hash not valid")
+}
+
+type FakeOrgStore struct {
+ orgs []int64
+}
+
+func NewFakeOrgStore(t *testing.T, orgs []int64) FakeOrgStore {
+ t.Helper()
+
+ return FakeOrgStore{
+ orgs: orgs,
+ }
+}
+
+func (f *FakeOrgStore) GetOrgs(_ context.Context) ([]int64, error) {
+ return f.orgs, nil
+}
+
+type FakeKVStore struct {
+ mtx sync.Mutex
+ store map[int64]map[string]map[string]string
+}
+
+func NewFakeKVStore(t *testing.T) *FakeKVStore {
+ t.Helper()
+
+ return &FakeKVStore{
+ store: map[int64]map[string]map[string]string{},
+ }
+}
+
+func (fkv *FakeKVStore) Get(_ context.Context, orgId int64, namespace string, key string) (string, bool, error) {
+ fkv.mtx.Lock()
+ defer fkv.mtx.Unlock()
+ org, ok := fkv.store[orgId]
+ if !ok {
+ return "", false, nil
+ }
+ k, ok := org[namespace]
+ if !ok {
+ return "", false, nil
+ }
+
+ v, ok := k[key]
+ if !ok {
+ return "", false, nil
+ }
+
+ return v, true, nil
+}
+func (fkv *FakeKVStore) Set(_ context.Context, orgId int64, namespace string, key string, value string) error {
+ fkv.mtx.Lock()
+ defer fkv.mtx.Unlock()
+ org, ok := fkv.store[orgId]
+ if !ok {
+ fkv.store[orgId] = map[string]map[string]string{}
+ }
+ _, ok = org[namespace]
+ if !ok {
+ fkv.store[orgId][namespace] = map[string]string{}
+ }
+
+ fkv.store[orgId][namespace][key] = value
+
+ return nil
+}
+func (fkv *FakeKVStore) Del(_ context.Context, orgId int64, namespace string, key string) error {
+ fkv.mtx.Lock()
+ defer fkv.mtx.Unlock()
+ org, ok := fkv.store[orgId]
+ if !ok {
+ return nil
+ }
+ _, ok = org[namespace]
+ if !ok {
+ return nil
+ }
+
+ delete(fkv.store[orgId][namespace], key)
+
+ return nil
+}
+
+func (fkv *FakeKVStore) Keys(ctx context.Context, orgID int64, namespace string, keyPrefix string) ([]kvstore.Key, error) {
+ fkv.mtx.Lock()
+ defer fkv.mtx.Unlock()
+ var keys []kvstore.Key
+ for orgIDFromStore, namespaceMap := range fkv.store {
+ if orgID != kvstore.AllOrganizations && orgID != orgIDFromStore {
+ continue
+ }
+ if keyMap, exists := namespaceMap[namespace]; exists {
+ for k := range keyMap {
+ if strings.HasPrefix(k, keyPrefix) {
+ keys = append(keys, kvstore.Key{
+ OrgId: orgIDFromStore,
+ Namespace: namespace,
+ Key: keyPrefix,
+ })
+ }
+ }
+ }
+ }
+ return keys, nil
+}
+
+func (fkv *FakeKVStore) GetAll(ctx context.Context, orgId int64, namespace string) (map[int64]map[string]string, error) {
+ return nil, nil
+}
+
+type fakeState struct {
+ data string
+}
+
+func (fs *fakeState) MarshalBinary() ([]byte, error) {
+ return []byte(fs.data), nil
+}