diff --git a/clients/pkg/logentry/metric/metricvec.go b/clients/pkg/logentry/metric/metricvec.go index 07f73c20873d3..666e3712eb7a7 100644 --- a/clients/pkg/logentry/metric/metricvec.go +++ b/clients/pkg/logentry/metric/metricvec.go @@ -84,6 +84,12 @@ func (c *metricVec) Delete(labels model.LabelSet) bool { return ok } +func (c *metricVec) DeleteAll() { + c.mtx.Lock() + defer c.mtx.Unlock() + c.metrics = map[model.Fingerprint]prometheus.Metric{} +} + // prune will remove all metrics which implement the Expirable interface and have expired // it does not take out a lock on the metrics map so whoever calls this function should do so. func (c *metricVec) prune() { diff --git a/clients/pkg/logentry/stages/decolorize.go b/clients/pkg/logentry/stages/decolorize.go index bac7274b6bad3..26e4f8b53a0dd 100644 --- a/clients/pkg/logentry/stages/decolorize.go +++ b/clients/pkg/logentry/stages/decolorize.go @@ -33,3 +33,8 @@ func (m *decolorizeStage) Run(in chan Entry) chan Entry { func (m *decolorizeStage) Name() string { return StageTypeDecolorize } + +// Cleanup implements Stage. +func (*decolorizeStage) Cleanup() { + // no-op +} diff --git a/clients/pkg/logentry/stages/drop.go b/clients/pkg/logentry/stages/drop.go index 19a2e6c378075..1a19ca294b658 100644 --- a/clients/pkg/logentry/stages/drop.go +++ b/clients/pkg/logentry/stages/drop.go @@ -266,3 +266,8 @@ func (m *dropStage) shouldDrop(e Entry) bool { func (m *dropStage) Name() string { return StageTypeDrop } + +// Cleanup implements Stage. +func (*dropStage) Cleanup() { + // no-op +} diff --git a/clients/pkg/logentry/stages/eventlogmessage.go b/clients/pkg/logentry/stages/eventlogmessage.go index e637c5c920984..bf591a3f5325e 100644 --- a/clients/pkg/logentry/stages/eventlogmessage.go +++ b/clients/pkg/logentry/stages/eventlogmessage.go @@ -142,6 +142,11 @@ func (m *eventLogMessageStage) Name() string { return StageTypeEventLogMessage } +// Cleanup implements Stage. +func (*eventLogMessageStage) Cleanup() { + // no-op +} + // Sanitize a input string to convert it into a valid prometheus label // TODO: switch to prometheus/prometheus/util/strutil/SanitizeFullLabelName func SanitizeFullLabelName(input string) string { diff --git a/clients/pkg/logentry/stages/extensions.go b/clients/pkg/logentry/stages/extensions.go index f25ffe02e8403..236cae7225292 100644 --- a/clients/pkg/logentry/stages/extensions.go +++ b/clients/pkg/logentry/stages/extensions.go @@ -59,6 +59,11 @@ func (c *cri) Name() string { return "cri" } +// Cleanup implements Stage. +func (*cri) Cleanup() { + // no-op +} + // implements Stage interface func (c *cri) Run(entry chan Entry) chan Entry { entry = c.base.Run(entry) diff --git a/clients/pkg/logentry/stages/geoip.go b/clients/pkg/logentry/stages/geoip.go index d127ecf89814b..0786f584d96c7 100644 --- a/clients/pkg/logentry/stages/geoip.go +++ b/clients/pkg/logentry/stages/geoip.go @@ -123,6 +123,11 @@ func (g *geoIPStage) Name() string { return StageTypeGeoIP } +// Cleanup implements Stage. +func (*geoIPStage) Cleanup() { + // no-op +} + func (g *geoIPStage) process(labels model.LabelSet, extracted map[string]interface{}, _ *time.Time, _ *string) { var ip net.IP if g.cfgs.Source != nil { diff --git a/clients/pkg/logentry/stages/json.go b/clients/pkg/logentry/stages/json.go index 36f8f66c0358c..de32bedff2bcd 100644 --- a/clients/pkg/logentry/stages/json.go +++ b/clients/pkg/logentry/stages/json.go @@ -188,3 +188,8 @@ func (j *jsonStage) processEntry(extracted map[string]interface{}, entry *string func (j *jsonStage) Name() string { return StageTypeJSON } + +// Cleanup implements Stage. +func (*jsonStage) Cleanup() { + // no-op +} diff --git a/clients/pkg/logentry/stages/limit.go b/clients/pkg/logentry/stages/limit.go index d5489221e6ac0..7538bd316c500 100644 --- a/clients/pkg/logentry/stages/limit.go +++ b/clients/pkg/logentry/stages/limit.go @@ -138,6 +138,11 @@ func (m *limitStage) Name() string { return StageTypeLimit } +// Cleanup implements Stage. +func (*limitStage) Cleanup() { + // no-op +} + func getDropCountByLabelMetric(registerer prometheus.Registerer) *prometheus.CounterVec { return util.RegisterCounterVec(registerer, "logentry", "dropped_lines_by_label_total", "A count of all log lines dropped as a result of a pipeline stage", diff --git a/clients/pkg/logentry/stages/match.go b/clients/pkg/logentry/stages/match.go index 3b4addbb0de12..2e547fdfcd7c2 100644 --- a/clients/pkg/logentry/stages/match.go +++ b/clients/pkg/logentry/stages/match.go @@ -206,3 +206,8 @@ func (m *matcherStage) processLogQL(e Entry) (Entry, bool) { func (m *matcherStage) Name() string { return StageTypeMatch } + +// Cleanup implements Stage. +func (*matcherStage) Cleanup() { + // no-op +} diff --git a/clients/pkg/logentry/stages/metrics.go b/clients/pkg/logentry/stages/metrics.go index 14386e3b43a40..3aee4cb68a953 100644 --- a/clients/pkg/logentry/stages/metrics.go +++ b/clients/pkg/logentry/stages/metrics.go @@ -128,11 +128,11 @@ func newMetricStage(logger log.Logger, config interface{}, registry prometheus.R metrics[name] = collector } } - return toStage(&metricStage{ + return &metricStage{ logger: logger, cfg: *cfgs, metrics: metrics, - }), nil + }, nil } // metricStage creates and updates prometheus metrics based on extracted pipeline data @@ -142,6 +142,19 @@ type metricStage struct { metrics map[string]prometheus.Collector } +func (m *metricStage) Run(in chan Entry) chan Entry { + out := make(chan Entry) + go func() { + defer close(out) + + for e := range in { + m.Process(e.Labels, e.Extracted, &e.Timestamp, &e.Line) + out <- e + } + }() + return out +} + // Process implements Stage func (m *metricStage) Process(labels model.LabelSet, extracted map[string]interface{}, _ *time.Time, entry *string) { for name, collector := range m.metrics { @@ -178,6 +191,20 @@ func (m *metricStage) Name() string { return StageTypeMetric } +// Cleanup implements Stage. +func (m *metricStage) Cleanup() { + for _, collector := range m.metrics { + switch vec := collector.(type) { + case *metric.Counters: + vec.DeleteAll() + case *metric.Gauges: + vec.DeleteAll() + case *metric.Histograms: + vec.DeleteAll() + } + } +} + // recordCounter will update a counter metric // nolint:goconst func (m *metricStage) recordCounter(name string, counter *metric.Counters, labels model.LabelSet, v interface{}) { diff --git a/clients/pkg/logentry/stages/metrics_test.go b/clients/pkg/logentry/stages/metrics_test.go index 6a14e6c80c1ee..4f5fec89e7f34 100644 --- a/clients/pkg/logentry/stages/metrics_test.go +++ b/clients/pkg/logentry/stages/metrics_test.go @@ -127,6 +127,13 @@ func TestMetricsPipeline(t *testing.T) { strings.NewReader(expectedMetrics)); err != nil { t.Fatalf("mismatch metrics: %v", err) } + + pl.Cleanup() + + if err := testutil.GatherAndCompare(registry, + strings.NewReader("")); err != nil { + t.Fatalf("mismatch metrics: %v", err) + } } func TestNegativeGauge(t *testing.T) { @@ -435,7 +442,7 @@ func TestDefaultIdleDuration(t *testing.T) { if err != nil { t.Fatalf("failed to create stage with metrics: %v", err) } - assert.Equal(t, int64(5*time.Minute.Seconds()), ms.(*stageProcessor).Processor.(*metricStage).cfg["total_keys"].maxIdleSec) + assert.Equal(t, int64(5*time.Minute.Seconds()), ms.(*metricStage).cfg["total_keys"].maxIdleSec) } var ( diff --git a/clients/pkg/logentry/stages/multiline.go b/clients/pkg/logentry/stages/multiline.go index 199ff438a9390..9b6ff9bb790ae 100644 --- a/clients/pkg/logentry/stages/multiline.go +++ b/clients/pkg/logentry/stages/multiline.go @@ -229,3 +229,8 @@ func (m *multilineStage) flush(out chan Entry, s *multilineState) { func (m *multilineStage) Name() string { return StageTypeMultiline } + +// Cleanup implements Stage. +func (*multilineStage) Cleanup() { + // no-op +} diff --git a/clients/pkg/logentry/stages/pack.go b/clients/pkg/logentry/stages/pack.go index 737fa8d36b796..e06e86dd7fc39 100644 --- a/clients/pkg/logentry/stages/pack.go +++ b/clients/pkg/logentry/stages/pack.go @@ -218,3 +218,8 @@ func (m *packStage) pack(e Entry) Entry { func (m *packStage) Name() string { return StageTypePack } + +// Cleanup implements Stage. +func (*packStage) Cleanup() { + // no-op +} diff --git a/clients/pkg/logentry/stages/pipeline.go b/clients/pkg/logentry/stages/pipeline.go index c20a7784c511c..fbc5b653d725f 100644 --- a/clients/pkg/logentry/stages/pipeline.go +++ b/clients/pkg/logentry/stages/pipeline.go @@ -30,6 +30,13 @@ type Pipeline struct { dropCount *prometheus.CounterVec } +// Cleanup implements Stage. +func (p *Pipeline) Cleanup() { + for _, s := range p.stages { + s.Cleanup() + } +} + // NewPipeline creates a new log entry pipeline from a configuration func NewPipeline(logger log.Logger, stgs PipelineStages, jobName *string, registerer prometheus.Registerer) (*Pipeline, error) { st := []Stage{} @@ -169,6 +176,7 @@ func (p *Pipeline) Wrap(next api.EntryHandler) api.EntryHandler { return api.NewEntryHandler(handlerIn, func() { once.Do(func() { close(handlerIn) }) wg.Wait() + p.Cleanup() }) } diff --git a/clients/pkg/logentry/stages/sampling.go b/clients/pkg/logentry/stages/sampling.go index 73d4b25402967..6340ec6de3cd9 100644 --- a/clients/pkg/logentry/stages/sampling.go +++ b/clients/pkg/logentry/stages/sampling.go @@ -111,3 +111,8 @@ func (m *samplingStage) randomNumber() uint64 { func (m *samplingStage) Name() string { return StageTypeSampling } + +// Cleanup implements Stage. +func (*samplingStage) Cleanup() { + // no-op +} diff --git a/clients/pkg/logentry/stages/stage.go b/clients/pkg/logentry/stages/stage.go index 1c19face4044d..86415738469f4 100644 --- a/clients/pkg/logentry/stages/stage.go +++ b/clients/pkg/logentry/stages/stage.go @@ -62,6 +62,7 @@ type Entry struct { type Stage interface { Name() string Run(chan Entry) chan Entry + Cleanup() } func (entry *Entry) copy() *Entry { @@ -228,3 +229,8 @@ func New(logger log.Logger, jobName *string, stageType string, } return creator(params) } + +// Cleanup implements Stage. +func (*stageProcessor) Cleanup() { + // no-op +} diff --git a/clients/pkg/logentry/stages/structuredmetadata.go b/clients/pkg/logentry/stages/structuredmetadata.go index cdab88a956c7f..e8222c1f49e91 100644 --- a/clients/pkg/logentry/stages/structuredmetadata.go +++ b/clients/pkg/logentry/stages/structuredmetadata.go @@ -33,6 +33,11 @@ func (s *structuredMetadataStage) Name() string { return StageTypeStructuredMetadata } +// Cleanup implements Stage. +func (*structuredMetadataStage) Cleanup() { + // no-op +} + func (s *structuredMetadataStage) Run(in chan Entry) chan Entry { return RunWith(in, func(e Entry) Entry { processLabelsConfigs(s.logger, e.Extracted, s.cfgs, func(labelName model.LabelName, labelValue model.LabelValue) { diff --git a/clients/pkg/promtail/targets/file/filetarget.go b/clients/pkg/promtail/targets/file/filetarget.go index 2c52cbead922f..563e6fb8c9de7 100644 --- a/clients/pkg/promtail/targets/file/filetarget.go +++ b/clients/pkg/promtail/targets/file/filetarget.go @@ -4,6 +4,7 @@ import ( "flag" "os" "path/filepath" + "sync" "time" "github.com/bmatcuk/doublestar" @@ -24,6 +25,8 @@ const ( FilenameLabel = "filename" ) +var errFileTargetStopped = errors.New("File target is stopped") + // Config describes behavior for Target type Config struct { SyncPeriod time.Duration `mapstructure:"sync_period" yaml:"sync_period"` @@ -92,12 +95,14 @@ type FileTarget struct { fileEventWatcher chan fsnotify.Event targetEventHandler chan fileTargetEvent watches map[string]struct{} + watchesMutex sync.Mutex path string pathExclude string quit chan struct{} done chan struct{} - readers map[string]Reader + readers map[string]Reader + readersMutex sync.Mutex targetConfig *Config watchConfig WatchConfig @@ -150,7 +155,7 @@ func NewFileTarget( // Ready if at least one file is being tailed func (t *FileTarget) Ready() bool { - return len(t.readers) > 0 + return t.getReadersLen() > 0 } // Stop the target. @@ -178,17 +183,21 @@ func (t *FileTarget) Labels() model.LabelSet { // Details implements a Target func (t *FileTarget) Details() interface{} { files := map[string]int64{} + t.readersMutex.Lock() for fileName := range t.readers { files[fileName], _ = t.positions.Get(fileName) } + t.readersMutex.Unlock() return files } func (t *FileTarget) run() { defer func() { + t.readersMutex.Lock() for _, v := range t.readers { v.Stop() } + t.readersMutex.Unlock() level.Info(t.logger).Log("msg", "filetarget: watcher closed, tailer stopped, positions saved", "path", t.path) close(t.done) }() @@ -216,6 +225,11 @@ func (t *FileTarget) run() { } case <-ticker.C: err := t.sync() + if errors.Is(err, errFileTargetStopped) { + // This file target has been stopped. + // This is normal and there is no need to log an error. + return + } if err != nil { level.Error(t.logger).Log("msg", "error running sync function", "error", err) } @@ -281,15 +295,28 @@ func (t *FileTarget) sync() error { } // Add any directories which are not already being watched. + t.watchesMutex.Lock() toStartWatching := missing(t.watches, dirs) - t.startWatching(toStartWatching) + t.watchesMutex.Unlock() + err := t.startWatching(toStartWatching) + if errors.Is(err, errFileTargetStopped) { + return err + } // Remove any directories which no longer need watching. + t.watchesMutex.Lock() toStopWatching := missing(dirs, t.watches) - t.stopWatching(toStopWatching) + t.watchesMutex.Unlock() + + err = t.stopWatching(toStopWatching) + if errors.Is(err, errFileTargetStopped) { + return err + } // fsnotify.Watcher doesn't allow us to see what is currently being watched so we have to track it ourselves. + t.watchesMutex.Lock() t.watches = dirs + t.watchesMutex.Unlock() // Check if any running tailers have stopped because of errors and remove them from the running list // (They will be restarted in startTailing) @@ -299,41 +326,55 @@ func (t *FileTarget) sync() error { t.startTailing(matches) // Stop tailing any files which no longer exist + t.readersMutex.Lock() toStopTailing := toStopTailing(matches, t.readers) + t.readersMutex.Unlock() t.stopTailingAndRemovePosition(toStopTailing) return nil } -func (t *FileTarget) startWatching(dirs map[string]struct{}) { +func (t *FileTarget) startWatching(dirs map[string]struct{}) error { for dir := range dirs { - if _, ok := t.watches[dir]; ok { + if _, ok := t.getWatch(dir); ok { continue } + level.Info(t.logger).Log("msg", "watching new directory", "directory", dir) - t.targetEventHandler <- fileTargetEvent{ + select { + case <-t.quit: + return errFileTargetStopped + case t.targetEventHandler <- fileTargetEvent{ path: dir, eventType: fileTargetEventWatchStart, + }: } } + return nil } -func (t *FileTarget) stopWatching(dirs map[string]struct{}) { +func (t *FileTarget) stopWatching(dirs map[string]struct{}) error { for dir := range dirs { - if _, ok := t.watches[dir]; !ok { + if _, ok := t.getWatch(dir); !ok { continue } + level.Info(t.logger).Log("msg", "removing directory from watcher", "directory", dir) - t.targetEventHandler <- fileTargetEvent{ + select { + case <-t.quit: + return errFileTargetStopped + case t.targetEventHandler <- fileTargetEvent{ path: dir, eventType: fileTargetEventWatchStop, + }: } } + return nil } func (t *FileTarget) startTailing(ps []string) { for _, p := range ps { - if _, ok := t.readers[p]; ok { + if _, ok := t.getReader(p); ok { continue } @@ -387,7 +428,7 @@ func (t *FileTarget) startTailing(ps []string) { } reader = tailer } - t.readers[p] = reader + t.setReader(p, reader) } } @@ -395,10 +436,10 @@ func (t *FileTarget) startTailing(ps []string) { // Call this when a file no longer exists and you want to remove all traces of it. func (t *FileTarget) stopTailingAndRemovePosition(ps []string) { for _, p := range ps { - if reader, ok := t.readers[p]; ok { + if reader, ok := t.getReader(p); ok { reader.Stop() t.positions.Remove(reader.Path()) - delete(t.readers, p) + t.removeReader(p) } } } @@ -406,6 +447,7 @@ func (t *FileTarget) stopTailingAndRemovePosition(ps []string) { // pruneStoppedTailers removes any tailers which have stopped running from // the list of active tailers. This allows them to be restarted if there were errors. func (t *FileTarget) pruneStoppedTailers() { + t.readersMutex.Lock() toRemove := make([]string, 0, len(t.readers)) for k, t := range t.readers { if !t.IsRunning() { @@ -415,6 +457,45 @@ func (t *FileTarget) pruneStoppedTailers() { for _, tr := range toRemove { delete(t.readers, tr) } + t.readersMutex.Unlock() +} + +func (t *FileTarget) getReadersLen() int { + t.readersMutex.Lock() + defer t.readersMutex.Unlock() + return len(t.readers) +} + +func (t *FileTarget) getReader(val string) (Reader, bool) { + t.readersMutex.Lock() + defer t.readersMutex.Unlock() + reader, ok := t.readers[val] + return reader, ok +} + +func (t *FileTarget) setReader(val string, reader Reader) { + t.readersMutex.Lock() + defer t.readersMutex.Unlock() + t.readers[val] = reader +} + +func (t *FileTarget) getWatch(val string) (struct{}, bool) { + t.watchesMutex.Lock() + defer t.watchesMutex.Unlock() + fileTarget, ok := t.watches[val] + return fileTarget, ok +} + +func (t *FileTarget) removeReader(val string) { + t.readersMutex.Lock() + defer t.readersMutex.Unlock() + delete(t.readers, val) +} + +func (t *FileTarget) getWatchesLen() int { + t.watchesMutex.Lock() + defer t.watchesMutex.Unlock() + return len(t.watches) } func toStopTailing(nt []string, et map[string]Reader) []string { @@ -442,7 +523,7 @@ func toStopTailing(nt []string, et map[string]Reader) []string { func (t *FileTarget) reportSize(ms []string) { for _, m := range ms { // Ask the tailer to update the size if a tailer exists, this keeps position and size metrics in sync - if reader, ok := t.readers[m]; ok { + if reader, ok := t.getReader(m); ok { err := reader.MarkPositionAndSize() if err != nil { level.Warn(t.logger).Log("msg", "failed to get file size from tailer, ", "file", m, "error", err) @@ -459,7 +540,6 @@ func (t *FileTarget) reportSize(ms []string) { } t.metrics.totalBytes.WithLabelValues(m).Set(float64(fi.Size())) } - } } diff --git a/clients/pkg/promtail/targets/file/filetarget_test.go b/clients/pkg/promtail/targets/file/filetarget_test.go index f3cde7bf819a4..0b72191634bde 100644 --- a/clients/pkg/promtail/targets/file/filetarget_test.go +++ b/clients/pkg/promtail/targets/file/filetarget_test.go @@ -329,6 +329,93 @@ func TestFileTarget_StopsTailersCleanly_Parallel(t *testing.T) { ps.Stop() } +// Make sure that Stop() doesn't hang if FileTarget is waiting on a channel send. +func TestFileTarget_StopAbruptly(t *testing.T) { + w := log.NewSyncWriter(os.Stderr) + logger := log.NewLogfmtLogger(w) + + dirName := newTestLogDirectories(t) + positionsFileName := filepath.Join(dirName, "positions.yml") + logDir1 := filepath.Join(dirName, "log1") + logDir2 := filepath.Join(dirName, "log2") + logDir3 := filepath.Join(dirName, "log3") + + logfile1 := filepath.Join(logDir1, "test1.log") + logfile2 := filepath.Join(logDir2, "test1.log") + logfile3 := filepath.Join(logDir3, "test1.log") + + ps, err := positions.New(logger, positions.Config{ + SyncPeriod: 10 * time.Millisecond, + PositionsFile: positionsFileName, + }) + require.NoError(t, err) + + client := fake.New(func() {}) + defer client.Stop() + + // fakeHandler has to be a buffered channel so that we can call the len() function on it. + // We need to call len() to check if the channel is full. + fakeHandler := make(chan fileTargetEvent, 1) + pathToWatch := filepath.Join(dirName, "**", "*.log") + registry := prometheus.NewRegistry() + target, err := NewFileTarget(NewMetrics(registry), logger, client, ps, pathToWatch, "", nil, nil, &Config{ + SyncPeriod: 10 * time.Millisecond, + }, DefaultWatchConig, nil, fakeHandler, "", nil) + assert.NoError(t, err) + + // Create a directory, still nothing is watched. + err = os.MkdirAll(logDir1, 0750) + assert.NoError(t, err) + _, err = os.Create(logfile1) + assert.NoError(t, err) + + // There should be only one WatchStart event in the channel so far. + ftEvent := <-fakeHandler + require.Equal(t, fileTargetEventWatchStart, ftEvent.eventType) + + requireEventually(t, func() bool { + return target.getReadersLen() == 1 + }, "expected 1 tailer to be created") + + require.NoError(t, testutil.GatherAndCompare(registry, bytes.NewBufferString(` + # HELP promtail_files_active_total Number of active files. + # TYPE promtail_files_active_total gauge + promtail_files_active_total 1 + `), "promtail_files_active_total")) + + // Create two directories - one more than the buffer of fakeHandler, + // so that the file target hands until we call Stop(). + err = os.MkdirAll(logDir2, 0750) + assert.NoError(t, err) + _, err = os.Create(logfile2) + assert.NoError(t, err) + + err = os.MkdirAll(logDir3, 0750) + assert.NoError(t, err) + _, err = os.Create(logfile3) + assert.NoError(t, err) + + // Wait until the file target is waiting on a channel send due to a full channel buffer. + requireEventually(t, func() bool { + return len(fakeHandler) == 1 + }, "expected an event in the fakeHandler channel") + + // If FileHandler works well, then it will stop waiting for + // the blocked fakeHandler and stop cleanly. + // This is why this time we don't drain fakeHandler. + requireEventually(t, func() bool { + target.Stop() + ps.Stop() + return true + }, "expected FileTarget not to hang") + + require.NoError(t, testutil.GatherAndCompare(registry, bytes.NewBufferString(` + # HELP promtail_files_active_total Number of active files. + # TYPE promtail_files_active_total gauge + promtail_files_active_total 0 + `), "promtail_files_active_total")) +} + func TestFileTargetPathExclusion(t *testing.T) { w := log.NewSyncWriter(os.Stderr) logger := log.NewLogfmtLogger(w) diff --git a/docs/sources/send-data/promtail/configuration.md b/docs/sources/send-data/promtail/configuration.md index cbd5c7fc717e2..fb6f1b802c3b6 100644 --- a/docs/sources/send-data/promtail/configuration.md +++ b/docs/sources/send-data/promtail/configuration.md @@ -681,7 +681,8 @@ The metrics stage allows for defining metrics from the extracted data. Created metrics are not pushed to Loki and are instead exposed via Promtail's `/metrics` endpoint. Prometheus should be configured to scrape Promtail to be -able to retrieve the metrics configured by this stage. +able to retrieve the metrics configured by this stage. +If Promtail's configuration is reloaded, all metrics will be reset. ```yaml diff --git a/docs/sources/send-data/promtail/stages/metrics.md b/docs/sources/send-data/promtail/stages/metrics.md index 055f7e076e7a5..b034bd6d6d6a1 100644 --- a/docs/sources/send-data/promtail/stages/metrics.md +++ b/docs/sources/send-data/promtail/stages/metrics.md @@ -13,7 +13,8 @@ The `metrics` stage is an action stage that allows for defining and updating metrics based on data from the extracted map. Note that created metrics are not pushed to Loki and are instead exposed via Promtail's `/metrics` endpoint. Prometheus should be configured to scrape Promtail to be able to retrieve the -metrics configured by this stage. +metrics configured by this stage. If Promtail's configuration is reloaded, +all metrics will be reset. ## Schema