From fa7ae67e828cb3cbf833518e9b93023ab14c178f Mon Sep 17 00:00:00 2001 From: Alvar Penning Date: Tue, 29 Jul 2025 16:18:43 +0200 Subject: [PATCH 01/38] Initial Icinga Notifications Source This is the first version to use Icinga DB as an event source for Icinga Notifications. If configured accordingly, Icinga DB forwards events crafted from the Redis pipeline to the Icinga Notifications API. This required a small refactoring of the history synchronization to allow hooking into the Redis stream. Afterwards, the newly introduced notifications package handles the rest. Note: As part of this architectural change, Icinga Notifications offers filters to be evaluated by Icinga DB. At the moment, these are SQL queries being executed on the Icinga DB relational database. Either consider both Icinga DB and Icinga Notifications to be part of the same trust domain or consider the security implications. Furthermore, this change requires a change on Icinga Notifications as well. This will not work with the current version 0.1.1. --- cmd/icingadb/main.go | 16 +- config.example.yml | 15 + internal/config/config.go | 17 +- pkg/icingadb/history/sync.go | 68 ++- pkg/notifications/notifications.go | 700 +++++++++++++++++++++++++++++ 5 files changed, 810 insertions(+), 6 deletions(-) create mode 100644 pkg/notifications/notifications.go diff --git a/cmd/icingadb/main.go b/cmd/icingadb/main.go index e96e96d08..bd2a14ce1 100644 --- a/cmd/icingadb/main.go +++ b/cmd/icingadb/main.go @@ -3,6 +3,7 @@ package main import ( "context" "fmt" + "github.com/icinga/icinga-go-library/database" "github.com/icinga/icinga-go-library/logging" "github.com/icinga/icinga-go-library/redis" "github.com/icinga/icinga-go-library/utils" @@ -15,6 +16,7 @@ import ( v1 "github.com/icinga/icingadb/pkg/icingadb/v1" "github.com/icinga/icingadb/pkg/icingaredis" "github.com/icinga/icingadb/pkg/icingaredis/telemetry" + "github.com/icinga/icingadb/pkg/notifications" "github.com/okzk/sdnotify" "github.com/pkg/errors" "go.uber.org/zap" @@ -168,10 +170,22 @@ func run() int { sig := make(chan os.Signal, 1) signal.Notify(sig, os.Interrupt, syscall.SIGTERM, syscall.SIGHUP) + var notificationsSourceCallback func(database.Entity) + if cfg := cmd.Config.NotificationsSource; cfg.ApiBaseUrl != "" { + logger.Info("Starting Icinga Notifications source") + + notificationsSource := notifications.NewNotificationsSource( + ctx, + db, + logs.GetChildLogger("notifications-source"), + cfg) + notificationsSourceCallback = notificationsSource.Submit + } + go func() { logger.Info("Starting history sync") - if err := hs.Sync(ctx); err != nil && !utils.IsContextCanceled(err) { + if err := hs.Sync(ctx, notificationsSourceCallback); err != nil && !utils.IsContextCanceled(err) { logger.Fatalf("%+v", err) } }() diff --git a/config.example.yml b/config.example.yml index a8e66b9a8..419b20c3b 100644 --- a/config.example.yml +++ b/config.example.yml @@ -139,3 +139,18 @@ redis: # flapping: # notification: # state: + +# Icinga DB can act as an event source for Icinga Notifications. If the following block is not empty, Icinga DB will +# submit events to the Icinga Notifications API. +# notifications-source: +# # URL to the API root. +# api-base-url: http://localhost:5680 +# +# # Source or username to authenticate against the /process-event API. +# user: source-2 +# +# # Password for the defined user. +# password: 1234567812345678 +# +# # URL to the Icinga Web 2 to craft event URLs. +# icinga-web2-base-url: http://localhost/icingaweb2 \ No newline at end of file diff --git a/internal/config/config.go b/internal/config/config.go index 503e6dbf6..10134ed60 100644 --- a/internal/config/config.go +++ b/internal/config/config.go @@ -15,10 +15,11 @@ const DefaultConfigPath = "/etc/icingadb/config.yml" // Config defines Icinga DB config. type Config struct { - Database database.Config `yaml:"database" envPrefix:"DATABASE_"` - Redis redis.Config `yaml:"redis" envPrefix:"REDIS_"` - Logging logging.Config `yaml:"logging" envPrefix:"LOGGING_"` - Retention RetentionConfig `yaml:"retention" envPrefix:"RETENTION_"` + Database database.Config `yaml:"database" envPrefix:"DATABASE_"` + Redis redis.Config `yaml:"redis" envPrefix:"REDIS_"` + Logging logging.Config `yaml:"logging" envPrefix:"LOGGING_"` + Retention RetentionConfig `yaml:"retention" envPrefix:"RETENTION_"` + NotificationsSource NotificationsConfig `yaml:"notifications-source" envPrefix:"NOTIFICATIONS_SOURCE_"` } func (c *Config) SetDefaults() { @@ -109,3 +110,11 @@ func (r *RetentionConfig) Validate() error { return r.Options.Validate() } + +// NotificationsConfig defines Icinga DB as an Icinga Notifications source. +type NotificationsConfig struct { + ApiBaseUrl string `yaml:"api-base-url" env:"API_BASE_URL"` + User string `yaml:"user" env:"USER"` + Password string `yaml:"password" env:"PASSWORD,unset"` + IcingaWeb2BaseUrl string `yaml:"icinga-web2-base-url" env:"ICINGA_WEB2_BASE_URL"` +} diff --git a/pkg/icingadb/history/sync.go b/pkg/icingadb/history/sync.go index 51976f1a1..30c7f5a24 100644 --- a/pkg/icingadb/history/sync.go +++ b/pkg/icingadb/history/sync.go @@ -2,6 +2,7 @@ package history import ( "context" + "fmt" "github.com/icinga/icinga-go-library/com" "github.com/icinga/icinga-go-library/database" "github.com/icinga/icinga-go-library/logging" @@ -37,7 +38,10 @@ func NewSync(db *database.DB, redis *redis.Client, logger *logging.Logger) *Sync } // Sync synchronizes Redis history streams from s.redis to s.db and deletes the original data on success. -func (s Sync) Sync(ctx context.Context) error { +// +// If not nil, the callback function is appended to each synchronization pipeline and called before the entry is deleted +// from Redis. +func (s Sync) Sync(ctx context.Context, callback func(database.Entity)) error { g, ctx := errgroup.WithContext(ctx) for key, pipeline := range syncPipelines { @@ -63,6 +67,10 @@ func (s Sync) Sync(ctx context.Context) error { // forward the entry after it has completed its own sync so that later stages can rely on previous stages being // executed successfully. + if callback != nil { + pipeline = append(pipeline, makeCallbackStageFunc(callback)) + } + ch := make([]chan redis.XMessage, len(pipeline)+1) for i := range ch { if i == 0 { @@ -361,6 +369,64 @@ func userNotificationStage(ctx context.Context, s Sync, key string, in <-chan re })(ctx, s, key, in, out) } +// makeCallbackStageFunc creates a new stageFunc calling the given callback function for each message. +// +// The callback call is blocking and the message will be forwarded to the out channel after the function has returned. +// Thus, please ensure this function does not block too long. +func makeCallbackStageFunc(callback func(database.Entity)) stageFunc { + return func(ctx context.Context, _ Sync, key string, in <-chan redis.XMessage, out chan<- redis.XMessage) error { + defer close(out) + + var structPtr database.Entity + switch key { // keep in sync with syncPipelines below + case "notification": + structPtr = (*v1.NotificationHistory)(nil) + case "state": + structPtr = (*v1.StateHistory)(nil) + case "downtime": + structPtr = (*v1.DowntimeHistory)(nil) + case "comment": + structPtr = (*v1.CommentHistory)(nil) + case "flapping": + structPtr = (*v1.FlappingHistory)(nil) + case "acknowledgement": + structPtr = (*v1.AcknowledgementHistory)(nil) + default: + return fmt.Errorf("unsupported key %q", key) + } + + structifier := structify.MakeMapStructifier( + reflect.TypeOf(structPtr).Elem(), + "json", + contracts.SafeInit) + + for { + select { + case msg, ok := <-in: + if !ok { + return nil + } + + val, err := structifier(msg.Values) + if err != nil { + return errors.Wrapf(err, "can't structify values %#v for %s", msg.Values, key) + } + + entity, ok := val.(database.Entity) + if !ok { + return fmt.Errorf("structifier returned %T, expected %T", val, structPtr) + } + callback(entity) + + out <- msg + + case <-ctx.Done(): + return ctx.Err() + } + } + } +} + var syncPipelines = map[string][]stageFunc{ "notification": { writeOneEntityStage((*v1.NotificationHistory)(nil)), // notification_history diff --git a/pkg/notifications/notifications.go b/pkg/notifications/notifications.go new file mode 100644 index 000000000..4e9937b4e --- /dev/null +++ b/pkg/notifications/notifications.go @@ -0,0 +1,700 @@ +package notifications + +import ( + "bytes" + "context" + "database/sql" + "encoding/json" + "fmt" + "github.com/icinga/icinga-go-library/backoff" + "github.com/icinga/icinga-go-library/database" + "github.com/icinga/icinga-go-library/logging" + "github.com/icinga/icinga-go-library/retry" + "github.com/icinga/icinga-go-library/types" + "github.com/icinga/icingadb/internal/config" + "github.com/icinga/icingadb/pkg/common" + v1history "github.com/icinga/icingadb/pkg/icingadb/v1/history" + "github.com/pkg/errors" + "go.uber.org/zap" + "go.uber.org/zap/zapcore" + "io" + "net/http" + "net/url" + "strings" + "sync" + "time" +) + +// IcingaNotificationsEvent represents an event to be processed by Icinga Notifications. +// +// https://github.com/Icinga/icinga-notifications/blob/v0.1.1/internal/event/event.go#L27 +type IcingaNotificationsEvent struct { + Name string `json:"name"` + URL string `json:"url"` + Tags map[string]string `json:"tags"` + + Type string `json:"type"` + Severity string `json:"severity,omitempty"` + Username string `json:"username"` + Message string `json:"message"` + + Mute bool `json:"mute,omitempty"` + MuteReason string `json:"mute_reason,omitempty"` +} + +// MarshalLogObject implements the zapcore.ObjectMarshaler interface. +func (event IcingaNotificationsEvent) MarshalLogObject(encoder zapcore.ObjectEncoder) error { + encoder.AddString("name", event.Name) + encoder.AddString("type", event.Type) + return nil +} + +// List of IcingaNotificationsEvent.Type defined by Icinga Notifications. +// +// https://github.com/Icinga/icinga-notifications/blob/v0.1.1/internal/event/event.go#L49-L62 +const ( + TypeAcknowledgementCleared = "acknowledgement-cleared" + TypeAcknowledgementSet = "acknowledgement-set" + TypeCustom = "custom" + TypeDowntimeEnd = "downtime-end" + TypeDowntimeRemoved = "downtime-removed" + TypeDowntimeStart = "downtime-start" + TypeFlappingEnd = "flapping-end" + TypeFlappingStart = "flapping-start" + TypeIncidentAge = "incident-age" + TypeMute = "mute" + TypeState = "state" + TypeUnmute = "unmute" +) + +// Severities inspired by Icinga Notifications. +// +// https://github.com/Icinga/icinga-notifications/blob/v0.1.1/internal/event/severity.go#L9 +const ( + SeverityOK = "ok" + SeverityDebug = "debug" + SeverityInfo = "info" + SeverityNotice = "notice" + SeverityWarning = "warning" + SeverityErr = "err" + SeverityCrit = "crit" + SeverityAlert = "alert" + SeverityEmerg = "emerg" +) + +// RuleResp describes a rule response object from Icinga Notifications /event-rules API. +type RuleResp struct { + Id int64 + Name string + ObjectFilterExpr string +} + +// Source is an Icinga Notifications compatible source implementation to push events to Icinga Notifications. +// +// A new Source should be created by the NewNotificationsSource function. New history entries can be submitted by +// calling the Source.Submit method. +type Source struct { + config.NotificationsConfig + + inputCh chan database.Entity // inputCh is a buffered channel used to submit history entries to the worker. + db *database.DB + logger *logging.Logger + + rules map[int64]RuleResp + ruleVersion string + rulesMutex sync.RWMutex + + ctx context.Context + ctxCancel context.CancelFunc +} + +// ErrRulesOutdated implies that the rule version between Icinga DB and Icinga Notifications mismatches. +var ErrRulesOutdated = fmt.Errorf("rule version is outdated") + +// NewNotificationsSource creates a new Source connected to an existing database and logger. +// +// This function starts a worker goroutine in the background which can be stopped by ending the provided context. +func NewNotificationsSource( + ctx context.Context, + db *database.DB, + logger *logging.Logger, + cfg config.NotificationsConfig, +) *Source { + ctx, ctxCancel := context.WithCancel(ctx) + + source := &Source{ + NotificationsConfig: cfg, + + inputCh: make(chan database.Entity, 1<<10), // chosen by fair dice roll + db: db, + logger: logger, + + ctx: ctx, + ctxCancel: ctxCancel, + } + go source.worker() + + return source +} + +// fetchRules from Icinga Notifications /event-rules API endpoint and store both the new rules and the latest rule +// version in the Source struct. +func (s *Source) fetchRules(ctx context.Context, client *http.Client) error { + apiUrl, err := url.JoinPath(s.ApiBaseUrl, "/event-rules") + if err != nil { + return errors.Wrap(err, "cannot join API URL") + } + + req, err := http.NewRequestWithContext(ctx, http.MethodGet, apiUrl, nil) + if err != nil { + return errors.Wrap(err, "cannot create HTTP request") + } + req.SetBasicAuth(s.User, s.Password) + + resp, err := client.Do(req) + if err != nil { + return errors.Wrap(err, "cannot GET rules from Icinga Notifications") + } + + defer func() { + _, _ = io.Copy(io.Discard, resp.Body) + _ = resp.Body.Close() + }() + + if resp.StatusCode != http.StatusOK { + return fmt.Errorf("unexpected status code %q (%d) for rules", resp.Status, resp.StatusCode) + } + + type Response struct { + Version string + Rules map[int64]RuleResp + } + var r Response + + dec := json.NewDecoder(resp.Body) + if err := dec.Decode(&r); err != nil { + return errors.Wrap(err, "cannot decode rules from Icinga Notifications") + } + + s.rulesMutex.Lock() + s.rules = r.Rules + s.ruleVersion = r.Version + s.rulesMutex.Unlock() + + return nil +} + +// evaluateRulesForObject returns the rule IDs for each matching query. +// +// At the moment, each RuleResp.ObjectFilterExpr is executed as a SQL query after the parameters are being bound. If the +// query returns at least one line, the rule will match. Rules with an empty ObjectFilterExpr are a special case and +// will always match. +// +// For the queries, the following mapping is performed: +// - :host_id <- hostId +// - :service_id <- serviceId +// - :environment_id <- environmentId +// +// This allows a query like the following: +// +// > select * from host where id = :host_id and environment_id = :environment_id and name like 'prefix_%' +func (s *Source) evaluateRulesForObject(ctx context.Context, hostId, serviceId, environmentId types.Binary) ([]int64, error) { + s.rulesMutex.RLock() + defer s.rulesMutex.RUnlock() + + outRuleIds := make([]int64, 0, len(s.rules)) + + namedParams := map[string]any{ + "host_id": hostId, + "service_id": serviceId, + "environment_id": environmentId, + } + + for _, rule := range s.rules { + if rule.ObjectFilterExpr == "" { + outRuleIds = append(outRuleIds, rule.Id) + continue + } + + err := retry.WithBackoff( + ctx, + func(ctx context.Context) error { + query := s.db.Rebind(rule.ObjectFilterExpr) + rows, err := s.db.NamedQueryContext(ctx, query, namedParams) + if err != nil { + return err + } + defer func() { _ = rows.Close() }() + + if !rows.Next() { + return sql.ErrNoRows + } + return nil + }, + retry.Retryable, + backoff.DefaultBackoff, + retry.Settings{Timeout: retry.DefaultTimeout}) + + if err == nil { + outRuleIds = append(outRuleIds, rule.Id) + } else if errors.Is(err, sql.ErrNoRows) { + continue + } else { + return nil, errors.Wrapf(err, "cannot fetch rule %d from %q", rule.Id, rule.ObjectFilterExpr) + } + } + + return outRuleIds[:len(outRuleIds):len(outRuleIds)], nil +} + +// fetchHostServiceName for a host ID and a potential service ID from the Icinga DB relational database. +func (s *Source) fetchHostServiceName(ctx context.Context, hostId, serviceId, envId types.Binary) (host, service string, err error) { + err = retry.WithBackoff( + ctx, + func(ctx context.Context) error { + queryHost := s.db.Rebind("SELECT name FROM host WHERE id = ? AND environment_id = ?") + err := s.db.QueryRowxContext(ctx, queryHost, hostId, envId).Scan(&host) + if err != nil { + return errors.Wrap(err, "cannot select host") + } + + if serviceId != nil { + queryService := s.db.Rebind("SELECT name FROM service WHERE id = ? AND environment_id = ?") + err := s.db.QueryRowxContext(ctx, queryService, serviceId, envId).Scan(&service) + if err != nil { + return errors.Wrap(err, "cannot select service") + } + } + + return nil + }, + retry.Retryable, + backoff.DefaultBackoff, + retry.Settings{Timeout: retry.DefaultTimeout}) + return +} + +// rawurlencode mimics PHP's rawurlencode to be used for parameter encoding. +// +// Icinga Web uses rawurldecode instead of urldecode, which, as its main difference, does not honor the plus char ('+') +// as a valid substitution for space (' '). Unfortunately, Go's url.QueryEscape does this very substitution and +// url.PathEscape does a bit too less and has a misleading name on top. +// +// - https://www.php.net/manual/en/function.rawurlencode.php +// - https://github.com/php/php-src/blob/php-8.2.12/ext/standard/url.c#L538 +func rawurlencode(s string) string { + return strings.ReplaceAll(url.QueryEscape(s), "+", "%20") +} + +// buildCommonEvent creates an event.Event based on Host and (optional) Service attributes to be specified later. +// +// The new Event's Time will be the current timestamp. +// +// The following fields will NOT be populated and might be altered later: +// - Type +// - Severity +// - Username +// - Message +// - ID +func (s *Source) buildCommonEvent(host, service string) (*IcingaNotificationsEvent, error) { + var ( + eventName string + eventUrl *url.URL + eventTags map[string]string + ) + + eventUrl, err := url.Parse(s.IcingaWeb2BaseUrl) + if err != nil { + return nil, err + } + + if service != "" { + eventName = host + "!" + service + + eventUrl = eventUrl.JoinPath("/icingadb/service") + eventUrl.RawQuery = "name=" + rawurlencode(service) + "&host.name=" + rawurlencode(host) + + eventTags = map[string]string{ + "host": host, + "service": service, + } + } else { + eventName = host + + eventUrl = eventUrl.JoinPath("/icingadb/host") + eventUrl.RawQuery = "name=" + rawurlencode(host) + + eventTags = map[string]string{ + "host": host, + } + } + + return &IcingaNotificationsEvent{ + Name: eventName, + URL: eventUrl.String(), + Tags: eventTags, + }, nil +} + +// buildStateHistoryEvent from a state history entry. +func (s *Source) buildStateHistoryEvent(ctx context.Context, h *v1history.StateHistory) (*IcingaNotificationsEvent, error) { + hostName, serviceName, err := s.fetchHostServiceName(ctx, h.HostId, h.ServiceId, h.EnvironmentId) + if err != nil { + return nil, errors.Wrap(err, "cannot fetch host/service information") + } + + event, err := s.buildCommonEvent(hostName, serviceName) + if err != nil { + return nil, errors.Wrapf(err, "cannot build event for %q,%q", hostName, serviceName) + } + + event.Type = TypeState + + if serviceName != "" { + switch h.HardState { + case 0: + event.Severity = SeverityOK + case 1: + event.Severity = SeverityWarning + case 2: + event.Severity = SeverityCrit + case 3: + event.Severity = SeverityErr + default: + return nil, fmt.Errorf("unexpected service state %d", h.HardState) + } + } else { + switch h.HardState { + case 0: + event.Severity = SeverityOK + case 1: + event.Severity = SeverityCrit + default: + return nil, fmt.Errorf("unexpected host state %d", h.HardState) + } + } + + if h.Output.Valid { + event.Message = h.Output.String + } + if h.LongOutput.Valid { + event.Message += "\n" + h.LongOutput.String + } + + return event, nil +} + +// buildDowntimeHistoryEvent from a downtime history entry. +func (s *Source) buildDowntimeHistoryEvent(ctx context.Context, h *v1history.DowntimeHistory) (*IcingaNotificationsEvent, error) { + hostName, serviceName, err := s.fetchHostServiceName(ctx, h.HostId, h.ServiceId, h.EnvironmentId) + if err != nil { + return nil, errors.Wrap(err, "cannot fetch host/service information") + } + + event, err := s.buildCommonEvent(hostName, serviceName) + if err != nil { + return nil, errors.Wrapf(err, "cannot build event for %q,%q", hostName, serviceName) + } + + if h.HasBeenCancelled.Valid && h.HasBeenCancelled.Bool { + event.Type = TypeDowntimeRemoved + event.Message = "Downtime was cancelled" + + if h.CancelledBy.Valid { + event.Username = h.CancelledBy.String + } + } else if h.EndTime.Time().Compare(time.Now()) <= 0 { + event.Type = TypeDowntimeEnd + event.Message = "Downtime expired" + } else { + event.Type = TypeDowntimeStart + event.Username = h.Author + event.Message = h.Comment + event.Mute = true + event.MuteReason = "Checkable is in downtime" + } + + return event, nil +} + +// buildFlappingHistoryEvent from a flapping history entry. +func (s *Source) buildFlappingHistoryEvent(ctx context.Context, h *v1history.FlappingHistory) (*IcingaNotificationsEvent, error) { + hostName, serviceName, err := s.fetchHostServiceName(ctx, h.HostId, h.ServiceId, h.EnvironmentId) + if err != nil { + return nil, errors.Wrap(err, "cannot fetch host/service information") + } + + event, err := s.buildCommonEvent(hostName, serviceName) + if err != nil { + return nil, errors.Wrapf(err, "cannot build event for %q,%q", hostName, serviceName) + } + + if h.PercentStateChangeEnd.Valid { + event.Type = TypeFlappingEnd + event.Message = fmt.Sprintf( + "Checkable stopped flapping (Current flapping value %.2f%% < low threshold %.2f%%)", + h.PercentStateChangeEnd.Float64, h.FlappingThresholdLow) + } else if h.PercentStateChangeStart.Valid { + event.Type = TypeFlappingStart + event.Message = fmt.Sprintf( + "Checkable started flapping (Current flapping value %.2f%% > high threshold %.2f%%)", + h.PercentStateChangeStart.Float64, h.FlappingThresholdHigh) + event.Mute = true + event.MuteReason = "Checkable is flapping" + } else { + return nil, errors.New("flapping history entry has neither percent_state_change_start nor percent_state_change_end") + } + + return event, nil +} + +// buildAcknowledgementHistoryEvent from an acknowledgement history entry. +func (s *Source) buildAcknowledgementHistoryEvent(ctx context.Context, h *v1history.AcknowledgementHistory) (*IcingaNotificationsEvent, error) { + hostName, serviceName, err := s.fetchHostServiceName(ctx, h.HostId, h.ServiceId, h.EnvironmentId) + if err != nil { + return nil, errors.Wrap(err, "cannot fetch host/service information") + } + + event, err := s.buildCommonEvent(hostName, serviceName) + if err != nil { + return nil, errors.Wrapf(err, "cannot build event for %q,%q", hostName, serviceName) + } + + if !h.ClearTime.Time().IsZero() { + event.Type = TypeAcknowledgementCleared + event.Message = "Checkable was cleared" + + if h.ClearedBy.Valid { + event.Username = h.ClearedBy.String + } + } else if !h.SetTime.Time().IsZero() { + event.Type = TypeAcknowledgementSet + + if h.Comment.Valid { + event.Message = h.Comment.String + } else { + event.Message = "Checkable was acknowledged" + } + + if h.Author.Valid { + event.Username = h.Author.String + } + } else { + return nil, errors.New("acknowledgment history entry has neither a set_time nor a clear_time") + } + + return event, nil +} + +// submitEvent to the Icinga Notifications /process-event API endpoint. +// +// The event will be passed together with the Source.ruleVersion and all evaluated ruleIds to the endpoint. Even if no +// rules were evaluated, this method should be called. Thus, Icinga Notifications can dismiss the event, but Icinga DB +// would still be informed in case of a rule change. Otherwise, events might be dropped here which are now required. +// +// This method may return an ErrRulesOutdated error, implying that the Source.ruleVersion mismatches the version stored +// at Icinga Notifications. In this case, the rules must be refetched and the event requires another evaluation. +func (s *Source) submitEvent(ctx context.Context, client *http.Client, event *IcingaNotificationsEvent, ruleIds []int64) error { + jsonBody, err := json.Marshal(event) + if err != nil { + return errors.Wrap(err, "cannot encode event to JSON") + } + + apiUrl, err := url.JoinPath(s.ApiBaseUrl, "/process-event") + if err != nil { + return errors.Wrap(err, "cannot join API URL") + } + req, err := http.NewRequestWithContext(ctx, http.MethodPost, apiUrl, bytes.NewBuffer(jsonBody)) + if err != nil { + return errors.Wrap(err, "cannot create HTTP request") + } + req.SetBasicAuth(s.User, s.Password) + + s.rulesMutex.RLock() + ruleVersion := s.ruleVersion + s.rulesMutex.RUnlock() + + ruleIdsStrArr := make([]string, 0, len(ruleIds)) + for _, idInt := range ruleIds { + ruleIdsStrArr = append(ruleIdsStrArr, fmt.Sprintf("%d", idInt)) + } + ruleIdsStr := strings.Join(ruleIdsStrArr, ",") + + req.Header.Set("X-Rule-Ids", ruleIdsStr) + req.Header.Set("X-Rule-Version", ruleVersion) + + resp, err := client.Do(req) + if err != nil { + return errors.Wrap(err, "cannot POST HTTP request to Icinga Notifications") + } + + defer func() { + _, _ = io.Copy(io.Discard, resp.Body) + _ = resp.Body.Close() + }() + + // Refetching rules required. + if resp.StatusCode == http.StatusFailedDependency { + return ErrRulesOutdated + } + + // 200s are acceptable. + if 200 <= resp.StatusCode && resp.StatusCode <= 299 { + return nil + } + + // Ignoring superfluous state change. + if resp.StatusCode == http.StatusNotAcceptable { + return nil + } + + var buff bytes.Buffer + _, _ = io.Copy(&buff, &io.LimitedReader{ + R: resp.Body, + N: 1 << 20, // Limit the error message's length against memory exhaustion + }) + return fmt.Errorf("unexpected response from Icinga Notificatios, status %q (%d): %q", + resp.Status, resp.StatusCode, strings.TrimSpace(buff.String())) +} + +// worker is the background worker launched by NewNotificationsSource. +func (s *Source) worker() { + defer s.ctxCancel() + + client := &http.Client{} + + if err := retry.WithBackoff( + s.ctx, + func(ctx context.Context) error { return s.fetchRules(s.ctx, client) }, + func(_ error) bool { return true }, // For the moment, retry every potential error. + backoff.DefaultBackoff, + retry.Settings{ + Timeout: retry.DefaultTimeout, + OnRetryableError: func(elapsed time.Duration, attempt uint64, err, lastErr error) { + s.logger.Errorw("Cannot fetch rules from Icinga Notifications", + zap.Duration("elapsed", elapsed), + zap.Uint64("attempt", attempt), + zap.Error(err)) + }, + OnSuccess: func(_ time.Duration, attempt uint64, _ error) { + s.logger.Infow("Fetched rules from Icinga Notifications", zap.Uint64("attempt", attempt)) + }, + }, + ); err != nil { + s.logger.Fatalw("Cannot fetch rules from Icinga Notifications", zap.Error(err)) + } + + for { + select { + case <-s.ctx.Done(): + return + + case entity := <-s.inputCh: + var ( + event *IcingaNotificationsEvent + eventErr error + metaHistory v1history.HistoryTableMeta + ) + + // Keep the type switch in sync with syncPipelines from pkg/icingadb/history/sync.go + switch h := entity.(type) { + case *v1history.NotificationHistory: + // Ignore for the moment. + continue + + case *v1history.StateHistory: + if h.StateType != common.HardState { + continue + } + + event, eventErr = s.buildStateHistoryEvent(s.ctx, h) + metaHistory = h.HistoryTableMeta + + case *v1history.DowntimeHistory: + event, eventErr = s.buildDowntimeHistoryEvent(s.ctx, h) + metaHistory = h.HistoryTableMeta + + case *v1history.CommentHistory: + // Ignore for the moment. + continue + + case *v1history.FlappingHistory: + event, eventErr = s.buildFlappingHistoryEvent(s.ctx, h) + metaHistory = h.HistoryTableMeta + + case *v1history.AcknowledgementHistory: + event, eventErr = s.buildAcknowledgementHistoryEvent(s.ctx, h) + metaHistory = h.HistoryTableMeta + + default: + s.logger.Error("Cannot process unsupported type", + zap.String("type", fmt.Sprintf("%T", h))) + continue + } + + if eventErr != nil { + s.logger.Errorw("Cannot build event from history entry", + zap.String("type", fmt.Sprintf("%T", entity)), + zap.Error(eventErr)) + continue + } + if event == nil { + s.logger.Error("No event was fetched, but no error was reported. This REALLY SHOULD NOT happen.") + continue + } + + eventLogger := s.logger.With(zap.Object("event", event)) + + eventRuleIds, err := s.evaluateRulesForObject( + s.ctx, + metaHistory.HostId, + metaHistory.ServiceId, + metaHistory.EnvironmentId) + if err != nil { + eventLogger.Errorw("Cannot evaluate rules for event", zap.Error(err)) + continue + } + + eventLogger = eventLogger.With(zap.Any("rules", eventRuleIds)) + + err = s.submitEvent(s.ctx, client, event, eventRuleIds) + if errors.Is(err, ErrRulesOutdated) { + s.logger.Info("Icinga Notification rules were updated, triggering resync") + + if err := s.fetchRules(s.ctx, client); err != nil { + s.logger.Errorw("Cannot fetch rules from Icinga Notifications", zap.Error(err)) + } + go s.Submit(entity) + + continue + } else if err != nil { + eventLogger.Errorw("Cannot submit event to Icinga Notifications", zap.Error(err)) + continue + } + + eventLogger.Info("Submitted event to Icinga Notifications") + } + } +} + +// Submit a history entry to be processed by the Source's internal worker loop. +// +// Internally, a buffered channel is used for delivery. So this function should not block. Otherwise, it will abort +// after a second and an error is logged. +func (s *Source) Submit(entity database.Entity) { + select { + case <-s.ctx.Done(): + s.logger.Errorw("Source context is done, rejecting submission", + zap.String("submission", fmt.Sprintf("%+v", entity)), + zap.Error(s.ctx.Err())) + return + + case s.inputCh <- entity: + return + + case <-time.After(time.Second): + s.logger.Error("Source submission channel is blocking, rejecting submission", + zap.String("submission", fmt.Sprintf("%+v", entity))) + return + } +} From d5d96a20a51fc81f3b51278d67e09bf58957087c Mon Sep 17 00:00:00 2001 From: Yonas Habteab Date: Wed, 6 Aug 2025 11:58:42 +0200 Subject: [PATCH 02/38] Use the newly introduced notifications event utils from `igl` Most of the notifications related code from here were outsourced to Icinga Go Library, thus removes all the now obsolte ones from here. --- config.example.yml | 6 +- internal/config/config.go | 19 +- pkg/notifications/notifications.go | 435 ++++++++--------------------- 3 files changed, 125 insertions(+), 335 deletions(-) diff --git a/config.example.yml b/config.example.yml index 419b20c3b..10a064c3d 100644 --- a/config.example.yml +++ b/config.example.yml @@ -147,10 +147,10 @@ redis: # api-base-url: http://localhost:5680 # # # Source or username to authenticate against the /process-event API. -# user: source-2 +# username: source-2 # # # Password for the defined user. -# password: 1234567812345678 +# password: insecureinsecure # # # URL to the Icinga Web 2 to craft event URLs. -# icinga-web2-base-url: http://localhost/icingaweb2 \ No newline at end of file +# icingaweb2-base-url: http://localhost/icingaweb2 diff --git a/internal/config/config.go b/internal/config/config.go index 10134ed60..618a85fa0 100644 --- a/internal/config/config.go +++ b/internal/config/config.go @@ -4,6 +4,7 @@ import ( "github.com/creasty/defaults" "github.com/icinga/icinga-go-library/database" "github.com/icinga/icinga-go-library/logging" + "github.com/icinga/icinga-go-library/notifications" "github.com/icinga/icinga-go-library/redis" "github.com/icinga/icingadb/pkg/icingadb/history" "github.com/pkg/errors" @@ -15,11 +16,11 @@ const DefaultConfigPath = "/etc/icingadb/config.yml" // Config defines Icinga DB config. type Config struct { - Database database.Config `yaml:"database" envPrefix:"DATABASE_"` - Redis redis.Config `yaml:"redis" envPrefix:"REDIS_"` - Logging logging.Config `yaml:"logging" envPrefix:"LOGGING_"` - Retention RetentionConfig `yaml:"retention" envPrefix:"RETENTION_"` - NotificationsSource NotificationsConfig `yaml:"notifications-source" envPrefix:"NOTIFICATIONS_SOURCE_"` + Database database.Config `yaml:"database" envPrefix:"DATABASE_"` + Redis redis.Config `yaml:"redis" envPrefix:"REDIS_"` + Logging logging.Config `yaml:"logging" envPrefix:"LOGGING_"` + Retention RetentionConfig `yaml:"retention" envPrefix:"RETENTION_"` + NotificationsSource notifications.Config `yaml:"notifications-source" envPrefix:"NOTIFICATIONS_SOURCE_"` } func (c *Config) SetDefaults() { @@ -110,11 +111,3 @@ func (r *RetentionConfig) Validate() error { return r.Options.Validate() } - -// NotificationsConfig defines Icinga DB as an Icinga Notifications source. -type NotificationsConfig struct { - ApiBaseUrl string `yaml:"api-base-url" env:"API_BASE_URL"` - User string `yaml:"user" env:"USER"` - Password string `yaml:"password" env:"PASSWORD,unset"` - IcingaWeb2BaseUrl string `yaml:"icinga-web2-base-url" env:"ICINGA_WEB2_BASE_URL"` -} diff --git a/pkg/notifications/notifications.go b/pkg/notifications/notifications.go index 4e9937b4e..59a5ff12e 100644 --- a/pkg/notifications/notifications.go +++ b/pkg/notifications/notifications.go @@ -1,115 +1,47 @@ package notifications import ( - "bytes" "context" "database/sql" - "encoding/json" "fmt" + "net/url" + "sync" + "time" + "github.com/icinga/icinga-go-library/backoff" "github.com/icinga/icinga-go-library/database" "github.com/icinga/icinga-go-library/logging" + "github.com/icinga/icinga-go-library/notifications" + "github.com/icinga/icinga-go-library/notifications/event" "github.com/icinga/icinga-go-library/retry" "github.com/icinga/icinga-go-library/types" - "github.com/icinga/icingadb/internal/config" + "github.com/icinga/icinga-go-library/utils" "github.com/icinga/icingadb/pkg/common" v1history "github.com/icinga/icingadb/pkg/icingadb/v1/history" "github.com/pkg/errors" "go.uber.org/zap" "go.uber.org/zap/zapcore" - "io" - "net/http" - "net/url" - "strings" - "sync" - "time" ) -// IcingaNotificationsEvent represents an event to be processed by Icinga Notifications. -// -// https://github.com/Icinga/icinga-notifications/blob/v0.1.1/internal/event/event.go#L27 -type IcingaNotificationsEvent struct { - Name string `json:"name"` - URL string `json:"url"` - Tags map[string]string `json:"tags"` - - Type string `json:"type"` - Severity string `json:"severity,omitempty"` - Username string `json:"username"` - Message string `json:"message"` - - Mute bool `json:"mute,omitempty"` - MuteReason string `json:"mute_reason,omitempty"` -} - -// MarshalLogObject implements the zapcore.ObjectMarshaler interface. -func (event IcingaNotificationsEvent) MarshalLogObject(encoder zapcore.ObjectEncoder) error { - encoder.AddString("name", event.Name) - encoder.AddString("type", event.Type) - return nil -} - -// List of IcingaNotificationsEvent.Type defined by Icinga Notifications. -// -// https://github.com/Icinga/icinga-notifications/blob/v0.1.1/internal/event/event.go#L49-L62 -const ( - TypeAcknowledgementCleared = "acknowledgement-cleared" - TypeAcknowledgementSet = "acknowledgement-set" - TypeCustom = "custom" - TypeDowntimeEnd = "downtime-end" - TypeDowntimeRemoved = "downtime-removed" - TypeDowntimeStart = "downtime-start" - TypeFlappingEnd = "flapping-end" - TypeFlappingStart = "flapping-start" - TypeIncidentAge = "incident-age" - TypeMute = "mute" - TypeState = "state" - TypeUnmute = "unmute" -) - -// Severities inspired by Icinga Notifications. -// -// https://github.com/Icinga/icinga-notifications/blob/v0.1.1/internal/event/severity.go#L9 -const ( - SeverityOK = "ok" - SeverityDebug = "debug" - SeverityInfo = "info" - SeverityNotice = "notice" - SeverityWarning = "warning" - SeverityErr = "err" - SeverityCrit = "crit" - SeverityAlert = "alert" - SeverityEmerg = "emerg" -) - -// RuleResp describes a rule response object from Icinga Notifications /event-rules API. -type RuleResp struct { - Id int64 - Name string - ObjectFilterExpr string -} - // Source is an Icinga Notifications compatible source implementation to push events to Icinga Notifications. // // A new Source should be created by the NewNotificationsSource function. New history entries can be submitted by -// calling the Source.Submit method. +// calling the Source.Submit method. The Source will then process the history entries in a background worker goroutine. type Source struct { - config.NotificationsConfig + notifications.Config inputCh chan database.Entity // inputCh is a buffered channel used to submit history entries to the worker. db *database.DB logger *logging.Logger - rules map[int64]RuleResp - ruleVersion string - rulesMutex sync.RWMutex + rules *notifications.SourceRulesInfo // rules holds the latest rules fetched from Icinga Notifications. + rulesMutex sync.RWMutex // rulesMutex protects access to the rules field. ctx context.Context ctxCancel context.CancelFunc -} -// ErrRulesOutdated implies that the rule version between Icinga DB and Icinga Notifications mismatches. -var ErrRulesOutdated = fmt.Errorf("rule version is outdated") + notificationsClient *notifications.Client // The Icinga Notifications client used to interact with the API. +} // NewNotificationsSource creates a new Source connected to an existing database and logger. // @@ -118,70 +50,32 @@ func NewNotificationsSource( ctx context.Context, db *database.DB, logger *logging.Logger, - cfg config.NotificationsConfig, + cfg notifications.Config, ) *Source { ctx, ctxCancel := context.WithCancel(ctx) source := &Source{ - NotificationsConfig: cfg, + Config: cfg, inputCh: make(chan database.Entity, 1<<10), // chosen by fair dice roll db: db, logger: logger, + rules: ¬ifications.SourceRulesInfo{Version: notifications.EmptyRulesVersion}, + ctx: ctx, ctxCancel: ctxCancel, } - go source.worker() - - return source -} -// fetchRules from Icinga Notifications /event-rules API endpoint and store both the new rules and the latest rule -// version in the Source struct. -func (s *Source) fetchRules(ctx context.Context, client *http.Client) error { - apiUrl, err := url.JoinPath(s.ApiBaseUrl, "/event-rules") + client, err := notifications.NewClient(source.Config, "Icinga DB") if err != nil { - return errors.Wrap(err, "cannot join API URL") - } - - req, err := http.NewRequestWithContext(ctx, http.MethodGet, apiUrl, nil) - if err != nil { - return errors.Wrap(err, "cannot create HTTP request") - } - req.SetBasicAuth(s.User, s.Password) - - resp, err := client.Do(req) - if err != nil { - return errors.Wrap(err, "cannot GET rules from Icinga Notifications") - } - - defer func() { - _, _ = io.Copy(io.Discard, resp.Body) - _ = resp.Body.Close() - }() - - if resp.StatusCode != http.StatusOK { - return fmt.Errorf("unexpected status code %q (%d) for rules", resp.Status, resp.StatusCode) - } - - type Response struct { - Version string - Rules map[int64]RuleResp - } - var r Response - - dec := json.NewDecoder(resp.Body) - if err := dec.Decode(&r); err != nil { - return errors.Wrap(err, "cannot decode rules from Icinga Notifications") + logger.Fatalw("Cannot create Icinga Notifications client", zap.Error(err)) } + source.notificationsClient = client - s.rulesMutex.Lock() - s.rules = r.Rules - s.ruleVersion = r.Version - s.rulesMutex.Unlock() + go source.worker() - return nil + return source } // evaluateRulesForObject returns the rule IDs for each matching query. @@ -202,7 +96,7 @@ func (s *Source) evaluateRulesForObject(ctx context.Context, hostId, serviceId, s.rulesMutex.RLock() defer s.rulesMutex.RUnlock() - outRuleIds := make([]int64, 0, len(s.rules)) + outRuleIds := make([]int64, 0, len(s.rules.Rules)) namedParams := map[string]any{ "host_id": hostId, @@ -210,7 +104,7 @@ func (s *Source) evaluateRulesForObject(ctx context.Context, hostId, serviceId, "environment_id": environmentId, } - for _, rule := range s.rules { + for rule := range s.rules.Iter() { if rule.ObjectFilterExpr == "" { outRuleIds = append(outRuleIds, rule.Id) continue @@ -219,8 +113,13 @@ func (s *Source) evaluateRulesForObject(ctx context.Context, hostId, serviceId, err := retry.WithBackoff( ctx, func(ctx context.Context) error { - query := s.db.Rebind(rule.ObjectFilterExpr) - rows, err := s.db.NamedQueryContext(ctx, query, namedParams) + // The raw SQL query in the database is URL-encoded (mostly the space character is replaced by %20). + // So, we need to unescape it before passing it to the database. + query, err := url.QueryUnescape(rule.ObjectFilterExpr) + if err != nil { + return errors.Wrapf(err, "cannot unescape rule %d object filter expression %q", rule.Id, rule.ObjectFilterExpr) + } + rows, err := s.db.NamedQueryContext(ctx, s.db.Rebind(query), namedParams) if err != nil { return err } @@ -231,9 +130,10 @@ func (s *Source) evaluateRulesForObject(ctx context.Context, hostId, serviceId, } return nil }, - retry.Retryable, + func(_ error) bool { return false }, // Never retry an error, otherwise we'll block the worker unnecessarily. backoff.DefaultBackoff, - retry.Settings{Timeout: retry.DefaultTimeout}) + s.db.GetDefaultRetrySettings(), + ) if err == nil { outRuleIds = append(outRuleIds, rule.Id) @@ -274,45 +174,23 @@ func (s *Source) fetchHostServiceName(ctx context.Context, hostId, serviceId, en return } -// rawurlencode mimics PHP's rawurlencode to be used for parameter encoding. -// -// Icinga Web uses rawurldecode instead of urldecode, which, as its main difference, does not honor the plus char ('+') -// as a valid substitution for space (' '). Unfortunately, Go's url.QueryEscape does this very substitution and -// url.PathEscape does a bit too less and has a misleading name on top. +// buildCommonEvent creates an event.Event based on Host and (optional) Service names. // -// - https://www.php.net/manual/en/function.rawurlencode.php -// - https://github.com/php/php-src/blob/php-8.2.12/ext/standard/url.c#L538 -func rawurlencode(s string) string { - return strings.ReplaceAll(url.QueryEscape(s), "+", "%20") -} - -// buildCommonEvent creates an event.Event based on Host and (optional) Service attributes to be specified later. -// -// The new Event's Time will be the current timestamp. -// -// The following fields will NOT be populated and might be altered later: -// - Type -// - Severity -// - Username -// - Message -// - ID -func (s *Source) buildCommonEvent(host, service string) (*IcingaNotificationsEvent, error) { +// This function is used by all event builders to create a common event structure that includes the host and service +// names, the absolute URL to the Icinga Web 2 Icinga DB page for the host or service, and the tags for the event. +// Any event type-specific information (like severity, message, etc.) is added by the specific event builders. +func (s *Source) buildCommonEvent(host, service string) (*event.Event, error) { var ( eventName string eventUrl *url.URL eventTags map[string]string ) - eventUrl, err := url.Parse(s.IcingaWeb2BaseUrl) - if err != nil { - return nil, err - } - if service != "" { eventName = host + "!" + service - eventUrl = eventUrl.JoinPath("/icingadb/service") - eventUrl.RawQuery = "name=" + rawurlencode(service) + "&host.name=" + rawurlencode(host) + eventUrl = s.notificationsClient.JoinIcingaWeb2Path("/icingadb/service") + eventUrl.RawQuery = "name=" + utils.RawUrlEncode(service) + "&host.name=" + utils.RawUrlEncode(host) eventTags = map[string]string{ "host": host, @@ -321,269 +199,178 @@ func (s *Source) buildCommonEvent(host, service string) (*IcingaNotificationsEve } else { eventName = host - eventUrl = eventUrl.JoinPath("/icingadb/host") - eventUrl.RawQuery = "name=" + rawurlencode(host) + eventUrl = s.notificationsClient.JoinIcingaWeb2Path("/icingadb/host") + eventUrl.RawQuery = "name=" + utils.RawUrlEncode(host) eventTags = map[string]string{ "host": host, } } - return &IcingaNotificationsEvent{ + return &event.Event{ Name: eventName, URL: eventUrl.String(), Tags: eventTags, }, nil } -// buildStateHistoryEvent from a state history entry. -func (s *Source) buildStateHistoryEvent(ctx context.Context, h *v1history.StateHistory) (*IcingaNotificationsEvent, error) { +// buildStateHistoryEvent builds a fully initialized event.Event from a state history entry. +// +// The resulted event will have all the necessary information for a state change event, and must +// not be further modified by the caller. +func (s *Source) buildStateHistoryEvent(ctx context.Context, h *v1history.StateHistory) (*event.Event, error) { hostName, serviceName, err := s.fetchHostServiceName(ctx, h.HostId, h.ServiceId, h.EnvironmentId) if err != nil { return nil, errors.Wrap(err, "cannot fetch host/service information") } - event, err := s.buildCommonEvent(hostName, serviceName) + ev, err := s.buildCommonEvent(hostName, serviceName) if err != nil { return nil, errors.Wrapf(err, "cannot build event for %q,%q", hostName, serviceName) } - event.Type = TypeState + ev.Type = event.TypeState if serviceName != "" { switch h.HardState { case 0: - event.Severity = SeverityOK + ev.Severity = event.SeverityOK case 1: - event.Severity = SeverityWarning + ev.Severity = event.SeverityWarning case 2: - event.Severity = SeverityCrit + ev.Severity = event.SeverityCrit case 3: - event.Severity = SeverityErr + ev.Severity = event.SeverityErr default: return nil, fmt.Errorf("unexpected service state %d", h.HardState) } } else { switch h.HardState { case 0: - event.Severity = SeverityOK + ev.Severity = event.SeverityOK case 1: - event.Severity = SeverityCrit + ev.Severity = event.SeverityCrit default: return nil, fmt.Errorf("unexpected host state %d", h.HardState) } } if h.Output.Valid { - event.Message = h.Output.String + ev.Message = h.Output.String } if h.LongOutput.Valid { - event.Message += "\n" + h.LongOutput.String + ev.Message += "\n" + h.LongOutput.String } - return event, nil + return ev, nil } // buildDowntimeHistoryEvent from a downtime history entry. -func (s *Source) buildDowntimeHistoryEvent(ctx context.Context, h *v1history.DowntimeHistory) (*IcingaNotificationsEvent, error) { +func (s *Source) buildDowntimeHistoryEvent(ctx context.Context, h *v1history.DowntimeHistory) (*event.Event, error) { hostName, serviceName, err := s.fetchHostServiceName(ctx, h.HostId, h.ServiceId, h.EnvironmentId) if err != nil { return nil, errors.Wrap(err, "cannot fetch host/service information") } - event, err := s.buildCommonEvent(hostName, serviceName) + ev, err := s.buildCommonEvent(hostName, serviceName) if err != nil { return nil, errors.Wrapf(err, "cannot build event for %q,%q", hostName, serviceName) } if h.HasBeenCancelled.Valid && h.HasBeenCancelled.Bool { - event.Type = TypeDowntimeRemoved - event.Message = "Downtime was cancelled" + ev.Type = event.TypeDowntimeRemoved + ev.Message = "Downtime was cancelled" if h.CancelledBy.Valid { - event.Username = h.CancelledBy.String + ev.Username = h.CancelledBy.String } } else if h.EndTime.Time().Compare(time.Now()) <= 0 { - event.Type = TypeDowntimeEnd - event.Message = "Downtime expired" + ev.Type = event.TypeDowntimeEnd + ev.Message = "Downtime expired" } else { - event.Type = TypeDowntimeStart - event.Username = h.Author - event.Message = h.Comment - event.Mute = true - event.MuteReason = "Checkable is in downtime" + ev.Type = event.TypeDowntimeStart + ev.Username = h.Author + ev.Message = h.Comment + ev.Mute = types.MakeBool(true) + ev.MuteReason = "Checkable is in downtime" } - return event, nil + return ev, nil } // buildFlappingHistoryEvent from a flapping history entry. -func (s *Source) buildFlappingHistoryEvent(ctx context.Context, h *v1history.FlappingHistory) (*IcingaNotificationsEvent, error) { +func (s *Source) buildFlappingHistoryEvent(ctx context.Context, h *v1history.FlappingHistory) (*event.Event, error) { hostName, serviceName, err := s.fetchHostServiceName(ctx, h.HostId, h.ServiceId, h.EnvironmentId) if err != nil { return nil, errors.Wrap(err, "cannot fetch host/service information") } - event, err := s.buildCommonEvent(hostName, serviceName) + ev, err := s.buildCommonEvent(hostName, serviceName) if err != nil { return nil, errors.Wrapf(err, "cannot build event for %q,%q", hostName, serviceName) } if h.PercentStateChangeEnd.Valid { - event.Type = TypeFlappingEnd - event.Message = fmt.Sprintf( + ev.Type = event.TypeFlappingEnd + ev.Message = fmt.Sprintf( "Checkable stopped flapping (Current flapping value %.2f%% < low threshold %.2f%%)", h.PercentStateChangeEnd.Float64, h.FlappingThresholdLow) } else if h.PercentStateChangeStart.Valid { - event.Type = TypeFlappingStart - event.Message = fmt.Sprintf( + ev.Type = event.TypeFlappingStart + ev.Message = fmt.Sprintf( "Checkable started flapping (Current flapping value %.2f%% > high threshold %.2f%%)", h.PercentStateChangeStart.Float64, h.FlappingThresholdHigh) - event.Mute = true - event.MuteReason = "Checkable is flapping" + ev.Mute = types.MakeBool(true) + ev.MuteReason = "Checkable is flapping" } else { return nil, errors.New("flapping history entry has neither percent_state_change_start nor percent_state_change_end") } - return event, nil + return ev, nil } -// buildAcknowledgementHistoryEvent from an acknowledgement history entry. -func (s *Source) buildAcknowledgementHistoryEvent(ctx context.Context, h *v1history.AcknowledgementHistory) (*IcingaNotificationsEvent, error) { +// buildAcknowledgementHistoryEvent from an acknowledgment history entry. +func (s *Source) buildAcknowledgementHistoryEvent(ctx context.Context, h *v1history.AcknowledgementHistory) (*event.Event, error) { hostName, serviceName, err := s.fetchHostServiceName(ctx, h.HostId, h.ServiceId, h.EnvironmentId) if err != nil { return nil, errors.Wrap(err, "cannot fetch host/service information") } - event, err := s.buildCommonEvent(hostName, serviceName) + ev, err := s.buildCommonEvent(hostName, serviceName) if err != nil { return nil, errors.Wrapf(err, "cannot build event for %q,%q", hostName, serviceName) } if !h.ClearTime.Time().IsZero() { - event.Type = TypeAcknowledgementCleared - event.Message = "Checkable was cleared" + ev.Type = event.TypeAcknowledgementCleared + ev.Message = "Checkable was cleared" if h.ClearedBy.Valid { - event.Username = h.ClearedBy.String + ev.Username = h.ClearedBy.String } } else if !h.SetTime.Time().IsZero() { - event.Type = TypeAcknowledgementSet + ev.Type = event.TypeAcknowledgementSet if h.Comment.Valid { - event.Message = h.Comment.String + ev.Message = h.Comment.String } else { - event.Message = "Checkable was acknowledged" + ev.Message = "Checkable was acknowledged" } if h.Author.Valid { - event.Username = h.Author.String + ev.Username = h.Author.String } } else { return nil, errors.New("acknowledgment history entry has neither a set_time nor a clear_time") } - return event, nil -} - -// submitEvent to the Icinga Notifications /process-event API endpoint. -// -// The event will be passed together with the Source.ruleVersion and all evaluated ruleIds to the endpoint. Even if no -// rules were evaluated, this method should be called. Thus, Icinga Notifications can dismiss the event, but Icinga DB -// would still be informed in case of a rule change. Otherwise, events might be dropped here which are now required. -// -// This method may return an ErrRulesOutdated error, implying that the Source.ruleVersion mismatches the version stored -// at Icinga Notifications. In this case, the rules must be refetched and the event requires another evaluation. -func (s *Source) submitEvent(ctx context.Context, client *http.Client, event *IcingaNotificationsEvent, ruleIds []int64) error { - jsonBody, err := json.Marshal(event) - if err != nil { - return errors.Wrap(err, "cannot encode event to JSON") - } - - apiUrl, err := url.JoinPath(s.ApiBaseUrl, "/process-event") - if err != nil { - return errors.Wrap(err, "cannot join API URL") - } - req, err := http.NewRequestWithContext(ctx, http.MethodPost, apiUrl, bytes.NewBuffer(jsonBody)) - if err != nil { - return errors.Wrap(err, "cannot create HTTP request") - } - req.SetBasicAuth(s.User, s.Password) - - s.rulesMutex.RLock() - ruleVersion := s.ruleVersion - s.rulesMutex.RUnlock() - - ruleIdsStrArr := make([]string, 0, len(ruleIds)) - for _, idInt := range ruleIds { - ruleIdsStrArr = append(ruleIdsStrArr, fmt.Sprintf("%d", idInt)) - } - ruleIdsStr := strings.Join(ruleIdsStrArr, ",") - - req.Header.Set("X-Rule-Ids", ruleIdsStr) - req.Header.Set("X-Rule-Version", ruleVersion) - - resp, err := client.Do(req) - if err != nil { - return errors.Wrap(err, "cannot POST HTTP request to Icinga Notifications") - } - - defer func() { - _, _ = io.Copy(io.Discard, resp.Body) - _ = resp.Body.Close() - }() - - // Refetching rules required. - if resp.StatusCode == http.StatusFailedDependency { - return ErrRulesOutdated - } - - // 200s are acceptable. - if 200 <= resp.StatusCode && resp.StatusCode <= 299 { - return nil - } - - // Ignoring superfluous state change. - if resp.StatusCode == http.StatusNotAcceptable { - return nil - } - - var buff bytes.Buffer - _, _ = io.Copy(&buff, &io.LimitedReader{ - R: resp.Body, - N: 1 << 20, // Limit the error message's length against memory exhaustion - }) - return fmt.Errorf("unexpected response from Icinga Notificatios, status %q (%d): %q", - resp.Status, resp.StatusCode, strings.TrimSpace(buff.String())) + return ev, nil } // worker is the background worker launched by NewNotificationsSource. func (s *Source) worker() { defer s.ctxCancel() - client := &http.Client{} - - if err := retry.WithBackoff( - s.ctx, - func(ctx context.Context) error { return s.fetchRules(s.ctx, client) }, - func(_ error) bool { return true }, // For the moment, retry every potential error. - backoff.DefaultBackoff, - retry.Settings{ - Timeout: retry.DefaultTimeout, - OnRetryableError: func(elapsed time.Duration, attempt uint64, err, lastErr error) { - s.logger.Errorw("Cannot fetch rules from Icinga Notifications", - zap.Duration("elapsed", elapsed), - zap.Uint64("attempt", attempt), - zap.Error(err)) - }, - OnSuccess: func(_ time.Duration, attempt uint64, _ error) { - s.logger.Infow("Fetched rules from Icinga Notifications", zap.Uint64("attempt", attempt)) - }, - }, - ); err != nil { - s.logger.Fatalw("Cannot fetch rules from Icinga Notifications", zap.Error(err)) - } - for { select { case <-s.ctx.Done(): @@ -591,7 +378,7 @@ func (s *Source) worker() { case entity := <-s.inputCh: var ( - event *IcingaNotificationsEvent + ev *event.Event eventErr error metaHistory v1history.HistoryTableMeta ) @@ -607,11 +394,11 @@ func (s *Source) worker() { continue } - event, eventErr = s.buildStateHistoryEvent(s.ctx, h) + ev, eventErr = s.buildStateHistoryEvent(s.ctx, h) metaHistory = h.HistoryTableMeta case *v1history.DowntimeHistory: - event, eventErr = s.buildDowntimeHistoryEvent(s.ctx, h) + ev, eventErr = s.buildDowntimeHistoryEvent(s.ctx, h) metaHistory = h.HistoryTableMeta case *v1history.CommentHistory: @@ -619,11 +406,11 @@ func (s *Source) worker() { continue case *v1history.FlappingHistory: - event, eventErr = s.buildFlappingHistoryEvent(s.ctx, h) + ev, eventErr = s.buildFlappingHistoryEvent(s.ctx, h) metaHistory = h.HistoryTableMeta case *v1history.AcknowledgementHistory: - event, eventErr = s.buildAcknowledgementHistoryEvent(s.ctx, h) + ev, eventErr = s.buildAcknowledgementHistoryEvent(s.ctx, h) metaHistory = h.HistoryTableMeta default: @@ -638,12 +425,19 @@ func (s *Source) worker() { zap.Error(eventErr)) continue } - if event == nil { + if ev == nil { s.logger.Error("No event was fetched, but no error was reported. This REALLY SHOULD NOT happen.") continue } - eventLogger := s.logger.With(zap.Object("event", event)) + eventLogger := s.logger.With(zap.Object( + "event", + zapcore.ObjectMarshalerFunc(func(encoder zapcore.ObjectEncoder) error { + encoder.AddString("name", ev.Name) + encoder.AddString("type", ev.Type.String()) + return nil + }), + )) eventRuleIds, err := s.evaluateRulesForObject( s.ctx, @@ -657,13 +451,16 @@ func (s *Source) worker() { eventLogger = eventLogger.With(zap.Any("rules", eventRuleIds)) - err = s.submitEvent(s.ctx, client, event, eventRuleIds) - if errors.Is(err, ErrRulesOutdated) { - s.logger.Info("Icinga Notification rules were updated, triggering resync") + s.rulesMutex.RLock() + ruleVersion := s.rules.Version + s.rulesMutex.RUnlock() + + newEventRules, err := s.notificationsClient.ProcessEvent(s.ctx, ev, ruleVersion, eventRuleIds...) + if errors.Is(err, notifications.ErrRulesOutdated) { + s.rulesMutex.Lock() + s.rules = newEventRules + s.rulesMutex.Unlock() - if err := s.fetchRules(s.ctx, client); err != nil { - s.logger.Errorw("Cannot fetch rules from Icinga Notifications", zap.Error(err)) - } go s.Submit(entity) continue From 50750cf287c0a7ff997a5782387243984e960904 Mon Sep 17 00:00:00 2001 From: Yonas Habteab Date: Wed, 6 Aug 2025 12:04:24 +0200 Subject: [PATCH 03/38] Don't limit queries referncing to `{host,service}_id` & Env ID params Instead allow them to reference any columns of the database entity as long as that entity provides it. It also removes the retry mechanism used to execute the queries as this would block the worker unnecessarily. --- pkg/notifications/notifications.go | 85 ++++++++++++------------------ 1 file changed, 34 insertions(+), 51 deletions(-) diff --git a/pkg/notifications/notifications.go b/pkg/notifications/notifications.go index 59a5ff12e..d2d77a905 100644 --- a/pkg/notifications/notifications.go +++ b/pkg/notifications/notifications.go @@ -84,58 +84,47 @@ func NewNotificationsSource( // query returns at least one line, the rule will match. Rules with an empty ObjectFilterExpr are a special case and // will always match. // -// For the queries, the following mapping is performed: -// - :host_id <- hostId -// - :service_id <- serviceId -// - :environment_id <- environmentId +// The provided entity is passed as param to the queries, thus they are allowed to use all fields of that specific +// entity. Cross-table column references are not supported unless the provided entity provides the fields in one way +// or another. // // This allows a query like the following: // // > select * from host where id = :host_id and environment_id = :environment_id and name like 'prefix_%' -func (s *Source) evaluateRulesForObject(ctx context.Context, hostId, serviceId, environmentId types.Binary) ([]int64, error) { +// +// The :host_id and :environment_id parameters will be bound to the entity's ID and EnvironmentId fields, respectively. +func (s *Source) evaluateRulesForObject(ctx context.Context, entity database.Entity) ([]int64, error) { s.rulesMutex.RLock() defer s.rulesMutex.RUnlock() outRuleIds := make([]int64, 0, len(s.rules.Rules)) - namedParams := map[string]any{ - "host_id": hostId, - "service_id": serviceId, - "environment_id": environmentId, - } - for rule := range s.rules.Iter() { if rule.ObjectFilterExpr == "" { outRuleIds = append(outRuleIds, rule.Id) continue } - err := retry.WithBackoff( - ctx, - func(ctx context.Context) error { - // The raw SQL query in the database is URL-encoded (mostly the space character is replaced by %20). - // So, we need to unescape it before passing it to the database. - query, err := url.QueryUnescape(rule.ObjectFilterExpr) - if err != nil { - return errors.Wrapf(err, "cannot unescape rule %d object filter expression %q", rule.Id, rule.ObjectFilterExpr) - } - rows, err := s.db.NamedQueryContext(ctx, s.db.Rebind(query), namedParams) - if err != nil { - return err - } - defer func() { _ = rows.Close() }() + run := func() error { + // The raw SQL query in the database is URL-encoded (mostly the space character is replaced by %20). + // So, we need to unescape it before passing it to the database. + query, err := url.QueryUnescape(rule.ObjectFilterExpr) + if err != nil { + return errors.Wrapf(err, "cannot unescape rule %d object filter expression %q", rule.Id, rule.ObjectFilterExpr) + } + rows, err := s.db.NamedQueryContext(ctx, s.db.Rebind(query), entity) + if err != nil { + return err + } + defer func() { _ = rows.Close() }() - if !rows.Next() { - return sql.ErrNoRows - } - return nil - }, - func(_ error) bool { return false }, // Never retry an error, otherwise we'll block the worker unnecessarily. - backoff.DefaultBackoff, - s.db.GetDefaultRetrySettings(), - ) - - if err == nil { + if !rows.Next() { + return sql.ErrNoRows + } + return nil + } + + if err := run(); err == nil { outRuleIds = append(outRuleIds, rule.Id) } else if errors.Is(err, sql.ErrNoRows) { continue @@ -376,12 +365,14 @@ func (s *Source) worker() { case <-s.ctx.Done(): return - case entity := <-s.inputCh: - var ( - ev *event.Event - eventErr error - metaHistory v1history.HistoryTableMeta - ) + case entity, more := <-s.inputCh: + if !more { // Should never happen, but just in case. + s.logger.Debug("Input channel closed, stopping worker") + return + } + + var ev *event.Event + var eventErr error // Keep the type switch in sync with syncPipelines from pkg/icingadb/history/sync.go switch h := entity.(type) { @@ -395,11 +386,9 @@ func (s *Source) worker() { } ev, eventErr = s.buildStateHistoryEvent(s.ctx, h) - metaHistory = h.HistoryTableMeta case *v1history.DowntimeHistory: ev, eventErr = s.buildDowntimeHistoryEvent(s.ctx, h) - metaHistory = h.HistoryTableMeta case *v1history.CommentHistory: // Ignore for the moment. @@ -407,11 +396,9 @@ func (s *Source) worker() { case *v1history.FlappingHistory: ev, eventErr = s.buildFlappingHistoryEvent(s.ctx, h) - metaHistory = h.HistoryTableMeta case *v1history.AcknowledgementHistory: ev, eventErr = s.buildAcknowledgementHistoryEvent(s.ctx, h) - metaHistory = h.HistoryTableMeta default: s.logger.Error("Cannot process unsupported type", @@ -439,11 +426,7 @@ func (s *Source) worker() { }), )) - eventRuleIds, err := s.evaluateRulesForObject( - s.ctx, - metaHistory.HostId, - metaHistory.ServiceId, - metaHistory.EnvironmentId) + eventRuleIds, err := s.evaluateRulesForObject(s.ctx, entity) if err != nil { eventLogger.Errorw("Cannot evaluate rules for event", zap.Error(err)) continue From 66dd8a35f45b91b3f9c8efa5243e87f896415fa5 Mon Sep 17 00:00:00 2001 From: Yonas Habteab Date: Wed, 6 Aug 2025 12:11:44 +0200 Subject: [PATCH 04/38] Reevaluate rules immediately after refetching them Otherwise, posting the entity in a `go s.Submit(entity)` manner in the background will mess up the order of events as there might be another even in the queue affecting the same entity. Apart from that, the log entry "submitted event ..." is also downgraded to debug level, as it creates too much noise at the info level without saying anything relevant to an end user. --- pkg/notifications/notifications.go | 15 +++++++++------ 1 file changed, 9 insertions(+), 6 deletions(-) diff --git a/pkg/notifications/notifications.go b/pkg/notifications/notifications.go index d2d77a905..e43d70f45 100644 --- a/pkg/notifications/notifications.go +++ b/pkg/notifications/notifications.go @@ -426,14 +426,13 @@ func (s *Source) worker() { }), )) + reevaluateRules: eventRuleIds, err := s.evaluateRulesForObject(s.ctx, entity) if err != nil { eventLogger.Errorw("Cannot evaluate rules for event", zap.Error(err)) continue } - eventLogger = eventLogger.With(zap.Any("rules", eventRuleIds)) - s.rulesMutex.RLock() ruleVersion := s.rules.Version s.rulesMutex.RUnlock() @@ -444,15 +443,19 @@ func (s *Source) worker() { s.rules = newEventRules s.rulesMutex.Unlock() - go s.Submit(entity) + eventLogger.Debugw("Re-evaluating rules for event after fetching new rules", zap.String("rules_version", s.rules.Version)) - continue + // Re-evaluate the just fetched rules for the current event. + goto reevaluateRules } else if err != nil { - eventLogger.Errorw("Cannot submit event to Icinga Notifications", zap.Error(err)) + eventLogger.Errorw("Cannot submit event to Icinga Notifications", + zap.String("rules_version", s.rules.Version), + zap.Any("rules", eventRuleIds), + zap.Error(err)) continue } - eventLogger.Info("Submitted event to Icinga Notifications") + eventLogger.Debugw("Successfully submitted event to Icinga Notifications", zap.Any("rules", eventRuleIds)) } } } From b5be0319fa5cdfb1ce569d0f3e1beeb6ef751c74 Mon Sep 17 00:00:00 2001 From: Yonas Habteab Date: Wed, 6 Aug 2025 16:05:04 +0200 Subject: [PATCH 05/38] Retrieve host and service names from Redis Instead of retrieving the host and service names from the used RDBMs, this commit allows us to query them from Redis. This is done to avoid the overhead of database queries, especially when the host and service names are always to be found in Redis. The previous implementation simply perfomed two database queries with each received entity based on their IDs, but we can perform this operation more efficiently from Redis using the same filtering logic as before. Of course, we now have to maintain more code needed to handle the Redis operations, but this is a trade-off we should be willing to make for performance reasons. --- cmd/icingadb/main.go | 1 + go.mod | 2 +- pkg/notifications/notifications.go | 87 +++++++++++------------------ pkg/notifications/redis_fetch.go | 89 ++++++++++++++++++++++++++++++ 4 files changed, 122 insertions(+), 57 deletions(-) create mode 100644 pkg/notifications/redis_fetch.go diff --git a/cmd/icingadb/main.go b/cmd/icingadb/main.go index bd2a14ce1..c7fd403cd 100644 --- a/cmd/icingadb/main.go +++ b/cmd/icingadb/main.go @@ -177,6 +177,7 @@ func run() int { notificationsSource := notifications.NewNotificationsSource( ctx, db, + rc, logs.GetChildLogger("notifications-source"), cfg) notificationsSourceCallback = notificationsSource.Submit diff --git a/go.mod b/go.mod index cb6a0ac41..92d2f81b7 100644 --- a/go.mod +++ b/go.mod @@ -13,6 +13,7 @@ require ( github.com/mattn/go-sqlite3 v1.14.32 github.com/okzk/sdnotify v0.0.0-20180710141335-d9becc38acbd github.com/pkg/errors v0.9.1 + github.com/redis/go-redis/v9 v9.10.0 github.com/stretchr/testify v1.11.1 github.com/vbauerster/mpb/v6 v6.0.4 go.uber.org/zap v1.27.0 @@ -34,7 +35,6 @@ require ( github.com/mattn/go-isatty v0.0.20 // indirect github.com/mattn/go-runewidth v0.0.12 // indirect github.com/pmezard/go-difflib v1.0.0 // indirect - github.com/redis/go-redis/v9 v9.10.0 // indirect github.com/rivo/uniseg v0.2.0 // indirect github.com/ssgreg/journald v1.0.0 // indirect go.uber.org/multierr v1.11.0 // indirect diff --git a/pkg/notifications/notifications.go b/pkg/notifications/notifications.go index e43d70f45..416558819 100644 --- a/pkg/notifications/notifications.go +++ b/pkg/notifications/notifications.go @@ -8,12 +8,11 @@ import ( "sync" "time" - "github.com/icinga/icinga-go-library/backoff" "github.com/icinga/icinga-go-library/database" "github.com/icinga/icinga-go-library/logging" "github.com/icinga/icinga-go-library/notifications" "github.com/icinga/icinga-go-library/notifications/event" - "github.com/icinga/icinga-go-library/retry" + "github.com/icinga/icinga-go-library/redis" "github.com/icinga/icinga-go-library/types" "github.com/icinga/icinga-go-library/utils" "github.com/icinga/icingadb/pkg/common" @@ -41,6 +40,7 @@ type Source struct { ctxCancel context.CancelFunc notificationsClient *notifications.Client // The Icinga Notifications client used to interact with the API. + redisClient *redis.Client // redisClient is the Redis client used to fetch host and service names for events. } // NewNotificationsSource creates a new Source connected to an existing database and logger. @@ -49,6 +49,7 @@ type Source struct { func NewNotificationsSource( ctx context.Context, db *database.DB, + rc *redis.Client, logger *logging.Logger, cfg notifications.Config, ) *Source { @@ -61,7 +62,8 @@ func NewNotificationsSource( db: db, logger: logger, - rules: ¬ifications.SourceRulesInfo{Version: notifications.EmptyRulesVersion}, + rules: ¬ifications.SourceRulesInfo{Version: notifications.EmptyRulesVersion}, + redisClient: rc, ctx: ctx, ctxCancel: ctxCancel, @@ -136,63 +138,36 @@ func (s *Source) evaluateRulesForObject(ctx context.Context, entity database.Ent return outRuleIds[:len(outRuleIds):len(outRuleIds)], nil } -// fetchHostServiceName for a host ID and a potential service ID from the Icinga DB relational database. -func (s *Source) fetchHostServiceName(ctx context.Context, hostId, serviceId, envId types.Binary) (host, service string, err error) { - err = retry.WithBackoff( - ctx, - func(ctx context.Context) error { - queryHost := s.db.Rebind("SELECT name FROM host WHERE id = ? AND environment_id = ?") - err := s.db.QueryRowxContext(ctx, queryHost, hostId, envId).Scan(&host) - if err != nil { - return errors.Wrap(err, "cannot select host") - } - - if serviceId != nil { - queryService := s.db.Rebind("SELECT name FROM service WHERE id = ? AND environment_id = ?") - err := s.db.QueryRowxContext(ctx, queryService, serviceId, envId).Scan(&service) - if err != nil { - return errors.Wrap(err, "cannot select service") - } - } - - return nil - }, - retry.Retryable, - backoff.DefaultBackoff, - retry.Settings{Timeout: retry.DefaultTimeout}) - return -} - // buildCommonEvent creates an event.Event based on Host and (optional) Service names. // // This function is used by all event builders to create a common event structure that includes the host and service // names, the absolute URL to the Icinga Web 2 Icinga DB page for the host or service, and the tags for the event. // Any event type-specific information (like severity, message, etc.) is added by the specific event builders. -func (s *Source) buildCommonEvent(host, service string) (*event.Event, error) { +func (s *Source) buildCommonEvent(rlr *redisLookupResult) (*event.Event, error) { var ( eventName string eventUrl *url.URL eventTags map[string]string ) - if service != "" { - eventName = host + "!" + service + if rlr.ServiceName != "" { + eventName = rlr.HostName + "!" + rlr.ServiceName eventUrl = s.notificationsClient.JoinIcingaWeb2Path("/icingadb/service") - eventUrl.RawQuery = "name=" + utils.RawUrlEncode(service) + "&host.name=" + utils.RawUrlEncode(host) + eventUrl.RawQuery = "name=" + utils.RawUrlEncode(rlr.ServiceName) + "&host.name=" + utils.RawUrlEncode(rlr.HostName) eventTags = map[string]string{ - "host": host, - "service": service, + "host": rlr.HostName, + "service": rlr.ServiceName, } } else { - eventName = host + eventName = rlr.HostName eventUrl = s.notificationsClient.JoinIcingaWeb2Path("/icingadb/host") - eventUrl.RawQuery = "name=" + utils.RawUrlEncode(host) + eventUrl.RawQuery = "name=" + utils.RawUrlEncode(rlr.HostName) eventTags = map[string]string{ - "host": host, + "host": rlr.HostName, } } @@ -208,19 +183,19 @@ func (s *Source) buildCommonEvent(host, service string) (*event.Event, error) { // The resulted event will have all the necessary information for a state change event, and must // not be further modified by the caller. func (s *Source) buildStateHistoryEvent(ctx context.Context, h *v1history.StateHistory) (*event.Event, error) { - hostName, serviceName, err := s.fetchHostServiceName(ctx, h.HostId, h.ServiceId, h.EnvironmentId) + res, err := s.fetchHostServiceName(ctx, h.HostId, h.ServiceId) if err != nil { - return nil, errors.Wrap(err, "cannot fetch host/service information") + return nil, err } - ev, err := s.buildCommonEvent(hostName, serviceName) + ev, err := s.buildCommonEvent(res) if err != nil { - return nil, errors.Wrapf(err, "cannot build event for %q,%q", hostName, serviceName) + return nil, errors.Wrapf(err, "cannot build event for %q,%q", res.HostName, res.ServiceName) } ev.Type = event.TypeState - if serviceName != "" { + if res.ServiceName != "" { switch h.HardState { case 0: ev.Severity = event.SeverityOK @@ -256,14 +231,14 @@ func (s *Source) buildStateHistoryEvent(ctx context.Context, h *v1history.StateH // buildDowntimeHistoryEvent from a downtime history entry. func (s *Source) buildDowntimeHistoryEvent(ctx context.Context, h *v1history.DowntimeHistory) (*event.Event, error) { - hostName, serviceName, err := s.fetchHostServiceName(ctx, h.HostId, h.ServiceId, h.EnvironmentId) + res, err := s.fetchHostServiceName(ctx, h.HostId, h.ServiceId) if err != nil { - return nil, errors.Wrap(err, "cannot fetch host/service information") + return nil, err } - ev, err := s.buildCommonEvent(hostName, serviceName) + ev, err := s.buildCommonEvent(res) if err != nil { - return nil, errors.Wrapf(err, "cannot build event for %q,%q", hostName, serviceName) + return nil, errors.Wrapf(err, "cannot build event for %q,%q", res.HostName, res.ServiceName) } if h.HasBeenCancelled.Valid && h.HasBeenCancelled.Bool { @@ -289,14 +264,14 @@ func (s *Source) buildDowntimeHistoryEvent(ctx context.Context, h *v1history.Dow // buildFlappingHistoryEvent from a flapping history entry. func (s *Source) buildFlappingHistoryEvent(ctx context.Context, h *v1history.FlappingHistory) (*event.Event, error) { - hostName, serviceName, err := s.fetchHostServiceName(ctx, h.HostId, h.ServiceId, h.EnvironmentId) + res, err := s.fetchHostServiceName(ctx, h.HostId, h.ServiceId) if err != nil { - return nil, errors.Wrap(err, "cannot fetch host/service information") + return nil, err } - ev, err := s.buildCommonEvent(hostName, serviceName) + ev, err := s.buildCommonEvent(res) if err != nil { - return nil, errors.Wrapf(err, "cannot build event for %q,%q", hostName, serviceName) + return nil, errors.Wrapf(err, "cannot build event for %q,%q", res.HostName, res.ServiceName) } if h.PercentStateChangeEnd.Valid { @@ -320,14 +295,14 @@ func (s *Source) buildFlappingHistoryEvent(ctx context.Context, h *v1history.Fla // buildAcknowledgementHistoryEvent from an acknowledgment history entry. func (s *Source) buildAcknowledgementHistoryEvent(ctx context.Context, h *v1history.AcknowledgementHistory) (*event.Event, error) { - hostName, serviceName, err := s.fetchHostServiceName(ctx, h.HostId, h.ServiceId, h.EnvironmentId) + res, err := s.fetchHostServiceName(ctx, h.HostId, h.ServiceId) if err != nil { - return nil, errors.Wrap(err, "cannot fetch host/service information") + return nil, err } - ev, err := s.buildCommonEvent(hostName, serviceName) + ev, err := s.buildCommonEvent(res) if err != nil { - return nil, errors.Wrapf(err, "cannot build event for %q,%q", hostName, serviceName) + return nil, errors.Wrapf(err, "cannot build event for %q,%q", res.HostName, res.ServiceName) } if !h.ClearTime.Time().IsZero() { diff --git a/pkg/notifications/redis_fetch.go b/pkg/notifications/redis_fetch.go new file mode 100644 index 000000000..8d61d7760 --- /dev/null +++ b/pkg/notifications/redis_fetch.go @@ -0,0 +1,89 @@ +package notifications + +import ( + "context" + "encoding/json" + "errors" + "fmt" + "time" + + "github.com/icinga/icinga-go-library/backoff" + "github.com/icinga/icinga-go-library/retry" + "github.com/icinga/icinga-go-library/types" + "github.com/redis/go-redis/v9" +) + +// fetchHostServiceName retrieves the host and service names from Redis. +// +// It uses either the hostId or/and serviceId to fetch the corresponding names. If both are provided, +// the returned result will contain the host name and the service name accordingly. Otherwise, it will +// only contain the host name. +// +// Internally, it uses the Redis HGet command to fetch the data from the "icinga:host" and "icinga:service" hashes. +// If this operation couldn't be completed within a reasonable time (a hard coded 5 seconds), it will cancel the +// request and return an error indicating that the operation timed out. In case of the serviceId being set, the +// maximum execution time of the Redis HGet commands is 10s (5s for each HGet call). +func (s *Source) fetchHostServiceName(ctx context.Context, hostId, serviceId types.Binary) (*redisLookupResult, error) { + redisHGet := func(typ, field string, out *redisLookupResult) error { + ctx, cancel := context.WithTimeout(ctx, 5*time.Second) + defer cancel() + + err := retry.WithBackoff( + ctx, + func(ctx context.Context) error { return s.redisClient.HGet(ctx, "icinga:"+typ, field).Scan(out) }, + retry.Retryable, + backoff.DefaultBackoff, + retry.Settings{}, + ) + if err != nil { + if errors.Is(err, redis.Nil) { + return fmt.Errorf("%s with ID %s not found in Redis", typ, hostId) + } + return fmt.Errorf("failed to fetch %s with ID %s from Redis: %w", typ, field, err) + } + return nil + } + + var result redisLookupResult + if err := redisHGet("host", hostId.String(), &result); err != nil { + return nil, err + } + + result.HostName = result.Name + result.Name = "" // Clear the name field for the host, as we will fetch the service name next. + + if serviceId != nil { + if err := redisHGet("service", serviceId.String(), &result); err != nil { + return nil, err + } + result.ServiceName = result.Name + result.Name = "" // It's not needed anymore, clear it! + } + + return &result, nil +} + +// redisLookupResult defines the structure of the Redis message we're interested in. +type redisLookupResult struct { + HostName string `json:"-"` // Name of the host (never empty). + ServiceName string `json:"-"` // Name of the service (only set in service context). + + // Name is used to retrieve the host or service name from Redis. + // It should not be used for any other purpose apart from within the [Source.fetchHostServiceName] function. + Name string `json:"name"` +} + +// UnmarshalBinary implements the [encoding.BinaryUnmarshaler] interface for redisLookupResult. +// +// It unmarshals the binary data of the Redis HGet result into the redisLookupResult struct. +// This is required for the HGet().Scan() usage in the [Source.fetchHostServiceName] function to work correctly. +func (rlr *redisLookupResult) UnmarshalBinary(data []byte) error { + if len(data) == 0 { + return errors.New("empty data received for redisLookupResult") + } + + if err := json.Unmarshal(data, rlr); err != nil { + return fmt.Errorf("failed to unmarshal redis result: %w", err) + } + return nil +} From cd435af6d0a9fc8e9e4034ae611d20e10407c9e5 Mon Sep 17 00:00:00 2001 From: Yonas Habteab Date: Thu, 7 Aug 2025 09:11:33 +0200 Subject: [PATCH 06/38] Drop superfluous `rulesMutex` There won't be any concurrent access to the rules, so we don't need to guard it with a mutex. --- pkg/notifications/notifications.go | 15 ++------------- 1 file changed, 2 insertions(+), 13 deletions(-) diff --git a/pkg/notifications/notifications.go b/pkg/notifications/notifications.go index 416558819..5a469ecef 100644 --- a/pkg/notifications/notifications.go +++ b/pkg/notifications/notifications.go @@ -5,7 +5,6 @@ import ( "database/sql" "fmt" "net/url" - "sync" "time" "github.com/icinga/icinga-go-library/database" @@ -33,8 +32,7 @@ type Source struct { db *database.DB logger *logging.Logger - rules *notifications.SourceRulesInfo // rules holds the latest rules fetched from Icinga Notifications. - rulesMutex sync.RWMutex // rulesMutex protects access to the rules field. + rules *notifications.SourceRulesInfo // rules holds the latest rules fetched from Icinga Notifications. ctx context.Context ctxCancel context.CancelFunc @@ -96,9 +94,6 @@ func NewNotificationsSource( // // The :host_id and :environment_id parameters will be bound to the entity's ID and EnvironmentId fields, respectively. func (s *Source) evaluateRulesForObject(ctx context.Context, entity database.Entity) ([]int64, error) { - s.rulesMutex.RLock() - defer s.rulesMutex.RUnlock() - outRuleIds := make([]int64, 0, len(s.rules.Rules)) for rule := range s.rules.Iter() { @@ -408,15 +403,9 @@ func (s *Source) worker() { continue } - s.rulesMutex.RLock() - ruleVersion := s.rules.Version - s.rulesMutex.RUnlock() - - newEventRules, err := s.notificationsClient.ProcessEvent(s.ctx, ev, ruleVersion, eventRuleIds...) + newEventRules, err := s.notificationsClient.ProcessEvent(s.ctx, ev, s.rules.Version, eventRuleIds...) if errors.Is(err, notifications.ErrRulesOutdated) { - s.rulesMutex.Lock() s.rules = newEventRules - s.rulesMutex.Unlock() eventLogger.Debugw("Re-evaluating rules for event after fetching new rules", zap.String("rules_version", s.rules.Version)) From 6822d1129989d208b607d3101e6be38d01a9517a Mon Sep 17 00:00:00 2001 From: Yonas Habteab Date: Fri, 8 Aug 2025 15:08:06 +0200 Subject: [PATCH 07/38] go.mod(WIP): require not yet merged branch of IGL WIP because we might move the code elsewhere. --- go.mod | 4 ++-- go.sum | 8 ++++---- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/go.mod b/go.mod index 92d2f81b7..dcea1432e 100644 --- a/go.mod +++ b/go.mod @@ -7,14 +7,14 @@ require ( github.com/goccy/go-yaml v1.13.0 github.com/google/go-cmp v0.7.0 github.com/google/uuid v1.6.0 - github.com/icinga/icinga-go-library v0.7.2 + github.com/icinga/icinga-go-library v0.7.3-0.20250807134650-55c038b220d8 github.com/jessevdk/go-flags v1.6.1 github.com/jmoiron/sqlx v1.4.0 github.com/mattn/go-sqlite3 v1.14.32 github.com/okzk/sdnotify v0.0.0-20180710141335-d9becc38acbd github.com/pkg/errors v0.9.1 - github.com/redis/go-redis/v9 v9.10.0 github.com/stretchr/testify v1.11.1 + github.com/redis/go-redis/v9 v9.11.0 github.com/vbauerster/mpb/v6 v6.0.4 go.uber.org/zap v1.27.0 golang.org/x/sync v0.17.0 diff --git a/go.sum b/go.sum index c849198af..118d146ce 100644 --- a/go.sum +++ b/go.sum @@ -37,8 +37,8 @@ github.com/google/go-cmp v0.7.0 h1:wk8382ETsv4JYUZwIsn6YpYiWiBsYLSJiTsyBybVuN8= github.com/google/go-cmp v0.7.0/go.mod h1:pXiqmnSA92OHEEa9HXL2W4E7lf9JzCmGVUdgjX3N/iU= github.com/google/uuid v1.6.0 h1:NIvaJDMOsjHA8n1jAhLSgzrAzy1Hgr+hNrb57e+94F0= github.com/google/uuid v1.6.0/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= -github.com/icinga/icinga-go-library v0.7.2 h1:6ilUeE9F9OqxxJXNR9URWDf6zOqsdhjjR9w1MUXY9Kg= -github.com/icinga/icinga-go-library v0.7.2/go.mod h1:HZTiYD+N+9FZIVpPdUEJWJnc6sLvrIRO03jvkdkmUEU= +github.com/icinga/icinga-go-library v0.7.3-0.20250807134650-55c038b220d8 h1:YStwl7OlLUN87ROcZ8LQRTDKBhEx0V7EX5+5OMRwEnM= +github.com/icinga/icinga-go-library v0.7.3-0.20250807134650-55c038b220d8/go.mod h1:6xgV9o7JcGVg2I6CzPfefQZF4Ev8QeZnysiu82YaRY4= github.com/jessevdk/go-flags v1.6.1 h1:Cvu5U8UGrLay1rZfv/zP7iLpSHGUZ/Ou68T0iX1bBK4= github.com/jessevdk/go-flags v1.6.1/go.mod h1:Mk8T1hIAWpOiJiHa9rJASDK2UGWji0EuPGBnNLMooyc= github.com/jmoiron/sqlx v1.4.0 h1:1PLqN7S1UYp5t4SrVVnt4nUVNemrDAtxlulVe+Qgm3o= @@ -63,8 +63,8 @@ github.com/pkg/errors v0.9.1 h1:FEBLx1zS214owpjy7qsBeixbURkuhQAwrK5UwLGTwt4= github.com/pkg/errors v0.9.1/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= github.com/pmezard/go-difflib v1.0.0 h1:4DBwDE0NGyQoBHbLQYPwSUPoCMWR5BEzIk/f1lZbAQM= github.com/pmezard/go-difflib v1.0.0/go.mod h1:iKH77koFhYxTK1pcRnkKkqfTogsbg7gZNVY4sRDYZ/4= -github.com/redis/go-redis/v9 v9.10.0 h1:FxwK3eV8p/CQa0Ch276C7u2d0eNC9kCmAYQ7mCXCzVs= -github.com/redis/go-redis/v9 v9.10.0/go.mod h1:huWgSWd8mW6+m0VPhJjSSQ+d6Nh1VICQ6Q5lHuCH/Iw= +github.com/redis/go-redis/v9 v9.11.0 h1:E3S08Gl/nJNn5vkxd2i78wZxWAPNZgUNTp8WIJUAiIs= +github.com/redis/go-redis/v9 v9.11.0/go.mod h1:huWgSWd8mW6+m0VPhJjSSQ+d6Nh1VICQ6Q5lHuCH/Iw= github.com/rivo/uniseg v0.1.0/go.mod h1:J6wj4VEh+S6ZtnVlnTBMWIodfgj8LQOQFoIToxlJtxc= github.com/rivo/uniseg v0.2.0 h1:S1pD9weZBuJdFmowNwbpi7BJ8TNftyUImj/0WQi72jY= github.com/rivo/uniseg v0.2.0/go.mod h1:J6wj4VEh+S6ZtnVlnTBMWIodfgj8LQOQFoIToxlJtxc= From d3f22f48f57ea6ae41981518f4dc39cb34886386 Mon Sep 17 00:00:00 2001 From: Alvar Penning Date: Fri, 5 Sep 2025 13:52:38 +0200 Subject: [PATCH 08/38] Notifications: Address Code Review - Bump IGL to latest changes in Icinga/icinga-go-library#145. - Allow specifying which pipeline keys are relevant, ignore others. - Allow specifying which pipeline key should be parsed in which type. - Create history.DowntimeHistoryMeta as a chimera combining history.DowntimeHistory and history.HistoryDowntime to allow access event_type, distinguishing between downtime_start and downtime_end. - Trace times for submission steps in the worker. Turns out, the single threaded worker blocks roughly two seconds for each Client.ProcessEvent method call. This might sum up to minutes if lots of events are processed at once. My current theory is that the delay results in the expensive bcrypt hash comparison on Notifications. --- cmd/icingadb/main.go | 31 ++-- go.mod | 4 +- go.sum | 8 +- internal/config/config.go | 12 +- pkg/icingadb/history/sync.go | 89 +++++---- pkg/icingadb/v1/history/downtime.go | 6 + pkg/notifications/notifications.go | 272 +++++++++++++++++----------- pkg/notifications/redis_fetch.go | 6 +- 8 files changed, 255 insertions(+), 173 deletions(-) diff --git a/cmd/icingadb/main.go b/cmd/icingadb/main.go index c7fd403cd..27cb58f20 100644 --- a/cmd/icingadb/main.go +++ b/cmd/icingadb/main.go @@ -170,23 +170,26 @@ func run() int { sig := make(chan os.Signal, 1) signal.Notify(sig, os.Interrupt, syscall.SIGTERM, syscall.SIGHUP) - var notificationsSourceCallback func(database.Entity) - if cfg := cmd.Config.NotificationsSource; cfg.ApiBaseUrl != "" { - logger.Info("Starting Icinga Notifications source") - - notificationsSource := notifications.NewNotificationsSource( - ctx, - db, - rc, - logs.GetChildLogger("notifications-source"), - cfg) - notificationsSourceCallback = notificationsSource.Submit - } - go func() { + var callback func(database.Entity) + var callbackKeyStructPtr map[string]any + + if cfg := cmd.Config.NotificationsSource; cfg.ApiBaseUrl != "" { + logger.Info("Starting Icinga Notifications source") + + notificationsSource := notifications.NewNotificationsClient( + ctx, + db, + rc, + logs.GetChildLogger("notifications-source"), + cfg) + callback = notificationsSource.Submit + callbackKeyStructPtr = notifications.SyncKeyStructPtrs + } + logger.Info("Starting history sync") - if err := hs.Sync(ctx, notificationsSourceCallback); err != nil && !utils.IsContextCanceled(err) { + if err := hs.Sync(ctx, callbackKeyStructPtr, callback); err != nil && !utils.IsContextCanceled(err) { logger.Fatalf("%+v", err) } }() diff --git a/go.mod b/go.mod index dcea1432e..4df8f7977 100644 --- a/go.mod +++ b/go.mod @@ -7,14 +7,14 @@ require ( github.com/goccy/go-yaml v1.13.0 github.com/google/go-cmp v0.7.0 github.com/google/uuid v1.6.0 - github.com/icinga/icinga-go-library v0.7.3-0.20250807134650-55c038b220d8 + github.com/icinga/icinga-go-library v0.7.3-0.20250904130608-5032573a325e github.com/jessevdk/go-flags v1.6.1 github.com/jmoiron/sqlx v1.4.0 github.com/mattn/go-sqlite3 v1.14.32 github.com/okzk/sdnotify v0.0.0-20180710141335-d9becc38acbd github.com/pkg/errors v0.9.1 + github.com/redis/go-redis/v9 v9.13.0 github.com/stretchr/testify v1.11.1 - github.com/redis/go-redis/v9 v9.11.0 github.com/vbauerster/mpb/v6 v6.0.4 go.uber.org/zap v1.27.0 golang.org/x/sync v0.17.0 diff --git a/go.sum b/go.sum index 118d146ce..7fb54d086 100644 --- a/go.sum +++ b/go.sum @@ -37,8 +37,8 @@ github.com/google/go-cmp v0.7.0 h1:wk8382ETsv4JYUZwIsn6YpYiWiBsYLSJiTsyBybVuN8= github.com/google/go-cmp v0.7.0/go.mod h1:pXiqmnSA92OHEEa9HXL2W4E7lf9JzCmGVUdgjX3N/iU= github.com/google/uuid v1.6.0 h1:NIvaJDMOsjHA8n1jAhLSgzrAzy1Hgr+hNrb57e+94F0= github.com/google/uuid v1.6.0/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= -github.com/icinga/icinga-go-library v0.7.3-0.20250807134650-55c038b220d8 h1:YStwl7OlLUN87ROcZ8LQRTDKBhEx0V7EX5+5OMRwEnM= -github.com/icinga/icinga-go-library v0.7.3-0.20250807134650-55c038b220d8/go.mod h1:6xgV9o7JcGVg2I6CzPfefQZF4Ev8QeZnysiu82YaRY4= +github.com/icinga/icinga-go-library v0.7.3-0.20250904130608-5032573a325e h1:yZPWPPCHKozWRm9VedDHd8igJh9uI4U2CJdgl1On+/4= +github.com/icinga/icinga-go-library v0.7.3-0.20250904130608-5032573a325e/go.mod h1:exEJdfik2GPYrvZM6Gn4BXIBLIGg6OrCCMnILT+mTUs= github.com/jessevdk/go-flags v1.6.1 h1:Cvu5U8UGrLay1rZfv/zP7iLpSHGUZ/Ou68T0iX1bBK4= github.com/jessevdk/go-flags v1.6.1/go.mod h1:Mk8T1hIAWpOiJiHa9rJASDK2UGWji0EuPGBnNLMooyc= github.com/jmoiron/sqlx v1.4.0 h1:1PLqN7S1UYp5t4SrVVnt4nUVNemrDAtxlulVe+Qgm3o= @@ -63,8 +63,8 @@ github.com/pkg/errors v0.9.1 h1:FEBLx1zS214owpjy7qsBeixbURkuhQAwrK5UwLGTwt4= github.com/pkg/errors v0.9.1/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= github.com/pmezard/go-difflib v1.0.0 h1:4DBwDE0NGyQoBHbLQYPwSUPoCMWR5BEzIk/f1lZbAQM= github.com/pmezard/go-difflib v1.0.0/go.mod h1:iKH77koFhYxTK1pcRnkKkqfTogsbg7gZNVY4sRDYZ/4= -github.com/redis/go-redis/v9 v9.11.0 h1:E3S08Gl/nJNn5vkxd2i78wZxWAPNZgUNTp8WIJUAiIs= -github.com/redis/go-redis/v9 v9.11.0/go.mod h1:huWgSWd8mW6+m0VPhJjSSQ+d6Nh1VICQ6Q5lHuCH/Iw= +github.com/redis/go-redis/v9 v9.13.0 h1:PpmlVykE0ODh8P43U0HqC+2NXHXwG+GUtQyz+MPKGRg= +github.com/redis/go-redis/v9 v9.13.0/go.mod h1:huWgSWd8mW6+m0VPhJjSSQ+d6Nh1VICQ6Q5lHuCH/Iw= github.com/rivo/uniseg v0.1.0/go.mod h1:J6wj4VEh+S6ZtnVlnTBMWIodfgj8LQOQFoIToxlJtxc= github.com/rivo/uniseg v0.2.0 h1:S1pD9weZBuJdFmowNwbpi7BJ8TNftyUImj/0WQi72jY= github.com/rivo/uniseg v0.2.0/go.mod h1:J6wj4VEh+S6ZtnVlnTBMWIodfgj8LQOQFoIToxlJtxc= diff --git a/internal/config/config.go b/internal/config/config.go index 618a85fa0..359cf052e 100644 --- a/internal/config/config.go +++ b/internal/config/config.go @@ -4,7 +4,7 @@ import ( "github.com/creasty/defaults" "github.com/icinga/icinga-go-library/database" "github.com/icinga/icinga-go-library/logging" - "github.com/icinga/icinga-go-library/notifications" + "github.com/icinga/icinga-go-library/notifications/source" "github.com/icinga/icinga-go-library/redis" "github.com/icinga/icingadb/pkg/icingadb/history" "github.com/pkg/errors" @@ -16,11 +16,11 @@ const DefaultConfigPath = "/etc/icingadb/config.yml" // Config defines Icinga DB config. type Config struct { - Database database.Config `yaml:"database" envPrefix:"DATABASE_"` - Redis redis.Config `yaml:"redis" envPrefix:"REDIS_"` - Logging logging.Config `yaml:"logging" envPrefix:"LOGGING_"` - Retention RetentionConfig `yaml:"retention" envPrefix:"RETENTION_"` - NotificationsSource notifications.Config `yaml:"notifications-source" envPrefix:"NOTIFICATIONS_SOURCE_"` + Database database.Config `yaml:"database" envPrefix:"DATABASE_"` + Redis redis.Config `yaml:"redis" envPrefix:"REDIS_"` + Logging logging.Config `yaml:"logging" envPrefix:"LOGGING_"` + Retention RetentionConfig `yaml:"retention" envPrefix:"RETENTION_"` + NotificationsSource source.Config `yaml:"notifications-source" envPrefix:"NOTIFICATIONS_SOURCE_"` } func (c *Config) SetDefaults() { diff --git a/pkg/icingadb/history/sync.go b/pkg/icingadb/history/sync.go index 30c7f5a24..ac1c28e69 100644 --- a/pkg/icingadb/history/sync.go +++ b/pkg/icingadb/history/sync.go @@ -39,9 +39,15 @@ func NewSync(db *database.DB, redis *redis.Client, logger *logging.Logger) *Sync // Sync synchronizes Redis history streams from s.redis to s.db and deletes the original data on success. // -// If not nil, the callback function is appended to each synchronization pipeline and called before the entry is deleted -// from Redis. -func (s Sync) Sync(ctx context.Context, callback func(database.Entity)) error { +// An optional callback and callbackKeyStructPtr might be given. Both most either be nil or not nil. +// +// The callbackKeyStructPtr says which pipeline keys should be mapped to which type, identified by a struct pointer. If +// a key is missing from the map, it will not be used for the callback. The callback function itself shall not block. +func (s Sync) Sync(ctx context.Context, callbackKeyStructPtr map[string]any, callback func(database.Entity)) error { + if (callbackKeyStructPtr == nil) != (callback == nil) { + return fmt.Errorf("either both callbackKeyStructPtr and callback must be nil or none") + } + g, ctx := errgroup.WithContext(ctx) for key, pipeline := range syncPipelines { @@ -67,8 +73,13 @@ func (s Sync) Sync(ctx context.Context, callback func(database.Entity)) error { // forward the entry after it has completed its own sync so that later stages can rely on previous stages being // executed successfully. - if callback != nil { - pipeline = append(pipeline, makeCallbackStageFunc(callback)) + // Shadowed variable to allow appending custom callbacks. + pipeline := pipeline + if callbackKeyStructPtr != nil { + _, ok := callbackKeyStructPtr[key] + if ok { + pipeline = append(pipeline, makeCallbackStageFunc(callbackKeyStructPtr, callback)) + } } ch := make([]chan redis.XMessage, len(pipeline)+1) @@ -371,28 +382,17 @@ func userNotificationStage(ctx context.Context, s Sync, key string, in <-chan re // makeCallbackStageFunc creates a new stageFunc calling the given callback function for each message. // +// The keyStructPtrs map decides what kind of database.Entity type will be used for the input data based on the key. +// // The callback call is blocking and the message will be forwarded to the out channel after the function has returned. // Thus, please ensure this function does not block too long. -func makeCallbackStageFunc(callback func(database.Entity)) stageFunc { +func makeCallbackStageFunc(keyStructPtrs map[string]any, callback func(database.Entity)) stageFunc { return func(ctx context.Context, _ Sync, key string, in <-chan redis.XMessage, out chan<- redis.XMessage) error { defer close(out) - var structPtr database.Entity - switch key { // keep in sync with syncPipelines below - case "notification": - structPtr = (*v1.NotificationHistory)(nil) - case "state": - structPtr = (*v1.StateHistory)(nil) - case "downtime": - structPtr = (*v1.DowntimeHistory)(nil) - case "comment": - structPtr = (*v1.CommentHistory)(nil) - case "flapping": - structPtr = (*v1.FlappingHistory)(nil) - case "acknowledgement": - structPtr = (*v1.AcknowledgementHistory)(nil) - default: - return fmt.Errorf("unsupported key %q", key) + structPtr, ok := keyStructPtrs[key] + if !ok { + return fmt.Errorf("can't lookup struct pointer for key %q", key) } structifier := structify.MakeMapStructifier( @@ -409,7 +409,7 @@ func makeCallbackStageFunc(callback func(database.Entity)) stageFunc { val, err := structifier(msg.Values) if err != nil { - return errors.Wrapf(err, "can't structify values %#v for %s", msg.Values, key) + return errors.Wrapf(err, "can't structify values %#v for %q", msg.Values, key) } entity, ok := val.(database.Entity) @@ -427,32 +427,41 @@ func makeCallbackStageFunc(callback func(database.Entity)) stageFunc { } } +const ( + SyncPipelineAcknowledgement = "acknowledgement" + SyncPipelineComment = "comment" + SyncPipelineDowntime = "downtime" + SyncPipelineFlapping = "flapping" + SyncPipelineNotification = "notification" + SyncPipelineState = "state" +) + var syncPipelines = map[string][]stageFunc{ - "notification": { - writeOneEntityStage((*v1.NotificationHistory)(nil)), // notification_history - userNotificationStage, // user_notification_history (depends on notification_history) - writeOneEntityStage((*v1.HistoryNotification)(nil)), // history (depends on notification_history) + SyncPipelineAcknowledgement: { + writeOneEntityStage((*v1.AcknowledgementHistory)(nil)), // acknowledgement_history + writeOneEntityStage((*v1.HistoryAck)(nil)), // history (depends on acknowledgement_history) }, - "state": { - writeOneEntityStage((*v1.StateHistory)(nil)), // state_history - writeOneEntityStage((*v1.HistoryState)(nil)), // history (depends on state_history) - writeMultiEntityStage(stateHistoryToSlaEntity), // sla_history_state + SyncPipelineComment: { + writeOneEntityStage((*v1.CommentHistory)(nil)), // comment_history + writeOneEntityStage((*v1.HistoryComment)(nil)), // history (depends on comment_history) }, - "downtime": { + SyncPipelineDowntime: { writeOneEntityStage((*v1.DowntimeHistory)(nil)), // downtime_history writeOneEntityStage((*v1.HistoryDowntime)(nil)), // history (depends on downtime_history) writeOneEntityStage((*v1.SlaHistoryDowntime)(nil)), // sla_history_downtime }, - "comment": { - writeOneEntityStage((*v1.CommentHistory)(nil)), // comment_history - writeOneEntityStage((*v1.HistoryComment)(nil)), // history (depends on comment_history) - }, - "flapping": { + SyncPipelineFlapping: { writeOneEntityStage((*v1.FlappingHistory)(nil)), // flapping_history writeOneEntityStage((*v1.HistoryFlapping)(nil)), // history (depends on flapping_history) }, - "acknowledgement": { - writeOneEntityStage((*v1.AcknowledgementHistory)(nil)), // acknowledgement_history - writeOneEntityStage((*v1.HistoryAck)(nil)), // history (depends on acknowledgement_history) + SyncPipelineNotification: { + writeOneEntityStage((*v1.NotificationHistory)(nil)), // notification_history + userNotificationStage, // user_notification_history (depends on notification_history) + writeOneEntityStage((*v1.HistoryNotification)(nil)), // history (depends on notification_history) + }, + SyncPipelineState: { + writeOneEntityStage((*v1.StateHistory)(nil)), // state_history + writeOneEntityStage((*v1.HistoryState)(nil)), // history (depends on state_history) + writeMultiEntityStage(stateHistoryToSlaEntity), // sla_history_state }, } diff --git a/pkg/icingadb/v1/history/downtime.go b/pkg/icingadb/v1/history/downtime.go index 969cd4728..bbbfbbc15 100644 --- a/pkg/icingadb/v1/history/downtime.go +++ b/pkg/icingadb/v1/history/downtime.go @@ -88,6 +88,12 @@ func (*HistoryDowntime) TableName() string { return "history" } +type DowntimeHistoryMeta struct { + DowntimeHistoryEntity `json:",inline"` + DowntimeHistory `json:",inline"` + HistoryMeta `json:",inline"` +} + type SlaHistoryDowntime struct { DowntimeHistoryEntity `json:",inline"` HistoryTableMeta `json:",inline"` diff --git a/pkg/notifications/notifications.go b/pkg/notifications/notifications.go index 5a469ecef..1a0ddd239 100644 --- a/pkg/notifications/notifications.go +++ b/pkg/notifications/notifications.go @@ -2,80 +2,119 @@ package notifications import ( "context" - "database/sql" "fmt" "net/url" "time" "github.com/icinga/icinga-go-library/database" "github.com/icinga/icinga-go-library/logging" - "github.com/icinga/icinga-go-library/notifications" "github.com/icinga/icinga-go-library/notifications/event" + "github.com/icinga/icinga-go-library/notifications/source" "github.com/icinga/icinga-go-library/redis" "github.com/icinga/icinga-go-library/types" "github.com/icinga/icinga-go-library/utils" "github.com/icinga/icingadb/pkg/common" + "github.com/icinga/icingadb/pkg/icingadb/history" v1history "github.com/icinga/icingadb/pkg/icingadb/v1/history" "github.com/pkg/errors" "go.uber.org/zap" "go.uber.org/zap/zapcore" + "slices" + "strings" ) -// Source is an Icinga Notifications compatible source implementation to push events to Icinga Notifications. +// submission of a [database.Entity] to the Client. +type submission struct { + entity database.Entity + traces map[string]time.Time +} + +// MarshalLogObject implements [zapcore.ObjectMarshaler] to print a debug trace. +func (sub submission) MarshalLogObject(encoder zapcore.ObjectEncoder) error { + encoder.AddString("type", fmt.Sprintf("%T", sub.entity)) + + if len(sub.traces) < 1 { + return nil + } + + tracesKeys := slices.SortedFunc(func(yield func(string) bool) { + for key := range sub.traces { + if !yield(key) { + return + } + } + }, func(a string, b string) int { + return sub.traces[a].Compare(sub.traces[b]) + }) + + relTraces := make([]string, 0, len(tracesKeys)-1) + for i := 1; i < len(tracesKeys); i++ { + relTraces = append(relTraces, fmt.Sprintf("%s: %v", + tracesKeys[i], + sub.traces[tracesKeys[i]].Sub(sub.traces[tracesKeys[i-1]]))) + } + + encoder.AddDuration("processing_time", sub.traces[tracesKeys[len(tracesKeys)-1]].Sub(sub.traces[tracesKeys[0]])) + encoder.AddString("trace", strings.Join(relTraces, ", ")) + + return nil +} + +// Client is an Icinga Notifications compatible client implementation to push events to Icinga Notifications. // -// A new Source should be created by the NewNotificationsSource function. New history entries can be submitted by -// calling the Source.Submit method. The Source will then process the history entries in a background worker goroutine. -type Source struct { - notifications.Config +// A new Client should be created by the NewNotificationsClient function. New history entries can be submitted by +// calling the Source.Submit method. The Client will then process the history entries in a background worker goroutine. +type Client struct { + source.Config - inputCh chan database.Entity // inputCh is a buffered channel used to submit history entries to the worker. + inputCh chan submission // inputCh is a buffered channel used to submit history entries to the worker. db *database.DB logger *logging.Logger - rules *notifications.SourceRulesInfo // rules holds the latest rules fetched from Icinga Notifications. + rules *source.RulesInfo // rules holds the latest rules fetched from Icinga Notifications. ctx context.Context ctxCancel context.CancelFunc - notificationsClient *notifications.Client // The Icinga Notifications client used to interact with the API. - redisClient *redis.Client // redisClient is the Redis client used to fetch host and service names for events. + notificationsClient *source.Client // The Icinga Notifications client used to interact with the API. + redisClient *redis.Client // redisClient is the Redis client used to fetch host and service names for events. } -// NewNotificationsSource creates a new Source connected to an existing database and logger. +// NewNotificationsClient creates a new Client connected to an existing database and logger. // // This function starts a worker goroutine in the background which can be stopped by ending the provided context. -func NewNotificationsSource( +func NewNotificationsClient( ctx context.Context, db *database.DB, rc *redis.Client, logger *logging.Logger, - cfg notifications.Config, -) *Source { + cfg source.Config, +) *Client { ctx, ctxCancel := context.WithCancel(ctx) - source := &Source{ + client := &Client{ Config: cfg, - inputCh: make(chan database.Entity, 1<<10), // chosen by fair dice roll + inputCh: make(chan submission, 1<<10), // chosen by fair dice roll db: db, logger: logger, - rules: ¬ifications.SourceRulesInfo{Version: notifications.EmptyRulesVersion}, + rules: &source.RulesInfo{Version: source.EmptyRulesVersion}, redisClient: rc, ctx: ctx, ctxCancel: ctxCancel, } - client, err := notifications.NewClient(source.Config, "Icinga DB") + notificationsClient, err := source.NewClient(client.Config, "Icinga DB") if err != nil { logger.Fatalw("Cannot create Icinga Notifications client", zap.Error(err)) } - source.notificationsClient = client + client.notificationsClient = notificationsClient - go source.worker() + go client.worker() - return source + return client } // evaluateRulesForObject returns the rule IDs for each matching query. @@ -93,7 +132,7 @@ func NewNotificationsSource( // > select * from host where id = :host_id and environment_id = :environment_id and name like 'prefix_%' // // The :host_id and :environment_id parameters will be bound to the entity's ID and EnvironmentId fields, respectively. -func (s *Source) evaluateRulesForObject(ctx context.Context, entity database.Entity) ([]int64, error) { +func (s *Client) evaluateRulesForObject(ctx context.Context, entity database.Entity) ([]int64, error) { outRuleIds := make([]int64, 0, len(s.rules.Rules)) for rule := range s.rules.Iter() { @@ -102,35 +141,33 @@ func (s *Source) evaluateRulesForObject(ctx context.Context, entity database.Ent continue } - run := func() error { + evaluates, err := func() (bool, error) { // The raw SQL query in the database is URL-encoded (mostly the space character is replaced by %20). // So, we need to unescape it before passing it to the database. query, err := url.QueryUnescape(rule.ObjectFilterExpr) if err != nil { - return errors.Wrapf(err, "cannot unescape rule %d object filter expression %q", rule.Id, rule.ObjectFilterExpr) + return false, errors.Wrapf(err, "cannot unescape rule %d object filter expression %q", rule.Id, rule.ObjectFilterExpr) } rows, err := s.db.NamedQueryContext(ctx, s.db.Rebind(query), entity) if err != nil { - return err + return false, err } defer func() { _ = rows.Close() }() if !rows.Next() { - return sql.ErrNoRows + return false, nil } - return nil - } - - if err := run(); err == nil { - outRuleIds = append(outRuleIds, rule.Id) - } else if errors.Is(err, sql.ErrNoRows) { - continue - } else { + return true, nil + }() + if err != nil { return nil, errors.Wrapf(err, "cannot fetch rule %d from %q", rule.Id, rule.ObjectFilterExpr) + } else if !evaluates { + continue } + outRuleIds = append(outRuleIds, rule.Id) } - return outRuleIds[:len(outRuleIds):len(outRuleIds)], nil + return outRuleIds, nil } // buildCommonEvent creates an event.Event based on Host and (optional) Service names. @@ -138,38 +175,38 @@ func (s *Source) evaluateRulesForObject(ctx context.Context, entity database.Ent // This function is used by all event builders to create a common event structure that includes the host and service // names, the absolute URL to the Icinga Web 2 Icinga DB page for the host or service, and the tags for the event. // Any event type-specific information (like severity, message, etc.) is added by the specific event builders. -func (s *Source) buildCommonEvent(rlr *redisLookupResult) (*event.Event, error) { +func (s *Client) buildCommonEvent(rlr *redisLookupResult) (*event.Event, error) { var ( - eventName string - eventUrl *url.URL - eventTags map[string]string + objectName string + objectUrl *url.URL + objectTags map[string]string ) if rlr.ServiceName != "" { - eventName = rlr.HostName + "!" + rlr.ServiceName + objectName = rlr.HostName + "!" + rlr.ServiceName - eventUrl = s.notificationsClient.JoinIcingaWeb2Path("/icingadb/service") - eventUrl.RawQuery = "name=" + utils.RawUrlEncode(rlr.ServiceName) + "&host.name=" + utils.RawUrlEncode(rlr.HostName) + objectUrl = s.notificationsClient.JoinIcingaWeb2Path("/icingadb/service") + objectUrl.RawQuery = "name=" + utils.RawUrlEncode(rlr.ServiceName) + "&host.name=" + utils.RawUrlEncode(rlr.HostName) - eventTags = map[string]string{ + objectTags = map[string]string{ "host": rlr.HostName, "service": rlr.ServiceName, } } else { - eventName = rlr.HostName + objectName = rlr.HostName - eventUrl = s.notificationsClient.JoinIcingaWeb2Path("/icingadb/host") - eventUrl.RawQuery = "name=" + utils.RawUrlEncode(rlr.HostName) + objectUrl = s.notificationsClient.JoinIcingaWeb2Path("/icingadb/host") + objectUrl.RawQuery = "name=" + utils.RawUrlEncode(rlr.HostName) - eventTags = map[string]string{ + objectTags = map[string]string{ "host": rlr.HostName, } } return &event.Event{ - Name: eventName, - URL: eventUrl.String(), - Tags: eventTags, + Name: objectName, + URL: objectUrl.String(), + Tags: objectTags, }, nil } @@ -177,7 +214,7 @@ func (s *Source) buildCommonEvent(rlr *redisLookupResult) (*event.Event, error) // // The resulted event will have all the necessary information for a state change event, and must // not be further modified by the caller. -func (s *Source) buildStateHistoryEvent(ctx context.Context, h *v1history.StateHistory) (*event.Event, error) { +func (s *Client) buildStateHistoryEvent(ctx context.Context, h *v1history.StateHistory) (*event.Event, error) { res, err := s.fetchHostServiceName(ctx, h.HostId, h.ServiceId) if err != nil { return nil, err @@ -224,8 +261,8 @@ func (s *Source) buildStateHistoryEvent(ctx context.Context, h *v1history.StateH return ev, nil } -// buildDowntimeHistoryEvent from a downtime history entry. -func (s *Source) buildDowntimeHistoryEvent(ctx context.Context, h *v1history.DowntimeHistory) (*event.Event, error) { +// buildDowntimeHistoryMetaEvent from a downtime history entry. +func (s *Client) buildDowntimeHistoryMetaEvent(ctx context.Context, h *v1history.DowntimeHistoryMeta) (*event.Event, error) { res, err := s.fetchHostServiceName(ctx, h.HostId, h.ServiceId) if err != nil { return nil, err @@ -236,29 +273,36 @@ func (s *Source) buildDowntimeHistoryEvent(ctx context.Context, h *v1history.Dow return nil, errors.Wrapf(err, "cannot build event for %q,%q", res.HostName, res.ServiceName) } - if h.HasBeenCancelled.Valid && h.HasBeenCancelled.Bool { - ev.Type = event.TypeDowntimeRemoved - ev.Message = "Downtime was cancelled" - - if h.CancelledBy.Valid { - ev.Username = h.CancelledBy.String - } - } else if h.EndTime.Time().Compare(time.Now()) <= 0 { - ev.Type = event.TypeDowntimeEnd - ev.Message = "Downtime expired" - } else { + switch h.EventType { + case "downtime_start": ev.Type = event.TypeDowntimeStart ev.Username = h.Author ev.Message = h.Comment ev.Mute = types.MakeBool(true) ev.MuteReason = "Checkable is in downtime" + + case "downtime_end": + if h.HasBeenCancelled.Valid && h.HasBeenCancelled.Bool { + ev.Type = event.TypeDowntimeRemoved + ev.Message = "Downtime was cancelled" + + if h.CancelledBy.Valid { + ev.Username = h.CancelledBy.String + } + } else { + ev.Type = event.TypeDowntimeEnd + ev.Message = "Downtime expired" + } + + default: + return nil, fmt.Errorf("unexpected event type %q", h.EventType) } return ev, nil } // buildFlappingHistoryEvent from a flapping history entry. -func (s *Source) buildFlappingHistoryEvent(ctx context.Context, h *v1history.FlappingHistory) (*event.Event, error) { +func (s *Client) buildFlappingHistoryEvent(ctx context.Context, h *v1history.FlappingHistory) (*event.Event, error) { res, err := s.fetchHostServiceName(ctx, h.HostId, h.ServiceId) if err != nil { return nil, err @@ -289,7 +333,7 @@ func (s *Source) buildFlappingHistoryEvent(ctx context.Context, h *v1history.Fla } // buildAcknowledgementHistoryEvent from an acknowledgment history entry. -func (s *Source) buildAcknowledgementHistoryEvent(ctx context.Context, h *v1history.AcknowledgementHistory) (*event.Event, error) { +func (s *Client) buildAcknowledgementHistoryEvent(ctx context.Context, h *v1history.AcknowledgementHistory) (*event.Event, error) { res, err := s.fetchHostServiceName(ctx, h.HostId, h.ServiceId) if err != nil { return nil, err @@ -326,8 +370,8 @@ func (s *Source) buildAcknowledgementHistoryEvent(ctx context.Context, h *v1hist return ev, nil } -// worker is the background worker launched by NewNotificationsSource. -func (s *Source) worker() { +// worker is the background worker launched by NewNotificationsClient. +func (s *Client) worker() { defer s.ctxCancel() for { @@ -335,55 +379,50 @@ func (s *Source) worker() { case <-s.ctx.Done(): return - case entity, more := <-s.inputCh: + case sub, more := <-s.inputCh: if !more { // Should never happen, but just in case. s.logger.Debug("Input channel closed, stopping worker") return } + sub.traces["worker_start"] = time.Now() + var ev *event.Event var eventErr error // Keep the type switch in sync with syncPipelines from pkg/icingadb/history/sync.go - switch h := entity.(type) { - case *v1history.NotificationHistory: - // Ignore for the moment. - continue + switch h := sub.entity.(type) { + case *v1history.AcknowledgementHistory: + ev, eventErr = s.buildAcknowledgementHistoryEvent(s.ctx, h) + + case *v1history.DowntimeHistoryMeta: + ev, eventErr = s.buildDowntimeHistoryMetaEvent(s.ctx, h) + + case *v1history.FlappingHistory: + ev, eventErr = s.buildFlappingHistoryEvent(s.ctx, h) case *v1history.StateHistory: if h.StateType != common.HardState { continue } - ev, eventErr = s.buildStateHistoryEvent(s.ctx, h) - case *v1history.DowntimeHistory: - ev, eventErr = s.buildDowntimeHistoryEvent(s.ctx, h) - - case *v1history.CommentHistory: - // Ignore for the moment. - continue - - case *v1history.FlappingHistory: - ev, eventErr = s.buildFlappingHistoryEvent(s.ctx, h) - - case *v1history.AcknowledgementHistory: - ev, eventErr = s.buildAcknowledgementHistoryEvent(s.ctx, h) - default: s.logger.Error("Cannot process unsupported type", + zap.Object("submission", sub), zap.String("type", fmt.Sprintf("%T", h))) continue } if eventErr != nil { s.logger.Errorw("Cannot build event from history entry", - zap.String("type", fmt.Sprintf("%T", entity)), + zap.Object("submission", sub), + zap.String("type", fmt.Sprintf("%T", sub.entity)), zap.Error(eventErr)) continue - } - if ev == nil { - s.logger.Error("No event was fetched, but no error was reported. This REALLY SHOULD NOT happen.") + } else if ev == nil { + // This really should not happen. + s.logger.Errorw("No event was fetched, but no error was reported.", zap.Object("submission", sub)) continue } @@ -396,52 +435,77 @@ func (s *Source) worker() { }), )) + sub.traces["evaluate_jump_pre"] = time.Now() reevaluateRules: - eventRuleIds, err := s.evaluateRulesForObject(s.ctx, entity) + sub.traces["evaluate_jump_last"] = time.Now() + eventRuleIds, err := s.evaluateRulesForObject(s.ctx, sub.entity) if err != nil { - eventLogger.Errorw("Cannot evaluate rules for event", zap.Error(err)) + eventLogger.Errorw("Cannot evaluate rules for event", + zap.Object("submission", sub), + zap.Error(err)) continue } + sub.traces["process_last"] = time.Now() newEventRules, err := s.notificationsClient.ProcessEvent(s.ctx, ev, s.rules.Version, eventRuleIds...) - if errors.Is(err, notifications.ErrRulesOutdated) { + if errors.Is(err, source.ErrRulesOutdated) { s.rules = newEventRules - eventLogger.Debugw("Re-evaluating rules for event after fetching new rules", zap.String("rules_version", s.rules.Version)) + eventLogger.Infow("Re-evaluating rules for event after fetching new rules", + zap.Object("submission", sub), + zap.String("rules_version", s.rules.Version)) // Re-evaluate the just fetched rules for the current event. goto reevaluateRules } else if err != nil { eventLogger.Errorw("Cannot submit event to Icinga Notifications", + zap.Object("submission", sub), zap.String("rules_version", s.rules.Version), zap.Any("rules", eventRuleIds), zap.Error(err)) continue } - eventLogger.Debugw("Successfully submitted event to Icinga Notifications", zap.Any("rules", eventRuleIds)) + sub.traces["worker_fin"] = time.Now() + eventLogger.Debugw("Successfully submitted event to Icinga Notifications", + zap.Object("submission", sub), + zap.Any("rules", eventRuleIds)) } } } -// Submit a history entry to be processed by the Source's internal worker loop. +// Submit a history entry to be processed by the Client's internal worker loop. // // Internally, a buffered channel is used for delivery. So this function should not block. Otherwise, it will abort // after a second and an error is logged. -func (s *Source) Submit(entity database.Entity) { +func (s *Client) Submit(entity database.Entity) { + sub := submission{ + entity: entity, + traces: map[string]time.Time{ + "submit": time.Now(), + }, + } + select { case <-s.ctx.Done(): - s.logger.Errorw("Source context is done, rejecting submission", - zap.String("submission", fmt.Sprintf("%+v", entity)), + s.logger.Errorw("Client context is done, rejecting submission", + zap.Object("submission", sub), zap.Error(s.ctx.Err())) return - case s.inputCh <- entity: + case s.inputCh <- sub: return case <-time.After(time.Second): - s.logger.Error("Source submission channel is blocking, rejecting submission", - zap.String("submission", fmt.Sprintf("%+v", entity))) + s.logger.Error("Client submission channel is blocking, rejecting submission", + zap.Object("submission", sub)) return } } + +var SyncKeyStructPtrs = map[string]any{ + history.SyncPipelineAcknowledgement: (*v1history.AcknowledgementHistory)(nil), + history.SyncPipelineDowntime: (*v1history.DowntimeHistoryMeta)(nil), + history.SyncPipelineFlapping: (*v1history.FlappingHistory)(nil), + history.SyncPipelineState: (*v1history.StateHistory)(nil), +} diff --git a/pkg/notifications/redis_fetch.go b/pkg/notifications/redis_fetch.go index 8d61d7760..7d7ec61dc 100644 --- a/pkg/notifications/redis_fetch.go +++ b/pkg/notifications/redis_fetch.go @@ -23,7 +23,7 @@ import ( // If this operation couldn't be completed within a reasonable time (a hard coded 5 seconds), it will cancel the // request and return an error indicating that the operation timed out. In case of the serviceId being set, the // maximum execution time of the Redis HGet commands is 10s (5s for each HGet call). -func (s *Source) fetchHostServiceName(ctx context.Context, hostId, serviceId types.Binary) (*redisLookupResult, error) { +func (s *Client) fetchHostServiceName(ctx context.Context, hostId, serviceId types.Binary) (*redisLookupResult, error) { redisHGet := func(typ, field string, out *redisLookupResult) error { ctx, cancel := context.WithTimeout(ctx, 5*time.Second) defer cancel() @@ -69,14 +69,14 @@ type redisLookupResult struct { ServiceName string `json:"-"` // Name of the service (only set in service context). // Name is used to retrieve the host or service name from Redis. - // It should not be used for any other purpose apart from within the [Source.fetchHostServiceName] function. + // It should not be used for any other purpose apart from within the [Client.fetchHostServiceName] function. Name string `json:"name"` } // UnmarshalBinary implements the [encoding.BinaryUnmarshaler] interface for redisLookupResult. // // It unmarshals the binary data of the Redis HGet result into the redisLookupResult struct. -// This is required for the HGet().Scan() usage in the [Source.fetchHostServiceName] function to work correctly. +// This is required for the HGet().Scan() usage in the [Client.fetchHostServiceName] function to work correctly. func (rlr *redisLookupResult) UnmarshalBinary(data []byte) error { if len(data) == 0 { return errors.New("empty data received for redisLookupResult") From 076e41a8937fe8d58a6c8a997034ae5a8de10cfc Mon Sep 17 00:00:00 2001 From: Alvar Penning Date: Tue, 9 Sep 2025 12:09:02 +0200 Subject: [PATCH 09/38] notifications: IGL Changes For Rules The rules and rule version is now part of the Event. Also rename the Client method receiver variable. --- go.mod | 2 +- go.sum | 4 +- pkg/notifications/notifications.go | 89 +++++++++++++++--------------- pkg/notifications/redis_fetch.go | 4 +- 4 files changed, 51 insertions(+), 48 deletions(-) diff --git a/go.mod b/go.mod index 4df8f7977..59125620b 100644 --- a/go.mod +++ b/go.mod @@ -7,7 +7,7 @@ require ( github.com/goccy/go-yaml v1.13.0 github.com/google/go-cmp v0.7.0 github.com/google/uuid v1.6.0 - github.com/icinga/icinga-go-library v0.7.3-0.20250904130608-5032573a325e + github.com/icinga/icinga-go-library v0.7.3-0.20250909100113-20db23663e45 github.com/jessevdk/go-flags v1.6.1 github.com/jmoiron/sqlx v1.4.0 github.com/mattn/go-sqlite3 v1.14.32 diff --git a/go.sum b/go.sum index 7fb54d086..ab75a6aaf 100644 --- a/go.sum +++ b/go.sum @@ -37,8 +37,8 @@ github.com/google/go-cmp v0.7.0 h1:wk8382ETsv4JYUZwIsn6YpYiWiBsYLSJiTsyBybVuN8= github.com/google/go-cmp v0.7.0/go.mod h1:pXiqmnSA92OHEEa9HXL2W4E7lf9JzCmGVUdgjX3N/iU= github.com/google/uuid v1.6.0 h1:NIvaJDMOsjHA8n1jAhLSgzrAzy1Hgr+hNrb57e+94F0= github.com/google/uuid v1.6.0/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= -github.com/icinga/icinga-go-library v0.7.3-0.20250904130608-5032573a325e h1:yZPWPPCHKozWRm9VedDHd8igJh9uI4U2CJdgl1On+/4= -github.com/icinga/icinga-go-library v0.7.3-0.20250904130608-5032573a325e/go.mod h1:exEJdfik2GPYrvZM6Gn4BXIBLIGg6OrCCMnILT+mTUs= +github.com/icinga/icinga-go-library v0.7.3-0.20250909100113-20db23663e45 h1:Wz6ttTYgYB7y8FH7snBSnnllLuzhE0QSp6m3P9b/QfM= +github.com/icinga/icinga-go-library v0.7.3-0.20250909100113-20db23663e45/go.mod h1:uCENf5EVhNVvXTvhB+jXiwRB2NdLlz8cymseOM4qmI0= github.com/jessevdk/go-flags v1.6.1 h1:Cvu5U8UGrLay1rZfv/zP7iLpSHGUZ/Ou68T0iX1bBK4= github.com/jessevdk/go-flags v1.6.1/go.mod h1:Mk8T1hIAWpOiJiHa9rJASDK2UGWji0EuPGBnNLMooyc= github.com/jmoiron/sqlx v1.4.0 h1:1PLqN7S1UYp5t4SrVVnt4nUVNemrDAtxlulVe+Qgm3o= diff --git a/pkg/notifications/notifications.go b/pkg/notifications/notifications.go index 1a0ddd239..f150cf2b5 100644 --- a/pkg/notifications/notifications.go +++ b/pkg/notifications/notifications.go @@ -132,10 +132,10 @@ func NewNotificationsClient( // > select * from host where id = :host_id and environment_id = :environment_id and name like 'prefix_%' // // The :host_id and :environment_id parameters will be bound to the entity's ID and EnvironmentId fields, respectively. -func (s *Client) evaluateRulesForObject(ctx context.Context, entity database.Entity) ([]int64, error) { - outRuleIds := make([]int64, 0, len(s.rules.Rules)) +func (client *Client) evaluateRulesForObject(ctx context.Context, entity database.Entity) ([]int64, error) { + outRuleIds := make([]int64, 0, len(client.rules.Rules)) - for rule := range s.rules.Iter() { + for rule := range client.rules.Iter() { if rule.ObjectFilterExpr == "" { outRuleIds = append(outRuleIds, rule.Id) continue @@ -148,7 +148,7 @@ func (s *Client) evaluateRulesForObject(ctx context.Context, entity database.Ent if err != nil { return false, errors.Wrapf(err, "cannot unescape rule %d object filter expression %q", rule.Id, rule.ObjectFilterExpr) } - rows, err := s.db.NamedQueryContext(ctx, s.db.Rebind(query), entity) + rows, err := client.db.NamedQueryContext(ctx, client.db.Rebind(query), entity) if err != nil { return false, err } @@ -175,7 +175,7 @@ func (s *Client) evaluateRulesForObject(ctx context.Context, entity database.Ent // This function is used by all event builders to create a common event structure that includes the host and service // names, the absolute URL to the Icinga Web 2 Icinga DB page for the host or service, and the tags for the event. // Any event type-specific information (like severity, message, etc.) is added by the specific event builders. -func (s *Client) buildCommonEvent(rlr *redisLookupResult) (*event.Event, error) { +func (client *Client) buildCommonEvent(rlr *redisLookupResult) (*event.Event, error) { var ( objectName string objectUrl *url.URL @@ -185,7 +185,7 @@ func (s *Client) buildCommonEvent(rlr *redisLookupResult) (*event.Event, error) if rlr.ServiceName != "" { objectName = rlr.HostName + "!" + rlr.ServiceName - objectUrl = s.notificationsClient.JoinIcingaWeb2Path("/icingadb/service") + objectUrl = client.notificationsClient.JoinIcingaWeb2Path("/icingadb/service") objectUrl.RawQuery = "name=" + utils.RawUrlEncode(rlr.ServiceName) + "&host.name=" + utils.RawUrlEncode(rlr.HostName) objectTags = map[string]string{ @@ -195,7 +195,7 @@ func (s *Client) buildCommonEvent(rlr *redisLookupResult) (*event.Event, error) } else { objectName = rlr.HostName - objectUrl = s.notificationsClient.JoinIcingaWeb2Path("/icingadb/host") + objectUrl = client.notificationsClient.JoinIcingaWeb2Path("/icingadb/host") objectUrl.RawQuery = "name=" + utils.RawUrlEncode(rlr.HostName) objectTags = map[string]string{ @@ -214,13 +214,13 @@ func (s *Client) buildCommonEvent(rlr *redisLookupResult) (*event.Event, error) // // The resulted event will have all the necessary information for a state change event, and must // not be further modified by the caller. -func (s *Client) buildStateHistoryEvent(ctx context.Context, h *v1history.StateHistory) (*event.Event, error) { - res, err := s.fetchHostServiceName(ctx, h.HostId, h.ServiceId) +func (client *Client) buildStateHistoryEvent(ctx context.Context, h *v1history.StateHistory) (*event.Event, error) { + res, err := client.fetchHostServiceName(ctx, h.HostId, h.ServiceId) if err != nil { return nil, err } - ev, err := s.buildCommonEvent(res) + ev, err := client.buildCommonEvent(res) if err != nil { return nil, errors.Wrapf(err, "cannot build event for %q,%q", res.HostName, res.ServiceName) } @@ -262,13 +262,13 @@ func (s *Client) buildStateHistoryEvent(ctx context.Context, h *v1history.StateH } // buildDowntimeHistoryMetaEvent from a downtime history entry. -func (s *Client) buildDowntimeHistoryMetaEvent(ctx context.Context, h *v1history.DowntimeHistoryMeta) (*event.Event, error) { - res, err := s.fetchHostServiceName(ctx, h.HostId, h.ServiceId) +func (client *Client) buildDowntimeHistoryMetaEvent(ctx context.Context, h *v1history.DowntimeHistoryMeta) (*event.Event, error) { + res, err := client.fetchHostServiceName(ctx, h.HostId, h.ServiceId) if err != nil { return nil, err } - ev, err := s.buildCommonEvent(res) + ev, err := client.buildCommonEvent(res) if err != nil { return nil, errors.Wrapf(err, "cannot build event for %q,%q", res.HostName, res.ServiceName) } @@ -302,13 +302,13 @@ func (s *Client) buildDowntimeHistoryMetaEvent(ctx context.Context, h *v1history } // buildFlappingHistoryEvent from a flapping history entry. -func (s *Client) buildFlappingHistoryEvent(ctx context.Context, h *v1history.FlappingHistory) (*event.Event, error) { - res, err := s.fetchHostServiceName(ctx, h.HostId, h.ServiceId) +func (client *Client) buildFlappingHistoryEvent(ctx context.Context, h *v1history.FlappingHistory) (*event.Event, error) { + res, err := client.fetchHostServiceName(ctx, h.HostId, h.ServiceId) if err != nil { return nil, err } - ev, err := s.buildCommonEvent(res) + ev, err := client.buildCommonEvent(res) if err != nil { return nil, errors.Wrapf(err, "cannot build event for %q,%q", res.HostName, res.ServiceName) } @@ -333,13 +333,13 @@ func (s *Client) buildFlappingHistoryEvent(ctx context.Context, h *v1history.Fla } // buildAcknowledgementHistoryEvent from an acknowledgment history entry. -func (s *Client) buildAcknowledgementHistoryEvent(ctx context.Context, h *v1history.AcknowledgementHistory) (*event.Event, error) { - res, err := s.fetchHostServiceName(ctx, h.HostId, h.ServiceId) +func (client *Client) buildAcknowledgementHistoryEvent(ctx context.Context, h *v1history.AcknowledgementHistory) (*event.Event, error) { + res, err := client.fetchHostServiceName(ctx, h.HostId, h.ServiceId) if err != nil { return nil, err } - ev, err := s.buildCommonEvent(res) + ev, err := client.buildCommonEvent(res) if err != nil { return nil, errors.Wrapf(err, "cannot build event for %q,%q", res.HostName, res.ServiceName) } @@ -371,17 +371,17 @@ func (s *Client) buildAcknowledgementHistoryEvent(ctx context.Context, h *v1hist } // worker is the background worker launched by NewNotificationsClient. -func (s *Client) worker() { - defer s.ctxCancel() +func (client *Client) worker() { + defer client.ctxCancel() for { select { - case <-s.ctx.Done(): + case <-client.ctx.Done(): return - case sub, more := <-s.inputCh: + case sub, more := <-client.inputCh: if !more { // Should never happen, but just in case. - s.logger.Debug("Input channel closed, stopping worker") + client.logger.Debug("Input channel closed, stopping worker") return } @@ -393,40 +393,40 @@ func (s *Client) worker() { // Keep the type switch in sync with syncPipelines from pkg/icingadb/history/sync.go switch h := sub.entity.(type) { case *v1history.AcknowledgementHistory: - ev, eventErr = s.buildAcknowledgementHistoryEvent(s.ctx, h) + ev, eventErr = client.buildAcknowledgementHistoryEvent(client.ctx, h) case *v1history.DowntimeHistoryMeta: - ev, eventErr = s.buildDowntimeHistoryMetaEvent(s.ctx, h) + ev, eventErr = client.buildDowntimeHistoryMetaEvent(client.ctx, h) case *v1history.FlappingHistory: - ev, eventErr = s.buildFlappingHistoryEvent(s.ctx, h) + ev, eventErr = client.buildFlappingHistoryEvent(client.ctx, h) case *v1history.StateHistory: if h.StateType != common.HardState { continue } - ev, eventErr = s.buildStateHistoryEvent(s.ctx, h) + ev, eventErr = client.buildStateHistoryEvent(client.ctx, h) default: - s.logger.Error("Cannot process unsupported type", + client.logger.Error("Cannot process unsupported type", zap.Object("submission", sub), zap.String("type", fmt.Sprintf("%T", h))) continue } if eventErr != nil { - s.logger.Errorw("Cannot build event from history entry", + client.logger.Errorw("Cannot build event from history entry", zap.Object("submission", sub), zap.String("type", fmt.Sprintf("%T", sub.entity)), zap.Error(eventErr)) continue } else if ev == nil { // This really should not happen. - s.logger.Errorw("No event was fetched, but no error was reported.", zap.Object("submission", sub)) + client.logger.Errorw("No event was fetched, but no error was reported.", zap.Object("submission", sub)) continue } - eventLogger := s.logger.With(zap.Object( + eventLogger := client.logger.With(zap.Object( "event", zapcore.ObjectMarshalerFunc(func(encoder zapcore.ObjectEncoder) error { encoder.AddString("name", ev.Name) @@ -438,7 +438,7 @@ func (s *Client) worker() { sub.traces["evaluate_jump_pre"] = time.Now() reevaluateRules: sub.traces["evaluate_jump_last"] = time.Now() - eventRuleIds, err := s.evaluateRulesForObject(s.ctx, sub.entity) + eventRuleIds, err := client.evaluateRulesForObject(client.ctx, sub.entity) if err != nil { eventLogger.Errorw("Cannot evaluate rules for event", zap.Object("submission", sub), @@ -446,21 +446,24 @@ func (s *Client) worker() { continue } + ev.RulesVersion = client.rules.Version + ev.RuleIds = eventRuleIds + sub.traces["process_last"] = time.Now() - newEventRules, err := s.notificationsClient.ProcessEvent(s.ctx, ev, s.rules.Version, eventRuleIds...) + newEventRules, err := client.notificationsClient.ProcessEvent(client.ctx, ev) if errors.Is(err, source.ErrRulesOutdated) { - s.rules = newEventRules + client.rules = newEventRules eventLogger.Infow("Re-evaluating rules for event after fetching new rules", zap.Object("submission", sub), - zap.String("rules_version", s.rules.Version)) + zap.String("rules_version", client.rules.Version)) // Re-evaluate the just fetched rules for the current event. goto reevaluateRules } else if err != nil { eventLogger.Errorw("Cannot submit event to Icinga Notifications", zap.Object("submission", sub), - zap.String("rules_version", s.rules.Version), + zap.String("rules_version", client.rules.Version), zap.Any("rules", eventRuleIds), zap.Error(err)) continue @@ -478,7 +481,7 @@ func (s *Client) worker() { // // Internally, a buffered channel is used for delivery. So this function should not block. Otherwise, it will abort // after a second and an error is logged. -func (s *Client) Submit(entity database.Entity) { +func (client *Client) Submit(entity database.Entity) { sub := submission{ entity: entity, traces: map[string]time.Time{ @@ -487,17 +490,17 @@ func (s *Client) Submit(entity database.Entity) { } select { - case <-s.ctx.Done(): - s.logger.Errorw("Client context is done, rejecting submission", + case <-client.ctx.Done(): + client.logger.Errorw("Client context is done, rejecting submission", zap.Object("submission", sub), - zap.Error(s.ctx.Err())) + zap.Error(client.ctx.Err())) return - case s.inputCh <- sub: + case client.inputCh <- sub: return case <-time.After(time.Second): - s.logger.Error("Client submission channel is blocking, rejecting submission", + client.logger.Error("Client submission channel is blocking, rejecting submission", zap.Object("submission", sub)) return } diff --git a/pkg/notifications/redis_fetch.go b/pkg/notifications/redis_fetch.go index 7d7ec61dc..3cacde16c 100644 --- a/pkg/notifications/redis_fetch.go +++ b/pkg/notifications/redis_fetch.go @@ -23,14 +23,14 @@ import ( // If this operation couldn't be completed within a reasonable time (a hard coded 5 seconds), it will cancel the // request and return an error indicating that the operation timed out. In case of the serviceId being set, the // maximum execution time of the Redis HGet commands is 10s (5s for each HGet call). -func (s *Client) fetchHostServiceName(ctx context.Context, hostId, serviceId types.Binary) (*redisLookupResult, error) { +func (client *Client) fetchHostServiceName(ctx context.Context, hostId, serviceId types.Binary) (*redisLookupResult, error) { redisHGet := func(typ, field string, out *redisLookupResult) error { ctx, cancel := context.WithTimeout(ctx, 5*time.Second) defer cancel() err := retry.WithBackoff( ctx, - func(ctx context.Context) error { return s.redisClient.HGet(ctx, "icinga:"+typ, field).Scan(out) }, + func(ctx context.Context) error { return client.redisClient.HGet(ctx, "icinga:"+typ, field).Scan(out) }, retry.Retryable, backoff.DefaultBackoff, retry.Settings{}, From 84d3cfa6a438e7a9d353863778a9680c85ea8841 Mon Sep 17 00:00:00 2001 From: Alvar Penning Date: Thu, 25 Sep 2025 14:45:17 +0200 Subject: [PATCH 10/38] history: Retry failing callback submissions Do not silently drop failing callback submissions - such as Icinga Notification during restarts or network disruptions -, but switch the internal makeCallbackStageFunc stageFunc into a backlog mode. This resulted in multiple changes, including removing the background worker for notifications.Client, as otherwise the event submission status could not be propagated back. --- cmd/icingadb/main.go | 2 +- pkg/icingadb/history/sync.go | 161 +++++++++++++++-- pkg/icingaredis/telemetry/stats.go | 8 +- pkg/notifications/notifications.go | 266 ++++++++++------------------- 4 files changed, 241 insertions(+), 196 deletions(-) diff --git a/cmd/icingadb/main.go b/cmd/icingadb/main.go index 27cb58f20..ed5392253 100644 --- a/cmd/icingadb/main.go +++ b/cmd/icingadb/main.go @@ -171,7 +171,7 @@ func run() int { signal.Notify(sig, os.Interrupt, syscall.SIGTERM, syscall.SIGHUP) go func() { - var callback func(database.Entity) + var callback func(database.Entity) bool var callbackKeyStructPtr map[string]any if cfg := cmd.Config.NotificationsSource; cfg.ApiBaseUrl != "" { diff --git a/pkg/icingadb/history/sync.go b/pkg/icingadb/history/sync.go index ac1c28e69..82ba93747 100644 --- a/pkg/icingadb/history/sync.go +++ b/pkg/icingadb/history/sync.go @@ -16,9 +16,11 @@ import ( v1 "github.com/icinga/icingadb/pkg/icingadb/v1/history" "github.com/icinga/icingadb/pkg/icingaredis/telemetry" "github.com/pkg/errors" + "go.uber.org/zap" "golang.org/x/sync/errgroup" "reflect" "sync" + "time" ) // Sync specifies the source and destination of a history sync. @@ -43,7 +45,7 @@ func NewSync(db *database.DB, redis *redis.Client, logger *logging.Logger) *Sync // // The callbackKeyStructPtr says which pipeline keys should be mapped to which type, identified by a struct pointer. If // a key is missing from the map, it will not be used for the callback. The callback function itself shall not block. -func (s Sync) Sync(ctx context.Context, callbackKeyStructPtr map[string]any, callback func(database.Entity)) error { +func (s Sync) Sync(ctx context.Context, callbackKeyStructPtr map[string]any, callback func(database.Entity) bool) error { if (callbackKeyStructPtr == nil) != (callback == nil) { return fmt.Errorf("either both callbackKeyStructPtr and callback must be nil or none") } @@ -72,14 +74,20 @@ func (s Sync) Sync(ctx context.Context, callbackKeyStructPtr map[string]any, cal // it has processed it, even if the stage itself does not do anything with this specific entry. It should only // forward the entry after it has completed its own sync so that later stages can rely on previous stages being // executed successfully. + // + // If a callback exists for this key, it will be appended to the pipeline. Thus, it is executed after every + // other pipeline action, but before deleteFromRedis. + + var hasCallbackStage bool + if callbackKeyStructPtr != nil { + _, exists := callbackKeyStructPtr[key] + hasCallbackStage = exists + } // Shadowed variable to allow appending custom callbacks. pipeline := pipeline - if callbackKeyStructPtr != nil { - _, ok := callbackKeyStructPtr[key] - if ok { - pipeline = append(pipeline, makeCallbackStageFunc(callbackKeyStructPtr, callback)) - } + if hasCallbackStage { + pipeline = append(pipeline, makeCallbackStageFunc(callbackKeyStructPtr, callback)) } ch := make([]chan redis.XMessage, len(pipeline)+1) @@ -171,7 +179,6 @@ func (s Sync) deleteFromRedis(ctx context.Context, key string, input <-chan redi } counter.Add(uint64(len(ids))) - telemetry.Stats.History.Add(uint64(len(ids))) case <-ctx.Done(): return ctx.Err() } @@ -380,14 +387,46 @@ func userNotificationStage(ctx context.Context, s Sync, key string, in <-chan re })(ctx, s, key, in, out) } +// countElementStage increments the [Stats.History] counter. +// +// This stageFunc should be called last in a [syncPipeline]. Thus, it is still executed before the final +// Sync.deleteFromRedis call in Sync.Sync. Furthermore, an optional callback function will be appended after this stage, +// resulting in an incremented history state counter for synchronized history, but stalling callback actions. +func countElementStage(ctx context.Context, _ Sync, _ string, in <-chan redis.XMessage, out chan<- redis.XMessage) error { + defer close(out) + + for { + select { + case msg, ok := <-in: + if !ok { + return nil + } + + telemetry.Stats.History.Add(1) + out <- msg + + case <-ctx.Done(): + return ctx.Err() + } + } +} + // makeCallbackStageFunc creates a new stageFunc calling the given callback function for each message. // // The keyStructPtrs map decides what kind of database.Entity type will be used for the input data based on the key. // // The callback call is blocking and the message will be forwarded to the out channel after the function has returned. // Thus, please ensure this function does not block too long. -func makeCallbackStageFunc(keyStructPtrs map[string]any, callback func(database.Entity)) stageFunc { - return func(ctx context.Context, _ Sync, key string, in <-chan redis.XMessage, out chan<- redis.XMessage) error { +// +// If the callback function returns false, the stageFunc switches to a backlog mode, retrying the failed messages and +// every subsequent message until there are no messages left. Only after a message was successfully handled by the +// callback method, it will be forwarded to the out channel. Thus, this stage might "block" or "hold back" certain +// messages during unhappy callback times. +// +// For each successfully submitted message, [telemetry.State.Callback] is incremented. Thus, a delta between +// [telemetry.State.History] and [telemetry.State.Callback] indicates blocking callbacks. +func makeCallbackStageFunc(keyStructPtrs map[string]any, callback func(database.Entity) bool) stageFunc { + return func(ctx context.Context, s Sync, key string, in <-chan redis.XMessage, out chan<- redis.XMessage) error { defer close(out) structPtr, ok := keyStructPtrs[key] @@ -400,6 +439,28 @@ func makeCallbackStageFunc(keyStructPtrs map[string]any, callback func(database. "json", contracts.SafeInit) + makeEntity := func(values map[string]interface{}) (database.Entity, error) { + val, err := structifier(values) + if err != nil { + return nil, errors.Wrapf(err, "can't structify values %#v for %q", values, key) + } + + entity, ok := val.(database.Entity) + if !ok { + return nil, fmt.Errorf("structifier returned %T which does not implement database.Entity", val) + } + + return entity, nil + } + + backlogLastId := "" + backlogMsgCounter := 0 + + const backlogTimerMinInterval, backlogTimerMaxInterval = 10 * time.Millisecond, time.Minute + backlogTimerInterval := backlogTimerMinInterval + backlogTimer := time.NewTimer(backlogTimerInterval) + _ = backlogTimer.Stop() + for { select { case msg, ok := <-in: @@ -407,18 +468,80 @@ func makeCallbackStageFunc(keyStructPtrs map[string]any, callback func(database. return nil } - val, err := structifier(msg.Values) + // Only submit the entity directly if there is no backlog. + // The second check covers a potential corner case if the XRANGE below races this stream. + if backlogLastId != "" && backlogLastId != msg.ID { + continue + } + + entity, err := makeEntity(msg.Values) if err != nil { - return errors.Wrapf(err, "can't structify values %#v for %q", msg.Values, key) + return err } - entity, ok := val.(database.Entity) - if !ok { - return fmt.Errorf("structifier returned %T, expected %T", val, structPtr) + if callback(entity) { + out <- msg + telemetry.Stats.Callback.Add(1) + backlogLastId = "" + } else { + backlogLastId = msg.ID + backlogMsgCounter = 0 + backlogTimerInterval = backlogTimerMinInterval + _ = backlogTimer.Reset(backlogTimerInterval) + s.logger.Warnw("Failed to submit entity to callback, entering into backlog", + zap.String("key", key), + zap.String("id", backlogLastId)) + } + + case <-backlogTimer.C: + if backlogLastId == "" { // Should never happen. + return fmt.Errorf("backlog timer logic for %q was called while backlogLastId was empty", key) + } + + logger := s.logger.With( + zap.String("key", key), + zap.String("last-id", backlogLastId)) + + logger.Debug("Trying to advance backlog of callback elements") + + xrangeCmd := s.redis.XRangeN(ctx, "icinga:history:stream:"+key, backlogLastId, "+", 2) + msgs, err := xrangeCmd.Result() + if err != nil { + return errors.Wrapf(err, "XRANGE %q to %q on stream %q failed", backlogLastId, "+", key) } - callback(entity) - out <- msg + if len(msgs) < 1 || len(msgs) > 2 { + return fmt.Errorf("XRANGE %q to %q on stream %q returned %d messages, not 1 or 2", + backlogLastId, "+", key, len(msgs)) + } + + msg := msgs[0] + entity, err := makeEntity(msg.Values) + if err != nil { + return errors.Wrapf(err, "can't structify backlog value %q for %q", backlogLastId, key) + } + + if callback(entity) { + out <- msg + backlogMsgCounter++ + telemetry.Stats.Callback.Add(1) + + if len(msgs) == 1 { + backlogLastId = "" + logger.Infow("Finished rolling back backlog of callback elements", zap.Int("delay", backlogMsgCounter)) + } else { + backlogLastId = msgs[1].ID + backlogTimerInterval = backlogTimerMinInterval + _ = backlogTimer.Reset(backlogTimerInterval) + logger.Debugw("Advanced backlog", + zap.String("new-last-id", backlogLastId), + zap.Duration("delay", backlogTimerInterval)) + } + } else { + backlogTimerInterval = min(backlogTimerMaxInterval, backlogTimerInterval*2) + _ = backlogTimer.Reset(backlogTimerInterval) + logger.Warnw("Failed to roll back callback elements", zap.Duration("delay", backlogTimerInterval)) + } case <-ctx.Done(): return ctx.Err() @@ -440,28 +563,34 @@ var syncPipelines = map[string][]stageFunc{ SyncPipelineAcknowledgement: { writeOneEntityStage((*v1.AcknowledgementHistory)(nil)), // acknowledgement_history writeOneEntityStage((*v1.HistoryAck)(nil)), // history (depends on acknowledgement_history) + countElementStage, }, SyncPipelineComment: { writeOneEntityStage((*v1.CommentHistory)(nil)), // comment_history writeOneEntityStage((*v1.HistoryComment)(nil)), // history (depends on comment_history) + countElementStage, }, SyncPipelineDowntime: { writeOneEntityStage((*v1.DowntimeHistory)(nil)), // downtime_history writeOneEntityStage((*v1.HistoryDowntime)(nil)), // history (depends on downtime_history) writeOneEntityStage((*v1.SlaHistoryDowntime)(nil)), // sla_history_downtime + countElementStage, }, SyncPipelineFlapping: { writeOneEntityStage((*v1.FlappingHistory)(nil)), // flapping_history writeOneEntityStage((*v1.HistoryFlapping)(nil)), // history (depends on flapping_history) + countElementStage, }, SyncPipelineNotification: { writeOneEntityStage((*v1.NotificationHistory)(nil)), // notification_history userNotificationStage, // user_notification_history (depends on notification_history) writeOneEntityStage((*v1.HistoryNotification)(nil)), // history (depends on notification_history) + countElementStage, }, SyncPipelineState: { writeOneEntityStage((*v1.StateHistory)(nil)), // state_history writeOneEntityStage((*v1.HistoryState)(nil)), // history (depends on state_history) writeMultiEntityStage(stateHistoryToSlaEntity), // sla_history_state + countElementStage, }, } diff --git a/pkg/icingaredis/telemetry/stats.go b/pkg/icingaredis/telemetry/stats.go index 78f5c5c67..a34845128 100644 --- a/pkg/icingaredis/telemetry/stats.go +++ b/pkg/icingaredis/telemetry/stats.go @@ -14,7 +14,12 @@ import ( var Stats struct { // Config & co. are to be increased by the T sync once for every T object synced. - Config, State, History, Overdue, HistoryCleanup com.Counter + Config com.Counter + State com.Counter + History com.Counter + Callback com.Counter + Overdue com.Counter + HistoryCleanup com.Counter } // WriteStats periodically forwards Stats to Redis for being monitored by Icinga 2. @@ -23,6 +28,7 @@ func WriteStats(ctx context.Context, client *redis.Client, logger *logging.Logge "config_sync": &Stats.Config, "state_sync": &Stats.State, "history_sync": &Stats.History, + "callback_sync": &Stats.Callback, "overdue_sync": &Stats.Overdue, "history_cleanup": &Stats.HistoryCleanup, } diff --git a/pkg/notifications/notifications.go b/pkg/notifications/notifications.go index f150cf2b5..5b1f53426 100644 --- a/pkg/notifications/notifications.go +++ b/pkg/notifications/notifications.go @@ -3,9 +3,6 @@ package notifications import ( "context" "fmt" - "net/url" - "time" - "github.com/icinga/icinga-go-library/database" "github.com/icinga/icinga-go-library/logging" "github.com/icinga/icinga-go-library/notifications/event" @@ -19,70 +16,31 @@ import ( "github.com/pkg/errors" "go.uber.org/zap" "go.uber.org/zap/zapcore" - "slices" - "strings" + "net/url" + "sync" ) -// submission of a [database.Entity] to the Client. -type submission struct { - entity database.Entity - traces map[string]time.Time -} - -// MarshalLogObject implements [zapcore.ObjectMarshaler] to print a debug trace. -func (sub submission) MarshalLogObject(encoder zapcore.ObjectEncoder) error { - encoder.AddString("type", fmt.Sprintf("%T", sub.entity)) - - if len(sub.traces) < 1 { - return nil - } - - tracesKeys := slices.SortedFunc(func(yield func(string) bool) { - for key := range sub.traces { - if !yield(key) { - return - } - } - }, func(a string, b string) int { - return sub.traces[a].Compare(sub.traces[b]) - }) - - relTraces := make([]string, 0, len(tracesKeys)-1) - for i := 1; i < len(tracesKeys); i++ { - relTraces = append(relTraces, fmt.Sprintf("%s: %v", - tracesKeys[i], - sub.traces[tracesKeys[i]].Sub(sub.traces[tracesKeys[i-1]]))) - } - - encoder.AddDuration("processing_time", sub.traces[tracesKeys[len(tracesKeys)-1]].Sub(sub.traces[tracesKeys[0]])) - encoder.AddString("trace", strings.Join(relTraces, ", ")) - - return nil -} - // Client is an Icinga Notifications compatible client implementation to push events to Icinga Notifications. // // A new Client should be created by the NewNotificationsClient function. New history entries can be submitted by -// calling the Source.Submit method. The Client will then process the history entries in a background worker goroutine. +// calling the Client.Submit method. type Client struct { source.Config - inputCh chan submission // inputCh is a buffered channel used to submit history entries to the worker. - db *database.DB - logger *logging.Logger + db *database.DB + logger *logging.Logger rules *source.RulesInfo // rules holds the latest rules fetched from Icinga Notifications. - ctx context.Context - ctxCancel context.CancelFunc + ctx context.Context notificationsClient *source.Client // The Icinga Notifications client used to interact with the API. redisClient *redis.Client // redisClient is the Redis client used to fetch host and service names for events. + + submissionMutex sync.Mutex } // NewNotificationsClient creates a new Client connected to an existing database and logger. -// -// This function starts a worker goroutine in the background which can be stopped by ending the provided context. func NewNotificationsClient( ctx context.Context, db *database.DB, @@ -90,20 +48,16 @@ func NewNotificationsClient( logger *logging.Logger, cfg source.Config, ) *Client { - ctx, ctxCancel := context.WithCancel(ctx) - client := &Client{ Config: cfg, - inputCh: make(chan submission, 1<<10), // chosen by fair dice roll - db: db, - logger: logger, + db: db, + logger: logger, rules: &source.RulesInfo{Version: source.EmptyRulesVersion}, redisClient: rc, - ctx: ctx, - ctxCancel: ctxCancel, + ctx: ctx, } notificationsClient, err := source.NewClient(client.Config, "Icinga DB") @@ -112,8 +66,6 @@ func NewNotificationsClient( } client.notificationsClient = notificationsClient - go client.worker() - return client } @@ -370,140 +322,98 @@ func (client *Client) buildAcknowledgementHistoryEvent(ctx context.Context, h *v return ev, nil } -// worker is the background worker launched by NewNotificationsClient. -func (client *Client) worker() { - defer client.ctxCancel() - - for { - select { - case <-client.ctx.Done(): - return - - case sub, more := <-client.inputCh: - if !more { // Should never happen, but just in case. - client.logger.Debug("Input channel closed, stopping worker") - return - } - - sub.traces["worker_start"] = time.Now() - - var ev *event.Event - var eventErr error +// Submit this [database.Entity] to the Icinga Notifications API. +// +// Based on the entity's type, a different kind of event will be constructed. The event will be sent to the API in a +// blocking fashion. +// +// Returns true if this entity was processed or cannot be processed any further. Returns false if this entity should be +// retried later. +// +// This method usees the Client's logger. +func (client *Client) Submit(entity database.Entity) bool { + client.submissionMutex.Lock() + defer client.submissionMutex.Unlock() + + if client.ctx.Err() != nil { + client.logger.Error("Cannot process submitted entity as client context is done") + return true + } - // Keep the type switch in sync with syncPipelines from pkg/icingadb/history/sync.go - switch h := sub.entity.(type) { - case *v1history.AcknowledgementHistory: - ev, eventErr = client.buildAcknowledgementHistoryEvent(client.ctx, h) + var ev *event.Event + var eventErr error - case *v1history.DowntimeHistoryMeta: - ev, eventErr = client.buildDowntimeHistoryMetaEvent(client.ctx, h) + // Keep the type switch in sync with syncPipelines from pkg/icingadb/history/sync.go + switch h := entity.(type) { + case *v1history.AcknowledgementHistory: + ev, eventErr = client.buildAcknowledgementHistoryEvent(client.ctx, h) - case *v1history.FlappingHistory: - ev, eventErr = client.buildFlappingHistoryEvent(client.ctx, h) + case *v1history.DowntimeHistoryMeta: + ev, eventErr = client.buildDowntimeHistoryMetaEvent(client.ctx, h) - case *v1history.StateHistory: - if h.StateType != common.HardState { - continue - } - ev, eventErr = client.buildStateHistoryEvent(client.ctx, h) + case *v1history.FlappingHistory: + ev, eventErr = client.buildFlappingHistoryEvent(client.ctx, h) - default: - client.logger.Error("Cannot process unsupported type", - zap.Object("submission", sub), - zap.String("type", fmt.Sprintf("%T", h))) - continue - } + case *v1history.StateHistory: + if h.StateType != common.HardState { + return true + } + ev, eventErr = client.buildStateHistoryEvent(client.ctx, h) - if eventErr != nil { - client.logger.Errorw("Cannot build event from history entry", - zap.Object("submission", sub), - zap.String("type", fmt.Sprintf("%T", sub.entity)), - zap.Error(eventErr)) - continue - } else if ev == nil { - // This really should not happen. - client.logger.Errorw("No event was fetched, but no error was reported.", zap.Object("submission", sub)) - continue - } + default: + client.logger.Error("Cannot process unsupported type", zap.String("type", fmt.Sprintf("%T", h))) + return true + } - eventLogger := client.logger.With(zap.Object( - "event", - zapcore.ObjectMarshalerFunc(func(encoder zapcore.ObjectEncoder) error { - encoder.AddString("name", ev.Name) - encoder.AddString("type", ev.Type.String()) - return nil - }), - )) - - sub.traces["evaluate_jump_pre"] = time.Now() - reevaluateRules: - sub.traces["evaluate_jump_last"] = time.Now() - eventRuleIds, err := client.evaluateRulesForObject(client.ctx, sub.entity) - if err != nil { - eventLogger.Errorw("Cannot evaluate rules for event", - zap.Object("submission", sub), - zap.Error(err)) - continue - } + if eventErr != nil { + client.logger.Errorw("Cannot build event from history entry", + zap.String("type", fmt.Sprintf("%T", entity)), + zap.Error(eventErr)) + return true + } else if ev == nil { + // This really should not happen. + client.logger.Errorw("No event was built, but no error was reported", + zap.String("type", fmt.Sprintf("%T", entity))) + return true + } - ev.RulesVersion = client.rules.Version - ev.RuleIds = eventRuleIds - - sub.traces["process_last"] = time.Now() - newEventRules, err := client.notificationsClient.ProcessEvent(client.ctx, ev) - if errors.Is(err, source.ErrRulesOutdated) { - client.rules = newEventRules - - eventLogger.Infow("Re-evaluating rules for event after fetching new rules", - zap.Object("submission", sub), - zap.String("rules_version", client.rules.Version)) - - // Re-evaluate the just fetched rules for the current event. - goto reevaluateRules - } else if err != nil { - eventLogger.Errorw("Cannot submit event to Icinga Notifications", - zap.Object("submission", sub), - zap.String("rules_version", client.rules.Version), - zap.Any("rules", eventRuleIds), - zap.Error(err)) - continue - } + eventLogger := client.logger.With(zap.Object( + "event", + zapcore.ObjectMarshalerFunc(func(encoder zapcore.ObjectEncoder) error { + encoder.AddString("name", ev.Name) + encoder.AddString("type", ev.Type.String()) + return nil + }), + )) - sub.traces["worker_fin"] = time.Now() - eventLogger.Debugw("Successfully submitted event to Icinga Notifications", - zap.Object("submission", sub), - zap.Any("rules", eventRuleIds)) - } + eventRuleIds, err := client.evaluateRulesForObject(client.ctx, entity) + if err != nil { + eventLogger.Errorw("Cannot evaluate rules for event, will be retried", zap.Error(err)) + return false } -} -// Submit a history entry to be processed by the Client's internal worker loop. -// -// Internally, a buffered channel is used for delivery. So this function should not block. Otherwise, it will abort -// after a second and an error is logged. -func (client *Client) Submit(entity database.Entity) { - sub := submission{ - entity: entity, - traces: map[string]time.Time{ - "submit": time.Now(), - }, - } + ev.RulesVersion = client.rules.Version + ev.RuleIds = eventRuleIds - select { - case <-client.ctx.Done(): - client.logger.Errorw("Client context is done, rejecting submission", - zap.Object("submission", sub), - zap.Error(client.ctx.Err())) - return + newEventRules, err := client.notificationsClient.ProcessEvent(client.ctx, ev) + if errors.Is(err, source.ErrRulesOutdated) { + eventLogger.Infow("Cannot submit event to Icinga Notifications due to rule changes, will be retried", + zap.String("old_rules_version", client.rules.Version), + zap.String("new_rules_version", newEventRules.Version)) - case client.inputCh <- sub: - return + client.rules = newEventRules - case <-time.After(time.Second): - client.logger.Error("Client submission channel is blocking, rejecting submission", - zap.Object("submission", sub)) - return + return false + } else if err != nil { + eventLogger.Errorw("Cannot submit event to Icinga Notifications, will be retried", + zap.String("rules_version", client.rules.Version), + zap.Any("rules", eventRuleIds), + zap.Error(err)) + return false } + + eventLogger.Debugw("Successfully submitted event to Icinga Notifications", zap.Any("rules", eventRuleIds)) + return true } var SyncKeyStructPtrs = map[string]any{ From 1dcd1e010689cd35ccba2868cf7d802d71e5cf96 Mon Sep 17 00:00:00 2001 From: Alvar Penning Date: Mon, 29 Sep 2025 17:05:28 +0200 Subject: [PATCH 11/38] notifications: Send relative Icinga Web 2 URLs There is no need to let each Icinga Notifications source know the root URL of Icinga Web 2. Since the latest IGL and IN change, partly URLs relative to Icinga Web 2 are supported. --- config.example.yml | 3 --- go.mod | 4 ++-- go.sum | 8 ++++---- pkg/notifications/notifications.go | 19 ++++++++++--------- 4 files changed, 16 insertions(+), 18 deletions(-) diff --git a/config.example.yml b/config.example.yml index 10a064c3d..c4d47c9ca 100644 --- a/config.example.yml +++ b/config.example.yml @@ -151,6 +151,3 @@ redis: # # # Password for the defined user. # password: insecureinsecure -# -# # URL to the Icinga Web 2 to craft event URLs. -# icingaweb2-base-url: http://localhost/icingaweb2 diff --git a/go.mod b/go.mod index 59125620b..372a80584 100644 --- a/go.mod +++ b/go.mod @@ -7,13 +7,13 @@ require ( github.com/goccy/go-yaml v1.13.0 github.com/google/go-cmp v0.7.0 github.com/google/uuid v1.6.0 - github.com/icinga/icinga-go-library v0.7.3-0.20250909100113-20db23663e45 + github.com/icinga/icinga-go-library v0.7.3-0.20250929125610-2d45c2002dfa github.com/jessevdk/go-flags v1.6.1 github.com/jmoiron/sqlx v1.4.0 github.com/mattn/go-sqlite3 v1.14.32 github.com/okzk/sdnotify v0.0.0-20180710141335-d9becc38acbd github.com/pkg/errors v0.9.1 - github.com/redis/go-redis/v9 v9.13.0 + github.com/redis/go-redis/v9 v9.14.0 github.com/stretchr/testify v1.11.1 github.com/vbauerster/mpb/v6 v6.0.4 go.uber.org/zap v1.27.0 diff --git a/go.sum b/go.sum index ab75a6aaf..033cb9350 100644 --- a/go.sum +++ b/go.sum @@ -37,8 +37,8 @@ github.com/google/go-cmp v0.7.0 h1:wk8382ETsv4JYUZwIsn6YpYiWiBsYLSJiTsyBybVuN8= github.com/google/go-cmp v0.7.0/go.mod h1:pXiqmnSA92OHEEa9HXL2W4E7lf9JzCmGVUdgjX3N/iU= github.com/google/uuid v1.6.0 h1:NIvaJDMOsjHA8n1jAhLSgzrAzy1Hgr+hNrb57e+94F0= github.com/google/uuid v1.6.0/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= -github.com/icinga/icinga-go-library v0.7.3-0.20250909100113-20db23663e45 h1:Wz6ttTYgYB7y8FH7snBSnnllLuzhE0QSp6m3P9b/QfM= -github.com/icinga/icinga-go-library v0.7.3-0.20250909100113-20db23663e45/go.mod h1:uCENf5EVhNVvXTvhB+jXiwRB2NdLlz8cymseOM4qmI0= +github.com/icinga/icinga-go-library v0.7.3-0.20250929125610-2d45c2002dfa h1:lGWIyytdvC5HsokVpBUaKIsRdpoWaucPFXBziuKBQ+U= +github.com/icinga/icinga-go-library v0.7.3-0.20250929125610-2d45c2002dfa/go.mod h1:nQXvnUNKDpJfOLin9cVveUFmiqipMYkoHo5P8O3kjXs= github.com/jessevdk/go-flags v1.6.1 h1:Cvu5U8UGrLay1rZfv/zP7iLpSHGUZ/Ou68T0iX1bBK4= github.com/jessevdk/go-flags v1.6.1/go.mod h1:Mk8T1hIAWpOiJiHa9rJASDK2UGWji0EuPGBnNLMooyc= github.com/jmoiron/sqlx v1.4.0 h1:1PLqN7S1UYp5t4SrVVnt4nUVNemrDAtxlulVe+Qgm3o= @@ -63,8 +63,8 @@ github.com/pkg/errors v0.9.1 h1:FEBLx1zS214owpjy7qsBeixbURkuhQAwrK5UwLGTwt4= github.com/pkg/errors v0.9.1/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= github.com/pmezard/go-difflib v1.0.0 h1:4DBwDE0NGyQoBHbLQYPwSUPoCMWR5BEzIk/f1lZbAQM= github.com/pmezard/go-difflib v1.0.0/go.mod h1:iKH77koFhYxTK1pcRnkKkqfTogsbg7gZNVY4sRDYZ/4= -github.com/redis/go-redis/v9 v9.13.0 h1:PpmlVykE0ODh8P43U0HqC+2NXHXwG+GUtQyz+MPKGRg= -github.com/redis/go-redis/v9 v9.13.0/go.mod h1:huWgSWd8mW6+m0VPhJjSSQ+d6Nh1VICQ6Q5lHuCH/Iw= +github.com/redis/go-redis/v9 v9.14.0 h1:u4tNCjXOyzfgeLN+vAZaW1xUooqWDqVEsZN0U01jfAE= +github.com/redis/go-redis/v9 v9.14.0/go.mod h1:huWgSWd8mW6+m0VPhJjSSQ+d6Nh1VICQ6Q5lHuCH/Iw= github.com/rivo/uniseg v0.1.0/go.mod h1:J6wj4VEh+S6ZtnVlnTBMWIodfgj8LQOQFoIToxlJtxc= github.com/rivo/uniseg v0.2.0 h1:S1pD9weZBuJdFmowNwbpi7BJ8TNftyUImj/0WQi72jY= github.com/rivo/uniseg v0.2.0/go.mod h1:J6wj4VEh+S6ZtnVlnTBMWIodfgj8LQOQFoIToxlJtxc= diff --git a/pkg/notifications/notifications.go b/pkg/notifications/notifications.go index 5b1f53426..566d368b0 100644 --- a/pkg/notifications/notifications.go +++ b/pkg/notifications/notifications.go @@ -10,6 +10,7 @@ import ( "github.com/icinga/icinga-go-library/redis" "github.com/icinga/icinga-go-library/types" "github.com/icinga/icinga-go-library/utils" + "github.com/icinga/icingadb/internal" "github.com/icinga/icingadb/pkg/common" "github.com/icinga/icingadb/pkg/icingadb/history" v1history "github.com/icinga/icingadb/pkg/icingadb/v1/history" @@ -54,13 +55,13 @@ func NewNotificationsClient( db: db, logger: logger, - rules: &source.RulesInfo{Version: source.EmptyRulesVersion}, + rules: &source.RulesInfo{}, redisClient: rc, ctx: ctx, } - notificationsClient, err := source.NewClient(client.Config, "Icinga DB") + notificationsClient, err := source.NewClient(client.Config, fmt.Sprintf("Icinga DB %s", internal.Version.Version)) if err != nil { logger.Fatalw("Cannot create Icinga Notifications client", zap.Error(err)) } @@ -84,8 +85,8 @@ func NewNotificationsClient( // > select * from host where id = :host_id and environment_id = :environment_id and name like 'prefix_%' // // The :host_id and :environment_id parameters will be bound to the entity's ID and EnvironmentId fields, respectively. -func (client *Client) evaluateRulesForObject(ctx context.Context, entity database.Entity) ([]int64, error) { - outRuleIds := make([]int64, 0, len(client.rules.Rules)) +func (client *Client) evaluateRulesForObject(ctx context.Context, entity database.Entity) ([]string, error) { + outRuleIds := make([]string, 0, len(client.rules.Rules)) for rule := range client.rules.Iter() { if rule.ObjectFilterExpr == "" { @@ -98,7 +99,7 @@ func (client *Client) evaluateRulesForObject(ctx context.Context, entity databas // So, we need to unescape it before passing it to the database. query, err := url.QueryUnescape(rule.ObjectFilterExpr) if err != nil { - return false, errors.Wrapf(err, "cannot unescape rule %d object filter expression %q", rule.Id, rule.ObjectFilterExpr) + return false, errors.Wrapf(err, "cannot unescape rule %q object filter expression %q", rule.Id, rule.ObjectFilterExpr) } rows, err := client.db.NamedQueryContext(ctx, client.db.Rebind(query), entity) if err != nil { @@ -112,7 +113,7 @@ func (client *Client) evaluateRulesForObject(ctx context.Context, entity databas return true, nil }() if err != nil { - return nil, errors.Wrapf(err, "cannot fetch rule %d from %q", rule.Id, rule.ObjectFilterExpr) + return nil, errors.Wrapf(err, "cannot fetch rule %q from %q", rule.Id, rule.ObjectFilterExpr) } else if !evaluates { continue } @@ -130,14 +131,14 @@ func (client *Client) evaluateRulesForObject(ctx context.Context, entity databas func (client *Client) buildCommonEvent(rlr *redisLookupResult) (*event.Event, error) { var ( objectName string - objectUrl *url.URL + objectUrl url.URL objectTags map[string]string ) if rlr.ServiceName != "" { objectName = rlr.HostName + "!" + rlr.ServiceName - objectUrl = client.notificationsClient.JoinIcingaWeb2Path("/icingadb/service") + objectUrl.Path = "/icingadb/service" objectUrl.RawQuery = "name=" + utils.RawUrlEncode(rlr.ServiceName) + "&host.name=" + utils.RawUrlEncode(rlr.HostName) objectTags = map[string]string{ @@ -147,7 +148,7 @@ func (client *Client) buildCommonEvent(rlr *redisLookupResult) (*event.Event, er } else { objectName = rlr.HostName - objectUrl = client.notificationsClient.JoinIcingaWeb2Path("/icingadb/host") + objectUrl.Path = "/icingadb/host" objectUrl.RawQuery = "name=" + utils.RawUrlEncode(rlr.HostName) objectTags = map[string]string{ From b4a9755b15c4f75ac450dffd46f8ba22963b807e Mon Sep 17 00:00:00 2001 From: Alvar Penning Date: Tue, 30 Sep 2025 14:42:06 +0200 Subject: [PATCH 12/38] notifications: Don't abort for faulty object rules When a faulty - like syntactical incorrect - object filter expression was loaded, each evaluation fails. However, prior to this change, the submission logic was exited, making Icinga DB unable to recover. Now, the event will be considered as no rule has matched and new rule version can be loaded. --- pkg/notifications/notifications.go | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/pkg/notifications/notifications.go b/pkg/notifications/notifications.go index 566d368b0..2d9147c40 100644 --- a/pkg/notifications/notifications.go +++ b/pkg/notifications/notifications.go @@ -337,7 +337,7 @@ func (client *Client) Submit(entity database.Entity) bool { defer client.submissionMutex.Unlock() if client.ctx.Err() != nil { - client.logger.Error("Cannot process submitted entity as client context is done") + client.logger.Errorw("Cannot process submitted entity as client context is done", zap.Error(client.ctx.Err())) return true } @@ -389,8 +389,10 @@ func (client *Client) Submit(entity database.Entity) bool { eventRuleIds, err := client.evaluateRulesForObject(client.ctx, entity) if err != nil { - eventLogger.Errorw("Cannot evaluate rules for event, will be retried", zap.Error(err)) - return false + // While returning false would be more correct, this would result in never being able to refetch new rule + // versions. Consider an invalid object filter expression, which is now impossible to get rid of. + eventLogger.Errorw("Cannot evaluate rules for event, assuming no rule matched", zap.Error(err)) + eventRuleIds = []string{} } ev.RulesVersion = client.rules.Version From 7d1b80c19b85d0513b3e3cb506b11234a4e80b67 Mon Sep 17 00:00:00 2001 From: Alvar Penning Date: Thu, 2 Oct 2025 10:00:06 +0200 Subject: [PATCH 13/38] notifications: Reflect RulesInfo IGL update The RulesInfo type was simplified. Rules are no longer a custom struct, but just represented by the map key and a filter expression string. --- go.mod | 2 +- go.sum | 4 ++-- pkg/notifications/notifications.go | 37 +++++++++++++++--------------- 3 files changed, 22 insertions(+), 21 deletions(-) diff --git a/go.mod b/go.mod index 372a80584..b6ab6584f 100644 --- a/go.mod +++ b/go.mod @@ -7,7 +7,7 @@ require ( github.com/goccy/go-yaml v1.13.0 github.com/google/go-cmp v0.7.0 github.com/google/uuid v1.6.0 - github.com/icinga/icinga-go-library v0.7.3-0.20250929125610-2d45c2002dfa + github.com/icinga/icinga-go-library v0.7.3-0.20251002071510-577e857e2417 github.com/jessevdk/go-flags v1.6.1 github.com/jmoiron/sqlx v1.4.0 github.com/mattn/go-sqlite3 v1.14.32 diff --git a/go.sum b/go.sum index 033cb9350..db2c137c9 100644 --- a/go.sum +++ b/go.sum @@ -37,8 +37,8 @@ github.com/google/go-cmp v0.7.0 h1:wk8382ETsv4JYUZwIsn6YpYiWiBsYLSJiTsyBybVuN8= github.com/google/go-cmp v0.7.0/go.mod h1:pXiqmnSA92OHEEa9HXL2W4E7lf9JzCmGVUdgjX3N/iU= github.com/google/uuid v1.6.0 h1:NIvaJDMOsjHA8n1jAhLSgzrAzy1Hgr+hNrb57e+94F0= github.com/google/uuid v1.6.0/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= -github.com/icinga/icinga-go-library v0.7.3-0.20250929125610-2d45c2002dfa h1:lGWIyytdvC5HsokVpBUaKIsRdpoWaucPFXBziuKBQ+U= -github.com/icinga/icinga-go-library v0.7.3-0.20250929125610-2d45c2002dfa/go.mod h1:nQXvnUNKDpJfOLin9cVveUFmiqipMYkoHo5P8O3kjXs= +github.com/icinga/icinga-go-library v0.7.3-0.20251002071510-577e857e2417 h1:OToIpiQyD84lyZ/hEGV5LNBtQWQxPQj/7U4F3FyITD0= +github.com/icinga/icinga-go-library v0.7.3-0.20251002071510-577e857e2417/go.mod h1:nQXvnUNKDpJfOLin9cVveUFmiqipMYkoHo5P8O3kjXs= github.com/jessevdk/go-flags v1.6.1 h1:Cvu5U8UGrLay1rZfv/zP7iLpSHGUZ/Ou68T0iX1bBK4= github.com/jessevdk/go-flags v1.6.1/go.mod h1:Mk8T1hIAWpOiJiHa9rJASDK2UGWji0EuPGBnNLMooyc= github.com/jmoiron/sqlx v1.4.0 h1:1PLqN7S1UYp5t4SrVVnt4nUVNemrDAtxlulVe+Qgm3o= diff --git a/pkg/notifications/notifications.go b/pkg/notifications/notifications.go index 2d9147c40..7ee4612c4 100644 --- a/pkg/notifications/notifications.go +++ b/pkg/notifications/notifications.go @@ -3,6 +3,9 @@ package notifications import ( "context" "fmt" + "net/url" + "sync" + "github.com/icinga/icinga-go-library/database" "github.com/icinga/icinga-go-library/logging" "github.com/icinga/icinga-go-library/notifications/event" @@ -17,8 +20,6 @@ import ( "github.com/pkg/errors" "go.uber.org/zap" "go.uber.org/zap/zapcore" - "net/url" - "sync" ) // Client is an Icinga Notifications compatible client implementation to push events to Icinga Notifications. @@ -31,7 +32,7 @@ type Client struct { db *database.DB logger *logging.Logger - rules *source.RulesInfo // rules holds the latest rules fetched from Icinga Notifications. + rulesInfo *source.RulesInfo // rulesInfo holds the latest rulesInfo fetched from Icinga Notifications. ctx context.Context @@ -55,7 +56,7 @@ func NewNotificationsClient( db: db, logger: logger, - rules: &source.RulesInfo{}, + rulesInfo: &source.RulesInfo{}, redisClient: rc, ctx: ctx, @@ -72,8 +73,8 @@ func NewNotificationsClient( // evaluateRulesForObject returns the rule IDs for each matching query. // -// At the moment, each RuleResp.ObjectFilterExpr is executed as a SQL query after the parameters are being bound. If the -// query returns at least one line, the rule will match. Rules with an empty ObjectFilterExpr are a special case and +// At the moment, each rule filter expression is executed as a SQL query after the parameters are being bound. If the +// query returns at least one line, the rule will match. Rules with an empty filter expression are a special case and // will always match. // // The provided entity is passed as param to the queries, thus they are allowed to use all fields of that specific @@ -86,20 +87,20 @@ func NewNotificationsClient( // // The :host_id and :environment_id parameters will be bound to the entity's ID and EnvironmentId fields, respectively. func (client *Client) evaluateRulesForObject(ctx context.Context, entity database.Entity) ([]string, error) { - outRuleIds := make([]string, 0, len(client.rules.Rules)) + outRuleIds := make([]string, 0, len(client.rulesInfo.Rules)) - for rule := range client.rules.Iter() { - if rule.ObjectFilterExpr == "" { - outRuleIds = append(outRuleIds, rule.Id) + for id, filterExpr := range client.rulesInfo.Rules { + if filterExpr == "" { + outRuleIds = append(outRuleIds, id) continue } evaluates, err := func() (bool, error) { // The raw SQL query in the database is URL-encoded (mostly the space character is replaced by %20). // So, we need to unescape it before passing it to the database. - query, err := url.QueryUnescape(rule.ObjectFilterExpr) + query, err := url.QueryUnescape(filterExpr) if err != nil { - return false, errors.Wrapf(err, "cannot unescape rule %q object filter expression %q", rule.Id, rule.ObjectFilterExpr) + return false, errors.Wrapf(err, "cannot unescape rule %q object filter expression %q", id, filterExpr) } rows, err := client.db.NamedQueryContext(ctx, client.db.Rebind(query), entity) if err != nil { @@ -113,11 +114,11 @@ func (client *Client) evaluateRulesForObject(ctx context.Context, entity databas return true, nil }() if err != nil { - return nil, errors.Wrapf(err, "cannot fetch rule %q from %q", rule.Id, rule.ObjectFilterExpr) + return nil, errors.Wrapf(err, "cannot fetch rule %q from %q", id, filterExpr) } else if !evaluates { continue } - outRuleIds = append(outRuleIds, rule.Id) + outRuleIds = append(outRuleIds, id) } return outRuleIds, nil @@ -395,21 +396,21 @@ func (client *Client) Submit(entity database.Entity) bool { eventRuleIds = []string{} } - ev.RulesVersion = client.rules.Version + ev.RulesVersion = client.rulesInfo.Version ev.RuleIds = eventRuleIds newEventRules, err := client.notificationsClient.ProcessEvent(client.ctx, ev) if errors.Is(err, source.ErrRulesOutdated) { eventLogger.Infow("Cannot submit event to Icinga Notifications due to rule changes, will be retried", - zap.String("old_rules_version", client.rules.Version), + zap.String("old_rules_version", client.rulesInfo.Version), zap.String("new_rules_version", newEventRules.Version)) - client.rules = newEventRules + client.rulesInfo = newEventRules return false } else if err != nil { eventLogger.Errorw("Cannot submit event to Icinga Notifications, will be retried", - zap.String("rules_version", client.rules.Version), + zap.String("rules_version", client.rulesInfo.Version), zap.Any("rules", eventRuleIds), zap.Error(err)) return false From 597d41d4b27065fb95536ee5359a3afdb67386d3 Mon Sep 17 00:00:00 2001 From: Alvar Penning Date: Thu, 2 Oct 2025 11:13:29 +0200 Subject: [PATCH 14/38] Document Notification Source Briefly describe the required configuration for Icinga Notifications Source next to mentioning it in the About section. --- config.example.yml | 4 ++-- doc/01-About.md | 4 ++++ doc/03-Configuration.md | 16 +++++++++++++++- 3 files changed, 21 insertions(+), 3 deletions(-) diff --git a/config.example.yml b/config.example.yml index c4d47c9ca..de5706abf 100644 --- a/config.example.yml +++ b/config.example.yml @@ -146,8 +146,8 @@ redis: # # URL to the API root. # api-base-url: http://localhost:5680 # -# # Source or username to authenticate against the /process-event API. -# username: source-2 +# Username to authenticate against the Icinga Notifications API. +# username: icingadb # # # Password for the defined user. # password: insecureinsecure diff --git a/doc/01-About.md b/doc/01-About.md index c880ac00c..fea23d846 100644 --- a/doc/01-About.md +++ b/doc/01-About.md @@ -34,6 +34,10 @@ Icinga DB Web also connects to the Icinga 2 API with its Command Transport to ac These are the components of Icinga DB embedded into an Icinga setup with Icinga 2 and Icinga Web 2. +Since the Icinga DB daemon always receives the latest information from Redis®, it is an ideal candidate to distribute information further. +In addition to inserting data into a relational database, Icinga DB can also forward events to [Icinga Notifications](https://icinga.com/docs/icinga-notifications/), +as described in the [configuration section](03-Configuration.md#notifications-source-configuration). + ## Installation To install Icinga DB see [Installation](02-Installation.md). diff --git a/doc/03-Configuration.md b/doc/03-Configuration.md index cccfd2233..a1a3a0693 100644 --- a/doc/03-Configuration.md +++ b/doc/03-Configuration.md @@ -146,7 +146,7 @@ ICINGADB_LOGGING_OPTIONS=database:error,high-availability:debug | runtime-updates | Runtime updates of config objects after the initial config synchronization. | | telemetry | Reporting of Icinga DB status to Icinga 2 via Redis® (for monitoring purposes). | -## Retention +## Retention Configuration By default, no historical data is deleted, which means that the longer the data is retained, the more disk space is required to store it. History retention is an optional feature that allows to @@ -174,6 +174,20 @@ ICINGADB_RETENTION_OPTIONS=comment:356 | count | **Optional.** Number of old historical data a single query can delete in a `"DELETE FROM ... LIMIT count"` manner. Defaults to `5000`. | | options | **Optional.** Map of history category to number of days to retain its data. Available categories are `acknowledgement`, `comment`, `downtime`, `flapping`, `notification` and `state`. | +## Notifications Source Configuration + +Icinga DB can act as an event source for [Icinga Notifications](https://icinga.com/docs/icinga-notifications/). +If configured, Icinga DB will submit events to the Icinga Notifications API. + +For YAML configuration, the options are part of the `notifications-source` dictionary. +For environment variables, each option is prefixed with `ICINGADB_NOTIFICATIONS_SOURCE_`. + +| Option | Description | +|--------------|-----------------------------------------------------------------------------------| +| api-base-url | **Optional.** Icinga Notifications API base URL, such as `http://localhost:5680`. | +| username | **Optional.** Icinga Notifications API user for this source. | +| password | **Optional.** Icinga Notifications API user password. | + ## Appendix ### Duration String From cf4bd92611be38ff324f5b3ab46a8e6b7a41ec1e Mon Sep 17 00:00:00 2001 From: Alvar Penning Date: Tue, 21 Oct 2025 10:15:40 +0200 Subject: [PATCH 15/38] Configurable callback sync telemetry stat name Refactor the telemetry.Stats to allow custom names. This enabled dynamic callback names for the Redis history sync, used by Icinga Notifications. --- cmd/icingadb/main.go | 18 +++++-- pkg/icingadb/history/retention.go | 2 +- pkg/icingadb/history/sync.go | 42 ++++++++++----- pkg/icingadb/overdue/sync.go | 2 +- pkg/icingadb/runtime_updates.go | 4 +- pkg/icingadb/sync.go | 4 +- pkg/icingaredis/telemetry/stats.go | 70 ++++++++++++++++++------- pkg/icingaredis/telemetry/stats_test.go | 44 ++++++++++++++++ 8 files changed, 144 insertions(+), 42 deletions(-) create mode 100644 pkg/icingaredis/telemetry/stats_test.go diff --git a/cmd/icingadb/main.go b/cmd/icingadb/main.go index ed5392253..c47cea839 100644 --- a/cmd/icingadb/main.go +++ b/cmd/icingadb/main.go @@ -171,8 +171,11 @@ func run() int { signal.Notify(sig, os.Interrupt, syscall.SIGTERM, syscall.SIGHUP) go func() { - var callback func(database.Entity) bool - var callbackKeyStructPtr map[string]any + var ( + callbackName string + callbackKeyStructPtr map[string]any + callbackFn func(database.Entity) bool + ) if cfg := cmd.Config.NotificationsSource; cfg.ApiBaseUrl != "" { logger.Info("Starting Icinga Notifications source") @@ -183,13 +186,20 @@ func run() int { rc, logs.GetChildLogger("notifications-source"), cfg) - callback = notificationsSource.Submit + + callbackName = "notifications_sync" callbackKeyStructPtr = notifications.SyncKeyStructPtrs + callbackFn = notificationsSource.Submit } logger.Info("Starting history sync") - if err := hs.Sync(ctx, callbackKeyStructPtr, callback); err != nil && !utils.IsContextCanceled(err) { + if err := hs.Sync( + ctx, + callbackName, + callbackKeyStructPtr, + callbackFn, + ); err != nil && !utils.IsContextCanceled(err) { logger.Fatalf("%+v", err) } }() diff --git a/pkg/icingadb/history/retention.go b/pkg/icingadb/history/retention.go index 2d3a6de1a..e9d893bdc 100644 --- a/pkg/icingadb/history/retention.go +++ b/pkg/icingadb/history/retention.go @@ -230,7 +230,7 @@ func (r *Retention) Start(ctx context.Context) error { deleted, err := stmt.CleanupOlderThan( ctx, r.db, e.Id, r.count, olderThan, - database.OnSuccessIncrement[struct{}](&telemetry.Stats.HistoryCleanup), + database.OnSuccessIncrement[struct{}](telemetry.Stats.Get(telemetry.StatHistoryCleanup)), ) if err != nil { select { diff --git a/pkg/icingadb/history/sync.go b/pkg/icingadb/history/sync.go index 82ba93747..49fbfdf6f 100644 --- a/pkg/icingadb/history/sync.go +++ b/pkg/icingadb/history/sync.go @@ -41,13 +41,23 @@ func NewSync(db *database.DB, redis *redis.Client, logger *logging.Logger) *Sync // Sync synchronizes Redis history streams from s.redis to s.db and deletes the original data on success. // -// An optional callback and callbackKeyStructPtr might be given. Both most either be nil or not nil. +// It is possible to enable a callback functionality, e.g., for the Icinga Notifications integration. To do so, the +// optional callbackFn and callbackKeyStructPtr must be set. Both must either be nil or not nil. If set, the additional +// callbackName must also be set, to be used in [telemetry.Stats]. // // The callbackKeyStructPtr says which pipeline keys should be mapped to which type, identified by a struct pointer. If -// a key is missing from the map, it will not be used for the callback. The callback function itself shall not block. -func (s Sync) Sync(ctx context.Context, callbackKeyStructPtr map[string]any, callback func(database.Entity) bool) error { - if (callbackKeyStructPtr == nil) != (callback == nil) { - return fmt.Errorf("either both callbackKeyStructPtr and callback must be nil or none") +// a key is missing from the map, it will not be used for the callback. The callbackFn function shall not block. +func (s Sync) Sync( + ctx context.Context, + callbackName string, + callbackKeyStructPtr map[string]any, + callbackFn func(database.Entity) bool, +) error { + if (callbackKeyStructPtr == nil) != (callbackFn == nil) { + return fmt.Errorf("either both callbackKeyStructPtr and callbackFn must be nil or none") + } + if (callbackKeyStructPtr != nil) && (callbackName == "") { + return fmt.Errorf("if callbackKeyStructPtr and callbackFn are set, a callbackName is required") } g, ctx := errgroup.WithContext(ctx) @@ -87,7 +97,7 @@ func (s Sync) Sync(ctx context.Context, callbackKeyStructPtr map[string]any, cal // Shadowed variable to allow appending custom callbacks. pipeline := pipeline if hasCallbackStage { - pipeline = append(pipeline, makeCallbackStageFunc(callbackKeyStructPtr, callback)) + pipeline = append(pipeline, makeCallbackStageFunc(callbackName, callbackKeyStructPtr, callbackFn)) } ch := make([]chan redis.XMessage, len(pipeline)+1) @@ -402,7 +412,7 @@ func countElementStage(ctx context.Context, _ Sync, _ string, in <-chan redis.XM return nil } - telemetry.Stats.History.Add(1) + telemetry.Stats.Get(telemetry.StatHistory).Add(1) out <- msg case <-ctx.Done(): @@ -423,9 +433,13 @@ func countElementStage(ctx context.Context, _ Sync, _ string, in <-chan redis.XM // callback method, it will be forwarded to the out channel. Thus, this stage might "block" or "hold back" certain // messages during unhappy callback times. // -// For each successfully submitted message, [telemetry.State.Callback] is incremented. Thus, a delta between -// [telemetry.State.History] and [telemetry.State.Callback] indicates blocking callbacks. -func makeCallbackStageFunc(keyStructPtrs map[string]any, callback func(database.Entity) bool) stageFunc { +// For each successfully submitted message, the telemetry stat named after this callback is incremented. Thus, a delta +// between [telemetry.StatHistory] and this stat indicates blocking callbacks. +func makeCallbackStageFunc( + name string, + keyStructPtrs map[string]any, + fn func(database.Entity) bool, +) stageFunc { return func(ctx context.Context, s Sync, key string, in <-chan redis.XMessage, out chan<- redis.XMessage) error { defer close(out) @@ -479,9 +493,9 @@ func makeCallbackStageFunc(keyStructPtrs map[string]any, callback func(database. return err } - if callback(entity) { + if fn(entity) { out <- msg - telemetry.Stats.Callback.Add(1) + telemetry.Stats.Get(name).Add(1) backlogLastId = "" } else { backlogLastId = msg.ID @@ -521,10 +535,10 @@ func makeCallbackStageFunc(keyStructPtrs map[string]any, callback func(database. return errors.Wrapf(err, "can't structify backlog value %q for %q", backlogLastId, key) } - if callback(entity) { + if fn(entity) { out <- msg backlogMsgCounter++ - telemetry.Stats.Callback.Add(1) + telemetry.Stats.Get(name).Add(1) if len(msgs) == 1 { backlogLastId = "" diff --git a/pkg/icingadb/overdue/sync.go b/pkg/icingadb/overdue/sync.go index b1b2f488c..049f217da 100644 --- a/pkg/icingadb/overdue/sync.go +++ b/pkg/icingadb/overdue/sync.go @@ -219,7 +219,7 @@ func (s Sync) updateOverdue( } counter.Add(uint64(len(ids))) - telemetry.Stats.Overdue.Add(uint64(len(ids))) + telemetry.Stats.Get(telemetry.StatOverdue).Add(uint64(len(ids))) var op func(ctx context.Context, key string, members ...any) *redis.IntCmd if overdue { diff --git a/pkg/icingadb/runtime_updates.go b/pkg/icingadb/runtime_updates.go index 888f4b4d9..e5b5efb26 100644 --- a/pkg/icingadb/runtime_updates.go +++ b/pkg/icingadb/runtime_updates.go @@ -184,7 +184,7 @@ func (r *RuntimeUpdates) Sync( return r.db.NamedBulkExec( ctx, cvStmt, cvCount, sem, customvars, database.SplitOnDupId[database.Entity], database.OnSuccessIncrement[database.Entity](&counter), - database.OnSuccessIncrement[database.Entity](&telemetry.Stats.Config), + database.OnSuccessIncrement[database.Entity](telemetry.Stats.Get(telemetry.StatConfig)), ) }) @@ -204,7 +204,7 @@ func (r *RuntimeUpdates) Sync( return r.db.NamedBulkExec( ctx, cvFlatStmt, cvFlatCount, sem, flatCustomvars, database.SplitOnDupId[database.Entity], database.OnSuccessIncrement[database.Entity](&counter), - database.OnSuccessIncrement[database.Entity](&telemetry.Stats.Config), + database.OnSuccessIncrement[database.Entity](telemetry.Stats.Get(telemetry.StatConfig)), ) }) diff --git a/pkg/icingadb/sync.go b/pkg/icingadb/sync.go index b353c7848..73e4d2e39 100644 --- a/pkg/icingadb/sync.go +++ b/pkg/icingadb/sync.go @@ -225,8 +225,8 @@ func (s Sync) SyncCustomvars(ctx context.Context) error { func getCounterForEntity(e database.Entity) *com.Counter { switch e.(type) { case *v1.HostState, *v1.ServiceState: - return &telemetry.Stats.State + return telemetry.Stats.Get(telemetry.StatState) default: - return &telemetry.Stats.Config + return telemetry.Stats.Get(telemetry.StatConfig) } } diff --git a/pkg/icingaredis/telemetry/stats.go b/pkg/icingaredis/telemetry/stats.go index a34845128..bbe8c03cb 100644 --- a/pkg/icingaredis/telemetry/stats.go +++ b/pkg/icingaredis/telemetry/stats.go @@ -2,40 +2,74 @@ package telemetry import ( "context" + "fmt" "github.com/icinga/icinga-go-library/com" "github.com/icinga/icinga-go-library/logging" "github.com/icinga/icinga-go-library/periodic" "github.com/icinga/icinga-go-library/redis" "github.com/icinga/icinga-go-library/utils" "go.uber.org/zap" + "iter" "strconv" + "sync" "time" ) -var Stats struct { - // Config & co. are to be increased by the T sync once for every T object synced. - Config com.Counter - State com.Counter - History com.Counter - Callback com.Counter - Overdue com.Counter - HistoryCleanup com.Counter +// StatsKeeper holds multiple [com.Counter] values by name, to be used for statistics in WriteStats. +type StatsKeeper struct { + m sync.Map } -// WriteStats periodically forwards Stats to Redis for being monitored by Icinga 2. -func WriteStats(ctx context.Context, client *redis.Client, logger *logging.Logger) { - counters := map[string]*com.Counter{ - "config_sync": &Stats.Config, - "state_sync": &Stats.State, - "history_sync": &Stats.History, - "callback_sync": &Stats.Callback, - "overdue_sync": &Stats.Overdue, - "history_cleanup": &Stats.HistoryCleanup, +// Get or create a [com.Counter] by its name. +func (statsKeeper *StatsKeeper) Get(key string) *com.Counter { + ctrAny, _ := statsKeeper.m.LoadOrStore(key, &com.Counter{}) + + ctr, ok := ctrAny.(*com.Counter) + if !ok { + // Should not happen unless someone messes with the internal map. + panic(fmt.Sprintf( + "StatsKeeper.Get(%q) returned something of type %T, not *com.Counter", + key, ctrAny)) } + return ctr +} + +// Iterator over all keys and their [com.Counter]. +func (statsKeeper *StatsKeeper) Iterator() iter.Seq2[string, *com.Counter] { + return func(yield func(string, *com.Counter) bool) { + statsKeeper.m.Range(func(keyAny, ctrAny any) bool { + key, keyOk := keyAny.(string) + ctr, ctrOk := ctrAny.(*com.Counter) + if !keyOk || !ctrOk { + // Should not happen unless someone messes with the internal map. + panic(fmt.Sprintf( + "iterating StatsKeeper on key %q got types (%T, %T), not (string, *com.Counter)", + keyAny, keyAny, ctrAny)) + } + + return yield(key, ctr) + }) + } +} + +// Stats is the singleton StatsKeeper to be used to access a [com.Counter]. +var Stats = &StatsKeeper{} + +// Keys for different well known Stats entries. +const ( + StatConfig = "config_sync" + StatState = "state_sync" + StatHistory = "history_sync" + StatOverdue = "overdue_sync" + StatHistoryCleanup = "history_cleanup" +) + +// WriteStats periodically forwards Stats to Redis for being monitored by Icinga 2. +func WriteStats(ctx context.Context, client *redis.Client, logger *logging.Logger) { periodic.Start(ctx, time.Second, func(_ periodic.Tick) { var data []string - for kind, counter := range counters { + for kind, counter := range Stats.Iterator() { if cnt := counter.Reset(); cnt > 0 { data = append(data, kind, strconv.FormatUint(cnt, 10)) } diff --git a/pkg/icingaredis/telemetry/stats_test.go b/pkg/icingaredis/telemetry/stats_test.go new file mode 100644 index 000000000..4b851e989 --- /dev/null +++ b/pkg/icingaredis/telemetry/stats_test.go @@ -0,0 +1,44 @@ +package telemetry + +import ( + "github.com/stretchr/testify/assert" + "testing" +) + +func TestStatsKeeper(t *testing.T) { + desiredState := map[string]uint64{ + "foo": 23, + "bar": 42, + "baz": 0, + } + + stats := &StatsKeeper{} + + // Populate based on desiredState + for key, counterValue := range desiredState { + ctr := stats.Get(key) + ctr.Add(counterValue) + } + + // Check if desiredState is set + for key, counterValue := range desiredState { + ctr := stats.Get(key) + assert.Equal(t, counterValue, ctr.Val()) + } + + // Get reference, change value, compare + fooKey := "foo" + fooCtr := stats.Get(fooKey) + assert.Equal(t, desiredState[fooKey], fooCtr.Reset()) + assert.Equal(t, uint64(0), fooCtr.Val()) + assert.Equal(t, uint64(0), stats.Get(fooKey).Val()) + fooCtr.Add(desiredState[fooKey]) + assert.Equal(t, desiredState[fooKey], stats.Get(fooKey).Val()) + + // Range over + for key, ctr := range stats.Iterator() { + desired, ok := desiredState[key] + assert.True(t, ok) + assert.Equal(t, desired, ctr.Val()) + } +} From c24bc27dc3238042736d870a576acfeb995b1c4e Mon Sep 17 00:00:00 2001 From: Alvar Penning Date: Tue, 21 Oct 2025 16:36:04 +0200 Subject: [PATCH 16/38] Minor Tweaks for Icinga Notifications Integration - Don't validate notifications config in a background Goroutine. - Clip pipeline slice to avoid reusing capability twice. - Rework notification Client.buildCommonEvent and depending methods. - Resubmit events after updating rules in one go. - Simplify Client.fetchHostServiceName based on Julian's suggestion. Co-Authored-By: Julian Brost --- cmd/icingadb/main.go | 29 ++--- pkg/icingadb/history/sync.go | 5 +- pkg/icingadb/v1/history/downtime.go | 3 + pkg/notifications/notifications.go | 157 +++++++++++++--------------- pkg/notifications/redis_fetch.go | 55 +++++----- 5 files changed, 121 insertions(+), 128 deletions(-) diff --git a/cmd/icingadb/main.go b/cmd/icingadb/main.go index c47cea839..aca620313 100644 --- a/cmd/icingadb/main.go +++ b/cmd/icingadb/main.go @@ -170,7 +170,7 @@ func run() int { sig := make(chan os.Signal, 1) signal.Notify(sig, os.Interrupt, syscall.SIGTERM, syscall.SIGHUP) - go func() { + { var ( callbackName string callbackKeyStructPtr map[string]any @@ -180,29 +180,34 @@ func run() int { if cfg := cmd.Config.NotificationsSource; cfg.ApiBaseUrl != "" { logger.Info("Starting Icinga Notifications source") - notificationsSource := notifications.NewNotificationsClient( + notificationsSource, err := notifications.NewNotificationsClient( ctx, db, rc, logs.GetChildLogger("notifications-source"), cfg) + if err != nil { + logger.Fatalw("Can't create Icinga Notifications client from config", zap.Error(err)) + } callbackName = "notifications_sync" callbackKeyStructPtr = notifications.SyncKeyStructPtrs callbackFn = notificationsSource.Submit } - logger.Info("Starting history sync") + go func() { + logger.Info("Starting history sync") - if err := hs.Sync( - ctx, - callbackName, - callbackKeyStructPtr, - callbackFn, - ); err != nil && !utils.IsContextCanceled(err) { - logger.Fatalf("%+v", err) - } - }() + if err := hs.Sync( + ctx, + callbackName, + callbackKeyStructPtr, + callbackFn, + ); err != nil && !utils.IsContextCanceled(err) { + logger.Fatalf("%+v", err) + } + }() + } // Main loop for { diff --git a/pkg/icingadb/history/sync.go b/pkg/icingadb/history/sync.go index 49fbfdf6f..f9f628384 100644 --- a/pkg/icingadb/history/sync.go +++ b/pkg/icingadb/history/sync.go @@ -19,6 +19,7 @@ import ( "go.uber.org/zap" "golang.org/x/sync/errgroup" "reflect" + "slices" "sync" "time" ) @@ -97,7 +98,7 @@ func (s Sync) Sync( // Shadowed variable to allow appending custom callbacks. pipeline := pipeline if hasCallbackStage { - pipeline = append(pipeline, makeCallbackStageFunc(callbackName, callbackKeyStructPtr, callbackFn)) + pipeline = append(slices.Clip(pipeline), makeCallbackStageFunc(callbackName, callbackKeyStructPtr, callbackFn)) } ch := make([]chan redis.XMessage, len(pipeline)+1) @@ -470,7 +471,7 @@ func makeCallbackStageFunc( backlogLastId := "" backlogMsgCounter := 0 - const backlogTimerMinInterval, backlogTimerMaxInterval = 10 * time.Millisecond, time.Minute + const backlogTimerMinInterval, backlogTimerMaxInterval = time.Millisecond, time.Minute backlogTimerInterval := backlogTimerMinInterval backlogTimer := time.NewTimer(backlogTimerInterval) _ = backlogTimer.Stop() diff --git a/pkg/icingadb/v1/history/downtime.go b/pkg/icingadb/v1/history/downtime.go index bbbfbbc15..e43f918d7 100644 --- a/pkg/icingadb/v1/history/downtime.go +++ b/pkg/icingadb/v1/history/downtime.go @@ -88,6 +88,9 @@ func (*HistoryDowntime) TableName() string { return "history" } +// DowntimeHistoryMeta is a combined HistoryMeta struct of DowntimeHistoryEntity and DowntimeHistory. +// +// It is used in the notifications package and became necessary as values of both structs were required. type DowntimeHistoryMeta struct { DowntimeHistoryEntity `json:",inline"` DowntimeHistory `json:",inline"` diff --git a/pkg/notifications/notifications.go b/pkg/notifications/notifications.go index 7ee4612c4..25df8094a 100644 --- a/pkg/notifications/notifications.go +++ b/pkg/notifications/notifications.go @@ -49,26 +49,25 @@ func NewNotificationsClient( rc *redis.Client, logger *logging.Logger, cfg source.Config, -) *Client { - client := &Client{ +) (*Client, error) { + notificationsClient, err := source.NewClient(cfg, "Icinga DB "+internal.Version.Version) + if err != nil { + return nil, err + } + + return &Client{ Config: cfg, db: db, logger: logger, - rulesInfo: &source.RulesInfo{}, - redisClient: rc, - ctx: ctx, - } - notificationsClient, err := source.NewClient(client.Config, fmt.Sprintf("Icinga DB %s", internal.Version.Version)) - if err != nil { - logger.Fatalw("Cannot create Icinga Notifications client", zap.Error(err)) - } - client.notificationsClient = notificationsClient + rulesInfo: &source.RulesInfo{}, - return client + notificationsClient: notificationsClient, + redisClient: rc, + }, nil } // evaluateRulesForObject returns the rule IDs for each matching query. @@ -124,34 +123,36 @@ func (client *Client) evaluateRulesForObject(ctx context.Context, entity databas return outRuleIds, nil } -// buildCommonEvent creates an event.Event based on Host and (optional) Service names. +// buildCommonEvent creates an event.Event based on Host and (optional) Service IDs. // // This function is used by all event builders to create a common event structure that includes the host and service -// names, the absolute URL to the Icinga Web 2 Icinga DB page for the host or service, and the tags for the event. +// names, an Icinga DB Web reference, and the tags for the event. // Any event type-specific information (like severity, message, etc.) is added by the specific event builders. -func (client *Client) buildCommonEvent(rlr *redisLookupResult) (*event.Event, error) { +func (client *Client) buildCommonEvent( + ctx context.Context, + hostId, serviceId types.Binary, +) (*event.Event, *redisLookupResult, error) { + rlr, err := client.fetchHostServiceName(ctx, hostId, serviceId) + if err != nil { + return nil, nil, err + } + var ( objectName string - objectUrl url.URL + objectUrl string objectTags map[string]string ) if rlr.ServiceName != "" { objectName = rlr.HostName + "!" + rlr.ServiceName - - objectUrl.Path = "/icingadb/service" - objectUrl.RawQuery = "name=" + utils.RawUrlEncode(rlr.ServiceName) + "&host.name=" + utils.RawUrlEncode(rlr.HostName) - + objectUrl = "/icingadb/service?name=" + utils.RawUrlEncode(rlr.ServiceName) + "&host.name=" + utils.RawUrlEncode(rlr.HostName) objectTags = map[string]string{ "host": rlr.HostName, "service": rlr.ServiceName, } } else { objectName = rlr.HostName - - objectUrl.Path = "/icingadb/host" - objectUrl.RawQuery = "name=" + utils.RawUrlEncode(rlr.HostName) - + objectUrl = "/icingadb/host?name=" + utils.RawUrlEncode(rlr.HostName) objectTags = map[string]string{ "host": rlr.HostName, } @@ -159,9 +160,9 @@ func (client *Client) buildCommonEvent(rlr *redisLookupResult) (*event.Event, er return &event.Event{ Name: objectName, - URL: objectUrl.String(), + URL: objectUrl, Tags: objectTags, - }, nil + }, rlr, nil } // buildStateHistoryEvent builds a fully initialized event.Event from a state history entry. @@ -169,19 +170,14 @@ func (client *Client) buildCommonEvent(rlr *redisLookupResult) (*event.Event, er // The resulted event will have all the necessary information for a state change event, and must // not be further modified by the caller. func (client *Client) buildStateHistoryEvent(ctx context.Context, h *v1history.StateHistory) (*event.Event, error) { - res, err := client.fetchHostServiceName(ctx, h.HostId, h.ServiceId) - if err != nil { - return nil, err - } - - ev, err := client.buildCommonEvent(res) + ev, rlr, err := client.buildCommonEvent(ctx, h.HostId, h.ServiceId) if err != nil { - return nil, errors.Wrapf(err, "cannot build event for %q,%q", res.HostName, res.ServiceName) + return nil, errors.Wrapf(err, "cannot build event for %q,%q", h.HostId, h.ServiceId) } ev.Type = event.TypeState - if res.ServiceName != "" { + if rlr.ServiceName != "" { switch h.HardState { case 0: ev.Severity = event.SeverityOK @@ -217,14 +213,9 @@ func (client *Client) buildStateHistoryEvent(ctx context.Context, h *v1history.S // buildDowntimeHistoryMetaEvent from a downtime history entry. func (client *Client) buildDowntimeHistoryMetaEvent(ctx context.Context, h *v1history.DowntimeHistoryMeta) (*event.Event, error) { - res, err := client.fetchHostServiceName(ctx, h.HostId, h.ServiceId) + ev, _, err := client.buildCommonEvent(ctx, h.HostId, h.ServiceId) if err != nil { - return nil, err - } - - ev, err := client.buildCommonEvent(res) - if err != nil { - return nil, errors.Wrapf(err, "cannot build event for %q,%q", res.HostName, res.ServiceName) + return nil, errors.Wrapf(err, "cannot build event for %q,%q", h.HostId, h.ServiceId) } switch h.EventType { @@ -236,6 +227,7 @@ func (client *Client) buildDowntimeHistoryMetaEvent(ctx context.Context, h *v1hi ev.MuteReason = "Checkable is in downtime" case "downtime_end": + ev.Mute = types.MakeBool(false) if h.HasBeenCancelled.Valid && h.HasBeenCancelled.Bool { ev.Type = event.TypeDowntimeRemoved ev.Message = "Downtime was cancelled" @@ -257,14 +249,9 @@ func (client *Client) buildDowntimeHistoryMetaEvent(ctx context.Context, h *v1hi // buildFlappingHistoryEvent from a flapping history entry. func (client *Client) buildFlappingHistoryEvent(ctx context.Context, h *v1history.FlappingHistory) (*event.Event, error) { - res, err := client.fetchHostServiceName(ctx, h.HostId, h.ServiceId) - if err != nil { - return nil, err - } - - ev, err := client.buildCommonEvent(res) + ev, _, err := client.buildCommonEvent(ctx, h.HostId, h.ServiceId) if err != nil { - return nil, errors.Wrapf(err, "cannot build event for %q,%q", res.HostName, res.ServiceName) + return nil, errors.Wrapf(err, "cannot build event for %q,%q", h.HostId, h.ServiceId) } if h.PercentStateChangeEnd.Valid { @@ -288,14 +275,9 @@ func (client *Client) buildFlappingHistoryEvent(ctx context.Context, h *v1histor // buildAcknowledgementHistoryEvent from an acknowledgment history entry. func (client *Client) buildAcknowledgementHistoryEvent(ctx context.Context, h *v1history.AcknowledgementHistory) (*event.Event, error) { - res, err := client.fetchHostServiceName(ctx, h.HostId, h.ServiceId) + ev, _, err := client.buildCommonEvent(ctx, h.HostId, h.ServiceId) if err != nil { - return nil, err - } - - ev, err := client.buildCommonEvent(res) - if err != nil { - return nil, errors.Wrapf(err, "cannot build event for %q,%q", res.HostName, res.ServiceName) + return nil, errors.Wrapf(err, "cannot build event for %q,%q", h.HostId, h.ServiceId) } if !h.ClearTime.Time().IsZero() { @@ -334,9 +316,6 @@ func (client *Client) buildAcknowledgementHistoryEvent(ctx context.Context, h *v // // This method usees the Client's logger. func (client *Client) Submit(entity database.Entity) bool { - client.submissionMutex.Lock() - defer client.submissionMutex.Unlock() - if client.ctx.Err() != nil { client.logger.Errorw("Cannot process submitted entity as client context is done", zap.Error(client.ctx.Err())) return true @@ -345,7 +324,7 @@ func (client *Client) Submit(entity database.Entity) bool { var ev *event.Event var eventErr error - // Keep the type switch in sync with syncPipelines from pkg/icingadb/history/sync.go + // Keep the type switch in sync with the values of SyncKeyStructPtrs below. switch h := entity.(type) { case *v1history.AcknowledgementHistory: ev, eventErr = client.buildAcknowledgementHistoryEvent(client.ctx, h) @@ -388,36 +367,48 @@ func (client *Client) Submit(entity database.Entity) bool { }), )) - eventRuleIds, err := client.evaluateRulesForObject(client.ctx, entity) - if err != nil { - // While returning false would be more correct, this would result in never being able to refetch new rule - // versions. Consider an invalid object filter expression, which is now impossible to get rid of. - eventLogger.Errorw("Cannot evaluate rules for event, assuming no rule matched", zap.Error(err)) - eventRuleIds = []string{} - } + // The following code accesses Client.rulesInfo. + client.submissionMutex.Lock() + defer client.submissionMutex.Unlock() + + // This loop allows resubmitting an event if the rules have changed. The first try would be the rule update, the + // second try would be the resubmit, and the third try would be for bad luck, e.g., when a second rule update just + // crept in between. If there are three subsequent rule updates, something is wrong. + for try := 0; try < 3; try++ { + eventRuleIds, err := client.evaluateRulesForObject(client.ctx, entity) + if err != nil { + // While returning false would be more correct, this would result in never being able to refetch new rule + // versions. Consider an invalid object filter expression, which is now impossible to get rid of. + eventLogger.Errorw("Cannot evaluate rules for event, assuming no rule matched", zap.Error(err)) + eventRuleIds = []string{} + } + + ev.RulesVersion = client.rulesInfo.Version + ev.RuleIds = eventRuleIds - ev.RulesVersion = client.rulesInfo.Version - ev.RuleIds = eventRuleIds + newEventRules, err := client.notificationsClient.ProcessEvent(client.ctx, ev) + if errors.Is(err, source.ErrRulesOutdated) { + eventLogger.Infow("Received a rule update from Icinga Notification, resubmitting event", + zap.String("old_rules_version", client.rulesInfo.Version), + zap.String("new_rules_version", newEventRules.Version)) - newEventRules, err := client.notificationsClient.ProcessEvent(client.ctx, ev) - if errors.Is(err, source.ErrRulesOutdated) { - eventLogger.Infow("Cannot submit event to Icinga Notifications due to rule changes, will be retried", - zap.String("old_rules_version", client.rulesInfo.Version), - zap.String("new_rules_version", newEventRules.Version)) + client.rulesInfo = newEventRules - client.rulesInfo = newEventRules + continue + } else if err != nil { + eventLogger.Errorw("Cannot submit event to Icinga Notifications, will be retried", + zap.String("rules_version", client.rulesInfo.Version), + zap.Any("rules", eventRuleIds), + zap.Error(err)) + return false + } - return false - } else if err != nil { - eventLogger.Errorw("Cannot submit event to Icinga Notifications, will be retried", - zap.String("rules_version", client.rulesInfo.Version), - zap.Any("rules", eventRuleIds), - zap.Error(err)) - return false + eventLogger.Debugw("Successfully submitted event to Icinga Notifications", zap.Any("rules", eventRuleIds)) + return true } - eventLogger.Debugw("Successfully submitted event to Icinga Notifications", zap.Any("rules", eventRuleIds)) - return true + eventLogger.Error("Received three rule updates from Icinga Notifications in a row, event will be retried") + return false } var SyncKeyStructPtrs = map[string]any{ diff --git a/pkg/notifications/redis_fetch.go b/pkg/notifications/redis_fetch.go index 3cacde16c..c221400d3 100644 --- a/pkg/notifications/redis_fetch.go +++ b/pkg/notifications/redis_fetch.go @@ -24,40 +24,52 @@ import ( // request and return an error indicating that the operation timed out. In case of the serviceId being set, the // maximum execution time of the Redis HGet commands is 10s (5s for each HGet call). func (client *Client) fetchHostServiceName(ctx context.Context, hostId, serviceId types.Binary) (*redisLookupResult, error) { - redisHGet := func(typ, field string, out *redisLookupResult) error { + getNameFromRedis := func(typ, id string) (string, error) { ctx, cancel := context.WithTimeout(ctx, 5*time.Second) defer cancel() + var data string err := retry.WithBackoff( ctx, - func(ctx context.Context) error { return client.redisClient.HGet(ctx, "icinga:"+typ, field).Scan(out) }, + func(ctx context.Context) (err error) { + data, err = client.redisClient.HGet(ctx, "icinga:"+typ, id).Result() + return + }, retry.Retryable, backoff.DefaultBackoff, retry.Settings{}, ) if err != nil { if errors.Is(err, redis.Nil) { - return fmt.Errorf("%s with ID %s not found in Redis", typ, hostId) + return "", fmt.Errorf("%s with ID %s not found in Redis", typ, hostId) } - return fmt.Errorf("failed to fetch %s with ID %s from Redis: %w", typ, field, err) + return "", fmt.Errorf("failed to fetch %s with ID %s from Redis: %w", typ, id, err) } - return nil + + var result struct { + Name string `json:"name"` + } + + if err := json.Unmarshal([]byte(data), &result); err != nil { + return "", fmt.Errorf("failed to unmarshal redis result: %w", err) + } + + return result.Name, nil } var result redisLookupResult - if err := redisHGet("host", hostId.String(), &result); err != nil { + var err error + + result.HostName, err = getNameFromRedis("host", hostId.String()) + if err != nil { return nil, err } - result.HostName = result.Name - result.Name = "" // Clear the name field for the host, as we will fetch the service name next. - if serviceId != nil { - if err := redisHGet("service", serviceId.String(), &result); err != nil { + result.ServiceName, err = getNameFromRedis("service", serviceId.String()) + if err != nil { return nil, err } - result.ServiceName = result.Name - result.Name = "" // It's not needed anymore, clear it! } return &result, nil @@ -67,23 +79,4 @@ func (client *Client) fetchHostServiceName(ctx context.Context, hostId, serviceI type redisLookupResult struct { HostName string `json:"-"` // Name of the host (never empty). ServiceName string `json:"-"` // Name of the service (only set in service context). - - // Name is used to retrieve the host or service name from Redis. - // It should not be used for any other purpose apart from within the [Client.fetchHostServiceName] function. - Name string `json:"name"` -} - -// UnmarshalBinary implements the [encoding.BinaryUnmarshaler] interface for redisLookupResult. -// -// It unmarshals the binary data of the Redis HGet result into the redisLookupResult struct. -// This is required for the HGet().Scan() usage in the [Client.fetchHostServiceName] function to work correctly. -func (rlr *redisLookupResult) UnmarshalBinary(data []byte) error { - if len(data) == 0 { - return errors.New("empty data received for redisLookupResult") - } - - if err := json.Unmarshal(data, rlr); err != nil { - return fmt.Errorf("failed to unmarshal redis result: %w", err) - } - return nil } From 72dce9b8971b0308590ddc7b982588478a708c73 Mon Sep 17 00:00:00 2001 From: Alvar Penning Date: Wed, 22 Oct 2025 18:07:23 +0200 Subject: [PATCH 17/38] notifications: Fetch customvars from Redis After reintroducing Event.ExtraTags in the IGL and Icinga Notifications, Icinga DB populates events by their custom variables. At the moment, the required customvars are fetched from Redis for each event. Due to the Redis schema, at least on HGETALL with manual filtering is required. This might be a good candidate for further caching, and cache invalidation. --- go.mod | 4 +- go.sum | 8 +- pkg/icingadb/history/sync.go | 2 +- pkg/notifications/notifications.go | 29 +++--- pkg/notifications/redis_fetch.go | 148 ++++++++++++++++++++++++----- 5 files changed, 145 insertions(+), 46 deletions(-) diff --git a/go.mod b/go.mod index b6ab6584f..81b24f41d 100644 --- a/go.mod +++ b/go.mod @@ -7,13 +7,12 @@ require ( github.com/goccy/go-yaml v1.13.0 github.com/google/go-cmp v0.7.0 github.com/google/uuid v1.6.0 - github.com/icinga/icinga-go-library v0.7.3-0.20251002071510-577e857e2417 + github.com/icinga/icinga-go-library v0.7.3-0.20251022120618-6600889adc38 github.com/jessevdk/go-flags v1.6.1 github.com/jmoiron/sqlx v1.4.0 github.com/mattn/go-sqlite3 v1.14.32 github.com/okzk/sdnotify v0.0.0-20180710141335-d9becc38acbd github.com/pkg/errors v0.9.1 - github.com/redis/go-redis/v9 v9.14.0 github.com/stretchr/testify v1.11.1 github.com/vbauerster/mpb/v6 v6.0.4 go.uber.org/zap v1.27.0 @@ -35,6 +34,7 @@ require ( github.com/mattn/go-isatty v0.0.20 // indirect github.com/mattn/go-runewidth v0.0.12 // indirect github.com/pmezard/go-difflib v1.0.0 // indirect + github.com/redis/go-redis/v9 v9.14.1 // indirect github.com/rivo/uniseg v0.2.0 // indirect github.com/ssgreg/journald v1.0.0 // indirect go.uber.org/multierr v1.11.0 // indirect diff --git a/go.sum b/go.sum index db2c137c9..d544422f6 100644 --- a/go.sum +++ b/go.sum @@ -37,8 +37,8 @@ github.com/google/go-cmp v0.7.0 h1:wk8382ETsv4JYUZwIsn6YpYiWiBsYLSJiTsyBybVuN8= github.com/google/go-cmp v0.7.0/go.mod h1:pXiqmnSA92OHEEa9HXL2W4E7lf9JzCmGVUdgjX3N/iU= github.com/google/uuid v1.6.0 h1:NIvaJDMOsjHA8n1jAhLSgzrAzy1Hgr+hNrb57e+94F0= github.com/google/uuid v1.6.0/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= -github.com/icinga/icinga-go-library v0.7.3-0.20251002071510-577e857e2417 h1:OToIpiQyD84lyZ/hEGV5LNBtQWQxPQj/7U4F3FyITD0= -github.com/icinga/icinga-go-library v0.7.3-0.20251002071510-577e857e2417/go.mod h1:nQXvnUNKDpJfOLin9cVveUFmiqipMYkoHo5P8O3kjXs= +github.com/icinga/icinga-go-library v0.7.3-0.20251022120618-6600889adc38 h1:5RNrPZCwvqm2/06i9dUCJtcBV+tR8WgUKtHne2sOaA8= +github.com/icinga/icinga-go-library v0.7.3-0.20251022120618-6600889adc38/go.mod h1:L80M/ufoqFJJjZcdnfsTp6eFl06vm3JuvSWlGcDf708= github.com/jessevdk/go-flags v1.6.1 h1:Cvu5U8UGrLay1rZfv/zP7iLpSHGUZ/Ou68T0iX1bBK4= github.com/jessevdk/go-flags v1.6.1/go.mod h1:Mk8T1hIAWpOiJiHa9rJASDK2UGWji0EuPGBnNLMooyc= github.com/jmoiron/sqlx v1.4.0 h1:1PLqN7S1UYp5t4SrVVnt4nUVNemrDAtxlulVe+Qgm3o= @@ -63,8 +63,8 @@ github.com/pkg/errors v0.9.1 h1:FEBLx1zS214owpjy7qsBeixbURkuhQAwrK5UwLGTwt4= github.com/pkg/errors v0.9.1/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= github.com/pmezard/go-difflib v1.0.0 h1:4DBwDE0NGyQoBHbLQYPwSUPoCMWR5BEzIk/f1lZbAQM= github.com/pmezard/go-difflib v1.0.0/go.mod h1:iKH77koFhYxTK1pcRnkKkqfTogsbg7gZNVY4sRDYZ/4= -github.com/redis/go-redis/v9 v9.14.0 h1:u4tNCjXOyzfgeLN+vAZaW1xUooqWDqVEsZN0U01jfAE= -github.com/redis/go-redis/v9 v9.14.0/go.mod h1:huWgSWd8mW6+m0VPhJjSSQ+d6Nh1VICQ6Q5lHuCH/Iw= +github.com/redis/go-redis/v9 v9.14.1 h1:nDCrEiJmfOWhD76xlaw+HXT0c9hfNWeXgl0vIRYSDvQ= +github.com/redis/go-redis/v9 v9.14.1/go.mod h1:huWgSWd8mW6+m0VPhJjSSQ+d6Nh1VICQ6Q5lHuCH/Iw= github.com/rivo/uniseg v0.1.0/go.mod h1:J6wj4VEh+S6ZtnVlnTBMWIodfgj8LQOQFoIToxlJtxc= github.com/rivo/uniseg v0.2.0 h1:S1pD9weZBuJdFmowNwbpi7BJ8TNftyUImj/0WQi72jY= github.com/rivo/uniseg v0.2.0/go.mod h1:J6wj4VEh+S6ZtnVlnTBMWIodfgj8LQOQFoIToxlJtxc= diff --git a/pkg/icingadb/history/sync.go b/pkg/icingadb/history/sync.go index f9f628384..6cf8a499a 100644 --- a/pkg/icingadb/history/sync.go +++ b/pkg/icingadb/history/sync.go @@ -543,7 +543,7 @@ func makeCallbackStageFunc( if len(msgs) == 1 { backlogLastId = "" - logger.Infow("Finished rolling back backlog of callback elements", zap.Int("delay", backlogMsgCounter)) + logger.Infow("Finished rolling back backlog of callback elements", zap.Int("elements", backlogMsgCounter)) } else { backlogLastId = msgs[1].ID backlogTimerInterval = backlogTimerMinInterval diff --git a/pkg/notifications/notifications.go b/pkg/notifications/notifications.go index 25df8094a..630eaa901 100644 --- a/pkg/notifications/notifications.go +++ b/pkg/notifications/notifications.go @@ -39,7 +39,7 @@ type Client struct { notificationsClient *source.Client // The Icinga Notifications client used to interact with the API. redisClient *redis.Client // redisClient is the Redis client used to fetch host and service names for events. - submissionMutex sync.Mutex + submissionMutex sync.Mutex // submissionMutex protects not concurrent safe struct fields in Client.Submit, i.e., rulesInfo. } // NewNotificationsClient creates a new Client connected to an existing database and logger. @@ -132,7 +132,7 @@ func (client *Client) buildCommonEvent( ctx context.Context, hostId, serviceId types.Binary, ) (*event.Event, *redisLookupResult, error) { - rlr, err := client.fetchHostServiceName(ctx, hostId, serviceId) + rlr, err := client.fetchHostServiceFromRedis(ctx, hostId, serviceId) if err != nil { return nil, nil, err } @@ -143,25 +143,26 @@ func (client *Client) buildCommonEvent( objectTags map[string]string ) - if rlr.ServiceName != "" { - objectName = rlr.HostName + "!" + rlr.ServiceName - objectUrl = "/icingadb/service?name=" + utils.RawUrlEncode(rlr.ServiceName) + "&host.name=" + utils.RawUrlEncode(rlr.HostName) + if rlr.serviceName != "" { + objectName = rlr.hostName + "!" + rlr.serviceName + objectUrl = "/icingadb/service?name=" + utils.RawUrlEncode(rlr.serviceName) + "&host.name=" + utils.RawUrlEncode(rlr.hostName) objectTags = map[string]string{ - "host": rlr.HostName, - "service": rlr.ServiceName, + "host": rlr.hostName, + "service": rlr.serviceName, } } else { - objectName = rlr.HostName - objectUrl = "/icingadb/host?name=" + utils.RawUrlEncode(rlr.HostName) + objectName = rlr.hostName + objectUrl = "/icingadb/host?name=" + utils.RawUrlEncode(rlr.hostName) objectTags = map[string]string{ - "host": rlr.HostName, + "host": rlr.hostName, } } return &event.Event{ - Name: objectName, - URL: objectUrl, - Tags: objectTags, + Name: objectName, + URL: objectUrl, + Tags: objectTags, + ExtraTags: rlr.CustomVars(), }, rlr, nil } @@ -177,7 +178,7 @@ func (client *Client) buildStateHistoryEvent(ctx context.Context, h *v1history.S ev.Type = event.TypeState - if rlr.ServiceName != "" { + if rlr.serviceName != "" { switch h.HardState { case 0: ev.Severity = event.SeverityOK diff --git a/pkg/notifications/redis_fetch.go b/pkg/notifications/redis_fetch.go index c221400d3..af9666dd2 100644 --- a/pkg/notifications/redis_fetch.go +++ b/pkg/notifications/redis_fetch.go @@ -3,36 +3,56 @@ package notifications import ( "context" "encoding/json" - "errors" "fmt" "time" "github.com/icinga/icinga-go-library/backoff" "github.com/icinga/icinga-go-library/retry" "github.com/icinga/icinga-go-library/types" - "github.com/redis/go-redis/v9" ) -// fetchHostServiceName retrieves the host and service names from Redis. +// redisCustomVar is a customvar entry from Redis. +type redisCustomVar struct { + EnvironmentID types.Binary `json:"environment_id"` + Name string `json:"name"` + Value string `json:"value"` +} + +// redisLookupResult defines the structure of the Redis message we're interested in. +type redisLookupResult struct { + hostName string + serviceName string + customVars []*redisCustomVar +} + +// CustomVars returns a mapping of customvar names to values. +func (result redisLookupResult) CustomVars() map[string]string { + m := make(map[string]string) + for _, customvar := range result.customVars { + m[customvar.Name] = customvar.Value + } + + return m +} + +// fetchHostServiceFromRedis retrieves the host and service names and customvars from Redis. // // It uses either the hostId or/and serviceId to fetch the corresponding names. If both are provided, // the returned result will contain the host name and the service name accordingly. Otherwise, it will // only contain the host name. // -// Internally, it uses the Redis HGet command to fetch the data from the "icinga:host" and "icinga:service" hashes. -// If this operation couldn't be completed within a reasonable time (a hard coded 5 seconds), it will cancel the -// request and return an error indicating that the operation timed out. In case of the serviceId being set, the -// maximum execution time of the Redis HGet commands is 10s (5s for each HGet call). -func (client *Client) fetchHostServiceName(ctx context.Context, hostId, serviceId types.Binary) (*redisLookupResult, error) { - getNameFromRedis := func(typ, id string) (string, error) { - ctx, cancel := context.WithTimeout(ctx, 5*time.Second) - defer cancel() +// The function has a hard coded timeout of five seconds for all HGET and HGETALL commands together. +func (client *Client) fetchHostServiceFromRedis(ctx context.Context, hostId, serviceId types.Binary) (*redisLookupResult, error) { + var cancel context.CancelFunc + ctx, cancel = context.WithTimeout(ctx, 5*time.Second) + defer cancel() + hgetFromRedis := func(key, id string) (string, error) { var data string err := retry.WithBackoff( ctx, func(ctx context.Context) (err error) { - data, err = client.redisClient.HGet(ctx, "icinga:"+typ, id).Result() + data, err = client.redisClient.HGet(ctx, key, id).Result() return }, retry.Retryable, @@ -40,16 +60,21 @@ func (client *Client) fetchHostServiceName(ctx context.Context, hostId, serviceI retry.Settings{}, ) if err != nil { - if errors.Is(err, redis.Nil) { - return "", fmt.Errorf("%s with ID %s not found in Redis", typ, hostId) - } - return "", fmt.Errorf("failed to fetch %s with ID %s from Redis: %w", typ, id, err) + return "", fmt.Errorf("redis hget %q, %q failed: %w", key, id, err) + } + + return data, nil + } + + getNameFromRedis := func(typ, id string) (string, error) { + data, err := hgetFromRedis("icinga:"+typ, id) + if err != nil { + return "", err } var result struct { Name string `json:"name"` } - if err := json.Unmarshal([]byte(data), &result); err != nil { return "", fmt.Errorf("failed to unmarshal redis result: %w", err) } @@ -57,26 +82,99 @@ func (client *Client) fetchHostServiceName(ctx context.Context, hostId, serviceI return result.Name, nil } + getCustomVarFromRedis := func(id string) (*redisCustomVar, error) { + data, err := hgetFromRedis("icinga:customvar", id) + if err != nil { + return nil, err + } + + customvar := new(redisCustomVar) + if err := json.Unmarshal([]byte(data), customvar); err != nil { + return nil, fmt.Errorf("failed to unmarshal redis result: %w", err) + } + + return customvar, nil + } + + getObjectCustomVarsFromRedis := func(typ, id string) ([]*redisCustomVar, error) { + var resMap map[string]string + err := retry.WithBackoff( + ctx, + func(ctx context.Context) (err error) { + res := client.redisClient.HGetAll(ctx, "icinga:"+typ+":customvar") + if err = res.Err(); err != nil { + return + } + + resMap, err = res.Result() + return + }, + retry.Retryable, + backoff.DefaultBackoff, + retry.Settings{}, + ) + if err != nil { + return nil, fmt.Errorf("failed to HGETALL icinga:%s:customvar from Redis: %w", typ, err) + } + + var result struct { + CustomvarId string `json:"customvar_id"` + HostId string `json:"host_id"` + ServiceId string `json:"service_id"` + } + + var customvars []*redisCustomVar + for _, res := range resMap { + if err := json.Unmarshal([]byte(res), &result); err != nil { + return nil, fmt.Errorf("failed to unmarshal redis result: %w", err) + } + + switch typ { + case "host": + if result.HostId != id { + continue + } + case "service": + if result.ServiceId != id { + continue + } + default: + panic(fmt.Sprintf("unexpected object type %q", typ)) + } + + customvar, err := getCustomVarFromRedis(result.CustomvarId) + if err != nil { + return nil, fmt.Errorf("failed to fetch customvar: %w", err) + } + customvars = append(customvars, customvar) + } + + return customvars, nil + } + var result redisLookupResult var err error - result.HostName, err = getNameFromRedis("host", hostId.String()) + result.hostName, err = getNameFromRedis("host", hostId.String()) if err != nil { return nil, err } if serviceId != nil { - result.ServiceName, err = getNameFromRedis("service", serviceId.String()) + result.serviceName, err = getNameFromRedis("service", serviceId.String()) if err != nil { return nil, err } } - return &result, nil -} + if serviceId == nil { + result.customVars, err = getObjectCustomVarsFromRedis("host", hostId.String()) + } else { + result.customVars, err = getObjectCustomVarsFromRedis("service", serviceId.String()) + } + if err != nil { + return nil, err + } -// redisLookupResult defines the structure of the Redis message we're interested in. -type redisLookupResult struct { - HostName string `json:"-"` // Name of the host (never empty). - ServiceName string `json:"-"` // Name of the service (only set in service context). + return &result, nil } From 4be195087edc16a06bfb0480711716310015ca83 Mon Sep 17 00:00:00 2001 From: Alvar Penning Date: Mon, 27 Oct 2025 14:36:42 +0100 Subject: [PATCH 18/38] notifications: Evaluate Icinga DB Web Rule Filter The rules are no longer just plain SQL queries, but have now their own JSON format, introduced by Icinga DB Web. This format is now supported by Client.evaluateRulesForObject. - https://github.com/Icinga/icingadb-web/pull/1289 - https://github.com/Icinga/icingadb/pull/998#issuecomment-3442298348 --- pkg/notifications/notifications.go | 124 +++++++++++++++++++++++------ 1 file changed, 99 insertions(+), 25 deletions(-) diff --git a/pkg/notifications/notifications.go b/pkg/notifications/notifications.go index 630eaa901..8743ed365 100644 --- a/pkg/notifications/notifications.go +++ b/pkg/notifications/notifications.go @@ -2,8 +2,9 @@ package notifications import ( "context" + "encoding/json" "fmt" - "net/url" + "slices" "sync" "github.com/icinga/icinga-go-library/database" @@ -70,22 +71,41 @@ func NewNotificationsClient( }, nil } -// evaluateRulesForObject returns the rule IDs for each matching query. +// evaluateRulesForObject checks each rule against the Icinga DB SQL database and returns matching rule IDs. // -// At the moment, each rule filter expression is executed as a SQL query after the parameters are being bound. If the -// query returns at least one line, the rule will match. Rules with an empty filter expression are a special case and -// will always match. +// Within the Icinga Notifications relation database, the rules are stored in rule.object_filter as a JSON object +// created by Icinga DB Web. This object contains SQL queries with bindvars for the Icinga DB relational database, to be +// executed with the given host, service and environment IDs. If this query returns at least one row, the rule is +// considered as matching. // -// The provided entity is passed as param to the queries, thus they are allowed to use all fields of that specific -// entity. Cross-table column references are not supported unless the provided entity provides the fields in one way -// or another. -// -// This allows a query like the following: -// -// > select * from host where id = :host_id and environment_id = :environment_id and name like 'prefix_%' -// -// The :host_id and :environment_id parameters will be bound to the entity's ID and EnvironmentId fields, respectively. -func (client *Client) evaluateRulesForObject(ctx context.Context, entity database.Entity) ([]string, error) { +// Icinga DB Web's JSON structure is described in: +// - https://github.com/Icinga/icingadb-web/pull/1289 +// - https://github.com/Icinga/icingadb/pull/998#issuecomment-3442298348 +func (client *Client) evaluateRulesForObject(ctx context.Context, hostId, serviceId, environmentId types.Binary) ([]string, error) { + const ( + icingaDbWebRuleVersion = 1 + icingaDbWebRuleTypeAll = "all" + icingaDbWebRuleTypeHost = "host" + icingaDbWebRuleTypeService = "service" + ) + + type IcingaDbWebQuery struct { + Query string `json:"query"` + Parameters []string `json:"parameters"` + } + + type IcingaDbWebRule struct { + Version int `json:"version"` // expect icingaDbWebRuleVersion + Config struct { + Type string `json:"type"` // expect one of [all, host, service] + Filter string `json:"filter"` // Icinga DB Web filter expression + } `json:"config"` + Queries struct { + Host *IcingaDbWebQuery `json:"host"` + Service *IcingaDbWebQuery `json:"service,omitempty"` + } `json:"queries"` + } + outRuleIds := make([]string, 0, len(client.rulesInfo.Rules)) for id, filterExpr := range client.rulesInfo.Rules { @@ -94,14 +114,57 @@ func (client *Client) evaluateRulesForObject(ctx context.Context, entity databas continue } - evaluates, err := func() (bool, error) { - // The raw SQL query in the database is URL-encoded (mostly the space character is replaced by %20). - // So, we need to unescape it before passing it to the database. - query, err := url.QueryUnescape(filterExpr) - if err != nil { - return false, errors.Wrapf(err, "cannot unescape rule %q object filter expression %q", id, filterExpr) + var webRule IcingaDbWebRule + if err := json.Unmarshal([]byte(filterExpr), &webRule); err != nil { + return nil, errors.Wrap(err, "cannot decode rule filter expression as JSON into struct") + } + + if version := webRule.Version; version != icingaDbWebRuleVersion { + return nil, errors.Errorf("decoded rule filter expression .Version is %d, %d expected", version, icingaDbWebRuleVersion) + } + if cfgType := webRule.Config.Type; !slices.Contains( + []string{icingaDbWebRuleTypeAll, icingaDbWebRuleTypeHost, icingaDbWebRuleTypeService}, cfgType) { + return nil, errors.Errorf("decoded rule filter expression contains unsupported .Config.Type %q", cfgType) + } + if cfgType := webRule.Config.Type; cfgType != icingaDbWebRuleTypeService && webRule.Queries.Host == nil { + return nil, errors.Errorf("decoded rule filter expression for .Config.Type %q with an empty .Queries.Host", cfgType) + } + if cfgType := webRule.Config.Type; cfgType != icingaDbWebRuleTypeHost && webRule.Queries.Service == nil { + return nil, errors.Errorf("decoded rule filter expression for .Config.Type %q with an empty .Queries.Service", cfgType) + } + + var webQuery IcingaDbWebQuery + if !serviceId.Valid() { + if webRule.Config.Type == icingaDbWebRuleTypeService { + continue + } + webQuery = *webRule.Queries.Host + } else { + if webRule.Config.Type == icingaDbWebRuleTypeHost { + continue + } + webQuery = *webRule.Queries.Service + } + + queryArgs := make([]any, 0, len(webQuery.Parameters)) + for _, param := range webQuery.Parameters { + switch param { + case ":host_id": + queryArgs = append(queryArgs, hostId.String()) + case ":service_id": + if !serviceId.Valid() { + return nil, errors.New("host rule filter expression contains :service_id for replacement") + } + queryArgs = append(queryArgs, serviceId.String()) + case ":environment_id": + queryArgs = append(queryArgs, environmentId.String()) + default: + queryArgs = append(queryArgs, param) } - rows, err := client.db.NamedQueryContext(ctx, client.db.Rebind(query), entity) + } + + evaluates, err := func() (bool, error) { + rows, err := client.db.QueryContext(ctx, client.db.Rebind(webQuery.Query), queryArgs...) if err != nil { return false, err } @@ -322,25 +385,32 @@ func (client *Client) Submit(entity database.Entity) bool { return true } - var ev *event.Event - var eventErr error + var ( + ev *event.Event + eventErr error + metaHistory v1history.HistoryTableMeta + ) // Keep the type switch in sync with the values of SyncKeyStructPtrs below. switch h := entity.(type) { case *v1history.AcknowledgementHistory: ev, eventErr = client.buildAcknowledgementHistoryEvent(client.ctx, h) + metaHistory = h.HistoryTableMeta case *v1history.DowntimeHistoryMeta: ev, eventErr = client.buildDowntimeHistoryMetaEvent(client.ctx, h) + metaHistory = h.HistoryTableMeta case *v1history.FlappingHistory: ev, eventErr = client.buildFlappingHistoryEvent(client.ctx, h) + metaHistory = h.HistoryTableMeta case *v1history.StateHistory: if h.StateType != common.HardState { return true } ev, eventErr = client.buildStateHistoryEvent(client.ctx, h) + metaHistory = h.HistoryTableMeta default: client.logger.Error("Cannot process unsupported type", zap.String("type", fmt.Sprintf("%T", h))) @@ -376,7 +446,11 @@ func (client *Client) Submit(entity database.Entity) bool { // second try would be the resubmit, and the third try would be for bad luck, e.g., when a second rule update just // crept in between. If there are three subsequent rule updates, something is wrong. for try := 0; try < 3; try++ { - eventRuleIds, err := client.evaluateRulesForObject(client.ctx, entity) + eventRuleIds, err := client.evaluateRulesForObject( + client.ctx, + metaHistory.HostId, + metaHistory.ServiceId, + metaHistory.EnvironmentId) if err != nil { // While returning false would be more correct, this would result in never being able to refetch new rule // versions. Consider an invalid object filter expression, which is now impossible to get rid of. From 17addb97920071d88cd6160402e5fa33430a5b2d Mon Sep 17 00:00:00 2001 From: Alvar Penning Date: Fri, 31 Oct 2025 10:25:39 +0100 Subject: [PATCH 19/38] history: StreamSorter for Notifications Callback The StreamSorter was added to history, allowing to collect messages from multiple Redis streams, sorting them based on the timestamp in the Stream ID, and ejecting them back. This is used for the callback stage, required by Icinga Notification. In the Notification context, an ordered stream is required. Despite my best intention, it felt like I have created an Erlang. --- pkg/icingadb/history/sorter.go | 321 ++++++++++++++++++++++++++++ pkg/icingadb/history/sorter_test.go | 317 +++++++++++++++++++++++++++ pkg/icingadb/history/sync.go | 152 +++++-------- 3 files changed, 691 insertions(+), 99 deletions(-) create mode 100644 pkg/icingadb/history/sorter.go create mode 100644 pkg/icingadb/history/sorter_test.go diff --git a/pkg/icingadb/history/sorter.go b/pkg/icingadb/history/sorter.go new file mode 100644 index 000000000..af55168ca --- /dev/null +++ b/pkg/icingadb/history/sorter.go @@ -0,0 +1,321 @@ +package history + +import ( + "context" + "github.com/icinga/icinga-go-library/logging" + "github.com/icinga/icinga-go-library/redis" + "github.com/pkg/errors" + "go.uber.org/zap" + "math" + "regexp" + "sort" + "strconv" + "time" +) + +// parseRedisStreamId parses a Redis Stream ID and returns the timestamp in ms and the sequence number, or an error. +func parseRedisStreamId(redisStreamId string) (int64, int64, error) { + re := regexp.MustCompile(`^(\d+)-(\d+)$`) + matches := re.FindStringSubmatch(redisStreamId) + if len(matches) != 3 { + return 0, 0, errors.Errorf("value %q does not satisfy Redis Stream ID regex", redisStreamId) + } + + ms, err := strconv.ParseInt(matches[1], 10, 64) + if err != nil { + return 0, 0, errors.Wrapf( + err, + "timestamp part of the Redis Stream ID %q cannot be parsed to int", redisStreamId) + } + + seq, err := strconv.ParseInt(matches[2], 10, 64) + if err != nil { + return 0, 0, errors.Wrapf( + err, + "sequence number of the Redis Stream ID %q cannot be parsed to int", redisStreamId) + } + + return ms, seq, nil +} + +// streamSorterSubmission is one submission to a StreamSorter, allowing to be sorted by the Redis Stream ID - both via +// timestamp and the sequence number as a fallback - as well as the submission timestamp for duplicates if milliseconds +// are not precise enough. +type streamSorterSubmission struct { + // msg is the Redis message to be forwarded to out after this submission was sorted. + msg redis.XMessage + args any + out chan<- redis.XMessage + + // Required for sorting. + streamIdMs int64 // streamIdMs is the Redis Stream ID timestamp part (milliseconds) + streamIdSeq int64 // streamIdSeq is the Redis Stream ID sequence number + submitTimeNs int64 // submitTimeNs is the timestamp when the element was submitted (in nanoseconds) +} + +// streamSorterSubmissions implements sort.Interface for []streamSorterSubmission. +type streamSorterSubmissions []streamSorterSubmission + +func (subs streamSorterSubmissions) Len() int { return len(subs) } + +func (subs streamSorterSubmissions) Swap(i, j int) { subs[i], subs[j] = subs[j], subs[i] } + +func (subs streamSorterSubmissions) Less(i, j int) bool { + a, b := subs[i], subs[j] + if a.streamIdMs != b.streamIdMs { + return a.streamIdMs < b.streamIdMs + } + if a.streamIdSeq != b.streamIdSeq { + return a.streamIdSeq < b.streamIdSeq + } + return a.submitTimeNs < b.submitTimeNs +} + +// StreamSorter accepts multiple [redis.XMessage] via Submit and ejects them in an ordered fashion. +// +// Internally, two goroutines are used. One collects the submissions and puts them into buckets based on the second +// of the Redis Stream ID. After three seconds, each bucket is being sorted and ejected to the other goroutine. There, +// each element is passed to the callback function in order. Only if the callback function has succeeded, it is removed +// from the top of the queue. +// +// Thus, if a message is received delayed for more than three seconds, it will be relayed out of order while an error is +// being logged. The StreamSorter is only able to ensure order to a certain degree of chaos. +// +// The callback function receives the [redis.XMessage] together with generic args passed in Submit for additional +// context. If the callback function returns true, the element will be removed from the queue. Otherwise, the element +// will be kept at top of the queue and retried next time. +type StreamSorter struct { + logger *logging.Logger + callbackFn func(redis.XMessage, any) bool + submissionCh chan streamSorterSubmission + + // verbose implies a verbose debug logging. Don't think one want to have this outside the tests. + verbose bool +} + +// NewStreamSorter creates a StreamSorter honoring the given context and returning elements to the callback function. +func NewStreamSorter( + ctx context.Context, + logger *logging.Logger, + callbackFn func(msg redis.XMessage, args any) bool, +) *StreamSorter { + sorter := &StreamSorter{ + logger: logger, + callbackFn: callbackFn, + submissionCh: make(chan streamSorterSubmission), + } + + _ = context.AfterFunc(ctx, func() { close(sorter.submissionCh) }) + + ch := make(chan []streamSorterSubmission) + go sorter.submissionWorker(ctx, ch) + go sorter.queueWorker(ctx, ch) + + return sorter +} + +// submissionWorker listens ton submissionCh populated by Submit, fills buckets and ejects them into out, linked to +// the queueWorker goroutine for further processing. +func (sorter *StreamSorter) submissionWorker(ctx context.Context, out chan<- []streamSorterSubmission) { + // slots defines how many second slots should be kept for sorting + const slots = 3 + // buckets maps timestamp in seconds to streamSorterSubmissions made within this second + buckets := make(map[int64][]streamSorterSubmission) + + defer close(out) + + ticker := time.NewTicker(time.Second) + defer ticker.Stop() + + for { + select { + case <-ctx.Done(): + return + + case submission := <-sorter.submissionCh: + curBucketId := time.Now().Unix() + bucketId := submission.streamIdMs / 1_000 + if minBucketId := curBucketId - slots; bucketId < minBucketId { + sorter.logger.Errorw("Received message with Stream ID from the far past, put in last bucket", + zap.String("id", submission.msg.ID), + zap.Int64("buckets-behind", minBucketId-bucketId)) + bucketId = minBucketId + } else if bucketId > curBucketId { + sorter.logger.Warnw("Received message with Stream ID from the future", + zap.String("id", submission.msg.ID), + zap.Int64("buckets-ahead", bucketId-curBucketId)) + } + + if sorter.verbose { + sorter.logger.Debugw("Insert submission into bucket", + zap.String("id", submission.msg.ID), + zap.Int64("bucket-id", bucketId)) + } + + bucket, ok := buckets[bucketId] + if !ok { + bucket = make([]streamSorterSubmission, 0, 1) + } + buckets[bucketId] = append(bucket, submission) + + case <-ticker.C: + // Search the smallest bucket ID older than slots+1 seconds by iterating over the keys. This is fast due to + // slots being 3 and the submission code eliminates inserts from the far past. Usually, the latest bucket ID + // should be "time.Now().Unix() - slots - 1", but I raced this with a very busy submission channel. + bucketId := int64(math.MaxInt64) + bucketSup := time.Now().Unix() - slots - 1 + for k := range buckets { + if k <= bucketSup { + bucketId = min(bucketId, k) + } + } + + bucket, ok := buckets[bucketId] + if !ok { + continue + } + delete(buckets, bucketId) + + sort.Sort(streamSorterSubmissions(bucket)) + out <- bucket + + if sorter.verbose { + sorter.logger.Debugw("Ejected submission bucket to callback worker", + zap.Int64("bucket-id", bucketId), + zap.Int("bucket-size", len(bucket))) + } + } + } +} + +// queueWorker receives sorted streamSorterSubmissions from submissionWorker and forwards them to the callback. +func (sorter *StreamSorter) queueWorker(ctx context.Context, in <-chan []streamSorterSubmission) { + // Each streamSorterSubmission received bucket-wise from in is stored in the queue slice. From there on, the slice + // head is passed to the callback function. The queueEventCh has a buffer capacity of 1 to allow both filling and + // consuming in the same goroutine. + queue := make([]streamSorterSubmission, 0, 1024) + queueEventCh := make(chan struct{}, 1) + defer close(queueEventCh) + + // queueEvent places something in queueEventCh w/o deadlocking + queueEvent := func() { + // Always drain channel first. Consider positive <-queueEventCh case followed by <-in. Within <-in, a second + // struct{}{} would be sent, effectively deadlocking. + for len(queueEventCh) > 0 { + <-queueEventCh + } + queueEventCh <- struct{}{} + } + + // The actual callback function is executed concurrently as it might block longer than expected. A blocking select + // would result in the queue not being populated, effectively blocking the submissionWorker. Thus, the callbackFn is + // started in a goroutine, signaling back its success status via callbackCh. If no callback is active, the channel + // is nil. Furthermore, an exponential backoff for sequentially failing callbacks is in place. + const callbackMaxDelay = 10 * time.Second + var callbackDelay time.Duration + var callbackCh chan bool + callbackFn := func(submission streamSorterSubmission) { + select { + case <-ctx.Done(): + return + case <-time.After(callbackDelay): + } + + start := time.Now() + success := sorter.callbackFn(submission.msg, submission.args) + if success { + submission.out <- submission.msg + callbackDelay = 0 + } else { + callbackDelay = min(2*max(time.Millisecond, callbackDelay), callbackMaxDelay) + } + + if sorter.verbose { + sorter.logger.Debugw("Callback finished", + zap.String("id", submission.msg.ID), + zap.Bool("success", success), + zap.Duration("duration", time.Since(start)), + zap.Duration("next-delay", callbackDelay)) + } + + callbackCh <- success + } + + for { + select { + case <-ctx.Done(): + return + + case submissions, ok := <-in: + if !ok { + return + } + + queue = append(queue, submissions...) + queueEvent() + + if sorter.verbose { + sorter.logger.Debugw("Queue worker received new submissions", + zap.Int("queue-size", len(queue))) + } + + case <-queueEventCh: + if len(queue) == 0 { + continue + } + + if callbackCh != nil { + continue + } + callbackCh = make(chan bool) + go callbackFn(queue[0]) + + case success := <-callbackCh: + if success { + queue = queue[1:] + } + + close(callbackCh) + callbackCh = nil + + if len(queue) > 0 { + queueEvent() + } else { + if sorter.verbose { + sorter.logger.Debug("Queue worker finished processing queue") + } + } + } + } +} + +// Submit a [redis.XMessage] to the StreamSorter. +// +// After the message was sorted and successfully passed to the callback including the optional args, it will be +// forwarded to the out channel. +// +// This method returns an error for malformed Redis Stream IDs or if the internal submission channel blocks for over a +// second. Usually, this both should not happen. +func (sorter *StreamSorter) Submit(msg redis.XMessage, args any, out chan<- redis.XMessage) error { + ms, seq, err := parseRedisStreamId(msg.ID) + if err != nil { + return errors.Wrap(err, "cannot parse Redis Stream ID") + } + + submission := streamSorterSubmission{ + msg: msg, + args: args, + out: out, + streamIdMs: ms, + streamIdSeq: seq, + submitTimeNs: time.Now().UnixNano(), + } + + select { + case sorter.submissionCh <- submission: + return nil + + case <-time.After(time.Second): + return errors.New("submission timed out") + } +} diff --git a/pkg/icingadb/history/sorter_test.go b/pkg/icingadb/history/sorter_test.go new file mode 100644 index 000000000..f5551e42d --- /dev/null +++ b/pkg/icingadb/history/sorter_test.go @@ -0,0 +1,317 @@ +// #nosec G404 -- Allow math/rand for the tests +package history + +import ( + "cmp" + "fmt" + "github.com/icinga/icinga-go-library/logging" + "github.com/icinga/icinga-go-library/redis" + "github.com/stretchr/testify/require" + "go.uber.org/zap/zaptest" + "math/rand" + "sort" + "strconv" + "strings" + "sync" + "testing" + "time" +) + +func Test_redisStreamIdToMs(t *testing.T) { + tests := []struct { + name string + input string + wantMs int64 + wantSeq int64 + wantErr bool + }{ + { + name: "epoch", + input: "0-0", + }, + { + name: "valid", + input: "1761658169701-0", + wantMs: 1761658169701, + }, + { + name: "valid sequence", + input: "1761658169701-23", + wantMs: 1761658169701, + wantSeq: 23, + }, + { + name: "invalid format", + input: "23-42-23", + wantErr: true, + }, + { + name: "invalid field types", + input: "0x23-0x42", + wantErr: true, + }, + { + name: "number too big", + input: "22222222222222222222222222222222222222222222222222222222222222222222222222222222222222222-0", + wantErr: true, + }, + } + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + gotMs, gotSeq, err := parseRedisStreamId(tt.input) + require.Equal(t, tt.wantErr, err != nil, "error differs %v", err) + require.Equal(t, tt.wantMs, gotMs, "ms from Redis Stream ID differs") + require.Equal(t, tt.wantSeq, gotSeq, "seq from Redis Stream ID differs") + }) + } +} + +func Test_streamSorterSubmissions(t *testing.T) { + submissions := []streamSorterSubmission{ + {streamIdMs: 0, streamIdSeq: 0, submitTimeNs: 0}, + {streamIdMs: 1, streamIdSeq: 0, submitTimeNs: 0}, + {streamIdMs: 1, streamIdSeq: 1, submitTimeNs: 0}, + {streamIdMs: 2, streamIdSeq: 0, submitTimeNs: 0}, + {streamIdMs: 2, streamIdSeq: 0, submitTimeNs: 1}, + {streamIdMs: 3, streamIdSeq: 0, submitTimeNs: 0}, + {streamIdMs: 3, streamIdSeq: 1, submitTimeNs: 0}, + {streamIdMs: 3, streamIdSeq: 1, submitTimeNs: 1}, + {streamIdMs: 3, streamIdSeq: 1, submitTimeNs: 2}, + } + + submissionsRand := make([]streamSorterSubmission, 0, len(submissions)) + for _, i := range rand.Perm(len(submissions)) { + submissionsRand = append(submissionsRand, submissions[i]) + } + + sort.Sort(streamSorterSubmissions(submissionsRand)) + require.Equal(t, submissions, submissionsRand) +} + +func TestStreamSorter(t *testing.T) { + tests := []struct { + name string + messages int + producers int + callbackMaxDelayMs int + callbackSuccessPercent int + expectTimeout bool + outMaxDelayMs int + }{ + { + name: "baseline", + messages: 10, + producers: 1, + callbackSuccessPercent: 100, + }, + { + name: "simple", + messages: 100, + producers: 10, + callbackSuccessPercent: 100, + }, + { + name: "many producers", + messages: 100, + producers: 100, + callbackSuccessPercent: 100, + }, + { + name: "many messages", + messages: 1000, + producers: 10, + callbackSuccessPercent: 100, + }, + { + name: "callback a bit unreliable", + messages: 50, + producers: 10, + callbackSuccessPercent: 70, + }, + { + name: "callback coin flip", + messages: 50, + producers: 10, + callbackSuccessPercent: 50, + }, + { + name: "callback unreliable", + messages: 25, + producers: 5, + callbackSuccessPercent: 30, + }, + { + name: "callback total rejection", + messages: 10, + producers: 1, + callbackSuccessPercent: 0, + expectTimeout: true, + }, + { + name: "callback slow", + messages: 100, + producers: 10, + callbackMaxDelayMs: 3000, + callbackSuccessPercent: 100, + }, + { + name: "out slow", + messages: 100, + producers: 10, + callbackSuccessPercent: 100, + outMaxDelayMs: 1000, + }, + { + name: "pure chaos", + messages: 50, + producers: 10, + callbackMaxDelayMs: 3000, + callbackSuccessPercent: 50, + outMaxDelayMs: 1000, + }, + } + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + t.Parallel() + + // Callback functions after reordering + var ( + callbackCollection []string + callbackCollectionMutex sync.Mutex + callbackFn = func(msg redis.XMessage, _ any) bool { + if tt.callbackMaxDelayMs > 0 { + time.Sleep(time.Duration(rand.Int63n(int64(tt.callbackMaxDelayMs))) * time.Millisecond) + } + + if rand.Int63n(100)+1 > int64(tt.callbackSuccessPercent) { + return false + } + + callbackCollectionMutex.Lock() + defer callbackCollectionMutex.Unlock() + callbackCollection = append(callbackCollection, msg.ID) + return true + } + ) + + // Out channel after reordering and callback + var ( + outCounterCh = make(chan struct{}) + outConsumer = func(out chan redis.XMessage) { + for { + if tt.outMaxDelayMs > 0 { + time.Sleep(time.Duration(rand.Int63n(int64(tt.outMaxDelayMs))) * time.Millisecond) + } + + _, ok := <-out + if !ok { + return + } + outCounterCh <- struct{}{} + } + } + ) + + // Decreasing counter for messages to be sent + var ( + inCounter = tt.messages + inCounterMutex sync.Mutex + ) + + sorter := NewStreamSorter( + t.Context(), + logging.NewLogger(zaptest.NewLogger(t).Sugar(), time.Second), + callbackFn) + sorter.verbose = true + + for range tt.producers { + out := make(chan redis.XMessage) + defer close(out) // no leakage, general cleanup after finishing test run + + go func() { + for { + time.Sleep(time.Duration(rand.Int63n(250)) * time.Millisecond) + + inCounterMutex.Lock() + isFin := inCounter <= 0 + if !isFin { + inCounter-- + } + inCounterMutex.Unlock() + + if isFin { + return + } + + ms := time.Now().UnixMilli() + rand.Int63n(2_000) - 1_000 + seq := rand.Int63n(100) + + // Add 10% time travelers + if rand.Int63n(10) == 9 { + distanceMs := int64(1_500) + if rand.Int63n(2) > 0 { + // Don't go back too far. Otherwise, elements would be out of order. Three seconds max. + ms -= distanceMs + } else { + ms += distanceMs + } + } + + msg := redis.XMessage{ID: fmt.Sprintf("%d-%d", ms, seq)} + require.NoError(t, sorter.Submit(msg, nil, out)) + } + }() + + go outConsumer(out) + } + + var outCounter int + breakFor: + for { + select { + case <-outCounterCh: + outCounter++ + if outCounter == tt.messages { + break breakFor + } + + case <-time.After(2 * time.Minute): + if tt.expectTimeout { + return + } + t.Fatalf("Collecting messages timed out after receiving %d out of %d messages", + outCounter, tt.messages) + } + } + if tt.expectTimeout { + t.Fatal("Timeout was expected") + } + + callbackCollectionMutex.Lock() + for i := 0; i < len(callbackCollection)-1; i++ { + parse := func(id string) (int64, int64) { + parts := strings.Split(id, "-") + ms, err1 := strconv.ParseInt(parts[0], 10, 64) + seq, err2 := strconv.ParseInt(parts[1], 10, 64) + require.NoError(t, cmp.Or(err1, err2)) + return ms, seq + } + + a, b := callbackCollection[i], callbackCollection[i+1] + aMs, aSeq := parse(a) + bMs, bSeq := parse(b) + + switch { + case aMs < bMs: + case aMs == bMs: + if aSeq > bSeq { + t.Errorf("collection in wrong order: %q before %q", a, b) + } + case aMs > bMs: + t.Errorf("collection in wrong order: %q before %q", a, b) + } + } + callbackCollectionMutex.Unlock() + }) + } +} diff --git a/pkg/icingadb/history/sync.go b/pkg/icingadb/history/sync.go index 6cf8a499a..7467742e9 100644 --- a/pkg/icingadb/history/sync.go +++ b/pkg/icingadb/history/sync.go @@ -21,7 +21,6 @@ import ( "reflect" "slices" "sync" - "time" ) // Sync specifies the source and destination of a history sync. @@ -61,6 +60,11 @@ func (s Sync) Sync( return fmt.Errorf("if callbackKeyStructPtr and callbackFn are set, a callbackName is required") } + var callbackStageFn stageFunc + if callbackKeyStructPtr != nil { + callbackStageFn = makeSortedCallbackStageFunc(ctx, s.logger, callbackName, callbackKeyStructPtr, callbackFn) + } + g, ctx := errgroup.WithContext(ctx) for key, pipeline := range syncPipelines { @@ -98,7 +102,7 @@ func (s Sync) Sync( // Shadowed variable to allow appending custom callbacks. pipeline := pipeline if hasCallbackStage { - pipeline = append(slices.Clip(pipeline), makeCallbackStageFunc(callbackName, callbackKeyStructPtr, callbackFn)) + pipeline = append(slices.Clip(pipeline), callbackStageFn) } ch := make([]chan redis.XMessage, len(pipeline)+1) @@ -422,39 +426,40 @@ func countElementStage(ctx context.Context, _ Sync, _ string, in <-chan redis.XM } } -// makeCallbackStageFunc creates a new stageFunc calling the given callback function for each message. +// makeSortedCallbackStageFunc creates a new stageFunc calling the callback function after reordering messages. +// +// This stageFunc is designed to be used by multiple channels. The internal sorting logic - realized by a StreamSorter - +// results in all messages to be sorted based on their Redis Stream ID and be ejected to the callback function in this +// order. // // The keyStructPtrs map decides what kind of database.Entity type will be used for the input data based on the key. // // The callback call is blocking and the message will be forwarded to the out channel after the function has returned. // Thus, please ensure this function does not block too long. // -// If the callback function returns false, the stageFunc switches to a backlog mode, retrying the failed messages and -// every subsequent message until there are no messages left. Only after a message was successfully handled by the -// callback method, it will be forwarded to the out channel. Thus, this stage might "block" or "hold back" certain -// messages during unhappy callback times. +// If the callback function returns false, the message will be retried after an increasing backoff. All subsequent +// messages will wait until this one succeeds. // // For each successfully submitted message, the telemetry stat named after this callback is incremented. Thus, a delta // between [telemetry.StatHistory] and this stat indicates blocking callbacks. -func makeCallbackStageFunc( +func makeSortedCallbackStageFunc( + ctx context.Context, + logger *logging.Logger, name string, keyStructPtrs map[string]any, fn func(database.Entity) bool, ) stageFunc { - return func(ctx context.Context, s Sync, key string, in <-chan redis.XMessage, out chan<- redis.XMessage) error { - defer close(out) - - structPtr, ok := keyStructPtrs[key] - if !ok { - return fmt.Errorf("can't lookup struct pointer for key %q", key) - } - - structifier := structify.MakeMapStructifier( - reflect.TypeOf(structPtr).Elem(), - "json", - contracts.SafeInit) + sorterCallbackFn := func(msg redis.XMessage, args any) bool { + makeEntity := func(key string, values map[string]interface{}) (database.Entity, error) { + structPtr, ok := keyStructPtrs[key] + if !ok { + return nil, fmt.Errorf("key is not part of keyStructPtrs") + } - makeEntity := func(values map[string]interface{}) (database.Entity, error) { + structifier := structify.MakeMapStructifier( + reflect.TypeOf(structPtr).Elem(), + "json", + contracts.SafeInit) val, err := structifier(values) if err != nil { return nil, errors.Wrapf(err, "can't structify values %#v for %q", values, key) @@ -468,13 +473,32 @@ func makeCallbackStageFunc( return entity, nil } - backlogLastId := "" - backlogMsgCounter := 0 + key, ok := args.(string) + if !ok { + // Shall not happen; set to string some thirty lines below + panic(fmt.Sprintf("args is of type %T, not string", args)) + } + + entity, err := makeEntity(key, msg.Values) + if err != nil { + logger.Errorw("Failed to create database.Entity out of Redis stream message", + zap.Error(err), + zap.String("key", key), + zap.String("id", msg.ID)) + return false + } - const backlogTimerMinInterval, backlogTimerMaxInterval = time.Millisecond, time.Minute - backlogTimerInterval := backlogTimerMinInterval - backlogTimer := time.NewTimer(backlogTimerInterval) - _ = backlogTimer.Stop() + success := fn(entity) + if success { + telemetry.Stats.Get(name).Add(1) + } + return success + } + + sorter := NewStreamSorter(ctx, logger, sorterCallbackFn) + + return func(ctx context.Context, s Sync, key string, in <-chan redis.XMessage, out chan<- redis.XMessage) error { + defer close(out) for { select { @@ -483,79 +507,9 @@ func makeCallbackStageFunc( return nil } - // Only submit the entity directly if there is no backlog. - // The second check covers a potential corner case if the XRANGE below races this stream. - if backlogLastId != "" && backlogLastId != msg.ID { - continue - } - - entity, err := makeEntity(msg.Values) - if err != nil { - return err - } - - if fn(entity) { - out <- msg - telemetry.Stats.Get(name).Add(1) - backlogLastId = "" - } else { - backlogLastId = msg.ID - backlogMsgCounter = 0 - backlogTimerInterval = backlogTimerMinInterval - _ = backlogTimer.Reset(backlogTimerInterval) - s.logger.Warnw("Failed to submit entity to callback, entering into backlog", - zap.String("key", key), - zap.String("id", backlogLastId)) - } - - case <-backlogTimer.C: - if backlogLastId == "" { // Should never happen. - return fmt.Errorf("backlog timer logic for %q was called while backlogLastId was empty", key) - } - - logger := s.logger.With( - zap.String("key", key), - zap.String("last-id", backlogLastId)) - - logger.Debug("Trying to advance backlog of callback elements") - - xrangeCmd := s.redis.XRangeN(ctx, "icinga:history:stream:"+key, backlogLastId, "+", 2) - msgs, err := xrangeCmd.Result() - if err != nil { - return errors.Wrapf(err, "XRANGE %q to %q on stream %q failed", backlogLastId, "+", key) - } - - if len(msgs) < 1 || len(msgs) > 2 { - return fmt.Errorf("XRANGE %q to %q on stream %q returned %d messages, not 1 or 2", - backlogLastId, "+", key, len(msgs)) - } - - msg := msgs[0] - entity, err := makeEntity(msg.Values) + err := sorter.Submit(msg, key, out) if err != nil { - return errors.Wrapf(err, "can't structify backlog value %q for %q", backlogLastId, key) - } - - if fn(entity) { - out <- msg - backlogMsgCounter++ - telemetry.Stats.Get(name).Add(1) - - if len(msgs) == 1 { - backlogLastId = "" - logger.Infow("Finished rolling back backlog of callback elements", zap.Int("elements", backlogMsgCounter)) - } else { - backlogLastId = msgs[1].ID - backlogTimerInterval = backlogTimerMinInterval - _ = backlogTimer.Reset(backlogTimerInterval) - logger.Debugw("Advanced backlog", - zap.String("new-last-id", backlogLastId), - zap.Duration("delay", backlogTimerInterval)) - } - } else { - backlogTimerInterval = min(backlogTimerMaxInterval, backlogTimerInterval*2) - _ = backlogTimer.Reset(backlogTimerInterval) - logger.Warnw("Failed to roll back callback elements", zap.Duration("delay", backlogTimerInterval)) + s.logger.Errorw("Failed to submit Redis stream event to stream sorter", zap.Error(err)) } case <-ctx.Done(): From aba4b2ecd16fc3c690fb43de2ff0ace8910e1fb5 Mon Sep 17 00:00:00 2001 From: Alvar Penning Date: Fri, 31 Oct 2025 11:54:36 +0100 Subject: [PATCH 20/38] notifications: Custom Vars From SQL, Output Format Rework the prior custom variable fetching code to no longer fetch everything in a looping fashion from Redis, but send SQL queries for custom variables now. In addition, for service objects now contain both the service and host custom variables, prefixed by "host.vars." or "service.vars.". --- pkg/notifications/fetch.go | 159 +++++++++++++++++++++++++ pkg/notifications/notifications.go | 33 +++--- pkg/notifications/redis_fetch.go | 180 ----------------------------- 3 files changed, 175 insertions(+), 197 deletions(-) create mode 100644 pkg/notifications/fetch.go delete mode 100644 pkg/notifications/redis_fetch.go diff --git a/pkg/notifications/fetch.go b/pkg/notifications/fetch.go new file mode 100644 index 000000000..0c2b81c9b --- /dev/null +++ b/pkg/notifications/fetch.go @@ -0,0 +1,159 @@ +package notifications + +import ( + "context" + "encoding/json" + "github.com/icinga/icinga-go-library/backoff" + "github.com/icinga/icinga-go-library/retry" + "github.com/icinga/icinga-go-library/types" + "github.com/pkg/errors" + "golang.org/x/sync/errgroup" + "time" +) + +// fetchHostServiceFromRedis retrieves the host and service names from Redis. +// +// If serviceId is nil, only the host name is fetched. Otherwise, both host and service name is fetched. +func (client *Client) fetchHostServiceFromRedis( + ctx context.Context, + hostId, serviceId types.Binary, +) (hostName string, serviceName string, err error) { + getNameFromRedis := func(ctx context.Context, typ, id string) (string, error) { + key := "icinga:" + typ + + var data string + if err := retry.WithBackoff( + ctx, + func(ctx context.Context) (err error) { + data, err = client.redisClient.HGet(ctx, key, id).Result() + return + }, + retry.Retryable, + backoff.DefaultBackoff, + retry.Settings{}, + ); err != nil { + return "", errors.Wrapf(err, "redis HGET %q, %q failed", key, id) + } + + var result struct { + Name string `json:"name"` + } + if err := json.Unmarshal([]byte(data), &result); err != nil { + return "", errors.Wrap(err, "failed to unmarshal redis result") + } + + return result.Name, nil + } + + hostName, err = getNameFromRedis(ctx, "host", hostId.String()) + if err != nil { + return + } + + if serviceId != nil { + serviceName, err = getNameFromRedis(ctx, "service", serviceId.String()) + if err != nil { + return + } + } + + return +} + +// fetchCustomVarFromSql retrieves custom variables for the hsot and service from SQL. +// +// If serviceId is nil, only the host custom vars are fetched. Otherwise, both host and service custom vars are fetched. +func (client *Client) fetchCustomVarFromSql( + ctx context.Context, + hostId, serviceId types.Binary, +) (map[string]string, error) { + type customVar struct { + Name string `db:"name"` + Value string `db:"value"` + } + + getCustomVarsFromSql := func(ctx context.Context, typ string, id types.Binary) ([]customVar, error) { + stmt, err := client.db.Preparex(client.db.Rebind( + `SELECT customvar.name AS name, customvar.value AS value + FROM ` + typ + `_customvar + LEFT JOIN customvar + ON ` + typ + `_customvar.customvar_id = customvar.id + WHERE ` + typ + `_customvar.` + typ + `_id = ?`)) + if err != nil { + return nil, err + } + + var customVars []customVar + if err := stmt.SelectContext(ctx, &customVars, id); err != nil { + return nil, err + } + + return customVars, nil + } + + customVars := make(map[string]string) + + hostVars, err := getCustomVarsFromSql(ctx, "host", hostId) + if err != nil { + return nil, err + } + + for _, hostVar := range hostVars { + customVars["host.vars."+hostVar.Name] = hostVar.Value + } + + if serviceId != nil { + serviceVars, err := getCustomVarsFromSql(ctx, "service", serviceId) + if err != nil { + return nil, err + } + + for _, serviceVar := range serviceVars { + customVars["service.vars."+serviceVar.Name] = serviceVar.Value + } + } + + return customVars, nil +} + +// hostServiceInformation contains the host name, an optional service name, and all custom variables. +// +// Returned from Client.fetchHostServiceData. +type hostServiceInformation struct { + hostName string + serviceName string + customVars map[string]string +} + +// fetchHostServiceData resolves the object names and fetches the associated custom variables. +// +// If serviceId is not nil, both host and service data will be queried. Otherwise, only host information is fetched. To +// acquire the information, the fetchHostServiceFromRedis and fetchCustomVarFromSql methods are used concurrently with +// a timeout of three seconds. +func (client *Client) fetchHostServiceData( + ctx context.Context, + hostId, serviceId types.Binary, +) (*hostServiceInformation, error) { + ctx, cancel := context.WithTimeout(ctx, 3*time.Second) + defer cancel() + + ret := &hostServiceInformation{} + g, ctx := errgroup.WithContext(ctx) + + g.Go(func() error { + var err error + ret.hostName, ret.serviceName, err = client.fetchHostServiceFromRedis(ctx, hostId, serviceId) + return err + }) + g.Go(func() error { + var err error + ret.customVars, err = client.fetchCustomVarFromSql(ctx, hostId, serviceId) + return err + }) + + if err := g.Wait(); err != nil { + return nil, err + } + + return ret, nil +} diff --git a/pkg/notifications/notifications.go b/pkg/notifications/notifications.go index 8743ed365..c4ab1101f 100644 --- a/pkg/notifications/notifications.go +++ b/pkg/notifications/notifications.go @@ -4,9 +4,6 @@ import ( "context" "encoding/json" "fmt" - "slices" - "sync" - "github.com/icinga/icinga-go-library/database" "github.com/icinga/icinga-go-library/logging" "github.com/icinga/icinga-go-library/notifications/event" @@ -21,6 +18,8 @@ import ( "github.com/pkg/errors" "go.uber.org/zap" "go.uber.org/zap/zapcore" + "slices" + "sync" ) // Client is an Icinga Notifications compatible client implementation to push events to Icinga Notifications. @@ -194,8 +193,8 @@ func (client *Client) evaluateRulesForObject(ctx context.Context, hostId, servic func (client *Client) buildCommonEvent( ctx context.Context, hostId, serviceId types.Binary, -) (*event.Event, *redisLookupResult, error) { - rlr, err := client.fetchHostServiceFromRedis(ctx, hostId, serviceId) +) (*event.Event, *hostServiceInformation, error) { + info, err := client.fetchHostServiceData(ctx, hostId, serviceId) if err != nil { return nil, nil, err } @@ -206,18 +205,18 @@ func (client *Client) buildCommonEvent( objectTags map[string]string ) - if rlr.serviceName != "" { - objectName = rlr.hostName + "!" + rlr.serviceName - objectUrl = "/icingadb/service?name=" + utils.RawUrlEncode(rlr.serviceName) + "&host.name=" + utils.RawUrlEncode(rlr.hostName) + if info.serviceName != "" { + objectName = info.hostName + "!" + info.serviceName + objectUrl = "/icingadb/service?name=" + utils.RawUrlEncode(info.serviceName) + "&host.name=" + utils.RawUrlEncode(info.hostName) objectTags = map[string]string{ - "host": rlr.hostName, - "service": rlr.serviceName, + "host": info.hostName, + "service": info.serviceName, } } else { - objectName = rlr.hostName - objectUrl = "/icingadb/host?name=" + utils.RawUrlEncode(rlr.hostName) + objectName = info.hostName + objectUrl = "/icingadb/host?name=" + utils.RawUrlEncode(info.hostName) objectTags = map[string]string{ - "host": rlr.hostName, + "host": info.hostName, } } @@ -225,8 +224,8 @@ func (client *Client) buildCommonEvent( Name: objectName, URL: objectUrl, Tags: objectTags, - ExtraTags: rlr.CustomVars(), - }, rlr, nil + ExtraTags: info.customVars, + }, info, nil } // buildStateHistoryEvent builds a fully initialized event.Event from a state history entry. @@ -234,14 +233,14 @@ func (client *Client) buildCommonEvent( // The resulted event will have all the necessary information for a state change event, and must // not be further modified by the caller. func (client *Client) buildStateHistoryEvent(ctx context.Context, h *v1history.StateHistory) (*event.Event, error) { - ev, rlr, err := client.buildCommonEvent(ctx, h.HostId, h.ServiceId) + ev, info, err := client.buildCommonEvent(ctx, h.HostId, h.ServiceId) if err != nil { return nil, errors.Wrapf(err, "cannot build event for %q,%q", h.HostId, h.ServiceId) } ev.Type = event.TypeState - if rlr.serviceName != "" { + if info.serviceName != "" { switch h.HardState { case 0: ev.Severity = event.SeverityOK diff --git a/pkg/notifications/redis_fetch.go b/pkg/notifications/redis_fetch.go deleted file mode 100644 index af9666dd2..000000000 --- a/pkg/notifications/redis_fetch.go +++ /dev/null @@ -1,180 +0,0 @@ -package notifications - -import ( - "context" - "encoding/json" - "fmt" - "time" - - "github.com/icinga/icinga-go-library/backoff" - "github.com/icinga/icinga-go-library/retry" - "github.com/icinga/icinga-go-library/types" -) - -// redisCustomVar is a customvar entry from Redis. -type redisCustomVar struct { - EnvironmentID types.Binary `json:"environment_id"` - Name string `json:"name"` - Value string `json:"value"` -} - -// redisLookupResult defines the structure of the Redis message we're interested in. -type redisLookupResult struct { - hostName string - serviceName string - customVars []*redisCustomVar -} - -// CustomVars returns a mapping of customvar names to values. -func (result redisLookupResult) CustomVars() map[string]string { - m := make(map[string]string) - for _, customvar := range result.customVars { - m[customvar.Name] = customvar.Value - } - - return m -} - -// fetchHostServiceFromRedis retrieves the host and service names and customvars from Redis. -// -// It uses either the hostId or/and serviceId to fetch the corresponding names. If both are provided, -// the returned result will contain the host name and the service name accordingly. Otherwise, it will -// only contain the host name. -// -// The function has a hard coded timeout of five seconds for all HGET and HGETALL commands together. -func (client *Client) fetchHostServiceFromRedis(ctx context.Context, hostId, serviceId types.Binary) (*redisLookupResult, error) { - var cancel context.CancelFunc - ctx, cancel = context.WithTimeout(ctx, 5*time.Second) - defer cancel() - - hgetFromRedis := func(key, id string) (string, error) { - var data string - err := retry.WithBackoff( - ctx, - func(ctx context.Context) (err error) { - data, err = client.redisClient.HGet(ctx, key, id).Result() - return - }, - retry.Retryable, - backoff.DefaultBackoff, - retry.Settings{}, - ) - if err != nil { - return "", fmt.Errorf("redis hget %q, %q failed: %w", key, id, err) - } - - return data, nil - } - - getNameFromRedis := func(typ, id string) (string, error) { - data, err := hgetFromRedis("icinga:"+typ, id) - if err != nil { - return "", err - } - - var result struct { - Name string `json:"name"` - } - if err := json.Unmarshal([]byte(data), &result); err != nil { - return "", fmt.Errorf("failed to unmarshal redis result: %w", err) - } - - return result.Name, nil - } - - getCustomVarFromRedis := func(id string) (*redisCustomVar, error) { - data, err := hgetFromRedis("icinga:customvar", id) - if err != nil { - return nil, err - } - - customvar := new(redisCustomVar) - if err := json.Unmarshal([]byte(data), customvar); err != nil { - return nil, fmt.Errorf("failed to unmarshal redis result: %w", err) - } - - return customvar, nil - } - - getObjectCustomVarsFromRedis := func(typ, id string) ([]*redisCustomVar, error) { - var resMap map[string]string - err := retry.WithBackoff( - ctx, - func(ctx context.Context) (err error) { - res := client.redisClient.HGetAll(ctx, "icinga:"+typ+":customvar") - if err = res.Err(); err != nil { - return - } - - resMap, err = res.Result() - return - }, - retry.Retryable, - backoff.DefaultBackoff, - retry.Settings{}, - ) - if err != nil { - return nil, fmt.Errorf("failed to HGETALL icinga:%s:customvar from Redis: %w", typ, err) - } - - var result struct { - CustomvarId string `json:"customvar_id"` - HostId string `json:"host_id"` - ServiceId string `json:"service_id"` - } - - var customvars []*redisCustomVar - for _, res := range resMap { - if err := json.Unmarshal([]byte(res), &result); err != nil { - return nil, fmt.Errorf("failed to unmarshal redis result: %w", err) - } - - switch typ { - case "host": - if result.HostId != id { - continue - } - case "service": - if result.ServiceId != id { - continue - } - default: - panic(fmt.Sprintf("unexpected object type %q", typ)) - } - - customvar, err := getCustomVarFromRedis(result.CustomvarId) - if err != nil { - return nil, fmt.Errorf("failed to fetch customvar: %w", err) - } - customvars = append(customvars, customvar) - } - - return customvars, nil - } - - var result redisLookupResult - var err error - - result.hostName, err = getNameFromRedis("host", hostId.String()) - if err != nil { - return nil, err - } - - if serviceId != nil { - result.serviceName, err = getNameFromRedis("service", serviceId.String()) - if err != nil { - return nil, err - } - } - - if serviceId == nil { - result.customVars, err = getObjectCustomVarsFromRedis("host", hostId.String()) - } else { - result.customVars, err = getObjectCustomVarsFromRedis("service", serviceId.String()) - } - if err != nil { - return nil, err - } - - return &result, nil -} From b901f67aa7dc95f526e075df1e6e88afdcfdda6c Mon Sep 17 00:00:00 2001 From: Alvar Penning Date: Fri, 31 Oct 2025 13:35:27 +0100 Subject: [PATCH 21/38] go.mod: Bump Icinga Go Library --- go.mod | 4 ++-- go.sum | 8 ++++---- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/go.mod b/go.mod index 81b24f41d..b3675c116 100644 --- a/go.mod +++ b/go.mod @@ -7,7 +7,7 @@ require ( github.com/goccy/go-yaml v1.13.0 github.com/google/go-cmp v0.7.0 github.com/google/uuid v1.6.0 - github.com/icinga/icinga-go-library v0.7.3-0.20251022120618-6600889adc38 + github.com/icinga/icinga-go-library v0.7.3-0.20251029100725-d59f989509ea github.com/jessevdk/go-flags v1.6.1 github.com/jmoiron/sqlx v1.4.0 github.com/mattn/go-sqlite3 v1.14.32 @@ -34,7 +34,7 @@ require ( github.com/mattn/go-isatty v0.0.20 // indirect github.com/mattn/go-runewidth v0.0.12 // indirect github.com/pmezard/go-difflib v1.0.0 // indirect - github.com/redis/go-redis/v9 v9.14.1 // indirect + github.com/redis/go-redis/v9 v9.16.0 // indirect github.com/rivo/uniseg v0.2.0 // indirect github.com/ssgreg/journald v1.0.0 // indirect go.uber.org/multierr v1.11.0 // indirect diff --git a/go.sum b/go.sum index d544422f6..190bc7471 100644 --- a/go.sum +++ b/go.sum @@ -37,8 +37,8 @@ github.com/google/go-cmp v0.7.0 h1:wk8382ETsv4JYUZwIsn6YpYiWiBsYLSJiTsyBybVuN8= github.com/google/go-cmp v0.7.0/go.mod h1:pXiqmnSA92OHEEa9HXL2W4E7lf9JzCmGVUdgjX3N/iU= github.com/google/uuid v1.6.0 h1:NIvaJDMOsjHA8n1jAhLSgzrAzy1Hgr+hNrb57e+94F0= github.com/google/uuid v1.6.0/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= -github.com/icinga/icinga-go-library v0.7.3-0.20251022120618-6600889adc38 h1:5RNrPZCwvqm2/06i9dUCJtcBV+tR8WgUKtHne2sOaA8= -github.com/icinga/icinga-go-library v0.7.3-0.20251022120618-6600889adc38/go.mod h1:L80M/ufoqFJJjZcdnfsTp6eFl06vm3JuvSWlGcDf708= +github.com/icinga/icinga-go-library v0.7.3-0.20251029100725-d59f989509ea h1:aCp3iiDJnfK0E1deHUhR2aRDgeLENoYArliP7jgo5HY= +github.com/icinga/icinga-go-library v0.7.3-0.20251029100725-d59f989509ea/go.mod h1:2QwCwu10qdiPhYu2lbAmcCmPzMbWHLWW/vcvRf/MXnI= github.com/jessevdk/go-flags v1.6.1 h1:Cvu5U8UGrLay1rZfv/zP7iLpSHGUZ/Ou68T0iX1bBK4= github.com/jessevdk/go-flags v1.6.1/go.mod h1:Mk8T1hIAWpOiJiHa9rJASDK2UGWji0EuPGBnNLMooyc= github.com/jmoiron/sqlx v1.4.0 h1:1PLqN7S1UYp5t4SrVVnt4nUVNemrDAtxlulVe+Qgm3o= @@ -63,8 +63,8 @@ github.com/pkg/errors v0.9.1 h1:FEBLx1zS214owpjy7qsBeixbURkuhQAwrK5UwLGTwt4= github.com/pkg/errors v0.9.1/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= github.com/pmezard/go-difflib v1.0.0 h1:4DBwDE0NGyQoBHbLQYPwSUPoCMWR5BEzIk/f1lZbAQM= github.com/pmezard/go-difflib v1.0.0/go.mod h1:iKH77koFhYxTK1pcRnkKkqfTogsbg7gZNVY4sRDYZ/4= -github.com/redis/go-redis/v9 v9.14.1 h1:nDCrEiJmfOWhD76xlaw+HXT0c9hfNWeXgl0vIRYSDvQ= -github.com/redis/go-redis/v9 v9.14.1/go.mod h1:huWgSWd8mW6+m0VPhJjSSQ+d6Nh1VICQ6Q5lHuCH/Iw= +github.com/redis/go-redis/v9 v9.16.0 h1:OotgqgLSRCmzfqChbQyG1PHC3tLNR89DG4jdOERSEP4= +github.com/redis/go-redis/v9 v9.16.0/go.mod h1:u410H11HMLoB+TP67dz8rL9s6QW2j76l0//kSOd3370= github.com/rivo/uniseg v0.1.0/go.mod h1:J6wj4VEh+S6ZtnVlnTBMWIodfgj8LQOQFoIToxlJtxc= github.com/rivo/uniseg v0.2.0 h1:S1pD9weZBuJdFmowNwbpi7BJ8TNftyUImj/0WQi72jY= github.com/rivo/uniseg v0.2.0/go.mod h1:J6wj4VEh+S6ZtnVlnTBMWIodfgj8LQOQFoIToxlJtxc= From 70c91337a97df22c9a7e57b36ab4decb0e8beb1c Mon Sep 17 00:00:00 2001 From: Alvar Penning Date: Fri, 31 Oct 2025 14:09:06 +0100 Subject: [PATCH 22/38] config.example.yml: Reformat notifications-source --- config.example.yml | 18 +++++++++--------- 1 file changed, 9 insertions(+), 9 deletions(-) diff --git a/config.example.yml b/config.example.yml index de5706abf..a110be19f 100644 --- a/config.example.yml +++ b/config.example.yml @@ -142,12 +142,12 @@ redis: # Icinga DB can act as an event source for Icinga Notifications. If the following block is not empty, Icinga DB will # submit events to the Icinga Notifications API. -# notifications-source: -# # URL to the API root. -# api-base-url: http://localhost:5680 -# -# Username to authenticate against the Icinga Notifications API. -# username: icingadb -# -# # Password for the defined user. -# password: insecureinsecure +#notifications-source: + # URL to the API root. +# api-base-url: http://localhost:5680 + + # Username to authenticate against the Icinga Notifications API. +# username: icingadb + + # Password for the defined user. +# password: insecureinsecure From 74cb26ae5b951d7c39bf3059a54d3c0d4ca4c683 Mon Sep 17 00:00:00 2001 From: Alvar Penning Date: Fri, 31 Oct 2025 14:27:53 +0100 Subject: [PATCH 23/38] history: SyncCallbackConf For Common Callback Conf Refactor multiple variables into common struct to ease handling. --- cmd/icingadb/main.go | 22 ++++++------------ pkg/icingadb/history/sync.go | 44 ++++++++++++++++++------------------ 2 files changed, 29 insertions(+), 37 deletions(-) diff --git a/cmd/icingadb/main.go b/cmd/icingadb/main.go index aca620313..ed1aecb9b 100644 --- a/cmd/icingadb/main.go +++ b/cmd/icingadb/main.go @@ -3,7 +3,6 @@ package main import ( "context" "fmt" - "github.com/icinga/icinga-go-library/database" "github.com/icinga/icinga-go-library/logging" "github.com/icinga/icinga-go-library/redis" "github.com/icinga/icinga-go-library/utils" @@ -171,11 +170,7 @@ func run() int { signal.Notify(sig, os.Interrupt, syscall.SIGTERM, syscall.SIGHUP) { - var ( - callbackName string - callbackKeyStructPtr map[string]any - callbackFn func(database.Entity) bool - ) + var callbackCfg *history.SyncCallbackConf if cfg := cmd.Config.NotificationsSource; cfg.ApiBaseUrl != "" { logger.Info("Starting Icinga Notifications source") @@ -190,20 +185,17 @@ func run() int { logger.Fatalw("Can't create Icinga Notifications client from config", zap.Error(err)) } - callbackName = "notifications_sync" - callbackKeyStructPtr = notifications.SyncKeyStructPtrs - callbackFn = notificationsSource.Submit + callbackCfg = &history.SyncCallbackConf{ + Name: "notifications_sync", + KeyStructPtr: notifications.SyncKeyStructPtrs, + Fn: notificationsSource.Submit, + } } go func() { logger.Info("Starting history sync") - if err := hs.Sync( - ctx, - callbackName, - callbackKeyStructPtr, - callbackFn, - ); err != nil && !utils.IsContextCanceled(err) { + if err := hs.Sync(ctx, callbackCfg); err != nil && !utils.IsContextCanceled(err) { logger.Fatalf("%+v", err) } }() diff --git a/pkg/icingadb/history/sync.go b/pkg/icingadb/history/sync.go index 7467742e9..3e553a930 100644 --- a/pkg/icingadb/history/sync.go +++ b/pkg/icingadb/history/sync.go @@ -30,6 +30,17 @@ type Sync struct { logger *logging.Logger } +// SyncCallbackConf configures a callback stage given to Sync.Sync. +type SyncCallbackConf struct { + // Name of this callback, used in [telemetry.Stats]. + Name string + // KeyStructPtr says which pipeline keys should be mapped to which type, identified by a struct pointer. If + // a key is missing from the map, it will not be used for the callback. + KeyStructPtr map[string]any + // Fn is the actual callback function. + Fn func(database.Entity) bool +} + // NewSync creates a new Sync. func NewSync(db *database.DB, redis *redis.Client, logger *logging.Logger) *Sync { return &Sync{ @@ -42,27 +53,16 @@ func NewSync(db *database.DB, redis *redis.Client, logger *logging.Logger) *Sync // Sync synchronizes Redis history streams from s.redis to s.db and deletes the original data on success. // // It is possible to enable a callback functionality, e.g., for the Icinga Notifications integration. To do so, the -// optional callbackFn and callbackKeyStructPtr must be set. Both must either be nil or not nil. If set, the additional -// callbackName must also be set, to be used in [telemetry.Stats]. -// -// The callbackKeyStructPtr says which pipeline keys should be mapped to which type, identified by a struct pointer. If -// a key is missing from the map, it will not be used for the callback. The callbackFn function shall not block. -func (s Sync) Sync( - ctx context.Context, - callbackName string, - callbackKeyStructPtr map[string]any, - callbackFn func(database.Entity) bool, -) error { - if (callbackKeyStructPtr == nil) != (callbackFn == nil) { - return fmt.Errorf("either both callbackKeyStructPtr and callbackFn must be nil or none") - } - if (callbackKeyStructPtr != nil) && (callbackName == "") { - return fmt.Errorf("if callbackKeyStructPtr and callbackFn are set, a callbackName is required") - } - +// callbackCfg must be set according to the SyncCallbackConf struct documentation. +func (s Sync) Sync(ctx context.Context, callbackCfg *SyncCallbackConf) error { var callbackStageFn stageFunc - if callbackKeyStructPtr != nil { - callbackStageFn = makeSortedCallbackStageFunc(ctx, s.logger, callbackName, callbackKeyStructPtr, callbackFn) + if callbackCfg != nil { + callbackStageFn = makeSortedCallbackStageFunc( + ctx, + s.logger, + callbackCfg.Name, + callbackCfg.KeyStructPtr, + callbackCfg.Fn) } g, ctx := errgroup.WithContext(ctx) @@ -94,8 +94,8 @@ func (s Sync) Sync( // other pipeline action, but before deleteFromRedis. var hasCallbackStage bool - if callbackKeyStructPtr != nil { - _, exists := callbackKeyStructPtr[key] + if callbackCfg != nil { + _, exists := callbackCfg.KeyStructPtr[key] hasCallbackStage = exists } From e283ac0d664937f64d205714193705a1303c7a68 Mon Sep 17 00:00:00 2001 From: Alvar Penning Date: Fri, 31 Oct 2025 15:23:50 +0100 Subject: [PATCH 24/38] notifications: Simplify Icinga DB Web Rule Evaluation - Ignore the "config" part of the JSON struct which is only relevant for Icinga DB Web. - Remove unnecessary string conversions. - Small code changes/improvements. --- pkg/notifications/notifications.go | 46 ++++++++---------------------- 1 file changed, 12 insertions(+), 34 deletions(-) diff --git a/pkg/notifications/notifications.go b/pkg/notifications/notifications.go index c4ab1101f..2d19bbb0c 100644 --- a/pkg/notifications/notifications.go +++ b/pkg/notifications/notifications.go @@ -18,7 +18,6 @@ import ( "github.com/pkg/errors" "go.uber.org/zap" "go.uber.org/zap/zapcore" - "slices" "sync" ) @@ -81,12 +80,7 @@ func NewNotificationsClient( // - https://github.com/Icinga/icingadb-web/pull/1289 // - https://github.com/Icinga/icingadb/pull/998#issuecomment-3442298348 func (client *Client) evaluateRulesForObject(ctx context.Context, hostId, serviceId, environmentId types.Binary) ([]string, error) { - const ( - icingaDbWebRuleVersion = 1 - icingaDbWebRuleTypeAll = "all" - icingaDbWebRuleTypeHost = "host" - icingaDbWebRuleTypeService = "service" - ) + const icingaDbWebRuleVersion = 1 type IcingaDbWebQuery struct { Query string `json:"query"` @@ -95,12 +89,8 @@ func (client *Client) evaluateRulesForObject(ctx context.Context, hostId, servic type IcingaDbWebRule struct { Version int `json:"version"` // expect icingaDbWebRuleVersion - Config struct { - Type string `json:"type"` // expect one of [all, host, service] - Filter string `json:"filter"` // Icinga DB Web filter expression - } `json:"config"` Queries struct { - Host *IcingaDbWebQuery `json:"host"` + Host *IcingaDbWebQuery `json:"host,omitempty"` Service *IcingaDbWebQuery `json:"service,omitempty"` } `json:"queries"` } @@ -117,29 +107,20 @@ func (client *Client) evaluateRulesForObject(ctx context.Context, hostId, servic if err := json.Unmarshal([]byte(filterExpr), &webRule); err != nil { return nil, errors.Wrap(err, "cannot decode rule filter expression as JSON into struct") } - if version := webRule.Version; version != icingaDbWebRuleVersion { return nil, errors.Errorf("decoded rule filter expression .Version is %d, %d expected", version, icingaDbWebRuleVersion) } - if cfgType := webRule.Config.Type; !slices.Contains( - []string{icingaDbWebRuleTypeAll, icingaDbWebRuleTypeHost, icingaDbWebRuleTypeService}, cfgType) { - return nil, errors.Errorf("decoded rule filter expression contains unsupported .Config.Type %q", cfgType) - } - if cfgType := webRule.Config.Type; cfgType != icingaDbWebRuleTypeService && webRule.Queries.Host == nil { - return nil, errors.Errorf("decoded rule filter expression for .Config.Type %q with an empty .Queries.Host", cfgType) - } - if cfgType := webRule.Config.Type; cfgType != icingaDbWebRuleTypeHost && webRule.Queries.Service == nil { - return nil, errors.Errorf("decoded rule filter expression for .Config.Type %q with an empty .Queries.Service", cfgType) - } var webQuery IcingaDbWebQuery if !serviceId.Valid() { - if webRule.Config.Type == icingaDbWebRuleTypeService { + // Evaluate rule for a host object + if webRule.Queries.Host == nil { continue } webQuery = *webRule.Queries.Host } else { - if webRule.Config.Type == icingaDbWebRuleTypeHost { + // Evaluate rule for a service object + if webRule.Queries.Service == nil { continue } webQuery = *webRule.Queries.Service @@ -149,34 +130,31 @@ func (client *Client) evaluateRulesForObject(ctx context.Context, hostId, servic for _, param := range webQuery.Parameters { switch param { case ":host_id": - queryArgs = append(queryArgs, hostId.String()) + queryArgs = append(queryArgs, hostId) case ":service_id": if !serviceId.Valid() { return nil, errors.New("host rule filter expression contains :service_id for replacement") } - queryArgs = append(queryArgs, serviceId.String()) + queryArgs = append(queryArgs, serviceId) case ":environment_id": - queryArgs = append(queryArgs, environmentId.String()) + queryArgs = append(queryArgs, environmentId) default: queryArgs = append(queryArgs, param) } } - evaluates, err := func() (bool, error) { + matches, err := func() (bool, error) { rows, err := client.db.QueryContext(ctx, client.db.Rebind(webQuery.Query), queryArgs...) if err != nil { return false, err } defer func() { _ = rows.Close() }() - if !rows.Next() { - return false, nil - } - return true, nil + return rows.Next(), nil }() if err != nil { return nil, errors.Wrapf(err, "cannot fetch rule %q from %q", id, filterExpr) - } else if !evaluates { + } else if !matches { continue } outRuleIds = append(outRuleIds, id) From e475a5ef9153101466c06df84e28339262893e58 Mon Sep 17 00:00:00 2001 From: Alvar Penning Date: Fri, 31 Oct 2025 15:51:24 +0100 Subject: [PATCH 25/38] telemetry: Undo Stats rework Effectively reverting cf4bd92611be38ff324f5b3ab46a8e6b7a41ec1e and passing a pointer to the relevant com.Counter to the history sync. --- cmd/icingadb/main.go | 2 +- pkg/icingadb/history/retention.go | 2 +- pkg/icingadb/history/sync.go | 16 +++--- pkg/icingadb/overdue/sync.go | 2 +- pkg/icingadb/runtime_updates.go | 4 +- pkg/icingadb/sync.go | 4 +- pkg/icingaredis/telemetry/stats.go | 72 +++++++------------------ pkg/icingaredis/telemetry/stats_test.go | 44 --------------- 8 files changed, 35 insertions(+), 111 deletions(-) delete mode 100644 pkg/icingaredis/telemetry/stats_test.go diff --git a/cmd/icingadb/main.go b/cmd/icingadb/main.go index ed1aecb9b..dd2b5b0a2 100644 --- a/cmd/icingadb/main.go +++ b/cmd/icingadb/main.go @@ -186,7 +186,7 @@ func run() int { } callbackCfg = &history.SyncCallbackConf{ - Name: "notifications_sync", + StatPtr: &telemetry.Stats.NotificationSync, KeyStructPtr: notifications.SyncKeyStructPtrs, Fn: notificationsSource.Submit, } diff --git a/pkg/icingadb/history/retention.go b/pkg/icingadb/history/retention.go index e9d893bdc..2d3a6de1a 100644 --- a/pkg/icingadb/history/retention.go +++ b/pkg/icingadb/history/retention.go @@ -230,7 +230,7 @@ func (r *Retention) Start(ctx context.Context) error { deleted, err := stmt.CleanupOlderThan( ctx, r.db, e.Id, r.count, olderThan, - database.OnSuccessIncrement[struct{}](telemetry.Stats.Get(telemetry.StatHistoryCleanup)), + database.OnSuccessIncrement[struct{}](&telemetry.Stats.HistoryCleanup), ) if err != nil { select { diff --git a/pkg/icingadb/history/sync.go b/pkg/icingadb/history/sync.go index 3e553a930..073d352ff 100644 --- a/pkg/icingadb/history/sync.go +++ b/pkg/icingadb/history/sync.go @@ -32,8 +32,8 @@ type Sync struct { // SyncCallbackConf configures a callback stage given to Sync.Sync. type SyncCallbackConf struct { - // Name of this callback, used in [telemetry.Stats]. - Name string + // StatPtr refers a [com.Counter] from the [telemetry.Stats] struct, e.g., Stats.NotificationSync. + StatPtr *com.Counter // KeyStructPtr says which pipeline keys should be mapped to which type, identified by a struct pointer. If // a key is missing from the map, it will not be used for the callback. KeyStructPtr map[string]any @@ -60,7 +60,7 @@ func (s Sync) Sync(ctx context.Context, callbackCfg *SyncCallbackConf) error { callbackStageFn = makeSortedCallbackStageFunc( ctx, s.logger, - callbackCfg.Name, + callbackCfg.StatPtr, callbackCfg.KeyStructPtr, callbackCfg.Fn) } @@ -417,7 +417,7 @@ func countElementStage(ctx context.Context, _ Sync, _ string, in <-chan redis.XM return nil } - telemetry.Stats.Get(telemetry.StatHistory).Add(1) + telemetry.Stats.History.Add(1) out <- msg case <-ctx.Done(): @@ -440,12 +440,12 @@ func countElementStage(ctx context.Context, _ Sync, _ string, in <-chan redis.XM // If the callback function returns false, the message will be retried after an increasing backoff. All subsequent // messages will wait until this one succeeds. // -// For each successfully submitted message, the telemetry stat named after this callback is incremented. Thus, a delta -// between [telemetry.StatHistory] and this stat indicates blocking callbacks. +// For each successfully submitted message, the telemetry stat referenced via a pointer s incremented. Thus, a delta +// between telemetry.Stats.History and this stat indicates blocking callbacks. func makeSortedCallbackStageFunc( ctx context.Context, logger *logging.Logger, - name string, + statPtr *com.Counter, keyStructPtrs map[string]any, fn func(database.Entity) bool, ) stageFunc { @@ -490,7 +490,7 @@ func makeSortedCallbackStageFunc( success := fn(entity) if success { - telemetry.Stats.Get(name).Add(1) + statPtr.Add(1) } return success } diff --git a/pkg/icingadb/overdue/sync.go b/pkg/icingadb/overdue/sync.go index 049f217da..b1b2f488c 100644 --- a/pkg/icingadb/overdue/sync.go +++ b/pkg/icingadb/overdue/sync.go @@ -219,7 +219,7 @@ func (s Sync) updateOverdue( } counter.Add(uint64(len(ids))) - telemetry.Stats.Get(telemetry.StatOverdue).Add(uint64(len(ids))) + telemetry.Stats.Overdue.Add(uint64(len(ids))) var op func(ctx context.Context, key string, members ...any) *redis.IntCmd if overdue { diff --git a/pkg/icingadb/runtime_updates.go b/pkg/icingadb/runtime_updates.go index e5b5efb26..888f4b4d9 100644 --- a/pkg/icingadb/runtime_updates.go +++ b/pkg/icingadb/runtime_updates.go @@ -184,7 +184,7 @@ func (r *RuntimeUpdates) Sync( return r.db.NamedBulkExec( ctx, cvStmt, cvCount, sem, customvars, database.SplitOnDupId[database.Entity], database.OnSuccessIncrement[database.Entity](&counter), - database.OnSuccessIncrement[database.Entity](telemetry.Stats.Get(telemetry.StatConfig)), + database.OnSuccessIncrement[database.Entity](&telemetry.Stats.Config), ) }) @@ -204,7 +204,7 @@ func (r *RuntimeUpdates) Sync( return r.db.NamedBulkExec( ctx, cvFlatStmt, cvFlatCount, sem, flatCustomvars, database.SplitOnDupId[database.Entity], database.OnSuccessIncrement[database.Entity](&counter), - database.OnSuccessIncrement[database.Entity](telemetry.Stats.Get(telemetry.StatConfig)), + database.OnSuccessIncrement[database.Entity](&telemetry.Stats.Config), ) }) diff --git a/pkg/icingadb/sync.go b/pkg/icingadb/sync.go index 73e4d2e39..b353c7848 100644 --- a/pkg/icingadb/sync.go +++ b/pkg/icingadb/sync.go @@ -225,8 +225,8 @@ func (s Sync) SyncCustomvars(ctx context.Context) error { func getCounterForEntity(e database.Entity) *com.Counter { switch e.(type) { case *v1.HostState, *v1.ServiceState: - return telemetry.Stats.Get(telemetry.StatState) + return &telemetry.Stats.State default: - return telemetry.Stats.Get(telemetry.StatConfig) + return &telemetry.Stats.Config } } diff --git a/pkg/icingaredis/telemetry/stats.go b/pkg/icingaredis/telemetry/stats.go index bbe8c03cb..106e81af6 100644 --- a/pkg/icingaredis/telemetry/stats.go +++ b/pkg/icingaredis/telemetry/stats.go @@ -2,74 +2,42 @@ package telemetry import ( "context" - "fmt" "github.com/icinga/icinga-go-library/com" "github.com/icinga/icinga-go-library/logging" "github.com/icinga/icinga-go-library/periodic" "github.com/icinga/icinga-go-library/redis" "github.com/icinga/icinga-go-library/utils" "go.uber.org/zap" - "iter" "strconv" - "sync" "time" ) -// StatsKeeper holds multiple [com.Counter] values by name, to be used for statistics in WriteStats. -type StatsKeeper struct { - m sync.Map +var Stats struct { + // Config & co. are to be increased by the T sync once for every T object synced. + Config com.Counter + State com.Counter + History com.Counter + Callback com.Counter + Overdue com.Counter + HistoryCleanup com.Counter + NotificationSync com.Counter } -// Get or create a [com.Counter] by its name. -func (statsKeeper *StatsKeeper) Get(key string) *com.Counter { - ctrAny, _ := statsKeeper.m.LoadOrStore(key, &com.Counter{}) - - ctr, ok := ctrAny.(*com.Counter) - if !ok { - // Should not happen unless someone messes with the internal map. - panic(fmt.Sprintf( - "StatsKeeper.Get(%q) returned something of type %T, not *com.Counter", - key, ctrAny)) - } - - return ctr -} - -// Iterator over all keys and their [com.Counter]. -func (statsKeeper *StatsKeeper) Iterator() iter.Seq2[string, *com.Counter] { - return func(yield func(string, *com.Counter) bool) { - statsKeeper.m.Range(func(keyAny, ctrAny any) bool { - key, keyOk := keyAny.(string) - ctr, ctrOk := ctrAny.(*com.Counter) - if !keyOk || !ctrOk { - // Should not happen unless someone messes with the internal map. - panic(fmt.Sprintf( - "iterating StatsKeeper on key %q got types (%T, %T), not (string, *com.Counter)", - keyAny, keyAny, ctrAny)) - } - - return yield(key, ctr) - }) - } -} - -// Stats is the singleton StatsKeeper to be used to access a [com.Counter]. -var Stats = &StatsKeeper{} - -// Keys for different well known Stats entries. -const ( - StatConfig = "config_sync" - StatState = "state_sync" - StatHistory = "history_sync" - StatOverdue = "overdue_sync" - StatHistoryCleanup = "history_cleanup" -) - // WriteStats periodically forwards Stats to Redis for being monitored by Icinga 2. func WriteStats(ctx context.Context, client *redis.Client, logger *logging.Logger) { + counters := map[string]*com.Counter{ + "config_sync": &Stats.Config, + "state_sync": &Stats.State, + "history_sync": &Stats.History, + "callback_sync": &Stats.Callback, + "overdue_sync": &Stats.Overdue, + "history_cleanup": &Stats.HistoryCleanup, + "notification_sync": &Stats.NotificationSync, + } + periodic.Start(ctx, time.Second, func(_ periodic.Tick) { var data []string - for kind, counter := range Stats.Iterator() { + for kind, counter := range counters { if cnt := counter.Reset(); cnt > 0 { data = append(data, kind, strconv.FormatUint(cnt, 10)) } diff --git a/pkg/icingaredis/telemetry/stats_test.go b/pkg/icingaredis/telemetry/stats_test.go deleted file mode 100644 index 4b851e989..000000000 --- a/pkg/icingaredis/telemetry/stats_test.go +++ /dev/null @@ -1,44 +0,0 @@ -package telemetry - -import ( - "github.com/stretchr/testify/assert" - "testing" -) - -func TestStatsKeeper(t *testing.T) { - desiredState := map[string]uint64{ - "foo": 23, - "bar": 42, - "baz": 0, - } - - stats := &StatsKeeper{} - - // Populate based on desiredState - for key, counterValue := range desiredState { - ctr := stats.Get(key) - ctr.Add(counterValue) - } - - // Check if desiredState is set - for key, counterValue := range desiredState { - ctr := stats.Get(key) - assert.Equal(t, counterValue, ctr.Val()) - } - - // Get reference, change value, compare - fooKey := "foo" - fooCtr := stats.Get(fooKey) - assert.Equal(t, desiredState[fooKey], fooCtr.Reset()) - assert.Equal(t, uint64(0), fooCtr.Val()) - assert.Equal(t, uint64(0), stats.Get(fooKey).Val()) - fooCtr.Add(desiredState[fooKey]) - assert.Equal(t, desiredState[fooKey], stats.Get(fooKey).Val()) - - // Range over - for key, ctr := range stats.Iterator() { - desired, ok := desiredState[key] - assert.True(t, ok) - assert.Equal(t, desired, ctr.Val()) - } -} From e708d40873549a9bde24a4e6abedf762d94c8c5e Mon Sep 17 00:00:00 2001 From: Alvar Penning Date: Mon, 3 Nov 2025 09:22:24 +0100 Subject: [PATCH 26/38] history.parseRedisStreamId: Remove regex With parseRedisStreamId being in the "hot path", the quite simple regular expression was exchanged with an even simpler string split. However, as network operations precede and follow this, the benefit of this optimization might be questionable. --- pkg/icingadb/history/sorter.go | 13 ++++++------- pkg/icingadb/history/sorter_test.go | 20 ++++++++++++++++++++ 2 files changed, 26 insertions(+), 7 deletions(-) diff --git a/pkg/icingadb/history/sorter.go b/pkg/icingadb/history/sorter.go index af55168ca..172daf384 100644 --- a/pkg/icingadb/history/sorter.go +++ b/pkg/icingadb/history/sorter.go @@ -7,28 +7,27 @@ import ( "github.com/pkg/errors" "go.uber.org/zap" "math" - "regexp" "sort" "strconv" + "strings" "time" ) // parseRedisStreamId parses a Redis Stream ID and returns the timestamp in ms and the sequence number, or an error. func parseRedisStreamId(redisStreamId string) (int64, int64, error) { - re := regexp.MustCompile(`^(\d+)-(\d+)$`) - matches := re.FindStringSubmatch(redisStreamId) - if len(matches) != 3 { - return 0, 0, errors.Errorf("value %q does not satisfy Redis Stream ID regex", redisStreamId) + dashPos := strings.IndexRune(redisStreamId, '-') + if dashPos <= 0 { + return 0, 0, errors.Errorf("value %q does not satisfy Redis Stream ID pattern", redisStreamId) } - ms, err := strconv.ParseInt(matches[1], 10, 64) + ms, err := strconv.ParseInt(redisStreamId[:dashPos], 10, 64) if err != nil { return 0, 0, errors.Wrapf( err, "timestamp part of the Redis Stream ID %q cannot be parsed to int", redisStreamId) } - seq, err := strconv.ParseInt(matches[2], 10, 64) + seq, err := strconv.ParseInt(redisStreamId[dashPos+1:], 10, 64) if err != nil { return 0, 0, errors.Wrapf( err, diff --git a/pkg/icingadb/history/sorter_test.go b/pkg/icingadb/history/sorter_test.go index f5551e42d..48960ac53 100644 --- a/pkg/icingadb/history/sorter_test.go +++ b/pkg/icingadb/history/sorter_test.go @@ -45,6 +45,26 @@ func Test_redisStreamIdToMs(t *testing.T) { input: "23-42-23", wantErr: true, }, + { + name: "missing first part", + input: "-23", + wantErr: true, + }, + { + name: "missing second part", + input: "23-", + wantErr: true, + }, + { + name: "only dash", + input: "-", + wantErr: true, + }, + { + name: "just invalid", + input: "oops", + wantErr: true, + }, { name: "invalid field types", input: "0x23-0x42", From e4c14e72f74e7626674af2cfd7a8c85fb9790194 Mon Sep 17 00:00:00 2001 From: Alvar Penning Date: Wed, 5 Nov 2025 09:33:12 +0100 Subject: [PATCH 27/38] history.StreamSorter: Various Fixes - Store the streamSorterSubmission submission time as a time.Time instead of a nanosecond timestamp, comparing the time.Timer's monotonic clock. - Replace time-based buckets in StreamSorter.submissionWorker by a heap to be pushed and popped. However, only submissions of a certain age are being forwarded. Reduces complexity quite a bit. - Reduce complexity of StreamSorter.queueWorker by getting rid of unnecessary channel signals by checking for new queue events for processing at the loop start. --- pkg/icingadb/history/sorter.go | 210 ++++++++++++++-------------- pkg/icingadb/history/sorter_test.go | 21 +-- 2 files changed, 117 insertions(+), 114 deletions(-) diff --git a/pkg/icingadb/history/sorter.go b/pkg/icingadb/history/sorter.go index 172daf384..a6eeed922 100644 --- a/pkg/icingadb/history/sorter.go +++ b/pkg/icingadb/history/sorter.go @@ -1,13 +1,14 @@ package history import ( + "container/heap" "context" + "fmt" "github.com/icinga/icinga-go-library/logging" "github.com/icinga/icinga-go-library/redis" "github.com/pkg/errors" "go.uber.org/zap" - "math" - "sort" + "go.uber.org/zap/zapcore" "strconv" "strings" "time" @@ -47,18 +48,30 @@ type streamSorterSubmission struct { out chan<- redis.XMessage // Required for sorting. - streamIdMs int64 // streamIdMs is the Redis Stream ID timestamp part (milliseconds) - streamIdSeq int64 // streamIdSeq is the Redis Stream ID sequence number - submitTimeNs int64 // submitTimeNs is the timestamp when the element was submitted (in nanoseconds) + streamIdMs int64 // streamIdMs is the Redis Stream ID timestamp part (milliseconds) + streamIdSeq int64 // streamIdSeq is the Redis Stream ID sequence number + submitTime time.Time // submitTime is the timestamp when the element was submitted } -// streamSorterSubmissions implements sort.Interface for []streamSorterSubmission. +// MarshalLogObject implements [zapcore.ObjectMarshaler]. +func (sub streamSorterSubmission) MarshalLogObject(encoder zapcore.ObjectEncoder) error { + encoder.AddInt64("redis-id-ms", sub.streamIdMs) + encoder.AddInt64("redis-id-seq", sub.streamIdSeq) + encoder.AddTime("submit-time", sub.submitTime) + + return nil +} + +// streamSorterSubmissions implements [heap.Interface] for []streamSorterSubmission. type streamSorterSubmissions []streamSorterSubmission +// Len implements [sort.Interface] required by [heap.Interface]. func (subs streamSorterSubmissions) Len() int { return len(subs) } +// Swap implements [sort.Interface] required by [heap.Interface]. func (subs streamSorterSubmissions) Swap(i, j int) { subs[i], subs[j] = subs[j], subs[i] } +// Less implements [sort.Interface] required by [heap.Interface]. func (subs streamSorterSubmissions) Less(i, j int) bool { a, b := subs[i], subs[j] if a.streamIdMs != b.streamIdMs { @@ -67,18 +80,37 @@ func (subs streamSorterSubmissions) Less(i, j int) bool { if a.streamIdSeq != b.streamIdSeq { return a.streamIdSeq < b.streamIdSeq } - return a.submitTimeNs < b.submitTimeNs + return a.submitTime.Before(b.submitTime) +} + +// Push implements [heap.Interface]. +func (subs *streamSorterSubmissions) Push(x any) { + sub, ok := x.(streamSorterSubmission) + if !ok { + panic(fmt.Sprintf("streamSorterSubmissions.Push received x of %T", x)) + } + + *subs = append(*subs, sub) +} + +// Pop implements [heap.Interface]. +func (subs *streamSorterSubmissions) Pop() any { + old := *subs + n := len(old) + x := old[n-1] + *subs = old[0 : n-1] + return x } // StreamSorter accepts multiple [redis.XMessage] via Submit and ejects them in an ordered fashion. // -// Internally, two goroutines are used. One collects the submissions and puts them into buckets based on the second -// of the Redis Stream ID. After three seconds, each bucket is being sorted and ejected to the other goroutine. There, -// each element is passed to the callback function in order. Only if the callback function has succeeded, it is removed -// from the top of the queue. +// Internally, two goroutines are used. The first one collects the submissions and sorts them into a heap based on the +// Redis Stream ID. After being in the heap for at least three seconds, a submission is forwarded to the other +// goroutine. There, each element is passed to the callback function in order. Only if the callback function has +// succeeded, it is removed from the top of the queue. // -// Thus, if a message is received delayed for more than three seconds, it will be relayed out of order while an error is -// being logged. The StreamSorter is only able to ensure order to a certain degree of chaos. +// Thus, if a message is received delayed for more than three seconds, it will be relayed out of order. The StreamSorter +// is only able to ensure order to a certain degree of chaos. // // The callback function receives the [redis.XMessage] together with generic args passed in Submit for additional // context. If the callback function returns true, the element will be removed from the queue. Otherwise, the element @@ -106,23 +138,24 @@ func NewStreamSorter( _ = context.AfterFunc(ctx, func() { close(sorter.submissionCh) }) - ch := make(chan []streamSorterSubmission) + ch := make(chan streamSorterSubmission) go sorter.submissionWorker(ctx, ch) go sorter.queueWorker(ctx, ch) return sorter } -// submissionWorker listens ton submissionCh populated by Submit, fills buckets and ejects them into out, linked to -// the queueWorker goroutine for further processing. -func (sorter *StreamSorter) submissionWorker(ctx context.Context, out chan<- []streamSorterSubmission) { - // slots defines how many second slots should be kept for sorting - const slots = 3 - // buckets maps timestamp in seconds to streamSorterSubmissions made within this second - buckets := make(map[int64][]streamSorterSubmission) - +// submissionWorker listens ton submissionCh populated by Submit, fills the heap and ejects streamSorterSubmissions into +// out, linked to the queueWorker goroutine for further processing. +func (sorter *StreamSorter) submissionWorker(ctx context.Context, out chan<- streamSorterSubmission) { defer close(out) + // When a streamSorterSubmission is created in the Submit method, the current time.Time is added to the struct. + // Only if the submission was at least three seconds (submissionMinAge) ago, a popped submission from the heap will + // be forwarded to the other goroutine for future processing. + const submissionMinAge = 3 * time.Second + submissionHeap := &streamSorterSubmissions{} + ticker := time.NewTicker(time.Second) defer ticker.Stop() @@ -131,81 +164,58 @@ func (sorter *StreamSorter) submissionWorker(ctx context.Context, out chan<- []s case <-ctx.Done(): return - case submission := <-sorter.submissionCh: - curBucketId := time.Now().Unix() - bucketId := submission.streamIdMs / 1_000 - if minBucketId := curBucketId - slots; bucketId < minBucketId { - sorter.logger.Errorw("Received message with Stream ID from the far past, put in last bucket", - zap.String("id", submission.msg.ID), - zap.Int64("buckets-behind", minBucketId-bucketId)) - bucketId = minBucketId - } else if bucketId > curBucketId { - sorter.logger.Warnw("Received message with Stream ID from the future", - zap.String("id", submission.msg.ID), - zap.Int64("buckets-ahead", bucketId-curBucketId)) - } - + case sub := <-sorter.submissionCh: if sorter.verbose { - sorter.logger.Debugw("Insert submission into bucket", - zap.String("id", submission.msg.ID), - zap.Int64("bucket-id", bucketId)) + sorter.logger.Debugw("Push submission to heap", zap.Object("submission", sub)) } - bucket, ok := buckets[bucketId] - if !ok { - bucket = make([]streamSorterSubmission, 0, 1) - } - buckets[bucketId] = append(bucket, submission) + heap.Push(submissionHeap, sub) case <-ticker.C: - // Search the smallest bucket ID older than slots+1 seconds by iterating over the keys. This is fast due to - // slots being 3 and the submission code eliminates inserts from the far past. Usually, the latest bucket ID - // should be "time.Now().Unix() - slots - 1", but I raced this with a very busy submission channel. - bucketId := int64(math.MaxInt64) - bucketSup := time.Now().Unix() - slots - 1 - for k := range buckets { - if k <= bucketSup { - bucketId = min(bucketId, k) + start := time.Now() + submissionCounter := 0 + + for submissionHeap.Len() > 0 { + x := heap.Pop(submissionHeap) + sub, ok := x.(streamSorterSubmission) + if !ok { + panic(fmt.Sprintf("invalid type %T from submission heap", x)) } - } - bucket, ok := buckets[bucketId] - if !ok { - continue - } - delete(buckets, bucketId) + if time.Since(sub.submitTime) < submissionMinAge { + if sorter.verbose { + sorter.logger.Debugw("Stopped popping heap as submission is not old enough", + zap.Object("submission", sub), + zap.Int("submissions", submissionCounter), + zap.Duration("duration", time.Since(start))) + } - sort.Sort(streamSorterSubmissions(bucket)) - out <- bucket + heap.Push(submissionHeap, sub) + break + } - if sorter.verbose { - sorter.logger.Debugw("Ejected submission bucket to callback worker", - zap.Int64("bucket-id", bucketId), - zap.Int("bucket-size", len(bucket))) + out <- sub + submissionCounter++ + } + + if sorter.verbose && submissionCounter > 0 { + sorter.logger.Debugw("Ejected submissions to callback worker", + zap.Int("submissions", submissionCounter), + zap.Duration("duration", time.Since(start))) } } } } // queueWorker receives sorted streamSorterSubmissions from submissionWorker and forwards them to the callback. -func (sorter *StreamSorter) queueWorker(ctx context.Context, in <-chan []streamSorterSubmission) { - // Each streamSorterSubmission received bucket-wise from in is stored in the queue slice. From there on, the slice - // head is passed to the callback function. The queueEventCh has a buffer capacity of 1 to allow both filling and - // consuming in the same goroutine. +func (sorter *StreamSorter) queueWorker(ctx context.Context, in <-chan streamSorterSubmission) { + // Each streamSorterSubmission received from "in" is stored in the queue slice. From there on, the slice head is + // passed to the callback function. The queueEventCh has a buffer capacity of 1 to allow both filling and consuming + // in the same goroutine. queue := make([]streamSorterSubmission, 0, 1024) queueEventCh := make(chan struct{}, 1) defer close(queueEventCh) - // queueEvent places something in queueEventCh w/o deadlocking - queueEvent := func() { - // Always drain channel first. Consider positive <-queueEventCh case followed by <-in. Within <-in, a second - // struct{}{} would be sent, effectively deadlocking. - for len(queueEventCh) > 0 { - <-queueEventCh - } - queueEventCh <- struct{}{} - } - // The actual callback function is executed concurrently as it might block longer than expected. A blocking select // would result in the queue not being populated, effectively blocking the submissionWorker. Thus, the callbackFn is // started in a goroutine, signaling back its success status via callbackCh. If no callback is active, the channel @@ -241,34 +251,28 @@ func (sorter *StreamSorter) queueWorker(ctx context.Context, in <-chan []streamS } for { + if len(queue) > 0 && callbackCh == nil { + callbackCh = make(chan bool) + go callbackFn(queue[0]) + } + select { case <-ctx.Done(): return - case submissions, ok := <-in: + case sub, ok := <-in: if !ok { return } - queue = append(queue, submissions...) - queueEvent() + queue = append(queue, sub) if sorter.verbose { - sorter.logger.Debugw("Queue worker received new submissions", + sorter.logger.Debugw("Queue worker received new submission", + zap.Object("submission", sub), zap.Int("queue-size", len(queue))) } - case <-queueEventCh: - if len(queue) == 0 { - continue - } - - if callbackCh != nil { - continue - } - callbackCh = make(chan bool) - go callbackFn(queue[0]) - case success := <-callbackCh: if success { queue = queue[1:] @@ -277,12 +281,8 @@ func (sorter *StreamSorter) queueWorker(ctx context.Context, in <-chan []streamS close(callbackCh) callbackCh = nil - if len(queue) > 0 { - queueEvent() - } else { - if sorter.verbose { - sorter.logger.Debug("Queue worker finished processing queue") - } + if sorter.verbose && len(queue) == 0 { + sorter.logger.Debug("Queue worker finished processing queue") } } } @@ -302,12 +302,12 @@ func (sorter *StreamSorter) Submit(msg redis.XMessage, args any, out chan<- redi } submission := streamSorterSubmission{ - msg: msg, - args: args, - out: out, - streamIdMs: ms, - streamIdSeq: seq, - submitTimeNs: time.Now().UnixNano(), + msg: msg, + args: args, + out: out, + streamIdMs: ms, + streamIdSeq: seq, + submitTime: time.Now(), } select { diff --git a/pkg/icingadb/history/sorter_test.go b/pkg/icingadb/history/sorter_test.go index 48960ac53..7b5539ce0 100644 --- a/pkg/icingadb/history/sorter_test.go +++ b/pkg/icingadb/history/sorter_test.go @@ -87,16 +87,19 @@ func Test_redisStreamIdToMs(t *testing.T) { } func Test_streamSorterSubmissions(t *testing.T) { + mkSubmitTime := func(offset int) time.Time { + return time.Date(2009, 11, 10, 23, 0, 0, offset, time.UTC) + } submissions := []streamSorterSubmission{ - {streamIdMs: 0, streamIdSeq: 0, submitTimeNs: 0}, - {streamIdMs: 1, streamIdSeq: 0, submitTimeNs: 0}, - {streamIdMs: 1, streamIdSeq: 1, submitTimeNs: 0}, - {streamIdMs: 2, streamIdSeq: 0, submitTimeNs: 0}, - {streamIdMs: 2, streamIdSeq: 0, submitTimeNs: 1}, - {streamIdMs: 3, streamIdSeq: 0, submitTimeNs: 0}, - {streamIdMs: 3, streamIdSeq: 1, submitTimeNs: 0}, - {streamIdMs: 3, streamIdSeq: 1, submitTimeNs: 1}, - {streamIdMs: 3, streamIdSeq: 1, submitTimeNs: 2}, + {streamIdMs: 0, streamIdSeq: 0, submitTime: mkSubmitTime(0)}, + {streamIdMs: 1, streamIdSeq: 0, submitTime: mkSubmitTime(0)}, + {streamIdMs: 1, streamIdSeq: 1, submitTime: mkSubmitTime(0)}, + {streamIdMs: 2, streamIdSeq: 0, submitTime: mkSubmitTime(0)}, + {streamIdMs: 2, streamIdSeq: 0, submitTime: mkSubmitTime(1)}, + {streamIdMs: 3, streamIdSeq: 0, submitTime: mkSubmitTime(0)}, + {streamIdMs: 3, streamIdSeq: 1, submitTime: mkSubmitTime(0)}, + {streamIdMs: 3, streamIdSeq: 1, submitTime: mkSubmitTime(1)}, + {streamIdMs: 3, streamIdSeq: 1, submitTime: mkSubmitTime(2)}, } submissionsRand := make([]streamSorterSubmission, 0, len(submissions)) From 8a86d357087bcef293d3ca5f41a35a3c4f0c5b68 Mon Sep 17 00:00:00 2001 From: Alvar Penning Date: Wed, 5 Nov 2025 13:10:14 +0100 Subject: [PATCH 28/38] history.StreamSorter: Cleanup Output Channel Introduce the StreamSorter.CloseOutput method to remove all submissions for a certain output channel from both workers before closing the channel. The motivation behind this change is to have a single point where the output channel is closed while no submissions are being sent into an already closed channel. --- pkg/icingadb/history/sorter.go | 114 ++++++++++++++++++++++++---- pkg/icingadb/history/sorter_test.go | 25 +++++- pkg/icingadb/history/sync.go | 12 ++- 3 files changed, 131 insertions(+), 20 deletions(-) diff --git a/pkg/icingadb/history/sorter.go b/pkg/icingadb/history/sorter.go index a6eeed922..3b1467fbd 100644 --- a/pkg/icingadb/history/sorter.go +++ b/pkg/icingadb/history/sorter.go @@ -9,6 +9,7 @@ import ( "github.com/pkg/errors" "go.uber.org/zap" "go.uber.org/zap/zapcore" + "slices" "strconv" "strings" "time" @@ -116,9 +117,12 @@ func (subs *streamSorterSubmissions) Pop() any { // context. If the callback function returns true, the element will be removed from the queue. Otherwise, the element // will be kept at top of the queue and retried next time. type StreamSorter struct { - logger *logging.Logger - callbackFn func(redis.XMessage, any) bool - submissionCh chan streamSorterSubmission + ctx context.Context + logger *logging.Logger + callbackFn func(redis.XMessage, any) bool + submissionCh chan streamSorterSubmission + closeChSubmission chan chan<- redis.XMessage + closeChQueue chan chan<- redis.XMessage // verbose implies a verbose debug logging. Don't think one want to have this outside the tests. verbose bool @@ -131,23 +135,30 @@ func NewStreamSorter( callbackFn func(msg redis.XMessage, args any) bool, ) *StreamSorter { sorter := &StreamSorter{ - logger: logger, - callbackFn: callbackFn, - submissionCh: make(chan streamSorterSubmission), + ctx: ctx, + logger: logger, + callbackFn: callbackFn, + submissionCh: make(chan streamSorterSubmission), + closeChSubmission: make(chan chan<- redis.XMessage), + closeChQueue: make(chan chan<- redis.XMessage), } - _ = context.AfterFunc(ctx, func() { close(sorter.submissionCh) }) + _ = context.AfterFunc(ctx, func() { + close(sorter.submissionCh) + close(sorter.closeChSubmission) + close(sorter.closeChQueue) + }) ch := make(chan streamSorterSubmission) - go sorter.submissionWorker(ctx, ch) - go sorter.queueWorker(ctx, ch) + go sorter.submissionWorker(ch) + go sorter.queueWorker(ch) return sorter } // submissionWorker listens ton submissionCh populated by Submit, fills the heap and ejects streamSorterSubmissions into // out, linked to the queueWorker goroutine for further processing. -func (sorter *StreamSorter) submissionWorker(ctx context.Context, out chan<- streamSorterSubmission) { +func (sorter *StreamSorter) submissionWorker(out chan<- streamSorterSubmission) { defer close(out) // When a streamSorterSubmission is created in the Submit method, the current time.Time is added to the struct. @@ -161,16 +172,41 @@ func (sorter *StreamSorter) submissionWorker(ctx context.Context, out chan<- str for { select { - case <-ctx.Done(): + case <-sorter.ctx.Done(): return - case sub := <-sorter.submissionCh: + case sub, ok := <-sorter.submissionCh: + if !ok { + return + } + if sorter.verbose { sorter.logger.Debugw("Push submission to heap", zap.Object("submission", sub)) } heap.Push(submissionHeap, sub) + case ch, ok := <-sorter.closeChSubmission: + if !ok { + return + } + + bkp := &streamSorterSubmissions{} + for submissionHeap.Len() > 0 { + x := heap.Pop(submissionHeap) + sub, ok := x.(streamSorterSubmission) + if !ok { + panic(fmt.Sprintf("invalid type %T from submission heap", x)) + } + + if sub.out == ch { + continue + } + + bkp.Push(sub) + } + submissionHeap = bkp + case <-ticker.C: start := time.Now() submissionCounter := 0 @@ -208,7 +244,7 @@ func (sorter *StreamSorter) submissionWorker(ctx context.Context, out chan<- str } // queueWorker receives sorted streamSorterSubmissions from submissionWorker and forwards them to the callback. -func (sorter *StreamSorter) queueWorker(ctx context.Context, in <-chan streamSorterSubmission) { +func (sorter *StreamSorter) queueWorker(in <-chan streamSorterSubmission) { // Each streamSorterSubmission received from "in" is stored in the queue slice. From there on, the slice head is // passed to the callback function. The queueEventCh has a buffer capacity of 1 to allow both filling and consuming // in the same goroutine. @@ -225,7 +261,7 @@ func (sorter *StreamSorter) queueWorker(ctx context.Context, in <-chan streamSor var callbackCh chan bool callbackFn := func(submission streamSorterSubmission) { select { - case <-ctx.Done(): + case <-sorter.ctx.Done(): return case <-time.After(callbackDelay): } @@ -233,6 +269,13 @@ func (sorter *StreamSorter) queueWorker(ctx context.Context, in <-chan streamSor start := time.Now() success := sorter.callbackFn(submission.msg, submission.args) if success { + defer func() { + // Ensure not to panic if the out channel was closed via CloseOutput in the meantime. + if r := recover(); r != nil { + sorter.logger.Error("Recovered from sending submission", zap.Any("recovery", r)) + } + }() + submission.out <- submission.msg callbackDelay = 0 } else { @@ -257,7 +300,7 @@ func (sorter *StreamSorter) queueWorker(ctx context.Context, in <-chan streamSor } select { - case <-ctx.Done(): + case <-sorter.ctx.Done(): return case sub, ok := <-in: @@ -273,8 +316,18 @@ func (sorter *StreamSorter) queueWorker(ctx context.Context, in <-chan streamSor zap.Int("queue-size", len(queue))) } + case ch, ok := <-sorter.closeChQueue: + if !ok { + return + } + + queue = slices.DeleteFunc(queue, func(sub streamSorterSubmission) bool { + return sub.out == ch + }) + case success := <-callbackCh: - if success { + // The len(queue) part is necessary as sorter.closeChQueue might interfere. + if success && len(queue) > 0 { queue = queue[1:] } @@ -311,6 +364,9 @@ func (sorter *StreamSorter) Submit(msg redis.XMessage, args any, out chan<- redi } select { + case <-sorter.ctx.Done(): + return sorter.ctx.Err() + case sorter.submissionCh <- submission: return nil @@ -318,3 +374,29 @@ func (sorter *StreamSorter) Submit(msg redis.XMessage, args any, out chan<- redi return errors.New("submission timed out") } } + +// CloseOutput clears all submissions targeting this output channel and closes the channel afterwards. +// +// This will only result in submissions with this out channel to be removed from both the submissionWorker's heap and +// the queueWorker's queue. In case such a submission is already in the actual submission process, it might still be +// tried, but sending it to the out channel is recovered internally. +// +// As filtering/recreating the caches is potentially expensive, only call this method if required. In the current +// architecture of sync.go, this is fine. +func (sorter *StreamSorter) CloseOutput(out chan<- redis.XMessage) error { + for _, ch := range []chan chan<- redis.XMessage{sorter.closeChSubmission, sorter.closeChQueue} { + select { + case <-sorter.ctx.Done(): + return sorter.ctx.Err() + + case ch <- out: + + case <-time.After(time.Second): + return errors.New("sending to channel for closing timed out") + } + } + + close(out) + + return nil +} diff --git a/pkg/icingadb/history/sorter_test.go b/pkg/icingadb/history/sorter_test.go index 7b5539ce0..a3ab7a31d 100644 --- a/pkg/icingadb/history/sorter_test.go +++ b/pkg/icingadb/history/sorter_test.go @@ -116,6 +116,7 @@ func TestStreamSorter(t *testing.T) { name string messages int producers int + producersEarlyClose int callbackMaxDelayMs int callbackSuccessPercent int expectTimeout bool @@ -184,6 +185,15 @@ func TestStreamSorter(t *testing.T) { callbackSuccessPercent: 100, outMaxDelayMs: 1000, }, + { + name: "producer out early close", + messages: 100, + producers: 10, + producersEarlyClose: 5, + callbackMaxDelayMs: 1000, + callbackSuccessPercent: 100, + expectTimeout: true, + }, { name: "pure chaos", messages: 50, @@ -247,9 +257,13 @@ func TestStreamSorter(t *testing.T) { callbackFn) sorter.verbose = true - for range tt.producers { + for i := range tt.producers { + earlyClose := i < tt.producersEarlyClose + out := make(chan redis.XMessage) - defer close(out) // no leakage, general cleanup after finishing test run + if !earlyClose { + defer func() { _ = sorter.CloseOutput(out) }() // no leakage, general cleanup + } go func() { for { @@ -282,6 +296,13 @@ func TestStreamSorter(t *testing.T) { msg := redis.XMessage{ID: fmt.Sprintf("%d-%d", ms, seq)} require.NoError(t, sorter.Submit(msg, nil, out)) + + // 25% chance of closing for early closing producers + if earlyClose && rand.Int63n(4) == 3 { + require.NoError(t, sorter.CloseOutput(out)) + t.Log("Successfully closed producer early") + return + } } }() diff --git a/pkg/icingadb/history/sync.go b/pkg/icingadb/history/sync.go index 073d352ff..1b248b587 100644 --- a/pkg/icingadb/history/sync.go +++ b/pkg/icingadb/history/sync.go @@ -498,7 +498,13 @@ func makeSortedCallbackStageFunc( sorter := NewStreamSorter(ctx, logger, sorterCallbackFn) return func(ctx context.Context, s Sync, key string, in <-chan redis.XMessage, out chan<- redis.XMessage) error { - defer close(out) + defer func() { + if err := sorter.CloseOutput(out); err != nil { + s.logger.Errorw("Closing stream sorter output failed", + zap.String("key", key), + zap.Error(err)) + } + }() for { select { @@ -509,7 +515,9 @@ func makeSortedCallbackStageFunc( err := sorter.Submit(msg, key, out) if err != nil { - s.logger.Errorw("Failed to submit Redis stream event to stream sorter", zap.Error(err)) + s.logger.Errorw("Failed to submit Redis stream event to stream sorter", + zap.String("key", key), + zap.Error(err)) } case <-ctx.Done(): From ffedb952264aaa688a856b0da7d28ef43aa30fd8 Mon Sep 17 00:00:00 2001 From: Alvar Penning Date: Wed, 5 Nov 2025 13:13:39 +0100 Subject: [PATCH 29/38] telemetry: Remove leftover Stats.Callback Seems to be forgotten in e475a5ef9153101466c06df84e28339262893e58. --- pkg/icingaredis/telemetry/stats.go | 2 -- 1 file changed, 2 deletions(-) diff --git a/pkg/icingaredis/telemetry/stats.go b/pkg/icingaredis/telemetry/stats.go index 106e81af6..3a5a4fdb4 100644 --- a/pkg/icingaredis/telemetry/stats.go +++ b/pkg/icingaredis/telemetry/stats.go @@ -17,7 +17,6 @@ var Stats struct { Config com.Counter State com.Counter History com.Counter - Callback com.Counter Overdue com.Counter HistoryCleanup com.Counter NotificationSync com.Counter @@ -29,7 +28,6 @@ func WriteStats(ctx context.Context, client *redis.Client, logger *logging.Logge "config_sync": &Stats.Config, "state_sync": &Stats.State, "history_sync": &Stats.History, - "callback_sync": &Stats.Callback, "overdue_sync": &Stats.Overdue, "history_cleanup": &Stats.HistoryCleanup, "notification_sync": &Stats.NotificationSync, From 1829c7dda1f1b752af5054c006f75aac811f89f5 Mon Sep 17 00:00:00 2001 From: Alvar Penning Date: Wed, 5 Nov 2025 13:24:27 +0100 Subject: [PATCH 30/38] notifications: Fix flat customvars Next to minor changes, the custom variables are now fetched from customvar_flat, being in their flat format. --- pkg/notifications/fetch.go | 38 +++++++++++++++++++++++++------------- 1 file changed, 25 insertions(+), 13 deletions(-) diff --git a/pkg/notifications/fetch.go b/pkg/notifications/fetch.go index 0c2b81c9b..58b1a0d7a 100644 --- a/pkg/notifications/fetch.go +++ b/pkg/notifications/fetch.go @@ -22,7 +22,7 @@ func (client *Client) fetchHostServiceFromRedis( key := "icinga:" + typ var data string - if err := retry.WithBackoff( + err := retry.WithBackoff( ctx, func(ctx context.Context) (err error) { data, err = client.redisClient.HGet(ctx, key, id).Result() @@ -31,7 +31,8 @@ func (client *Client) fetchHostServiceFromRedis( retry.Retryable, backoff.DefaultBackoff, retry.Settings{}, - ); err != nil { + ) + if err != nil { return "", errors.Wrapf(err, "redis HGET %q, %q failed", key, id) } @@ -60,24 +61,35 @@ func (client *Client) fetchHostServiceFromRedis( return } -// fetchCustomVarFromSql retrieves custom variables for the hsot and service from SQL. +// customVar is used as an internal representation in Client.fetchCustomVarFromSql. +type customVar struct { + Name string `db:"name"` + Value types.String `db:"value"` +} + +// getValue returns this customvar's value as a string, transforming SQL NULLs to empty strings. +func (cv customVar) getValue() string { + if cv.Value.Valid { + return cv.Value.String + } + return "" +} + +// fetchCustomVarFromSql retrieves custom variables for the host and service from SQL. // // If serviceId is nil, only the host custom vars are fetched. Otherwise, both host and service custom vars are fetched. func (client *Client) fetchCustomVarFromSql( ctx context.Context, hostId, serviceId types.Binary, ) (map[string]string, error) { - type customVar struct { - Name string `db:"name"` - Value string `db:"value"` - } - getCustomVarsFromSql := func(ctx context.Context, typ string, id types.Binary) ([]customVar, error) { stmt, err := client.db.Preparex(client.db.Rebind( - `SELECT customvar.name AS name, customvar.value AS value + `SELECT + customvar_flat.flatname AS name, + customvar_flat.flatvalue AS value FROM ` + typ + `_customvar - LEFT JOIN customvar - ON ` + typ + `_customvar.customvar_id = customvar.id + JOIN customvar_flat + ON ` + typ + `_customvar.customvar_id = customvar_flat.customvar_id WHERE ` + typ + `_customvar.` + typ + `_id = ?`)) if err != nil { return nil, err @@ -99,7 +111,7 @@ func (client *Client) fetchCustomVarFromSql( } for _, hostVar := range hostVars { - customVars["host.vars."+hostVar.Name] = hostVar.Value + customVars["host.vars."+hostVar.Name] = hostVar.getValue() } if serviceId != nil { @@ -109,7 +121,7 @@ func (client *Client) fetchCustomVarFromSql( } for _, serviceVar := range serviceVars { - customVars["service.vars."+serviceVar.Name] = serviceVar.Value + customVars["service.vars."+serviceVar.Name] = serviceVar.getValue() } } From 013aa8d377e5c06b05a192a3b91e5c05f84ac552 Mon Sep 17 00:00:00 2001 From: Alvar Penning Date: Wed, 5 Nov 2025 17:16:14 +0100 Subject: [PATCH 31/38] history.streamSorterSubmissions: Use Pointer Next to some other small cleanups, the streamSorterSubmissions slice type now references pointers. --- pkg/icingadb/history/sorter.go | 47 +++++++++++++---------------- pkg/icingadb/history/sorter_test.go | 6 ++-- 2 files changed, 24 insertions(+), 29 deletions(-) diff --git a/pkg/icingadb/history/sorter.go b/pkg/icingadb/history/sorter.go index 3b1467fbd..d079094f0 100644 --- a/pkg/icingadb/history/sorter.go +++ b/pkg/icingadb/history/sorter.go @@ -64,7 +64,7 @@ func (sub streamSorterSubmission) MarshalLogObject(encoder zapcore.ObjectEncoder } // streamSorterSubmissions implements [heap.Interface] for []streamSorterSubmission. -type streamSorterSubmissions []streamSorterSubmission +type streamSorterSubmissions []*streamSorterSubmission // Len implements [sort.Interface] required by [heap.Interface]. func (subs streamSorterSubmissions) Len() int { return len(subs) } @@ -86,7 +86,7 @@ func (subs streamSorterSubmissions) Less(i, j int) bool { // Push implements [heap.Interface]. func (subs *streamSorterSubmissions) Push(x any) { - sub, ok := x.(streamSorterSubmission) + sub, ok := x.(*streamSorterSubmission) if !ok { panic(fmt.Sprintf("streamSorterSubmissions.Push received x of %T", x)) } @@ -120,7 +120,7 @@ type StreamSorter struct { ctx context.Context logger *logging.Logger callbackFn func(redis.XMessage, any) bool - submissionCh chan streamSorterSubmission + submissionCh chan *streamSorterSubmission closeChSubmission chan chan<- redis.XMessage closeChQueue chan chan<- redis.XMessage @@ -138,7 +138,7 @@ func NewStreamSorter( ctx: ctx, logger: logger, callbackFn: callbackFn, - submissionCh: make(chan streamSorterSubmission), + submissionCh: make(chan *streamSorterSubmission), closeChSubmission: make(chan chan<- redis.XMessage), closeChQueue: make(chan chan<- redis.XMessage), } @@ -149,7 +149,7 @@ func NewStreamSorter( close(sorter.closeChQueue) }) - ch := make(chan streamSorterSubmission) + ch := make(chan *streamSorterSubmission) go sorter.submissionWorker(ch) go sorter.queueWorker(ch) @@ -158,7 +158,7 @@ func NewStreamSorter( // submissionWorker listens ton submissionCh populated by Submit, fills the heap and ejects streamSorterSubmissions into // out, linked to the queueWorker goroutine for further processing. -func (sorter *StreamSorter) submissionWorker(out chan<- streamSorterSubmission) { +func (sorter *StreamSorter) submissionWorker(out chan<- *streamSorterSubmission) { defer close(out) // When a streamSorterSubmission is created in the Submit method, the current time.Time is added to the struct. @@ -194,7 +194,7 @@ func (sorter *StreamSorter) submissionWorker(out chan<- streamSorterSubmission) bkp := &streamSorterSubmissions{} for submissionHeap.Len() > 0 { x := heap.Pop(submissionHeap) - sub, ok := x.(streamSorterSubmission) + sub, ok := x.(*streamSorterSubmission) if !ok { panic(fmt.Sprintf("invalid type %T from submission heap", x)) } @@ -212,24 +212,22 @@ func (sorter *StreamSorter) submissionWorker(out chan<- streamSorterSubmission) submissionCounter := 0 for submissionHeap.Len() > 0 { - x := heap.Pop(submissionHeap) - sub, ok := x.(streamSorterSubmission) - if !ok { - panic(fmt.Sprintf("invalid type %T from submission heap", x)) - } - - if time.Since(sub.submitTime) < submissionMinAge { + if peek := (*submissionHeap)[0]; time.Since(peek.submitTime) < submissionMinAge { if sorter.verbose { sorter.logger.Debugw("Stopped popping heap as submission is not old enough", - zap.Object("submission", sub), + zap.Object("submission", peek), zap.Int("submissions", submissionCounter), zap.Duration("duration", time.Since(start))) } - - heap.Push(submissionHeap, sub) break } + x := heap.Pop(submissionHeap) + sub, ok := x.(*streamSorterSubmission) + if !ok { + panic(fmt.Sprintf("invalid type %T from submission heap", x)) + } + out <- sub submissionCounter++ } @@ -244,13 +242,10 @@ func (sorter *StreamSorter) submissionWorker(out chan<- streamSorterSubmission) } // queueWorker receives sorted streamSorterSubmissions from submissionWorker and forwards them to the callback. -func (sorter *StreamSorter) queueWorker(in <-chan streamSorterSubmission) { +func (sorter *StreamSorter) queueWorker(in <-chan *streamSorterSubmission) { // Each streamSorterSubmission received from "in" is stored in the queue slice. From there on, the slice head is - // passed to the callback function. The queueEventCh has a buffer capacity of 1 to allow both filling and consuming - // in the same goroutine. - queue := make([]streamSorterSubmission, 0, 1024) - queueEventCh := make(chan struct{}, 1) - defer close(queueEventCh) + // passed to the callback function. + queue := make([]*streamSorterSubmission, 0, 1024) // The actual callback function is executed concurrently as it might block longer than expected. A blocking select // would result in the queue not being populated, effectively blocking the submissionWorker. Thus, the callbackFn is @@ -259,7 +254,7 @@ func (sorter *StreamSorter) queueWorker(in <-chan streamSorterSubmission) { const callbackMaxDelay = 10 * time.Second var callbackDelay time.Duration var callbackCh chan bool - callbackFn := func(submission streamSorterSubmission) { + callbackFn := func(submission *streamSorterSubmission) { select { case <-sorter.ctx.Done(): return @@ -321,7 +316,7 @@ func (sorter *StreamSorter) queueWorker(in <-chan streamSorterSubmission) { return } - queue = slices.DeleteFunc(queue, func(sub streamSorterSubmission) bool { + queue = slices.DeleteFunc(queue, func(sub *streamSorterSubmission) bool { return sub.out == ch }) @@ -354,7 +349,7 @@ func (sorter *StreamSorter) Submit(msg redis.XMessage, args any, out chan<- redi return errors.Wrap(err, "cannot parse Redis Stream ID") } - submission := streamSorterSubmission{ + submission := &streamSorterSubmission{ msg: msg, args: args, out: out, diff --git a/pkg/icingadb/history/sorter_test.go b/pkg/icingadb/history/sorter_test.go index a3ab7a31d..b6c6fd6b2 100644 --- a/pkg/icingadb/history/sorter_test.go +++ b/pkg/icingadb/history/sorter_test.go @@ -90,7 +90,7 @@ func Test_streamSorterSubmissions(t *testing.T) { mkSubmitTime := func(offset int) time.Time { return time.Date(2009, 11, 10, 23, 0, 0, offset, time.UTC) } - submissions := []streamSorterSubmission{ + submissions := streamSorterSubmissions{ {streamIdMs: 0, streamIdSeq: 0, submitTime: mkSubmitTime(0)}, {streamIdMs: 1, streamIdSeq: 0, submitTime: mkSubmitTime(0)}, {streamIdMs: 1, streamIdSeq: 1, submitTime: mkSubmitTime(0)}, @@ -102,12 +102,12 @@ func Test_streamSorterSubmissions(t *testing.T) { {streamIdMs: 3, streamIdSeq: 1, submitTime: mkSubmitTime(2)}, } - submissionsRand := make([]streamSorterSubmission, 0, len(submissions)) + submissionsRand := make(streamSorterSubmissions, 0, len(submissions)) for _, i := range rand.Perm(len(submissions)) { submissionsRand = append(submissionsRand, submissions[i]) } - sort.Sort(streamSorterSubmissions(submissionsRand)) + sort.Sort(submissionsRand) require.Equal(t, submissions, submissionsRand) } From db436f5064cacd22a2ace83007a9f867ec2c82d7 Mon Sep 17 00:00:00 2001 From: Julian Brost Date: Thu, 6 Nov 2025 09:40:25 +0100 Subject: [PATCH 32/38] StreamSorter: improve output channel close behavior and simplify implementation This commit is pretty much an overhaul of the implementation to allow for a more straight-forward way to close the output channel. The main changes to the implementation are: - StreamSorter now provides a method PipelineFunc that can directly be used in a history sync pipeline. This allows StreamSorter to handle the in + out stream pair internally, so that it closes out after in was closed and all messages from it were passed to out. - The two worker goroutines were combined into a single one and the secondary queue was removed. All pending messages remain in the heap and will only be removed from the heap when they are about to be passed to the callback. - The worker now handles all operations (send and close) on the output stream. --- pkg/icingadb/history/sorter.go | 407 ++++++++++++++-------------- pkg/icingadb/history/sorter_test.go | 17 +- pkg/icingadb/history/sync.go | 39 +-- 3 files changed, 216 insertions(+), 247 deletions(-) diff --git a/pkg/icingadb/history/sorter.go b/pkg/icingadb/history/sorter.go index d079094f0..ba81954d3 100644 --- a/pkg/icingadb/history/sorter.go +++ b/pkg/icingadb/history/sorter.go @@ -9,7 +9,6 @@ import ( "github.com/pkg/errors" "go.uber.org/zap" "go.uber.org/zap/zapcore" - "slices" "strconv" "strings" "time" @@ -44,9 +43,9 @@ func parseRedisStreamId(redisStreamId string) (int64, int64, error) { // are not precise enough. type streamSorterSubmission struct { // msg is the Redis message to be forwarded to out after this submission was sorted. - msg redis.XMessage - args any - out chan<- redis.XMessage + msg redis.XMessage + key string + out chan<- redis.XMessage // Required for sorting. streamIdMs int64 // streamIdMs is the Redis Stream ID timestamp part (milliseconds) @@ -55,10 +54,11 @@ type streamSorterSubmission struct { } // MarshalLogObject implements [zapcore.ObjectMarshaler]. -func (sub streamSorterSubmission) MarshalLogObject(encoder zapcore.ObjectEncoder) error { +func (sub *streamSorterSubmission) MarshalLogObject(encoder zapcore.ObjectEncoder) error { encoder.AddInt64("redis-id-ms", sub.streamIdMs) encoder.AddInt64("redis-id-seq", sub.streamIdSeq) encoder.AddTime("submit-time", sub.submitTime) + encoder.AddString("out", fmt.Sprint(sub.out)) return nil } @@ -103,26 +103,37 @@ func (subs *streamSorterSubmissions) Pop() any { return x } -// StreamSorter accepts multiple [redis.XMessage] via Submit and ejects them in an ordered fashion. -// -// Internally, two goroutines are used. The first one collects the submissions and sorts them into a heap based on the -// Redis Stream ID. After being in the heap for at least three seconds, a submission is forwarded to the other -// goroutine. There, each element is passed to the callback function in order. Only if the callback function has -// succeeded, it is removed from the top of the queue. +// Peek returns the smallest element from the heap without removing it, or nil if the heap is empty. +func (subs streamSorterSubmissions) Peek() *streamSorterSubmission { + if len(subs) > 0 { + return subs[0] + } else { + return nil + } +} + +// StreamSorter is a helper that can used to intercept messages from different history sync pipelines and passes them +// to a callback in the order given by their Redis Stream ID (sorted across all involved streams). // -// Thus, if a message is received delayed for more than three seconds, it will be relayed out of order. The StreamSorter -// is only able to ensure order to a certain degree of chaos. +// After a message is received, it is kept in a priority queue for three seconds to wait for possible messages from +// another stream with a smaller ID. Thus, if a message is received delayed for more than three seconds, it will be +// relayed out of order. The StreamSorter is only able to ensure order to a certain degree of chaos. // -// The callback function receives the [redis.XMessage] together with generic args passed in Submit for additional -// context. If the callback function returns true, the element will be removed from the queue. Otherwise, the element -// will be kept at top of the queue and retried next time. +// The callback function receives the [redis.XMessage] together with the Redis stream name (key) for additional +// context. The callback function is supposed to return true on success. Otherwise, the callback will be retried until +// it succeeds. type StreamSorter struct { - ctx context.Context - logger *logging.Logger - callbackFn func(redis.XMessage, any) bool - submissionCh chan *streamSorterSubmission - closeChSubmission chan chan<- redis.XMessage - closeChQueue chan chan<- redis.XMessage + ctx context.Context + logger *logging.Logger + callbackFn func(redis.XMessage, string) bool + submissionCh chan *streamSorterSubmission + + // registerOutCh is used by PipelineFunc() to register output channels with worker() + registerOutCh chan chan<- redis.XMessage + + // closeOutCh is used by PipelineFunc() to signal to worker() that there will be no more submissions destined for + // that output channel and it can be closed by the worker after it processed all pending submissions for it. + closeOutCh chan chan<- redis.XMessage // verbose implies a verbose debug logging. Don't think one want to have this outside the tests. verbose bool @@ -132,218 +143,170 @@ type StreamSorter struct { func NewStreamSorter( ctx context.Context, logger *logging.Logger, - callbackFn func(msg redis.XMessage, args any) bool, + callbackFn func(msg redis.XMessage, key string) bool, ) *StreamSorter { sorter := &StreamSorter{ - ctx: ctx, - logger: logger, - callbackFn: callbackFn, - submissionCh: make(chan *streamSorterSubmission), - closeChSubmission: make(chan chan<- redis.XMessage), - closeChQueue: make(chan chan<- redis.XMessage), + ctx: ctx, + logger: logger, + callbackFn: callbackFn, + submissionCh: make(chan *streamSorterSubmission), + registerOutCh: make(chan chan<- redis.XMessage), + closeOutCh: make(chan chan<- redis.XMessage), } - _ = context.AfterFunc(ctx, func() { - close(sorter.submissionCh) - close(sorter.closeChSubmission) - close(sorter.closeChQueue) - }) - - ch := make(chan *streamSorterSubmission) - go sorter.submissionWorker(ch) - go sorter.queueWorker(ch) + go sorter.worker() return sorter } -// submissionWorker listens ton submissionCh populated by Submit, fills the heap and ejects streamSorterSubmissions into -// out, linked to the queueWorker goroutine for further processing. -func (sorter *StreamSorter) submissionWorker(out chan<- *streamSorterSubmission) { - defer close(out) +// startCallback initiates the callback in a background goroutine and returns a channel that is closed once the callback +// has succeeded. It retries the callback with a backoff until it signal success by returning true. +func (sorter *StreamSorter) startCallback(msg redis.XMessage, key string) <-chan struct{} { + callbackCh := make(chan struct{}) - // When a streamSorterSubmission is created in the Submit method, the current time.Time is added to the struct. - // Only if the submission was at least three seconds (submissionMinAge) ago, a popped submission from the heap will - // be forwarded to the other goroutine for future processing. - const submissionMinAge = 3 * time.Second - submissionHeap := &streamSorterSubmissions{} + go func() { + defer close(callbackCh) - ticker := time.NewTicker(time.Second) - defer ticker.Stop() + const callbackMaxDelay = 10 * time.Second + callbackDelay := time.Duration(0) - for { - select { - case <-sorter.ctx.Done(): - return - - case sub, ok := <-sorter.submissionCh: - if !ok { + for { + select { + case <-sorter.ctx.Done(): return + case <-time.After(callbackDelay): } + start := time.Now() + success := sorter.callbackFn(msg, key) + if sorter.verbose { - sorter.logger.Debugw("Push submission to heap", zap.Object("submission", sub)) + sorter.logger.Debugw("Callback finished", + zap.String("id", msg.ID), + zap.Bool("success", success), + zap.Duration("duration", time.Since(start)), + zap.Duration("next-delay", callbackDelay)) } - heap.Push(submissionHeap, sub) - - case ch, ok := <-sorter.closeChSubmission: - if !ok { + if success { return + } else { + callbackDelay = min(2*max(time.Millisecond, callbackDelay), callbackMaxDelay) } - bkp := &streamSorterSubmissions{} - for submissionHeap.Len() > 0 { - x := heap.Pop(submissionHeap) - sub, ok := x.(*streamSorterSubmission) - if !ok { - panic(fmt.Sprintf("invalid type %T from submission heap", x)) - } - - if sub.out == ch { - continue - } - - bkp.Push(sub) - } - submissionHeap = bkp - - case <-ticker.C: - start := time.Now() - submissionCounter := 0 - - for submissionHeap.Len() > 0 { - if peek := (*submissionHeap)[0]; time.Since(peek.submitTime) < submissionMinAge { - if sorter.verbose { - sorter.logger.Debugw("Stopped popping heap as submission is not old enough", - zap.Object("submission", peek), - zap.Int("submissions", submissionCounter), - zap.Duration("duration", time.Since(start))) - } - break - } - - x := heap.Pop(submissionHeap) - sub, ok := x.(*streamSorterSubmission) - if !ok { - panic(fmt.Sprintf("invalid type %T from submission heap", x)) - } - - out <- sub - submissionCounter++ - } - - if sorter.verbose && submissionCounter > 0 { - sorter.logger.Debugw("Ejected submissions to callback worker", - zap.Int("submissions", submissionCounter), - zap.Duration("duration", time.Since(start))) - } } - } + }() + + return callbackCh } -// queueWorker receives sorted streamSorterSubmissions from submissionWorker and forwards them to the callback. -func (sorter *StreamSorter) queueWorker(in <-chan *streamSorterSubmission) { - // Each streamSorterSubmission received from "in" is stored in the queue slice. From there on, the slice head is - // passed to the callback function. - queue := make([]*streamSorterSubmission, 0, 1024) - - // The actual callback function is executed concurrently as it might block longer than expected. A blocking select - // would result in the queue not being populated, effectively blocking the submissionWorker. Thus, the callbackFn is - // started in a goroutine, signaling back its success status via callbackCh. If no callback is active, the channel - // is nil. Furthermore, an exponential backoff for sequentially failing callbacks is in place. - const callbackMaxDelay = 10 * time.Second - var callbackDelay time.Duration - var callbackCh chan bool - callbackFn := func(submission *streamSorterSubmission) { - select { - case <-sorter.ctx.Done(): - return - case <-time.After(callbackDelay): - } +// worker +func (sorter *StreamSorter) worker() { + // When a streamSorterSubmission is created in the submit method, the current time.Time is added to the struct. + // Only if the submission was at least three seconds (submissionMinAge) ago, a popped submission from the heap will + // be forwarded to the other goroutine for future processing. + const submissionMinAge = 3 * time.Second + var submissionHeap streamSorterSubmissions - start := time.Now() - success := sorter.callbackFn(submission.msg, submission.args) - if success { - defer func() { - // Ensure not to panic if the out channel was closed via CloseOutput in the meantime. - if r := recover(); r != nil { - sorter.logger.Error("Recovered from sending submission", zap.Any("recovery", r)) - } - }() + type OutputState struct { + pending int + close bool + } - submission.out <- submission.msg - callbackDelay = 0 - } else { - callbackDelay = min(2*max(time.Millisecond, callbackDelay), callbackMaxDelay) - } + registeredOutputs := make(map[chan<- redis.XMessage]*OutputState) - if sorter.verbose { - sorter.logger.Debugw("Callback finished", - zap.String("id", submission.msg.ID), - zap.Bool("success", success), - zap.Duration("duration", time.Since(start)), - zap.Duration("next-delay", callbackDelay)) + // Close all registered outputs when we exit. + defer func() { + for out := range registeredOutputs { + close(out) } + }() - callbackCh <- success - } + var runningSubmission *streamSorterSubmission + var runningCallbackCh <-chan struct{} for { - if len(queue) > 0 && callbackCh == nil { - callbackCh = make(chan bool) - go callbackFn(queue[0]) + // Sanity check + if (runningSubmission == nil) != (runningCallbackCh == nil) { + panic(fmt.Sprintf("inconsistent state: runningSubmission=%#v and runningCallbackCh=%#v", runningSubmission, runningCallbackCh)) } - select { - case <-sorter.ctx.Done(): - return - - case sub, ok := <-in: - if !ok { - return + var nextSubmissionDue <-chan time.Time + if runningCallbackCh == nil { + if next := submissionHeap.Peek(); next != nil { + if submissionAge := time.Since(next.submitTime); submissionAge >= submissionMinAge { + runningCallbackCh = sorter.startCallback(next.msg, next.key) + runningSubmission = next + heap.Pop(&submissionHeap) + } else { + nextSubmissionDue = time.After(submissionMinAge - submissionAge) + } } + } - queue = append(queue, sub) + select { + case out := <-sorter.registerOutCh: + if sorter.verbose { + sorter.logger.Debugw("worker: register output", zap.String("out", fmt.Sprint(out))) + } + if _, ok := registeredOutputs[out]; ok { + panic("attempting to register the same output channel twice") + } + registeredOutputs[out] = &OutputState{} + // This function is now responsible for closing out. + case out := <-sorter.closeOutCh: if sorter.verbose { - sorter.logger.Debugw("Queue worker received new submission", - zap.Object("submission", sub), - zap.Int("queue-size", len(queue))) + sorter.logger.Debugw("worker: request close output", zap.String("out", fmt.Sprint(out))) + } + if state := registeredOutputs[out]; state == nil { + panic("requested to close unknown output channel") + } else if state.pending > 0 { + // Still pending work, mark the output and wait for it to complete. + state.close = true + } else { + // Output can be closed and unregistered immediately + close(out) + delete(registeredOutputs, out) } - case ch, ok := <-sorter.closeChQueue: - if !ok { - return + case sub := <-sorter.submissionCh: + if sorter.verbose { + sorter.logger.Debugw("Push submission to heap", zap.Object("submission", sub)) } - queue = slices.DeleteFunc(queue, func(sub *streamSorterSubmission) bool { - return sub.out == ch - }) + if state := registeredOutputs[sub.out]; state == nil { + panic("submission for an unknown output channel") + } else { + state.pending++ + heap.Push(&submissionHeap, sub) + } - case success := <-callbackCh: - // The len(queue) part is necessary as sorter.closeChQueue might interfere. - if success && len(queue) > 0 { - queue = queue[1:] + case <-nextSubmissionDue: + // Loop start processing of the next submission. + continue + + case <-runningCallbackCh: + out := runningSubmission.out + out <- runningSubmission.msg + state := registeredOutputs[out] + state.pending-- + if state.close && state.pending == 0 { + close(out) + delete(registeredOutputs, out) } - close(callbackCh) - callbackCh = nil + runningCallbackCh = nil + runningSubmission = nil - if sorter.verbose && len(queue) == 0 { - sorter.logger.Debug("Queue worker finished processing queue") - } + case <-sorter.ctx.Done(): + return } } } -// Submit a [redis.XMessage] to the StreamSorter. -// -// After the message was sorted and successfully passed to the callback including the optional args, it will be -// forwarded to the out channel. -// -// This method returns an error for malformed Redis Stream IDs or if the internal submission channel blocks for over a -// second. Usually, this both should not happen. -func (sorter *StreamSorter) Submit(msg redis.XMessage, args any, out chan<- redis.XMessage) error { +// submit a [redis.XMessage] to the StreamSorter. +func (sorter *StreamSorter) submit(msg redis.XMessage, key string, out chan<- redis.XMessage) error { ms, seq, err := parseRedisStreamId(msg.ID) if err != nil { return errors.Wrap(err, "cannot parse Redis Stream ID") @@ -351,7 +314,7 @@ func (sorter *StreamSorter) Submit(msg redis.XMessage, args any, out chan<- redi submission := &streamSorterSubmission{ msg: msg, - args: args, + key: key, out: out, streamIdMs: ms, streamIdSeq: seq, @@ -370,28 +333,64 @@ func (sorter *StreamSorter) Submit(msg redis.XMessage, args any, out chan<- redi } } -// CloseOutput clears all submissions targeting this output channel and closes the channel afterwards. -// -// This will only result in submissions with this out channel to be removed from both the submissionWorker's heap and -// the queueWorker's queue. In case such a submission is already in the actual submission process, it might still be -// tried, but sending it to the out channel is recovered internally. +// PipelineFunc implements the interface expected for a history sync pipeline stage. // -// As filtering/recreating the caches is potentially expensive, only call this method if required. In the current -// architecture of sync.go, this is fine. -func (sorter *StreamSorter) CloseOutput(out chan<- redis.XMessage) error { - for _, ch := range []chan chan<- redis.XMessage{sorter.closeChSubmission, sorter.closeChQueue} { +// This method of a single StreamSorter can be inserted into multiple history sync pipelines and will forward all +// messages from in to out as expected from a pipeline stage. In between, all messages are processed by the +// StreamSorter, which correlates the messages from different pipelines and additionally passes them to a callback +// according to its specification (see the comment on the StreamSorter type). +func (sorter *StreamSorter) PipelineFunc( + ctx context.Context, + s Sync, + key string, + in <-chan redis.XMessage, + out chan<- redis.XMessage, +) error { + + // Register output channel with worker. + select { + case sorter.registerOutCh <- out: + // Success, worker is now responsible for closing the channel. + + case <-ctx.Done(): + close(out) + return ctx.Err() + + case <-sorter.ctx.Done(): + close(out) + return sorter.ctx.Err() + } + + // If we exit, signal to the worker that no more work for this channel will be submitted. + defer func() { select { + case sorter.closeOutCh <- out: + // Success, worker will close the output channel eventually. + case <-sorter.ctx.Done(): - return sorter.ctx.Err() + // Worker will quit entirely, closing all output channels. + } + }() - case ch <- out: + for { + select { + case msg, ok := <-in: + if !ok { + return nil + } - case <-time.After(time.Second): - return errors.New("sending to channel for closing timed out") - } - } + err := sorter.submit(msg, key, out) + if err != nil { + s.logger.Errorw("Failed to submit Redis stream event to stream sorter", + zap.String("key", key), + zap.Error(err)) + } - close(out) + case <-ctx.Done(): + return ctx.Err() - return nil + case <-sorter.ctx.Done(): + return sorter.ctx.Err() + } + } } diff --git a/pkg/icingadb/history/sorter_test.go b/pkg/icingadb/history/sorter_test.go index b6c6fd6b2..f14399101 100644 --- a/pkg/icingadb/history/sorter_test.go +++ b/pkg/icingadb/history/sorter_test.go @@ -3,6 +3,7 @@ package history import ( "cmp" + "context" "fmt" "github.com/icinga/icinga-go-library/logging" "github.com/icinga/icinga-go-library/redis" @@ -192,7 +193,6 @@ func TestStreamSorter(t *testing.T) { producersEarlyClose: 5, callbackMaxDelayMs: 1000, callbackSuccessPercent: 100, - expectTimeout: true, }, { name: "pure chaos", @@ -211,7 +211,7 @@ func TestStreamSorter(t *testing.T) { var ( callbackCollection []string callbackCollectionMutex sync.Mutex - callbackFn = func(msg redis.XMessage, _ any) bool { + callbackFn = func(msg redis.XMessage, _ string) bool { if tt.callbackMaxDelayMs > 0 { time.Sleep(time.Duration(rand.Int63n(int64(tt.callbackMaxDelayMs))) * time.Millisecond) } @@ -260,9 +260,14 @@ func TestStreamSorter(t *testing.T) { for i := range tt.producers { earlyClose := i < tt.producersEarlyClose + in := make(chan redis.XMessage) out := make(chan redis.XMessage) + go func() { + require.NoError(t, sorter.PipelineFunc(context.Background(), Sync{}, "", in, out)) + }() + if !earlyClose { - defer func() { _ = sorter.CloseOutput(out) }() // no leakage, general cleanup + defer close(in) // no leakage, general cleanup } go func() { @@ -295,12 +300,12 @@ func TestStreamSorter(t *testing.T) { } msg := redis.XMessage{ID: fmt.Sprintf("%d-%d", ms, seq)} - require.NoError(t, sorter.Submit(msg, nil, out)) + in <- msg // 25% chance of closing for early closing producers if earlyClose && rand.Int63n(4) == 3 { - require.NoError(t, sorter.CloseOutput(out)) - t.Log("Successfully closed producer early") + close(in) + t.Log("closed producer early") return } } diff --git a/pkg/icingadb/history/sync.go b/pkg/icingadb/history/sync.go index 1b248b587..7d7f7dd1a 100644 --- a/pkg/icingadb/history/sync.go +++ b/pkg/icingadb/history/sync.go @@ -449,7 +449,7 @@ func makeSortedCallbackStageFunc( keyStructPtrs map[string]any, fn func(database.Entity) bool, ) stageFunc { - sorterCallbackFn := func(msg redis.XMessage, args any) bool { + sorterCallbackFn := func(msg redis.XMessage, key string) bool { makeEntity := func(key string, values map[string]interface{}) (database.Entity, error) { structPtr, ok := keyStructPtrs[key] if !ok { @@ -473,12 +473,6 @@ func makeSortedCallbackStageFunc( return entity, nil } - key, ok := args.(string) - if !ok { - // Shall not happen; set to string some thirty lines below - panic(fmt.Sprintf("args is of type %T, not string", args)) - } - entity, err := makeEntity(key, msg.Values) if err != nil { logger.Errorw("Failed to create database.Entity out of Redis stream message", @@ -495,36 +489,7 @@ func makeSortedCallbackStageFunc( return success } - sorter := NewStreamSorter(ctx, logger, sorterCallbackFn) - - return func(ctx context.Context, s Sync, key string, in <-chan redis.XMessage, out chan<- redis.XMessage) error { - defer func() { - if err := sorter.CloseOutput(out); err != nil { - s.logger.Errorw("Closing stream sorter output failed", - zap.String("key", key), - zap.Error(err)) - } - }() - - for { - select { - case msg, ok := <-in: - if !ok { - return nil - } - - err := sorter.Submit(msg, key, out) - if err != nil { - s.logger.Errorw("Failed to submit Redis stream event to stream sorter", - zap.String("key", key), - zap.Error(err)) - } - - case <-ctx.Done(): - return ctx.Err() - } - } - } + return NewStreamSorter(ctx, logger, sorterCallbackFn).PipelineFunc } const ( From 33cb78d0824d5a97a54631ec110069a71666a094 Mon Sep 17 00:00:00 2001 From: Alvar Penning Date: Thu, 6 Nov 2025 17:41:06 +0100 Subject: [PATCH 33/38] history.StreamSorter: Few comments, No Data Races After Julian reworked big parts of the StreamSorter for the better, I went over the code multiple times and added a few comments for parts I had to read twice. Within the tests, there might be a data race when zaptest is used after the test's context is done. Since there were a few log messages potentially occurring after the test's end, a guard was added to ensure no verbose log messages are being produced if the context is done. --- pkg/icingadb/history/sorter.go | 70 +++++++++++++++++------------ pkg/icingadb/history/sorter_test.go | 2 +- 2 files changed, 43 insertions(+), 29 deletions(-) diff --git a/pkg/icingadb/history/sorter.go b/pkg/icingadb/history/sorter.go index ba81954d3..cecf4c1ac 100644 --- a/pkg/icingadb/history/sorter.go +++ b/pkg/icingadb/history/sorter.go @@ -135,8 +135,8 @@ type StreamSorter struct { // that output channel and it can be closed by the worker after it processed all pending submissions for it. closeOutCh chan chan<- redis.XMessage - // verbose implies a verbose debug logging. Don't think one want to have this outside the tests. - verbose bool + // isVerbose implies a isVerbose debug logging. Don't think one want to have this outside the tests. + isVerbose bool } // NewStreamSorter creates a StreamSorter honoring the given context and returning elements to the callback function. @@ -159,6 +159,22 @@ func NewStreamSorter( return sorter } +// verbose produces a debug log messages if StreamSorter.isVerbose is set. +func (sorter *StreamSorter) verbose(msg string, keysAndValues ...any) { + // When used in tests and the test context is done, using the logger results in a data race. Since there are a few + // log messages which might occur after the test has finished, better not log at all. + // https://github.com/uber-go/zap/issues/687#issuecomment-473382859 + if sorter.ctx.Err() != nil { + return + } + + if !sorter.isVerbose { + return + } + + sorter.logger.Debugw(msg, keysAndValues...) +} + // startCallback initiates the callback in a background goroutine and returns a channel that is closed once the callback // has succeeded. It retries the callback with a backoff until it signal success by returning true. func (sorter *StreamSorter) startCallback(msg redis.XMessage, key string) <-chan struct{} { @@ -170,7 +186,7 @@ func (sorter *StreamSorter) startCallback(msg redis.XMessage, key string) <-chan const callbackMaxDelay = 10 * time.Second callbackDelay := time.Duration(0) - for { + for try := 0; ; try++ { select { case <-sorter.ctx.Done(): return @@ -180,34 +196,36 @@ func (sorter *StreamSorter) startCallback(msg redis.XMessage, key string) <-chan start := time.Now() success := sorter.callbackFn(msg, key) - if sorter.verbose { - sorter.logger.Debugw("Callback finished", - zap.String("id", msg.ID), - zap.Bool("success", success), - zap.Duration("duration", time.Since(start)), - zap.Duration("next-delay", callbackDelay)) - } + sorter.verbose("startCallback: finished executing callbackFn", + zap.String("id", msg.ID), + zap.Bool("success", success), + zap.Int("try", try), + zap.Duration("duration", time.Since(start)), + zap.Duration("next-delay", callbackDelay)) if success { return } else { - callbackDelay = min(2*max(time.Millisecond, callbackDelay), callbackMaxDelay) + callbackDelay = min(max(time.Millisecond, 2*callbackDelay), callbackMaxDelay) } - } }() return callbackCh } -// worker +// worker is the background worker, started in a goroutine from NewStreamSorter, reacts upon messages from the channels, +// and runs until the StreamSorter.ctx is done. func (sorter *StreamSorter) worker() { // When a streamSorterSubmission is created in the submit method, the current time.Time is added to the struct. // Only if the submission was at least three seconds (submissionMinAge) ago, a popped submission from the heap will - // be forwarded to the other goroutine for future processing. + // be passed to startCallback in its own goroutine to execute the callback function. const submissionMinAge = 3 * time.Second var submissionHeap streamSorterSubmissions + // Each registered output is stored in the registeredOutputs map, mapping output channels to the following struct. + // It counts pending submissions in the heap for each received submission from submissionCh and can be marked as + // closed to be cleaned up after its work is done. type OutputState struct { pending int close bool @@ -222,13 +240,15 @@ func (sorter *StreamSorter) worker() { } }() + // If a submission is currently given to the callback via startCallback, these two variables are not nil. After the + // callback has finished, the channel will be closed. var runningSubmission *streamSorterSubmission var runningCallbackCh <-chan struct{} for { - // Sanity check if (runningSubmission == nil) != (runningCallbackCh == nil) { - panic(fmt.Sprintf("inconsistent state: runningSubmission=%#v and runningCallbackCh=%#v", runningSubmission, runningCallbackCh)) + panic(fmt.Sprintf("inconsistent state: runningSubmission=%#v and runningCallbackCh=%#v", + runningSubmission, runningCallbackCh)) } var nextSubmissionDue <-chan time.Time @@ -246,9 +266,7 @@ func (sorter *StreamSorter) worker() { select { case out := <-sorter.registerOutCh: - if sorter.verbose { - sorter.logger.Debugw("worker: register output", zap.String("out", fmt.Sprint(out))) - } + sorter.verbose("worker: register output", zap.String("out", fmt.Sprint(out))) if _, ok := registeredOutputs[out]; ok { panic("attempting to register the same output channel twice") } @@ -256,9 +274,7 @@ func (sorter *StreamSorter) worker() { // This function is now responsible for closing out. case out := <-sorter.closeOutCh: - if sorter.verbose { - sorter.logger.Debugw("worker: request close output", zap.String("out", fmt.Sprint(out))) - } + sorter.verbose("worker: request close output", zap.String("out", fmt.Sprint(out))) if state := registeredOutputs[out]; state == nil { panic("requested to close unknown output channel") } else if state.pending > 0 { @@ -271,9 +287,7 @@ func (sorter *StreamSorter) worker() { } case sub := <-sorter.submissionCh: - if sorter.verbose { - sorter.logger.Debugw("Push submission to heap", zap.Object("submission", sub)) - } + sorter.verbose("worker: push submission to heap", zap.Object("submission", sub)) if state := registeredOutputs[sub.out]; state == nil { panic("submission for an unknown output channel") @@ -322,14 +336,14 @@ func (sorter *StreamSorter) submit(msg redis.XMessage, key string, out chan<- re } select { - case <-sorter.ctx.Done(): - return sorter.ctx.Err() - case sorter.submissionCh <- submission: return nil case <-time.After(time.Second): return errors.New("submission timed out") + + case <-sorter.ctx.Done(): + return sorter.ctx.Err() } } diff --git a/pkg/icingadb/history/sorter_test.go b/pkg/icingadb/history/sorter_test.go index f14399101..6e43124ed 100644 --- a/pkg/icingadb/history/sorter_test.go +++ b/pkg/icingadb/history/sorter_test.go @@ -255,7 +255,7 @@ func TestStreamSorter(t *testing.T) { t.Context(), logging.NewLogger(zaptest.NewLogger(t).Sugar(), time.Second), callbackFn) - sorter.verbose = true + sorter.isVerbose = true for i := range tt.producers { earlyClose := i < tt.producersEarlyClose From 2476251d581945cacab7e205ef3fdb79a482b168 Mon Sep 17 00:00:00 2001 From: Alvar Penning Date: Thu, 6 Nov 2025 17:44:03 +0100 Subject: [PATCH 34/38] notifications.Client: Allow Parameters of any Type The parameters can not only be strings, but anything to PHP's liking. In one example, an integer was observed. Since Parameters is converted to an []any later anyways, this is no real change in behavior. --- pkg/notifications/notifications.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/pkg/notifications/notifications.go b/pkg/notifications/notifications.go index 2d19bbb0c..983eabc83 100644 --- a/pkg/notifications/notifications.go +++ b/pkg/notifications/notifications.go @@ -83,8 +83,8 @@ func (client *Client) evaluateRulesForObject(ctx context.Context, hostId, servic const icingaDbWebRuleVersion = 1 type IcingaDbWebQuery struct { - Query string `json:"query"` - Parameters []string `json:"parameters"` + Query string `json:"query"` + Parameters []any `json:"parameters"` } type IcingaDbWebRule struct { From d769a0c3748ecfc422071af25f11d5333a54182d Mon Sep 17 00:00:00 2001 From: Alvar Penning Date: Fri, 7 Nov 2025 16:43:04 +0100 Subject: [PATCH 35/38] notifications: Import StreamSorter Logic The whole StreamSorter logic is only required for Icinga Notifications. Thus, the implementation was moved from the history package to the notifications package, removing some unnecessary generalizations on the way. This results in big changes to be made in the notifications package, while other modules are mostly not affected. --- cmd/icingadb/main.go | 11 +- pkg/icingadb/history/sync.go | 131 +++--------------- pkg/notifications/notifications.go | 69 ++++++++- .../history => notifications}/sorter.go | 9 +- .../history => notifications}/sorter_test.go | 5 +- 5 files changed, 94 insertions(+), 131 deletions(-) rename pkg/{icingadb/history => notifications}/sorter.go (97%) rename pkg/{icingadb/history => notifications}/sorter_test.go (98%) diff --git a/cmd/icingadb/main.go b/cmd/icingadb/main.go index dd2b5b0a2..f7d43c12e 100644 --- a/cmd/icingadb/main.go +++ b/cmd/icingadb/main.go @@ -170,8 +170,7 @@ func run() int { signal.Notify(sig, os.Interrupt, syscall.SIGTERM, syscall.SIGHUP) { - var callbackCfg *history.SyncCallbackConf - + var extraStages map[string]history.StageFunc if cfg := cmd.Config.NotificationsSource; cfg.ApiBaseUrl != "" { logger.Info("Starting Icinga Notifications source") @@ -185,17 +184,13 @@ func run() int { logger.Fatalw("Can't create Icinga Notifications client from config", zap.Error(err)) } - callbackCfg = &history.SyncCallbackConf{ - StatPtr: &telemetry.Stats.NotificationSync, - KeyStructPtr: notifications.SyncKeyStructPtrs, - Fn: notificationsSource.Submit, - } + extraStages = notificationsSource.SyncExtraStages() } go func() { logger.Info("Starting history sync") - if err := hs.Sync(ctx, callbackCfg); err != nil && !utils.IsContextCanceled(err) { + if err := hs.Sync(ctx, extraStages); err != nil && !utils.IsContextCanceled(err) { logger.Fatalf("%+v", err) } }() diff --git a/pkg/icingadb/history/sync.go b/pkg/icingadb/history/sync.go index 7d7f7dd1a..6c547815c 100644 --- a/pkg/icingadb/history/sync.go +++ b/pkg/icingadb/history/sync.go @@ -2,7 +2,6 @@ package history import ( "context" - "fmt" "github.com/icinga/icinga-go-library/com" "github.com/icinga/icinga-go-library/database" "github.com/icinga/icinga-go-library/logging" @@ -16,7 +15,6 @@ import ( v1 "github.com/icinga/icingadb/pkg/icingadb/v1/history" "github.com/icinga/icingadb/pkg/icingaredis/telemetry" "github.com/pkg/errors" - "go.uber.org/zap" "golang.org/x/sync/errgroup" "reflect" "slices" @@ -30,17 +28,6 @@ type Sync struct { logger *logging.Logger } -// SyncCallbackConf configures a callback stage given to Sync.Sync. -type SyncCallbackConf struct { - // StatPtr refers a [com.Counter] from the [telemetry.Stats] struct, e.g., Stats.NotificationSync. - StatPtr *com.Counter - // KeyStructPtr says which pipeline keys should be mapped to which type, identified by a struct pointer. If - // a key is missing from the map, it will not be used for the callback. - KeyStructPtr map[string]any - // Fn is the actual callback function. - Fn func(database.Entity) bool -} - // NewSync creates a new Sync. func NewSync(db *database.DB, redis *redis.Client, logger *logging.Logger) *Sync { return &Sync{ @@ -52,19 +39,9 @@ func NewSync(db *database.DB, redis *redis.Client, logger *logging.Logger) *Sync // Sync synchronizes Redis history streams from s.redis to s.db and deletes the original data on success. // -// It is possible to enable a callback functionality, e.g., for the Icinga Notifications integration. To do so, the -// callbackCfg must be set according to the SyncCallbackConf struct documentation. -func (s Sync) Sync(ctx context.Context, callbackCfg *SyncCallbackConf) error { - var callbackStageFn stageFunc - if callbackCfg != nil { - callbackStageFn = makeSortedCallbackStageFunc( - ctx, - s.logger, - callbackCfg.StatPtr, - callbackCfg.KeyStructPtr, - callbackCfg.Fn) - } - +// The optional extraStages parameter allows specifying an additional extra stage for each pipeline, identified by their +// key. This stage is executed after every other stage, but before the entry gets deleted from Redis. +func (s Sync) Sync(ctx context.Context, extraStages map[string]StageFunc) error { g, ctx := errgroup.WithContext(ctx) for key, pipeline := range syncPipelines { @@ -90,19 +67,16 @@ func (s Sync) Sync(ctx context.Context, callbackCfg *SyncCallbackConf) error { // forward the entry after it has completed its own sync so that later stages can rely on previous stages being // executed successfully. // - // If a callback exists for this key, it will be appended to the pipeline. Thus, it is executed after every + // If an extra stage exists for this key, it will be appended to the pipeline. Thus, it is executed after every // other pipeline action, but before deleteFromRedis. - var hasCallbackStage bool - if callbackCfg != nil { - _, exists := callbackCfg.KeyStructPtr[key] - hasCallbackStage = exists - } - // Shadowed variable to allow appending custom callbacks. pipeline := pipeline - if hasCallbackStage { - pipeline = append(slices.Clip(pipeline), callbackStageFn) + if extraStages != nil { + extraStage, ok := extraStages[key] + if ok { + pipeline = append(slices.Clip(pipeline), extraStage) + } } ch := make([]chan redis.XMessage, len(pipeline)+1) @@ -200,19 +174,19 @@ func (s Sync) deleteFromRedis(ctx context.Context, key string, input <-chan redi } } -// stageFunc is a function type that represents a sync pipeline stage. It is called with a context (it should stop +// StageFunc is a function type that represents a sync pipeline stage. It is called with a context (it should stop // once that context is canceled), the Sync instance (for access to Redis, SQL database, logging), the key (information // about which pipeline this function is running in, i.e. "notification"), an in channel for the stage to read history // events from and an out channel to forward history entries to after processing them successfully. A stage function // is supposed to forward each message from in to out, even if the event is not relevant for the current stage. On // error conditions, the message must not be forwarded to the next stage so that the event is not deleted from Redis // and can be processed at a later time. -type stageFunc func(ctx context.Context, s Sync, key string, in <-chan redis.XMessage, out chan<- redis.XMessage) error +type StageFunc func(ctx context.Context, s Sync, key string, in <-chan redis.XMessage, out chan<- redis.XMessage) error -// writeOneEntityStage creates a stageFunc from a pointer to a struct implementing the v1.UpserterEntity interface. +// writeOneEntityStage creates a StageFunc from a pointer to a struct implementing the v1.UpserterEntity interface. // For each history event it receives, it parses that event into a new instance of that entity type and writes it to // the database. It writes exactly one entity to the database for each history event. -func writeOneEntityStage(structPtr any) stageFunc { +func writeOneEntityStage(structPtr any) StageFunc { structifier := structify.MakeMapStructifier( reflect.TypeOf(structPtr).Elem(), "json", @@ -231,9 +205,9 @@ func writeOneEntityStage(structPtr any) stageFunc { }) } -// writeMultiEntityStage creates a stageFunc from a function that takes a history event as an input and returns a +// writeMultiEntityStage creates a StageFunc from a function that takes a history event as an input and returns a // (potentially empty) slice of v1.UpserterEntity instances that it then inserts into the database. -func writeMultiEntityStage(entryToEntities func(entry redis.XMessage) ([]v1.UpserterEntity, error)) stageFunc { +func writeMultiEntityStage(entryToEntities func(entry redis.XMessage) ([]v1.UpserterEntity, error)) StageFunc { return func(ctx context.Context, s Sync, key string, in <-chan redis.XMessage, out chan<- redis.XMessage) error { type State struct { Message redis.XMessage // Original event from Redis. @@ -345,7 +319,7 @@ func writeMultiEntityStage(entryToEntities func(entry redis.XMessage) ([]v1.Upse } } -// userNotificationStage is a specialized stageFunc that populates the user_notification_history table. It is executed +// userNotificationStage is a specialized StageFunc that populates the user_notification_history table. It is executed // on the notification history stream and uses the users_notified_ids attribute to create an entry in the // user_notification_history relation table for each user ID. func userNotificationStage(ctx context.Context, s Sync, key string, in <-chan redis.XMessage, out chan<- redis.XMessage) error { @@ -404,9 +378,8 @@ func userNotificationStage(ctx context.Context, s Sync, key string, in <-chan re // countElementStage increments the [Stats.History] counter. // -// This stageFunc should be called last in a [syncPipeline]. Thus, it is still executed before the final -// Sync.deleteFromRedis call in Sync.Sync. Furthermore, an optional callback function will be appended after this stage, -// resulting in an incremented history state counter for synchronized history, but stalling callback actions. +// This StageFunc should be called last in each syncPipeline. Thus, it is executed before the final +// Sync.deleteFromRedis call in Sync.Sync, but before optional extra stages, potentially blocking. func countElementStage(ctx context.Context, _ Sync, _ string, in <-chan redis.XMessage, out chan<- redis.XMessage) error { defer close(out) @@ -426,72 +399,6 @@ func countElementStage(ctx context.Context, _ Sync, _ string, in <-chan redis.XM } } -// makeSortedCallbackStageFunc creates a new stageFunc calling the callback function after reordering messages. -// -// This stageFunc is designed to be used by multiple channels. The internal sorting logic - realized by a StreamSorter - -// results in all messages to be sorted based on their Redis Stream ID and be ejected to the callback function in this -// order. -// -// The keyStructPtrs map decides what kind of database.Entity type will be used for the input data based on the key. -// -// The callback call is blocking and the message will be forwarded to the out channel after the function has returned. -// Thus, please ensure this function does not block too long. -// -// If the callback function returns false, the message will be retried after an increasing backoff. All subsequent -// messages will wait until this one succeeds. -// -// For each successfully submitted message, the telemetry stat referenced via a pointer s incremented. Thus, a delta -// between telemetry.Stats.History and this stat indicates blocking callbacks. -func makeSortedCallbackStageFunc( - ctx context.Context, - logger *logging.Logger, - statPtr *com.Counter, - keyStructPtrs map[string]any, - fn func(database.Entity) bool, -) stageFunc { - sorterCallbackFn := func(msg redis.XMessage, key string) bool { - makeEntity := func(key string, values map[string]interface{}) (database.Entity, error) { - structPtr, ok := keyStructPtrs[key] - if !ok { - return nil, fmt.Errorf("key is not part of keyStructPtrs") - } - - structifier := structify.MakeMapStructifier( - reflect.TypeOf(structPtr).Elem(), - "json", - contracts.SafeInit) - val, err := structifier(values) - if err != nil { - return nil, errors.Wrapf(err, "can't structify values %#v for %q", values, key) - } - - entity, ok := val.(database.Entity) - if !ok { - return nil, fmt.Errorf("structifier returned %T which does not implement database.Entity", val) - } - - return entity, nil - } - - entity, err := makeEntity(key, msg.Values) - if err != nil { - logger.Errorw("Failed to create database.Entity out of Redis stream message", - zap.Error(err), - zap.String("key", key), - zap.String("id", msg.ID)) - return false - } - - success := fn(entity) - if success { - statPtr.Add(1) - } - return success - } - - return NewStreamSorter(ctx, logger, sorterCallbackFn).PipelineFunc -} - const ( SyncPipelineAcknowledgement = "acknowledgement" SyncPipelineComment = "comment" @@ -501,7 +408,7 @@ const ( SyncPipelineState = "state" ) -var syncPipelines = map[string][]stageFunc{ +var syncPipelines = map[string][]StageFunc{ SyncPipelineAcknowledgement: { writeOneEntityStage((*v1.AcknowledgementHistory)(nil)), // acknowledgement_history writeOneEntityStage((*v1.HistoryAck)(nil)), // history (depends on acknowledgement_history) diff --git a/pkg/notifications/notifications.go b/pkg/notifications/notifications.go index 983eabc83..288a086d8 100644 --- a/pkg/notifications/notifications.go +++ b/pkg/notifications/notifications.go @@ -9,15 +9,19 @@ import ( "github.com/icinga/icinga-go-library/notifications/event" "github.com/icinga/icinga-go-library/notifications/source" "github.com/icinga/icinga-go-library/redis" + "github.com/icinga/icinga-go-library/structify" "github.com/icinga/icinga-go-library/types" "github.com/icinga/icinga-go-library/utils" "github.com/icinga/icingadb/internal" "github.com/icinga/icingadb/pkg/common" + "github.com/icinga/icingadb/pkg/contracts" "github.com/icinga/icingadb/pkg/icingadb/history" v1history "github.com/icinga/icingadb/pkg/icingadb/v1/history" + "github.com/icinga/icingadb/pkg/icingaredis/telemetry" "github.com/pkg/errors" "go.uber.org/zap" "go.uber.org/zap/zapcore" + "reflect" "sync" ) @@ -463,9 +467,64 @@ func (client *Client) Submit(entity database.Entity) bool { return false } -var SyncKeyStructPtrs = map[string]any{ - history.SyncPipelineAcknowledgement: (*v1history.AcknowledgementHistory)(nil), - history.SyncPipelineDowntime: (*v1history.DowntimeHistoryMeta)(nil), - history.SyncPipelineFlapping: (*v1history.FlappingHistory)(nil), - history.SyncPipelineState: (*v1history.StateHistory)(nil), +// SyncExtraStages returns a map of history sync keys to [history.StageFunc] to be used for [history.Sync]. +// +// Passing the return value of this method as the extraStages parameter to [history.Sync] results in forwarding events +// from the Icinga DB history stream to Icinga Notifications after being resorted via the StreamSorter. +func (client *Client) SyncExtraStages() map[string]history.StageFunc { + var syncKeyStructPtrs = map[string]any{ + history.SyncPipelineAcknowledgement: (*v1history.AcknowledgementHistory)(nil), + history.SyncPipelineDowntime: (*v1history.DowntimeHistoryMeta)(nil), + history.SyncPipelineFlapping: (*v1history.FlappingHistory)(nil), + history.SyncPipelineState: (*v1history.StateHistory)(nil), + } + + sorterCallbackFn := func(msg redis.XMessage, key string) bool { + makeEntity := func(key string, values map[string]interface{}) (database.Entity, error) { + structPtr, ok := syncKeyStructPtrs[key] + if !ok { + return nil, fmt.Errorf("key is not part of keyStructPtrs") + } + + structifier := structify.MakeMapStructifier( + reflect.TypeOf(structPtr).Elem(), + "json", + contracts.SafeInit) + val, err := structifier(values) + if err != nil { + return nil, errors.Wrapf(err, "can't structify values %#v for %q", values, key) + } + + entity, ok := val.(database.Entity) + if !ok { + return nil, fmt.Errorf("structifier returned %T which does not implement database.Entity", val) + } + + return entity, nil + } + + entity, err := makeEntity(key, msg.Values) + if err != nil { + client.logger.Errorw("Failed to create database.Entity out of Redis stream message", + zap.Error(err), + zap.String("key", key), + zap.String("id", msg.ID)) + return false + } + + success := client.Submit(entity) + if success { + telemetry.Stats.NotificationSync.Add(1) + } + return success + } + + pipelineFn := NewStreamSorter(client.ctx, client.logger, sorterCallbackFn).PipelineFunc + + extraStages := make(map[string]history.StageFunc) + for k := range syncKeyStructPtrs { + extraStages[k] = pipelineFn + } + + return extraStages } diff --git a/pkg/icingadb/history/sorter.go b/pkg/notifications/sorter.go similarity index 97% rename from pkg/icingadb/history/sorter.go rename to pkg/notifications/sorter.go index cecf4c1ac..51235e790 100644 --- a/pkg/icingadb/history/sorter.go +++ b/pkg/notifications/sorter.go @@ -1,4 +1,4 @@ -package history +package notifications import ( "container/heap" @@ -6,6 +6,7 @@ import ( "fmt" "github.com/icinga/icinga-go-library/logging" "github.com/icinga/icinga-go-library/redis" + "github.com/icinga/icingadb/pkg/icingadb/history" "github.com/pkg/errors" "go.uber.org/zap" "go.uber.org/zap/zapcore" @@ -347,7 +348,7 @@ func (sorter *StreamSorter) submit(msg redis.XMessage, key string, out chan<- re } } -// PipelineFunc implements the interface expected for a history sync pipeline stage. +// PipelineFunc implements the [history.StageFunc] type expected for a history sync pipeline stage. // // This method of a single StreamSorter can be inserted into multiple history sync pipelines and will forward all // messages from in to out as expected from a pipeline stage. In between, all messages are processed by the @@ -355,7 +356,7 @@ func (sorter *StreamSorter) submit(msg redis.XMessage, key string, out chan<- re // according to its specification (see the comment on the StreamSorter type). func (sorter *StreamSorter) PipelineFunc( ctx context.Context, - s Sync, + _ history.Sync, key string, in <-chan redis.XMessage, out chan<- redis.XMessage, @@ -395,7 +396,7 @@ func (sorter *StreamSorter) PipelineFunc( err := sorter.submit(msg, key, out) if err != nil { - s.logger.Errorw("Failed to submit Redis stream event to stream sorter", + sorter.logger.Errorw("Failed to submit Redis stream event to stream sorter", zap.String("key", key), zap.Error(err)) } diff --git a/pkg/icingadb/history/sorter_test.go b/pkg/notifications/sorter_test.go similarity index 98% rename from pkg/icingadb/history/sorter_test.go rename to pkg/notifications/sorter_test.go index 6e43124ed..2b7e11c9e 100644 --- a/pkg/icingadb/history/sorter_test.go +++ b/pkg/notifications/sorter_test.go @@ -1,5 +1,5 @@ // #nosec G404 -- Allow math/rand for the tests -package history +package notifications import ( "cmp" @@ -7,6 +7,7 @@ import ( "fmt" "github.com/icinga/icinga-go-library/logging" "github.com/icinga/icinga-go-library/redis" + "github.com/icinga/icingadb/pkg/icingadb/history" "github.com/stretchr/testify/require" "go.uber.org/zap/zaptest" "math/rand" @@ -263,7 +264,7 @@ func TestStreamSorter(t *testing.T) { in := make(chan redis.XMessage) out := make(chan redis.XMessage) go func() { - require.NoError(t, sorter.PipelineFunc(context.Background(), Sync{}, "", in, out)) + require.NoError(t, sorter.PipelineFunc(context.Background(), history.Sync{}, "", in, out)) }() if !earlyClose { From d6752572fe129042fb8e6db445785f48affc071e Mon Sep 17 00:00:00 2001 From: Alvar Penning Date: Mon, 10 Nov 2025 10:08:54 +0100 Subject: [PATCH 36/38] notifications: TypeAcknowledgementCleared Message Change the message for TypeAcknowledgementCleared events to a more obvious one. --- pkg/notifications/notifications.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pkg/notifications/notifications.go b/pkg/notifications/notifications.go index 288a086d8..f5313a2cd 100644 --- a/pkg/notifications/notifications.go +++ b/pkg/notifications/notifications.go @@ -327,7 +327,7 @@ func (client *Client) buildAcknowledgementHistoryEvent(ctx context.Context, h *v if !h.ClearTime.Time().IsZero() { ev.Type = event.TypeAcknowledgementCleared - ev.Message = "Checkable was cleared" + ev.Message = "Acknowledgement was cleared" if h.ClearedBy.Valid { ev.Username = h.ClearedBy.String From a254086481c51dc1ae39aa79cc97c4033d82cfc6 Mon Sep 17 00:00:00 2001 From: Alvar Penning Date: Mon, 10 Nov 2025 17:00:56 +0100 Subject: [PATCH 37/38] notifications: Mute and Unmute Events Populate the Event's Mute field for muting and unmuting for flapping events and acknowledgements. --- pkg/notifications/notifications.go | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/pkg/notifications/notifications.go b/pkg/notifications/notifications.go index f5313a2cd..d79ee8bce 100644 --- a/pkg/notifications/notifications.go +++ b/pkg/notifications/notifications.go @@ -304,6 +304,7 @@ func (client *Client) buildFlappingHistoryEvent(ctx context.Context, h *v1histor ev.Message = fmt.Sprintf( "Checkable stopped flapping (Current flapping value %.2f%% < low threshold %.2f%%)", h.PercentStateChangeEnd.Float64, h.FlappingThresholdLow) + ev.Mute = types.MakeBool(false) } else if h.PercentStateChangeStart.Valid { ev.Type = event.TypeFlappingStart ev.Message = fmt.Sprintf( @@ -328,12 +329,15 @@ func (client *Client) buildAcknowledgementHistoryEvent(ctx context.Context, h *v if !h.ClearTime.Time().IsZero() { ev.Type = event.TypeAcknowledgementCleared ev.Message = "Acknowledgement was cleared" + ev.Mute = types.MakeBool(false) if h.ClearedBy.Valid { ev.Username = h.ClearedBy.String } } else if !h.SetTime.Time().IsZero() { ev.Type = event.TypeAcknowledgementSet + ev.Mute = types.MakeBool(true) + ev.MuteReason = "Checkable was acknowledged" if h.Comment.Valid { ev.Message = h.Comment.String From cd7a5979d57a57ce0178109009fd53141f4e18c2 Mon Sep 17 00:00:00 2001 From: Alvar Penning Date: Mon, 10 Nov 2025 17:26:01 +0100 Subject: [PATCH 38/38] notifications: Speed up StreamSorter Tests Allow configurable timeouts for the StreamSorter, to set them to a fraction of their default for the tests. Now the tests are done in three seconds instead of three minutes. While doing so, another race condition with the test logging was unveiled. Since this race results from a closing test context and test logger, there was not much to do and I decided to just drop the logging message, which was used only for tests anyway. --- pkg/notifications/sorter.go | 30 ++++++++++++++++++------------ pkg/notifications/sorter_test.go | 18 ++++++++++-------- 2 files changed, 28 insertions(+), 20 deletions(-) diff --git a/pkg/notifications/sorter.go b/pkg/notifications/sorter.go index 51235e790..4fb6aa54c 100644 --- a/pkg/notifications/sorter.go +++ b/pkg/notifications/sorter.go @@ -136,6 +136,13 @@ type StreamSorter struct { // that output channel and it can be closed by the worker after it processed all pending submissions for it. closeOutCh chan chan<- redis.XMessage + // The following fields should only be changed for the tests. + + // callbackMaxDelay is the maximum delay for continuously failing callbacks. Defaults to 10s. + callbackMaxDelay time.Duration + // submissionMinAge is the minimum age for a submission before being forwarded. Defaults to 3s. + submissionMinAge time.Duration + // isVerbose implies a isVerbose debug logging. Don't think one want to have this outside the tests. isVerbose bool } @@ -147,12 +154,14 @@ func NewStreamSorter( callbackFn func(msg redis.XMessage, key string) bool, ) *StreamSorter { sorter := &StreamSorter{ - ctx: ctx, - logger: logger, - callbackFn: callbackFn, - submissionCh: make(chan *streamSorterSubmission), - registerOutCh: make(chan chan<- redis.XMessage), - closeOutCh: make(chan chan<- redis.XMessage), + ctx: ctx, + logger: logger, + callbackFn: callbackFn, + submissionCh: make(chan *streamSorterSubmission), + registerOutCh: make(chan chan<- redis.XMessage), + closeOutCh: make(chan chan<- redis.XMessage), + callbackMaxDelay: 10 * time.Second, + submissionMinAge: 3 * time.Second, } go sorter.worker() @@ -184,7 +193,6 @@ func (sorter *StreamSorter) startCallback(msg redis.XMessage, key string) <-chan go func() { defer close(callbackCh) - const callbackMaxDelay = 10 * time.Second callbackDelay := time.Duration(0) for try := 0; ; try++ { @@ -207,7 +215,7 @@ func (sorter *StreamSorter) startCallback(msg redis.XMessage, key string) <-chan if success { return } else { - callbackDelay = min(max(time.Millisecond, 2*callbackDelay), callbackMaxDelay) + callbackDelay = min(max(time.Millisecond, 2*callbackDelay), sorter.callbackMaxDelay) } } }() @@ -221,7 +229,6 @@ func (sorter *StreamSorter) worker() { // When a streamSorterSubmission is created in the submit method, the current time.Time is added to the struct. // Only if the submission was at least three seconds (submissionMinAge) ago, a popped submission from the heap will // be passed to startCallback in its own goroutine to execute the callback function. - const submissionMinAge = 3 * time.Second var submissionHeap streamSorterSubmissions // Each registered output is stored in the registeredOutputs map, mapping output channels to the following struct. @@ -255,12 +262,12 @@ func (sorter *StreamSorter) worker() { var nextSubmissionDue <-chan time.Time if runningCallbackCh == nil { if next := submissionHeap.Peek(); next != nil { - if submissionAge := time.Since(next.submitTime); submissionAge >= submissionMinAge { + if submissionAge := time.Since(next.submitTime); submissionAge >= sorter.submissionMinAge { runningCallbackCh = sorter.startCallback(next.msg, next.key) runningSubmission = next heap.Pop(&submissionHeap) } else { - nextSubmissionDue = time.After(submissionMinAge - submissionAge) + nextSubmissionDue = time.After(sorter.submissionMinAge - submissionAge) } } } @@ -275,7 +282,6 @@ func (sorter *StreamSorter) worker() { // This function is now responsible for closing out. case out := <-sorter.closeOutCh: - sorter.verbose("worker: request close output", zap.String("out", fmt.Sprint(out))) if state := registeredOutputs[out]; state == nil { panic("requested to close unknown output channel") } else if state.pending > 0 { diff --git a/pkg/notifications/sorter_test.go b/pkg/notifications/sorter_test.go index 2b7e11c9e..21f490796 100644 --- a/pkg/notifications/sorter_test.go +++ b/pkg/notifications/sorter_test.go @@ -214,7 +214,7 @@ func TestStreamSorter(t *testing.T) { callbackCollectionMutex sync.Mutex callbackFn = func(msg redis.XMessage, _ string) bool { if tt.callbackMaxDelayMs > 0 { - time.Sleep(time.Duration(rand.Int63n(int64(tt.callbackMaxDelayMs))) * time.Millisecond) + time.Sleep(time.Duration(rand.Int63n(int64(tt.callbackMaxDelayMs))) * time.Microsecond) } if rand.Int63n(100)+1 > int64(tt.callbackSuccessPercent) { @@ -234,7 +234,7 @@ func TestStreamSorter(t *testing.T) { outConsumer = func(out chan redis.XMessage) { for { if tt.outMaxDelayMs > 0 { - time.Sleep(time.Duration(rand.Int63n(int64(tt.outMaxDelayMs))) * time.Millisecond) + time.Sleep(time.Duration(rand.Int63n(int64(tt.outMaxDelayMs))) * time.Microsecond) } _, ok := <-out @@ -256,6 +256,8 @@ func TestStreamSorter(t *testing.T) { t.Context(), logging.NewLogger(zaptest.NewLogger(t).Sugar(), time.Second), callbackFn) + sorter.callbackMaxDelay = 100 * time.Millisecond + sorter.submissionMinAge = 30 * time.Millisecond sorter.isVerbose = true for i := range tt.producers { @@ -273,7 +275,7 @@ func TestStreamSorter(t *testing.T) { go func() { for { - time.Sleep(time.Duration(rand.Int63n(250)) * time.Millisecond) + time.Sleep(time.Duration(rand.Int63n(250)) * time.Microsecond) inCounterMutex.Lock() isFin := inCounter <= 0 @@ -286,14 +288,14 @@ func TestStreamSorter(t *testing.T) { return } - ms := time.Now().UnixMilli() + rand.Int63n(2_000) - 1_000 - seq := rand.Int63n(100) + ms := time.Now().UnixMilli() + rand.Int63n(20) - 10 + seq := rand.Int63n(1_000) // Add 10% time travelers if rand.Int63n(10) == 9 { - distanceMs := int64(1_500) + distanceMs := int64(5) if rand.Int63n(2) > 0 { - // Don't go back too far. Otherwise, elements would be out of order. Three seconds max. + // Don't go back too far. Otherwise, elements would be out of order - submissionMinAge. ms -= distanceMs } else { ms += distanceMs @@ -325,7 +327,7 @@ func TestStreamSorter(t *testing.T) { break breakFor } - case <-time.After(2 * time.Minute): + case <-time.After(3 * time.Second): if tt.expectTimeout { return }