Add alertmanager notification support to Prometheus.

Alert definitions now also have mandatory SUMMARY and DESCRIPTION fields
that get sent along a firing alert to the alert manager.
This commit is contained in:
Julius Volz 2013-07-30 17:18:07 +02:00
parent 8eb1e8e679
commit 35ee2cd3cb
16 changed files with 1062 additions and 510 deletions

23
main.go
View File

@ -24,6 +24,7 @@ import (
"github.com/prometheus/client_golang/extraction" "github.com/prometheus/client_golang/extraction"
"github.com/prometheus/prometheus/config" "github.com/prometheus/prometheus/config"
"github.com/prometheus/prometheus/notification"
"github.com/prometheus/prometheus/retrieval" "github.com/prometheus/prometheus/retrieval"
"github.com/prometheus/prometheus/rules" "github.com/prometheus/prometheus/rules"
"github.com/prometheus/prometheus/storage/metric" "github.com/prometheus/prometheus/storage/metric"
@ -62,6 +63,9 @@ var (
arenaFlushInterval = flag.Duration("arena.flushInterval", 15*time.Minute, "The period at which the in-memory arena is flushed to disk.") arenaFlushInterval = flag.Duration("arena.flushInterval", 15*time.Minute, "The period at which the in-memory arena is flushed to disk.")
arenaTTL = flag.Duration("arena.ttl", 10*time.Minute, "The relative age of values to purge to disk from memory.") arenaTTL = flag.Duration("arena.ttl", 10*time.Minute, "The relative age of values to purge to disk from memory.")
alertmanagerUrl = flag.String("alertmanager.url", "", "The URL of the alert manager to send notifications to.")
notificationQueueCapacity = flag.Int("alertmanager.notificationQueueCapacity", 100, "The size of the queue for pending alert manager notifications.")
) )
type prometheus struct { type prometheus struct {
@ -77,7 +81,9 @@ type prometheus struct {
unwrittenSamples chan *extraction.Result unwrittenSamples chan *extraction.Result
storage *metric.TieredStorage ruleManager rules.RuleManager
notifications chan rules.NotificationReqs
storage *metric.TieredStorage
} }
func (p *prometheus) interruptHandler() { func (p *prometheus) interruptHandler() {
@ -146,7 +152,10 @@ func (p *prometheus) close() {
p.curationMutex.Lock() p.curationMutex.Lock()
p.ruleManager.Stop()
p.storage.Close() p.storage.Close()
close(p.notifications)
close(p.stopBackgroundOperations) close(p.stopBackgroundOperations)
close(p.curationState) close(p.curationState)
close(p.databaseStates) close(p.databaseStates)
@ -209,14 +218,20 @@ func main() {
targetManager := retrieval.NewTargetManager(unwrittenSamples, *concurrentRetrievalAllowance) targetManager := retrieval.NewTargetManager(unwrittenSamples, *concurrentRetrievalAllowance)
targetManager.AddTargetsFromConfig(conf) targetManager.AddTargetsFromConfig(conf)
notifications := make(chan rules.NotificationReqs, *notificationQueueCapacity)
// Queue depth will need to be exposed // Queue depth will need to be exposed
ruleManager := rules.NewRuleManager(unwrittenSamples, conf.EvaluationInterval(), ts) ruleManager := rules.NewRuleManager(unwrittenSamples, notifications, conf.EvaluationInterval(), ts)
err = ruleManager.AddRulesFromConfig(conf) err = ruleManager.AddRulesFromConfig(conf)
if err != nil { if err != nil {
log.Fatalf("Error loading rule files: %v", err) log.Fatalf("Error loading rule files: %v", err)
} }
go ruleManager.Run() go ruleManager.Run()
// Queue depth will need to be exposed
notificationHandler := notification.NewNotificationHandler(*alertmanagerUrl, notifications)
go notificationHandler.Run()
flags := map[string]string{} flags := map[string]string{}
flag.VisitAll(func(f *flag.Flag) { flag.VisitAll(func(f *flag.Flag) {
@ -272,7 +287,9 @@ func main() {
stopBackgroundOperations: make(chan bool, 1), stopBackgroundOperations: make(chan bool, 1),
storage: ts, ruleManager: ruleManager,
notifications: notifications,
storage: ts,
} }
defer prometheus.close() defer prometheus.close()

View File

@ -0,0 +1,105 @@
// Copyright 2013 Prometheus Team
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
package notification
import (
"bytes"
"encoding/json"
"flag"
"io/ioutil"
"log"
"net/http"
"time"
clientmodel "github.com/prometheus/client_golang/model"
"github.com/prometheus/prometheus/rules"
"github.com/prometheus/prometheus/utility"
)
const (
alertmanagerApiEventsPath = "/api/events"
contentTypeJson = "application/json"
)
var (
deadline = flag.Duration("alertmanager.httpDeadline", 10*time.Second, "Alert manager HTTP API timeout.")
)
// NotificationHandler is responsible for dispatching alert notifications to an
// alert manager service.
type NotificationHandler struct {
// The URL of the alert manager to send notifications to.
alertmanagerUrl string
// Buffer of notifications that have not yet been sent.
pendingNotifications <-chan rules.NotificationReqs
// HTTP client with custom timeout settings.
httpClient http.Client
}
// Construct a new NotificationHandler.
func NewNotificationHandler(alertmanagerUrl string, notificationReqs <-chan rules.NotificationReqs) *NotificationHandler {
return &NotificationHandler{
alertmanagerUrl: alertmanagerUrl,
pendingNotifications: notificationReqs,
httpClient: utility.NewDeadlineClient(*deadline),
}
}
// Send a list of notifications to the configured alert manager.
func (n *NotificationHandler) sendNotifications(reqs rules.NotificationReqs) error {
alerts := make([]map[string]interface{}, 0, len(reqs))
for _, req := range reqs {
alerts = append(alerts, map[string]interface{}{
"Summary": req.Rule.Summary,
"Description": req.Rule.Description,
"Labels": req.ActiveAlert.Labels.Merge(clientmodel.LabelSet{
rules.AlertNameLabel: clientmodel.LabelValue(req.Rule.Name()),
}),
"Payload": map[string]interface{}{
"Value": req.ActiveAlert.Value,
"ActiveSince": req.ActiveAlert.ActiveSince,
},
})
}
buf, err := json.Marshal(alerts)
if err != nil {
return err
}
resp, err := n.httpClient.Post(
n.alertmanagerUrl+alertmanagerApiEventsPath,
contentTypeJson,
bytes.NewBuffer(buf),
)
if err != nil {
return err
}
defer resp.Body.Close()
_, err = ioutil.ReadAll(resp.Body)
if err != nil {
return err
}
// BUG: Do we need to check the response code?
return nil
}
// Continusouly dispatch notifications.
func (n *NotificationHandler) Run() {
for reqs := range n.pendingNotifications {
if err := n.sendNotifications(reqs); err != nil {
log.Println("Error sending notification:", err)
}
}
}

View File

@ -1,30 +0,0 @@
package retrieval
import (
"net"
"net/http"
"time"
)
// NewDeadlineClient returns a new http.Client which will time out long running
// requests.
func NewDeadlineClient(timeout time.Duration) http.Client {
return http.Client{
Transport: &http.Transport{
// We need to disable keepalive, becasue we set a deadline on the
// underlying connection.
DisableKeepAlives: true,
Dial: func(netw, addr string) (c net.Conn, err error) {
start := time.Now()
c, err = net.DialTimeout(netw, addr, timeout)
if err == nil {
c.SetDeadline(start.Add(timeout))
}
return
},
},
}
}

View File

@ -25,6 +25,8 @@ import (
"github.com/prometheus/client_golang/extraction" "github.com/prometheus/client_golang/extraction"
clientmodel "github.com/prometheus/client_golang/model" clientmodel "github.com/prometheus/client_golang/model"
"github.com/prometheus/prometheus/utility"
) )
const ( const (
@ -143,7 +145,7 @@ func NewTarget(address string, deadline time.Duration, baseLabels clientmodel.La
address: address, address: address,
Deadline: deadline, Deadline: deadline,
baseLabels: baseLabels, baseLabels: baseLabels,
client: NewDeadlineClient(deadline), client: utility.NewDeadlineClient(deadline),
} }
scheduler := &healthScheduler{ scheduler := &healthScheduler{

View File

@ -101,7 +101,11 @@ type AlertingRule struct {
// output vector before an alert transitions from PENDING to FIRING state. // output vector before an alert transitions from PENDING to FIRING state.
holdDuration time.Duration holdDuration time.Duration
// Extra labels to attach to the resulting alert sample vectors. // Extra labels to attach to the resulting alert sample vectors.
labels clientmodel.LabelSet Labels clientmodel.LabelSet
// Short alert summary, suitable for email subjects.
Summary string
// More detailed alert description.
Description string
// Protects the below. // Protects the below.
mutex sync.Mutex mutex sync.Mutex
@ -110,7 +114,9 @@ type AlertingRule struct {
activeAlerts map[clientmodel.Fingerprint]*Alert activeAlerts map[clientmodel.Fingerprint]*Alert
} }
func (rule *AlertingRule) Name() string { return rule.name } func (rule *AlertingRule) Name() string {
return rule.name
}
func (rule *AlertingRule) EvalRaw(timestamp time.Time, storage *metric.TieredStorage) (ast.Vector, error) { func (rule *AlertingRule) EvalRaw(timestamp time.Time, storage *metric.TieredStorage) (ast.Vector, error) {
return ast.EvalVectorInstant(rule.vector, timestamp, storage, stats.NewTimerGroup()) return ast.EvalVectorInstant(rule.vector, timestamp, storage, stats.NewTimerGroup())
@ -137,6 +143,7 @@ func (rule *AlertingRule) Eval(timestamp time.Time, storage *metric.TieredStorag
if alert, ok := rule.activeAlerts[*fp]; !ok { if alert, ok := rule.activeAlerts[*fp]; !ok {
labels := clientmodel.LabelSet{} labels := clientmodel.LabelSet{}
labels.MergeFromMetric(sample.Metric) labels.MergeFromMetric(sample.Metric)
labels = labels.Merge(rule.Labels)
if _, ok := labels[clientmodel.MetricNameLabel]; ok { if _, ok := labels[clientmodel.MetricNameLabel]; ok {
delete(labels, clientmodel.MetricNameLabel) delete(labels, clientmodel.MetricNameLabel)
} }
@ -183,7 +190,7 @@ func (rule *AlertingRule) ToDotGraph() string {
} }
func (rule *AlertingRule) String() string { func (rule *AlertingRule) String() string {
return fmt.Sprintf("ALERT %s IF %s FOR %s WITH %s", rule.name, rule.vector, utility.DurationToString(rule.holdDuration), rule.labels) return fmt.Sprintf("ALERT %s IF %s FOR %s WITH %s", rule.name, rule.vector, utility.DurationToString(rule.holdDuration), rule.Labels)
} }
func (rule *AlertingRule) HTMLSnippet() template.HTML { func (rule *AlertingRule) HTMLSnippet() template.HTML {
@ -198,7 +205,7 @@ func (rule *AlertingRule) HTMLSnippet() template.HTML {
ConsoleLinkForExpression(rule.vector.String()), ConsoleLinkForExpression(rule.vector.String()),
rule.vector, rule.vector,
utility.DurationToString(rule.holdDuration), utility.DurationToString(rule.holdDuration),
rule.labels)) rule.Labels))
} }
func (rule *AlertingRule) State() AlertState { func (rule *AlertingRule) State() AlertState {
@ -226,12 +233,15 @@ func (rule *AlertingRule) ActiveAlerts() []Alert {
} }
// Construct a new AlertingRule. // Construct a new AlertingRule.
func NewAlertingRule(name string, vector ast.VectorNode, holdDuration time.Duration, labels clientmodel.LabelSet) *AlertingRule { func NewAlertingRule(name string, vector ast.VectorNode, holdDuration time.Duration, labels clientmodel.LabelSet, summary string, description string) *AlertingRule {
return &AlertingRule{ return &AlertingRule{
name: name, name: name,
vector: vector, vector: vector,
holdDuration: holdDuration, holdDuration: holdDuration,
labels: labels, Labels: labels,
Summary: summary,
Description: description,
activeAlerts: map[clientmodel.Fingerprint]*Alert{}, activeAlerts: map[clientmodel.Fingerprint]*Alert{},
} }
} }

View File

@ -3,11 +3,14 @@ dc_http_request_rate5m = sum(rate(http_request_count[5m])) by (dc)
// A simple test alerting rule. // A simple test alerting rule.
ALERT GlobalRequestRateLow IF(dc_http_request_rate5m < 10000) FOR 5m WITH { ALERT GlobalRequestRateLow IF(dc_http_request_rate5m < 10000) FOR 5m WITH {
description = "Global HTTP request rate low!", service = "testservice"
summary = "Request rate low" /* ... more fields here ... */
/* ... more fields here ... */ }
} SUMMARY "Global request rate low"
DESCRIPTION "The global request rate is low"
foo = bar{label1="value1"} foo = bar{label1="value1"}
ALERT BazAlert IF(foo > 10) WITH {} ALERT BazAlert IF(foo > 10) WITH {}
SUMMARY "Baz"
DESCRIPTION "BazAlert"

View File

@ -3,11 +3,13 @@ dc_http_request_rate5m = sum(rate(http_request_count[5m])) by (dc)
// A simple test alerting rule with a syntax error (invalid duration string "5"). // A simple test alerting rule with a syntax error (invalid duration string "5").
ALERT GlobalRequestRateLow IF(dc_http_request_rate5m < 10000) FOR 5 WITH { ALERT GlobalRequestRateLow IF(dc_http_request_rate5m < 10000) FOR 5 WITH {
description = "Global HTTP request rate low!", description = "Global HTTP request rate low!",
summary = "Request rate low" summary = "Request rate low"
/* ... more fields here ... */ /* ... more fields here ... */
} }
SUMMARY "summary"
DESCRIPTION "description"
foo = bar{label1="value1"} foo = bar{label1="value1"}
ALERT BazAlert IF(foo > 10) WITH {} ALERT BazAlert IF(foo > 10) WITH {} SUMMARY "summary" DESCRIPTION "description"

View File

@ -30,7 +30,7 @@ func CreateRecordingRule(name string, labels clientmodel.LabelSet, expr ast.Node
return NewRecordingRule(name, labels, expr.(ast.VectorNode), permanent), nil return NewRecordingRule(name, labels, expr.(ast.VectorNode), permanent), nil
} }
func CreateAlertingRule(name string, expr ast.Node, holdDurationStr string, labels clientmodel.LabelSet) (*AlertingRule, error) { func CreateAlertingRule(name string, expr ast.Node, holdDurationStr string, labels clientmodel.LabelSet, summary string, description string) (*AlertingRule, error) {
if _, ok := expr.(ast.VectorNode); !ok { if _, ok := expr.(ast.VectorNode); !ok {
return nil, fmt.Errorf("Alert rule expression %v does not evaluate to vector type", expr) return nil, fmt.Errorf("Alert rule expression %v does not evaluate to vector type", expr)
} }
@ -38,7 +38,7 @@ func CreateAlertingRule(name string, expr ast.Node, holdDurationStr string, labe
if err != nil { if err != nil {
return nil, err return nil, err
} }
return NewAlertingRule(name, expr.(ast.VectorNode), holdDuration, labels), nil return NewAlertingRule(name, expr.(ast.VectorNode), holdDuration, labels, summary, description), nil
} }
func NewFunctionCall(name string, args []ast.Node) (ast.Node, error) { func NewFunctionCall(name string, args []ast.Node) (ast.Node, error) {

View File

@ -1,15 +1,15 @@
// Copyright 2013 Prometheus Team /* Copyright 2013 Prometheus Team
// Licensed under the Apache License, Version 2.0 (the "License"); * Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License. * you may not use this file except in compliance with the License.
// You may obtain a copy of the License at * You may obtain a copy of the License at
// *
// http://www.apache.org/licenses/LICENSE-2.0 * http://www.apache.org/licenses/LICENSE-2.0
// *
// Unless required by applicable law or agreed to in writing, software * Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS, * distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and * See the License for the specific language governing permissions and
// limitations under the License. * limitations under the License. */
%{ %{
package rules package rules
@ -73,6 +73,8 @@ ALERT|alert return ALERT
IF|if return IF IF|if return IF
FOR|for return FOR FOR|for return FOR
WITH|with return WITH WITH|with return WITH
SUMMARY|summary return SUMMARY
DESCRIPTION|description return DESCRIPTION
PERMANENT|permanent return PERMANENT PERMANENT|permanent return PERMANENT
BY|by return GROUP_OP BY|by return GROUP_OP

File diff suppressed because it is too large Load Diff

View File

@ -38,24 +38,36 @@ type RuleManager interface {
AlertingRules() []*AlertingRule AlertingRules() []*AlertingRule
} }
// A request for sending an alert notification to the alert manager. This needs
// to be defined in this package to prevent a circular import between
// rules<->notification.
type NotificationReq struct {
Rule *AlertingRule
ActiveAlert Alert
}
type NotificationReqs []*NotificationReq
type ruleManager struct { type ruleManager struct {
// Protects the rules list. // Protects the rules list.
sync.Mutex sync.Mutex
rules []Rule rules []Rule
results chan<- *extraction.Result results chan<- *extraction.Result
done chan bool notifications chan<- NotificationReqs
interval time.Duration done chan bool
storage *metric.TieredStorage interval time.Duration
storage *metric.TieredStorage
} }
func NewRuleManager(results chan<- *extraction.Result, interval time.Duration, storage *metric.TieredStorage) RuleManager { func NewRuleManager(results chan<- *extraction.Result, notifications chan<- NotificationReqs, interval time.Duration, storage *metric.TieredStorage) RuleManager {
manager := &ruleManager{ manager := &ruleManager{
results: results, results: results,
rules: []Rule{}, notifications: notifications,
done: make(chan bool), rules: []Rule{},
interval: interval, done: make(chan bool),
storage: storage, interval: interval,
storage: storage,
} }
return manager return manager
} }
@ -84,6 +96,27 @@ func (m *ruleManager) Stop() {
} }
} }
func (m *ruleManager) queueAlertNotifications(rule *AlertingRule) {
activeAlerts := rule.ActiveAlerts()
if len(activeAlerts) == 0 {
return
}
notifications := make(NotificationReqs, 0, len(activeAlerts))
for _, aa := range activeAlerts {
if aa.State != FIRING {
// BUG: In the future, make AlertManager support pending alerts?
continue
}
notifications = append(notifications, &NotificationReq{
Rule: rule,
ActiveAlert: aa,
})
}
m.notifications <- notifications
}
func (m *ruleManager) runIteration(results chan<- *extraction.Result) { func (m *ruleManager) runIteration(results chan<- *extraction.Result) {
now := time.Now() now := time.Now()
wg := sync.WaitGroup{} wg := sync.WaitGroup{}
@ -105,6 +138,10 @@ func (m *ruleManager) runIteration(results chan<- *extraction.Result) {
Samples: samples, Samples: samples,
Err: err, Err: err,
} }
if alertingRule, ok := rule.(*AlertingRule); ok {
m.queueAlertNotifications(alertingRule)
}
}(rule) }(rule)
} }

View File

@ -41,7 +41,7 @@
%token <num> NUMBER %token <num> NUMBER
%token PERMANENT GROUP_OP %token PERMANENT GROUP_OP
%token <str> AGGR_OP CMP_OP ADDITIVE_OP MULT_OP %token <str> AGGR_OP CMP_OP ADDITIVE_OP MULT_OP
%token ALERT IF FOR WITH %token ALERT IF FOR WITH SUMMARY DESCRIPTION
%type <ruleNodeSlice> func_arg_list %type <ruleNodeSlice> func_arg_list
%type <labelNameSlice> label_list grouping_opts %type <labelNameSlice> label_list grouping_opts
@ -75,9 +75,9 @@ rules_stat : qualifier IDENTIFIER rule_labels '=' rule_expr
if err != nil { yylex.Error(err.Error()); return 1 } if err != nil { yylex.Error(err.Error()); return 1 }
yylex.(*RulesLexer).parsedRules = append(yylex.(*RulesLexer).parsedRules, rule) yylex.(*RulesLexer).parsedRules = append(yylex.(*RulesLexer).parsedRules, rule)
} }
| ALERT IDENTIFIER IF rule_expr for_duration WITH rule_labels | ALERT IDENTIFIER IF rule_expr for_duration WITH rule_labels SUMMARY STRING DESCRIPTION STRING
{ {
rule, err := CreateAlertingRule($2, $4, $5, $7) rule, err := CreateAlertingRule($2, $4, $5, $7, $9, $11)
if err != nil { yylex.Error(err.Error()); return 1 } if err != nil { yylex.Error(err.Error()); return 1 }
yylex.(*RulesLexer).parsedRules = append(yylex.(*RulesLexer).parsedRules, rule) yylex.(*RulesLexer).parsedRules = append(yylex.(*RulesLexer).parsedRules, rule)
} }

View File

@ -38,6 +38,8 @@ const ALERT = 57358
const IF = 57359 const IF = 57359
const FOR = 57360 const FOR = 57360
const WITH = 57361 const WITH = 57361
const SUMMARY = 57362
const DESCRIPTION = 57363
var yyToknames = []string{ var yyToknames = []string{
"START_RULES", "START_RULES",
@ -56,6 +58,8 @@ var yyToknames = []string{
"IF", "IF",
"FOR", "FOR",
"WITH", "WITH",
"SUMMARY",
"DESCRIPTION",
" =", " =",
} }
var yyStatenames = []string{} var yyStatenames = []string{}
@ -83,35 +87,37 @@ const yyPrivate = 57344
var yyTokenNames []string var yyTokenNames []string
var yyStates []string var yyStates []string
const yyLast = 97 const yyLast = 101
var yyAct = []int{ var yyAct = []int{
20, 38, 34, 17, 33, 43, 6, 67, 15, 66, 20, 38, 34, 43, 33, 17, 6, 18, 16, 17,
19, 18, 16, 17, 15, 45, 59, 44, 60, 27, 19, 15, 59, 18, 16, 17, 16, 17, 15, 27,
28, 29, 16, 17, 15, 41, 40, 18, 16, 17, 28, 29, 15, 60, 23, 41, 40, 49, 15, 22,
18, 16, 17, 22, 15, 23, 21, 46, 47, 49, 15, 8, 35, 67, 10, 66, 45, 9, 44, 50,
15, 8, 35, 15, 10, 51, 22, 9, 50, 53, 18, 16, 17, 46, 47, 51, 18, 16, 17, 53,
52, 48, 61, 57, 18, 16, 17, 39, 8, 7, 52, 7, 32, 57, 30, 15, 39, 8, 35, 48,
32, 10, 65, 42, 9, 56, 30, 15, 8, 35, 10, 15, 65, 9, 8, 22, 71, 10, 21, 68,
54, 10, 62, 37, 9, 14, 7, 26, 68, 64, 9, 61, 42, 14, 37, 56, 62, 7, 26, 13,
39, 13, 25, 24, 2, 3, 7, 11, 5, 4, 72, 70, 54, 69, 7, 64, 39, 25, 24, 2,
1, 58, 12, 36, 55, 63, 31, 3, 11, 5, 4, 1, 58, 12, 36, 55, 63,
31,
} }
var yyPact = []int{ var yyPact = []int{
80, -1000, -1000, 52, 65, -1000, 17, 52, 12, 11, 85, -1000, -1000, 58, 63, -1000, 33, 58, 42, -2,
-1000, -1000, 77, 76, -1000, 69, 52, 52, 52, 41, -1000, -1000, 82, 81, -1000, 70, 58, 58, 58, 27,
-1000, 35, 51, 52, 25, 46, -22, -12, -18, 8, -1000, 25, 50, 58, 6, 55, -26, -10, -17, 2,
-1000, -8, -1000, -1000, 17, -1000, 15, -1000, -1000, 31, -1000, 11, -1000, -1000, 33, -1000, 19, -1000, -1000, 37,
14, 28, 52, -1000, -1000, 62, -1000, 74, 63, 54, 0, 17, 58, -1000, -1000, 51, -1000, 80, 75, 64,
52, -2, -1000, -1000, -1000, -1000, -6, 17, 33, 64, 58, -6, -1000, -1000, -1000, -1000, -3, 33, 52, 68,
73, 25, -1000, -16, -1000, -1000, -1000, 72, -1000, 79, 6, -1000, 8, -1000, 49, -1000, 77, 74, -1000,
45, 73, -1000,
} }
var yyPgo = []int{ var yyPgo = []int{
0, 96, 95, 94, 1, 93, 0, 2, 4, 92, 0, 100, 99, 98, 1, 97, 0, 2, 4, 96,
91, 90, 89, 88, 87, 95, 94, 93, 92, 91,
} }
var yyR1 = []int{ var yyR1 = []int{
@ -122,20 +128,21 @@ var yyR1 = []int{
} }
var yyR2 = []int{ var yyR2 = []int{
0, 2, 2, 0, 2, 1, 5, 7, 0, 2, 0, 2, 2, 0, 2, 1, 5, 11, 0, 2,
0, 1, 0, 3, 2, 1, 3, 3, 3, 2, 0, 1, 0, 3, 2, 1, 3, 3, 3, 2,
4, 3, 4, 5, 3, 3, 3, 1, 0, 4, 4, 3, 4, 5, 3, 3, 3, 1, 0, 4,
1, 3, 1, 3, 1, 1, 1, 3, 1, 3, 1, 1,
} }
var yyChk = []int{ var yyChk = []int{
-1000, -11, 4, 5, -12, -13, -7, 24, 6, 12, -1000, -11, 4, 5, -12, -13, -7, 26, 6, 12,
9, -14, -9, 16, 10, 26, 14, 15, 13, -7, 9, -14, -9, 16, 10, 28, 14, 15, 13, -7,
-6, 24, 21, 24, 6, 6, 8, -7, -7, -7, -6, 26, 23, 26, 6, 6, 8, -7, -7, -7,
25, -1, 25, -8, -7, 7, -5, 22, -4, 6, 27, -1, 27, -8, -7, 7, -5, 24, -4, 6,
-7, -6, 17, 27, 25, 23, 22, 23, 20, 25, -7, -6, 17, 29, 27, 25, 24, 25, 22, 27,
20, -7, -8, -4, 7, -3, 11, -7, -10, 18, 22, -7, -8, -4, 7, -3, 11, -7, -10, 18,
24, 19, 8, -2, 6, -6, 25, 23, 6, 26, 19, 8, -2, 6, -6, 27, 25, 20, 6,
7, 21, 7,
} }
var yyDef = []int{ var yyDef = []int{
@ -145,7 +152,8 @@ var yyDef = []int{
18, 0, 21, 32, 34, 35, 0, 14, 15, 0, 18, 0, 21, 32, 34, 35, 0, 14, 15, 0,
0, 0, 0, 22, 20, 0, 13, 0, 0, 28, 0, 0, 0, 22, 20, 0, 13, 0, 0, 28,
0, 8, 33, 16, 17, 23, 0, 6, 0, 0, 0, 8, 33, 16, 17, 23, 0, 6, 0, 0,
0, 12, 9, 0, 30, 7, 29, 0, 31, 0, 12, 9, 0, 30, 0, 29, 0, 0, 31,
0, 0, 7,
} }
var yyTok1 = []int{ var yyTok1 = []int{
@ -153,20 +161,20 @@ var yyTok1 = []int{
3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3,
3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3,
3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3,
24, 25, 3, 3, 23, 3, 3, 3, 3, 3, 26, 27, 3, 3, 25, 3, 3, 3, 3, 3,
3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3,
3, 20, 3, 3, 3, 3, 3, 3, 3, 3, 3, 22, 3, 3, 3, 3, 3, 3, 3, 3,
3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3,
3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3,
3, 26, 3, 27, 3, 3, 3, 3, 3, 3, 3, 28, 3, 29, 3, 3, 3, 3, 3, 3,
3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3,
3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3,
3, 3, 3, 21, 3, 22, 3, 3, 3, 23, 3, 24,
} }
var yyTok2 = []int{ var yyTok2 = []int{
2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11,
12, 13, 14, 15, 16, 17, 18, 19, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21,
} }
var yyTok3 = []int{ var yyTok3 = []int{
0, 0,
@ -410,7 +418,7 @@ yydefault:
case 7: case 7:
//line parser.y:79 //line parser.y:79
{ {
rule, err := CreateAlertingRule(yyS[yypt-5].str, yyS[yypt-3].ruleNode, yyS[yypt-2].str, yyS[yypt-0].labelSet) rule, err := CreateAlertingRule(yyS[yypt-9].str, yyS[yypt-7].ruleNode, yyS[yypt-6].str, yyS[yypt-4].labelSet, yyS[yypt-2].str, yyS[yypt-0].str)
if err != nil { yylex.Error(err.Error()); return 1 } if err != nil { yylex.Error(err.Error()); return 1 }
yylex.(*RulesLexer).parsedRules = append(yylex.(*RulesLexer).parsedRules, rule) yylex.(*RulesLexer).parsedRules = append(yylex.(*RulesLexer).parsedRules, rule)
} }

View File

@ -511,18 +511,18 @@ func TestAlertingRule(t *testing.T) {
// Labels in expected output need to be alphabetically sorted. // Labels in expected output need to be alphabetically sorted.
var evalOutputs = [][]string{ var evalOutputs = [][]string{
{ {
`ALERTS{alertname="HttpRequestRateLow", alertstate="pending", group="canary", instance="0", job="app-server"} => 1 @[%v]`, `ALERTS{alertname="HttpRequestRateLow", alertstate="pending", group="canary", instance="0", job="app-server", severity="critical"} => 1 @[%v]`,
`ALERTS{alertname="HttpRequestRateLow", alertstate="pending", group="canary", instance="1", job="app-server"} => 1 @[%v]`, `ALERTS{alertname="HttpRequestRateLow", alertstate="pending", group="canary", instance="1", job="app-server", severity="critical"} => 1 @[%v]`,
}, },
{ {
`ALERTS{alertname="HttpRequestRateLow", alertstate="pending", group="canary", instance="0", job="app-server"} => 0 @[%v]`, `ALERTS{alertname="HttpRequestRateLow", alertstate="pending", group="canary", instance="0", job="app-server", severity="critical"} => 0 @[%v]`,
`ALERTS{alertname="HttpRequestRateLow", alertstate="firing", group="canary", instance="0", job="app-server"} => 1 @[%v]`, `ALERTS{alertname="HttpRequestRateLow", alertstate="firing", group="canary", instance="0", job="app-server", severity="critical"} => 1 @[%v]`,
`ALERTS{alertname="HttpRequestRateLow", alertstate="pending", group="canary", instance="1", job="app-server"} => 0 @[%v]`, `ALERTS{alertname="HttpRequestRateLow", alertstate="pending", group="canary", instance="1", job="app-server", severity="critical"} => 0 @[%v]`,
`ALERTS{alertname="HttpRequestRateLow", alertstate="firing", group="canary", instance="1", job="app-server"} => 1 @[%v]`, `ALERTS{alertname="HttpRequestRateLow", alertstate="firing", group="canary", instance="1", job="app-server", severity="critical"} => 1 @[%v]`,
}, },
{ {
`ALERTS{alertname="HttpRequestRateLow", alertstate="firing", group="canary", instance="1", job="app-server"} => 0 @[%v]`, `ALERTS{alertname="HttpRequestRateLow", alertstate="firing", group="canary", instance="1", job="app-server", severity="critical"} => 0 @[%v]`,
`ALERTS{alertname="HttpRequestRateLow", alertstate="firing", group="canary", instance="0", job="app-server"} => 0 @[%v]`, `ALERTS{alertname="HttpRequestRateLow", alertstate="firing", group="canary", instance="0", job="app-server", severity="critical"} => 0 @[%v]`,
}, },
{ {
/* empty */ /* empty */
@ -542,9 +542,9 @@ func TestAlertingRule(t *testing.T) {
} }
alertName := "HttpRequestRateLow" alertName := "HttpRequestRateLow"
alertLabels := clientmodel.LabelSet{ alertLabels := clientmodel.LabelSet{
"summary": "HTTP request rate is low", "severity": "critical",
} }
rule := NewAlertingRule(alertName, alertExpr.(ast.VectorNode), time.Minute, alertLabels) rule := NewAlertingRule(alertName, alertExpr.(ast.VectorNode), time.Minute, alertLabels, "summary", "description")
for i, expected := range evalOutputs { for i, expected := range evalOutputs {
evalTime := testStartTime.Add(testSampleInterval * time.Duration(i)) evalTime := testStartTime.Add(testSampleInterval * time.Duration(i))

View File

@ -0,0 +1,43 @@
// Copyright 2013 Prometheus Team
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
package utility
import (
"net"
"net/http"
"time"
)
// NewDeadlineClient returns a new http.Client which will time out long running
// requests.
func NewDeadlineClient(timeout time.Duration) http.Client {
return http.Client{
Transport: &http.Transport{
// We need to disable keepalive, becasue we set a deadline on the
// underlying connection.
DisableKeepAlives: true,
Dial: func(netw, addr string) (c net.Conn, err error) {
start := time.Now()
c, err = net.DialTimeout(netw, addr, timeout)
if err == nil {
c.SetDeadline(start.Add(timeout))
}
return
},
},
}
}

View File

@ -48,7 +48,7 @@ func (serv MetricsService) Query(expr string, asText string) string {
rb := serv.ResponseBuilder() rb := serv.ResponseBuilder()
serv.setAccessControlHeaders(rb) serv.setAccessControlHeaders(rb)
var format ast.OutputFormat var format ast.OutputFormat
// BUG(julius): Use Content-Type negotiation. // BUG(julius): Use Content-Type negotiation.
if asText == "" { if asText == "" {
format = ast.JSON format = ast.JSON
rb.SetContentType(gorest.Application_Json) rb.SetContentType(gorest.Application_Json)