From 7504b5ce7ccb5f38413a5342e6c1ec01da5ef99b Mon Sep 17 00:00:00 2001 From: jessicagreben Date: Mon, 27 Jul 2020 07:44:49 -0700 Subject: [PATCH 01/21] add rule importer with tsdb block writer Signed-off-by: jessicagreben --- cmd/promtool/main.go | 54 +++++++ importers/rules.go | 256 +++++++++++++++++++++++++++++++++ tsdb/importer/blocks/multi.go | 120 ++++++++++++++++ tsdb/importer/blocks/writer.go | 139 ++++++++++++++++++ 4 files changed, 569 insertions(+) create mode 100644 importers/rules.go create mode 100644 tsdb/importer/blocks/multi.go create mode 100644 tsdb/importer/blocks/writer.go diff --git a/cmd/promtool/main.go b/cmd/promtool/main.go index f99f63cdd..edddaf8b0 100644 --- a/cmd/promtool/main.go +++ b/cmd/promtool/main.go @@ -28,6 +28,7 @@ import ( "strings" "time" + "github.com/go-kit/kit/log" "github.com/google/pprof/profile" "github.com/pkg/errors" "github.com/prometheus/client_golang/api" @@ -40,6 +41,7 @@ import ( "gopkg.in/alecthomas/kingpin.v2" "github.com/prometheus/prometheus/config" + "github.com/prometheus/prometheus/importers" "github.com/prometheus/prometheus/pkg/rulefmt" ) @@ -128,6 +130,20 @@ func main() { dumpMinTime := tsdbDumpCmd.Flag("min-time", "Minimum timestamp to dump.").Default(strconv.FormatInt(math.MinInt64, 10)).Int64() dumpMaxTime := tsdbDumpCmd.Flag("max-time", "Maximum timestamp to dump.").Default(strconv.FormatInt(math.MaxInt64, 10)).Int64() + backfillCmd := app.Command("backfill", "Backfill Prometheus data.") + backfillRuleCmd := backfillCmd.Command("rules", "Backfill Prometheus data for new rules.") + backfillRuleURL := backfillRuleCmd.Flag("url", "Prometheus API url.").Required().String() + backfillRuleEvalInterval := backfillRuleCmd.Flag("evaluation_interval", "How frequently to evaluate rules when backfilling."). + Default("-3h").Duration() + backfillRuleStart := backfillRuleCmd.Flag("start", "The time to start backfilling the new rule from. It is required. Start time should be RFC3339 or Unix timestamp."). + Required().Duration() + backfillRuleEnd := backfillRuleCmd.Flag("end", "If an end time is provided, the new rule backfilling will end at this time. The default will backfill to the 3 hrs ago. End time should be RFC3339 or Unix timestamp."). + Default("").Duration() + backfillRuleFiles := backfillRuleCmd.Arg( + "rule-files", + "The file containing the new rule that needs to be backfilled.", + ).Required().ExistingFiles() + parsedCmd := kingpin.MustParse(app.Parse(os.Args[1:])) var p printer @@ -183,6 +199,9 @@ func main() { case tsdbDumpCmd.FullCommand(): os.Exit(checkErr(dumpSamples(*dumpPath, *dumpMinTime, *dumpMaxTime))) + + case backfillRuleCmd.FullCommand(): + os.Exit(BackfillRule(*backfillRuleURL, *backfillRuleStart, *backfillRuleEnd, *backfillRuleEvalInterval, *backfillRuleFiles...)) } } @@ -747,3 +766,38 @@ func (j *jsonPrinter) printLabelValues(v model.LabelValues) { //nolint:errcheck json.NewEncoder(os.Stdout).Encode(v) } + +// BackfillRule backfills rules from the files provided +func BackfillRule(url string, start, end, evalInterval time.Duration, files ...string) int { + ctx := context.Background() + cfg := importers.RuleConfig{ + Start: start.String(), + End: end.String(), + EvalInterval: evalInterval, + URL: url, + } + logger := log.NewLogfmtLogger(log.NewSyncWriter(os.Stderr)) + ruleImporter := importers.NewRuleImporter(logger, cfg) + err := ruleImporter.Init() + if err != nil { + fmt.Fprintln(os.Stderr, "rule importer init error", err) + return 1 + } + + errs := ruleImporter.Parse(ctx, files) + for _, err := range errs { + if err != nil { + fmt.Fprintln(os.Stderr, "rule importer parse error", err) + return 1 + } + } + + errs = ruleImporter.ImportAll(ctx) + for _, err := range errs { + if err != nil { + fmt.Fprintln(os.Stderr, "rule importer error", err) + } + } + + return 0 +} diff --git a/importers/rules.go b/importers/rules.go new file mode 100644 index 000000000..a103eadb1 --- /dev/null +++ b/importers/rules.go @@ -0,0 +1,256 @@ +// Copyright 2020 The Prometheus Authors +// 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 importers + +import ( + "context" + "fmt" + "io/ioutil" + "math" + "net/url" + "os" + "sort" + "strconv" + "time" + + "github.com/go-kit/kit/log" + "github.com/pkg/errors" + "github.com/prometheus/client_golang/api" + v1 "github.com/prometheus/client_golang/api/prometheus/v1" + "github.com/prometheus/common/model" + "github.com/prometheus/prometheus/pkg/labels" + plabels "github.com/prometheus/prometheus/pkg/labels" + "github.com/prometheus/prometheus/pkg/rulefmt" + "github.com/prometheus/prometheus/promql" + "github.com/prometheus/prometheus/promql/parser" + "github.com/prometheus/prometheus/rules" + "github.com/prometheus/prometheus/tsdb/importer/blocks" +) + +// RuleImporter is the importer for rules +type RuleImporter struct { + logger log.Logger + + config RuleConfig + groups map[string]*rules.Group + + apiClient v1.API + + writer *blocks.MultiWriter +} + +// RuleConfig is the config for the rule importer +type RuleConfig struct { + Start string + End string + EvalInterval time.Duration + URL string +} + +// NewRuleImporter creates a new rule importer +func NewRuleImporter(logger log.Logger, config RuleConfig) *RuleImporter { + return &RuleImporter{ + config: config, + } +} + +// Init initializes the rule importer which creates a new block writer +// and creates an Prometheus API client +func (importer *RuleImporter) Init() error { + // create new block writer + newBlockDir, err := ioutil.TempDir("", "rule_blocks") + if err != nil { + return err + } + importer.writer = blocks.NewMultiWriter(importer.logger, newBlockDir, importer.config.EvalInterval.Nanoseconds()) + + // create api client + config := api.Config{ + Address: importer.config.URL, + } + c, err := api.NewClient(config) + if err != nil { + return err + } + importer.apiClient = v1.NewAPI(c) + return nil +} + +// Close cleans up any open resources +func (importer *RuleImporter) Close() error { + return importer.writer.Close() +} + +// Parse parses the groups and rules from a list of rules files +func (importer *RuleImporter) Parse(ctx context.Context, files []string) (errs []error) { + groups := make(map[string]*rules.Group) + + for _, file := range files { + ruleGroups, errs := rulefmt.ParseFile(file) + if errs != nil { + return errs + } + + for _, ruleGroup := range ruleGroups.Groups { + itv := importer.config.EvalInterval + if ruleGroup.Interval != 0 { + itv = time.Duration(ruleGroup.Interval) + } + + rulez := make([]rules.Rule, 0, len(ruleGroup.Rules)) + for _, r := range ruleGroup.Rules { + expr, err := parser.ParseExpr(r.Expr.Value) + if err != nil { + return []error{errors.Wrap(err, file)} + } + + rulez = append(rulez, rules.NewRecordingRule( + r.Record.Value, + expr, + labels.FromMap(r.Labels), + )) + } + + groups[file+";"+ruleGroup.Name] = rules.NewGroup(rules.GroupOptions{ + Name: ruleGroup.Name, + File: file, + Interval: itv, + Rules: rulez, + }) + } + } + + importer.groups = groups + return errs +} + +// ImportAll evaluates all the groups and rules and creates new time series +// and stores in new blocks +func (importer *RuleImporter) ImportAll(ctx context.Context) []error { + var errs = []error{} + for _, group := range importer.groups { + for _, rule := range group.Rules() { + err := importer.ImportRule(ctx, rule) + if err != nil { + errs = append(errs, err) + } + } + } + err := importer.CreateBlocks() + if err != nil { + errs = append(errs, err) + } + return errs +} + +func (importer *RuleImporter) queryFn(ctx context.Context, q string, t time.Time) (promql.Vector, error) { + val, warnings, err := importer.apiClient.Query(ctx, q, t) + if err != nil { + return promql.Vector{}, err + } + if warnings != nil { + fmt.Fprint(os.Stderr, "warning api.Query:", warnings) + } + + switch val.Type() { + case model.ValVector: + valVector := val.(model.Vector) + return modelToPromqlVector(valVector), nil + case model.ValScalar: + valScalar := val.(*model.Scalar) + return promql.Vector{promql.Sample{ + Metric: labels.Labels{}, + Point: promql.Point{T: int64(valScalar.Timestamp), V: float64(valScalar.Value)}, + }}, nil + default: + return nil, errors.New("rule result is wrong type") + } +} + +func modelToPromqlVector(modelValue model.Vector) promql.Vector { + result := make(promql.Vector, 0, len(modelValue)) + + for _, value := range modelValue { + labels := make(labels.Labels, 0, len(value.Metric)) + + for k, v := range value.Metric { + labels = append(labels, plabels.Label{ + Name: string(k), + Value: string(v), + }) + } + sort.Sort(labels) + + result = append(result, promql.Sample{ + Metric: labels, + Point: promql.Point{T: int64(value.Timestamp), V: float64(value.Value)}, + }) + } + return result +} + +// ImportRule imports the historical data for a single rule +func (importer *RuleImporter) ImportRule(ctx context.Context, rule rules.Rule) error { + ts, err := parseTime(importer.config.Start) + if err != nil { + return err + } + end, err := parseTime(importer.config.End) + if err != nil { + return err + } + url, err := url.Parse(importer.config.URL) + if err != nil { + return err + } + + appender := importer.writer.Appender() + for ts.Before(end) { + vector, err := rule.Eval(ctx, ts, importer.queryFn, url) + if err != nil { + return err + } + for _, sample := range vector { + // we don't AddFast here because we need to maintain the + // ref for each series bcs rule.Eval could return different labels, + // so that means you would need to map the ref to metric, but that is what Add does + // anyways so just use that + _, err := appender.Add(plabels.Labels{plabels.Label{Name: sample.String()}}, sample.T, sample.V) + if err != nil { + return err + } + } + + ts.Add(importer.config.EvalInterval) + // todo: 2 hr blocks? + } + return appender.Commit() +} + +func parseTime(s string) (time.Time, error) { + if t, err := strconv.ParseFloat(s, 64); err == nil { + s, ns := math.Modf(t) + return time.Unix(int64(s), int64(ns*float64(time.Second))).UTC(), nil + } + if t, err := time.Parse(time.RFC3339Nano, s); err == nil { + return t, nil + } + return time.Time{}, errors.Errorf("cannot parse %q to a valid timestamp", s) +} + +// CreateBlocks creates blocks for all the rule data +func (importer *RuleImporter) CreateBlocks() error { + _, err := importer.writer.Flush() + return err +} diff --git a/tsdb/importer/blocks/multi.go b/tsdb/importer/blocks/multi.go new file mode 100644 index 000000000..4007860d8 --- /dev/null +++ b/tsdb/importer/blocks/multi.go @@ -0,0 +1,120 @@ +// Copyright 2020 The Prometheus Authors +// 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 blocks + +import ( + "github.com/go-kit/kit/log" + "github.com/oklog/ulid" + "github.com/pkg/errors" + "github.com/prometheus/prometheus/pkg/labels" + "github.com/prometheus/prometheus/storage" + tsdb_errors "github.com/prometheus/prometheus/tsdb/errors" + "github.com/prometheus/prometheus/tsdb/index" +) + +type errAppender struct{ err error } + +func (a errAppender) Add(l labels.Labels, t int64, v float64) (uint64, error) { return 0, a.err } +func (a errAppender) AddFast(ref uint64, t int64, v float64) error { return a.err } +func (a errAppender) Commit() error { return a.err } +func (a errAppender) Rollback() error { return a.err } + +func rangeForTimestamp(t int64, width int64) (maxt int64) { + return (t/width)*width + width +} + +type MultiWriter struct { + blocks map[index.Range]Writer + activeAppenders map[index.Range]storage.Appender + + logger log.Logger + dir string + // TODO(bwplotka): Allow more complex compaction levels. + sizeMillis int64 +} + +func NewMultiWriter(logger log.Logger, dir string, sizeMillis int64) *MultiWriter { + return &MultiWriter{ + logger: logger, + dir: dir, + sizeMillis: sizeMillis, + blocks: map[index.Range]Writer{}, + activeAppenders: map[index.Range]storage.Appender{}, + } +} + +// Appender is not thread-safe. Returned Appender is not thread-save as well. +// TODO(bwplotka): Consider making it thread safe. +func (w *MultiWriter) Appender() storage.Appender { return w } + +func (w *MultiWriter) getOrCreate(t int64) storage.Appender { + maxt := rangeForTimestamp(t, w.sizeMillis) + hash := index.Range{Start: maxt - w.sizeMillis, End: maxt} + if a, ok := w.activeAppenders[hash]; ok { + return a + } + + nw, err := NewTSDBWriter(w.logger, w.dir) + if err != nil { + return errAppender{err: errors.Wrap(err, "new tsdb writer")} + } + + w.blocks[hash] = nw + w.activeAppenders[hash] = nw.Appender() + return w.activeAppenders[hash] +} + +func (w *MultiWriter) Add(l labels.Labels, t int64, v float64) (uint64, error) { + return w.getOrCreate(t).Add(l, t, v) +} + +func (w *MultiWriter) AddFast(ref uint64, t int64, v float64) error { + return w.getOrCreate(t).AddFast(ref, t, v) +} + +func (w *MultiWriter) Commit() error { + var merr tsdb_errors.MultiError + for _, a := range w.activeAppenders { + merr.Add(a.Commit()) + } + return merr.Err() +} + +func (w *MultiWriter) Rollback() error { + var merr tsdb_errors.MultiError + for _, a := range w.activeAppenders { + merr.Add(a.Rollback()) + } + return merr.Err() +} + +func (w *MultiWriter) Flush() ([]ulid.ULID, error) { + ids := make([]ulid.ULID, 0, len(w.blocks)) + for _, b := range w.blocks { + id, err := b.Flush() + if err != nil { + return nil, err + } + ids = append(ids, id...) + } + return ids, nil +} + +func (w *MultiWriter) Close() error { + var merr tsdb_errors.MultiError + for _, b := range w.blocks { + merr.Add(b.Close()) + } + return merr.Err() +} diff --git a/tsdb/importer/blocks/writer.go b/tsdb/importer/blocks/writer.go new file mode 100644 index 000000000..3af869e06 --- /dev/null +++ b/tsdb/importer/blocks/writer.go @@ -0,0 +1,139 @@ +// Copyright 2020 The Prometheus Authors +// 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 blocks + +import ( + "context" + "io/ioutil" + "math" + "os" + "time" + + "github.com/oklog/ulid" + "github.com/prometheus/prometheus/storage" + "github.com/prometheus/prometheus/tsdb" + + "github.com/go-kit/kit/log" + "github.com/go-kit/kit/log/level" + "github.com/pkg/errors" + "github.com/prometheus/prometheus/pkg/timestamp" + "github.com/prometheus/prometheus/tsdb/chunkenc" +) + +// Writer is interface to write time series into Prometheus blocks. +type Writer interface { + storage.Appendable + + // Flush writes current data to disk. + // The block or blocks will contain values accumulated by `Write`. + Flush() ([]ulid.ULID, error) + + // Close releases all resources. No append is allowed anymore to such writer. + Close() error +} + +var _ Writer = &TSDBWriter{} + +// Writer is a block writer that allows appending and flushing to disk. +type TSDBWriter struct { + logger log.Logger + dir string + + head *tsdb.Head + tmpDir string +} + +func durToMillis(t time.Duration) int64 { + return int64(t.Seconds() * 1000) +} + +// NewTSDBWriter create new block writer. +// +// The returned writer accumulates all series in memory until `Flush` is called. +// +// Note that the writer will not check if the target directory exists or +// contains anything at all. It is the caller's responsibility to +// ensure that the resulting blocks do not overlap etc. +// Writer ensures the block flush is atomic (via rename). +func NewTSDBWriter(logger log.Logger, dir string) (*TSDBWriter, error) { + res := &TSDBWriter{ + logger: logger, + dir: dir, + } + return res, res.initHead() +} + +// initHead creates and initialises new head. +func (w *TSDBWriter) initHead() error { + logger := w.logger + + // Keep Registerer and WAL nil as we don't use them. + // Put huge chunkRange; It has to be equal then expected block size. + // Since we don't have info about block size here, set it to large number. + + tmpDir, err := ioutil.TempDir(os.TempDir(), "head") + if err != nil { + return errors.Wrap(err, "create temp dir") + } + w.tmpDir = tmpDir + + h, err := tsdb.NewHead(nil, logger, nil, durToMillis(9999*time.Hour), w.tmpDir, nil, tsdb.DefaultStripeSize, nil) + if err != nil { + return errors.Wrap(err, "tsdb.NewHead") + } + + w.head = h + return w.head.Init(math.MinInt64) +} + +// Appender is not thread-safe. Returned Appender is thread-save however. +func (w *TSDBWriter) Appender() storage.Appender { + return w.head.Appender() +} + +// Flush implements Writer interface. This is where actual block writing +// happens. After flush completes, no write can be done. +func (w *TSDBWriter) Flush() ([]ulid.ULID, error) { + seriesCount := w.head.NumSeries() + if w.head.NumSeries() == 0 { + return nil, errors.New("no series appended; aborting.") + } + + mint := w.head.MinTime() + maxt := w.head.MaxTime() + 1 + level.Info(w.logger).Log("msg", "flushing", "series_count", seriesCount, "mint", timestamp.Time(mint), "maxt", timestamp.Time(maxt)) + + // Flush head to disk as a block. + compactor, err := tsdb.NewLeveledCompactor( + context.Background(), + nil, + w.logger, + []int64{durToMillis(2 * time.Hour)}, // Does not matter, used only for planning. + chunkenc.NewPool()) + if err != nil { + return nil, errors.Wrap(err, "create leveled compactor") + } + id, err := compactor.Write(w.dir, w.head, mint, maxt, nil) + if err != nil { + return nil, errors.Wrap(err, "compactor write") + } + // TODO(bwplotka): Potential truncate head, and allow writer reuse. Currently truncating fails with + // truncate chunks.HeadReadWriter: maxt of the files are not set. + return []ulid.ULID{id}, nil +} + +func (w *TSDBWriter) Close() error { + _ = os.RemoveAll(w.tmpDir) + return w.head.Close() +} From dfa510086be77f0a180362d35f7e16a079e911c9 Mon Sep 17 00:00:00 2001 From: jessicagreben Date: Sun, 13 Sep 2020 08:07:59 -0700 Subject: [PATCH 02/21] add alignment, mv rule importer to promtool dir, add queryRange Signed-off-by: jessicagreben --- cmd/promtool/main.go | 36 +++--- cmd/promtool/rules.go | 216 +++++++++++++++++++++++++++++++++++ importers/rules.go | 256 ------------------------------------------ rules/manager.go | 30 ++--- 4 files changed, 252 insertions(+), 286 deletions(-) create mode 100644 cmd/promtool/rules.go delete mode 100644 importers/rules.go diff --git a/cmd/promtool/main.go b/cmd/promtool/main.go index edddaf8b0..0c78d93e7 100644 --- a/cmd/promtool/main.go +++ b/cmd/promtool/main.go @@ -41,7 +41,6 @@ import ( "gopkg.in/alecthomas/kingpin.v2" "github.com/prometheus/prometheus/config" - "github.com/prometheus/prometheus/importers" "github.com/prometheus/prometheus/pkg/rulefmt" ) @@ -132,16 +131,16 @@ func main() { backfillCmd := app.Command("backfill", "Backfill Prometheus data.") backfillRuleCmd := backfillCmd.Command("rules", "Backfill Prometheus data for new rules.") - backfillRuleURL := backfillRuleCmd.Flag("url", "Prometheus API url.").Required().String() - backfillRuleEvalInterval := backfillRuleCmd.Flag("evaluation_interval", "How frequently to evaluate rules when backfilling."). - Default("-3h").Duration() backfillRuleStart := backfillRuleCmd.Flag("start", "The time to start backfilling the new rule from. It is required. Start time should be RFC3339 or Unix timestamp."). - Required().Duration() - backfillRuleEnd := backfillRuleCmd.Flag("end", "If an end time is provided, the new rule backfilling will end at this time. The default will backfill to the 3 hrs ago. End time should be RFC3339 or Unix timestamp."). - Default("").Duration() + Required().String() + backfillRuleEnd := backfillRuleCmd.Flag("end", "If an end time is provided, all recording rules in the rule files provided will be backfilled to the end time. Default will backfill up to 3 hrs ago. End time should be RFC3339 or Unix timestamp."). + Default("-3h").String() + backfillRuleURL := backfillRuleCmd.Flag("url", "Prometheus API url with the data where the rule will be backfilled from.").Default("localhost:9090").String() + backfillRuleEvalInterval := backfillRuleCmd.Flag("evaluation_interval", "How frequently to evaluate rules when backfilling."). + Default("15s").Duration() backfillRuleFiles := backfillRuleCmd.Arg( "rule-files", - "The file containing the new rule that needs to be backfilled.", + "A list of one or more files containing recording rules to be backfilled. All recording rules listed in the files will be backfilled. Alerting rules are not evaluated.", ).Required().ExistingFiles() parsedCmd := kingpin.MustParse(app.Parse(os.Args[1:])) @@ -767,24 +766,29 @@ func (j *jsonPrinter) printLabelValues(v model.LabelValues) { json.NewEncoder(os.Stdout).Encode(v) } -// BackfillRule backfills rules from the files provided -func BackfillRule(url string, start, end, evalInterval time.Duration, files ...string) int { +// BackfillRule backfills rules from the files provided. +func BackfillRule(url, start, end string, evalInterval time.Duration, files ...string) int { ctx := context.Background() - cfg := importers.RuleConfig{ - Start: start.String(), - End: end.String(), + stime, etime, err := parseStartTimeAndEndTime(start, end) + if err != nil { + fmt.Fprintln(os.Stderr, err) + return 1 + } + cfg := RuleImporterConfig{ + Start: stime, + End: etime, EvalInterval: evalInterval, URL: url, } logger := log.NewLogfmtLogger(log.NewSyncWriter(os.Stderr)) - ruleImporter := importers.NewRuleImporter(logger, cfg) - err := ruleImporter.Init() + ruleImporter := NewRuleImporter(logger, cfg) + err = ruleImporter.Init() if err != nil { fmt.Fprintln(os.Stderr, "rule importer init error", err) return 1 } - errs := ruleImporter.Parse(ctx, files) + errs := ruleImporter.LoadGroups(ctx, files) for _, err := range errs { if err != nil { fmt.Fprintln(os.Stderr, "rule importer parse error", err) diff --git a/cmd/promtool/rules.go b/cmd/promtool/rules.go new file mode 100644 index 000000000..52bf22688 --- /dev/null +++ b/cmd/promtool/rules.go @@ -0,0 +1,216 @@ +// Copyright 2020 The Prometheus Authors +// 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 main + +import ( + "context" + "fmt" + "io/ioutil" + "os" + "time" + + "github.com/go-kit/kit/log" + "github.com/pkg/errors" + "github.com/prometheus/client_golang/api" + v1 "github.com/prometheus/client_golang/api/prometheus/v1" + "github.com/prometheus/common/model" + "github.com/prometheus/prometheus/pkg/labels" + "github.com/prometheus/prometheus/rules" + "github.com/prometheus/prometheus/tsdb/importer/blocks" +) + +const blockSize = 2 // in hours + +// RuleImporter is the importer to backfill rules. +type RuleImporter struct { + logger log.Logger + config RuleImporterConfig + + groups map[string]*rules.Group + groupLoader rules.GroupLoader + + apiClient v1.API + + writer *blocks.MultiWriter +} + +// RuleImporterConfig is the config for the rule importer. +type RuleImporterConfig struct { + Start time.Time + End time.Time + EvalInterval time.Duration + URL string +} + +// NewRuleImporter creates a new rule importer that can be used to backfill rules. +func NewRuleImporter(logger log.Logger, config RuleImporterConfig) *RuleImporter { + return &RuleImporter{ + config: config, + groupLoader: rules.FileLoader{}, + } +} + +// Init initializes the rule importer which creates a new block writer +// and creates an Prometheus API client. +func (importer *RuleImporter) Init() error { + // todo: clean up dir + newBlockDir, err := ioutil.TempDir("", "rule_blocks") + if err != nil { + return err + } + importer.writer = blocks.NewMultiWriter(importer.logger, newBlockDir, importer.config.EvalInterval.Nanoseconds()) + + config := api.Config{ + Address: importer.config.URL, + } + c, err := api.NewClient(config) + if err != nil { + return err + } + importer.apiClient = v1.NewAPI(c) + return nil +} + +// Close cleans up any open resources. +func (importer *RuleImporter) Close() error { + // todo: clean up any dirs that were created + return importer.writer.Close() +} + +// LoadGroups reads groups from a list of rule files. +func (importer *RuleImporter) LoadGroups(ctx context.Context, filenames []string) (errs []error) { + groups := make(map[string]*rules.Group) + + for _, filename := range filenames { + rgs, errs := importer.groupLoader.Load(filename) + if errs != nil { + return errs + } + + for _, ruleGroup := range rgs.Groups { + + itv := importer.config.EvalInterval + if ruleGroup.Interval != 0 { + itv = time.Duration(ruleGroup.Interval) + } + + rgRules := make([]rules.Rule, 0, len(ruleGroup.Rules)) + + for _, r := range ruleGroup.Rules { + + expr, err := importer.groupLoader.Parse(r.Expr.Value) + if err != nil { + return []error{errors.Wrap(err, filename)} + } + + rgRules = append(rgRules, rules.NewRecordingRule( + r.Record.Value, + expr, + labels.FromMap(r.Labels), + )) + } + + groups[rules.GroupKey(filename, ruleGroup.Name)] = rules.NewGroup(rules.GroupOptions{ + Name: ruleGroup.Name, + File: filename, + Interval: itv, + Rules: rgRules, + }) + } + } + + importer.groups = groups + return nil +} + +// ImportAll evaluates all the groups and rules and creates new time series +// and stores them in new blocks. +func (importer *RuleImporter) ImportAll(ctx context.Context) []error { + var errs = []error{} + for _, group := range importer.groups { + stimeWithAlignment := group.EvalTimestamp(importer.config.Start.UnixNano()) + + for _, r := range group.Rules() { + err := importer.ImportRule(ctx, r.Query().String(), stimeWithAlignment, group.Interval()) + if err != nil { + errs = append(errs, err) + } + } + } + _, err := importer.writer.Flush() + if err != nil { + errs = append(errs, err) + } + return errs +} + +// ImportRule imports the historical data for a single rule. +func (importer *RuleImporter) ImportRule(ctx context.Context, ruleExpr string, stimeWithAlignment time.Time, internval time.Duration) error { + ts := stimeWithAlignment + + appender := importer.writer.Appender() + + for ts.Before(importer.config.End) { + currentBlockEnd := ts.Add(blockSize * time.Hour) + if currentBlockEnd.After(importer.config.End) { + currentBlockEnd = importer.config.End + } + + val, warnings, err := importer.apiClient.QueryRange(ctx, + ruleExpr, + v1.Range{ + Start: ts, + End: currentBlockEnd, + Step: importer.config.EvalInterval, + }, + ) + if err != nil { + return err + } + if warnings != nil { + fmt.Fprint(os.Stderr, "warning api.QueryRange:", warnings) + } + + var matrix model.Matrix + switch val.Type() { + case model.ValMatrix: + matrix = val.(model.Matrix) + for _, sample := range matrix { + currentLabels := make(labels.Labels, 0, len(sample.Metric)) + for k, v := range sample.Metric { + currentLabels = append(currentLabels, labels.Label{ + Name: string(k), + Value: string(v), + }) + } + for _, value := range sample.Values { + _, err := appender.Add(currentLabels, value.Timestamp.Unix(), float64(value.Value)) + if err != nil { + // todo: handle other errors, i.e. ErrOutOfOrderSample and ErrDuplicateSampleForTimestamp + return err + } + } + } + default: + return errors.New("rule result is wrong type") + } + + ts = currentBlockEnd + } + _, err := importer.writer.Flush() + if err != nil { + return err + } + return appender.Commit() +} diff --git a/importers/rules.go b/importers/rules.go deleted file mode 100644 index a103eadb1..000000000 --- a/importers/rules.go +++ /dev/null @@ -1,256 +0,0 @@ -// Copyright 2020 The Prometheus Authors -// 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 importers - -import ( - "context" - "fmt" - "io/ioutil" - "math" - "net/url" - "os" - "sort" - "strconv" - "time" - - "github.com/go-kit/kit/log" - "github.com/pkg/errors" - "github.com/prometheus/client_golang/api" - v1 "github.com/prometheus/client_golang/api/prometheus/v1" - "github.com/prometheus/common/model" - "github.com/prometheus/prometheus/pkg/labels" - plabels "github.com/prometheus/prometheus/pkg/labels" - "github.com/prometheus/prometheus/pkg/rulefmt" - "github.com/prometheus/prometheus/promql" - "github.com/prometheus/prometheus/promql/parser" - "github.com/prometheus/prometheus/rules" - "github.com/prometheus/prometheus/tsdb/importer/blocks" -) - -// RuleImporter is the importer for rules -type RuleImporter struct { - logger log.Logger - - config RuleConfig - groups map[string]*rules.Group - - apiClient v1.API - - writer *blocks.MultiWriter -} - -// RuleConfig is the config for the rule importer -type RuleConfig struct { - Start string - End string - EvalInterval time.Duration - URL string -} - -// NewRuleImporter creates a new rule importer -func NewRuleImporter(logger log.Logger, config RuleConfig) *RuleImporter { - return &RuleImporter{ - config: config, - } -} - -// Init initializes the rule importer which creates a new block writer -// and creates an Prometheus API client -func (importer *RuleImporter) Init() error { - // create new block writer - newBlockDir, err := ioutil.TempDir("", "rule_blocks") - if err != nil { - return err - } - importer.writer = blocks.NewMultiWriter(importer.logger, newBlockDir, importer.config.EvalInterval.Nanoseconds()) - - // create api client - config := api.Config{ - Address: importer.config.URL, - } - c, err := api.NewClient(config) - if err != nil { - return err - } - importer.apiClient = v1.NewAPI(c) - return nil -} - -// Close cleans up any open resources -func (importer *RuleImporter) Close() error { - return importer.writer.Close() -} - -// Parse parses the groups and rules from a list of rules files -func (importer *RuleImporter) Parse(ctx context.Context, files []string) (errs []error) { - groups := make(map[string]*rules.Group) - - for _, file := range files { - ruleGroups, errs := rulefmt.ParseFile(file) - if errs != nil { - return errs - } - - for _, ruleGroup := range ruleGroups.Groups { - itv := importer.config.EvalInterval - if ruleGroup.Interval != 0 { - itv = time.Duration(ruleGroup.Interval) - } - - rulez := make([]rules.Rule, 0, len(ruleGroup.Rules)) - for _, r := range ruleGroup.Rules { - expr, err := parser.ParseExpr(r.Expr.Value) - if err != nil { - return []error{errors.Wrap(err, file)} - } - - rulez = append(rulez, rules.NewRecordingRule( - r.Record.Value, - expr, - labels.FromMap(r.Labels), - )) - } - - groups[file+";"+ruleGroup.Name] = rules.NewGroup(rules.GroupOptions{ - Name: ruleGroup.Name, - File: file, - Interval: itv, - Rules: rulez, - }) - } - } - - importer.groups = groups - return errs -} - -// ImportAll evaluates all the groups and rules and creates new time series -// and stores in new blocks -func (importer *RuleImporter) ImportAll(ctx context.Context) []error { - var errs = []error{} - for _, group := range importer.groups { - for _, rule := range group.Rules() { - err := importer.ImportRule(ctx, rule) - if err != nil { - errs = append(errs, err) - } - } - } - err := importer.CreateBlocks() - if err != nil { - errs = append(errs, err) - } - return errs -} - -func (importer *RuleImporter) queryFn(ctx context.Context, q string, t time.Time) (promql.Vector, error) { - val, warnings, err := importer.apiClient.Query(ctx, q, t) - if err != nil { - return promql.Vector{}, err - } - if warnings != nil { - fmt.Fprint(os.Stderr, "warning api.Query:", warnings) - } - - switch val.Type() { - case model.ValVector: - valVector := val.(model.Vector) - return modelToPromqlVector(valVector), nil - case model.ValScalar: - valScalar := val.(*model.Scalar) - return promql.Vector{promql.Sample{ - Metric: labels.Labels{}, - Point: promql.Point{T: int64(valScalar.Timestamp), V: float64(valScalar.Value)}, - }}, nil - default: - return nil, errors.New("rule result is wrong type") - } -} - -func modelToPromqlVector(modelValue model.Vector) promql.Vector { - result := make(promql.Vector, 0, len(modelValue)) - - for _, value := range modelValue { - labels := make(labels.Labels, 0, len(value.Metric)) - - for k, v := range value.Metric { - labels = append(labels, plabels.Label{ - Name: string(k), - Value: string(v), - }) - } - sort.Sort(labels) - - result = append(result, promql.Sample{ - Metric: labels, - Point: promql.Point{T: int64(value.Timestamp), V: float64(value.Value)}, - }) - } - return result -} - -// ImportRule imports the historical data for a single rule -func (importer *RuleImporter) ImportRule(ctx context.Context, rule rules.Rule) error { - ts, err := parseTime(importer.config.Start) - if err != nil { - return err - } - end, err := parseTime(importer.config.End) - if err != nil { - return err - } - url, err := url.Parse(importer.config.URL) - if err != nil { - return err - } - - appender := importer.writer.Appender() - for ts.Before(end) { - vector, err := rule.Eval(ctx, ts, importer.queryFn, url) - if err != nil { - return err - } - for _, sample := range vector { - // we don't AddFast here because we need to maintain the - // ref for each series bcs rule.Eval could return different labels, - // so that means you would need to map the ref to metric, but that is what Add does - // anyways so just use that - _, err := appender.Add(plabels.Labels{plabels.Label{Name: sample.String()}}, sample.T, sample.V) - if err != nil { - return err - } - } - - ts.Add(importer.config.EvalInterval) - // todo: 2 hr blocks? - } - return appender.Commit() -} - -func parseTime(s string) (time.Time, error) { - if t, err := strconv.ParseFloat(s, 64); err == nil { - s, ns := math.Modf(t) - return time.Unix(int64(s), int64(ns*float64(time.Second))).UTC(), nil - } - if t, err := time.Parse(time.RFC3339Nano, s); err == nil { - return t, nil - } - return time.Time{}, errors.Errorf("cannot parse %q to a valid timestamp", s) -} - -// CreateBlocks creates blocks for all the rule data -func (importer *RuleImporter) CreateBlocks() error { - _, err := importer.writer.Flush() - return err -} diff --git a/rules/manager.go b/rules/manager.go index c4344fa68..ea7d10455 100644 --- a/rules/manager.go +++ b/rules/manager.go @@ -200,6 +200,8 @@ type Rule interface { Eval(context.Context, time.Time, QueryFunc, *url.URL) (promql.Vector, error) // String returns a human-readable string representation of the rule. String() string + // Query returns the rule query expression. + Query() parser.Expr // SetLastErr sets the current error experienced by the rule. SetLastError(error) // LastErr returns the last error experienced by the rule. @@ -262,7 +264,7 @@ func NewGroup(o GroupOptions) *Group { metrics = NewGroupMetrics(o.Opts.Registerer) } - key := groupKey(o.File, o.Name) + key := GroupKey(o.File, o.Name) metrics.evalTotal.WithLabelValues(key) metrics.evalFailures.WithLabelValues(key) metrics.groupLastEvalTime.WithLabelValues(key) @@ -302,7 +304,7 @@ func (g *Group) run(ctx context.Context) { defer close(g.terminated) // Wait an initial amount to have consistently slotted intervals. - evalTimestamp := g.evalTimestamp().Add(g.interval) + evalTimestamp := g.EvalTimestamp(time.Now().UnixNano()).Add(g.interval) select { case <-time.After(time.Until(evalTimestamp)): case <-g.done: @@ -455,7 +457,7 @@ func (g *Group) GetEvaluationDuration() time.Duration { // setEvaluationDuration sets the time in seconds the last evaluation took. func (g *Group) setEvaluationDuration(dur time.Duration) { - g.metrics.groupLastDuration.WithLabelValues(groupKey(g.file, g.name)).Set(dur.Seconds()) + g.metrics.groupLastDuration.WithLabelValues(GroupKey(g.file, g.name)).Set(dur.Seconds()) g.mtx.Lock() defer g.mtx.Unlock() @@ -471,19 +473,19 @@ func (g *Group) GetEvaluationTimestamp() time.Time { // setEvaluationTimestamp updates evaluationTimestamp to the timestamp of when the rule group was last evaluated. func (g *Group) setEvaluationTimestamp(ts time.Time) { - g.metrics.groupLastEvalTime.WithLabelValues(groupKey(g.file, g.name)).Set(float64(ts.UnixNano()) / 1e9) + g.metrics.groupLastEvalTime.WithLabelValues(GroupKey(g.file, g.name)).Set(float64(ts.UnixNano()) / 1e9) g.mtx.Lock() defer g.mtx.Unlock() g.evaluationTimestamp = ts } -// evalTimestamp returns the immediately preceding consistently slotted evaluation time. -func (g *Group) evalTimestamp() time.Time { +// EvalTimestamp returns the immediately preceding consistently slotted evaluation time. +func (g *Group) EvalTimestamp(startTime int64) time.Time { var ( offset = int64(g.hash() % uint64(g.interval)) - now = time.Now().UnixNano() - adjNow = now - offset + start = startTime + adjNow = start - offset base = adjNow - (adjNow % int64(g.interval)) ) @@ -567,7 +569,7 @@ func (g *Group) Eval(ctx context.Context, ts time.Time) { rule.SetEvaluationTimestamp(t) }(time.Now()) - g.metrics.evalTotal.WithLabelValues(groupKey(g.File(), g.Name())).Inc() + g.metrics.evalTotal.WithLabelValues(GroupKey(g.File(), g.Name())).Inc() vector, err := rule.Eval(ctx, ts, g.opts.QueryFunc, g.opts.ExternalURL) if err != nil { @@ -576,7 +578,7 @@ func (g *Group) Eval(ctx context.Context, ts time.Time) { if _, ok := err.(promql.ErrQueryCanceled); !ok { level.Warn(g.logger).Log("msg", "Evaluating rule failed", "rule", rule, "err", err) } - g.metrics.evalFailures.WithLabelValues(groupKey(g.File(), g.Name())).Inc() + g.metrics.evalFailures.WithLabelValues(GroupKey(g.File(), g.Name())).Inc() return } @@ -929,7 +931,7 @@ func (m *Manager) Update(interval time.Duration, files []string, externalLabels // check if new group equals with the old group, if yes then skip it. // If not equals, stop it and wait for it to finish the current iteration. // Then copy it into the new group. - gn := groupKey(newg.file, newg.name) + gn := GroupKey(newg.file, newg.name) oldg, ok := m.groups[gn] delete(m.groups, gn) @@ -1042,7 +1044,7 @@ func (m *Manager) LoadGroups( )) } - groups[groupKey(fn, rg.Name)] = NewGroup(GroupOptions{ + groups[GroupKey(fn, rg.Name)] = NewGroup(GroupOptions{ Name: rg.Name, File: fn, Interval: itv, @@ -1057,8 +1059,8 @@ func (m *Manager) LoadGroups( return groups, nil } -// Group names need not be unique across filenames. -func groupKey(file, name string) string { +// GroupKey group names need not be unique across filenames. +func GroupKey(file, name string) string { return file + ";" + name } From 2e526cf2a7b26706701878e2095f6c8127160269 Mon Sep 17 00:00:00 2001 From: jessicagreben Date: Sun, 13 Sep 2020 08:38:32 -0700 Subject: [PATCH 03/21] add output dir parameter Signed-off-by: jessicagreben --- cmd/promtool/main.go | 9 ++++++--- cmd/promtool/rules.go | 18 +++++------------- 2 files changed, 11 insertions(+), 16 deletions(-) diff --git a/cmd/promtool/main.go b/cmd/promtool/main.go index 0c78d93e7..8197047bc 100644 --- a/cmd/promtool/main.go +++ b/cmd/promtool/main.go @@ -135,6 +135,7 @@ func main() { Required().String() backfillRuleEnd := backfillRuleCmd.Flag("end", "If an end time is provided, all recording rules in the rule files provided will be backfilled to the end time. Default will backfill up to 3 hrs ago. End time should be RFC3339 or Unix timestamp."). Default("-3h").String() + backfillOutputDir := backfillRuleCmd.Flag("output dir", "The filepath on the local filesystem to write the output to. Output will be blocks containing the data of the backfilled recording rules.").Default("backfilldata/").String() backfillRuleURL := backfillRuleCmd.Flag("url", "Prometheus API url with the data where the rule will be backfilled from.").Default("localhost:9090").String() backfillRuleEvalInterval := backfillRuleCmd.Flag("evaluation_interval", "How frequently to evaluate rules when backfilling."). Default("15s").Duration() @@ -200,7 +201,7 @@ func main() { os.Exit(checkErr(dumpSamples(*dumpPath, *dumpMinTime, *dumpMaxTime))) case backfillRuleCmd.FullCommand(): - os.Exit(BackfillRule(*backfillRuleURL, *backfillRuleStart, *backfillRuleEnd, *backfillRuleEvalInterval, *backfillRuleFiles...)) + os.Exit(BackfillRule(*backfillRuleURL, *backfillRuleStart, *backfillRuleEnd, *backfillOutputDir, *backfillRuleEvalInterval, *backfillRuleFiles...)) } } @@ -766,8 +767,9 @@ func (j *jsonPrinter) printLabelValues(v model.LabelValues) { json.NewEncoder(os.Stdout).Encode(v) } -// BackfillRule backfills rules from the files provided. -func BackfillRule(url, start, end string, evalInterval time.Duration, files ...string) int { +// BackfillRule backfills recording rules from the files provided. The output are blocks of data +// at the outputDir location. +func BackfillRule(url, start, end, outputDir string, evalInterval time.Duration, files ...string) int { ctx := context.Background() stime, etime, err := parseStartTimeAndEndTime(start, end) if err != nil { @@ -777,6 +779,7 @@ func BackfillRule(url, start, end string, evalInterval time.Duration, files ...s cfg := RuleImporterConfig{ Start: stime, End: etime, + OutputDir: outputDir, EvalInterval: evalInterval, URL: url, } diff --git a/cmd/promtool/rules.go b/cmd/promtool/rules.go index 52bf22688..5fa100e49 100644 --- a/cmd/promtool/rules.go +++ b/cmd/promtool/rules.go @@ -16,7 +16,6 @@ package main import ( "context" "fmt" - "io/ioutil" "os" "time" @@ -49,6 +48,7 @@ type RuleImporter struct { type RuleImporterConfig struct { Start time.Time End time.Time + OutputDir string EvalInterval time.Duration URL string } @@ -64,12 +64,10 @@ func NewRuleImporter(logger log.Logger, config RuleImporterConfig) *RuleImporter // Init initializes the rule importer which creates a new block writer // and creates an Prometheus API client. func (importer *RuleImporter) Init() error { - // todo: clean up dir - newBlockDir, err := ioutil.TempDir("", "rule_blocks") - if err != nil { - return err - } - importer.writer = blocks.NewMultiWriter(importer.logger, newBlockDir, importer.config.EvalInterval.Nanoseconds()) + importer.writer = blocks.NewMultiWriter(importer.logger, + importer.config.OutputDir, + importer.config.EvalInterval.Nanoseconds(), + ) config := api.Config{ Address: importer.config.URL, @@ -84,7 +82,6 @@ func (importer *RuleImporter) Init() error { // Close cleans up any open resources. func (importer *RuleImporter) Close() error { - // todo: clean up any dirs that were created return importer.writer.Close() } @@ -99,21 +96,17 @@ func (importer *RuleImporter) LoadGroups(ctx context.Context, filenames []string } for _, ruleGroup := range rgs.Groups { - itv := importer.config.EvalInterval if ruleGroup.Interval != 0 { itv = time.Duration(ruleGroup.Interval) } rgRules := make([]rules.Rule, 0, len(ruleGroup.Rules)) - for _, r := range ruleGroup.Rules { - expr, err := importer.groupLoader.Parse(r.Expr.Value) if err != nil { return []error{errors.Wrap(err, filename)} } - rgRules = append(rgRules, rules.NewRecordingRule( r.Record.Value, expr, @@ -158,7 +151,6 @@ func (importer *RuleImporter) ImportAll(ctx context.Context) []error { // ImportRule imports the historical data for a single rule. func (importer *RuleImporter) ImportRule(ctx context.Context, ruleExpr string, stimeWithAlignment time.Time, internval time.Duration) error { ts := stimeWithAlignment - appender := importer.writer.Appender() for ts.Before(importer.config.End) { From 3ed6457dd4d2f04ef8ad9f54366bbc349951c2c5 Mon Sep 17 00:00:00 2001 From: jessicagreben Date: Sat, 17 Oct 2020 08:36:58 -0700 Subject: [PATCH 04/21] use blockwriter, rm multiwriter code Signed-off-by: jessicagreben --- cmd/promtool/main.go | 19 +++-- cmd/promtool/rules.go | 6 +- tsdb/importer/blocks/multi.go | 120 ---------------------------- tsdb/importer/blocks/writer.go | 139 --------------------------------- 4 files changed, 12 insertions(+), 272 deletions(-) delete mode 100644 tsdb/importer/blocks/multi.go delete mode 100644 tsdb/importer/blocks/writer.go diff --git a/cmd/promtool/main.go b/cmd/promtool/main.go index e389bf75b..74804fdf6 100644 --- a/cmd/promtool/main.go +++ b/cmd/promtool/main.go @@ -143,8 +143,8 @@ func main() { Default("-3h").String() backfillOutputDir := backfillRuleCmd.Flag("output dir", "The filepath on the local filesystem to write the output to. Output will be blocks containing the data of the backfilled recording rules.").Default("backfilldata/").String() backfillRuleURL := backfillRuleCmd.Flag("url", "Prometheus API url with the data where the rule will be backfilled from.").Default("localhost:9090").String() - backfillRuleEvalInterval := backfillRuleCmd.Flag("evaluation_interval", "How frequently to evaluate rules when backfilling."). - Default("15s").Duration() + backfillRuleEvalInterval := backfillRuleCmd.Flag("evaluation_interval default", "How frequently to evaluate rules when backfilling. evaluation interval in the rules file will take precedence."). + Default("60s").Duration() backfillRuleFiles := backfillRuleCmd.Arg( "rule-files", "A list of one or more files containing recording rules to be backfilled. All recording rules listed in the files will be backfilled. Alerting rules are not evaluated.", @@ -207,7 +207,7 @@ func main() { os.Exit(checkErr(dumpSamples(*dumpPath, *dumpMinTime, *dumpMaxTime))) case backfillRuleCmd.FullCommand(): - os.Exit(BackfillRule(*backfillRuleURL, *backfillRuleStart, *backfillRuleEnd, *backfillOutputDir, *backfillRuleEvalInterval, *backfillRuleFiles...)) + os.Exit(checkErr(BackfillRule(*backfillRuleURL, *backfillRuleStart, *backfillRuleEnd, *backfillOutputDir, *backfillRuleEvalInterval, *backfillRuleFiles...))) } } @@ -785,12 +785,12 @@ func (j *jsonPrinter) printLabelValues(v model.LabelValues) { // BackfillRule backfills recording rules from the files provided. The output are blocks of data // at the outputDir location. -func BackfillRule(url, start, end, outputDir string, evalInterval time.Duration, files ...string) int { +func BackfillRule(url, start, end, outputDir string, evalInterval time.Duration, files ...string) error { ctx := context.Background() stime, etime, err := parseStartTimeAndEndTime(start, end) if err != nil { fmt.Fprintln(os.Stderr, err) - return 1 + return err } cfg := RuleImporterConfig{ Start: stime, @@ -801,17 +801,16 @@ func BackfillRule(url, start, end, outputDir string, evalInterval time.Duration, } logger := log.NewLogfmtLogger(log.NewSyncWriter(os.Stderr)) ruleImporter := NewRuleImporter(logger, cfg) - err = ruleImporter.Init() - if err != nil { + if err = ruleImporter.Init(); err != nil { fmt.Fprintln(os.Stderr, "rule importer init error", err) - return 1 + return err } errs := ruleImporter.LoadGroups(ctx, files) for _, err := range errs { if err != nil { fmt.Fprintln(os.Stderr, "rule importer parse error", err) - return 1 + return err } } @@ -822,5 +821,5 @@ func BackfillRule(url, start, end, outputDir string, evalInterval time.Duration, } } - return 0 + return nil } diff --git a/cmd/promtool/rules.go b/cmd/promtool/rules.go index 5fa100e49..cb0586804 100644 --- a/cmd/promtool/rules.go +++ b/cmd/promtool/rules.go @@ -26,7 +26,7 @@ import ( "github.com/prometheus/common/model" "github.com/prometheus/prometheus/pkg/labels" "github.com/prometheus/prometheus/rules" - "github.com/prometheus/prometheus/tsdb/importer/blocks" + "github.com/prometheus/prometheus/tsdb" ) const blockSize = 2 // in hours @@ -64,9 +64,9 @@ func NewRuleImporter(logger log.Logger, config RuleImporterConfig) *RuleImporter // Init initializes the rule importer which creates a new block writer // and creates an Prometheus API client. func (importer *RuleImporter) Init() error { - importer.writer = blocks.NewMultiWriter(importer.logger, + importer.writer = tsdb.NewBlockWriter(importer.logger, importer.config.OutputDir, - importer.config.EvalInterval.Nanoseconds(), + (blockSize * time.Hour).Milliseconds() ) config := api.Config{ diff --git a/tsdb/importer/blocks/multi.go b/tsdb/importer/blocks/multi.go deleted file mode 100644 index 4007860d8..000000000 --- a/tsdb/importer/blocks/multi.go +++ /dev/null @@ -1,120 +0,0 @@ -// Copyright 2020 The Prometheus Authors -// 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 blocks - -import ( - "github.com/go-kit/kit/log" - "github.com/oklog/ulid" - "github.com/pkg/errors" - "github.com/prometheus/prometheus/pkg/labels" - "github.com/prometheus/prometheus/storage" - tsdb_errors "github.com/prometheus/prometheus/tsdb/errors" - "github.com/prometheus/prometheus/tsdb/index" -) - -type errAppender struct{ err error } - -func (a errAppender) Add(l labels.Labels, t int64, v float64) (uint64, error) { return 0, a.err } -func (a errAppender) AddFast(ref uint64, t int64, v float64) error { return a.err } -func (a errAppender) Commit() error { return a.err } -func (a errAppender) Rollback() error { return a.err } - -func rangeForTimestamp(t int64, width int64) (maxt int64) { - return (t/width)*width + width -} - -type MultiWriter struct { - blocks map[index.Range]Writer - activeAppenders map[index.Range]storage.Appender - - logger log.Logger - dir string - // TODO(bwplotka): Allow more complex compaction levels. - sizeMillis int64 -} - -func NewMultiWriter(logger log.Logger, dir string, sizeMillis int64) *MultiWriter { - return &MultiWriter{ - logger: logger, - dir: dir, - sizeMillis: sizeMillis, - blocks: map[index.Range]Writer{}, - activeAppenders: map[index.Range]storage.Appender{}, - } -} - -// Appender is not thread-safe. Returned Appender is not thread-save as well. -// TODO(bwplotka): Consider making it thread safe. -func (w *MultiWriter) Appender() storage.Appender { return w } - -func (w *MultiWriter) getOrCreate(t int64) storage.Appender { - maxt := rangeForTimestamp(t, w.sizeMillis) - hash := index.Range{Start: maxt - w.sizeMillis, End: maxt} - if a, ok := w.activeAppenders[hash]; ok { - return a - } - - nw, err := NewTSDBWriter(w.logger, w.dir) - if err != nil { - return errAppender{err: errors.Wrap(err, "new tsdb writer")} - } - - w.blocks[hash] = nw - w.activeAppenders[hash] = nw.Appender() - return w.activeAppenders[hash] -} - -func (w *MultiWriter) Add(l labels.Labels, t int64, v float64) (uint64, error) { - return w.getOrCreate(t).Add(l, t, v) -} - -func (w *MultiWriter) AddFast(ref uint64, t int64, v float64) error { - return w.getOrCreate(t).AddFast(ref, t, v) -} - -func (w *MultiWriter) Commit() error { - var merr tsdb_errors.MultiError - for _, a := range w.activeAppenders { - merr.Add(a.Commit()) - } - return merr.Err() -} - -func (w *MultiWriter) Rollback() error { - var merr tsdb_errors.MultiError - for _, a := range w.activeAppenders { - merr.Add(a.Rollback()) - } - return merr.Err() -} - -func (w *MultiWriter) Flush() ([]ulid.ULID, error) { - ids := make([]ulid.ULID, 0, len(w.blocks)) - for _, b := range w.blocks { - id, err := b.Flush() - if err != nil { - return nil, err - } - ids = append(ids, id...) - } - return ids, nil -} - -func (w *MultiWriter) Close() error { - var merr tsdb_errors.MultiError - for _, b := range w.blocks { - merr.Add(b.Close()) - } - return merr.Err() -} diff --git a/tsdb/importer/blocks/writer.go b/tsdb/importer/blocks/writer.go deleted file mode 100644 index 3af869e06..000000000 --- a/tsdb/importer/blocks/writer.go +++ /dev/null @@ -1,139 +0,0 @@ -// Copyright 2020 The Prometheus Authors -// 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 blocks - -import ( - "context" - "io/ioutil" - "math" - "os" - "time" - - "github.com/oklog/ulid" - "github.com/prometheus/prometheus/storage" - "github.com/prometheus/prometheus/tsdb" - - "github.com/go-kit/kit/log" - "github.com/go-kit/kit/log/level" - "github.com/pkg/errors" - "github.com/prometheus/prometheus/pkg/timestamp" - "github.com/prometheus/prometheus/tsdb/chunkenc" -) - -// Writer is interface to write time series into Prometheus blocks. -type Writer interface { - storage.Appendable - - // Flush writes current data to disk. - // The block or blocks will contain values accumulated by `Write`. - Flush() ([]ulid.ULID, error) - - // Close releases all resources. No append is allowed anymore to such writer. - Close() error -} - -var _ Writer = &TSDBWriter{} - -// Writer is a block writer that allows appending and flushing to disk. -type TSDBWriter struct { - logger log.Logger - dir string - - head *tsdb.Head - tmpDir string -} - -func durToMillis(t time.Duration) int64 { - return int64(t.Seconds() * 1000) -} - -// NewTSDBWriter create new block writer. -// -// The returned writer accumulates all series in memory until `Flush` is called. -// -// Note that the writer will not check if the target directory exists or -// contains anything at all. It is the caller's responsibility to -// ensure that the resulting blocks do not overlap etc. -// Writer ensures the block flush is atomic (via rename). -func NewTSDBWriter(logger log.Logger, dir string) (*TSDBWriter, error) { - res := &TSDBWriter{ - logger: logger, - dir: dir, - } - return res, res.initHead() -} - -// initHead creates and initialises new head. -func (w *TSDBWriter) initHead() error { - logger := w.logger - - // Keep Registerer and WAL nil as we don't use them. - // Put huge chunkRange; It has to be equal then expected block size. - // Since we don't have info about block size here, set it to large number. - - tmpDir, err := ioutil.TempDir(os.TempDir(), "head") - if err != nil { - return errors.Wrap(err, "create temp dir") - } - w.tmpDir = tmpDir - - h, err := tsdb.NewHead(nil, logger, nil, durToMillis(9999*time.Hour), w.tmpDir, nil, tsdb.DefaultStripeSize, nil) - if err != nil { - return errors.Wrap(err, "tsdb.NewHead") - } - - w.head = h - return w.head.Init(math.MinInt64) -} - -// Appender is not thread-safe. Returned Appender is thread-save however. -func (w *TSDBWriter) Appender() storage.Appender { - return w.head.Appender() -} - -// Flush implements Writer interface. This is where actual block writing -// happens. After flush completes, no write can be done. -func (w *TSDBWriter) Flush() ([]ulid.ULID, error) { - seriesCount := w.head.NumSeries() - if w.head.NumSeries() == 0 { - return nil, errors.New("no series appended; aborting.") - } - - mint := w.head.MinTime() - maxt := w.head.MaxTime() + 1 - level.Info(w.logger).Log("msg", "flushing", "series_count", seriesCount, "mint", timestamp.Time(mint), "maxt", timestamp.Time(maxt)) - - // Flush head to disk as a block. - compactor, err := tsdb.NewLeveledCompactor( - context.Background(), - nil, - w.logger, - []int64{durToMillis(2 * time.Hour)}, // Does not matter, used only for planning. - chunkenc.NewPool()) - if err != nil { - return nil, errors.Wrap(err, "create leveled compactor") - } - id, err := compactor.Write(w.dir, w.head, mint, maxt, nil) - if err != nil { - return nil, errors.Wrap(err, "compactor write") - } - // TODO(bwplotka): Potential truncate head, and allow writer reuse. Currently truncating fails with - // truncate chunks.HeadReadWriter: maxt of the files are not set. - return []ulid.ULID{id}, nil -} - -func (w *TSDBWriter) Close() error { - _ = os.RemoveAll(w.tmpDir) - return w.head.Close() -} From 6980bcf6716426f8a5057bf0cb88f9a3bc443920 Mon Sep 17 00:00:00 2001 From: jessicagreben Date: Sat, 31 Oct 2020 06:40:24 -0700 Subject: [PATCH 05/21] unexport backfiller Signed-off-by: jessicagreben --- cmd/promtool/main.go | 10 ++++---- cmd/promtool/rules.go | 55 +++++++++++++++++++++++-------------------- rules/manager.go | 37 ++++++----------------------- 3 files changed, 41 insertions(+), 61 deletions(-) diff --git a/cmd/promtool/main.go b/cmd/promtool/main.go index 74804fdf6..d594ba1f2 100644 --- a/cmd/promtool/main.go +++ b/cmd/promtool/main.go @@ -792,7 +792,7 @@ func BackfillRule(url, start, end, outputDir string, evalInterval time.Duration, fmt.Fprintln(os.Stderr, err) return err } - cfg := RuleImporterConfig{ + cfg := ruleImporterConfig{ Start: stime, End: etime, OutputDir: outputDir, @@ -800,13 +800,13 @@ func BackfillRule(url, start, end, outputDir string, evalInterval time.Duration, URL: url, } logger := log.NewLogfmtLogger(log.NewSyncWriter(os.Stderr)) - ruleImporter := NewRuleImporter(logger, cfg) - if err = ruleImporter.Init(); err != nil { + ruleImporter := newRuleImporter(logger, cfg) + if err = ruleImporter.init(); err != nil { fmt.Fprintln(os.Stderr, "rule importer init error", err) return err } - errs := ruleImporter.LoadGroups(ctx, files) + errs := ruleImporter.loadGroups(ctx, files) for _, err := range errs { if err != nil { fmt.Fprintln(os.Stderr, "rule importer parse error", err) @@ -814,7 +814,7 @@ func BackfillRule(url, start, end, outputDir string, evalInterval time.Duration, } } - errs = ruleImporter.ImportAll(ctx) + errs = ruleImporter.importAll(ctx) for _, err := range errs { if err != nil { fmt.Fprintln(os.Stderr, "rule importer error", err) diff --git a/cmd/promtool/rules.go b/cmd/promtool/rules.go index cb0586804..85425cc0c 100644 --- a/cmd/promtool/rules.go +++ b/cmd/promtool/rules.go @@ -31,21 +31,21 @@ import ( const blockSize = 2 // in hours -// RuleImporter is the importer to backfill rules. -type RuleImporter struct { +// ruleImporter is the importer to backfill rules. +type ruleImporter struct { logger log.Logger - config RuleImporterConfig + config ruleImporterConfig groups map[string]*rules.Group groupLoader rules.GroupLoader apiClient v1.API - writer *blocks.MultiWriter + writer *tsdb.BlockWriter } -// RuleImporterConfig is the config for the rule importer. -type RuleImporterConfig struct { +// ruleImporterConfig is the config for the rule importer. +type ruleImporterConfig struct { Start time.Time End time.Time OutputDir string @@ -53,21 +53,25 @@ type RuleImporterConfig struct { URL string } -// NewRuleImporter creates a new rule importer that can be used to backfill rules. -func NewRuleImporter(logger log.Logger, config RuleImporterConfig) *RuleImporter { - return &RuleImporter{ +// newRuleImporter creates a new rule importer that can be used to backfill rules. +func newRuleImporter(logger log.Logger, config ruleImporterConfig) *ruleImporter { + return &ruleImporter{ config: config, groupLoader: rules.FileLoader{}, } } -// Init initializes the rule importer which creates a new block writer +// init initializes the rule importer which creates a new block writer // and creates an Prometheus API client. -func (importer *RuleImporter) Init() error { - importer.writer = tsdb.NewBlockWriter(importer.logger, +func (importer *ruleImporter) init() error { + w, err := tsdb.NewBlockWriter(importer.logger, importer.config.OutputDir, - (blockSize * time.Hour).Milliseconds() + (blockSize * time.Hour).Milliseconds(), ) + if err != nil { + return err + } + importer.writer = w config := api.Config{ Address: importer.config.URL, @@ -80,13 +84,13 @@ func (importer *RuleImporter) Init() error { return nil } -// Close cleans up any open resources. -func (importer *RuleImporter) Close() error { +// close cleans up any open resources. +func (importer *ruleImporter) close() error { return importer.writer.Close() } -// LoadGroups reads groups from a list of rule files. -func (importer *RuleImporter) LoadGroups(ctx context.Context, filenames []string) (errs []error) { +// loadGroups reads groups from a list of rule files. +func (importer *ruleImporter) loadGroups(ctx context.Context, filenames []string) (errs []error) { groups := make(map[string]*rules.Group) for _, filename := range filenames { @@ -127,31 +131,31 @@ func (importer *RuleImporter) LoadGroups(ctx context.Context, filenames []string return nil } -// ImportAll evaluates all the groups and rules and creates new time series +// importAll evaluates all the groups and rules and creates new time series // and stores them in new blocks. -func (importer *RuleImporter) ImportAll(ctx context.Context) []error { +func (importer *ruleImporter) importAll(ctx context.Context) []error { var errs = []error{} for _, group := range importer.groups { stimeWithAlignment := group.EvalTimestamp(importer.config.Start.UnixNano()) for _, r := range group.Rules() { - err := importer.ImportRule(ctx, r.Query().String(), stimeWithAlignment, group.Interval()) + err := importer.importRule(ctx, r.Query().String(), stimeWithAlignment, group.Interval()) if err != nil { errs = append(errs, err) } } } - _, err := importer.writer.Flush() + _, err := importer.writer.Flush(ctx) if err != nil { errs = append(errs, err) } return errs } -// ImportRule imports the historical data for a single rule. -func (importer *RuleImporter) ImportRule(ctx context.Context, ruleExpr string, stimeWithAlignment time.Time, internval time.Duration) error { +// importRule imports the historical data for a single rule. +func (importer *ruleImporter) importRule(ctx context.Context, ruleExpr string, stimeWithAlignment time.Time, internval time.Duration) error { ts := stimeWithAlignment - appender := importer.writer.Appender() + appender := importer.writer.Appender(ctx) for ts.Before(importer.config.End) { currentBlockEnd := ts.Add(blockSize * time.Hour) @@ -189,7 +193,6 @@ func (importer *RuleImporter) ImportRule(ctx context.Context, ruleExpr string, s for _, value := range sample.Values { _, err := appender.Add(currentLabels, value.Timestamp.Unix(), float64(value.Value)) if err != nil { - // todo: handle other errors, i.e. ErrOutOfOrderSample and ErrDuplicateSampleForTimestamp return err } } @@ -200,7 +203,7 @@ func (importer *RuleImporter) ImportRule(ctx context.Context, ruleExpr string, s ts = currentBlockEnd } - _, err := importer.writer.Flush() + _, err := importer.writer.Flush(ctx) if err != nil { return err } diff --git a/rules/manager.go b/rules/manager.go index 63a7ae72b..40cfc0659 100644 --- a/rules/manager.go +++ b/rules/manager.go @@ -280,13 +280,9 @@ func NewGroup(o GroupOptions) *Group { metrics = NewGroupMetrics(o.Opts.Registerer) } -<<<<<<< HEAD key := GroupKey(o.File, o.Name) -======= - key := groupKey(o.File, o.Name) metrics.iterationsMissed.WithLabelValues(key) metrics.iterationsScheduled.WithLabelValues(key) ->>>>>>> master metrics.evalTotal.WithLabelValues(key) metrics.evalFailures.WithLabelValues(key) metrics.groupLastEvalTime.WithLabelValues(key) @@ -342,7 +338,7 @@ func (g *Group) run(ctx context.Context) { }) iter := func() { - g.metrics.iterationsScheduled.WithLabelValues(groupKey(g.file, g.name)).Inc() + g.metrics.iterationsScheduled.WithLabelValues(GroupKey(g.file, g.name)).Inc() start := time.Now() g.Eval(ctx, evalTimestamp) @@ -394,8 +390,8 @@ func (g *Group) run(ctx context.Context) { case <-tick.C: missed := (time.Since(evalTimestamp) / g.interval) - 1 if missed > 0 { - g.metrics.iterationsMissed.WithLabelValues(groupKey(g.file, g.name)).Add(float64(missed)) - g.metrics.iterationsScheduled.WithLabelValues(groupKey(g.file, g.name)).Add(float64(missed)) + g.metrics.iterationsMissed.WithLabelValues(GroupKey(g.file, g.name)).Add(float64(missed)) + g.metrics.iterationsScheduled.WithLabelValues(GroupKey(g.file, g.name)).Add(float64(missed)) } evalTimestamp = evalTimestamp.Add((missed + 1) * g.interval) iter() @@ -416,8 +412,8 @@ func (g *Group) run(ctx context.Context) { case <-tick.C: missed := (time.Since(evalTimestamp) / g.interval) - 1 if missed > 0 { - g.metrics.iterationsMissed.WithLabelValues(groupKey(g.file, g.name)).Add(float64(missed)) - g.metrics.iterationsScheduled.WithLabelValues(groupKey(g.file, g.name)).Add(float64(missed)) + g.metrics.iterationsMissed.WithLabelValues(GroupKey(g.file, g.name)).Add(float64(missed)) + g.metrics.iterationsScheduled.WithLabelValues(GroupKey(g.file, g.name)).Add(float64(missed)) } evalTimestamp = evalTimestamp.Add((missed + 1) * g.interval) iter() @@ -478,15 +474,9 @@ func (g *Group) GetEvaluationTime() time.Duration { return g.evaluationTime } -<<<<<<< HEAD // setEvaluationDuration sets the time in seconds the last evaluation took. func (g *Group) setEvaluationDuration(dur time.Duration) { g.metrics.groupLastDuration.WithLabelValues(GroupKey(g.file, g.name)).Set(dur.Seconds()) -======= -// setEvaluationTime sets the time in seconds the last evaluation took. -func (g *Group) setEvaluationTime(dur time.Duration) { - g.metrics.groupLastDuration.WithLabelValues(groupKey(g.file, g.name)).Set(dur.Seconds()) ->>>>>>> master g.mtx.Lock() defer g.mtx.Unlock() @@ -500,15 +490,9 @@ func (g *Group) GetLastEvaluation() time.Time { return g.lastEvaluation } -<<<<<<< HEAD // setEvaluationTimestamp updates evaluationTimestamp to the timestamp of when the rule group was last evaluated. func (g *Group) setEvaluationTimestamp(ts time.Time) { g.metrics.groupLastEvalTime.WithLabelValues(GroupKey(g.file, g.name)).Set(float64(ts.UnixNano()) / 1e9) -======= -// setLastEvaluation updates lastEvaluation to the timestamp of when the rule group was last evaluated. -func (g *Group) setLastEvaluation(ts time.Time) { - g.metrics.groupLastEvalTime.WithLabelValues(groupKey(g.file, g.name)).Set(float64(ts.UnixNano()) / 1e9) ->>>>>>> master g.mtx.Lock() defer g.mtx.Unlock() @@ -519,8 +503,7 @@ func (g *Group) setLastEvaluation(ts time.Time) { func (g *Group) EvalTimestamp(startTime int64) time.Time { var ( offset = int64(g.hash() % uint64(g.interval)) - start = startTime - adjNow = start - offset + adjNow = startTime - offset base = adjNow - (adjNow % int64(g.interval)) ) @@ -615,13 +598,7 @@ func (g *Group) Eval(ctx context.Context, ts time.Time) { if _, ok := err.(promql.ErrQueryCanceled); !ok { level.Warn(g.logger).Log("msg", "Evaluating rule failed", "rule", rule, "err", err) } -<<<<<<< HEAD g.metrics.evalFailures.WithLabelValues(GroupKey(g.File(), g.Name())).Inc() -======= - sp.SetTag("error", true) - sp.LogKV("error", err) - g.metrics.evalFailures.WithLabelValues(groupKey(g.File(), g.Name())).Inc() ->>>>>>> master return } samplesTotal += float64(len(vector)) @@ -683,7 +660,7 @@ func (g *Group) Eval(ctx context.Context, ts time.Time) { }(i, rule) } if g.metrics != nil { - g.metrics.groupSamples.WithLabelValues(groupKey(g.File(), g.Name())).Set(samplesTotal) + g.metrics.groupSamples.WithLabelValues(GroupKey(g.File(), g.Name())).Set(samplesTotal) } g.cleanupStaleSeries(ctx, ts) } From 61c9a89120a24eae6112278d8019f0a2bed5f544 Mon Sep 17 00:00:00 2001 From: jessicagreben Date: Sat, 31 Oct 2020 07:11:54 -0700 Subject: [PATCH 06/21] use milliseconds for blocksize Signed-off-by: jessicagreben --- cmd/promtool/rules.go | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/cmd/promtool/rules.go b/cmd/promtool/rules.go index 85425cc0c..33a09283d 100644 --- a/cmd/promtool/rules.go +++ b/cmd/promtool/rules.go @@ -29,8 +29,6 @@ import ( "github.com/prometheus/prometheus/tsdb" ) -const blockSize = 2 // in hours - // ruleImporter is the importer to backfill rules. type ruleImporter struct { logger log.Logger @@ -66,7 +64,7 @@ func newRuleImporter(logger log.Logger, config ruleImporterConfig) *ruleImporter func (importer *ruleImporter) init() error { w, err := tsdb.NewBlockWriter(importer.logger, importer.config.OutputDir, - (blockSize * time.Hour).Milliseconds(), + tsdb.DefaultBlockDuration, ) if err != nil { return err @@ -158,7 +156,7 @@ func (importer *ruleImporter) importRule(ctx context.Context, ruleExpr string, s appender := importer.writer.Appender(ctx) for ts.Before(importer.config.End) { - currentBlockEnd := ts.Add(blockSize * time.Hour) + currentBlockEnd := ts.Add(time.Duration(tsdb.DefaultBlockDuration) * time.Millisecond) if currentBlockEnd.After(importer.config.End) { currentBlockEnd = importer.config.End } From 75654715d3be4476a473239dcc28a042e9ecbed6 Mon Sep 17 00:00:00 2001 From: jessicagreben Date: Sun, 1 Nov 2020 07:54:04 -0800 Subject: [PATCH 07/21] fix panics Signed-off-by: jessicagreben --- cmd/promtool/main.go | 29 +++++++++++++++++++++++------ cmd/promtool/rules.go | 1 + rules/manager.go | 8 ++++---- tsdb/blockwriter.go | 3 --- 4 files changed, 28 insertions(+), 13 deletions(-) diff --git a/cmd/promtool/main.go b/cmd/promtool/main.go index d594ba1f2..f32aa16df 100644 --- a/cmd/promtool/main.go +++ b/cmd/promtool/main.go @@ -139,11 +139,10 @@ func main() { backfillRuleCmd := backfillCmd.Command("rules", "Backfill Prometheus data for new rules.") backfillRuleStart := backfillRuleCmd.Flag("start", "The time to start backfilling the new rule from. It is required. Start time should be RFC3339 or Unix timestamp."). Required().String() - backfillRuleEnd := backfillRuleCmd.Flag("end", "If an end time is provided, all recording rules in the rule files provided will be backfilled to the end time. Default will backfill up to 3 hrs ago. End time should be RFC3339 or Unix timestamp."). - Default("-3h").String() + backfillRuleEnd := backfillRuleCmd.Flag("end", "If an end time is provided, all recording rules in the rule files provided will be backfilled to the end time. Default will backfill up to 3 hrs ago. End time should be RFC3339 or Unix timestamp.").String() backfillOutputDir := backfillRuleCmd.Flag("output dir", "The filepath on the local filesystem to write the output to. Output will be blocks containing the data of the backfilled recording rules.").Default("backfilldata/").String() - backfillRuleURL := backfillRuleCmd.Flag("url", "Prometheus API url with the data where the rule will be backfilled from.").Default("localhost:9090").String() - backfillRuleEvalInterval := backfillRuleCmd.Flag("evaluation_interval default", "How frequently to evaluate rules when backfilling. evaluation interval in the rules file will take precedence."). + backfillRuleURL := backfillRuleCmd.Flag("url", "Prometheus API url with the data where the rule will be backfilled from.").Default("http://localhost:9090").String() + backfillRuleEvalInterval := backfillRuleCmd.Flag("evaluation_interval_default", "How frequently to evaluate rules when backfilling if a value is not set in the rules file."). Default("60s").Duration() backfillRuleFiles := backfillRuleCmd.Arg( "rule-files", @@ -787,11 +786,29 @@ func (j *jsonPrinter) printLabelValues(v model.LabelValues) { // at the outputDir location. func BackfillRule(url, start, end, outputDir string, evalInterval time.Duration, files ...string) error { ctx := context.Background() - stime, etime, err := parseStartTimeAndEndTime(start, end) + var stime, etime time.Time + var err error + if end == "" { + etime = time.Now().Add(-3 * time.Hour) + } else { + etime, err = parseTime(end) + if err != nil { + fmt.Fprintln(os.Stderr, "error parsing end time:", err) + return err + } + } + + stime, err = parseTime(start) if err != nil { - fmt.Fprintln(os.Stderr, err) + fmt.Fprintln(os.Stderr, "error parsing start time:", err) return err } + + if !stime.Before(etime) { + fmt.Fprintln(os.Stderr, "start time is not before end time") + return nil + } + cfg := ruleImporterConfig{ Start: stime, End: etime, diff --git a/cmd/promtool/rules.go b/cmd/promtool/rules.go index 33a09283d..476c1c599 100644 --- a/cmd/promtool/rules.go +++ b/cmd/promtool/rules.go @@ -121,6 +121,7 @@ func (importer *ruleImporter) loadGroups(ctx context.Context, filenames []string File: filename, Interval: itv, Rules: rgRules, + Opts: &rules.ManagerOptions{}, }) } } diff --git a/rules/manager.go b/rules/manager.go index 40cfc0659..575eac409 100644 --- a/rules/manager.go +++ b/rules/manager.go @@ -474,8 +474,8 @@ func (g *Group) GetEvaluationTime() time.Duration { return g.evaluationTime } -// setEvaluationDuration sets the time in seconds the last evaluation took. -func (g *Group) setEvaluationDuration(dur time.Duration) { +// setEvaluationTime sets the time in seconds the last evaluation took. +func (g *Group) setEvaluationTime(dur time.Duration) { g.metrics.groupLastDuration.WithLabelValues(GroupKey(g.file, g.name)).Set(dur.Seconds()) g.mtx.Lock() @@ -490,8 +490,8 @@ func (g *Group) GetLastEvaluation() time.Time { return g.lastEvaluation } -// setEvaluationTimestamp updates evaluationTimestamp to the timestamp of when the rule group was last evaluated. -func (g *Group) setEvaluationTimestamp(ts time.Time) { +// setLastEvaluation updates evaluationTimestamp to the timestamp of when the rule group was last evaluated. +func (g *Group) setLastEvaluation(ts time.Time) { g.metrics.groupLastEvalTime.WithLabelValues(GroupKey(g.file, g.name)).Set(float64(ts.UnixNano()) / 1e9) g.mtx.Lock() diff --git a/tsdb/blockwriter.go b/tsdb/blockwriter.go index bb6184169..b859f86d9 100644 --- a/tsdb/blockwriter.go +++ b/tsdb/blockwriter.go @@ -24,7 +24,6 @@ import ( "github.com/oklog/ulid" "github.com/pkg/errors" - "github.com/prometheus/prometheus/pkg/timestamp" "github.com/prometheus/prometheus/storage" "github.com/prometheus/prometheus/tsdb/chunkenc" ) @@ -85,7 +84,6 @@ func (w *BlockWriter) Appender(ctx context.Context) storage.Appender { // Flush implements the Writer interface. This is where actual block writing // happens. After flush completes, no writes can be done. func (w *BlockWriter) Flush(ctx context.Context) (ulid.ULID, error) { - seriesCount := w.head.NumSeries() if w.head.NumSeries() == 0 { return ulid.ULID{}, errors.New("no series appended, aborting") } @@ -94,7 +92,6 @@ func (w *BlockWriter) Flush(ctx context.Context) (ulid.ULID, error) { // Add +1 millisecond to block maxt because block intervals are half-open: [b.MinTime, b.MaxTime). // Because of this block intervals are always +1 than the total samples it includes. maxt := w.head.MaxTime() + 1 - level.Info(w.logger).Log("msg", "flushing", "series_count", seriesCount, "mint", timestamp.Time(mint), "maxt", timestamp.Time(maxt)) compactor, err := NewLeveledCompactor(ctx, nil, From 19dee0a5699c67191d352aa3c86d4ab4c341a879 Mon Sep 17 00:00:00 2001 From: jessicagreben Date: Sun, 22 Nov 2020 14:24:38 -0800 Subject: [PATCH 08/21] add name and labels to metric, eval all rules for each block Signed-off-by: jessicagreben --- cmd/promtool/rules.go | 136 ++++++++++++++++++++++++------------------ tsdb/blockwriter.go | 4 -- 2 files changed, 77 insertions(+), 63 deletions(-) diff --git a/cmd/promtool/rules.go b/cmd/promtool/rules.go index 476c1c599..4533e6ebb 100644 --- a/cmd/promtool/rules.go +++ b/cmd/promtool/rules.go @@ -20,12 +20,14 @@ import ( "time" "github.com/go-kit/kit/log" + "github.com/go-kit/kit/log/level" "github.com/pkg/errors" "github.com/prometheus/client_golang/api" v1 "github.com/prometheus/client_golang/api/prometheus/v1" "github.com/prometheus/common/model" "github.com/prometheus/prometheus/pkg/labels" "github.com/prometheus/prometheus/rules" + "github.com/prometheus/prometheus/storage" "github.com/prometheus/prometheus/tsdb" ) @@ -54,6 +56,7 @@ type ruleImporterConfig struct { // newRuleImporter creates a new rule importer that can be used to backfill rules. func newRuleImporter(logger log.Logger, config ruleImporterConfig) *ruleImporter { return &ruleImporter{ + logger: logger, config: config, groupLoader: rules.FileLoader{}, } @@ -134,77 +137,92 @@ func (importer *ruleImporter) loadGroups(ctx context.Context, filenames []string // and stores them in new blocks. func (importer *ruleImporter) importAll(ctx context.Context) []error { var errs = []error{} - for _, group := range importer.groups { + var currentBlockEnd time.Time + var appender storage.Appender + + for name, group := range importer.groups { + level.Info(importer.logger).Log("backfiller", fmt.Sprintf("processing group, name: %s", name)) stimeWithAlignment := group.EvalTimestamp(importer.config.Start.UnixNano()) - for _, r := range group.Rules() { - err := importer.importRule(ctx, r.Query().String(), stimeWithAlignment, group.Interval()) + ts := stimeWithAlignment + // a 2-hr block that contains all the data for each rule + for ts.Before(importer.config.End) { + currentBlockEnd = ts.Add(time.Duration(tsdb.DefaultBlockDuration) * time.Millisecond) + if currentBlockEnd.After(importer.config.End) { + currentBlockEnd = importer.config.End + } + // should we be creating a new appender for each block? + appender = importer.writer.Appender(ctx) + + for i, r := range group.Rules() { + level.Info(importer.logger).Log("backfiller", fmt.Sprintf("processing rule %d, name: %s", i+1, r.Name())) + err := importer.importRule(ctx, r.Query().String(), r.Name(), r.Labels(), ts, currentBlockEnd, appender) + if err != nil { + errs = append(errs, err) + } + } + + ts = currentBlockEnd + _, err := importer.writer.Flush(ctx) + if err != nil { + errs = append(errs, err) + } + + err = appender.Commit() if err != nil { errs = append(errs, err) } } } - _, err := importer.writer.Flush(ctx) - if err != nil { - errs = append(errs, err) - } return errs } // importRule imports the historical data for a single rule. -func (importer *ruleImporter) importRule(ctx context.Context, ruleExpr string, stimeWithAlignment time.Time, internval time.Duration) error { - ts := stimeWithAlignment - appender := importer.writer.Appender(ctx) - - for ts.Before(importer.config.End) { - currentBlockEnd := ts.Add(time.Duration(tsdb.DefaultBlockDuration) * time.Millisecond) - if currentBlockEnd.After(importer.config.End) { - currentBlockEnd = importer.config.End - } - - val, warnings, err := importer.apiClient.QueryRange(ctx, - ruleExpr, - v1.Range{ - Start: ts, - End: currentBlockEnd, - Step: importer.config.EvalInterval, - }, - ) - if err != nil { - return err - } - if warnings != nil { - fmt.Fprint(os.Stderr, "warning api.QueryRange:", warnings) - } - - var matrix model.Matrix - switch val.Type() { - case model.ValMatrix: - matrix = val.(model.Matrix) - for _, sample := range matrix { - currentLabels := make(labels.Labels, 0, len(sample.Metric)) - for k, v := range sample.Metric { - currentLabels = append(currentLabels, labels.Label{ - Name: string(k), - Value: string(v), - }) - } - for _, value := range sample.Values { - _, err := appender.Add(currentLabels, value.Timestamp.Unix(), float64(value.Value)) - if err != nil { - return err - } - } - } - default: - return errors.New("rule result is wrong type") - } - - ts = currentBlockEnd - } - _, err := importer.writer.Flush(ctx) +func (importer *ruleImporter) importRule(ctx context.Context, ruleExpr, ruleName string, ruleLabels labels.Labels, start, end time.Time, appender storage.Appender) error { + val, warnings, err := importer.apiClient.QueryRange(ctx, + ruleExpr, + v1.Range{ + Start: start, + End: end, + Step: importer.config.EvalInterval, + }, + ) if err != nil { return err } - return appender.Commit() + if warnings != nil { + fmt.Fprint(os.Stderr, "warning api.QueryRange:", warnings) + } + + var matrix model.Matrix + switch val.Type() { + case model.ValMatrix: + matrix = val.(model.Matrix) + for _, sample := range matrix { + + currentLabels := make(labels.Labels, 0, len(sample.Metric)) + currentLabels = append(currentLabels, labels.Label{ + Name: labels.MetricName, + Value: ruleName, + }) + for _, ruleLabel := range ruleLabels { + currentLabels = append(currentLabels, ruleLabel) + } + for k, v := range sample.Metric { + currentLabels = append(currentLabels, labels.Label{ + Name: string(k), + Value: string(v), + }) + } + for _, value := range sample.Values { + _, err := appender.Add(currentLabels, value.Timestamp.Unix(), float64(value.Value)) + if err != nil { + return err + } + } + } + default: + return errors.New("rule result is wrong type") + } + return nil } diff --git a/tsdb/blockwriter.go b/tsdb/blockwriter.go index b859f86d9..e1faf0196 100644 --- a/tsdb/blockwriter.go +++ b/tsdb/blockwriter.go @@ -84,10 +84,6 @@ func (w *BlockWriter) Appender(ctx context.Context) storage.Appender { // Flush implements the Writer interface. This is where actual block writing // happens. After flush completes, no writes can be done. func (w *BlockWriter) Flush(ctx context.Context) (ulid.ULID, error) { - if w.head.NumSeries() == 0 { - return ulid.ULID{}, errors.New("no series appended, aborting") - } - mint := w.head.MinTime() // Add +1 millisecond to block maxt because block intervals are half-open: [b.MinTime, b.MaxTime). // Because of this block intervals are always +1 than the total samples it includes. From 5dd3577424008e6828952092b0b36fee9adb5c67 Mon Sep 17 00:00:00 2001 From: jessicagreben Date: Sun, 22 Nov 2020 15:05:02 -0800 Subject: [PATCH 09/21] change name of promtool subcommand to create-blocks-from Signed-off-by: jessicagreben --- cmd/promtool/main.go | 20 ++++++++++---------- 1 file changed, 10 insertions(+), 10 deletions(-) diff --git a/cmd/promtool/main.go b/cmd/promtool/main.go index f32aa16df..a5e4f806f 100644 --- a/cmd/promtool/main.go +++ b/cmd/promtool/main.go @@ -135,16 +135,16 @@ func main() { dumpMinTime := tsdbDumpCmd.Flag("min-time", "Minimum timestamp to dump.").Default(strconv.FormatInt(math.MinInt64, 10)).Int64() dumpMaxTime := tsdbDumpCmd.Flag("max-time", "Maximum timestamp to dump.").Default(strconv.FormatInt(math.MaxInt64, 10)).Int64() - backfillCmd := app.Command("backfill", "Backfill Prometheus data.") - backfillRuleCmd := backfillCmd.Command("rules", "Backfill Prometheus data for new rules.") - backfillRuleStart := backfillRuleCmd.Flag("start", "The time to start backfilling the new rule from. It is required. Start time should be RFC3339 or Unix timestamp."). + createBlocksFromCmd := tsdbCmd.Command("create-blocks-from", "Create blocks from new input data.") + createBlocksFromRulesCmd := createBlocksFromCmd.Command("rules", "Create new blocks of data from Prometheus data for new rules.") + createBlocksFromRulesStart := createBlocksFromRulesCmd.Flag("start", "The time to start backfilling the new rule from. It is required. Start time should be RFC3339 or Unix timestamp."). Required().String() - backfillRuleEnd := backfillRuleCmd.Flag("end", "If an end time is provided, all recording rules in the rule files provided will be backfilled to the end time. Default will backfill up to 3 hrs ago. End time should be RFC3339 or Unix timestamp.").String() - backfillOutputDir := backfillRuleCmd.Flag("output dir", "The filepath on the local filesystem to write the output to. Output will be blocks containing the data of the backfilled recording rules.").Default("backfilldata/").String() - backfillRuleURL := backfillRuleCmd.Flag("url", "Prometheus API url with the data where the rule will be backfilled from.").Default("http://localhost:9090").String() - backfillRuleEvalInterval := backfillRuleCmd.Flag("evaluation_interval_default", "How frequently to evaluate rules when backfilling if a value is not set in the rules file."). + createBlocksFromRulesEnd := createBlocksFromRulesCmd.Flag("end", "If an end time is provided, all recording rules in the rule files provided will be backfilled to the end time. Default will backfill up to 3 hrs ago. End time should be RFC3339 or Unix timestamp.").String() + createBlocksFromRulesOutputDir := createBlocksFromRulesCmd.Flag("output dir", "The filepath on the local filesystem to write the output to. Output will be blocks containing the data of the backfilled recording rules.").Default("backfilldata/").String() + createBlocksFromRulesURL := createBlocksFromRulesCmd.Flag("url", "Prometheus API url with the data where the rule will be backfilled from.").Default("http://localhost:9090").String() + createBlocksFromRulesEvalInterval := createBlocksFromRulesCmd.Flag("evaluation-interval-default", "How frequently to evaluate rules when backfilling if a value is not set in the rules file."). Default("60s").Duration() - backfillRuleFiles := backfillRuleCmd.Arg( + createBlocksFromRulesFiles := createBlocksFromRulesCmd.Arg( "rule-files", "A list of one or more files containing recording rules to be backfilled. All recording rules listed in the files will be backfilled. Alerting rules are not evaluated.", ).Required().ExistingFiles() @@ -205,8 +205,8 @@ func main() { case tsdbDumpCmd.FullCommand(): os.Exit(checkErr(dumpSamples(*dumpPath, *dumpMinTime, *dumpMaxTime))) - case backfillRuleCmd.FullCommand(): - os.Exit(checkErr(BackfillRule(*backfillRuleURL, *backfillRuleStart, *backfillRuleEnd, *backfillOutputDir, *backfillRuleEvalInterval, *backfillRuleFiles...))) + case createBlocksFromRulesCmd.FullCommand(): + os.Exit(checkErr(BackfillRule(*createBlocksFromRulesURL, *createBlocksFromRulesStart, *createBlocksFromRulesEnd, *createBlocksFromRulesOutputDir, *createBlocksFromRulesEvalInterval, *createBlocksFromRulesFiles...))) } } From ee85c22adbc58308807b2c15d994d342afc4290e Mon Sep 17 00:00:00 2001 From: jessicagreben Date: Thu, 26 Nov 2020 08:30:06 -0800 Subject: [PATCH 10/21] flush samples to disk every 5k samples Signed-off-by: jessicagreben --- cmd/promtool/main.go | 31 +++++--- cmd/promtool/rules.go | 179 ++++++++++++++++++------------------------ 2 files changed, 96 insertions(+), 114 deletions(-) diff --git a/cmd/promtool/main.go b/cmd/promtool/main.go index a5e4f806f..3b87f0db8 100644 --- a/cmd/promtool/main.go +++ b/cmd/promtool/main.go @@ -44,6 +44,7 @@ import ( "github.com/prometheus/prometheus/discovery/file" "github.com/prometheus/prometheus/discovery/kubernetes" "github.com/prometheus/prometheus/pkg/rulefmt" + "github.com/prometheus/prometheus/tsdb" _ "github.com/prometheus/prometheus/discovery/install" // Register service discovery implementations. ) @@ -140,7 +141,7 @@ func main() { createBlocksFromRulesStart := createBlocksFromRulesCmd.Flag("start", "The time to start backfilling the new rule from. It is required. Start time should be RFC3339 or Unix timestamp."). Required().String() createBlocksFromRulesEnd := createBlocksFromRulesCmd.Flag("end", "If an end time is provided, all recording rules in the rule files provided will be backfilled to the end time. Default will backfill up to 3 hrs ago. End time should be RFC3339 or Unix timestamp.").String() - createBlocksFromRulesOutputDir := createBlocksFromRulesCmd.Flag("output dir", "The filepath on the local filesystem to write the output to. Output will be blocks containing the data of the backfilled recording rules.").Default("backfilldata/").String() + createBlocksFromRulesOutputDir := createBlocksFromRulesCmd.Flag("output dir", "The filepath on the local filesystem to write the output to. Output will be blocks containing the data of the backfilled recording rules. Don't use an active Prometheus data directory. If command is run many times with same start/end time, it will create duplicate series.").Default("backfilldata/").String() createBlocksFromRulesURL := createBlocksFromRulesCmd.Flag("url", "Prometheus API url with the data where the rule will be backfilled from.").Default("http://localhost:9090").String() createBlocksFromRulesEvalInterval := createBlocksFromRulesCmd.Flag("evaluation-interval-default", "How frequently to evaluate rules when backfilling if a value is not set in the rules file."). Default("60s").Duration() @@ -206,7 +207,7 @@ func main() { os.Exit(checkErr(dumpSamples(*dumpPath, *dumpMinTime, *dumpMaxTime))) case createBlocksFromRulesCmd.FullCommand(): - os.Exit(checkErr(BackfillRule(*createBlocksFromRulesURL, *createBlocksFromRulesStart, *createBlocksFromRulesEnd, *createBlocksFromRulesOutputDir, *createBlocksFromRulesEvalInterval, *createBlocksFromRulesFiles...))) + os.Exit(checkErr(CreateBlocksFromRules(*createBlocksFromRulesURL, *createBlocksFromRulesStart, *createBlocksFromRulesEnd, *createBlocksFromRulesOutputDir, *createBlocksFromRulesEvalInterval, *createBlocksFromRulesFiles...))) } } @@ -782,9 +783,9 @@ func (j *jsonPrinter) printLabelValues(v model.LabelValues) { json.NewEncoder(os.Stdout).Encode(v) } -// BackfillRule backfills recording rules from the files provided. The output are blocks of data +// CreateBlocksFromRules backfills recording rules from the files provided. The output are blocks of data // at the outputDir location. -func BackfillRule(url, start, end, outputDir string, evalInterval time.Duration, files ...string) error { +func CreateBlocksFromRules(url, start, end, outputDir string, evalInterval time.Duration, files ...string) error { ctx := context.Background() var stime, etime time.Time var err error @@ -809,19 +810,29 @@ func BackfillRule(url, start, end, outputDir string, evalInterval time.Duration, return nil } + logger := log.NewLogfmtLogger(log.NewSyncWriter(os.Stderr)) + writer, err := tsdb.NewBlockWriter(logger, + outputDir, + tsdb.DefaultBlockDuration, + ) + if err != nil { + fmt.Fprintln(os.Stderr, "new writer error", err) + return err + } + cfg := ruleImporterConfig{ Start: stime, End: etime, - OutputDir: outputDir, EvalInterval: evalInterval, - URL: url, } - logger := log.NewLogfmtLogger(log.NewSyncWriter(os.Stderr)) - ruleImporter := newRuleImporter(logger, cfg) - if err = ruleImporter.init(); err != nil { - fmt.Fprintln(os.Stderr, "rule importer init error", err) + c, err := api.NewClient(api.Config{ + Address: url, + }) + if err != nil { + fmt.Fprintln(os.Stderr, "new api client error", err) return err } + ruleImporter := newRuleImporter(logger, cfg, v1.NewAPI(c), newMultipleAppender(ctx, maxSamplesInMemory, writer)) errs := ruleImporter.loadGroups(ctx, files) for _, err := range errs { diff --git a/cmd/promtool/rules.go b/cmd/promtool/rules.go index 4533e6ebb..359dcc0aa 100644 --- a/cmd/promtool/rules.go +++ b/cmd/promtool/rules.go @@ -22,7 +22,6 @@ import ( "github.com/go-kit/kit/log" "github.com/go-kit/kit/log/level" "github.com/pkg/errors" - "github.com/prometheus/client_golang/api" v1 "github.com/prometheus/client_golang/api/prometheus/v1" "github.com/prometheus/common/model" "github.com/prometheus/prometheus/pkg/labels" @@ -31,160 +30,88 @@ import ( "github.com/prometheus/prometheus/tsdb" ) +const maxSamplesInMemory = 5000 + +type queryRangeAPI interface { + QueryRange(ctx context.Context, query string, r v1.Range) (model.Value, v1.Warnings, error) +} + // ruleImporter is the importer to backfill rules. type ruleImporter struct { logger log.Logger config ruleImporterConfig + apiClient queryRangeAPI + + appender *multipleAppender + groups map[string]*rules.Group - groupLoader rules.GroupLoader - - apiClient v1.API - - writer *tsdb.BlockWriter + ruleManager *rules.Manager } // ruleImporterConfig is the config for the rule importer. type ruleImporterConfig struct { Start time.Time End time.Time - OutputDir string EvalInterval time.Duration - URL string } // newRuleImporter creates a new rule importer that can be used to backfill rules. -func newRuleImporter(logger log.Logger, config ruleImporterConfig) *ruleImporter { +func newRuleImporter(logger log.Logger, config ruleImporterConfig, apiClient queryRangeAPI, appender *multipleAppender) *ruleImporter { return &ruleImporter{ logger: logger, config: config, - groupLoader: rules.FileLoader{}, + apiClient: apiClient, + appender: appender, + ruleManager: rules.NewManager(&rules.ManagerOptions{}), } } -// init initializes the rule importer which creates a new block writer -// and creates an Prometheus API client. -func (importer *ruleImporter) init() error { - w, err := tsdb.NewBlockWriter(importer.logger, - importer.config.OutputDir, - tsdb.DefaultBlockDuration, - ) - if err != nil { - return err - } - importer.writer = w - - config := api.Config{ - Address: importer.config.URL, - } - c, err := api.NewClient(config) - if err != nil { - return err - } - importer.apiClient = v1.NewAPI(c) - return nil -} - -// close cleans up any open resources. -func (importer *ruleImporter) close() error { - return importer.writer.Close() -} - -// loadGroups reads groups from a list of rule files. +// loadGroups parses groups from a list of rule files. func (importer *ruleImporter) loadGroups(ctx context.Context, filenames []string) (errs []error) { - groups := make(map[string]*rules.Group) - - for _, filename := range filenames { - rgs, errs := importer.groupLoader.Load(filename) - if errs != nil { - return errs - } - - for _, ruleGroup := range rgs.Groups { - itv := importer.config.EvalInterval - if ruleGroup.Interval != 0 { - itv = time.Duration(ruleGroup.Interval) - } - - rgRules := make([]rules.Rule, 0, len(ruleGroup.Rules)) - for _, r := range ruleGroup.Rules { - expr, err := importer.groupLoader.Parse(r.Expr.Value) - if err != nil { - return []error{errors.Wrap(err, filename)} - } - rgRules = append(rgRules, rules.NewRecordingRule( - r.Record.Value, - expr, - labels.FromMap(r.Labels), - )) - } - - groups[rules.GroupKey(filename, ruleGroup.Name)] = rules.NewGroup(rules.GroupOptions{ - Name: ruleGroup.Name, - File: filename, - Interval: itv, - Rules: rgRules, - Opts: &rules.ManagerOptions{}, - }) - } + groups, errs := importer.ruleManager.LoadGroups(importer.config.EvalInterval, labels.Labels{}, filenames...) + if errs != nil { + return errs } - importer.groups = groups return nil } // importAll evaluates all the groups and rules and creates new time series // and stores them in new blocks. -func (importer *ruleImporter) importAll(ctx context.Context) []error { - var errs = []error{} - var currentBlockEnd time.Time - var appender storage.Appender - +func (importer *ruleImporter) importAll(ctx context.Context) (errs []error) { for name, group := range importer.groups { level.Info(importer.logger).Log("backfiller", fmt.Sprintf("processing group, name: %s", name)) stimeWithAlignment := group.EvalTimestamp(importer.config.Start.UnixNano()) - ts := stimeWithAlignment - // a 2-hr block that contains all the data for each rule - for ts.Before(importer.config.End) { - currentBlockEnd = ts.Add(time.Duration(tsdb.DefaultBlockDuration) * time.Millisecond) + for stimeWithAlignment.Before(importer.config.End) { + + currentBlockEnd := stimeWithAlignment.Add(time.Duration(tsdb.DefaultBlockDuration) * time.Millisecond) if currentBlockEnd.After(importer.config.End) { currentBlockEnd = importer.config.End } - // should we be creating a new appender for each block? - appender = importer.writer.Appender(ctx) for i, r := range group.Rules() { level.Info(importer.logger).Log("backfiller", fmt.Sprintf("processing rule %d, name: %s", i+1, r.Name())) - err := importer.importRule(ctx, r.Query().String(), r.Name(), r.Labels(), ts, currentBlockEnd, appender) - if err != nil { + if err := importer.importRule(ctx, r.Query().String(), r.Name(), r.Labels(), stimeWithAlignment, currentBlockEnd); err != nil { errs = append(errs, err) } } - ts = currentBlockEnd - _, err := importer.writer.Flush(ctx) - if err != nil { - errs = append(errs, err) - } - - err = appender.Commit() - if err != nil { - errs = append(errs, err) - } + stimeWithAlignment = currentBlockEnd } } return errs } // importRule imports the historical data for a single rule. -func (importer *ruleImporter) importRule(ctx context.Context, ruleExpr, ruleName string, ruleLabels labels.Labels, start, end time.Time, appender storage.Appender) error { +func (importer *ruleImporter) importRule(ctx context.Context, ruleExpr, ruleName string, ruleLabels labels.Labels, start, end time.Time) error { val, warnings, err := importer.apiClient.QueryRange(ctx, ruleExpr, v1.Range{ Start: start, End: end, - Step: importer.config.EvalInterval, + Step: importer.config.EvalInterval, // todo: did we check if the rule has an interval? }, ) if err != nil { @@ -200,7 +127,7 @@ func (importer *ruleImporter) importRule(ctx context.Context, ruleExpr, ruleName matrix = val.(model.Matrix) for _, sample := range matrix { - currentLabels := make(labels.Labels, 0, len(sample.Metric)) + currentLabels := make(labels.Labels, 0, len(sample.Metric)+len(ruleLabels)+1) currentLabels = append(currentLabels, labels.Label{ Name: labels.MetricName, Value: ruleName, @@ -215,14 +142,58 @@ func (importer *ruleImporter) importRule(ctx context.Context, ruleExpr, ruleName }) } for _, value := range sample.Values { - _, err := appender.Add(currentLabels, value.Timestamp.Unix(), float64(value.Value)) - if err != nil { + if err := importer.appender.add(ctx, currentLabels, value.Timestamp.Unix(), float64(value.Value)); err != nil { return err } } } default: - return errors.New("rule result is wrong type") + return errors.New(fmt.Sprintf("rule result is wrong type %s", val.Type().String())) } return nil } + +// multipleAppender keeps track of how many series have been added to the current appender. +// If the max samples have been added, then all series are flushed to disk and commited and a new +// appender is created. +type multipleAppender struct { + maxSamplesInMemory int + currentSampleCount int + writer *tsdb.BlockWriter + appender storage.Appender +} + +func newMultipleAppender(ctx context.Context, maxSamplesInMemory int, blockWriter *tsdb.BlockWriter) *multipleAppender { + return &multipleAppender{ + maxSamplesInMemory: maxSamplesInMemory, + writer: blockWriter, + appender: blockWriter.Appender(ctx), + } +} + +func (m *multipleAppender) add(ctx context.Context, l labels.Labels, t int64, v float64) error { + if _, err := m.appender.Add(l, t, v); err != nil { + return err + } + m.currentSampleCount++ + if m.currentSampleCount > m.maxSamplesInMemory { + return m.flushAndCommit(ctx) + } + return nil +} + +func (m *multipleAppender) flushAndCommit(ctx context.Context) error { + if _, err := m.writer.Flush(ctx); err != nil { + return err + } + if err := m.appender.Commit(); err != nil { + return err + } + m.appender = m.writer.Appender(ctx) + m.currentSampleCount = 0 + return nil +} + +func (m *multipleAppender) close() error { + return m.writer.Close() +} From 2e9946e4d7407df3feaf673298e58c33b96f799b Mon Sep 17 00:00:00 2001 From: jessicagreben Date: Sat, 28 Nov 2020 07:58:33 -0800 Subject: [PATCH 11/21] add test Signed-off-by: jessicagreben --- cmd/promtool/main.go | 14 +-- cmd/promtool/rules.go | 132 ++++++++++++++-------------- cmd/promtool/rules_test.go | 171 +++++++++++++++++++++++++++++++++++++ 3 files changed, 244 insertions(+), 73 deletions(-) create mode 100644 cmd/promtool/rules_test.go diff --git a/cmd/promtool/main.go b/cmd/promtool/main.go index 2e9b032f3..5e6e7bea0 100644 --- a/cmd/promtool/main.go +++ b/cmd/promtool/main.go @@ -46,8 +46,6 @@ import ( "github.com/prometheus/prometheus/discovery/kubernetes" "github.com/prometheus/prometheus/pkg/rulefmt" "github.com/prometheus/prometheus/tsdb" - - _ "github.com/prometheus/prometheus/discovery/install" // Register service discovery implementations. ) func main() { @@ -216,7 +214,7 @@ func main() { os.Exit(checkErr(backfillOpenMetrics(*importFilePath, *importDBPath))) case importRulesCmd.FullCommand(): - os.Exit(checkErr(ImportRules(*importRulesURL, *importRulesStart, *importRulesEnd, *importRulesOutputDir, *importRulesEvalInterval, *importRulesFiles...))) + os.Exit(checkErr(importRules(*importRulesURL, *importRulesStart, *importRulesEnd, *importRulesOutputDir, *importRulesEvalInterval, *importRulesFiles...))) } } @@ -792,9 +790,9 @@ func (j *jsonPrinter) printLabelValues(v model.LabelValues) { json.NewEncoder(os.Stdout).Encode(v) } -// ImportRules backfills recording rules from the files provided. The output are blocks of data +// importRules backfills recording rules from the files provided. The output are blocks of data // at the outputDir location. -func ImportRules(url, start, end, outputDir string, evalInterval time.Duration, files ...string) error { +func importRules(url, start, end, outputDir string, evalInterval time.Duration, files ...string) error { ctx := context.Background() var stime, etime time.Time var err error @@ -828,6 +826,9 @@ func ImportRules(url, start, end, outputDir string, evalInterval time.Duration, fmt.Fprintln(os.Stderr, "new writer error", err) return err } + defer func() { + err = writer.Close() + }() cfg := ruleImporterConfig{ Start: stime, @@ -841,6 +842,7 @@ func ImportRules(url, start, end, outputDir string, evalInterval time.Duration, fmt.Fprintln(os.Stderr, "new api client error", err) return err } + const maxSamplesInMemory = 5000 ruleImporter := newRuleImporter(logger, cfg, v1.NewAPI(c), newMultipleAppender(ctx, maxSamplesInMemory, writer)) errs := ruleImporter.loadGroups(ctx, files) @@ -858,5 +860,5 @@ func ImportRules(url, start, end, outputDir string, evalInterval time.Duration, } } - return nil + return err } diff --git a/cmd/promtool/rules.go b/cmd/promtool/rules.go index 359dcc0aa..f7a2b2bc5 100644 --- a/cmd/promtool/rules.go +++ b/cmd/promtool/rules.go @@ -16,7 +16,6 @@ package main import ( "context" "fmt" - "os" "time" "github.com/go-kit/kit/log" @@ -25,18 +24,16 @@ import ( v1 "github.com/prometheus/client_golang/api/prometheus/v1" "github.com/prometheus/common/model" "github.com/prometheus/prometheus/pkg/labels" + "github.com/prometheus/prometheus/pkg/timestamp" "github.com/prometheus/prometheus/rules" "github.com/prometheus/prometheus/storage" "github.com/prometheus/prometheus/tsdb" ) -const maxSamplesInMemory = 5000 - type queryRangeAPI interface { QueryRange(ctx context.Context, query string, r v1.Range) (model.Value, v1.Warnings, error) } -// ruleImporter is the importer to backfill rules. type ruleImporter struct { logger log.Logger config ruleImporterConfig @@ -49,14 +46,14 @@ type ruleImporter struct { ruleManager *rules.Manager } -// ruleImporterConfig is the config for the rule importer. type ruleImporterConfig struct { Start time.Time End time.Time EvalInterval time.Duration } -// newRuleImporter creates a new rule importer that can be used to backfill rules. +// newRuleImporter creates a new rule importer that can be used to parse and evaluate recording rule files and create new series +// written to disk in blocks. func newRuleImporter(logger log.Logger, config ruleImporterConfig, apiClient queryRangeAPI, appender *multipleAppender) *ruleImporter { return &ruleImporter{ logger: logger, @@ -67,7 +64,7 @@ func newRuleImporter(logger log.Logger, config ruleImporterConfig, apiClient que } } -// loadGroups parses groups from a list of rule files. +// loadGroups parses groups from a list of recording rule files. func (importer *ruleImporter) loadGroups(ctx context.Context, filenames []string) (errs []error) { groups, errs := importer.ruleManager.LoadGroups(importer.config.EvalInterval, labels.Labels{}, filenames...) if errs != nil { @@ -77,78 +74,80 @@ func (importer *ruleImporter) loadGroups(ctx context.Context, filenames []string return nil } -// importAll evaluates all the groups and rules and creates new time series -// and stores them in new blocks. +// importAll evaluates all the recording rules and creates new time series and writes them to disk in blocks. func (importer *ruleImporter) importAll(ctx context.Context) (errs []error) { for name, group := range importer.groups { level.Info(importer.logger).Log("backfiller", fmt.Sprintf("processing group, name: %s", name)) + stimeWithAlignment := group.EvalTimestamp(importer.config.Start.UnixNano()) - - for stimeWithAlignment.Before(importer.config.End) { - - currentBlockEnd := stimeWithAlignment.Add(time.Duration(tsdb.DefaultBlockDuration) * time.Millisecond) - if currentBlockEnd.After(importer.config.End) { - currentBlockEnd = importer.config.End + for i, r := range group.Rules() { + level.Info(importer.logger).Log("backfiller", fmt.Sprintf("processing rule %d, name: %s", i+1, r.Name())) + if err := importer.importRule(ctx, r.Query().String(), r.Name(), r.Labels(), stimeWithAlignment, importer.config.End, group.Interval()); err != nil { + errs = append(errs, err) } - - for i, r := range group.Rules() { - level.Info(importer.logger).Log("backfiller", fmt.Sprintf("processing rule %d, name: %s", i+1, r.Name())) - if err := importer.importRule(ctx, r.Query().String(), r.Name(), r.Labels(), stimeWithAlignment, currentBlockEnd); err != nil { - errs = append(errs, err) - } - } - - stimeWithAlignment = currentBlockEnd } } + if err := importer.appender.flushAndCommit(ctx); err != nil { + errs = append(errs, err) + } return errs } -// importRule imports the historical data for a single rule. -func (importer *ruleImporter) importRule(ctx context.Context, ruleExpr, ruleName string, ruleLabels labels.Labels, start, end time.Time) error { - val, warnings, err := importer.apiClient.QueryRange(ctx, - ruleExpr, - v1.Range{ - Start: start, - End: end, - Step: importer.config.EvalInterval, // todo: did we check if the rule has an interval? - }, - ) - if err != nil { - return err - } - if warnings != nil { - fmt.Fprint(os.Stderr, "warning api.QueryRange:", warnings) - } +// importRule queries a prometheus API to evaluate rules at times in the past. +func (importer *ruleImporter) importRule(ctx context.Context, ruleExpr, ruleName string, ruleLabels labels.Labels, start, end time.Time, interval time.Duration) error { - var matrix model.Matrix - switch val.Type() { - case model.ValMatrix: - matrix = val.(model.Matrix) - for _, sample := range matrix { + // This loop breaks up the calls to the QueryRange API into 2 hr chunks so that we + // don't ever request too much data or take to long to process to avoid timeout. + for start.Before(end) { + currentBlockEnd := start.Add(time.Duration(tsdb.DefaultBlockDuration) * time.Millisecond) + if currentBlockEnd.After(end) { + currentBlockEnd = end + } - currentLabels := make(labels.Labels, 0, len(sample.Metric)+len(ruleLabels)+1) - currentLabels = append(currentLabels, labels.Label{ - Name: labels.MetricName, - Value: ruleName, - }) - for _, ruleLabel := range ruleLabels { - currentLabels = append(currentLabels, ruleLabel) - } - for k, v := range sample.Metric { + val, warnings, err := importer.apiClient.QueryRange(ctx, + ruleExpr, + v1.Range{ + Start: start, + End: end, + Step: interval, + }, + ) + if err != nil { + return err + } + if warnings != nil { + level.Warn(importer.logger).Log("backfiller", fmt.Sprintf("warnings QueryRange api: %v", warnings)) + } + + var matrix model.Matrix + switch val.Type() { + case model.ValMatrix: + matrix = val.(model.Matrix) + for _, sample := range matrix { + currentLabels := make(labels.Labels, 0, len(sample.Metric)+len(ruleLabels)+1) currentLabels = append(currentLabels, labels.Label{ - Name: string(k), - Value: string(v), + Name: labels.MetricName, + Value: ruleName, }) - } - for _, value := range sample.Values { - if err := importer.appender.add(ctx, currentLabels, value.Timestamp.Unix(), float64(value.Value)); err != nil { - return err + for _, ruleLabel := range ruleLabels { + currentLabels = append(currentLabels, ruleLabel) + } + for name, value := range sample.Metric { + currentLabels = append(currentLabels, labels.Label{ + Name: string(name), + Value: string(value), + }) + } + for _, value := range sample.Values { + if err := importer.appender.add(ctx, currentLabels, timestamp.FromTime(value.Timestamp.Time()), float64(value.Value)); err != nil { + return err + } } } + default: + return errors.New(fmt.Sprintf("rule result is wrong type %s", val.Type().String())) } - default: - return errors.New(fmt.Sprintf("rule result is wrong type %s", val.Type().String())) + start = currentBlockEnd } return nil } @@ -176,13 +175,16 @@ func (m *multipleAppender) add(ctx context.Context, l labels.Labels, t int64, v return err } m.currentSampleCount++ - if m.currentSampleCount > m.maxSamplesInMemory { + if m.currentSampleCount >= m.maxSamplesInMemory { return m.flushAndCommit(ctx) } return nil } func (m *multipleAppender) flushAndCommit(ctx context.Context) error { + if m.currentSampleCount == 0 { + return nil + } if _, err := m.writer.Flush(ctx); err != nil { return err } @@ -193,7 +195,3 @@ func (m *multipleAppender) flushAndCommit(ctx context.Context) error { m.currentSampleCount = 0 return nil } - -func (m *multipleAppender) close() error { - return m.writer.Close() -} diff --git a/cmd/promtool/rules_test.go b/cmd/promtool/rules_test.go new file mode 100644 index 000000000..ac667ed84 --- /dev/null +++ b/cmd/promtool/rules_test.go @@ -0,0 +1,171 @@ +// Copyright 2020 The Prometheus Authors +// 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 main + +import ( + "context" + "fmt" + "io/ioutil" + "math" + "os" + "testing" + "time" + + "github.com/go-kit/kit/log" + v1 "github.com/prometheus/client_golang/api/prometheus/v1" + "github.com/prometheus/common/model" + "github.com/prometheus/prometheus/pkg/labels" + "github.com/prometheus/prometheus/tsdb" + "github.com/stretchr/testify/require" +) + +const testMaxSampleCount = 500 + +type mockQueryRangeAPI struct{} + +func (mockAPI mockQueryRangeAPI) QueryRange(ctx context.Context, query string, r v1.Range) (model.Value, v1.Warnings, error) { + var testMatrix model.Matrix = []*model.SampleStream{ + { + Metric: model.Metric{ + "labelname1": "val1", + }, + Values: []model.SamplePair{ + { + Timestamp: model.Time(123456789123), + Value: 123, + }, + }, + }, + } + return testMatrix, v1.Warnings{}, nil +} + +// TestBackfillRuleIntegration is an integration test that runs all the rule importer code to confirm the parts work together. +func TestBackfillRuleIntegration(t *testing.T) { + tmpDir, err := ioutil.TempDir("", "backfilldata") + require.NoError(t, err) + defer func() { + require.NoError(t, os.RemoveAll(tmpDir)) + }() + start := time.Now().UTC() + ctx := context.Background() + + const ( + groupName = "test_group_name" + ruleName1 = "test_rule1_name" + ruleExpr = "test_expr" + ruleLabels = "test_label_name: test_label_value" + ) + + // Execute test two times in a row to simulate running the rule importer twice with the same data. + // We expect that duplicate blocks with the same series are created when run more than once. + for i := 0; i < 2; i++ { + ruleImporter, err := newTestRuleImporter(ctx, start, tmpDir) + require.NoError(t, err) + path := tmpDir + "/test.file" + require.NoError(t, createTestFiles(groupName, ruleName1, ruleExpr, ruleLabels, path)) + + // After loading/parsing the recording rule files make sure the parsing was correct. + errs := ruleImporter.loadGroups(ctx, []string{path}) + for _, err := range errs { + require.NoError(t, err) + } + const groupCount = 1 + require.Equal(t, groupCount, len(ruleImporter.groups)) + groupNameWithPath := path + ";" + groupName + group1 := ruleImporter.groups[groupNameWithPath] + require.NotNil(t, group1) + const defaultInterval = 60 + require.Equal(t, time.Duration(defaultInterval*time.Second), group1.Interval()) + gRules := group1.Rules() + const ruleCount = 1 + require.Equal(t, ruleCount, len(gRules)) + require.Equal(t, ruleName1, gRules[0].Name()) + require.Equal(t, ruleExpr, gRules[0].Query().String()) + require.Equal(t, 1, len(gRules[0].Labels())) + + // Backfill all recording rules then check the blocks to confirm the right + // data was created. + errs = ruleImporter.importAll(ctx) + for _, err := range errs { + require.NoError(t, err) + } + + opts := tsdb.DefaultOptions() + opts.AllowOverlappingBlocks = true + db, err := tsdb.Open(tmpDir, nil, nil, opts) + require.NoError(t, err) + + blocks := db.Blocks() + require.Equal(t, i+1, len(blocks)) + + q, err := db.Querier(context.Background(), math.MinInt64, math.MaxInt64) + require.NoError(t, err) + + selectedSeries := q.Select(false, nil, labels.MustNewMatcher(labels.MatchRegexp, "", ".*")) + var seriesCount, samplesCount int + for selectedSeries.Next() { + seriesCount++ + series := selectedSeries.At() + require.Equal(t, 3, len(series.Labels())) + it := series.Iterator() + for it.Next() { + samplesCount++ + _, v := it.At() + require.Equal(t, 123, v) + } + require.NoError(t, it.Err()) + } + require.NoError(t, selectedSeries.Err()) + require.Equal(t, 1, seriesCount) + require.Equal(t, 2, samplesCount) + require.NoError(t, q.Close()) + require.NoError(t, db.Close()) + } + +} + +func newTestRuleImporter(ctx context.Context, start time.Time, tmpDir string) (*ruleImporter, error) { + logger := log.NewNopLogger() + cfg := ruleImporterConfig{ + Start: start.Add(-1 * time.Hour), + End: start, + EvalInterval: 60 * time.Second, + } + writer, err := tsdb.NewBlockWriter(logger, + tmpDir, + tsdb.DefaultBlockDuration, + ) + if err != nil { + return nil, err + } + + app := newMultipleAppender(ctx, testMaxSampleCount, writer) + return newRuleImporter(logger, cfg, mockQueryRangeAPI{}, app), nil +} + +func createTestFiles(groupName, ruleName, ruleExpr, ruleLabels, path string) error { + x := fmt.Sprintf(` +groups: +- name: %s + rules: + - record: %s + expr: %s + labels: + %s +`, + groupName, ruleName, ruleExpr, ruleLabels, + ) + return ioutil.WriteFile(path, []byte(x), 0777) +} From cec3515fa3660de0307f88a3ee88778e6828722d Mon Sep 17 00:00:00 2001 From: jessicagreben Date: Mon, 30 Nov 2020 08:17:51 -0800 Subject: [PATCH 12/21] fix linter Signed-off-by: jessicagreben --- cmd/promtool/main.go | 8 ++++---- cmd/promtool/rules.go | 6 +++--- 2 files changed, 7 insertions(+), 7 deletions(-) diff --git a/cmd/promtool/main.go b/cmd/promtool/main.go index 5e6e7bea0..a1166a566 100644 --- a/cmd/promtool/main.go +++ b/cmd/promtool/main.go @@ -140,13 +140,13 @@ func main() { // TODO(aSquare14): add flag to set default block duration importFilePath := openMetricsImportCmd.Arg("input file", "OpenMetrics file to read samples from.").Required().String() importDBPath := openMetricsImportCmd.Arg("output directory", "Output directory for generated blocks.").Default(defaultDBPath).String() - importRulesCmd := importCmd.Command("rules", "Create new blocks of data from Prometheus data for new rules.") + importRulesCmd := importCmd.Command("rules", "Create new blocks of data from Prometheus data for new rules from recording rule files.") importRulesStart := importRulesCmd.Flag("start", "The time to start backfilling the new rule from. It is required. Start time should be RFC3339 or Unix timestamp."). Required().String() importRulesEnd := importRulesCmd.Flag("end", "If an end time is provided, all recording rules in the rule files provided will be backfilled to the end time. Default will backfill up to 3 hrs ago. End time should be RFC3339 or Unix timestamp.").String() - importRulesOutputDir := importRulesCmd.Flag("output dir", "The filepath on the local filesystem to write the output to. Output will be blocks containing the data of the backfilled recording rules. Don't use an active Prometheus data directory. If command is run many times with same start/end time, it will create duplicate series.").Default("backfilldata/").String() - importRulesURL := importRulesCmd.Flag("url", "Prometheus API url with the data where the rule will be backfilled from.").Default("http://localhost:9090").String() - importRulesEvalInterval := importRulesCmd.Flag("evaluation-interval-default", "How frequently to evaluate rules when backfilling if a value is not set in the rules file."). + importRulesOutputDir := importRulesCmd.Flag("output-dir", "The filepath on the local filesystem to write the output to. Output will be blocks containing the data of the backfilled recording rules. Don't use an active Prometheus data directory. If command is run many times with same start/end time, it will create duplicate series.").Default("backfilldata/").String() + importRulesURL := importRulesCmd.Flag("url", "The URL for the Prometheus API with the data where the rule will be backfilled from.").Default("http://localhost:9090").String() + importRulesEvalInterval := importRulesCmd.Flag("eval-interval-default", "How frequently to evaluate rules when backfilling if a value is not set in the recording rule files."). Default("60s").Duration() importRulesFiles := importRulesCmd.Arg( "rule-files", diff --git a/cmd/promtool/rules.go b/cmd/promtool/rules.go index f7a2b2bc5..c9a8b24fb 100644 --- a/cmd/promtool/rules.go +++ b/cmd/promtool/rules.go @@ -129,9 +129,9 @@ func (importer *ruleImporter) importRule(ctx context.Context, ruleExpr, ruleName Name: labels.MetricName, Value: ruleName, }) - for _, ruleLabel := range ruleLabels { - currentLabels = append(currentLabels, ruleLabel) - } + + currentLabels = append(currentLabels, ruleLabels...) + for name, value := range sample.Metric { currentLabels = append(currentLabels, labels.Label{ Name: string(name), From e32e4fcc5392864c88c2be30d950919f9cf0c134 Mon Sep 17 00:00:00 2001 From: jessicagreben Date: Mon, 30 Nov 2020 11:02:45 -0800 Subject: [PATCH 13/21] fix unit test Signed-off-by: jessicagreben --- cmd/promtool/rules.go | 4 ++-- cmd/promtool/rules_test.go | 26 ++++++++++++++++---------- 2 files changed, 18 insertions(+), 12 deletions(-) diff --git a/cmd/promtool/rules.go b/cmd/promtool/rules.go index c9a8b24fb..02c72a1a0 100644 --- a/cmd/promtool/rules.go +++ b/cmd/promtool/rules.go @@ -185,10 +185,10 @@ func (m *multipleAppender) flushAndCommit(ctx context.Context) error { if m.currentSampleCount == 0 { return nil } - if _, err := m.writer.Flush(ctx); err != nil { + if err := m.appender.Commit(); err != nil { return err } - if err := m.appender.Commit(); err != nil { + if _, err := m.writer.Flush(ctx); err != nil { return err } m.appender = m.writer.Appender(ctx) diff --git a/cmd/promtool/rules_test.go b/cmd/promtool/rules_test.go index ac667ed84..427495270 100644 --- a/cmd/promtool/rules_test.go +++ b/cmd/promtool/rules_test.go @@ -30,7 +30,12 @@ import ( "github.com/stretchr/testify/require" ) -const testMaxSampleCount = 500 +const ( + testMaxSampleCount = 50 + testValue = 123 +) + +var testTime = model.Time(time.Now().Add(-20 * time.Minute).Unix()) type mockQueryRangeAPI struct{} @@ -38,12 +43,12 @@ func (mockAPI mockQueryRangeAPI) QueryRange(ctx context.Context, query string, r var testMatrix model.Matrix = []*model.SampleStream{ { Metric: model.Metric{ - "labelname1": "val1", + "name1": "val1", }, Values: []model.SamplePair{ { - Timestamp: model.Time(123456789123), - Value: 123, + Timestamp: testTime, + Value: testValue, }, }, }, @@ -71,6 +76,7 @@ func TestBackfillRuleIntegration(t *testing.T) { // Execute test two times in a row to simulate running the rule importer twice with the same data. // We expect that duplicate blocks with the same series are created when run more than once. for i := 0; i < 2; i++ { + ruleImporter, err := newTestRuleImporter(ctx, start, tmpDir) require.NoError(t, err) path := tmpDir + "/test.file" @@ -122,14 +128,15 @@ func TestBackfillRuleIntegration(t *testing.T) { it := series.Iterator() for it.Next() { samplesCount++ - _, v := it.At() - require.Equal(t, 123, v) + ts, v := it.At() + require.Equal(t, float64(testValue), v) + require.Equal(t, int64(testTime), ts) } require.NoError(t, it.Err()) } require.NoError(t, selectedSeries.Err()) require.Equal(t, 1, seriesCount) - require.Equal(t, 2, samplesCount) + require.Equal(t, 1, samplesCount) require.NoError(t, q.Close()) require.NoError(t, db.Close()) } @@ -156,8 +163,7 @@ func newTestRuleImporter(ctx context.Context, start time.Time, tmpDir string) (* } func createTestFiles(groupName, ruleName, ruleExpr, ruleLabels, path string) error { - x := fmt.Sprintf(` -groups: + recordingRules := fmt.Sprintf(`groups: - name: %s rules: - record: %s @@ -167,5 +173,5 @@ groups: `, groupName, ruleName, ruleExpr, ruleLabels, ) - return ioutil.WriteFile(path, []byte(x), 0777) + return ioutil.WriteFile(path, []byte(recordingRules), 0777) } From f2db9dc722dc509285754afe63706b53523e955a Mon Sep 17 00:00:00 2001 From: jessicagreben Date: Wed, 24 Feb 2021 09:42:31 -0800 Subject: [PATCH 14/21] add multi rule integration tests Signed-off-by: jessicagreben --- cmd/promtool/rules_test.go | 249 +++++++++++++++++++++---------------- 1 file changed, 141 insertions(+), 108 deletions(-) diff --git a/cmd/promtool/rules_test.go b/cmd/promtool/rules_test.go index 427495270..8953841fb 100644 --- a/cmd/promtool/rules_test.go +++ b/cmd/promtool/rules_test.go @@ -15,10 +15,10 @@ package main import ( "context" - "fmt" "io/ioutil" "math" "os" + "path/filepath" "testing" "time" @@ -36,114 +36,127 @@ const ( ) var testTime = model.Time(time.Now().Add(-20 * time.Minute).Unix()) +var testTime2 = model.Time(time.Now().Add(-30 * time.Minute).Unix()) -type mockQueryRangeAPI struct{} +type mockQueryRangeAPI struct { + samples model.Matrix +} func (mockAPI mockQueryRangeAPI) QueryRange(ctx context.Context, query string, r v1.Range) (model.Value, v1.Warnings, error) { - var testMatrix model.Matrix = []*model.SampleStream{ - { - Metric: model.Metric{ - "name1": "val1", - }, - Values: []model.SamplePair{ - { - Timestamp: testTime, - Value: testValue, - }, - }, - }, - } - return testMatrix, v1.Warnings{}, nil + return mockAPI.samples, v1.Warnings{}, nil } // TestBackfillRuleIntegration is an integration test that runs all the rule importer code to confirm the parts work together. func TestBackfillRuleIntegration(t *testing.T) { - tmpDir, err := ioutil.TempDir("", "backfilldata") - require.NoError(t, err) - defer func() { - require.NoError(t, os.RemoveAll(tmpDir)) - }() - start := time.Now().UTC() - ctx := context.Background() - - const ( - groupName = "test_group_name" - ruleName1 = "test_rule1_name" - ruleExpr = "test_expr" - ruleLabels = "test_label_name: test_label_value" - ) - - // Execute test two times in a row to simulate running the rule importer twice with the same data. - // We expect that duplicate blocks with the same series are created when run more than once. - for i := 0; i < 2; i++ { - - ruleImporter, err := newTestRuleImporter(ctx, start, tmpDir) - require.NoError(t, err) - path := tmpDir + "/test.file" - require.NoError(t, createTestFiles(groupName, ruleName1, ruleExpr, ruleLabels, path)) - - // After loading/parsing the recording rule files make sure the parsing was correct. - errs := ruleImporter.loadGroups(ctx, []string{path}) - for _, err := range errs { - require.NoError(t, err) - } - const groupCount = 1 - require.Equal(t, groupCount, len(ruleImporter.groups)) - groupNameWithPath := path + ";" + groupName - group1 := ruleImporter.groups[groupNameWithPath] - require.NotNil(t, group1) - const defaultInterval = 60 - require.Equal(t, time.Duration(defaultInterval*time.Second), group1.Interval()) - gRules := group1.Rules() - const ruleCount = 1 - require.Equal(t, ruleCount, len(gRules)) - require.Equal(t, ruleName1, gRules[0].Name()) - require.Equal(t, ruleExpr, gRules[0].Query().String()) - require.Equal(t, 1, len(gRules[0].Labels())) - - // Backfill all recording rules then check the blocks to confirm the right - // data was created. - errs = ruleImporter.importAll(ctx) - for _, err := range errs { - require.NoError(t, err) - } - - opts := tsdb.DefaultOptions() - opts.AllowOverlappingBlocks = true - db, err := tsdb.Open(tmpDir, nil, nil, opts) - require.NoError(t, err) - - blocks := db.Blocks() - require.Equal(t, i+1, len(blocks)) - - q, err := db.Querier(context.Background(), math.MinInt64, math.MaxInt64) - require.NoError(t, err) - - selectedSeries := q.Select(false, nil, labels.MustNewMatcher(labels.MatchRegexp, "", ".*")) - var seriesCount, samplesCount int - for selectedSeries.Next() { - seriesCount++ - series := selectedSeries.At() - require.Equal(t, 3, len(series.Labels())) - it := series.Iterator() - for it.Next() { - samplesCount++ - ts, v := it.At() - require.Equal(t, float64(testValue), v) - require.Equal(t, int64(testTime), ts) - } - require.NoError(t, it.Err()) - } - require.NoError(t, selectedSeries.Err()) - require.Equal(t, 1, seriesCount) - require.Equal(t, 1, samplesCount) - require.NoError(t, q.Close()) - require.NoError(t, db.Close()) + var testCases = []struct { + name string + runcount int + expectedBlockCount int + expectedSeriesCount int + expectedSampleCount int + samples []*model.SampleStream + }{ + {"no samples", 1, 0, 0, 0, []*model.SampleStream{}}, + {"run importer once", 1, 1, 4, 4, []*model.SampleStream{{Metric: model.Metric{"name1": "val1"}, Values: []model.SamplePair{{Timestamp: testTime, Value: testValue}}}}}, + {"one importer twice", 2, 1, 4, 4, []*model.SampleStream{{Metric: model.Metric{"name1": "val1"}, Values: []model.SamplePair{{Timestamp: testTime, Value: testValue}, {Timestamp: testTime2, Value: testValue}}}}}, } + for _, tt := range testCases { + t.Run(tt.name, func(t *testing.T) { + tmpDir, err := ioutil.TempDir("", "backfilldata") + require.NoError(t, err) + defer func() { + require.NoError(t, os.RemoveAll(tmpDir)) + }() + start := time.Now().UTC() + ctx := context.Background() + // Execute the test more than once to simulate running the rule importer twice with the same data. + // We expect that duplicate blocks with the same series are created when run more than once. + for i := 0; i < tt.runcount; i++ { + ruleImporter, err := newTestRuleImporter(ctx, start, tmpDir, tt.samples) + require.NoError(t, err) + path1 := filepath.Join(tmpDir, "test.file") + require.NoError(t, createSingleRuleTestFiles(path1)) + path2 := filepath.Join(tmpDir, "test2.file") + require.NoError(t, createMultiRuleTestFiles(path2)) + + // Confirm that the rule files were loaded in correctly. + errs := ruleImporter.loadGroups(ctx, []string{path1, path2}) + for _, err := range errs { + require.NoError(t, err) + } + require.Equal(t, 3, len(ruleImporter.groups)) + group1 := ruleImporter.groups[path1+";group0"] + require.NotNil(t, group1) + const defaultInterval = 60 + require.Equal(t, time.Duration(defaultInterval*time.Second), group1.Interval()) + gRules := group1.Rules() + require.Equal(t, 1, len(gRules)) + require.Equal(t, "rule1", gRules[0].Name()) + require.Equal(t, "ruleExpr", gRules[0].Query().String()) + require.Equal(t, 1, len(gRules[0].Labels())) + + group2 := ruleImporter.groups[path2+";group2"] + require.NotNil(t, group2) + require.Equal(t, time.Duration(defaultInterval*time.Second), group2.Interval()) + g2Rules := group2.Rules() + require.Equal(t, 2, len(g2Rules)) + require.Equal(t, "grp2_rule1", g2Rules[0].Name()) + require.Equal(t, "grp2_rule1_expr", g2Rules[0].Query().String()) + require.Equal(t, 0, len(g2Rules[0].Labels())) + + // Backfill all recording rules then check the blocks to confirm the right data was created. + errs = ruleImporter.importAll(ctx) + for _, err := range errs { + require.NoError(t, err) + } + + opts := tsdb.DefaultOptions() + opts.AllowOverlappingBlocks = true + db, err := tsdb.Open(tmpDir, nil, nil, opts) + require.NoError(t, err) + + blocks := db.Blocks() + require.Equal(t, i+tt.expectedBlockCount, len(blocks)) + + q, err := db.Querier(context.Background(), math.MinInt64, math.MaxInt64) + require.NoError(t, err) + + selectedSeries := q.Select(false, nil, labels.MustNewMatcher(labels.MatchRegexp, "", ".*")) + var seriesCount, samplesCount int + for selectedSeries.Next() { + seriesCount++ + series := selectedSeries.At() + if len(series.Labels()) != 3 { + require.Equal(t, 2, len(series.Labels())) + x := labels.Labels{ + labels.Label{Name: "__name__", Value: "grp2_rule1"}, + labels.Label{Name: "name1", Value: "val1"}, + } + require.Equal(t, x, series.Labels()) + } else { + require.Equal(t, 3, len(series.Labels())) + } + it := series.Iterator() + for it.Next() { + samplesCount++ + ts, v := it.At() + require.Equal(t, float64(testValue), v) + require.Equal(t, int64(testTime), ts) + } + require.NoError(t, it.Err()) + } + require.NoError(t, selectedSeries.Err()) + require.Equal(t, tt.expectedSeriesCount, seriesCount) + require.Equal(t, tt.expectedSampleCount, samplesCount) + require.NoError(t, q.Close()) + require.NoError(t, db.Close()) + } + }) + } } -func newTestRuleImporter(ctx context.Context, start time.Time, tmpDir string) (*ruleImporter, error) { +func newTestRuleImporter(ctx context.Context, start time.Time, tmpDir string, testSamples model.Matrix) (*ruleImporter, error) { logger := log.NewNopLogger() cfg := ruleImporterConfig{ Start: start.Add(-1 * time.Hour), @@ -159,19 +172,39 @@ func newTestRuleImporter(ctx context.Context, start time.Time, tmpDir string) (* } app := newMultipleAppender(ctx, testMaxSampleCount, writer) - return newRuleImporter(logger, cfg, mockQueryRangeAPI{}, app), nil + return newRuleImporter(logger, cfg, mockQueryRangeAPI{ + samples: testSamples, + }, app), nil } -func createTestFiles(groupName, ruleName, ruleExpr, ruleLabels, path string) error { - recordingRules := fmt.Sprintf(`groups: -- name: %s +func createSingleRuleTestFiles(path string) error { + recordingRules := `groups: +- name: group0 rules: - - record: %s - expr: %s + - record: rule1 + expr: ruleExpr labels: - %s -`, - groupName, ruleName, ruleExpr, ruleLabels, - ) + testlabel11: testlabelvalue11 +` + return ioutil.WriteFile(path, []byte(recordingRules), 0777) +} + +func createMultiRuleTestFiles(path string) error { + recordingRules := `groups: +- name: group1 + rules: + - record: grp1_rule1 + expr: grp1_rule1_expr + labels: + testlabel11: testlabelvalue11 +- name: group2 + rules: + - record: grp2_rule1 + expr: grp2_rule1_expr + - record: grp2_rule2 + expr: grp2_rule2_expr + labels: + testlabel11: testlabelvalue11 +` return ioutil.WriteFile(path, []byte(recordingRules), 0777) } From 9fc53b7edfc4a041a3b90cf1f15e8ae8a87bbb62 Mon Sep 17 00:00:00 2001 From: jessicagreben Date: Mon, 1 Mar 2021 05:49:49 -0800 Subject: [PATCH 15/21] fix appender.Add -> appender.Append Signed-off-by: jessicagreben --- cmd/promtool/rules.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/cmd/promtool/rules.go b/cmd/promtool/rules.go index 02c72a1a0..473301921 100644 --- a/cmd/promtool/rules.go +++ b/cmd/promtool/rules.go @@ -171,7 +171,7 @@ func newMultipleAppender(ctx context.Context, maxSamplesInMemory int, blockWrite } func (m *multipleAppender) add(ctx context.Context, l labels.Labels, t int64, v float64) error { - if _, err := m.appender.Add(l, t, v); err != nil { + if _, err := m.appender.Append(0, l, t, v); err != nil { return err } m.currentSampleCount++ From 591e719dc1a411411a642d118da7344394873de6 Mon Sep 17 00:00:00 2001 From: jessicagreben Date: Mon, 1 Mar 2021 06:16:05 -0800 Subject: [PATCH 16/21] fix failing blockwriter tests Signed-off-by: jessicagreben --- tsdb/blockwriter_test.go | 4 ---- 1 file changed, 4 deletions(-) diff --git a/tsdb/blockwriter_test.go b/tsdb/blockwriter_test.go index 2be2d193c..9145a3128 100644 --- a/tsdb/blockwriter_test.go +++ b/tsdb/blockwriter_test.go @@ -36,10 +36,6 @@ func TestBlockWriter(t *testing.T) { w, err := NewBlockWriter(log.NewNopLogger(), outputDir, DefaultBlockDuration) require.NoError(t, err) - // Flush with no series results in error. - _, err = w.Flush(ctx) - require.EqualError(t, err, "no series appended, aborting") - // Add some series. app := w.Appender(ctx) ts1, v1 := int64(44), float64(7) From 7c26642460418654748c6bd33e1bbe8eab6e727c Mon Sep 17 00:00:00 2001 From: jessicagreben Date: Sun, 14 Mar 2021 10:10:55 -0700 Subject: [PATCH 17/21] add block alignment and write in 2 hr blocks Signed-off-by: jessicagreben --- .gitignore | 1 + cmd/promtool/main.go | 37 +++++---------- cmd/promtool/rules.go | 108 +++++++++++++++++++++++++----------------- 3 files changed, 77 insertions(+), 69 deletions(-) diff --git a/.gitignore b/.gitignore index c31d513e2..4e38927c5 100644 --- a/.gitignore +++ b/.gitignore @@ -11,6 +11,7 @@ benchmark.txt /cmd/prometheus/data /cmd/prometheus/debug /benchout +backfilldata !/.travis.yml !/.promu.yml diff --git a/cmd/promtool/main.go b/cmd/promtool/main.go index e48829393..1e201bbcf 100644 --- a/cmd/promtool/main.go +++ b/cmd/promtool/main.go @@ -46,7 +46,6 @@ import ( _ "github.com/prometheus/prometheus/discovery/install" // Register service discovery implementations. "github.com/prometheus/prometheus/discovery/kubernetes" "github.com/prometheus/prometheus/pkg/rulefmt" - "github.com/prometheus/prometheus/tsdb" ) func main() { @@ -151,10 +150,10 @@ func main() { importRulesCmd := importCmd.Command("rules", "Create new blocks of data from Prometheus data for new rules from recording rule files.") importRulesStart := importRulesCmd.Flag("start", "The time to start backfilling the new rule from. It is required. Start time should be RFC3339 or Unix timestamp."). Required().String() - importRulesEnd := importRulesCmd.Flag("end", "If an end time is provided, all recording rules in the rule files provided will be backfilled to the end time. Default will backfill up to 3 hrs ago. End time should be RFC3339 or Unix timestamp.").String() + importRulesEnd := importRulesCmd.Flag("end", "If an end time is provided, all recording rules in the rule files provided will be backfilled to the end time. Default will backfill up to 3 hours ago. End time should be RFC3339 or Unix timestamp.").String() importRulesOutputDir := importRulesCmd.Flag("output-dir", "The filepath on the local filesystem to write the output to. Output will be blocks containing the data of the backfilled recording rules. Don't use an active Prometheus data directory. If command is run many times with same start/end time, it will create duplicate series.").Default("backfilldata/").String() - importRulesURL := importRulesCmd.Flag("url", "The URL for the Prometheus API with the data where the rule will be backfilled from.").Default("http://localhost:9090").String() - importRulesEvalInterval := importRulesCmd.Flag("eval-interval-default", "How frequently to evaluate rules when backfilling if a value is not set in the recording rule files."). + importRulesURL := importRulesCmd.Flag("url", "The URL for the Prometheus API with the data where the rule will be backfilled from.").Default("http://localhost:9090").URL() + importRulesEvalInterval := importRulesCmd.Flag("eval-interval", "How frequently to evaluate rules when backfilling if a value is not set in the recording rule files."). Default("60s").Duration() importRulesFiles := importRulesCmd.Arg( "rule-files", @@ -838,7 +837,7 @@ func (j *jsonPrinter) printLabelValues(v model.LabelValues) { // importRules backfills recording rules from the files provided. The output are blocks of data // at the outputDir location. -func importRules(url, start, end, outputDir string, evalInterval time.Duration, files ...string) error { +func importRules(url *url.URL, start, end, outputDir string, evalInterval time.Duration, files ...string) error { ctx := context.Background() var stime, etime time.Time var err error @@ -863,33 +862,21 @@ func importRules(url, start, end, outputDir string, evalInterval time.Duration, return nil } - logger := log.NewLogfmtLogger(log.NewSyncWriter(os.Stderr)) - writer, err := tsdb.NewBlockWriter(logger, - outputDir, - tsdb.DefaultBlockDuration, - ) - if err != nil { - fmt.Fprintln(os.Stderr, "new writer error", err) - return err - } - defer func() { - err = writer.Close() - }() - cfg := ruleImporterConfig{ - Start: stime, - End: etime, - EvalInterval: evalInterval, + outputDir: outputDir, + start: stime, + end: etime, + evalInterval: evalInterval, } - c, err := api.NewClient(api.Config{ - Address: url, + client, err := api.NewClient(api.Config{ + Address: url.String(), }) if err != nil { fmt.Fprintln(os.Stderr, "new api client error", err) return err } - const maxSamplesInMemory = 5000 - ruleImporter := newRuleImporter(logger, cfg, v1.NewAPI(c), newMultipleAppender(ctx, maxSamplesInMemory, writer)) + + ruleImporter := newRuleImporter(log.NewLogfmtLogger(log.NewSyncWriter(os.Stderr)), cfg, v1.NewAPI(client)) errs := ruleImporter.loadGroups(ctx, files) for _, err := range errs { diff --git a/cmd/promtool/rules.go b/cmd/promtool/rules.go index 473301921..6d823794b 100644 --- a/cmd/promtool/rules.go +++ b/cmd/promtool/rules.go @@ -28,8 +28,11 @@ import ( "github.com/prometheus/prometheus/rules" "github.com/prometheus/prometheus/storage" "github.com/prometheus/prometheus/tsdb" + tsdb_errors "github.com/prometheus/prometheus/tsdb/errors" ) +const maxSamplesInMemory = 5000 + type queryRangeAPI interface { QueryRange(ctx context.Context, query string, r v1.Range) (model.Value, v1.Warnings, error) } @@ -40,33 +43,32 @@ type ruleImporter struct { apiClient queryRangeAPI - appender *multipleAppender - groups map[string]*rules.Group ruleManager *rules.Manager } type ruleImporterConfig struct { - Start time.Time - End time.Time - EvalInterval time.Duration + outputDir string + start time.Time + end time.Time + evalInterval time.Duration } // newRuleImporter creates a new rule importer that can be used to parse and evaluate recording rule files and create new series // written to disk in blocks. -func newRuleImporter(logger log.Logger, config ruleImporterConfig, apiClient queryRangeAPI, appender *multipleAppender) *ruleImporter { +func newRuleImporter(logger log.Logger, config ruleImporterConfig, apiClient queryRangeAPI) *ruleImporter { + return &ruleImporter{ logger: logger, config: config, apiClient: apiClient, - appender: appender, ruleManager: rules.NewManager(&rules.ManagerOptions{}), } } // loadGroups parses groups from a list of recording rule files. func (importer *ruleImporter) loadGroups(ctx context.Context, filenames []string) (errs []error) { - groups, errs := importer.ruleManager.LoadGroups(importer.config.EvalInterval, labels.Labels{}, filenames...) + groups, errs := importer.ruleManager.LoadGroups(importer.config.evalInterval, labels.Labels{}, filenames...) if errs != nil { return errs } @@ -79,46 +81,53 @@ func (importer *ruleImporter) importAll(ctx context.Context) (errs []error) { for name, group := range importer.groups { level.Info(importer.logger).Log("backfiller", fmt.Sprintf("processing group, name: %s", name)) - stimeWithAlignment := group.EvalTimestamp(importer.config.Start.UnixNano()) + stimeWithAlignment := group.EvalTimestamp(importer.config.start.UnixNano()) for i, r := range group.Rules() { level.Info(importer.logger).Log("backfiller", fmt.Sprintf("processing rule %d, name: %s", i+1, r.Name())) - if err := importer.importRule(ctx, r.Query().String(), r.Name(), r.Labels(), stimeWithAlignment, importer.config.End, group.Interval()); err != nil { + if err := importer.importRule(ctx, r.Query().String(), r.Name(), r.Labels(), stimeWithAlignment, importer.config.end, group.Interval()); err != nil { errs = append(errs, err) } } } - if err := importer.appender.flushAndCommit(ctx); err != nil { - errs = append(errs, err) - } return errs } // importRule queries a prometheus API to evaluate rules at times in the past. -func (importer *ruleImporter) importRule(ctx context.Context, ruleExpr, ruleName string, ruleLabels labels.Labels, start, end time.Time, interval time.Duration) error { - - // This loop breaks up the calls to the QueryRange API into 2 hr chunks so that we - // don't ever request too much data or take to long to process to avoid timeout. - for start.Before(end) { - currentBlockEnd := start.Add(time.Duration(tsdb.DefaultBlockDuration) * time.Millisecond) - if currentBlockEnd.After(end) { - currentBlockEnd = end - } +func (importer *ruleImporter) importRule(ctx context.Context, ruleExpr, ruleName string, ruleLabels labels.Labels, start, end time.Time, interval time.Duration) (err error) { + blockDuration := tsdb.DefaultBlockDuration + startOfBlock := blockDuration * (start.Unix() / blockDuration) + for t := startOfBlock; t <= end.Unix(); t = t + blockDuration { + endOfBlock := t + blockDuration val, warnings, err := importer.apiClient.QueryRange(ctx, ruleExpr, v1.Range{ - Start: start, - End: end, + Start: time.Unix(t, 0), + End: time.Unix(endOfBlock, 0), Step: interval, }, ) if err != nil { - return err + return errors.Wrap(err, "query range") } if warnings != nil { level.Warn(importer.logger).Log("backfiller", fmt.Sprintf("warnings QueryRange api: %v", warnings)) } + // To prevent races with compaction, a block writer only allows appending samples + // that are at most half a block size older than the most recent sample appended so far. + // However, in the way we use the block writer here, compaction doesn't happen, while we + // also need to append samples throughout the whole block range. To allow that, we + // pretend that the block is twice as large here, but only really add sample in the + // original interval later. + w, err := tsdb.NewBlockWriter(log.NewNopLogger(), importer.config.outputDir, 2*tsdb.DefaultBlockDuration) + if err != nil { + return errors.Wrap(err, "new block writer") + } + defer func() { + err = tsdb_errors.NewMulti(err, w.Close()).Err() + }() + app := newMultipleAppender(ctx, w) var matrix model.Matrix switch val.Type() { case model.ValMatrix: @@ -139,17 +148,29 @@ func (importer *ruleImporter) importRule(ctx context.Context, ruleExpr, ruleName }) } for _, value := range sample.Values { - if err := importer.appender.add(ctx, currentLabels, timestamp.FromTime(value.Timestamp.Time()), float64(value.Value)); err != nil { - return err + if err := app.add(ctx, currentLabels, timestamp.FromTime(value.Timestamp.Time()), float64(value.Value)); err != nil { + return errors.Wrap(err, "add") } } } default: return errors.New(fmt.Sprintf("rule result is wrong type %s", val.Type().String())) } - start = currentBlockEnd + + if err := app.flushAndCommit(ctx); err != nil { + return errors.Wrap(err, "flush and commit") + } + } + + return err +} + +func newMultipleAppender(ctx context.Context, blockWriter *tsdb.BlockWriter) *multipleAppender { + return &multipleAppender{ + maxSamplesInMemory: maxSamplesInMemory, + writer: blockWriter, + appender: blockWriter.Appender(ctx), } - return nil } // multipleAppender keeps track of how many series have been added to the current appender. @@ -162,36 +183,35 @@ type multipleAppender struct { appender storage.Appender } -func newMultipleAppender(ctx context.Context, maxSamplesInMemory int, blockWriter *tsdb.BlockWriter) *multipleAppender { - return &multipleAppender{ - maxSamplesInMemory: maxSamplesInMemory, - writer: blockWriter, - appender: blockWriter.Appender(ctx), - } -} - func (m *multipleAppender) add(ctx context.Context, l labels.Labels, t int64, v float64) error { if _, err := m.appender.Append(0, l, t, v); err != nil { - return err + return errors.Wrap(err, "multiappender append") } m.currentSampleCount++ if m.currentSampleCount >= m.maxSamplesInMemory { - return m.flushAndCommit(ctx) + return m.commit(ctx) } return nil } -func (m *multipleAppender) flushAndCommit(ctx context.Context) error { +func (m *multipleAppender) commit(ctx context.Context) error { if m.currentSampleCount == 0 { return nil } if err := m.appender.Commit(); err != nil { - return err - } - if _, err := m.writer.Flush(ctx); err != nil { - return err + return errors.Wrap(err, "multiappender commit") } m.appender = m.writer.Appender(ctx) m.currentSampleCount = 0 return nil } + +func (m *multipleAppender) flushAndCommit(ctx context.Context) error { + if err := m.commit(ctx); err != nil { + return err + } + if _, err := m.writer.Flush(ctx); err != nil { + return errors.Wrap(err, "multiappender flush") + } + return nil +} From e3a8132bb3ddcf5e3f7f8cb6197094c3130cf59f Mon Sep 17 00:00:00 2001 From: jessicagreben Date: Mon, 15 Mar 2021 12:44:58 -0700 Subject: [PATCH 18/21] fix block alignment, add sample alignment Signed-off-by: jessicagreben --- cmd/promtool/main.go | 1 - cmd/promtool/rules.go | 39 +++++++++++++++++++++++++++----------- cmd/promtool/rules_test.go | 27 +++++++++++++------------- 3 files changed, 41 insertions(+), 26 deletions(-) diff --git a/cmd/promtool/main.go b/cmd/promtool/main.go index 1e201bbcf..a0640c04d 100644 --- a/cmd/promtool/main.go +++ b/cmd/promtool/main.go @@ -877,7 +877,6 @@ func importRules(url *url.URL, start, end, outputDir string, evalInterval time.D } ruleImporter := newRuleImporter(log.NewLogfmtLogger(log.NewSyncWriter(os.Stderr)), cfg, v1.NewAPI(client)) - errs := ruleImporter.loadGroups(ctx, files) for _, err := range errs { if err != nil { diff --git a/cmd/promtool/rules.go b/cmd/promtool/rules.go index 6d823794b..5394c54b6 100644 --- a/cmd/promtool/rules.go +++ b/cmd/promtool/rules.go @@ -57,7 +57,6 @@ type ruleImporterConfig struct { // newRuleImporter creates a new rule importer that can be used to parse and evaluate recording rule files and create new series // written to disk in blocks. func newRuleImporter(logger log.Logger, config ruleImporterConfig, apiClient queryRangeAPI) *ruleImporter { - return &ruleImporter{ logger: logger, config: config, @@ -84,7 +83,7 @@ func (importer *ruleImporter) importAll(ctx context.Context) (errs []error) { stimeWithAlignment := group.EvalTimestamp(importer.config.start.UnixNano()) for i, r := range group.Rules() { level.Info(importer.logger).Log("backfiller", fmt.Sprintf("processing rule %d, name: %s", i+1, r.Name())) - if err := importer.importRule(ctx, r.Query().String(), r.Name(), r.Labels(), stimeWithAlignment, importer.config.end, group.Interval()); err != nil { + if err := importer.importRule(ctx, r.Query().String(), r.Name(), r.Labels(), stimeWithAlignment, importer.config.end, group); err != nil { errs = append(errs, err) } } @@ -93,18 +92,23 @@ func (importer *ruleImporter) importAll(ctx context.Context) (errs []error) { } // importRule queries a prometheus API to evaluate rules at times in the past. -func (importer *ruleImporter) importRule(ctx context.Context, ruleExpr, ruleName string, ruleLabels labels.Labels, start, end time.Time, interval time.Duration) (err error) { +func (importer *ruleImporter) importRule(ctx context.Context, ruleExpr, ruleName string, ruleLabels labels.Labels, start, end time.Time, grp *rules.Group) (err error) { blockDuration := tsdb.DefaultBlockDuration - startOfBlock := blockDuration * (start.Unix() / blockDuration) - for t := startOfBlock; t <= end.Unix(); t = t + blockDuration { - endOfBlock := t + blockDuration + startInMs := start.Unix() * int64(time.Second/time.Millisecond) + startOfBlock := blockDuration * (startInMs / blockDuration) + endInMs := end.Unix() * int64(time.Second/time.Millisecond) + for s := startOfBlock; s <= endInMs; s = s + blockDuration { + endOfBlock := s + blockDuration - 1 + + currStart := max(s/int64(time.Second/time.Millisecond), start.Unix()) + startWithAlignment := grp.EvalTimestamp(time.Unix(currStart, 0).UnixNano()) val, warnings, err := importer.apiClient.QueryRange(ctx, ruleExpr, v1.Range{ - Start: time.Unix(t, 0), - End: time.Unix(endOfBlock, 0), - Step: interval, + Start: startWithAlignment, + End: time.Unix(min(endOfBlock/int64(time.Second/time.Millisecond), end.Unix()), 0), + Step: grp.Interval(), }, ) if err != nil { @@ -174,8 +178,7 @@ func newMultipleAppender(ctx context.Context, blockWriter *tsdb.BlockWriter) *mu } // multipleAppender keeps track of how many series have been added to the current appender. -// If the max samples have been added, then all series are flushed to disk and commited and a new -// appender is created. +// If the max samples have been added, then all series are commited and a new appender is created. type multipleAppender struct { maxSamplesInMemory int currentSampleCount int @@ -215,3 +218,17 @@ func (m *multipleAppender) flushAndCommit(ctx context.Context) error { } return nil } + +func max(x, y int64) int64 { + if x > y { + return x + } + return y +} + +func min(x, y int64) int64 { + if x < y { + return x + } + return y +} diff --git a/cmd/promtool/rules_test.go b/cmd/promtool/rules_test.go index 8953841fb..a4c16102c 100644 --- a/cmd/promtool/rules_test.go +++ b/cmd/promtool/rules_test.go @@ -46,6 +46,12 @@ func (mockAPI mockQueryRangeAPI) QueryRange(ctx context.Context, query string, r return mockAPI.samples, v1.Warnings{}, nil } +func getTestProdData() []*model.SampleStream { + var result = []*model.SampleStream{} + + return result +} + // TestBackfillRuleIntegration is an integration test that runs all the rule importer code to confirm the parts work together. func TestBackfillRuleIntegration(t *testing.T) { var testCases = []struct { @@ -105,7 +111,7 @@ func TestBackfillRuleIntegration(t *testing.T) { require.Equal(t, "grp2_rule1_expr", g2Rules[0].Query().String()) require.Equal(t, 0, len(g2Rules[0].Labels())) - // Backfill all recording rules then check the blocks to confirm the right data was created. + // Backfill all recording rules then check the blocks to confirm the correct data was created. errs = ruleImporter.importAll(ctx) for _, err := range errs { require.NoError(t, err) @@ -159,22 +165,15 @@ func TestBackfillRuleIntegration(t *testing.T) { func newTestRuleImporter(ctx context.Context, start time.Time, tmpDir string, testSamples model.Matrix) (*ruleImporter, error) { logger := log.NewNopLogger() cfg := ruleImporterConfig{ - Start: start.Add(-1 * time.Hour), - End: start, - EvalInterval: 60 * time.Second, - } - writer, err := tsdb.NewBlockWriter(logger, - tmpDir, - tsdb.DefaultBlockDuration, - ) - if err != nil { - return nil, err + outputDir: tmpDir, + start: start.Add(-1 * time.Hour), + end: start, + evalInterval: 60 * time.Second, } - app := newMultipleAppender(ctx, testMaxSampleCount, writer) return newRuleImporter(logger, cfg, mockQueryRangeAPI{ samples: testSamples, - }, app), nil + }), nil } func createSingleRuleTestFiles(path string) error { @@ -182,7 +181,7 @@ func createSingleRuleTestFiles(path string) error { - name: group0 rules: - record: rule1 - expr: ruleExpr + expr: ruleExpr labels: testlabel11: testlabelvalue11 ` From 8de4da37160c3be773fcb2486dad358cf8c71a84 Mon Sep 17 00:00:00 2001 From: jessicagreben Date: Sat, 20 Mar 2021 12:38:30 -0700 Subject: [PATCH 19/21] add changes per comments, fix tests Signed-off-by: jessicagreben --- .gitignore | 2 +- cmd/promtool/main.go | 10 ++++----- cmd/promtool/rules.go | 18 +++++++-------- cmd/promtool/rules_test.go | 45 +++++++++++++++++++------------------- tsdb/blockwriter.go | 2 ++ 5 files changed, 39 insertions(+), 38 deletions(-) diff --git a/.gitignore b/.gitignore index 4e38927c5..de176eaab 100644 --- a/.gitignore +++ b/.gitignore @@ -11,7 +11,7 @@ benchmark.txt /cmd/prometheus/data /cmd/prometheus/debug /benchout -backfilldata +/cmd/promtool/data !/.travis.yml !/.promu.yml diff --git a/cmd/promtool/main.go b/cmd/promtool/main.go index a0640c04d..830282a48 100644 --- a/cmd/promtool/main.go +++ b/cmd/promtool/main.go @@ -148,11 +148,11 @@ func main() { importFilePath := openMetricsImportCmd.Arg("input file", "OpenMetrics file to read samples from.").Required().String() importDBPath := openMetricsImportCmd.Arg("output directory", "Output directory for generated blocks.").Default(defaultDBPath).String() importRulesCmd := importCmd.Command("rules", "Create new blocks of data from Prometheus data for new rules from recording rule files.") - importRulesStart := importRulesCmd.Flag("start", "The time to start backfilling the new rule from. It is required. Start time should be RFC3339 or Unix timestamp."). - Required().String() - importRulesEnd := importRulesCmd.Flag("end", "If an end time is provided, all recording rules in the rule files provided will be backfilled to the end time. Default will backfill up to 3 hours ago. End time should be RFC3339 or Unix timestamp.").String() - importRulesOutputDir := importRulesCmd.Flag("output-dir", "The filepath on the local filesystem to write the output to. Output will be blocks containing the data of the backfilled recording rules. Don't use an active Prometheus data directory. If command is run many times with same start/end time, it will create duplicate series.").Default("backfilldata/").String() importRulesURL := importRulesCmd.Flag("url", "The URL for the Prometheus API with the data where the rule will be backfilled from.").Default("http://localhost:9090").URL() + importRulesStart := importRulesCmd.Flag("start", "The time to start backfilling the new rule from. Must be a RFC3339 formated date or Unix timestamp. Required."). + Required().String() + importRulesEnd := importRulesCmd.Flag("end", "If an end time is provided, all recording rules in the rule files provided will be backfilled to the end time. Default will backfill up to 3 hours ago. Must be a RFC3339 formated date or Unix timestamp.").String() + importRulesOutputDir := importRulesCmd.Flag("output-dir", "Output directory for generated blocks.").Default("data/").String() importRulesEvalInterval := importRulesCmd.Flag("eval-interval", "How frequently to evaluate rules when backfilling if a value is not set in the recording rule files."). Default("60s").Duration() importRulesFiles := importRulesCmd.Arg( @@ -842,7 +842,7 @@ func importRules(url *url.URL, start, end, outputDir string, evalInterval time.D var stime, etime time.Time var err error if end == "" { - etime = time.Now().Add(-3 * time.Hour) + etime = time.Now().UTC().Add(-3 * time.Hour) } else { etime, err = parseTime(end) if err != nil { diff --git a/cmd/promtool/rules.go b/cmd/promtool/rules.go index 5394c54b6..261ab1a77 100644 --- a/cmd/promtool/rules.go +++ b/cmd/promtool/rules.go @@ -78,11 +78,11 @@ func (importer *ruleImporter) loadGroups(ctx context.Context, filenames []string // importAll evaluates all the recording rules and creates new time series and writes them to disk in blocks. func (importer *ruleImporter) importAll(ctx context.Context) (errs []error) { for name, group := range importer.groups { - level.Info(importer.logger).Log("backfiller", fmt.Sprintf("processing group, name: %s", name)) + level.Info(importer.logger).Log("backfiller", "processing group", "name", name) stimeWithAlignment := group.EvalTimestamp(importer.config.start.UnixNano()) for i, r := range group.Rules() { - level.Info(importer.logger).Log("backfiller", fmt.Sprintf("processing rule %d, name: %s", i+1, r.Name())) + level.Info(importer.logger).Log("backfiller", "processing rule", "id", i, "name", r.Name()) if err := importer.importRule(ctx, r.Query().String(), r.Name(), r.Labels(), stimeWithAlignment, importer.config.end, group); err != nil { errs = append(errs, err) } @@ -95,19 +95,18 @@ func (importer *ruleImporter) importAll(ctx context.Context) (errs []error) { func (importer *ruleImporter) importRule(ctx context.Context, ruleExpr, ruleName string, ruleLabels labels.Labels, start, end time.Time, grp *rules.Group) (err error) { blockDuration := tsdb.DefaultBlockDuration startInMs := start.Unix() * int64(time.Second/time.Millisecond) - startOfBlock := blockDuration * (startInMs / blockDuration) endInMs := end.Unix() * int64(time.Second/time.Millisecond) - for s := startOfBlock; s <= endInMs; s = s + blockDuration { - endOfBlock := s + blockDuration - 1 + for startOfBlock := blockDuration * (startInMs / blockDuration); startOfBlock <= endInMs; startOfBlock = startOfBlock + blockDuration { + endOfBlock := startOfBlock + blockDuration - 1 - currStart := max(s/int64(time.Second/time.Millisecond), start.Unix()) - startWithAlignment := grp.EvalTimestamp(time.Unix(currStart, 0).UnixNano()) + currStart := max(startOfBlock/int64(time.Second/time.Millisecond), start.Unix()) + startWithAlignment := grp.EvalTimestamp(time.Unix(currStart, 0).UTC().UnixNano()) val, warnings, err := importer.apiClient.QueryRange(ctx, ruleExpr, v1.Range{ Start: startWithAlignment, - End: time.Unix(min(endOfBlock/int64(time.Second/time.Millisecond), end.Unix()), 0), + End: time.Unix(min(endOfBlock/int64(time.Second/time.Millisecond), end.Unix()), 0).UTC(), Step: grp.Interval(), }, ) @@ -115,7 +114,7 @@ func (importer *ruleImporter) importRule(ctx context.Context, ruleExpr, ruleName return errors.Wrap(err, "query range") } if warnings != nil { - level.Warn(importer.logger).Log("backfiller", fmt.Sprintf("warnings QueryRange api: %v", warnings)) + level.Warn(importer.logger).Log("msg", "Range query returned warnings.", "warnings", warnings) } // To prevent races with compaction, a block writer only allows appending samples @@ -136,6 +135,7 @@ func (importer *ruleImporter) importRule(ctx context.Context, ruleExpr, ruleName switch val.Type() { case model.ValMatrix: matrix = val.(model.Matrix) + for _, sample := range matrix { currentLabels := make(labels.Labels, 0, len(sample.Metric)+len(ruleLabels)+1) currentLabels = append(currentLabels, labels.Label{ diff --git a/cmd/promtool/rules_test.go b/cmd/promtool/rules_test.go index a4c16102c..3459e8942 100644 --- a/cmd/promtool/rules_test.go +++ b/cmd/promtool/rules_test.go @@ -30,14 +30,6 @@ import ( "github.com/stretchr/testify/require" ) -const ( - testMaxSampleCount = 50 - testValue = 123 -) - -var testTime = model.Time(time.Now().Add(-20 * time.Minute).Unix()) -var testTime2 = model.Time(time.Now().Add(-30 * time.Minute).Unix()) - type mockQueryRangeAPI struct { samples model.Matrix } @@ -46,14 +38,19 @@ func (mockAPI mockQueryRangeAPI) QueryRange(ctx context.Context, query string, r return mockAPI.samples, v1.Warnings{}, nil } -func getTestProdData() []*model.SampleStream { - var result = []*model.SampleStream{} - - return result -} - // TestBackfillRuleIntegration is an integration test that runs all the rule importer code to confirm the parts work together. func TestBackfillRuleIntegration(t *testing.T) { + const ( + testMaxSampleCount = 50 + testValue = 123 + testValue2 = 98 + ) + var ( + start = time.Date(2009, time.November, 10, 6, 34, 0, 0, time.UTC) + testTime = model.Time(start.Add(-9 * time.Hour).Unix()) + testTime2 = model.Time(start.Add(-8 * time.Hour).Unix()) + ) + var testCases = []struct { name string runcount int @@ -63,8 +60,8 @@ func TestBackfillRuleIntegration(t *testing.T) { samples []*model.SampleStream }{ {"no samples", 1, 0, 0, 0, []*model.SampleStream{}}, - {"run importer once", 1, 1, 4, 4, []*model.SampleStream{{Metric: model.Metric{"name1": "val1"}, Values: []model.SamplePair{{Timestamp: testTime, Value: testValue}}}}}, - {"one importer twice", 2, 1, 4, 4, []*model.SampleStream{{Metric: model.Metric{"name1": "val1"}, Values: []model.SamplePair{{Timestamp: testTime, Value: testValue}, {Timestamp: testTime2, Value: testValue}}}}}, + {"run importer once", 1, 8, 4, 4, []*model.SampleStream{{Metric: model.Metric{"name1": "val1"}, Values: []model.SamplePair{{Timestamp: testTime, Value: testValue}}}}}, + {"one importer twice", 2, 8, 4, 8, []*model.SampleStream{{Metric: model.Metric{"name1": "val1"}, Values: []model.SamplePair{{Timestamp: testTime, Value: testValue}, {Timestamp: testTime2, Value: testValue2}}}}}, } for _, tt := range testCases { t.Run(tt.name, func(t *testing.T) { @@ -73,11 +70,10 @@ func TestBackfillRuleIntegration(t *testing.T) { defer func() { require.NoError(t, os.RemoveAll(tmpDir)) }() - start := time.Now().UTC() ctx := context.Background() // Execute the test more than once to simulate running the rule importer twice with the same data. - // We expect that duplicate blocks with the same series are created when run more than once. + // We expect duplicate blocks with the same series are created when run more than once. for i := 0; i < tt.runcount; i++ { ruleImporter, err := newTestRuleImporter(ctx, start, tmpDir, tt.samples) require.NoError(t, err) @@ -123,7 +119,7 @@ func TestBackfillRuleIntegration(t *testing.T) { require.NoError(t, err) blocks := db.Blocks() - require.Equal(t, i+tt.expectedBlockCount, len(blocks)) + require.Equal(t, (i+1)*tt.expectedBlockCount, len(blocks)) q, err := db.Querier(context.Background(), math.MinInt64, math.MaxInt64) require.NoError(t, err) @@ -147,8 +143,11 @@ func TestBackfillRuleIntegration(t *testing.T) { for it.Next() { samplesCount++ ts, v := it.At() - require.Equal(t, float64(testValue), v) - require.Equal(t, int64(testTime), ts) + if v == testValue { + require.Equal(t, int64(testTime), ts) + } else { + require.Equal(t, int64(testTime2), ts) + } } require.NoError(t, it.Err()) } @@ -166,8 +165,8 @@ func newTestRuleImporter(ctx context.Context, start time.Time, tmpDir string, te logger := log.NewNopLogger() cfg := ruleImporterConfig{ outputDir: tmpDir, - start: start.Add(-1 * time.Hour), - end: start, + start: start.Add(-10 * time.Hour), + end: start.Add(-7 * time.Hour), evalInterval: 60 * time.Second, } diff --git a/tsdb/blockwriter.go b/tsdb/blockwriter.go index d440ea9e9..2d19c7705 100644 --- a/tsdb/blockwriter.go +++ b/tsdb/blockwriter.go @@ -24,6 +24,7 @@ import ( "github.com/oklog/ulid" "github.com/pkg/errors" + "github.com/prometheus/prometheus/pkg/timestamp" "github.com/prometheus/prometheus/storage" "github.com/prometheus/prometheus/tsdb/chunkenc" ) @@ -93,6 +94,7 @@ func (w *BlockWriter) Flush(ctx context.Context) (ulid.ULID, error) { // Add +1 millisecond to block maxt because block intervals are half-open: [b.MinTime, b.MaxTime). // Because of this block intervals are always +1 than the total samples it includes. maxt := w.head.MaxTime() + 1 + level.Info(w.logger).Log("msg", "flushing", "series_count", w.head.NumSeries(), "mint", timestamp.Time(mint), "maxt", timestamp.Time(maxt)) compactor, err := NewLeveledCompactor(ctx, nil, From d89a1d999f68bb75845e670570b23183944888dc Mon Sep 17 00:00:00 2001 From: jessicagreben Date: Sun, 28 Mar 2021 12:13:58 -0700 Subject: [PATCH 20/21] add log with start/end times, close blocks before end of func Signed-off-by: jessicagreben --- cmd/promtool/rules.go | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/cmd/promtool/rules.go b/cmd/promtool/rules.go index 261ab1a77..90441a423 100644 --- a/cmd/promtool/rules.go +++ b/cmd/promtool/rules.go @@ -57,6 +57,7 @@ type ruleImporterConfig struct { // newRuleImporter creates a new rule importer that can be used to parse and evaluate recording rule files and create new series // written to disk in blocks. func newRuleImporter(logger log.Logger, config ruleImporterConfig, apiClient queryRangeAPI) *ruleImporter { + level.Info(logger).Log("backfiller", "new rule importer from start", config.start.Format(time.RFC822), " to end", config.end.Format(time.RFC822)) return &ruleImporter{ logger: logger, config: config, @@ -127,8 +128,11 @@ func (importer *ruleImporter) importRule(ctx context.Context, ruleExpr, ruleName if err != nil { return errors.Wrap(err, "new block writer") } + var closed bool defer func() { - err = tsdb_errors.NewMulti(err, w.Close()).Err() + if !closed { + err = tsdb_errors.NewMulti(err, w.Close()).Err() + } }() app := newMultipleAppender(ctx, w) var matrix model.Matrix @@ -160,6 +164,8 @@ func (importer *ruleImporter) importRule(ctx context.Context, ruleExpr, ruleName default: return errors.New(fmt.Sprintf("rule result is wrong type %s", val.Type().String())) } + err = tsdb_errors.NewMulti(err, w.Close()).Err() + closed = true if err := app.flushAndCommit(ctx); err != nil { return errors.Wrap(err, "flush and commit") From 896c828bb5327555a17d9053b5f5d32b96e40863 Mon Sep 17 00:00:00 2001 From: jessicagreben Date: Mon, 29 Mar 2021 06:45:12 -0700 Subject: [PATCH 21/21] close writer after flush Signed-off-by: jessicagreben --- cmd/promtool/rules.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/cmd/promtool/rules.go b/cmd/promtool/rules.go index 90441a423..e73b48241 100644 --- a/cmd/promtool/rules.go +++ b/cmd/promtool/rules.go @@ -164,12 +164,12 @@ func (importer *ruleImporter) importRule(ctx context.Context, ruleExpr, ruleName default: return errors.New(fmt.Sprintf("rule result is wrong type %s", val.Type().String())) } - err = tsdb_errors.NewMulti(err, w.Close()).Err() - closed = true if err := app.flushAndCommit(ctx); err != nil { return errors.Wrap(err, "flush and commit") } + err = tsdb_errors.NewMulti(err, w.Close()).Err() + closed = true } return err