From e569538630d6feb99406c6069e4fade3dacfe5e6 Mon Sep 17 00:00:00 2001 From: Ganesh Vernekar Date: Wed, 21 Oct 2020 18:32:23 +0530 Subject: [PATCH 1/8] Upgrade Prometheus to latest master Signed-off-by: Ganesh Vernekar --- go.mod | 2 +- go.sum | 4 +- .../prometheus/pkg/labels/labels.go | 36 +++-- .../remote/{max_gauge.go => max_timestamp.go} | 12 +- .../storage/remote/queue_manager.go | 8 +- .../prometheus/storage/remote/write.go | 6 +- .../prometheus/prometheus/tsdb/README.md | 8 +- .../prometheus/prometheus/tsdb/block.go | 2 +- .../prometheus/tsdb/chunks/head_chunks.go | 36 +++++ .../prometheus/prometheus/tsdb/compact.go | 6 + .../prometheus/prometheus/tsdb/db.go | 126 ++++++++++-------- .../prometheus/tsdb/fileutil/mmap.go | 2 +- .../prometheus/prometheus/tsdb/head.go | 47 +++++-- vendor/modules.txt | 2 +- 14 files changed, 207 insertions(+), 90 deletions(-) rename vendor/github.com/prometheus/prometheus/storage/remote/{max_gauge.go => max_timestamp.go} (80%) diff --git a/go.mod b/go.mod index 372cb7eb565..695a26c1c92 100644 --- a/go.mod +++ b/go.mod @@ -47,7 +47,7 @@ require ( github.com/prometheus/client_golang v1.7.1 github.com/prometheus/client_model v0.2.0 github.com/prometheus/common v0.14.0 - github.com/prometheus/prometheus v1.8.2-0.20201014093524-73e2ce1bd643 + github.com/prometheus/prometheus v1.8.2-0.20201021125713-2624d827fadc github.com/segmentio/fasthash v0.0.0-20180216231524-a72b379d632e github.com/sony/gobreaker v0.4.1 github.com/spf13/afero v1.2.2 diff --git a/go.sum b/go.sum index 561b24fd6c0..c66df93bc88 100644 --- a/go.sum +++ b/go.sum @@ -1081,8 +1081,8 @@ github.com/prometheus/prometheus v1.8.2-0.20200805082714-e0cf219f0de2/go.mod h1: github.com/prometheus/prometheus v1.8.2-0.20200819132913-cb830b0a9c78 h1:tHIAD+hgCIb86T0/Du7vGyfHa6J1+XsImQoY8Ete+c8= github.com/prometheus/prometheus v1.8.2-0.20200819132913-cb830b0a9c78/go.mod h1:zfAqy/MwhMFajB9E2n12/9gG2fvofIE9uKDtlZCDxqs= github.com/prometheus/prometheus v1.8.2-0.20200923143134-7e2db3d092f3/go.mod h1:9VNWoDFHOMovlubld5uKKxfCDcPBj2GMOCjcUFXkYaM= -github.com/prometheus/prometheus v1.8.2-0.20201014093524-73e2ce1bd643 h1:BDAexvKlOVjE5A8MlqRxzwkEpPl1/v6ydU1/J7kJtZc= -github.com/prometheus/prometheus v1.8.2-0.20201014093524-73e2ce1bd643/go.mod h1:XYjkJiog7fyQu3puQNivZPI2pNq1C/775EIoHfDvuvY= +github.com/prometheus/prometheus v1.8.2-0.20201021125713-2624d827fadc h1:/sVmawICge4nkNlVwuYdmwkTXk/HJJb8V5ilZx5/MaY= +github.com/prometheus/prometheus v1.8.2-0.20201021125713-2624d827fadc/go.mod h1:dCFGRXrHSoQYAAWc8XbmOfsgYEWoY7/XlTecchlehao= github.com/rafaeljusto/redigomock v0.0.0-20190202135759-257e089e14a1 h1:+kGqA4dNN5hn7WwvKdzHl0rdN5AEkbNZd0VjRltAiZg= github.com/rafaeljusto/redigomock v0.0.0-20190202135759-257e089e14a1/go.mod h1:JaY6n2sDr+z2WTsXkOmNRUfDy6FN0L6Nk7x06ndm4tY= github.com/rcrowley/go-metrics v0.0.0-20181016184325-3113b8401b8a/go.mod h1:bCqnVzQkZxMG4s8nGwiZ5l3QUCyqpo9Y+/ZMZ9VjZe4= diff --git a/vendor/github.com/prometheus/prometheus/pkg/labels/labels.go b/vendor/github.com/prometheus/prometheus/pkg/labels/labels.go index b6b859fa7ff..5c11cc2eeef 100644 --- a/vendor/github.com/prometheus/prometheus/pkg/labels/labels.go +++ b/vendor/github.com/prometheus/prometheus/pkg/labels/labels.go @@ -19,7 +19,7 @@ import ( "sort" "strconv" - "github.com/cespare/xxhash" + "github.com/cespare/xxhash/v2" ) // Well-known label names used by Prometheus components. @@ -29,10 +29,11 @@ const ( BucketLabel = "le" InstanceName = "instance" - sep = '\xff' labelSep = '\xfe' ) +var seps = []byte{'\xff'} + // Label is a key/value pair of strings. type Label struct { Name, Value string @@ -70,10 +71,10 @@ func (ls Labels) Bytes(buf []byte) []byte { b.WriteByte(labelSep) for i, l := range ls { if i > 0 { - b.WriteByte(sep) + b.WriteByte(seps[0]) } b.WriteString(l.Name) - b.WriteByte(sep) + b.WriteByte(seps[0]) b.WriteString(l.Value) } return b.Bytes() @@ -134,13 +135,26 @@ func (ls Labels) MatchLabels(on bool, names ...string) Labels { // Hash returns a hash value for the label set. func (ls Labels) Hash() uint64 { + // Use xxhash.Sum64(b) for fast path as it's faster. b := make([]byte, 0, 1024) + for i, v := range ls { + if len(b)+len(v.Name)+len(v.Value)+2 >= cap(b) { + // If labels entry is 1KB+ do not allocate whole entry. + h := xxhash.New() + _, _ = h.Write(b) + for _, v := range ls[i:] { + _, _ = h.WriteString(v.Name) + _, _ = h.Write(seps) + _, _ = h.WriteString(v.Value) + _, _ = h.Write(seps) + } + return h.Sum64() + } - for _, v := range ls { b = append(b, v.Name...) - b = append(b, sep) + b = append(b, seps[0]) b = append(b, v.Value...) - b = append(b, sep) + b = append(b, seps[0]) } return xxhash.Sum64(b) } @@ -157,9 +171,9 @@ func (ls Labels) HashForLabels(b []byte, names ...string) (uint64, []byte) { i++ } else { b = append(b, ls[i].Name...) - b = append(b, sep) + b = append(b, seps[0]) b = append(b, ls[i].Value...) - b = append(b, sep) + b = append(b, seps[0]) i++ j++ } @@ -181,9 +195,9 @@ func (ls Labels) HashWithoutLabels(b []byte, names ...string) (uint64, []byte) { continue } b = append(b, ls[i].Name...) - b = append(b, sep) + b = append(b, seps[0]) b = append(b, ls[i].Value...) - b = append(b, sep) + b = append(b, seps[0]) } return xxhash.Sum64(b), b } diff --git a/vendor/github.com/prometheus/prometheus/storage/remote/max_gauge.go b/vendor/github.com/prometheus/prometheus/storage/remote/max_timestamp.go similarity index 80% rename from vendor/github.com/prometheus/prometheus/storage/remote/max_gauge.go rename to vendor/github.com/prometheus/prometheus/storage/remote/max_timestamp.go index a56c2047d56..3a0a6d6fd4b 100644 --- a/vendor/github.com/prometheus/prometheus/storage/remote/max_gauge.go +++ b/vendor/github.com/prometheus/prometheus/storage/remote/max_timestamp.go @@ -19,13 +19,13 @@ import ( "github.com/prometheus/client_golang/prometheus" ) -type maxGauge struct { +type maxTimestamp struct { mtx sync.Mutex value float64 prometheus.Gauge } -func (m *maxGauge) Set(value float64) { +func (m *maxTimestamp) Set(value float64) { m.mtx.Lock() defer m.mtx.Unlock() if value > m.value { @@ -34,8 +34,14 @@ func (m *maxGauge) Set(value float64) { } } -func (m *maxGauge) Get() float64 { +func (m *maxTimestamp) Get() float64 { m.mtx.Lock() defer m.mtx.Unlock() return m.value } + +func (m *maxTimestamp) Collect(c chan<- prometheus.Metric) { + if m.Get() > 0 { + m.Gauge.Collect(c) + } +} diff --git a/vendor/github.com/prometheus/prometheus/storage/remote/queue_manager.go b/vendor/github.com/prometheus/prometheus/storage/remote/queue_manager.go index 1a79d6c0ce9..47b5726bdc3 100644 --- a/vendor/github.com/prometheus/prometheus/storage/remote/queue_manager.go +++ b/vendor/github.com/prometheus/prometheus/storage/remote/queue_manager.go @@ -56,7 +56,7 @@ type queueManagerMetrics struct { droppedSamplesTotal prometheus.Counter enqueueRetriesTotal prometheus.Counter sentBatchDuration prometheus.Histogram - highestSentTimestamp *maxGauge + highestSentTimestamp *maxTimestamp pendingSamples prometheus.Gauge shardCapacity prometheus.Gauge numShards prometheus.Gauge @@ -118,7 +118,7 @@ func newQueueManagerMetrics(r prometheus.Registerer, rn, e string) *queueManager Buckets: append(prometheus.DefBuckets, 25, 60, 120, 300), ConstLabels: constLabels, }) - m.highestSentTimestamp = &maxGauge{ + m.highestSentTimestamp = &maxTimestamp{ Gauge: prometheus.NewGauge(prometheus.GaugeOpts{ Namespace: namespace, Subsystem: subsystem, @@ -262,7 +262,7 @@ type QueueManager struct { metrics *queueManagerMetrics interner *pool - highestRecvTimestamp *maxGauge + highestRecvTimestamp *maxTimestamp } // NewQueueManager builds a new QueueManager. @@ -279,7 +279,7 @@ func NewQueueManager( client WriteClient, flushDeadline time.Duration, interner *pool, - highestRecvTimestamp *maxGauge, + highestRecvTimestamp *maxTimestamp, ) *QueueManager { if logger == nil { logger = log.NewNopLogger() diff --git a/vendor/github.com/prometheus/prometheus/storage/remote/write.go b/vendor/github.com/prometheus/prometheus/storage/remote/write.go index 8b0bf7622f3..9e93e0c18f0 100644 --- a/vendor/github.com/prometheus/prometheus/storage/remote/write.go +++ b/vendor/github.com/prometheus/prometheus/storage/remote/write.go @@ -53,7 +53,7 @@ type WriteStorage struct { interner *pool // For timestampTracker. - highestTimestamp *maxGauge + highestTimestamp *maxTimestamp } // NewWriteStorage creates and runs a WriteStorage. @@ -71,7 +71,7 @@ func NewWriteStorage(logger log.Logger, reg prometheus.Registerer, walDir string samplesIn: newEWMARate(ewmaWeight, shardUpdateDuration), walDir: walDir, interner: newPool(), - highestTimestamp: &maxGauge{ + highestTimestamp: &maxTimestamp{ Gauge: prometheus.NewGauge(prometheus.GaugeOpts{ Namespace: namespace, Subsystem: subsystem, @@ -202,7 +202,7 @@ type timestampTracker struct { writeStorage *WriteStorage samples int64 highestTimestamp int64 - highestRecvTimestamp *maxGauge + highestRecvTimestamp *maxTimestamp } // Add implements storage.Appender. diff --git a/vendor/github.com/prometheus/prometheus/tsdb/README.md b/vendor/github.com/prometheus/prometheus/tsdb/README.md index 61f86708820..248004b9d39 100644 --- a/vendor/github.com/prometheus/prometheus/tsdb/README.md +++ b/vendor/github.com/prometheus/prometheus/tsdb/README.md @@ -2,7 +2,7 @@ [![GoDoc](https://godoc.org/github.com/prometheus/prometheus/tsdb?status.svg)](https://godoc.org/github.com/prometheus/prometheus/tsdb) -This repository contains the Prometheus storage layer that is used in its 2.x releases. +This directory contains the Prometheus storage layer that is used in its 2.x releases. A writeup of its design can be found [here](https://fabxc.org/blog/2017-04-10-writing-a-tsdb/). @@ -11,3 +11,9 @@ Based on the Gorilla TSDB [white papers](http://www.vldb.org/pvldb/vol8/p1816-te Video: [Storing 16 Bytes at Scale](https://youtu.be/b_pEevMAC3I) from [PromCon 2017](https://promcon.io/2017-munich/). See also the [format documentation](docs/format/README.md). + +A series of blog posts explaining different components of TSDB: +* [The Head Block](https://ganeshvernekar.com/blog/prometheus-tsdb-the-head-block/) +* [WAL and Checkpoint](https://ganeshvernekar.com/blog/prometheus-tsdb-wal-and-checkpoint/) +* [Memory Mapping of Head Chunks from Disk](https://ganeshvernekar.com/blog/prometheus-tsdb-mmapping-head-chunks-from-disk/) +* [Persistent Block and its Index](https://ganeshvernekar.com/blog/prometheus-tsdb-persistent-block-and-its-index/) \ No newline at end of file diff --git a/vendor/github.com/prometheus/prometheus/tsdb/block.go b/vendor/github.com/prometheus/prometheus/tsdb/block.go index 0df30e846c3..c6de088d336 100644 --- a/vendor/github.com/prometheus/prometheus/tsdb/block.go +++ b/vendor/github.com/prometheus/prometheus/tsdb/block.go @@ -172,7 +172,7 @@ type BlockMetaCompaction struct { // ULIDs of all source head blocks that went into the block. Sources []ulid.ULID `json:"sources,omitempty"` // Indicates that during compaction it resulted in a block without any samples - // so it should be deleted on the next reload. + // so it should be deleted on the next reloadBlocks. Deletable bool `json:"deletable,omitempty"` // Short descriptions of the direct blocks that were used to create // this block. diff --git a/vendor/github.com/prometheus/prometheus/tsdb/chunks/head_chunks.go b/vendor/github.com/prometheus/prometheus/tsdb/chunks/head_chunks.go index 632682218cf..154149d0cc5 100644 --- a/vendor/github.com/prometheus/prometheus/tsdb/chunks/head_chunks.go +++ b/vendor/github.com/prometheus/prometheus/tsdb/chunks/head_chunks.go @@ -167,6 +167,11 @@ func (cdm *ChunkDiskMapper) openMMapFiles() (returnErr error) { return err } + files, err = repairLastChunkFile(files) + if err != nil { + return err + } + chkFileIndices := make([]int, 0, len(files)) for seq, fn := range files { f, err := fileutil.OpenMmapFile(fn) @@ -226,9 +231,40 @@ func listChunkFiles(dir string) (map[int]string, error) { } res[int(seq)] = filepath.Join(dir, fi.Name()) } + return res, nil } +// repairLastChunkFile deletes the last file if it's empty. +// Because we don't fsync when creating these file, we could end +// up with an empty file at the end during an abrupt shutdown. +func repairLastChunkFile(files map[int]string) (_ map[int]string, returnErr error) { + lastFile := -1 + for seq := range files { + if seq > lastFile { + lastFile = seq + } + } + + if lastFile <= 0 { + return files, nil + } + + info, err := os.Stat(files[lastFile]) + if err != nil { + return files, errors.Wrap(err, "file stat during last head chunk file repair") + } + if info.Size() == 0 { + // Corrupt file, hence remove it. + if err := os.RemoveAll(files[lastFile]); err != nil { + return files, errors.Wrap(err, "delete corrupted, empty head chunk file during last file repair") + } + delete(files, lastFile) + } + + return files, nil +} + // WriteChunk writes the chunk to the disk. // The returned chunk ref is the reference from where the chunk encoding starts for the chunk. func (cdm *ChunkDiskMapper) WriteChunk(seriesRef uint64, mint, maxt int64, chk chunkenc.Chunk) (chkRef uint64, err error) { diff --git a/vendor/github.com/prometheus/prometheus/tsdb/compact.go b/vendor/github.com/prometheus/prometheus/tsdb/compact.go index 74f54fdb9c8..fbc5b35833a 100644 --- a/vendor/github.com/prometheus/prometheus/tsdb/compact.go +++ b/vendor/github.com/prometheus/prometheus/tsdb/compact.go @@ -488,6 +488,12 @@ func (c *LeveledCompactor) Write(dest string, b BlockReader, mint, maxt int64, p } if meta.Stats.NumSamples == 0 { + level.Info(c.logger).Log( + "msg", "write block resulted in empty block", + "mint", meta.MinTime, + "maxt", meta.MaxTime, + "duration", time.Since(start), + ) return ulid.ULID{}, nil } diff --git a/vendor/github.com/prometheus/prometheus/tsdb/db.go b/vendor/github.com/prometheus/prometheus/tsdb/db.go index 4c70a405277..94cd401bb6b 100644 --- a/vendor/github.com/prometheus/prometheus/tsdb/db.go +++ b/vendor/github.com/prometheus/prometheus/tsdb/db.go @@ -216,7 +216,7 @@ func newDBMetrics(db *DB, r prometheus.Registerer) *dbMetrics { }) m.reloadsFailed = prometheus.NewCounter(prometheus.CounterOpts{ Name: "prometheus_tsdb_reloads_failures_total", - Help: "Number of times the database failed to reload block data from disk.", + Help: "Number of times the database failed to reloadBlocks block data from disk.", }) m.compactionsTriggered = prometheus.NewCounter(prometheus.CounterOpts{ Name: "prometheus_tsdb_compactions_triggered_total", @@ -769,20 +769,26 @@ func (a dbAppender) Commit() error { } // Compact data if possible. After successful compaction blocks are reloaded -// which will also trigger blocks to be deleted that fall out of the retention -// window. -// If no blocks are compacted, the retention window state doesn't change. Thus, -// this is sufficient to reliably delete old data. -// Old blocks are only deleted on reload based on the new block's parent information. -// See DB.reload documentation for further information. -func (db *DB) Compact() (err error) { +// which will also delete the blocks that fall out of the retention window. +// Old blocks are only deleted on reloadBlocks based on the new block's parent information. +// See DB.reloadBlocks documentation for further information. +func (db *DB) Compact() (returnErr error) { db.cmtx.Lock() defer db.cmtx.Unlock() defer func() { - if err != nil { + if returnErr != nil { db.metrics.compactionsFailed.Inc() } }() + + lastBlockMaxt := int64(math.MinInt64) + defer func() { + var merr tsdb_errors.MultiError + merr.Add(returnErr) + merr.Add(errors.Wrap(db.head.truncateWAL(lastBlockMaxt), "WAL truncation in Compact defer")) + returnErr = merr.Err() + }() + // Check whether we have pending head blocks that are ready to be persisted. // They have the highest priority. for { @@ -804,55 +810,59 @@ func (db *DB) Compact() (err error) { // so in order to make sure that overlaps are evaluated // consistently, we explicitly remove the last value // from the block interval here. - head := NewRangeHead(db.head, mint, maxt-1) - if err := db.compactHead(head); err != nil { - return err + if err := db.compactHead(NewRangeHead(db.head, mint, maxt-1)); err != nil { + return errors.Wrap(err, "compact head") } + // Consider only successful compactions for WAL truncation. + lastBlockMaxt = maxt + } + + // Clear some disk space before compacting blocks, especially important + // when Head compaction happened over a long time range. + if err := db.head.truncateWAL(lastBlockMaxt); err != nil { + return errors.Wrap(err, "WAL truncation in Compact") } return db.compactBlocks() } -// CompactHead compacts the given the RangeHead. -func (db *DB) CompactHead(head *RangeHead) (err error) { +// CompactHead compacts the given RangeHead. +func (db *DB) CompactHead(head *RangeHead) error { db.cmtx.Lock() defer db.cmtx.Unlock() - return db.compactHead(head) + if err := db.compactHead(head); err != nil { + return errors.Wrap(err, "compact head") + } + + if err := db.head.truncateWAL(head.BlockMaxTime()); err != nil { + return errors.Wrap(err, "WAL truncation") + } + return nil } -// compactHead compacts the given the RangeHead. +// compactHead compacts the given RangeHead. // The compaction mutex should be held before calling this method. -func (db *DB) compactHead(head *RangeHead) (err 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 := head.MaxTime() + 1 - uid, err := db.compactor.Write(db.dir, head, head.MinTime(), maxt, nil) +func (db *DB) compactHead(head *RangeHead) error { + uid, err := db.compactor.Write(db.dir, head, head.MinTime(), head.BlockMaxTime(), nil) if err != nil { return errors.Wrap(err, "persist head block") } runtime.GC() - - if err := db.reload(); err != nil { + if err := db.reloadBlocks(); err != nil { if errRemoveAll := os.RemoveAll(filepath.Join(db.dir, uid.String())); errRemoveAll != nil { var merr tsdb_errors.MultiError - merr.Add(errors.Wrap(err, "reload blocks")) - merr.Add(errors.Wrapf(errRemoveAll, "delete persisted head block after failed db reload:%s", uid)) + merr.Add(errors.Wrap(err, "reloadBlocks blocks")) + merr.Add(errors.Wrapf(errRemoveAll, "delete persisted head block after failed db reloadBlocks:%s", uid)) return merr.Err() } - return errors.Wrap(err, "reload blocks") + return errors.Wrap(err, "reloadBlocks blocks") } - if (uid == ulid.ULID{}) { - // Compaction resulted in an empty block. - // Head truncating during db.reload() depends on the persisted blocks and - // in this case no new block will be persisted so manually truncate the head. - if err = db.head.Truncate(maxt); err != nil { - return errors.Wrap(err, "head truncate failed (in compact)") - } + if err = db.head.truncateMemory(head.BlockMaxTime()); err != nil { + return errors.Wrap(err, "head memory truncate") } runtime.GC() - return nil } @@ -881,11 +891,11 @@ func (db *DB) compactBlocks() (err error) { } runtime.GC() - if err := db.reload(); err != nil { + if err := db.reloadBlocks(); err != nil { if err := os.RemoveAll(filepath.Join(db.dir, uid.String())); err != nil { - return errors.Wrapf(err, "delete compacted block after failed db reload:%s", uid) + return errors.Wrapf(err, "delete compacted block after failed db reloadBlocks:%s", uid) } - return errors.Wrap(err, "reload blocks") + return errors.Wrap(err, "reloadBlocks blocks") } runtime.GC() } @@ -904,9 +914,23 @@ func getBlock(allBlocks []*Block, id ulid.ULID) (*Block, bool) { return nil, false } -// reload blocks and trigger head truncation if new blocks appeared. +// reload reloads blocks and truncates the head and its WAL. +func (db *DB) reload() error { + if err := db.reloadBlocks(); err != nil { + return errors.Wrap(err, "reloadBlocks") + } + if len(db.blocks) == 0 { + return nil + } + if err := db.head.Truncate(db.blocks[len(db.blocks)-1].MaxTime()); err != nil { + return errors.Wrap(err, "head truncate") + } + return nil +} + +// reloadBlocks reloads blocks without touching head. // Blocks that are obsolete due to replacement or retention will be deleted. -func (db *DB) reload() (err error) { +func (db *DB) reloadBlocks() (err error) { defer func() { if err != nil { db.metrics.reloadsFailed.Inc() @@ -989,7 +1013,7 @@ func (db *DB) reload() (err error) { blockMetas = append(blockMetas, b.Meta()) } if overlaps := OverlappingBlocks(blockMetas); len(overlaps) > 0 { - level.Warn(db.logger).Log("msg", "Overlapping blocks found during reload", "detail", overlaps.String()) + level.Warn(db.logger).Log("msg", "Overlapping blocks found during reloadBlocks", "detail", overlaps.String()) } // Append blocks to old, deletable blocks, so we can close them. @@ -999,15 +1023,9 @@ func (db *DB) reload() (err error) { } } if err := db.deleteBlocks(deletable); err != nil { - return err + return errors.Wrapf(err, "delete %v blocks", len(deletable)) } - - // Garbage collect data in the head if the most recent persisted block - // covers data of its current time range. - if len(toLoad) == 0 { - return nil - } - return errors.Wrap(db.head.Truncate(toLoad[len(toLoad)-1].Meta().MaxTime), "head truncate failed") + return nil } func openBlocks(l log.Logger, dir string, loaded []*Block, chunkPool chunkenc.Pool) (blocks []*Block, corrupted map[ulid.ULID]error, err error) { @@ -1020,7 +1038,7 @@ func openBlocks(l log.Logger, dir string, loaded []*Block, chunkPool chunkenc.Po for _, bDir := range bDirs { meta, _, err := readMetaFile(bDir) if err != nil { - level.Error(l).Log("msg", "Failed to read meta.json for a block during reload. Skipping", "dir", bDir, "err", err) + level.Error(l).Log("msg", "Failed to read meta.json for a block during reloadBlocks. Skipping", "dir", bDir, "err", err) continue } @@ -1077,7 +1095,7 @@ func deletableBlocks(db *DB, blocks []*Block) map[ulid.ULID]struct{} { // set in the db options. func BeyondTimeRetention(db *DB, blocks []*Block) (deletable map[ulid.ULID]struct{}) { // Time retention is disabled or no blocks to work with. - if len(db.blocks) == 0 || db.opts.RetentionDuration == 0 { + if len(blocks) == 0 || db.opts.RetentionDuration == 0 { return } @@ -1100,7 +1118,7 @@ func BeyondTimeRetention(db *DB, blocks []*Block) (deletable map[ulid.ULID]struc // set in the db options. func BeyondSizeRetention(db *DB, blocks []*Block) (deletable map[ulid.ULID]struct{}) { // Size retention is disabled or no blocks to work with. - if len(db.blocks) == 0 || db.opts.MaxBytes <= 0 { + if len(blocks) == 0 || db.opts.MaxBytes <= 0 { return } @@ -1502,7 +1520,11 @@ func (db *DB) CleanTombstones() (err error) { newUIDs = append(newUIDs, *uid) } } - return errors.Wrap(db.reload(), "reload blocks") + + if err := db.reloadBlocks(); err != nil { + return errors.Wrap(err, "reload blocks") + } + return nil } func isBlockDir(fi os.FileInfo) bool { diff --git a/vendor/github.com/prometheus/prometheus/tsdb/fileutil/mmap.go b/vendor/github.com/prometheus/prometheus/tsdb/fileutil/mmap.go index 5cb12381039..4dbca4f9740 100644 --- a/vendor/github.com/prometheus/prometheus/tsdb/fileutil/mmap.go +++ b/vendor/github.com/prometheus/prometheus/tsdb/fileutil/mmap.go @@ -48,7 +48,7 @@ func OpenMmapFileWithSize(path string, size int) (mf *MmapFile, retErr error) { b, err := mmap(f, size) if err != nil { - return nil, errors.Wrap(err, "mmap") + return nil, errors.Wrapf(err, "mmap, size %d", size) } return &MmapFile{f: f, b: b}, nil diff --git a/vendor/github.com/prometheus/prometheus/tsdb/head.go b/vendor/github.com/prometheus/prometheus/tsdb/head.go index 3a577763d64..56b597f3df7 100644 --- a/vendor/github.com/prometheus/prometheus/tsdb/head.go +++ b/vendor/github.com/prometheus/prometheus/tsdb/head.go @@ -52,11 +52,12 @@ var ( // Head handles reads and writes of time series data within a time window. type Head struct { - chunkRange atomic.Int64 - numSeries atomic.Uint64 - minTime, maxTime atomic.Int64 // Current min and max of the samples included in the head. - minValidTime atomic.Int64 // Mint allowed to be added to the head. It shouldn't be lower than the maxt of the last persisted block. - lastSeriesID atomic.Uint64 + chunkRange atomic.Int64 + numSeries atomic.Uint64 + minTime, maxTime atomic.Int64 // Current min and max of the samples included in the head. + minValidTime atomic.Int64 // Mint allowed to be added to the head. It shouldn't be lower than the maxt of the last persisted block. + lastWALTruncationTime atomic.Int64 + lastSeriesID atomic.Uint64 metrics *headMetrics wal *wal.WAL @@ -323,6 +324,7 @@ func NewHead(r prometheus.Registerer, l log.Logger, wal *wal.WAL, chunkRange int h.chunkRange.Store(chunkRange) h.minTime.Store(math.MaxInt64) h.maxTime.Store(math.MinInt64) + h.lastWALTruncationTime.Store(math.MinInt64) h.metrics = newHeadMetrics(h, r) if pool == nil { @@ -776,8 +778,20 @@ func (h *Head) removeCorruptedMmappedChunks(err error) map[uint64][]*mmappedChun return mmappedChunks } -// Truncate removes old data before mint from the head. +// Truncate removes old data before mint from the head and WAL. func (h *Head) Truncate(mint int64) (err error) { + initialize := h.MinTime() == math.MaxInt64 + if err := h.truncateMemory(mint); err != nil { + return err + } + if initialize { + return nil + } + return h.truncateWAL(mint) +} + +// truncateMemory removes old data before mint from the head. +func (h *Head) truncateMemory(mint int64) (err error) { defer func() { if err != nil { h.metrics.headTruncateFail.Inc() @@ -813,11 +827,16 @@ func (h *Head) Truncate(mint int64) (err error) { if err := h.chunkDiskMapper.Truncate(mint); err != nil { return errors.Wrap(err, "truncate chunks.HeadReadWriter") } + return nil +} - if h.wal == nil { +// truncateWAL removes old data before mint from the WAL. +func (h *Head) truncateWAL(mint int64) error { + if h.wal == nil || mint <= h.lastWALTruncationTime.Load() { return nil } - start = time.Now() + start := time.Now() + h.lastWALTruncationTime.Store(mint) first, last, err := wal.Segments(h.wal.Dir()) if err != nil { @@ -825,8 +844,7 @@ func (h *Head) Truncate(mint int64) (err error) { } // Start a new segment, so low ingestion volume TSDB don't have more WAL than // needed. - err = h.wal.NextSegment() - if err != nil { + if err := h.wal.NextSegment(); err != nil { return errors.Wrap(err, "next segment") } last-- // Never consider last segment for checkpoint. @@ -950,10 +968,19 @@ func (h *RangeHead) MinTime() int64 { return h.mint } +// MaxTime returns the max time of actual data fetch-able from the head. +// This controls the chunks time range which is closed [b.MinTime, b.MaxTime]. func (h *RangeHead) MaxTime() int64 { return h.maxt } +// BlockMaxTime returns the max time of the potential block created from this head. +// It's different to MaxTime as we need to add +1 millisecond to block maxt because block +// intervals are half-open: [b.MinTime, b.MaxTime). Block intervals are always +1 than the total samples it includes. +func (h *RangeHead) BlockMaxTime() int64 { + return h.MaxTime() + 1 +} + func (h *RangeHead) NumSeries() uint64 { return h.head.NumSeries() } diff --git a/vendor/modules.txt b/vendor/modules.txt index 2e858829e35..b8fce6ff60f 100644 --- a/vendor/modules.txt +++ b/vendor/modules.txt @@ -483,7 +483,7 @@ github.com/prometheus/node_exporter/https github.com/prometheus/procfs github.com/prometheus/procfs/internal/fs github.com/prometheus/procfs/internal/util -# github.com/prometheus/prometheus v1.8.2-0.20201014093524-73e2ce1bd643 +# github.com/prometheus/prometheus v1.8.2-0.20201021125713-2624d827fadc ## explicit github.com/prometheus/prometheus/config github.com/prometheus/prometheus/discovery From 37792232f98cd6eec9b843cd811339d41451159b Mon Sep 17 00:00:00 2001 From: Ganesh Vernekar Date: Wed, 21 Oct 2020 20:40:18 +0530 Subject: [PATCH 2/8] Add another commit Signed-off-by: Ganesh Vernekar --- go.mod | 2 +- go.sum | 4 +- .../prometheus/prometheus/tsdb/db.go | 39 +++++++++++++------ vendor/modules.txt | 2 +- 4 files changed, 31 insertions(+), 16 deletions(-) diff --git a/go.mod b/go.mod index 695a26c1c92..ec6275e07bb 100644 --- a/go.mod +++ b/go.mod @@ -47,7 +47,7 @@ require ( github.com/prometheus/client_golang v1.7.1 github.com/prometheus/client_model v0.2.0 github.com/prometheus/common v0.14.0 - github.com/prometheus/prometheus v1.8.2-0.20201021125713-2624d827fadc + github.com/prometheus/prometheus v1.8.2-0.20201021150828-c5a832b39440 github.com/segmentio/fasthash v0.0.0-20180216231524-a72b379d632e github.com/sony/gobreaker v0.4.1 github.com/spf13/afero v1.2.2 diff --git a/go.sum b/go.sum index c66df93bc88..b781e3118e7 100644 --- a/go.sum +++ b/go.sum @@ -1081,8 +1081,8 @@ github.com/prometheus/prometheus v1.8.2-0.20200805082714-e0cf219f0de2/go.mod h1: github.com/prometheus/prometheus v1.8.2-0.20200819132913-cb830b0a9c78 h1:tHIAD+hgCIb86T0/Du7vGyfHa6J1+XsImQoY8Ete+c8= github.com/prometheus/prometheus v1.8.2-0.20200819132913-cb830b0a9c78/go.mod h1:zfAqy/MwhMFajB9E2n12/9gG2fvofIE9uKDtlZCDxqs= github.com/prometheus/prometheus v1.8.2-0.20200923143134-7e2db3d092f3/go.mod h1:9VNWoDFHOMovlubld5uKKxfCDcPBj2GMOCjcUFXkYaM= -github.com/prometheus/prometheus v1.8.2-0.20201021125713-2624d827fadc h1:/sVmawICge4nkNlVwuYdmwkTXk/HJJb8V5ilZx5/MaY= -github.com/prometheus/prometheus v1.8.2-0.20201021125713-2624d827fadc/go.mod h1:dCFGRXrHSoQYAAWc8XbmOfsgYEWoY7/XlTecchlehao= +github.com/prometheus/prometheus v1.8.2-0.20201021150828-c5a832b39440 h1:JaGDp2WhzxWFQ2GTXiEtssazFug/5Pp3r9ZHR2gV0N0= +github.com/prometheus/prometheus v1.8.2-0.20201021150828-c5a832b39440/go.mod h1:dCFGRXrHSoQYAAWc8XbmOfsgYEWoY7/XlTecchlehao= github.com/rafaeljusto/redigomock v0.0.0-20190202135759-257e089e14a1 h1:+kGqA4dNN5hn7WwvKdzHl0rdN5AEkbNZd0VjRltAiZg= github.com/rafaeljusto/redigomock v0.0.0-20190202135759-257e089e14a1/go.mod h1:JaY6n2sDr+z2WTsXkOmNRUfDy6FN0L6Nk7x06ndm4tY= github.com/rcrowley/go-metrics v0.0.0-20181016184325-3113b8401b8a/go.mod h1:bCqnVzQkZxMG4s8nGwiZ5l3QUCyqpo9Y+/ZMZ9VjZe4= diff --git a/vendor/github.com/prometheus/prometheus/tsdb/db.go b/vendor/github.com/prometheus/prometheus/tsdb/db.go index 94cd401bb6b..5be60b670bf 100644 --- a/vendor/github.com/prometheus/prometheus/tsdb/db.go +++ b/vendor/github.com/prometheus/prometheus/tsdb/db.go @@ -199,7 +199,7 @@ func newDBMetrics(db *DB, r prometheus.Registerer) *dbMetrics { }) m.symbolTableSize = prometheus.NewGaugeFunc(prometheus.GaugeOpts{ Name: "prometheus_tsdb_symbol_table_size_bytes", - Help: "Size of symbol table on disk (in bytes)", + Help: "Size of symbol table in memory for loaded blocks", }, func() float64 { db.mtx.RLock() blocks := db.blocks[:] @@ -553,7 +553,7 @@ func validateOpts(opts *Options, rngs []int64) (*Options, []int64) { return opts, rngs } -func open(dir string, l log.Logger, r prometheus.Registerer, opts *Options, rngs []int64) (db *DB, err error) { +func open(dir string, l log.Logger, r prometheus.Registerer, opts *Options, rngs []int64) (db *DB, returnedErr error) { if err := os.MkdirAll(dir, 0777); err != nil { return nil, err } @@ -595,6 +595,19 @@ func open(dir string, l log.Logger, r prometheus.Registerer, opts *Options, rngs chunkPool: chunkenc.NewPool(), blocksToDelete: opts.BlocksToDelete, } + defer func() { + // Close files if startup fails somewhere. + if returnedErr == nil { + return + } + + var merr tsdb_errors.MultiError + merr.Add(returnedErr) + merr.Add(errors.Wrap(db.Close(), "close DB after failed startup")) + + returnedErr = merr.Err() + }() + if db.blocksToDelete == nil { db.blocksToDelete = DefaultBlocksToDelete(db) } @@ -612,10 +625,10 @@ func open(dir string, l log.Logger, r prometheus.Registerer, opts *Options, rngs } ctx, cancel := context.WithCancel(context.Background()) - db.compactor, err = NewLeveledCompactor(ctx, r, l, rngs, db.chunkPool) - if err != nil { + db.compactor, returnedErr = NewLeveledCompactor(ctx, r, l, rngs, db.chunkPool) + if returnedErr != nil { cancel() - return nil, errors.Wrap(err, "create leveled compactor") + return nil, errors.Wrap(returnedErr, "create leveled compactor") } db.compactCancel = cancel @@ -627,15 +640,15 @@ func open(dir string, l log.Logger, r prometheus.Registerer, opts *Options, rngs if opts.WALSegmentSize > 0 { segmentSize = opts.WALSegmentSize } - wlog, err = wal.NewSize(l, r, walDir, segmentSize, opts.WALCompression) - if err != nil { - return nil, err + wlog, returnedErr = wal.NewSize(l, r, walDir, segmentSize, opts.WALCompression) + if returnedErr != nil { + return nil, returnedErr } } - db.head, err = NewHead(r, l, wlog, rngs[0], dir, db.chunkPool, opts.StripeSize, opts.SeriesLifecycleCallback) - if err != nil { - return nil, err + db.head, returnedErr = NewHead(r, l, wlog, rngs[0], dir, db.chunkPool, opts.StripeSize, opts.SeriesLifecycleCallback) + if returnedErr != nil { + return nil, returnedErr } // Register metrics after assigning the head block. @@ -1333,7 +1346,9 @@ func (db *DB) Close() error { if db.lockf != nil { merr.Add(db.lockf.Release()) } - merr.Add(db.head.Close()) + if db.head != nil { + merr.Add(db.head.Close()) + } return merr.Err() } diff --git a/vendor/modules.txt b/vendor/modules.txt index b8fce6ff60f..56a96350284 100644 --- a/vendor/modules.txt +++ b/vendor/modules.txt @@ -483,7 +483,7 @@ github.com/prometheus/node_exporter/https github.com/prometheus/procfs github.com/prometheus/procfs/internal/fs github.com/prometheus/procfs/internal/util -# github.com/prometheus/prometheus v1.8.2-0.20201021125713-2624d827fadc +# github.com/prometheus/prometheus v1.8.2-0.20201021150828-c5a832b39440 ## explicit github.com/prometheus/prometheus/config github.com/prometheus/prometheus/discovery From c8729f85e5457f21dd3f3c099c611bc9241f2e49 Mon Sep 17 00:00:00 2001 From: Marco Pracucci Date: Thu, 22 Oct 2020 16:53:21 +0200 Subject: [PATCH 3/8] Fixed test Signed-off-by: Marco Pracucci --- pkg/ingester/ingester_v2_test.go | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/pkg/ingester/ingester_v2_test.go b/pkg/ingester/ingester_v2_test.go index f5255455a53..96f1da1af95 100644 --- a/pkg/ingester/ingester_v2_test.go +++ b/pkg/ingester/ingester_v2_test.go @@ -1667,11 +1667,12 @@ func TestIngester_v2OpenExistingTSDBOnStartup(t *testing.T) { "should fail and rollback if an error occur while loading a TSDB on concurrency > number of TSDBs": { concurrency: 10, setup: func(t *testing.T, dir string) { - // Create a fake TSDB on disk with an empty chunks head segment file (it's invalid and - // opening TSDB should fail). + // Create a fake TSDB on disk with an empty chunks head segment file (it's invalid unless + // it's the last one and opening TSDB should fail). require.NoError(t, os.MkdirAll(filepath.Join(dir, "user0", "wal", ""), 0700)) require.NoError(t, os.MkdirAll(filepath.Join(dir, "user0", "chunks_head", ""), 0700)) require.NoError(t, ioutil.WriteFile(filepath.Join(dir, "user0", "chunks_head", "00000001"), nil, 0700)) + require.NoError(t, ioutil.WriteFile(filepath.Join(dir, "user0", "chunks_head", "00000002"), nil, 0700)) require.NoError(t, os.MkdirAll(filepath.Join(dir, "user1", "dummy"), 0700)) }, From 5db7820598b7a97c226d27e228b4931a48aa8308 Mon Sep 17 00:00:00 2001 From: Ganesh Vernekar Date: Wed, 28 Oct 2020 15:44:05 +0530 Subject: [PATCH 4/8] Upgrade Prometheus again to fix the panic Signed-off-by: Ganesh Vernekar --- go.mod | 2 +- go.sum | 8 +- .../prometheus/discovery/discovery.go | 2 +- .../prometheus/discovery/registry.go | 3 +- .../prometheus/notifier/notifier.go | 4 +- .../prometheus/pkg/textparse/interface.go | 16 +-- .../promql/parser/generated_parser.y.go | 7 +- .../prometheus/promql/parser/printer.go | 1 + .../prometheus/prometheus/promql/test.go | 1 + .../prometheus/prometheus/promql/value.go | 3 +- .../prometheus/prometheus/rules/alerting.go | 6 +- .../prometheus/prometheus/scrape/manager.go | 2 +- .../prometheus/prometheus/scrape/scrape.go | 39 ++++-- .../prometheus/prometheus/storage/fanout.go | 1 + .../prometheus/prometheus/storage/merge.go | 1 + .../prometheus/storage/remote/client.go | 2 +- .../prometheus/storage/remote/codec.go | 1 + .../prometheus/storage/remote/intern.go | 3 +- .../storage/remote/queue_manager.go | 2 +- .../prometheus/storage/remote/read.go | 1 + .../prometheus/storage/remote/storage.go | 4 +- .../prometheus/storage/remote/write.go | 1 + .../prometheus/template/template.go | 7 +- .../prometheus/prometheus/tsdb/block.go | 1 + .../prometheus/tsdb/chunks/chunks.go | 1 + .../prometheus/tsdb/chunks/head_chunks.go | 3 +- .../prometheus/prometheus/tsdb/compact.go | 1 + .../prometheus/prometheus/tsdb/db.go | 36 ++--- .../prometheus/prometheus/tsdb/head.go | 3 +- .../prometheus/prometheus/tsdb/index/index.go | 1 + .../prometheus/prometheus/tsdb/querier.go | 1 + .../prometheus/tsdb/record/record.go | 1 + .../prometheus/prometheus/tsdb/repair.go | 1 + .../prometheus/tsdb/tombstones/tombstones.go | 1 + .../prometheus/tsdb/tsdbblockutil.go | 1 + .../prometheus/prometheus/tsdb/wal.go | 1 + .../prometheus/tsdb/wal/checkpoint.go | 1 + .../prometheus/prometheus/tsdb/wal/wal.go | 1 + .../prometheus/prometheus/tsdb/wal/watcher.go | 1 + .../prometheus/util/testutil/directory.go | 18 +-- .../prometheus/util/testutil/testing.go | 129 ------------------ .../prometheus/prometheus/web/api/v1/api.go | 1 + .../x/tools/internal/gocommand/invoke.go | 30 +++- .../x/tools/internal/imports/fix.go | 34 +++-- .../x/tools/internal/imports/mod.go | 26 +++- vendor/modules.txt | 4 +- 46 files changed, 178 insertions(+), 236 deletions(-) diff --git a/go.mod b/go.mod index ec6275e07bb..0b7b7272977 100644 --- a/go.mod +++ b/go.mod @@ -47,7 +47,7 @@ require ( github.com/prometheus/client_golang v1.7.1 github.com/prometheus/client_model v0.2.0 github.com/prometheus/common v0.14.0 - github.com/prometheus/prometheus v1.8.2-0.20201021150828-c5a832b39440 + github.com/prometheus/prometheus v1.8.2-0.20201028100903-3245b3267b24 github.com/segmentio/fasthash v0.0.0-20180216231524-a72b379d632e github.com/sony/gobreaker v0.4.1 github.com/spf13/afero v1.2.2 diff --git a/go.sum b/go.sum index b781e3118e7..c3032c359f5 100644 --- a/go.sum +++ b/go.sum @@ -1081,8 +1081,8 @@ github.com/prometheus/prometheus v1.8.2-0.20200805082714-e0cf219f0de2/go.mod h1: github.com/prometheus/prometheus v1.8.2-0.20200819132913-cb830b0a9c78 h1:tHIAD+hgCIb86T0/Du7vGyfHa6J1+XsImQoY8Ete+c8= github.com/prometheus/prometheus v1.8.2-0.20200819132913-cb830b0a9c78/go.mod h1:zfAqy/MwhMFajB9E2n12/9gG2fvofIE9uKDtlZCDxqs= github.com/prometheus/prometheus v1.8.2-0.20200923143134-7e2db3d092f3/go.mod h1:9VNWoDFHOMovlubld5uKKxfCDcPBj2GMOCjcUFXkYaM= -github.com/prometheus/prometheus v1.8.2-0.20201021150828-c5a832b39440 h1:JaGDp2WhzxWFQ2GTXiEtssazFug/5Pp3r9ZHR2gV0N0= -github.com/prometheus/prometheus v1.8.2-0.20201021150828-c5a832b39440/go.mod h1:dCFGRXrHSoQYAAWc8XbmOfsgYEWoY7/XlTecchlehao= +github.com/prometheus/prometheus v1.8.2-0.20201028100903-3245b3267b24 h1:V/4Cj2GytqdqK7OMEz6c4LNjey3SNyfw3pg5jPKtJvQ= +github.com/prometheus/prometheus v1.8.2-0.20201028100903-3245b3267b24/go.mod h1:MDRkz271loM/PrYN+wUNEaTMDGSP760MQzB0yEjdgSQ= github.com/rafaeljusto/redigomock v0.0.0-20190202135759-257e089e14a1 h1:+kGqA4dNN5hn7WwvKdzHl0rdN5AEkbNZd0VjRltAiZg= github.com/rafaeljusto/redigomock v0.0.0-20190202135759-257e089e14a1/go.mod h1:JaY6n2sDr+z2WTsXkOmNRUfDy6FN0L6Nk7x06ndm4tY= github.com/rcrowley/go-metrics v0.0.0-20181016184325-3113b8401b8a/go.mod h1:bCqnVzQkZxMG4s8nGwiZ5l3QUCyqpo9Y+/ZMZ9VjZe4= @@ -1590,8 +1590,8 @@ golang.org/x/tools v0.0.0-20200804011535-6c149bb5ef0d/go.mod h1:njjCfa9FT2d7l9Bc golang.org/x/tools v0.0.0-20200822203824-307de81be3f4/go.mod h1:njjCfa9FT2d7l9Bc6FUM5FLjQPp3cFF28FI3qnDFljA= golang.org/x/tools v0.0.0-20200825202427-b303f430e36d/go.mod h1:njjCfa9FT2d7l9Bc6FUM5FLjQPp3cFF28FI3qnDFljA= golang.org/x/tools v0.0.0-20200904185747-39188db58858/go.mod h1:Cj7w3i3Rnn0Xh82ur9kSqwfTHTeVxaDqrfMjpcNT6bE= -golang.org/x/tools v0.0.0-20201008025239-9df69603baec h1:RY2OghEV/7X1MLaecgm1mwFd3sGvUddm5pGVSxQvX0c= -golang.org/x/tools v0.0.0-20201008025239-9df69603baec/go.mod h1:z6u4i615ZeAfBE4XtMziQW1fSVJXACjjbWkB/mvPzlU= +golang.org/x/tools v0.0.0-20201020161133-226fd2f889ca h1:pvScuB+UnCGDas2naNKUOXruM08MjwVcEdaweeynIqQ= +golang.org/x/tools v0.0.0-20201020161133-226fd2f889ca/go.mod h1:z6u4i615ZeAfBE4XtMziQW1fSVJXACjjbWkB/mvPzlU= golang.org/x/xerrors v0.0.0-20190717185122-a985d3407aa7/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= golang.org/x/xerrors v0.0.0-20191011141410-1b5146add898/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= golang.org/x/xerrors v0.0.0-20191204190536-9bdfabe68543 h1:E7g+9GITq07hpfrRu66IVDexMakfv52eLZ2CXBWiKr4= diff --git a/vendor/github.com/prometheus/prometheus/discovery/discovery.go b/vendor/github.com/prometheus/prometheus/discovery/discovery.go index d77b193b358..5b0402bdb02 100644 --- a/vendor/github.com/prometheus/prometheus/discovery/discovery.go +++ b/vendor/github.com/prometheus/prometheus/discovery/discovery.go @@ -18,8 +18,8 @@ import ( "reflect" "github.com/go-kit/kit/log" - "github.com/prometheus/common/config" + "github.com/prometheus/prometheus/discovery/targetgroup" ) diff --git a/vendor/github.com/prometheus/prometheus/discovery/registry.go b/vendor/github.com/prometheus/prometheus/discovery/registry.go index 5f9a40546ae..2ebb36cb290 100644 --- a/vendor/github.com/prometheus/prometheus/discovery/registry.go +++ b/vendor/github.com/prometheus/prometheus/discovery/registry.go @@ -21,8 +21,9 @@ import ( "strings" "sync" - "github.com/prometheus/prometheus/discovery/targetgroup" "gopkg.in/yaml.v2" + + "github.com/prometheus/prometheus/discovery/targetgroup" ) const ( diff --git a/vendor/github.com/prometheus/prometheus/notifier/notifier.go b/vendor/github.com/prometheus/prometheus/notifier/notifier.go index 29545bbee0e..5856613503e 100644 --- a/vendor/github.com/prometheus/prometheus/notifier/notifier.go +++ b/vendor/github.com/prometheus/prometheus/notifier/notifier.go @@ -32,13 +32,13 @@ import ( "github.com/go-kit/kit/log/level" "github.com/go-openapi/strfmt" "github.com/pkg/errors" - "go.uber.org/atomic" - "github.com/prometheus/alertmanager/api/v2/models" "github.com/prometheus/client_golang/prometheus" config_util "github.com/prometheus/common/config" "github.com/prometheus/common/model" "github.com/prometheus/common/version" + "go.uber.org/atomic" + "github.com/prometheus/prometheus/config" "github.com/prometheus/prometheus/discovery/targetgroup" "github.com/prometheus/prometheus/pkg/labels" diff --git a/vendor/github.com/prometheus/prometheus/pkg/textparse/interface.go b/vendor/github.com/prometheus/prometheus/pkg/textparse/interface.go index cfcd05e210f..557e566622b 100644 --- a/vendor/github.com/prometheus/prometheus/pkg/textparse/interface.go +++ b/vendor/github.com/prometheus/prometheus/pkg/textparse/interface.go @@ -85,12 +85,12 @@ const ( type MetricType string const ( - MetricTypeCounter = "counter" - MetricTypeGauge = "gauge" - MetricTypeHistogram = "histogram" - MetricTypeGaugeHistogram = "gaugehistogram" - MetricTypeSummary = "summary" - MetricTypeInfo = "info" - MetricTypeStateset = "stateset" - MetricTypeUnknown = "unknown" + MetricTypeCounter = MetricType("counter") + MetricTypeGauge = MetricType("gauge") + MetricTypeHistogram = MetricType("histogram") + MetricTypeGaugeHistogram = MetricType("gaugehistogram") + MetricTypeSummary = MetricType("summary") + MetricTypeInfo = MetricType("info") + MetricTypeStateset = MetricType("stateset") + MetricTypeUnknown = MetricType("unknown") ) diff --git a/vendor/github.com/prometheus/prometheus/promql/parser/generated_parser.y.go b/vendor/github.com/prometheus/prometheus/promql/parser/generated_parser.y.go index cf26fcc4724..4117f0e7150 100644 --- a/vendor/github.com/prometheus/prometheus/promql/parser/generated_parser.y.go +++ b/vendor/github.com/prometheus/prometheus/promql/parser/generated_parser.y.go @@ -3,11 +3,8 @@ //line generated_parser.y:15 package parser -import __yyfmt__ "fmt" - -//line generated_parser.y:15 - import ( + __yyfmt__ "fmt" "math" "sort" "strconv" @@ -15,7 +12,7 @@ import ( "github.com/prometheus/prometheus/pkg/labels" "github.com/prometheus/prometheus/pkg/value" -) +) //line generated_parser.y:15 //line generated_parser.y:28 type yySymType struct { diff --git a/vendor/github.com/prometheus/prometheus/promql/parser/printer.go b/vendor/github.com/prometheus/prometheus/promql/parser/printer.go index e187000af74..eef4aa8e418 100644 --- a/vendor/github.com/prometheus/prometheus/promql/parser/printer.go +++ b/vendor/github.com/prometheus/prometheus/promql/parser/printer.go @@ -20,6 +20,7 @@ import ( "time" "github.com/prometheus/common/model" + "github.com/prometheus/prometheus/pkg/labels" ) diff --git a/vendor/github.com/prometheus/prometheus/promql/test.go b/vendor/github.com/prometheus/prometheus/promql/test.go index bcd90200f87..7992c5b23a8 100644 --- a/vendor/github.com/prometheus/prometheus/promql/test.go +++ b/vendor/github.com/prometheus/prometheus/promql/test.go @@ -25,6 +25,7 @@ import ( "github.com/pkg/errors" "github.com/prometheus/common/model" + "github.com/prometheus/prometheus/pkg/labels" "github.com/prometheus/prometheus/promql/parser" "github.com/prometheus/prometheus/storage" diff --git a/vendor/github.com/prometheus/prometheus/promql/value.go b/vendor/github.com/prometheus/prometheus/promql/value.go index a28f06bbae8..fa3a71d352d 100644 --- a/vendor/github.com/prometheus/prometheus/promql/value.go +++ b/vendor/github.com/prometheus/prometheus/promql/value.go @@ -21,11 +21,10 @@ import ( "github.com/pkg/errors" - "github.com/prometheus/prometheus/tsdb/chunkenc" - "github.com/prometheus/prometheus/pkg/labels" "github.com/prometheus/prometheus/promql/parser" "github.com/prometheus/prometheus/storage" + "github.com/prometheus/prometheus/tsdb/chunkenc" ) func (Matrix) Type() parser.ValueType { return parser.ValueTypeMatrix } diff --git a/vendor/github.com/prometheus/prometheus/rules/alerting.go b/vendor/github.com/prometheus/prometheus/rules/alerting.go index c82bc9640ac..a4074221f9f 100644 --- a/vendor/github.com/prometheus/prometheus/rules/alerting.go +++ b/vendor/github.com/prometheus/prometheus/rules/alerting.go @@ -16,19 +16,17 @@ package rules import ( "context" "fmt" + html_template "html/template" "net/url" "strings" "sync" "time" - html_template "html/template" - - yaml "gopkg.in/yaml.v2" - "github.com/go-kit/kit/log" "github.com/go-kit/kit/log/level" "github.com/pkg/errors" "github.com/prometheus/common/model" + yaml "gopkg.in/yaml.v2" "github.com/prometheus/prometheus/pkg/labels" "github.com/prometheus/prometheus/pkg/rulefmt" diff --git a/vendor/github.com/prometheus/prometheus/scrape/manager.go b/vendor/github.com/prometheus/prometheus/scrape/manager.go index 26bb0d1a575..bf73561a125 100644 --- a/vendor/github.com/prometheus/prometheus/scrape/manager.go +++ b/vendor/github.com/prometheus/prometheus/scrape/manager.go @@ -26,8 +26,8 @@ import ( "github.com/go-kit/kit/log" "github.com/go-kit/kit/log/level" "github.com/pkg/errors" - "github.com/prometheus/client_golang/prometheus" + "github.com/prometheus/prometheus/config" "github.com/prometheus/prometheus/discovery/targetgroup" "github.com/prometheus/prometheus/pkg/labels" diff --git a/vendor/github.com/prometheus/prometheus/scrape/scrape.go b/vendor/github.com/prometheus/prometheus/scrape/scrape.go index d52b0ac7f31..f6aa49810db 100644 --- a/vendor/github.com/prometheus/prometheus/scrape/scrape.go +++ b/vendor/github.com/prometheus/prometheus/scrape/scrape.go @@ -192,7 +192,13 @@ type scrapePool struct { appendable storage.Appendable logger log.Logger - mtx sync.Mutex + // targetMtx protects activeTargets and droppedTargets from concurrent reads + // and writes. Only one of Sync/stop/reload may be called at once due to + // manager.mtxScrape so we only need to protect from concurrent reads from + // the ActiveTargets and DroppedTargets methods. This allows those two + // methods to always complete without having to wait on scrape loops to gracefull stop. + targetMtx sync.Mutex + config *config.ScrapeConfig client *http.Client // Targets and loops must always be synchronized to have the same @@ -273,8 +279,8 @@ func newScrapePool(cfg *config.ScrapeConfig, app storage.Appendable, jitterSeed } func (sp *scrapePool) ActiveTargets() []*Target { - sp.mtx.Lock() - defer sp.mtx.Unlock() + sp.targetMtx.Lock() + defer sp.targetMtx.Unlock() var tActive []*Target for _, t := range sp.activeTargets { @@ -284,8 +290,8 @@ func (sp *scrapePool) ActiveTargets() []*Target { } func (sp *scrapePool) DroppedTargets() []*Target { - sp.mtx.Lock() - defer sp.mtx.Unlock() + sp.targetMtx.Lock() + defer sp.targetMtx.Unlock() return sp.droppedTargets } @@ -294,8 +300,7 @@ func (sp *scrapePool) stop() { sp.cancel() var wg sync.WaitGroup - sp.mtx.Lock() - defer sp.mtx.Unlock() + sp.targetMtx.Lock() for fp, l := range sp.loops { wg.Add(1) @@ -308,6 +313,9 @@ func (sp *scrapePool) stop() { delete(sp.loops, fp) delete(sp.activeTargets, fp) } + + sp.targetMtx.Unlock() + wg.Wait() sp.client.CloseIdleConnections() @@ -326,9 +334,6 @@ func (sp *scrapePool) reload(cfg *config.ScrapeConfig) error { targetScrapePoolReloads.Inc() start := time.Now() - sp.mtx.Lock() - defer sp.mtx.Unlock() - client, err := config_util.NewClientFromConfig(cfg.HTTPClientConfig, cfg.JobName, false, false) if err != nil { targetScrapePoolReloadsFailed.Inc() @@ -352,6 +357,8 @@ func (sp *scrapePool) reload(cfg *config.ScrapeConfig) error { mrc = sp.config.MetricRelabelConfigs ) + sp.targetMtx.Lock() + forcedErr := sp.refreshTargetLimitErr() for fp, oldLoop := range sp.loops { var cache *scrapeCache @@ -387,6 +394,8 @@ func (sp *scrapePool) reload(cfg *config.ScrapeConfig) error { sp.loops[fp] = newLoop } + sp.targetMtx.Unlock() + wg.Wait() oldClient.CloseIdleConnections() targetReloadIntervalLength.WithLabelValues(interval.String()).Observe( @@ -398,11 +407,9 @@ func (sp *scrapePool) reload(cfg *config.ScrapeConfig) error { // Sync converts target groups into actual scrape targets and synchronizes // the currently running scraper with the resulting set and returns all scraped and dropped targets. func (sp *scrapePool) Sync(tgs []*targetgroup.Group) { - sp.mtx.Lock() - defer sp.mtx.Unlock() - start := time.Now() + sp.targetMtx.Lock() var all []*Target sp.droppedTargets = []*Target{} for _, tg := range tgs { @@ -419,6 +426,7 @@ func (sp *scrapePool) Sync(tgs []*targetgroup.Group) { } } } + sp.targetMtx.Unlock() sp.sync(all) targetSyncIntervalLength.WithLabelValues(sp.config.JobName).Observe( @@ -431,7 +439,6 @@ func (sp *scrapePool) Sync(tgs []*targetgroup.Group) { // scrape loops for new targets, and stops scrape loops for disappeared targets. // It returns after all stopped scrape loops terminated. func (sp *scrapePool) sync(targets []*Target) { - // This function expects that you have acquired the sp.mtx lock. var ( uniqueLoops = make(map[uint64]loop) interval = time.Duration(sp.config.ScrapeInterval) @@ -442,6 +449,7 @@ func (sp *scrapePool) sync(targets []*Target) { mrc = sp.config.MetricRelabelConfigs ) + sp.targetMtx.Lock() for _, t := range targets { hash := t.hash() @@ -487,6 +495,8 @@ func (sp *scrapePool) sync(targets []*Target) { } } + sp.targetMtx.Unlock() + targetScrapePoolTargetsAdded.WithLabelValues(sp.config.JobName).Set(float64(len(uniqueLoops))) forcedErr := sp.refreshTargetLimitErr() for _, l := range sp.loops { @@ -507,7 +517,6 @@ func (sp *scrapePool) sync(targets []*Target) { // refreshTargetLimitErr returns an error that can be passed to the scrape loops // if the number of targets exceeds the configured limit. func (sp *scrapePool) refreshTargetLimitErr() error { - // This function expects that you have acquired the sp.mtx lock. if sp.config == nil || sp.config.TargetLimit == 0 && !sp.targetLimitHit { return nil } diff --git a/vendor/github.com/prometheus/prometheus/storage/fanout.go b/vendor/github.com/prometheus/prometheus/storage/fanout.go index 62b2ef54ae0..82e65e91b38 100644 --- a/vendor/github.com/prometheus/prometheus/storage/fanout.go +++ b/vendor/github.com/prometheus/prometheus/storage/fanout.go @@ -19,6 +19,7 @@ import ( "github.com/go-kit/kit/log" "github.com/go-kit/kit/log/level" "github.com/prometheus/common/model" + "github.com/prometheus/prometheus/pkg/labels" tsdb_errors "github.com/prometheus/prometheus/tsdb/errors" ) diff --git a/vendor/github.com/prometheus/prometheus/storage/merge.go b/vendor/github.com/prometheus/prometheus/storage/merge.go index e3026cf5eed..1670a119a13 100644 --- a/vendor/github.com/prometheus/prometheus/storage/merge.go +++ b/vendor/github.com/prometheus/prometheus/storage/merge.go @@ -22,6 +22,7 @@ import ( "sync" "github.com/pkg/errors" + "github.com/prometheus/prometheus/pkg/labels" "github.com/prometheus/prometheus/tsdb/chunkenc" "github.com/prometheus/prometheus/tsdb/chunks" diff --git a/vendor/github.com/prometheus/prometheus/storage/remote/client.go b/vendor/github.com/prometheus/prometheus/storage/remote/client.go index b7965a2c98c..4e2b9e5b8ff 100644 --- a/vendor/github.com/prometheus/prometheus/storage/remote/client.go +++ b/vendor/github.com/prometheus/prometheus/storage/remote/client.go @@ -27,6 +27,7 @@ import ( "github.com/gogo/protobuf/proto" "github.com/golang/snappy" + "github.com/opentracing-contrib/go-stdlib/nethttp" "github.com/opentracing/opentracing-go" "github.com/pkg/errors" "github.com/prometheus/client_golang/prometheus" @@ -34,7 +35,6 @@ import ( "github.com/prometheus/common/model" "github.com/prometheus/common/version" - "github.com/opentracing-contrib/go-stdlib/nethttp" "github.com/prometheus/prometheus/prompb" ) diff --git a/vendor/github.com/prometheus/prometheus/storage/remote/codec.go b/vendor/github.com/prometheus/prometheus/storage/remote/codec.go index 47360dd57ba..474e4e40074 100644 --- a/vendor/github.com/prometheus/prometheus/storage/remote/codec.go +++ b/vendor/github.com/prometheus/prometheus/storage/remote/codec.go @@ -24,6 +24,7 @@ import ( "github.com/golang/snappy" "github.com/pkg/errors" "github.com/prometheus/common/model" + "github.com/prometheus/prometheus/pkg/labels" "github.com/prometheus/prometheus/prompb" "github.com/prometheus/prometheus/storage" diff --git a/vendor/github.com/prometheus/prometheus/storage/remote/intern.go b/vendor/github.com/prometheus/prometheus/storage/remote/intern.go index 98eec34141a..23047acd9bb 100644 --- a/vendor/github.com/prometheus/prometheus/storage/remote/intern.go +++ b/vendor/github.com/prometheus/prometheus/storage/remote/intern.go @@ -21,10 +21,9 @@ package remote import ( "sync" - "go.uber.org/atomic" - "github.com/prometheus/client_golang/prometheus" "github.com/prometheus/client_golang/prometheus/promauto" + "go.uber.org/atomic" ) var noReferenceReleases = promauto.NewCounter(prometheus.CounterOpts{ diff --git a/vendor/github.com/prometheus/prometheus/storage/remote/queue_manager.go b/vendor/github.com/prometheus/prometheus/storage/remote/queue_manager.go index 47b5726bdc3..6e04d651594 100644 --- a/vendor/github.com/prometheus/prometheus/storage/remote/queue_manager.go +++ b/vendor/github.com/prometheus/prometheus/storage/remote/queue_manager.go @@ -26,9 +26,9 @@ import ( "github.com/golang/snappy" "github.com/opentracing/opentracing-go" "github.com/opentracing/opentracing-go/ext" + "github.com/prometheus/client_golang/prometheus" "go.uber.org/atomic" - "github.com/prometheus/client_golang/prometheus" "github.com/prometheus/prometheus/config" "github.com/prometheus/prometheus/pkg/labels" "github.com/prometheus/prometheus/pkg/relabel" diff --git a/vendor/github.com/prometheus/prometheus/storage/remote/read.go b/vendor/github.com/prometheus/prometheus/storage/remote/read.go index 39822a6763f..4718b479786 100644 --- a/vendor/github.com/prometheus/prometheus/storage/remote/read.go +++ b/vendor/github.com/prometheus/prometheus/storage/remote/read.go @@ -17,6 +17,7 @@ import ( "context" "github.com/pkg/errors" + "github.com/prometheus/prometheus/pkg/labels" "github.com/prometheus/prometheus/storage" ) diff --git a/vendor/github.com/prometheus/prometheus/storage/remote/storage.go b/vendor/github.com/prometheus/prometheus/storage/remote/storage.go index 36d7c011456..5fb4bfbecbd 100644 --- a/vendor/github.com/prometheus/prometheus/storage/remote/storage.go +++ b/vendor/github.com/prometheus/prometheus/storage/remote/storage.go @@ -22,10 +22,10 @@ import ( "time" "github.com/go-kit/kit/log" - "gopkg.in/yaml.v2" - "github.com/prometheus/client_golang/prometheus" "github.com/prometheus/common/model" + "gopkg.in/yaml.v2" + "github.com/prometheus/prometheus/config" "github.com/prometheus/prometheus/pkg/labels" "github.com/prometheus/prometheus/pkg/logging" diff --git a/vendor/github.com/prometheus/prometheus/storage/remote/write.go b/vendor/github.com/prometheus/prometheus/storage/remote/write.go index 9e93e0c18f0..64d3be326dc 100644 --- a/vendor/github.com/prometheus/prometheus/storage/remote/write.go +++ b/vendor/github.com/prometheus/prometheus/storage/remote/write.go @@ -22,6 +22,7 @@ import ( "github.com/go-kit/kit/log" "github.com/prometheus/client_golang/prometheus" "github.com/prometheus/client_golang/prometheus/promauto" + "github.com/prometheus/prometheus/config" "github.com/prometheus/prometheus/pkg/labels" "github.com/prometheus/prometheus/storage" diff --git a/vendor/github.com/prometheus/prometheus/template/template.go b/vendor/github.com/prometheus/prometheus/template/template.go index 0a31c4e54a4..10a9241c796 100644 --- a/vendor/github.com/prometheus/prometheus/template/template.go +++ b/vendor/github.com/prometheus/prometheus/template/template.go @@ -17,20 +17,19 @@ import ( "bytes" "context" "fmt" + html_template "html/template" "math" "net/url" "regexp" "sort" "strings" - "time" - - html_template "html/template" text_template "text/template" + "time" "github.com/pkg/errors" - "github.com/prometheus/client_golang/prometheus" "github.com/prometheus/common/model" + "github.com/prometheus/prometheus/promql" "github.com/prometheus/prometheus/util/strutil" ) diff --git a/vendor/github.com/prometheus/prometheus/tsdb/block.go b/vendor/github.com/prometheus/prometheus/tsdb/block.go index c6de088d336..e177c112a51 100644 --- a/vendor/github.com/prometheus/prometheus/tsdb/block.go +++ b/vendor/github.com/prometheus/prometheus/tsdb/block.go @@ -26,6 +26,7 @@ import ( "github.com/go-kit/kit/log/level" "github.com/oklog/ulid" "github.com/pkg/errors" + "github.com/prometheus/prometheus/pkg/labels" "github.com/prometheus/prometheus/tsdb/chunkenc" "github.com/prometheus/prometheus/tsdb/chunks" diff --git a/vendor/github.com/prometheus/prometheus/tsdb/chunks/chunks.go b/vendor/github.com/prometheus/prometheus/tsdb/chunks/chunks.go index 6f7ea09b44f..af781e25970 100644 --- a/vendor/github.com/prometheus/prometheus/tsdb/chunks/chunks.go +++ b/vendor/github.com/prometheus/prometheus/tsdb/chunks/chunks.go @@ -27,6 +27,7 @@ import ( "strconv" "github.com/pkg/errors" + "github.com/prometheus/prometheus/tsdb/chunkenc" tsdb_errors "github.com/prometheus/prometheus/tsdb/errors" "github.com/prometheus/prometheus/tsdb/fileutil" diff --git a/vendor/github.com/prometheus/prometheus/tsdb/chunks/head_chunks.go b/vendor/github.com/prometheus/prometheus/tsdb/chunks/head_chunks.go index 154149d0cc5..6f67b8318ac 100644 --- a/vendor/github.com/prometheus/prometheus/tsdb/chunks/head_chunks.go +++ b/vendor/github.com/prometheus/prometheus/tsdb/chunks/head_chunks.go @@ -27,10 +27,11 @@ import ( "sync" "github.com/pkg/errors" + "go.uber.org/atomic" + "github.com/prometheus/prometheus/tsdb/chunkenc" tsdb_errors "github.com/prometheus/prometheus/tsdb/errors" "github.com/prometheus/prometheus/tsdb/fileutil" - "go.uber.org/atomic" ) // Head chunk file header fields constants. diff --git a/vendor/github.com/prometheus/prometheus/tsdb/compact.go b/vendor/github.com/prometheus/prometheus/tsdb/compact.go index fbc5b35833a..cd70ee52db9 100644 --- a/vendor/github.com/prometheus/prometheus/tsdb/compact.go +++ b/vendor/github.com/prometheus/prometheus/tsdb/compact.go @@ -29,6 +29,7 @@ import ( "github.com/oklog/ulid" "github.com/pkg/errors" "github.com/prometheus/client_golang/prometheus" + "github.com/prometheus/prometheus/storage" "github.com/prometheus/prometheus/tsdb/chunkenc" "github.com/prometheus/prometheus/tsdb/chunks" diff --git a/vendor/github.com/prometheus/prometheus/tsdb/db.go b/vendor/github.com/prometheus/prometheus/tsdb/db.go index 5be60b670bf..4c67e08fae6 100644 --- a/vendor/github.com/prometheus/prometheus/tsdb/db.go +++ b/vendor/github.com/prometheus/prometheus/tsdb/db.go @@ -34,16 +34,15 @@ import ( "github.com/oklog/ulid" "github.com/pkg/errors" "github.com/prometheus/client_golang/prometheus" + "golang.org/x/sync/errgroup" + "github.com/prometheus/prometheus/pkg/labels" "github.com/prometheus/prometheus/storage" "github.com/prometheus/prometheus/tsdb/chunkenc" tsdb_errors "github.com/prometheus/prometheus/tsdb/errors" "github.com/prometheus/prometheus/tsdb/fileutil" + _ "github.com/prometheus/prometheus/tsdb/goversion" // Load the package into main to make sure minium Go version is met. "github.com/prometheus/prometheus/tsdb/wal" - "golang.org/x/sync/errgroup" - - // Load the package into main to make sure minium Go version is met. - _ "github.com/prometheus/prometheus/tsdb/goversion" ) const ( @@ -553,7 +552,7 @@ func validateOpts(opts *Options, rngs []int64) (*Options, []int64) { return opts, rngs } -func open(dir string, l log.Logger, r prometheus.Registerer, opts *Options, rngs []int64) (db *DB, returnedErr error) { +func open(dir string, l log.Logger, r prometheus.Registerer, opts *Options, rngs []int64) (_ *DB, returnedErr error) { if err := os.MkdirAll(dir, 0777); err != nil { return nil, err } @@ -584,7 +583,7 @@ func open(dir string, l log.Logger, r prometheus.Registerer, opts *Options, rngs return nil, errors.Wrap(err, "remove tmp dirs") } - db = &DB{ + db := &DB{ dir: dir, logger: l, opts: opts, @@ -601,6 +600,8 @@ func open(dir string, l log.Logger, r prometheus.Registerer, opts *Options, rngs return } + close(db.donec) // DB is never run if it was an error, so close this channel here. + var merr tsdb_errors.MultiError merr.Add(returnedErr) merr.Add(errors.Wrap(db.Close(), "close DB after failed startup")) @@ -624,11 +625,12 @@ func open(dir string, l log.Logger, r prometheus.Registerer, opts *Options, rngs db.lockf = lockf } + var err error ctx, cancel := context.WithCancel(context.Background()) - db.compactor, returnedErr = NewLeveledCompactor(ctx, r, l, rngs, db.chunkPool) - if returnedErr != nil { + db.compactor, err = NewLeveledCompactor(ctx, r, l, rngs, db.chunkPool) + if err != nil { cancel() - return nil, errors.Wrap(returnedErr, "create leveled compactor") + return nil, errors.Wrap(err, "create leveled compactor") } db.compactCancel = cancel @@ -640,15 +642,15 @@ func open(dir string, l log.Logger, r prometheus.Registerer, opts *Options, rngs if opts.WALSegmentSize > 0 { segmentSize = opts.WALSegmentSize } - wlog, returnedErr = wal.NewSize(l, r, walDir, segmentSize, opts.WALCompression) - if returnedErr != nil { - return nil, returnedErr + wlog, err = wal.NewSize(l, r, walDir, segmentSize, opts.WALCompression) + if err != nil { + return nil, err } } - db.head, returnedErr = NewHead(r, l, wlog, rngs[0], dir, db.chunkPool, opts.StripeSize, opts.SeriesLifecycleCallback) - if returnedErr != nil { - return nil, returnedErr + db.head, err = NewHead(r, l, wlog, rngs[0], dir, db.chunkPool, opts.StripeSize, opts.SeriesLifecycleCallback) + if err != nil { + return nil, err } // Register metrics after assigning the head block. @@ -1326,7 +1328,9 @@ func (db *DB) Head() *Head { // Close the partition. func (db *DB) Close() error { close(db.stopc) - db.compactCancel() + if db.compactCancel != nil { + db.compactCancel() + } <-db.donec db.mtx.Lock() diff --git a/vendor/github.com/prometheus/prometheus/tsdb/head.go b/vendor/github.com/prometheus/prometheus/tsdb/head.go index 56b597f3df7..e4dfef20aec 100644 --- a/vendor/github.com/prometheus/prometheus/tsdb/head.go +++ b/vendor/github.com/prometheus/prometheus/tsdb/head.go @@ -28,6 +28,8 @@ import ( "github.com/oklog/ulid" "github.com/pkg/errors" "github.com/prometheus/client_golang/prometheus" + "go.uber.org/atomic" + "github.com/prometheus/prometheus/pkg/labels" "github.com/prometheus/prometheus/storage" "github.com/prometheus/prometheus/tsdb/chunkenc" @@ -38,7 +40,6 @@ import ( "github.com/prometheus/prometheus/tsdb/tombstones" "github.com/prometheus/prometheus/tsdb/tsdbutil" "github.com/prometheus/prometheus/tsdb/wal" - "go.uber.org/atomic" ) var ( diff --git a/vendor/github.com/prometheus/prometheus/tsdb/index/index.go b/vendor/github.com/prometheus/prometheus/tsdb/index/index.go index d7559b346ac..b3d8293f1b6 100644 --- a/vendor/github.com/prometheus/prometheus/tsdb/index/index.go +++ b/vendor/github.com/prometheus/prometheus/tsdb/index/index.go @@ -29,6 +29,7 @@ import ( "unsafe" "github.com/pkg/errors" + "github.com/prometheus/prometheus/pkg/labels" "github.com/prometheus/prometheus/tsdb/chunks" "github.com/prometheus/prometheus/tsdb/encoding" diff --git a/vendor/github.com/prometheus/prometheus/tsdb/querier.go b/vendor/github.com/prometheus/prometheus/tsdb/querier.go index 6df8cba9b4e..63c311ee23d 100644 --- a/vendor/github.com/prometheus/prometheus/tsdb/querier.go +++ b/vendor/github.com/prometheus/prometheus/tsdb/querier.go @@ -20,6 +20,7 @@ import ( "unicode/utf8" "github.com/pkg/errors" + "github.com/prometheus/prometheus/pkg/labels" "github.com/prometheus/prometheus/storage" "github.com/prometheus/prometheus/tsdb/chunkenc" diff --git a/vendor/github.com/prometheus/prometheus/tsdb/record/record.go b/vendor/github.com/prometheus/prometheus/tsdb/record/record.go index e7df28766c5..408882e8324 100644 --- a/vendor/github.com/prometheus/prometheus/tsdb/record/record.go +++ b/vendor/github.com/prometheus/prometheus/tsdb/record/record.go @@ -19,6 +19,7 @@ import ( "sort" "github.com/pkg/errors" + "github.com/prometheus/prometheus/pkg/labels" "github.com/prometheus/prometheus/tsdb/encoding" "github.com/prometheus/prometheus/tsdb/tombstones" diff --git a/vendor/github.com/prometheus/prometheus/tsdb/repair.go b/vendor/github.com/prometheus/prometheus/tsdb/repair.go index 02114cd4f67..49f55d5a8ac 100644 --- a/vendor/github.com/prometheus/prometheus/tsdb/repair.go +++ b/vendor/github.com/prometheus/prometheus/tsdb/repair.go @@ -23,6 +23,7 @@ import ( "github.com/go-kit/kit/log" "github.com/go-kit/kit/log/level" "github.com/pkg/errors" + tsdb_errors "github.com/prometheus/prometheus/tsdb/errors" "github.com/prometheus/prometheus/tsdb/fileutil" ) diff --git a/vendor/github.com/prometheus/prometheus/tsdb/tombstones/tombstones.go b/vendor/github.com/prometheus/prometheus/tsdb/tombstones/tombstones.go index d34a75af4d4..79b68adfbbd 100644 --- a/vendor/github.com/prometheus/prometheus/tsdb/tombstones/tombstones.go +++ b/vendor/github.com/prometheus/prometheus/tsdb/tombstones/tombstones.go @@ -27,6 +27,7 @@ import ( "github.com/go-kit/kit/log" "github.com/go-kit/kit/log/level" "github.com/pkg/errors" + "github.com/prometheus/prometheus/tsdb/encoding" tsdb_errors "github.com/prometheus/prometheus/tsdb/errors" "github.com/prometheus/prometheus/tsdb/fileutil" diff --git a/vendor/github.com/prometheus/prometheus/tsdb/tsdbblockutil.go b/vendor/github.com/prometheus/prometheus/tsdb/tsdbblockutil.go index 15f129d318d..be2c63f9f43 100644 --- a/vendor/github.com/prometheus/prometheus/tsdb/tsdbblockutil.go +++ b/vendor/github.com/prometheus/prometheus/tsdb/tsdbblockutil.go @@ -19,6 +19,7 @@ import ( "path/filepath" "github.com/go-kit/kit/log" + "github.com/prometheus/prometheus/storage" ) diff --git a/vendor/github.com/prometheus/prometheus/tsdb/wal.go b/vendor/github.com/prometheus/prometheus/tsdb/wal.go index 1e503a1481b..7856ac14c65 100644 --- a/vendor/github.com/prometheus/prometheus/tsdb/wal.go +++ b/vendor/github.com/prometheus/prometheus/tsdb/wal.go @@ -31,6 +31,7 @@ import ( "github.com/go-kit/kit/log/level" "github.com/pkg/errors" "github.com/prometheus/client_golang/prometheus" + "github.com/prometheus/prometheus/pkg/labels" "github.com/prometheus/prometheus/tsdb/encoding" "github.com/prometheus/prometheus/tsdb/fileutil" diff --git a/vendor/github.com/prometheus/prometheus/tsdb/wal/checkpoint.go b/vendor/github.com/prometheus/prometheus/tsdb/wal/checkpoint.go index 33e2e58cedb..c27f6f5b9fa 100644 --- a/vendor/github.com/prometheus/prometheus/tsdb/wal/checkpoint.go +++ b/vendor/github.com/prometheus/prometheus/tsdb/wal/checkpoint.go @@ -28,6 +28,7 @@ import ( "github.com/go-kit/kit/log" "github.com/go-kit/kit/log/level" "github.com/pkg/errors" + tsdb_errors "github.com/prometheus/prometheus/tsdb/errors" "github.com/prometheus/prometheus/tsdb/fileutil" "github.com/prometheus/prometheus/tsdb/record" diff --git a/vendor/github.com/prometheus/prometheus/tsdb/wal/wal.go b/vendor/github.com/prometheus/prometheus/tsdb/wal/wal.go index c5401ddf1e8..36df9ad6930 100644 --- a/vendor/github.com/prometheus/prometheus/tsdb/wal/wal.go +++ b/vendor/github.com/prometheus/prometheus/tsdb/wal/wal.go @@ -33,6 +33,7 @@ import ( "github.com/golang/snappy" "github.com/pkg/errors" "github.com/prometheus/client_golang/prometheus" + "github.com/prometheus/prometheus/tsdb/fileutil" ) diff --git a/vendor/github.com/prometheus/prometheus/tsdb/wal/watcher.go b/vendor/github.com/prometheus/prometheus/tsdb/wal/watcher.go index 1fb78005f8c..8670567f132 100644 --- a/vendor/github.com/prometheus/prometheus/tsdb/wal/watcher.go +++ b/vendor/github.com/prometheus/prometheus/tsdb/wal/watcher.go @@ -29,6 +29,7 @@ import ( "github.com/go-kit/kit/log/level" "github.com/pkg/errors" "github.com/prometheus/client_golang/prometheus" + "github.com/prometheus/prometheus/pkg/timestamp" "github.com/prometheus/prometheus/tsdb/record" ) diff --git a/vendor/github.com/prometheus/prometheus/util/testutil/directory.go b/vendor/github.com/prometheus/prometheus/util/testutil/directory.go index 4f0f08fe8f3..340d566c021 100644 --- a/vendor/github.com/prometheus/prometheus/util/testutil/directory.go +++ b/vendor/github.com/prometheus/prometheus/util/testutil/directory.go @@ -21,6 +21,8 @@ import ( "path/filepath" "strconv" "testing" + + "github.com/stretchr/testify/assert" ) const ( @@ -137,32 +139,32 @@ func NewTemporaryDirectory(name string, t T) (handler TemporaryDirectory) { func DirHash(t *testing.T, path string) []byte { hash := sha256.New() err := filepath.Walk(path, func(path string, info os.FileInfo, err error) error { - Ok(t, err) + assert.NoError(t, err) if info.IsDir() { return nil } f, err := os.Open(path) - Ok(t, err) + assert.NoError(t, err) defer f.Close() _, err = io.Copy(hash, f) - Ok(t, err) + assert.NoError(t, err) _, err = io.WriteString(hash, strconv.Itoa(int(info.Size()))) - Ok(t, err) + assert.NoError(t, err) _, err = io.WriteString(hash, info.Name()) - Ok(t, err) + assert.NoError(t, err) modTime, err := info.ModTime().GobEncode() - Ok(t, err) + assert.NoError(t, err) _, err = io.WriteString(hash, string(modTime)) - Ok(t, err) + assert.NoError(t, err) return nil }) - Ok(t, err) + assert.NoError(t, err) return hash.Sum(nil) } diff --git a/vendor/github.com/prometheus/prometheus/util/testutil/testing.go b/vendor/github.com/prometheus/prometheus/util/testutil/testing.go index 1645f80d5c3..8ec50cb00f4 100644 --- a/vendor/github.com/prometheus/prometheus/util/testutil/testing.go +++ b/vendor/github.com/prometheus/prometheus/util/testutil/testing.go @@ -23,140 +23,11 @@ package testutil import ( - "fmt" - "reflect" "testing" - "github.com/davecgh/go-spew/spew" - "github.com/pmezard/go-difflib/difflib" "go.uber.org/goleak" ) -// This package is imported by non-test code and therefore cannot import the -// testing package, which has side effects such as adding flags. Hence we use an -// interface to testing.{T,B}. -type TB interface { - Helper() - Fatalf(string, ...interface{}) -} - -// Assert fails the test if the condition is false. -func Assert(tb TB, condition bool, format string, a ...interface{}) { - tb.Helper() - if !condition { - tb.Fatalf("\033[31m"+format+"\033[39m\n", a...) - } -} - -// Ok fails the test if an err is not nil. -func Ok(tb TB, err error) { - tb.Helper() - if err != nil { - tb.Fatalf("\033[31munexpected error: %v\033[39m\n", err) - } -} - -// NotOk fails the test if an err is nil. -func NotOk(tb TB, err error, a ...interface{}) { - tb.Helper() - if err == nil { - if len(a) != 0 { - format := a[0].(string) - tb.Fatalf("\033[31m"+format+": expected error, got none\033[39m", a[1:]...) - } - tb.Fatalf("\033[31mexpected error, got none\033[39m") - } -} - -// Equals fails the test if exp is not equal to act. -func Equals(tb TB, exp, act interface{}, msgAndArgs ...interface{}) { - tb.Helper() - if !reflect.DeepEqual(exp, act) { - tb.Fatalf("\033[31m%s\n\nexp: %#v\n\ngot: %#v%s\033[39m\n", formatMessage(msgAndArgs), exp, act, diff(exp, act)) - } -} - -func typeAndKind(v interface{}) (reflect.Type, reflect.Kind) { - t := reflect.TypeOf(v) - k := t.Kind() - - if k == reflect.Ptr { - t = t.Elem() - k = t.Kind() - } - return t, k -} - -// diff returns a diff of both values as long as both are of the same type and -// are a struct, map, slice, array or string. Otherwise it returns an empty string. -func diff(expected interface{}, actual interface{}) string { - if expected == nil || actual == nil { - return "" - } - - et, ek := typeAndKind(expected) - at, _ := typeAndKind(actual) - if et != at { - return "" - } - - if ek != reflect.Struct && ek != reflect.Map && ek != reflect.Slice && ek != reflect.Array && ek != reflect.String { - return "" - } - - var e, a string - c := spew.ConfigState{ - Indent: " ", - DisablePointerAddresses: true, - DisableCapacities: true, - SortKeys: true, - } - if et != reflect.TypeOf("") { - e = c.Sdump(expected) - a = c.Sdump(actual) - } else { - e = reflect.ValueOf(expected).String() - a = reflect.ValueOf(actual).String() - } - - diff, _ := difflib.GetUnifiedDiffString(difflib.UnifiedDiff{ - A: difflib.SplitLines(e), - B: difflib.SplitLines(a), - FromFile: "Expected", - FromDate: "", - ToFile: "Actual", - ToDate: "", - Context: 1, - }) - return "\n\nDiff:\n" + diff -} - -// ErrorEqual compares Go errors for equality. -func ErrorEqual(tb TB, left, right error, msgAndArgs ...interface{}) { - tb.Helper() - if left == right { - return - } - - if left != nil && right != nil { - Equals(tb, left.Error(), right.Error(), msgAndArgs...) - return - } - - tb.Fatalf("\033[31m%s\n\nexp: %#v\n\ngot: %#v\033[39m\n", formatMessage(msgAndArgs), left, right) -} - -func formatMessage(msgAndArgs []interface{}) string { - if len(msgAndArgs) == 0 { - return "" - } - - if msg, ok := msgAndArgs[0].(string); ok { - return fmt.Sprintf("\n\nmsg: "+msg, msgAndArgs[1:]...) - } - return "" -} - // TolerantVerifyLeak verifies go leaks but excludes the go routines that are // launched as side effects of some of our dependencies. func TolerantVerifyLeak(m *testing.M) { diff --git a/vendor/github.com/prometheus/prometheus/web/api/v1/api.go b/vendor/github.com/prometheus/prometheus/web/api/v1/api.go index 284278e5277..ea4a2c3b1f4 100644 --- a/vendor/github.com/prometheus/prometheus/web/api/v1/api.go +++ b/vendor/github.com/prometheus/prometheus/web/api/v1/api.go @@ -37,6 +37,7 @@ import ( "github.com/prometheus/client_golang/prometheus" "github.com/prometheus/common/model" "github.com/prometheus/common/route" + "github.com/prometheus/prometheus/config" "github.com/prometheus/prometheus/pkg/gate" "github.com/prometheus/prometheus/pkg/labels" diff --git a/vendor/golang.org/x/tools/internal/gocommand/invoke.go b/vendor/golang.org/x/tools/internal/gocommand/invoke.go index f516e17623d..b5c061b0125 100644 --- a/vendor/golang.org/x/tools/internal/gocommand/invoke.go +++ b/vendor/golang.org/x/tools/internal/gocommand/invoke.go @@ -130,6 +130,8 @@ type Invocation struct { Verb string Args []string BuildFlags []string + ModFlag string + ModFile string Env []string WorkingDir string Logf func(format string, args ...interface{}) @@ -158,17 +160,35 @@ func (i *Invocation) run(ctx context.Context, stdout, stderr io.Writer) error { } goArgs := []string{i.Verb} + + appendModFile := func() { + if i.ModFile != "" { + goArgs = append(goArgs, "-modfile="+i.ModFile) + } + } + appendModFlag := func() { + if i.ModFlag != "" { + goArgs = append(goArgs, "-mod="+i.ModFlag) + } + } + switch i.Verb { + case "env", "version": + goArgs = append(goArgs, i.Args...) case "mod": - // mod needs the sub-verb before build flags. + // mod needs the sub-verb before flags. goArgs = append(goArgs, i.Args[0]) - goArgs = append(goArgs, i.BuildFlags...) + appendModFile() goArgs = append(goArgs, i.Args[1:]...) - case "env": - // env doesn't take build flags. + case "get": + goArgs = append(goArgs, i.BuildFlags...) + appendModFile() goArgs = append(goArgs, i.Args...) - default: + + default: // notably list and build. goArgs = append(goArgs, i.BuildFlags...) + appendModFile() + appendModFlag() goArgs = append(goArgs, i.Args...) } cmd := exec.Command("go", goArgs...) diff --git a/vendor/golang.org/x/tools/internal/imports/fix.go b/vendor/golang.org/x/tools/internal/imports/fix.go index 675d16c873b..d859617b774 100644 --- a/vendor/golang.org/x/tools/internal/imports/fix.go +++ b/vendor/golang.org/x/tools/internal/imports/fix.go @@ -83,7 +83,7 @@ type ImportFix struct { IdentName string // FixType is the type of fix this is (AddImport, DeleteImport, SetImportName). FixType ImportFixType - Relevance int // see pkg + Relevance float64 // see pkg } // An ImportInfo represents a single import statement. @@ -592,9 +592,9 @@ func getFixes(fset *token.FileSet, f *ast.File, filename string, env *ProcessEnv return fixes, nil } -// Highest relevance, used for the standard library. Chosen arbitrarily to -// match pre-existing gopls code. -const MaxRelevance = 7 +// MaxRelevance is the highest relevance, used for the standard library. +// Chosen arbitrarily to match pre-existing gopls code. +const MaxRelevance = 7.0 // getCandidatePkgs works with the passed callback to find all acceptable packages. // It deduplicates by import path, and uses a cached stdlib rather than reading @@ -607,6 +607,10 @@ func getCandidatePkgs(ctx context.Context, wrappedCallback *scanCallback, filena if err != nil { return err } + + var mu sync.Mutex // to guard asynchronous access to dupCheck + dupCheck := map[string]struct{}{} + // Start off with the standard library. for importPath, exports := range stdlib { p := &pkg{ @@ -615,14 +619,12 @@ func getCandidatePkgs(ctx context.Context, wrappedCallback *scanCallback, filena packageName: path.Base(importPath), relevance: MaxRelevance, } + dupCheck[importPath] = struct{}{} if notSelf(p) && wrappedCallback.dirFound(p) && wrappedCallback.packageNameLoaded(p) { wrappedCallback.exportsLoaded(p, exports) } } - var mu sync.Mutex - dupCheck := map[string]struct{}{} - scanFilter := &scanCallback{ rootFound: func(root gopathwalk.Root) bool { // Exclude goroot results -- getting them is relatively expensive, not cached, @@ -658,8 +660,8 @@ func getCandidatePkgs(ctx context.Context, wrappedCallback *scanCallback, filena return resolver.scan(ctx, scanFilter) } -func ScoreImportPaths(ctx context.Context, env *ProcessEnv, paths []string) (map[string]int, error) { - result := make(map[string]int) +func ScoreImportPaths(ctx context.Context, env *ProcessEnv, paths []string) (map[string]float64, error) { + result := make(map[string]float64) resolver, err := env.GetResolver() if err != nil { return nil, err @@ -802,6 +804,8 @@ type ProcessEnv struct { GocmdRunner *gocommand.Runner BuildFlags []string + ModFlag string + ModFile string // Env overrides the OS environment, and can be used to specify // GOPROXY, GO111MODULE, etc. PATH cannot be set here, because @@ -995,7 +999,7 @@ type Resolver interface { // loadExports may be called concurrently. loadExports(ctx context.Context, pkg *pkg, includeTest bool) (string, []string, error) // scoreImportPath returns the relevance for an import path. - scoreImportPath(ctx context.Context, path string) int + scoreImportPath(ctx context.Context, path string) float64 ClearForNewScan() } @@ -1260,10 +1264,10 @@ func packageDirToName(dir string) (packageName string, err error) { } type pkg struct { - dir string // absolute file path to pkg directory ("/usr/lib/go/src/net/http") - importPathShort string // vendorless import path ("net/http", "a/b") - packageName string // package name loaded from source if requested - relevance int // a weakly-defined score of how relevant a package is. 0 is most relevant. + dir string // absolute file path to pkg directory ("/usr/lib/go/src/net/http") + importPathShort string // vendorless import path ("net/http", "a/b") + packageName string // package name loaded from source if requested + relevance float64 // a weakly-defined score of how relevant a package is. 0 is most relevant. } type pkgDistance struct { @@ -1389,7 +1393,7 @@ func (r *gopathResolver) scan(ctx context.Context, callback *scanCallback) error return nil } -func (r *gopathResolver) scoreImportPath(ctx context.Context, path string) int { +func (r *gopathResolver) scoreImportPath(ctx context.Context, path string) float64 { if _, ok := stdlib[path]; ok { return MaxRelevance } diff --git a/vendor/golang.org/x/tools/internal/imports/mod.go b/vendor/golang.org/x/tools/internal/imports/mod.go index 94880d61604..8a83613c572 100644 --- a/vendor/golang.org/x/tools/internal/imports/mod.go +++ b/vendor/golang.org/x/tools/internal/imports/mod.go @@ -59,6 +59,8 @@ func (r *ModuleResolver) init() error { } inv := gocommand.Invocation{ BuildFlags: r.env.BuildFlags, + ModFlag: r.env.ModFlag, + ModFile: r.env.ModFile, Env: r.env.env(), Logf: r.env.Logf, WorkingDir: r.env.WorkingDir, @@ -487,7 +489,7 @@ func (r *ModuleResolver) scan(ctx context.Context, callback *scanCallback) error return nil } -func (r *ModuleResolver) scoreImportPath(ctx context.Context, path string) int { +func (r *ModuleResolver) scoreImportPath(ctx context.Context, path string) float64 { if _, ok := stdlib[path]; ok { return MaxRelevance } @@ -495,17 +497,31 @@ func (r *ModuleResolver) scoreImportPath(ctx context.Context, path string) int { return modRelevance(mod) } -func modRelevance(mod *gocommand.ModuleJSON) int { +func modRelevance(mod *gocommand.ModuleJSON) float64 { + var relevance float64 switch { case mod == nil: // out of scope return MaxRelevance - 4 case mod.Indirect: - return MaxRelevance - 3 + relevance = MaxRelevance - 3 case !mod.Main: - return MaxRelevance - 2 + relevance = MaxRelevance - 2 default: - return MaxRelevance - 1 // main module ties with stdlib + relevance = MaxRelevance - 1 // main module ties with stdlib } + + _, versionString, ok := module.SplitPathVersion(mod.Path) + if ok { + index := strings.Index(versionString, "v") + if index == -1 { + return relevance + } + if versionNumber, err := strconv.ParseFloat(versionString[index+1:], 64); err == nil { + relevance += versionNumber / 1000 + } + } + + return relevance } // canonicalize gets the result of canonicalizing the packages using the results diff --git a/vendor/modules.txt b/vendor/modules.txt index 56a96350284..8ca0369898f 100644 --- a/vendor/modules.txt +++ b/vendor/modules.txt @@ -483,7 +483,7 @@ github.com/prometheus/node_exporter/https github.com/prometheus/procfs github.com/prometheus/procfs/internal/fs github.com/prometheus/procfs/internal/util -# github.com/prometheus/prometheus v1.8.2-0.20201021150828-c5a832b39440 +# github.com/prometheus/prometheus v1.8.2-0.20201028100903-3245b3267b24 ## explicit github.com/prometheus/prometheus/config github.com/prometheus/prometheus/discovery @@ -845,7 +845,7 @@ golang.org/x/text/width # golang.org/x/time v0.0.0-20200630173020-3af7569d3a1e ## explicit golang.org/x/time/rate -# golang.org/x/tools v0.0.0-20201008025239-9df69603baec +# golang.org/x/tools v0.0.0-20201020161133-226fd2f889ca golang.org/x/tools/cmd/goimports golang.org/x/tools/go/ast/astutil golang.org/x/tools/go/gcexportdata From 2071c517f99491fbd221a29a2bb2f1158c0a45ea Mon Sep 17 00:00:00 2001 From: Ganesh Vernekar Date: Wed, 28 Oct 2020 15:50:24 +0530 Subject: [PATCH 5/8] Fix tests Signed-off-by: Ganesh Vernekar --- integration/e2e/scenario_test.go | 4 ++-- pkg/configs/legacy_promql/functions_test.go | 16 ++++++++-------- pkg/querier/chunk_tar_test.go | 4 ++-- pkg/querier/querier_test.go | 7 +++---- pkg/ruler/ruler_test.go | 3 +-- 5 files changed, 16 insertions(+), 18 deletions(-) diff --git a/integration/e2e/scenario_test.go b/integration/e2e/scenario_test.go index c2a5cb45f17..2a31e190107 100644 --- a/integration/e2e/scenario_test.go +++ b/integration/e2e/scenario_test.go @@ -10,7 +10,7 @@ import ( "time" "github.com/go-kit/kit/log" - "github.com/prometheus/prometheus/util/testutil" + "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" "github.com/thanos-io/thanos/pkg/objstore/s3" "gopkg.in/yaml.v2" @@ -23,7 +23,7 @@ const bktName = "cheesecake" func spinup(t *testing.T, networkName string) (*e2e.Scenario, *e2e.HTTPService, *e2e.HTTPService) { s, err := e2e.NewScenario(networkName) - testutil.Ok(t, err) + assert.NoError(t, err) m1 := e2edb.NewMinio(9000, bktName) m2 := e2edb.NewMinio(9001, bktName) diff --git a/pkg/configs/legacy_promql/functions_test.go b/pkg/configs/legacy_promql/functions_test.go index b11e0e14c52..efcda0cf308 100644 --- a/pkg/configs/legacy_promql/functions_test.go +++ b/pkg/configs/legacy_promql/functions_test.go @@ -20,7 +20,7 @@ import ( "github.com/prometheus/prometheus/pkg/labels" "github.com/prometheus/prometheus/pkg/timestamp" - "github.com/prometheus/prometheus/util/testutil" + "github.com/stretchr/testify/assert" ) func TestDeriv(t *testing.T) { @@ -35,21 +35,21 @@ func TestDeriv(t *testing.T) { metric := labels.FromStrings("__name__", "foo") _, err := a.Add(metric, 1493712816939, 1.0) - testutil.Ok(t, err) + assert.NoError(t, err) _, err = a.Add(metric, 1493712846939, 1.0) - testutil.Ok(t, err) + assert.NoError(t, err) err = a.Commit() - testutil.Ok(t, err) + assert.NoError(t, err) query, err := engine.NewInstantQuery(storage, "deriv(foo[30m])", timestamp.Time(1493712846939)) - testutil.Ok(t, err) + assert.NoError(t, err) result := query.Exec(context.Background()) - testutil.Ok(t, result.Err) + assert.NoError(t, result.Err) vec, _ := result.Vector() - testutil.Assert(t, len(vec) == 1, "Expected 1 result, got %d", len(vec)) - testutil.Assert(t, vec[0].V == 0.0, "Expected 0.0 as value, got %f", vec[0].V) + assert.True(t, len(vec) == 1, "Expected 1 result, got %d", len(vec)) + assert.True(t, vec[0].V == 0.0, "Expected 0.0 as value, got %f", vec[0].V) } diff --git a/pkg/querier/chunk_tar_test.go b/pkg/querier/chunk_tar_test.go index 5c7882ef6d3..af926a3ead4 100644 --- a/pkg/querier/chunk_tar_test.go +++ b/pkg/querier/chunk_tar_test.go @@ -14,7 +14,7 @@ import ( "github.com/pkg/errors" "github.com/prometheus/prometheus/promql" - "github.com/prometheus/prometheus/util/testutil" + "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" "github.com/weaveworks/common/user" @@ -53,7 +53,7 @@ func getTarDataFromEnv(t testing.TB) (query string, from, through time.Time, ste func runRangeQuery(t testing.TB, query string, from, through time.Time, step time.Duration, store chunkstore.ChunkStore) { dir, err := ioutil.TempDir("", t.Name()) - testutil.Ok(t, err) + assert.NoError(t, err) defer os.RemoveAll(dir) queryTracker := promql.NewActiveQueryTracker(dir, 1, util.Logger) diff --git a/pkg/querier/querier_test.go b/pkg/querier/querier_test.go index 957f6e1b2aa..71bf199d385 100644 --- a/pkg/querier/querier_test.go +++ b/pkg/querier/querier_test.go @@ -23,7 +23,6 @@ import ( "github.com/prometheus/prometheus/promql" "github.com/prometheus/prometheus/scrape" "github.com/prometheus/prometheus/storage" - "github.com/prometheus/prometheus/util/testutil" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" "github.com/weaveworks/common/user" @@ -262,7 +261,7 @@ func TestNoHistoricalQueryToIngester(t *testing.T) { } dir, err := ioutil.TempDir("", t.Name()) - testutil.Ok(t, err) + assert.NoError(t, err) defer os.RemoveAll(dir) queryTracker := promql.NewActiveQueryTracker(dir, 10, util.Logger) @@ -495,7 +494,7 @@ func mockDistibutorFor(t *testing.T, cs mockChunkStore, through model.Time) *moc func testQuery(t testing.TB, queryable storage.Queryable, end model.Time, q query) *promql.Result { dir, err := ioutil.TempDir("", "test_query") - testutil.Ok(t, err) + assert.NoError(t, err) defer os.RemoveAll(dir) queryTracker := promql.NewActiveQueryTracker(dir, 10, util.Logger) @@ -646,7 +645,7 @@ func TestShortTermQueryToLTS(t *testing.T) { } dir, err := ioutil.TempDir("", t.Name()) - testutil.Ok(t, err) + assert.NoError(t, err) defer os.RemoveAll(dir) queryTracker := promql.NewActiveQueryTracker(dir, 10, util.Logger) diff --git a/pkg/ruler/ruler_test.go b/pkg/ruler/ruler_test.go index f69f97144c2..620af6a8572 100644 --- a/pkg/ruler/ruler_test.go +++ b/pkg/ruler/ruler_test.go @@ -23,7 +23,6 @@ import ( "github.com/prometheus/prometheus/promql" promRules "github.com/prometheus/prometheus/rules" "github.com/prometheus/prometheus/storage" - "github.com/prometheus/prometheus/util/testutil" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" "github.com/weaveworks/common/user" @@ -87,7 +86,7 @@ func (r ruleLimits) RulerMaxRulesPerRuleGroup(_ string) int { func testSetup(t *testing.T, cfg Config) (*promql.Engine, storage.QueryableFunc, Pusher, log.Logger, RulesLimits, func()) { dir, err := ioutil.TempDir("", filepath.Base(t.Name())) - testutil.Ok(t, err) + assert.NoError(t, err) cleanup := func() { os.RemoveAll(dir) } From cb715372f44f8531e655ef419cbf897fdfdebad4 Mon Sep 17 00:00:00 2001 From: Marco Pracucci Date: Wed, 28 Oct 2020 12:30:53 +0100 Subject: [PATCH 6/8] Fixed test Signed-off-by: Marco Pracucci --- pkg/ingester/ingester_v2_test.go | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/pkg/ingester/ingester_v2_test.go b/pkg/ingester/ingester_v2_test.go index 96f1da1af95..f1c50a9fdd2 100644 --- a/pkg/ingester/ingester_v2_test.go +++ b/pkg/ingester/ingester_v2_test.go @@ -1691,11 +1691,12 @@ func TestIngester_v2OpenExistingTSDBOnStartup(t *testing.T) { require.NoError(t, os.MkdirAll(filepath.Join(dir, "user3", "dummy"), 0700)) require.NoError(t, os.MkdirAll(filepath.Join(dir, "user4", "dummy"), 0700)) - // Create a fake TSDB on disk with an empty chunks head segment file (it's invalid and - // opening TSDB should fail). + // Create a fake TSDB on disk with an empty chunks head segment file (it's invalid unless + // it's the last one and opening TSDB should fail). require.NoError(t, os.MkdirAll(filepath.Join(dir, "user2", "wal", ""), 0700)) require.NoError(t, os.MkdirAll(filepath.Join(dir, "user2", "chunks_head", ""), 0700)) require.NoError(t, ioutil.WriteFile(filepath.Join(dir, "user2", "chunks_head", "00000001"), nil, 0700)) + require.NoError(t, ioutil.WriteFile(filepath.Join(dir, "user2", "chunks_head", "00000002"), nil, 0700)) }, check: func(t *testing.T, i *Ingester) { require.Equal(t, 0, len(i.TSDBState.dbs)) From 68737a924541a11282094efc6ca75929e3bf9900 Mon Sep 17 00:00:00 2001 From: Ganesh Vernekar Date: Wed, 28 Oct 2020 19:04:12 +0530 Subject: [PATCH 7/8] CHANGELOG entry Signed-off-by: Ganesh Vernekar --- CHANGELOG.md | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index 24977830fc5..02866cf82cd 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -89,6 +89,10 @@ * [ENHANCEMENT] Blocks storage: reduced number of bucket listing operations to list block content (applies to newly created blocks only). #3363 * [ENHANCEMENT] Ruler: Include the tenant ID on the notifier logs. #3372 * [ENHANCEMENT] Blocks storage Compactor: Added `-compactor.enabled-tenants` and `-compactor.disabled-tenants` to explicitly enable or disable compaction of specific tenants. #3385 +* [ENHANCEMENT] [BUGFIX] Blocks storage: Upgraded Prometheus bringing in the following fixes: #3373 + - [ENHANCEMENT] Creating checkpoint only once even when there are multiple Head compactions in a single `Compact()` call. https://github.com/prometheus/prometheus/pull/8067 + - [BUGFIX] Read repair memory-mapped chunks file which can end up being empty on abrupt shutdowns combined with faulty disks. https://github.com/prometheus/prometheus/pull/8061 + - [BUGFIX] Close TSDB resources on failed startup preventing ingester OOMing. https://github.com/prometheus/prometheus/pull/8031 * [BUGFIX] No-longer-needed ingester operations for queries triggered by queriers and rulers are now canceled. #3178 * [BUGFIX] Ruler: directories in the configured `rules-path` will be removed on startup and shutdown in order to ensure they don't persist between runs. #3195 * [BUGFIX] Handle hash-collisions in the query path. #3192 From 8320fa00616544ee1a79330e69a7c3eccbe7b304 Mon Sep 17 00:00:00 2001 From: Marco Pracucci Date: Wed, 28 Oct 2020 14:46:10 +0100 Subject: [PATCH 8/8] Re-formatted CHANGELOG entry Signed-off-by: Marco Pracucci --- CHANGELOG.md | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 02866cf82cd..a841cea0b19 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -89,10 +89,9 @@ * [ENHANCEMENT] Blocks storage: reduced number of bucket listing operations to list block content (applies to newly created blocks only). #3363 * [ENHANCEMENT] Ruler: Include the tenant ID on the notifier logs. #3372 * [ENHANCEMENT] Blocks storage Compactor: Added `-compactor.enabled-tenants` and `-compactor.disabled-tenants` to explicitly enable or disable compaction of specific tenants. #3385 -* [ENHANCEMENT] [BUGFIX] Blocks storage: Upgraded Prometheus bringing in the following fixes: #3373 - - [ENHANCEMENT] Creating checkpoint only once even when there are multiple Head compactions in a single `Compact()` call. https://github.com/prometheus/prometheus/pull/8067 - - [BUGFIX] Read repair memory-mapped chunks file which can end up being empty on abrupt shutdowns combined with faulty disks. https://github.com/prometheus/prometheus/pull/8061 - - [BUGFIX] Close TSDB resources on failed startup preventing ingester OOMing. https://github.com/prometheus/prometheus/pull/8031 +* [ENHANCEMENT] Blocks storage ingester: Creating checkpoint only once even when there are multiple Head compactions in a single `Compact()` call. #3373 +* [BUGFIX] Blocks storage ingester: Read repair memory-mapped chunks file which can end up being empty on abrupt shutdowns combined with faulty disks. #3373 +* [BUGFIX] Blocks storage ingester: Close TSDB resources on failed startup preventing ingester OOMing. #3373 * [BUGFIX] No-longer-needed ingester operations for queries triggered by queriers and rulers are now canceled. #3178 * [BUGFIX] Ruler: directories in the configured `rules-path` will be removed on startup and shutdown in order to ensure they don't persist between runs. #3195 * [BUGFIX] Handle hash-collisions in the query path. #3192