diff --git a/main.go b/main.go index 53339fa23..45ccf192b 100644 --- a/main.go +++ b/main.go @@ -29,7 +29,7 @@ import ( "github.com/prometheus/prometheus/config" "github.com/prometheus/prometheus/notification" "github.com/prometheus/prometheus/retrieval" - "github.com/prometheus/prometheus/rules" + "github.com/prometheus/prometheus/rules/manager" "github.com/prometheus/prometheus/storage/metric/tiered" "github.com/prometheus/prometheus/storage/remote" "github.com/prometheus/prometheus/storage/remote/opentsdb" @@ -82,7 +82,7 @@ type prometheus struct { unwrittenSamples chan *extraction.Result - ruleManager rules.RuleManager + ruleManager manager.RuleManager targetManager retrieval.TargetManager notifications chan notification.NotificationReqs storage *tiered.TieredStorage @@ -272,7 +272,7 @@ func main() { notifications := make(chan notification.NotificationReqs, *notificationQueueCapacity) // Queue depth will need to be exposed - ruleManager := rules.NewRuleManager(&rules.RuleManagerOptions{ + ruleManager := manager.NewRuleManager(&manager.RuleManagerOptions{ Results: unwrittenSamples, Notifications: notifications, EvaluationInterval: conf.EvaluationInterval(), @@ -306,6 +306,10 @@ func main() { RuleManager: ruleManager, } + consolesHandler := &web.ConsolesHandler{ + Storage: ts, + } + databasesHandler := &web.DatabasesHandler{ Provider: ts.DiskStorage, RefreshInterval: 5 * time.Minute, @@ -341,6 +345,7 @@ func main() { StatusHandler: prometheusStatus, MetricsHandler: metricsService, DatabasesHandler: databasesHandler, + ConsolesHandler: consolesHandler, AlertsHandler: alertsHandler, QuitDelegate: prometheus.Close, diff --git a/notification/notification.go b/notification/notification.go index 793df9cc8..83fa9e40d 100644 --- a/notification/notification.go +++ b/notification/notification.go @@ -20,7 +20,6 @@ import ( "io" "io/ioutil" "net/http" - "text/template" "time" "github.com/golang/glog" @@ -84,49 +83,13 @@ func NewNotificationHandler(alertmanagerUrl string, notificationReqs <-chan Noti } } -// Interpolate alert information into summary/description templates. -func interpolateMessage(msg string, labels clientmodel.LabelSet, value clientmodel.SampleValue) string { - t := template.New("message") - - // Inject some convenience variables that are easier to remember for users - // who are not used to Go's templating system. - defs := - "{{$labels := .Labels}}" + - "{{$value := .Value}}" - - if _, err := t.Parse(defs + msg); err != nil { - glog.Warning("Error parsing template: ", err) - return msg - } - - l := map[string]string{} - for k, v := range labels { - l[string(k)] = string(v) - } - - tmplData := struct { - Labels map[string]string - Value clientmodel.SampleValue - }{ - Labels: l, - Value: value, - } - - var buf bytes.Buffer - if err := t.Execute(&buf, &tmplData); err != nil { - glog.Warning("Error executing template: ", err) - return msg - } - return buf.String() -} - // Send a list of notifications to the configured alert manager. func (n *NotificationHandler) sendNotifications(reqs NotificationReqs) error { alerts := make([]map[string]interface{}, 0, len(reqs)) for _, req := range reqs { alerts = append(alerts, map[string]interface{}{ - "Summary": interpolateMessage(req.Summary, req.Labels, req.Value), - "Description": interpolateMessage(req.Description, req.Labels, req.Value), + "Summary": req.Summary, + "Description": req.Description, "Labels": req.Labels, "Payload": map[string]interface{}{ "Value": req.Value, @@ -140,6 +103,7 @@ func (n *NotificationHandler) sendNotifications(reqs NotificationReqs) error { if err != nil { return err } + glog.V(1).Infoln("Sending notifications to alertmanager:", string(buf)) resp, err := n.httpClient.Post( n.alertmanagerUrl+alertmanagerApiEventsPath, contentTypeJson, diff --git a/notification/notification_test.go b/notification/notification_test.go index e01e6c42b..5bd856b89 100644 --- a/notification/notification_test.go +++ b/notification/notification_test.go @@ -80,27 +80,9 @@ func TestNotificationHandler(t *testing.T) { scenarios := []testNotificationScenario{ { // Correct message. - summary: "{{$labels.instance}} = {{$value}}", - description: "The alert value for {{$labels.instance}} is {{$value}}", - message: `[{"Description":"The alert value for testinstance is 0.3333333333333333","Labels":{"instance":"testinstance"},"Payload":{"ActiveSince":"0001-01-01T00:00:00Z","AlertingRule":"Test rule string","GeneratorUrl":"prometheus_url","Value":"0.333333"},"Summary":"testinstance = 0.3333333333333333"}]`, - }, - { - // Bad message referring to unknown label. - summary: "{{$labels.badlabel}} = {{$value}}", - description: "The alert value for {{$labels.badlabel}} is {{$value}}", - message: `[{"Description":"The alert value for \u003cno value\u003e is 0.3333333333333333","Labels":{"instance":"testinstance"},"Payload":{"ActiveSince":"0001-01-01T00:00:00Z","AlertingRule":"Test rule string","GeneratorUrl":"prometheus_url","Value":"0.333333"},"Summary":"\u003cno value\u003e = 0.3333333333333333"}]`, - }, - { - // Bad message referring to unknown variable. - summary: "{{$labels.instance}} = {{$badvar}}", - description: "The alert value for {{$labels.instance}} is {{$badvar}}", - message: `[{"Description":"The alert value for {{$labels.instance}} is {{$badvar}}","Labels":{"instance":"testinstance"},"Payload":{"ActiveSince":"0001-01-01T00:00:00Z","AlertingRule":"Test rule string","GeneratorUrl":"prometheus_url","Value":"0.333333"},"Summary":"{{$labels.instance}} = {{$badvar}}"}]`, - }, - { - // Bad message referring to unknown struct field. - summary: "{{$labels.instance}} = {{.Val}}", - description: "The alert value for {{$labels.instance}} is {{.Val}}", - message: `[{"Description":"The alert value for {{$labels.instance}} is {{.Val}}","Labels":{"instance":"testinstance"},"Payload":{"ActiveSince":"0001-01-01T00:00:00Z","AlertingRule":"Test rule string","GeneratorUrl":"prometheus_url","Value":"0.333333"},"Summary":"{{$labels.instance}} = {{.Val}}"}]`, + summary: "Summary", + description: "Description", + message: `[{"Description":"Description","Labels":{"instance":"testinstance"},"Payload":{"ActiveSince":"0001-01-01T00:00:00Z","AlertingRule":"Test rule string","GeneratorUrl":"prometheus_url","Value":"0.333333"},"Summary":"Summary"}]`, }, } diff --git a/rules/alerting.go b/rules/alerting.go index f7391f7ee..e43867e82 100644 --- a/rules/alerting.go +++ b/rules/alerting.go @@ -96,7 +96,7 @@ type AlertingRule struct { // The name of the alert. name string // The vector expression from which to generate alerts. - vector ast.VectorNode + Vector ast.VectorNode // The duration for which a labelset needs to persist in the expression // output vector before an alert transitions from PENDING to FIRING state. holdDuration time.Duration @@ -119,7 +119,7 @@ func (rule *AlertingRule) Name() string { } func (rule *AlertingRule) EvalRaw(timestamp clientmodel.Timestamp, storage metric.PreloadingPersistence) (ast.Vector, error) { - return ast.EvalVectorInstant(rule.vector, timestamp, storage, stats.NewTimerGroup()) + return ast.EvalVectorInstant(rule.Vector, timestamp, storage, stats.NewTimerGroup()) } func (rule *AlertingRule) Eval(timestamp clientmodel.Timestamp, storage metric.PreloadingPersistence) (ast.Vector, error) { @@ -185,12 +185,12 @@ func (rule *AlertingRule) ToDotGraph() string { %#p[shape="box",label="ALERT %s IF FOR %s"]; %#p -> %#p; %s - }`, &rule, rule.name, utility.DurationToString(rule.holdDuration), &rule, rule.vector, rule.vector.NodeTreeToDotGraph()) + }`, &rule, rule.name, utility.DurationToString(rule.holdDuration), &rule, rule.Vector, rule.Vector.NodeTreeToDotGraph()) return graph } 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 { @@ -202,8 +202,8 @@ func (rule *AlertingRule) HTMLSnippet() template.HTML { `ALERT %s IF %s FOR %s WITH %s`, ConsoleLinkForExpression(alertMetric.String()), rule.name, - ConsoleLinkForExpression(rule.vector.String()), - rule.vector, + ConsoleLinkForExpression(rule.Vector.String()), + rule.Vector, utility.DurationToString(rule.holdDuration), rule.Labels)) } @@ -236,7 +236,7 @@ func (rule *AlertingRule) ActiveAlerts() []Alert { func NewAlertingRule(name string, vector ast.VectorNode, holdDuration time.Duration, labels clientmodel.LabelSet, summary string, description string) *AlertingRule { return &AlertingRule{ name: name, - vector: vector, + Vector: vector, holdDuration: holdDuration, Labels: labels, Summary: summary, diff --git a/rules/ast/printer.go b/rules/ast/printer.go index cbef99958..0fefcdd1c 100644 --- a/rules/ast/printer.go +++ b/rules/ast/printer.go @@ -15,6 +15,7 @@ package ast import ( "encoding/json" + "errors" "fmt" "sort" "strings" @@ -200,6 +201,36 @@ func EvalToString(node Node, timestamp clientmodel.Timestamp, format OutputForma panic("Switch didn't cover all node types") } +// EvalToVector evaluates the given node into a Vector. Matrices aren't supported. +func EvalToVector(node Node, timestamp clientmodel.Timestamp, storage metric.PreloadingPersistence, queryStats *stats.TimerGroup) (Vector, error) { + viewTimer := queryStats.GetTimer(stats.TotalViewBuildingTime).Start() + viewAdapter, err := viewAdapterForInstantQuery(node, timestamp, storage, queryStats) + viewTimer.Stop() + if err != nil { + panic(err) + } + + evalTimer := queryStats.GetTimer(stats.InnerEvalTime).Start() + switch node.Type() { + case SCALAR: + scalar := node.(ScalarNode).Eval(timestamp, viewAdapter) + evalTimer.Stop() + return Vector{&clientmodel.Sample{Value: scalar}}, nil + case VECTOR: + vector := node.(VectorNode).Eval(timestamp, viewAdapter) + evalTimer.Stop() + return vector, nil + case MATRIX: + return nil, errors.New("Matrices not supported by EvalToVector") + case STRING: + str := node.(StringNode).Eval(timestamp, viewAdapter) + evalTimer.Stop() + return Vector{&clientmodel.Sample{ + Metric: clientmodel.Metric{"__value__": clientmodel.LabelValue(str)}}}, nil + } + panic("Switch didn't cover all node types") +} + // NodeTreeToDotGraph returns a DOT representation of the scalar // literal. func (node *ScalarLiteral) NodeTreeToDotGraph() string { diff --git a/rules/manager.go b/rules/manager/manager.go similarity index 68% rename from rules/manager.go rename to rules/manager/manager.go index b138d0640..a980402e2 100644 --- a/rules/manager.go +++ b/rules/manager/manager.go @@ -11,7 +11,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -package rules +package manager import ( "fmt" @@ -25,7 +25,9 @@ import ( "github.com/prometheus/prometheus/config" "github.com/prometheus/prometheus/notification" + "github.com/prometheus/prometheus/rules" "github.com/prometheus/prometheus/storage/metric" + "github.com/prometheus/prometheus/templates" ) type RuleManager interface { @@ -36,15 +38,15 @@ type RuleManager interface { // Stop the rule manager's rule evaluation cycles. Stop() // Return all rules. - Rules() []Rule + Rules() []rules.Rule // Return all alerting rules. - AlertingRules() []*AlertingRule + AlertingRules() []*rules.AlertingRule } type ruleManager struct { // Protects the rules list. sync.Mutex - rules []Rule + rules []rules.Rule done chan bool @@ -69,7 +71,7 @@ type RuleManagerOptions struct { func NewRuleManager(o *RuleManagerOptions) RuleManager { manager := &ruleManager{ - rules: []Rule{}, + rules: []rules.Rule{}, done: make(chan bool), interval: o.EvaluationInterval, @@ -92,7 +94,7 @@ func (m *ruleManager) Run() { m.runIteration(m.results) iterationDuration.Add(map[string]string{intervalLabel: m.interval.String()}, float64(time.Since(start)/time.Millisecond)) case <-m.done: - glog.Info("Rule manager exiting...") + glog.Info("rules.Rule manager exiting...") return } } @@ -105,7 +107,7 @@ func (m *ruleManager) Stop() { } } -func (m *ruleManager) queueAlertNotifications(rule *AlertingRule) { +func (m *ruleManager) queueAlertNotifications(rule *rules.AlertingRule, timestamp clientmodel.Timestamp) { activeAlerts := rule.ActiveAlerts() if len(activeAlerts) == 0 { return @@ -113,21 +115,46 @@ func (m *ruleManager) queueAlertNotifications(rule *AlertingRule) { notifications := make(notification.NotificationReqs, 0, len(activeAlerts)) for _, aa := range activeAlerts { - if aa.State != FIRING { + if aa.State != rules.FIRING { // BUG: In the future, make AlertManager support pending alerts? continue } + // Provide the alert information to the template. + l := map[string]string{} + for k, v := range aa.Labels { + l[string(k)] = string(v) + } + tmplData := struct { + Labels map[string]string + Value clientmodel.SampleValue + }{ + Labels: l, + Value: aa.Value, + } + // Inject some convenience variables that are easier to remember for users + // who are not used to Go's templating system. + defs := "{{$labels := .Labels}}{{$value := .Value}}" + + expand := func(text string) string { + result, err := templates.Expand(defs+text, "__alert_"+rule.Name(), tmplData, timestamp, m.storage) + if err != nil { + result = err.Error() + glog.Warningf("Error expanding alert template %v with data '%v': %v", rule.Name(), tmplData, err) + } + return result + } + notifications = append(notifications, ¬ification.NotificationReq{ - Summary: rule.Summary, - Description: rule.Description, + Summary: expand(rule.Summary), + Description: expand(rule.Description), Labels: aa.Labels.Merge(clientmodel.LabelSet{ - AlertNameLabel: clientmodel.LabelValue(rule.Name()), + rules.AlertNameLabel: clientmodel.LabelValue(rule.Name()), }), Value: aa.Value, ActiveSince: aa.ActiveSince.Time(), RuleString: rule.String(), - GeneratorUrl: m.prometheusUrl + ConsoleLinkForExpression(rule.vector.String()), + GeneratorUrl: m.prometheusUrl + rules.ConsoleLinkForExpression(rule.Vector.String()), }) } m.notifications <- notifications @@ -138,14 +165,14 @@ func (m *ruleManager) runIteration(results chan<- *extraction.Result) { wg := sync.WaitGroup{} m.Lock() - rules := make([]Rule, len(m.rules)) - copy(rules, m.rules) + rulesSnapshot := make([]rules.Rule, len(m.rules)) + copy(rulesSnapshot, m.rules) m.Unlock() - for _, rule := range rules { + for _, rule := range rulesSnapshot { wg.Add(1) // BUG(julius): Look at fixing thundering herd. - go func(rule Rule) { + go func(rule rules.Rule) { defer wg.Done() start := time.Now() @@ -160,10 +187,10 @@ func (m *ruleManager) runIteration(results chan<- *extraction.Result) { } switch r := rule.(type) { - case *AlertingRule: - m.queueAlertNotifications(r) + case *rules.AlertingRule: + m.queueAlertNotifications(r, now) recordOutcome(alertingRuleType, duration) - case *RecordingRule: + case *rules.RecordingRule: recordOutcome(recordingRuleType, duration) default: panic(fmt.Sprintf("Unknown rule type: %T", rule)) @@ -176,7 +203,7 @@ func (m *ruleManager) runIteration(results chan<- *extraction.Result) { func (m *ruleManager) AddRulesFromConfig(config config.Config) error { for _, ruleFile := range config.Global.RuleFile { - newRules, err := LoadRulesFromFile(ruleFile) + newRules, err := rules.LoadRulesFromFile(ruleFile) if err != nil { return fmt.Errorf("%s: %s", ruleFile, err) } @@ -187,22 +214,22 @@ func (m *ruleManager) AddRulesFromConfig(config config.Config) error { return nil } -func (m *ruleManager) Rules() []Rule { +func (m *ruleManager) Rules() []rules.Rule { m.Lock() defer m.Unlock() - rules := make([]Rule, len(m.rules)) + rules := make([]rules.Rule, len(m.rules)) copy(rules, m.rules) return rules } -func (m *ruleManager) AlertingRules() []*AlertingRule { +func (m *ruleManager) AlertingRules() []*rules.AlertingRule { m.Lock() defer m.Unlock() - alerts := []*AlertingRule{} + alerts := []*rules.AlertingRule{} for _, rule := range m.rules { - if alertingRule, ok := rule.(*AlertingRule); ok { + if alertingRule, ok := rule.(*rules.AlertingRule); ok { alerts = append(alerts, alertingRule) } } diff --git a/rules/telemetry.go b/rules/manager/telemetry.go similarity index 99% rename from rules/telemetry.go rename to rules/manager/telemetry.go index c644eb75f..dfcd761e6 100644 --- a/rules/telemetry.go +++ b/rules/manager/telemetry.go @@ -11,7 +11,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -package rules +package manager import ( "time" diff --git a/rules/rules_test.go b/rules/rules_test.go index e25e4af7c..22ddea906 100644 --- a/rules/rules_test.go +++ b/rules/rules_test.go @@ -62,37 +62,8 @@ func (t testTieredStorageCloser) Close() { t.directory.Close() } -// This is copied from storage/metric/helpers_test.go, which is unfortunate but -// presently required to make things work. -func NewTestTieredStorage(t testing.TB) (storage *tiered.TieredStorage, closer test.Closer) { - var directory test.TemporaryDirectory - directory = test.NewTemporaryDirectory("test_tiered_storage", t) - storage, err := tiered.NewTieredStorage(2500, 1000, 5*time.Second, 0*time.Second, directory.Path()) - - if err != nil { - if storage != nil { - storage.Close() - } - directory.Close() - t.Fatalf("Error creating storage: %s", err) - } - - if storage == nil { - directory.Close() - t.Fatalf("storage == nil") - } - started := make(chan bool) - go storage.Serve(started) - <-started - closer = &testTieredStorageCloser{ - storage: storage, - directory: directory, - } - return -} - func newTestStorage(t testing.TB) (storage *tiered.TieredStorage, closer test.Closer) { - storage, closer = NewTestTieredStorage(t) + storage, closer = tiered.NewTestTieredStorage(t) if storage == nil { t.Fatal("storage == nil") } diff --git a/storage/metric/tiered/helpers_test.go b/storage/metric/tiered/test_helpers.go similarity index 100% rename from storage/metric/tiered/helpers_test.go rename to storage/metric/tiered/test_helpers.go diff --git a/storage/raw/leveldb/test/fixtures.go b/storage/raw/leveldb/test/fixtures.go index d04339f46..f6d96d95c 100644 --- a/storage/raw/leveldb/test/fixtures.go +++ b/storage/raw/leveldb/test/fixtures.go @@ -15,6 +15,7 @@ package test import ( "testing" + "code.google.com/p/goprotobuf/proto" "github.com/prometheus/prometheus/storage/raw/leveldb" diff --git a/templates/templates.go b/templates/templates.go new file mode 100644 index 000000000..3a0eb8cab --- /dev/null +++ b/templates/templates.go @@ -0,0 +1,110 @@ +// 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 templates + +import ( + "bytes" + "errors" + "fmt" + "text/template" + + clientmodel "github.com/prometheus/client_golang/model" + + "github.com/prometheus/prometheus/rules" + "github.com/prometheus/prometheus/rules/ast" + "github.com/prometheus/prometheus/stats" + "github.com/prometheus/prometheus/storage/metric" +) + +// A version of vector that's easier to use from templates. +type sample struct { + Labels map[string]string + Value float64 +} +type queryResult []*sample + +// Expand a template, using the given data, time and storage. +func Expand(text string, name string, data interface{}, timestamp clientmodel.Timestamp, storage metric.PreloadingPersistence) (result string, resultErr error) { + + // It'd better to have no alert description than to kill the whole process + // if there's a bug in the template. Similarly with console templates. + defer func() { + if r := recover(); r != nil { + var ok bool + resultErr, ok = r.(error) + if !ok { + resultErr = fmt.Errorf("Panic expanding template: %v", r) + } + } + }() + + funcMap := template.FuncMap{ + "query": func(q string) (queryResult, error) { + return query(q, timestamp, storage) + }, + "first": func(v queryResult) (*sample, error) { + if len(v) > 0 { + return v[0], nil + } + return nil, errors.New("first() called on vector with no elements") + }, + "label": func(label string, s *sample) string { + return s.Labels[label] + }, + "value": func(s *sample) float64 { + return s.Value + }, + "strvalue": func(s *sample) string { + return s.Labels["__value__"] + }, + } + + var buffer bytes.Buffer + tmpl, err := template.New(name).Funcs(funcMap).Parse(text) + if err != nil { + return "", fmt.Errorf("Error parsing template %v: %v", name, err) + } + err = tmpl.Execute(&buffer, data) + if err != nil { + return "", fmt.Errorf("Error executing template %v: %v", name, err) + } + return buffer.String(), nil +} + +func query(q string, timestamp clientmodel.Timestamp, storage metric.PreloadingPersistence) (queryResult, error) { + exprNode, err := rules.LoadExprFromString(q) + if err != nil { + return nil, err + } + queryStats := stats.NewTimerGroup() + vector, err := ast.EvalToVector(exprNode, timestamp, storage, queryStats) + if err != nil { + return nil, err + } + + // ast.Vector is hard to work with in templates, so convert to + // base data types. + var result = make(queryResult, len(vector)) + for n, v := range vector { + s := sample{ + Value: float64(v.Value), + Labels: make(map[string]string), + } + for label, value := range v.Metric { + s.Labels[string(label)] = string(value) + } + result[n] = &s + } + return result, nil +} diff --git a/templates/templates_test.go b/templates/templates_test.go new file mode 100644 index 000000000..7b789973c --- /dev/null +++ b/templates/templates_test.go @@ -0,0 +1,109 @@ +// Copyright 2014 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 templates + +import ( + "testing" + + clientmodel "github.com/prometheus/client_golang/model" + + "github.com/prometheus/prometheus/storage/metric/tiered" +) + +type testTemplatesScenario struct { + text string + output string + shouldFail bool +} + +func TestTemplateExpansion(t *testing.T) { + scenarios := []testTemplatesScenario{ + { + // No template. + text: "plain text", + output: "plain text", + }, + { + // Simple value. + text: "{{ 1 }}", + output: "1", + }, + { + // Get value from query. + text: "{{ query \"metric{instance='a'}\" | first | value }}", + output: "11", + }, + { + // Get label from query. + text: "{{ query \"metric{instance='a'}\" | first | label \"instance\" }}", + output: "a", + }, + { + // Range over query. + text: "{{ range query \"metric\" }}{{.Labels.instance}}:{{.Value}}: {{end}}", + output: "a:11: b:21: ", + }, + { + // Unparsable template. + text: "{{", + shouldFail: true, + }, + { + // Error in function. + text: "{{ query \"missing\" | first }}", + shouldFail: true, + }, + { + // Panic. + text: "{{ (query \"missing\").banana }}", + shouldFail: true, + }, + } + + time := clientmodel.Timestamp(0) + + ts, _ := tiered.NewTestTieredStorage(t) + ts.AppendSamples(clientmodel.Samples{ + { + Metric: clientmodel.Metric{ + clientmodel.MetricNameLabel: "metric", + "instance": "a"}, + Value: 11, + }, + { + Metric: clientmodel.Metric{ + clientmodel.MetricNameLabel: "metric", + "instance": "b"}, + Value: 21, + }, + }) + + for _, s := range scenarios { + result, err := Expand(s.text, "test", nil, time, ts) + if s.shouldFail { + if err == nil { + t.Fatalf("Error not returned from %v", s.text) + } + continue + } + if err != nil { + t.Fatalf("Error returned from %v: %v", s.text, err) + continue + } + if result != s.output { + t.Fatalf("Error in result from %v: Expected '%v' Got '%v'", s.text, s.output, result) + continue + } + } +} diff --git a/web/alerts.go b/web/alerts.go index 5e88024ae..54db2e2fd 100644 --- a/web/alerts.go +++ b/web/alerts.go @@ -14,10 +14,12 @@ package web import ( - "github.com/prometheus/prometheus/rules" "net/http" "sort" "sync" + + "github.com/prometheus/prometheus/rules" + "github.com/prometheus/prometheus/rules/manager" ) type AlertStatus struct { @@ -26,7 +28,7 @@ type AlertStatus struct { } type AlertsHandler struct { - RuleManager rules.RuleManager + RuleManager manager.RuleManager mutex sync.Mutex } diff --git a/web/consoles.go b/web/consoles.go new file mode 100644 index 000000000..f352da6f8 --- /dev/null +++ b/web/consoles.go @@ -0,0 +1,74 @@ +// Copyright 2014 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 web + +import ( + "flag" + "io" + "io/ioutil" + "net/http" + "net/url" + + clientmodel "github.com/prometheus/client_golang/model" + "github.com/prometheus/prometheus/storage/metric" + "github.com/prometheus/prometheus/templates" +) + +var ( + consoleTemplatesPath = flag.String("consoleTemplates", "consoles", "Path to console template directory, available at /console") +) + +type ConsolesHandler struct { + Storage metric.PreloadingPersistence +} + +func (h *ConsolesHandler) ServeHTTP(w http.ResponseWriter, r *http.Request) { + file, err := http.Dir(*consoleTemplatesPath).Open(r.URL.Path) + if err != nil { + http.Error(w, err.Error(), http.StatusNotFound) + return + } + text, err := ioutil.ReadAll(file) + if err != nil { + http.Error(w, err.Error(), http.StatusInternalServerError) + return + } + + // Provide URL parameters as a map for easy use. Advanced users may have need for + // parameters beyond the first, so provide RawParams. + rawParams, err := url.ParseQuery(r.URL.RawQuery) + if err != nil { + http.Error(w, err.Error(), http.StatusBadRequest) + return + } + params := map[string]string{} + for k, v := range rawParams { + params[k] = v[0] + } + data := struct { + RawParams url.Values + Params map[string]string + }{ + RawParams: rawParams, + Params: params, + } + + now := clientmodel.Now() + result, err := templates.Expand(string(text), "__console_"+r.URL.Path, data, now, h.Storage) + if err != nil { + http.Error(w, err.Error(), http.StatusInternalServerError) + return + } + io.WriteString(w, result) +} diff --git a/web/status.go b/web/status.go index e77fbd193..37b5c1e54 100644 --- a/web/status.go +++ b/web/status.go @@ -19,7 +19,7 @@ import ( "time" "github.com/prometheus/prometheus/retrieval" - "github.com/prometheus/prometheus/rules" + "github.com/prometheus/prometheus/rules/manager" "github.com/prometheus/prometheus/storage/metric" ) @@ -30,7 +30,7 @@ type PrometheusStatusHandler struct { Config string Curation metric.CurationState Flags map[string]string - RuleManager rules.RuleManager + RuleManager manager.RuleManager TargetPools map[string]*retrieval.TargetPool Birth time.Time diff --git a/web/web.go b/web/web.go index b4d9530dc..26d3e0a57 100644 --- a/web/web.go +++ b/web/web.go @@ -46,6 +46,7 @@ type WebService struct { DatabasesHandler *DatabasesHandler MetricsHandler *api.MetricsService AlertsHandler *AlertsHandler + ConsolesHandler *ConsolesHandler QuitDelegate func() } @@ -65,6 +66,7 @@ func (w WebService) ServeForever() error { exp.Handle("/", w.StatusHandler) exp.Handle("/databases", w.DatabasesHandler) exp.Handle("/alerts", w.AlertsHandler) + exp.Handle("/consoles/", http.StripPrefix("/consoles/", w.ConsolesHandler)) exp.HandleFunc("/graph", graphHandler) exp.HandleFunc("/heap", dumpHeap)