Add a feature flag to enable the new discovery manager (#9537)

* Add a feature flag to enable the new manager

This PR creates a copy of the legacy manager and uses it by default.

It is a companion PR to #9349. With this PR, users can enable the new
discovery manager and provide us with any feedback / side effects that
the new behaviour might have.

Signed-off-by: Julien Pivotto <roidelapluie@inuits.eu>
This commit is contained in:
Julien Pivotto 2021-10-20 10:15:54 +02:00 committed by GitHub
parent 68bc92b520
commit 432005826d
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
6 changed files with 1809 additions and 10 deletions

View File

@ -58,6 +58,8 @@ import (
"github.com/prometheus/prometheus/config"
"github.com/prometheus/prometheus/discovery"
_ "github.com/prometheus/prometheus/discovery/install" // Register service discovery implementations.
"github.com/prometheus/prometheus/discovery/legacymanager"
"github.com/prometheus/prometheus/discovery/targetgroup"
"github.com/prometheus/prometheus/notifier"
"github.com/prometheus/prometheus/pkg/exemplar"
"github.com/prometheus/prometheus/pkg/labels"
@ -122,6 +124,7 @@ type flagConfig struct {
enablePromQLAtModifier bool
enablePromQLNegativeOffset bool
enableExpandExternalLabels bool
enableNewSDManager bool
prometheusURL string
corsRegexString string
@ -156,6 +159,9 @@ func (c *flagConfig) setFeatureListOptions(logger log.Logger) error {
case "extra-scrape-metrics":
c.scrape.ExtraMetrics = true
level.Info(logger).Log("msg", "Experimental additional scrape metrics")
case "new-service-discovery-manager":
c.enableNewSDManager = true
level.Info(logger).Log("msg", "Experimental service discovery manager")
case "":
continue
default:
@ -319,7 +325,7 @@ func main() {
a.Flag("query.max-samples", "Maximum number of samples a single query can load into memory. Note that queries will fail if they try to load more samples than this into memory, so this also limits the number of samples a query can return.").
Default("50000000").IntVar(&cfg.queryMaxSamples)
a.Flag("enable-feature", "Comma separated feature names to enable. Valid options: exemplar-storage, expand-external-labels, memory-snapshot-on-shutdown, promql-at-modifier, promql-negative-offset, remote-write-receiver, extra-scrape-metrics. See https://prometheus.io/docs/prometheus/latest/feature_flags/ for more details.").
a.Flag("enable-feature", "Comma separated feature names to enable. Valid options: exemplar-storage, expand-external-labels, memory-snapshot-on-shutdown, promql-at-modifier, promql-negative-offset, remote-write-receiver, extra-scrape-metrics, new-service-discovery-manager. See https://prometheus.io/docs/prometheus/latest/feature_flags/ for more details.").
Default("").StringsVar(&cfg.featureList)
promlogflag.AddFlags(a, &cfg.promlogConfig)
@ -459,11 +465,22 @@ func main() {
notifierManager = notifier.NewManager(&cfg.notifier, log.With(logger, "component", "notifier"))
ctxScrape, cancelScrape = context.WithCancel(context.Background())
discoveryManagerScrape = discovery.NewManager(ctxScrape, log.With(logger, "component", "discovery manager scrape"), discovery.Name("scrape"))
ctxNotify, cancelNotify = context.WithCancel(context.Background())
discoveryManagerNotify = discovery.NewManager(ctxNotify, log.With(logger, "component", "discovery manager notify"), discovery.Name("notify"))
discoveryManagerScrape discoveryManager
discoveryManagerNotify discoveryManager
)
if cfg.enableNewSDManager {
discovery.RegisterMetrics()
discoveryManagerScrape = discovery.NewManager(ctxScrape, log.With(logger, "component", "discovery manager scrape"), discovery.Name("scrape"))
discoveryManagerNotify = discovery.NewManager(ctxNotify, log.With(logger, "component", "discovery manager notify"), discovery.Name("notify"))
} else {
legacymanager.RegisterMetrics()
discoveryManagerScrape = legacymanager.NewManager(ctxScrape, log.With(logger, "component", "discovery manager scrape"), legacymanager.Name("scrape"))
discoveryManagerNotify = legacymanager.NewManager(ctxNotify, log.With(logger, "component", "discovery manager notify"), legacymanager.Name("notify"))
}
var (
scrapeManager = scrape.NewManager(&cfg.scrape, log.With(logger, "component", "scrape manager"), fanoutStorage)
opts = promql.EngineOpts{
@ -1346,3 +1363,12 @@ func (l jaegerLogger) Infof(msg string, args ...interface{}) {
keyvals := []interface{}{"msg", fmt.Sprintf(msg, args...)}
level.Info(l.logger).Log(keyvals...)
}
// discoveryManager interfaces the discovery manager. This is used to keep using
// the manager that restarts SD's on reload for a few releases until we feel
// the new manager can be enabled for all users.
type discoveryManager interface {
ApplyConfig(cfg map[string]discovery.Configs) error
Run() error
SyncCh() <-chan map[string][]*targetgroup.Group
}

View File

@ -0,0 +1,357 @@
// Copyright 2016 The Prometheus Authors
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
package legacymanager
import (
"context"
"fmt"
"reflect"
"sync"
"time"
"github.com/go-kit/log"
"github.com/go-kit/log/level"
"github.com/prometheus/client_golang/prometheus"
"github.com/prometheus/prometheus/discovery"
"github.com/prometheus/prometheus/discovery/targetgroup"
)
var (
failedConfigs = prometheus.NewGaugeVec(
prometheus.GaugeOpts{
Name: "prometheus_sd_failed_configs",
Help: "Current number of service discovery configurations that failed to load.",
},
[]string{"name"},
)
discoveredTargets = prometheus.NewGaugeVec(
prometheus.GaugeOpts{
Name: "prometheus_sd_discovered_targets",
Help: "Current number of discovered targets.",
},
[]string{"name", "config"},
)
receivedUpdates = prometheus.NewCounterVec(
prometheus.CounterOpts{
Name: "prometheus_sd_received_updates_total",
Help: "Total number of update events received from the SD providers.",
},
[]string{"name"},
)
delayedUpdates = prometheus.NewCounterVec(
prometheus.CounterOpts{
Name: "prometheus_sd_updates_delayed_total",
Help: "Total number of update events that couldn't be sent immediately.",
},
[]string{"name"},
)
sentUpdates = prometheus.NewCounterVec(
prometheus.CounterOpts{
Name: "prometheus_sd_updates_total",
Help: "Total number of update events sent to the SD consumers.",
},
[]string{"name"},
)
)
func RegisterMetrics() {
prometheus.MustRegister(failedConfigs, discoveredTargets, receivedUpdates, delayedUpdates, sentUpdates)
}
type poolKey struct {
setName string
provider string
}
// provider holds a Discoverer instance, its configuration and its subscribers.
type provider struct {
name string
d discovery.Discoverer
subs []string
config interface{}
}
// NewManager is the Discovery Manager constructor.
func NewManager(ctx context.Context, logger log.Logger, options ...func(*Manager)) *Manager {
if logger == nil {
logger = log.NewNopLogger()
}
mgr := &Manager{
logger: logger,
syncCh: make(chan map[string][]*targetgroup.Group),
targets: make(map[poolKey]map[string]*targetgroup.Group),
discoverCancel: []context.CancelFunc{},
ctx: ctx,
updatert: 5 * time.Second,
triggerSend: make(chan struct{}, 1),
}
for _, option := range options {
option(mgr)
}
return mgr
}
// Name sets the name of the manager.
func Name(n string) func(*Manager) {
return func(m *Manager) {
m.mtx.Lock()
defer m.mtx.Unlock()
m.name = n
}
}
// Manager maintains a set of discovery providers and sends each update to a map channel.
// Targets are grouped by the target set name.
type Manager struct {
logger log.Logger
name string
mtx sync.RWMutex
ctx context.Context
discoverCancel []context.CancelFunc
// Some Discoverers(eg. k8s) send only the updates for a given target group
// so we use map[tg.Source]*targetgroup.Group to know which group to update.
targets map[poolKey]map[string]*targetgroup.Group
// providers keeps track of SD providers.
providers []*provider
// The sync channel sends the updates as a map where the key is the job value from the scrape config.
syncCh chan map[string][]*targetgroup.Group
// How long to wait before sending updates to the channel. The variable
// should only be modified in unit tests.
updatert time.Duration
// The triggerSend channel signals to the manager that new updates have been received from providers.
triggerSend chan struct{}
}
// Run starts the background processing
func (m *Manager) Run() error {
go m.sender()
for range m.ctx.Done() {
m.cancelDiscoverers()
return m.ctx.Err()
}
return nil
}
// SyncCh returns a read only channel used by all the clients to receive target updates.
func (m *Manager) SyncCh() <-chan map[string][]*targetgroup.Group {
return m.syncCh
}
// ApplyConfig removes all running discovery providers and starts new ones using the provided config.
func (m *Manager) ApplyConfig(cfg map[string]discovery.Configs) error {
m.mtx.Lock()
defer m.mtx.Unlock()
for pk := range m.targets {
if _, ok := cfg[pk.setName]; !ok {
discoveredTargets.DeleteLabelValues(m.name, pk.setName)
}
}
m.cancelDiscoverers()
m.targets = make(map[poolKey]map[string]*targetgroup.Group)
m.providers = nil
m.discoverCancel = nil
failedCount := 0
for name, scfg := range cfg {
failedCount += m.registerProviders(scfg, name)
discoveredTargets.WithLabelValues(m.name, name).Set(0)
}
failedConfigs.WithLabelValues(m.name).Set(float64(failedCount))
for _, prov := range m.providers {
m.startProvider(m.ctx, prov)
}
return nil
}
// StartCustomProvider is used for sdtool. Only use this if you know what you're doing.
func (m *Manager) StartCustomProvider(ctx context.Context, name string, worker discovery.Discoverer) {
p := &provider{
name: name,
d: worker,
subs: []string{name},
}
m.providers = append(m.providers, p)
m.startProvider(ctx, p)
}
func (m *Manager) startProvider(ctx context.Context, p *provider) {
level.Debug(m.logger).Log("msg", "Starting provider", "provider", p.name, "subs", fmt.Sprintf("%v", p.subs))
ctx, cancel := context.WithCancel(ctx)
updates := make(chan []*targetgroup.Group)
m.discoverCancel = append(m.discoverCancel, cancel)
go p.d.Run(ctx, updates)
go m.updater(ctx, p, updates)
}
func (m *Manager) updater(ctx context.Context, p *provider, updates chan []*targetgroup.Group) {
for {
select {
case <-ctx.Done():
return
case tgs, ok := <-updates:
receivedUpdates.WithLabelValues(m.name).Inc()
if !ok {
level.Debug(m.logger).Log("msg", "Discoverer channel closed", "provider", p.name)
return
}
for _, s := range p.subs {
m.updateGroup(poolKey{setName: s, provider: p.name}, tgs)
}
select {
case m.triggerSend <- struct{}{}:
default:
}
}
}
}
func (m *Manager) sender() {
ticker := time.NewTicker(m.updatert)
defer ticker.Stop()
for {
select {
case <-m.ctx.Done():
return
case <-ticker.C: // Some discoverers send updates too often so we throttle these with the ticker.
select {
case <-m.triggerSend:
sentUpdates.WithLabelValues(m.name).Inc()
select {
case m.syncCh <- m.allGroups():
default:
delayedUpdates.WithLabelValues(m.name).Inc()
level.Debug(m.logger).Log("msg", "Discovery receiver's channel was full so will retry the next cycle")
select {
case m.triggerSend <- struct{}{}:
default:
}
}
default:
}
}
}
}
func (m *Manager) cancelDiscoverers() {
for _, c := range m.discoverCancel {
c()
}
}
func (m *Manager) updateGroup(poolKey poolKey, tgs []*targetgroup.Group) {
m.mtx.Lock()
defer m.mtx.Unlock()
if _, ok := m.targets[poolKey]; !ok {
m.targets[poolKey] = make(map[string]*targetgroup.Group)
}
for _, tg := range tgs {
if tg != nil { // Some Discoverers send nil target group so need to check for it to avoid panics.
m.targets[poolKey][tg.Source] = tg
}
}
}
func (m *Manager) allGroups() map[string][]*targetgroup.Group {
m.mtx.RLock()
defer m.mtx.RUnlock()
tSets := map[string][]*targetgroup.Group{}
n := map[string]int{}
for pkey, tsets := range m.targets {
for _, tg := range tsets {
// Even if the target group 'tg' is empty we still need to send it to the 'Scrape manager'
// to signal that it needs to stop all scrape loops for this target set.
tSets[pkey.setName] = append(tSets[pkey.setName], tg)
n[pkey.setName] += len(tg.Targets)
}
}
for setName, v := range n {
discoveredTargets.WithLabelValues(m.name, setName).Set(float64(v))
}
return tSets
}
// registerProviders returns a number of failed SD config.
func (m *Manager) registerProviders(cfgs discovery.Configs, setName string) int {
var (
failed int
added bool
)
add := func(cfg discovery.Config) {
for _, p := range m.providers {
if reflect.DeepEqual(cfg, p.config) {
p.subs = append(p.subs, setName)
added = true
return
}
}
typ := cfg.Name()
d, err := cfg.NewDiscoverer(discovery.DiscovererOptions{
Logger: log.With(m.logger, "discovery", typ),
})
if err != nil {
level.Error(m.logger).Log("msg", "Cannot create service discovery", "err", err, "type", typ)
failed++
return
}
m.providers = append(m.providers, &provider{
name: fmt.Sprintf("%s/%d", typ, len(m.providers)),
d: d,
config: cfg,
subs: []string{setName},
})
added = true
}
for _, cfg := range cfgs {
add(cfg)
}
if !added {
// Add an empty target group to force the refresh of the corresponding
// scrape pool and to notify the receiver that this target set has no
// current targets.
// It can happen because the combined set of SD configurations is empty
// or because we fail to instantiate all the SD configurations.
add(discovery.StaticConfig{{}})
}
return failed
}
// StaticProvider holds a list of target groups that never change.
type StaticProvider struct {
TargetGroups []*targetgroup.Group
}
// Run implements the Worker interface.
func (sd *StaticProvider) Run(ctx context.Context, ch chan<- []*targetgroup.Group) {
// We still have to consider that the consumer exits right away in which case
// the context will be canceled.
select {
case ch <- sd.TargetGroups:
case <-ctx.Done():
}
close(ch)
}

File diff suppressed because it is too large Load Diff

View File

@ -0,0 +1,259 @@
// Copyright 2020 The Prometheus Authors
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
package legacymanager
import (
"fmt"
"reflect"
"sort"
"strconv"
"strings"
"sync"
"gopkg.in/yaml.v2"
"github.com/prometheus/prometheus/discovery"
"github.com/prometheus/prometheus/discovery/targetgroup"
)
const (
configFieldPrefix = "AUTO_DISCOVERY_"
staticConfigsKey = "static_configs"
staticConfigsFieldName = configFieldPrefix + staticConfigsKey
)
var (
configNames = make(map[string]discovery.Config)
configFieldNames = make(map[reflect.Type]string)
configFields []reflect.StructField
configTypesMu sync.Mutex
configTypes = make(map[reflect.Type]reflect.Type)
emptyStructType = reflect.TypeOf(struct{}{})
configsType = reflect.TypeOf(discovery.Configs{})
)
// RegisterConfig registers the given Config type for YAML marshaling and unmarshaling.
func RegisterConfig(config discovery.Config) {
registerConfig(config.Name()+"_sd_configs", reflect.TypeOf(config), config)
}
func init() {
// N.B.: static_configs is the only Config type implemented by default.
// All other types are registered at init by their implementing packages.
elemTyp := reflect.TypeOf(&targetgroup.Group{})
registerConfig(staticConfigsKey, elemTyp, discovery.StaticConfig{})
}
func registerConfig(yamlKey string, elemType reflect.Type, config discovery.Config) {
name := config.Name()
if _, ok := configNames[name]; ok {
panic(fmt.Sprintf("discovery: Config named %q is already registered", name))
}
configNames[name] = config
fieldName := configFieldPrefix + yamlKey // Field must be exported.
configFieldNames[elemType] = fieldName
// Insert fields in sorted order.
i := sort.Search(len(configFields), func(k int) bool {
return fieldName < configFields[k].Name
})
configFields = append(configFields, reflect.StructField{}) // Add empty field at end.
copy(configFields[i+1:], configFields[i:]) // Shift fields to the right.
configFields[i] = reflect.StructField{ // Write new field in place.
Name: fieldName,
Type: reflect.SliceOf(elemType),
Tag: reflect.StructTag(`yaml:"` + yamlKey + `,omitempty"`),
}
}
func getConfigType(out reflect.Type) reflect.Type {
configTypesMu.Lock()
defer configTypesMu.Unlock()
if typ, ok := configTypes[out]; ok {
return typ
}
// Initial exported fields map one-to-one.
var fields []reflect.StructField
for i, n := 0, out.NumField(); i < n; i++ {
switch field := out.Field(i); {
case field.PkgPath == "" && field.Type != configsType:
fields = append(fields, field)
default:
fields = append(fields, reflect.StructField{
Name: "_" + field.Name, // Field must be unexported.
PkgPath: out.PkgPath(),
Type: emptyStructType,
})
}
}
// Append extra config fields on the end.
fields = append(fields, configFields...)
typ := reflect.StructOf(fields)
configTypes[out] = typ
return typ
}
// UnmarshalYAMLWithInlineConfigs helps implement yaml.Unmarshal for structs
// that have a Configs field that should be inlined.
func UnmarshalYAMLWithInlineConfigs(out interface{}, unmarshal func(interface{}) error) error {
outVal := reflect.ValueOf(out)
if outVal.Kind() != reflect.Ptr {
return fmt.Errorf("discovery: can only unmarshal into a struct pointer: %T", out)
}
outVal = outVal.Elem()
if outVal.Kind() != reflect.Struct {
return fmt.Errorf("discovery: can only unmarshal into a struct pointer: %T", out)
}
outTyp := outVal.Type()
cfgTyp := getConfigType(outTyp)
cfgPtr := reflect.New(cfgTyp)
cfgVal := cfgPtr.Elem()
// Copy shared fields (defaults) to dynamic value.
var configs *discovery.Configs
for i, n := 0, outVal.NumField(); i < n; i++ {
if outTyp.Field(i).Type == configsType {
configs = outVal.Field(i).Addr().Interface().(*discovery.Configs)
continue
}
if cfgTyp.Field(i).PkgPath != "" {
continue // Field is unexported: ignore.
}
cfgVal.Field(i).Set(outVal.Field(i))
}
if configs == nil {
return fmt.Errorf("discovery: Configs field not found in type: %T", out)
}
// Unmarshal into dynamic value.
if err := unmarshal(cfgPtr.Interface()); err != nil {
return replaceYAMLTypeError(err, cfgTyp, outTyp)
}
// Copy shared fields from dynamic value.
for i, n := 0, outVal.NumField(); i < n; i++ {
if cfgTyp.Field(i).PkgPath != "" {
continue // Field is unexported: ignore.
}
outVal.Field(i).Set(cfgVal.Field(i))
}
var err error
*configs, err = readConfigs(cfgVal, outVal.NumField())
return err
}
func readConfigs(structVal reflect.Value, startField int) (discovery.Configs, error) {
var (
configs discovery.Configs
targets []*targetgroup.Group
)
for i, n := startField, structVal.NumField(); i < n; i++ {
field := structVal.Field(i)
if field.Kind() != reflect.Slice {
panic("discovery: internal error: field is not a slice")
}
for k := 0; k < field.Len(); k++ {
val := field.Index(k)
if val.IsZero() || (val.Kind() == reflect.Ptr && val.Elem().IsZero()) {
key := configFieldNames[field.Type().Elem()]
key = strings.TrimPrefix(key, configFieldPrefix)
return nil, fmt.Errorf("empty or null section in %s", key)
}
switch c := val.Interface().(type) {
case *targetgroup.Group:
// Add index to the static config target groups for unique identification
// within scrape pool.
c.Source = strconv.Itoa(len(targets))
// Coalesce multiple static configs into a single static config.
targets = append(targets, c)
case discovery.Config:
configs = append(configs, c)
default:
panic("discovery: internal error: slice element is not a Config")
}
}
}
if len(targets) > 0 {
configs = append(configs, discovery.StaticConfig(targets))
}
return configs, nil
}
// MarshalYAMLWithInlineConfigs helps implement yaml.Marshal for structs
// that have a Configs field that should be inlined.
func MarshalYAMLWithInlineConfigs(in interface{}) (interface{}, error) {
inVal := reflect.ValueOf(in)
for inVal.Kind() == reflect.Ptr {
inVal = inVal.Elem()
}
inTyp := inVal.Type()
cfgTyp := getConfigType(inTyp)
cfgPtr := reflect.New(cfgTyp)
cfgVal := cfgPtr.Elem()
// Copy shared fields to dynamic value.
var configs *discovery.Configs
for i, n := 0, inTyp.NumField(); i < n; i++ {
if inTyp.Field(i).Type == configsType {
configs = inVal.Field(i).Addr().Interface().(*discovery.Configs)
}
if cfgTyp.Field(i).PkgPath != "" {
continue // Field is unexported: ignore.
}
cfgVal.Field(i).Set(inVal.Field(i))
}
if configs == nil {
return nil, fmt.Errorf("discovery: Configs field not found in type: %T", in)
}
if err := writeConfigs(cfgVal, *configs); err != nil {
return nil, err
}
return cfgPtr.Interface(), nil
}
func writeConfigs(structVal reflect.Value, configs discovery.Configs) error {
targets := structVal.FieldByName(staticConfigsFieldName).Addr().Interface().(*[]*targetgroup.Group)
for _, c := range configs {
if sc, ok := c.(discovery.StaticConfig); ok {
*targets = append(*targets, sc...)
continue
}
fieldName, ok := configFieldNames[reflect.TypeOf(c)]
if !ok {
return fmt.Errorf("discovery: cannot marshal unregistered Config type: %T", c)
}
field := structVal.FieldByName(fieldName)
field.Set(reflect.Append(field, reflect.ValueOf(c)))
}
return nil
}
func replaceYAMLTypeError(err error, oldTyp, newTyp reflect.Type) error {
if e, ok := err.(*yaml.TypeError); ok {
oldStr := oldTyp.String()
newStr := newTyp.String()
for i, s := range e.Errors {
e.Errors[i] = strings.Replace(s, oldStr, newStr, -1)
}
}
return err
}

View File

@ -65,7 +65,7 @@ var (
)
)
func init() {
func RegisterMetrics() {
prometheus.MustRegister(failedConfigs, discoveredTargets, receivedUpdates, delayedUpdates, sentUpdates)
}

View File

@ -1,9 +1,9 @@
---
title: Feature Flags
title: Feature flags
sort_rank: 11
---
# Feature Flags
# Feature flags
Here is a list of features that are disabled by default since they are breaking changes or are considered experimental.
Their behaviour can change in future releases which will be communicated via the [release changelog](https://github.com/prometheus/prometheus/blob/main/CHANGELOG.md).
@ -46,7 +46,7 @@ More details can be found [here](querying/basics.md#offset-modifier).
The remote write receiver allows Prometheus to accept remote write requests from other Prometheus servers. More details can be found [here](storage.md#overview).
## Exemplars Storage
## Exemplars storage
`--enable-feature=exemplar-storage`
@ -54,7 +54,7 @@ The remote write receiver allows Prometheus to accept remote write requests from
Exemplar storage is implemented as a fixed size circular buffer that stores exemplars in memory for all series. Enabling this feature will enable the storage of exemplars scraped by Prometheus. The flag `storage.exemplars.exemplars-limit` can be used to control the size of circular buffer by # of exemplars. An exemplar with just a `traceID=<jaeger-trace-id>` uses roughly 100 bytes of memory via the in-memory exemplar storage. If the exemplar storage is enabled, we will also append the exemplars to WAL for local persistence (for WAL duration).
## Memory Snapshot on Shutdown
## Memory snapshot on shutdown
`--enable-feature=memory-snapshot-on-shutdown`
@ -62,7 +62,7 @@ This takes the snapshot of the chunks that are in memory along with the series i
it on disk. This will reduce the startup time since the memory state can be restored with this snapshot and m-mapped
chunks without the need of WAL replay.
## Extra Scrape Metrics
## Extra scrape metrics
`--enable-feature=extra-scrape-metrics`
@ -71,3 +71,17 @@ When enabled, for each instance scrape, Prometheus stores a sample in the follow
- `scrape_timeout_seconds`. The configured `scrape_timeout` for a target. This allows you to measure each target to find out how close they are to timing out with `scrape_duration_seconds / scrape_timeout_seconds`.
- `scrape_sample_limit`. The configured `sample_limit` for a target. This allows you to measure each target
to find out how close they are to reaching the limit with `scrape_samples_post_metric_relabeling / scrape_sample_limit`. Note that `scrape_sample_limit` can be zero if there is no limit configured, which means that the query above can return `+Inf` for targets with no limit (as we divide by zero). If you want to query only for targets that do have a sample limit use this query: `scrape_samples_post_metric_relabeling / (scrape_sample_limit > 0)`.
## New service discovery manager
`--enable-feature=new-service-discovery-manager`
When enabled, Prometheus uses a new service discovery manager that does not
restart unchanged discoveries upon reloading. This makes reloads faster and reduces
pressure on service discoveries' sources.
Users are encouraged to test the new service discovery manager and report any
issues upstream.
In future releases, this new service discovery manager will become the default and
this feature flag will be ignored.