-
Notifications
You must be signed in to change notification settings - Fork 810
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Add compactor auto-forget from ring on unhealthy #6563
base: master
Are you sure you want to change the base?
Changes from 6 commits
67ac0ba
c7a4f68
fc66e3d
581310d
6dca9e9
0a90a81
788fcba
8262457
6a187fd
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,11 @@ | ||
package compactor | ||
|
||
import ( | ||
"context" | ||
|
||
"github.com/cortexproject/cortex/pkg/ring" | ||
) | ||
|
||
func (c *Compactor) OnRingInstanceHeartbeat(_ *ring.Lifecycler, ctx context.Context) { | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I would do similar to the basicLifecycler. Create the generic delegate for the ring and make compactor use it. |
||
c.ring.AutoForgetUnhealthy(ctx) | ||
} |
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -27,6 +27,12 @@ var ( | |
errInvalidTokensGeneratorStrategy = errors.New("invalid token generator strategy") | ||
) | ||
|
||
type LifecyclerDelegate interface { | ||
// OnRingInstanceHeartbeat is called while the instance is updating its heartbeat | ||
// in the ring. | ||
OnRingInstanceHeartbeat(lifecycler *Lifecycler, ctx context.Context) | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I'd love to see that the same function parameter to be reused. The current interface is a bit weird and we are not using the lifecycler at all in the compactor implementation |
||
} | ||
|
||
// LifecyclerConfig is the config to build a Lifecycler. | ||
type LifecyclerConfig struct { | ||
RingConfig Config `yaml:"ring"` | ||
|
@@ -108,6 +114,7 @@ type Lifecycler struct { | |
cfg LifecyclerConfig | ||
flushTransferer FlushTransferer | ||
KVStore kv.Client | ||
delegate LifecyclerDelegate | ||
|
||
actorChan chan func() | ||
autojoinChan chan struct{} | ||
|
@@ -150,6 +157,22 @@ type Lifecycler struct { | |
tg TokenGenerator | ||
} | ||
|
||
func NewLifecyclerWithDelegate( | ||
cfg LifecyclerConfig, | ||
flushTransferer FlushTransferer, | ||
ringName, ringKey string, | ||
autoJoinOnStartup, flushOnShutdown bool, | ||
logger log.Logger, | ||
reg prometheus.Registerer, | ||
delegate LifecyclerDelegate, | ||
) (*Lifecycler, error) { | ||
l, err := NewLifecycler(cfg, flushTransferer, ringName, ringKey, autoJoinOnStartup, flushOnShutdown, logger, reg) | ||
if l != nil { | ||
l.delegate = delegate | ||
} | ||
return l, err | ||
} | ||
|
||
// NewLifecycler creates new Lifecycler. It must be started via StartAsync. | ||
func NewLifecycler( | ||
cfg LifecyclerConfig, | ||
|
@@ -601,6 +624,9 @@ func (i *Lifecycler) heartbeat(ctx context.Context) { | |
i.lifecyclerMetrics.consulHeartbeats.Inc() | ||
ctx, cancel := context.WithTimeout(ctx, i.cfg.HeartbeatPeriod) | ||
defer cancel() | ||
if i.delegate != nil { | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Instead of explicitly checking |
||
i.delegate.OnRingInstanceHeartbeat(i, ctx) | ||
} | ||
if err := i.updateConsul(ctx); err != nil { | ||
level.Error(i.logger).Log("msg", "failed to write to the KV store, sleeping", "ring", i.RingName, "err", err) | ||
} | ||
|
Original file line number | Diff line number | Diff line change | ||
---|---|---|---|---|
|
@@ -142,12 +142,13 @@ var ( | |||
|
||||
// Config for a Ring | ||||
type Config struct { | ||||
KVStore kv.Config `yaml:"kvstore"` | ||||
HeartbeatTimeout time.Duration `yaml:"heartbeat_timeout"` | ||||
ReplicationFactor int `yaml:"replication_factor"` | ||||
ZoneAwarenessEnabled bool `yaml:"zone_awareness_enabled"` | ||||
ExcludedZones flagext.StringSliceCSV `yaml:"excluded_zones"` | ||||
DetailedMetricsEnabled bool `yaml:"detailed_metrics_enabled"` | ||||
KVStore kv.Config `yaml:"kvstore"` | ||||
HeartbeatTimeout time.Duration `yaml:"heartbeat_timeout"` | ||||
ReplicationFactor int `yaml:"replication_factor"` | ||||
ZoneAwarenessEnabled bool `yaml:"zone_awareness_enabled"` | ||||
ExcludedZones flagext.StringSliceCSV `yaml:"excluded_zones"` | ||||
DetailedMetricsEnabled bool `yaml:"detailed_metrics_enabled"` | ||||
AutoForgetUnhealthyPeriods int `yaml:"auto_forget_unhealthy_periods"` | ||||
|
||||
// Whether the shuffle-sharding subring cache is disabled. This option is set | ||||
// internally and never exposed to the user. | ||||
|
@@ -168,6 +169,7 @@ func (cfg *Config) RegisterFlagsWithPrefix(prefix string, f *flag.FlagSet) { | |||
f.IntVar(&cfg.ReplicationFactor, prefix+"distributor.replication-factor", 3, "The number of ingesters to write to and read from.") | ||||
f.BoolVar(&cfg.ZoneAwarenessEnabled, prefix+"distributor.zone-awareness-enabled", false, "True to enable the zone-awareness and replicate ingested samples across different availability zones.") | ||||
f.Var(&cfg.ExcludedZones, prefix+"distributor.excluded-zones", "Comma-separated list of zones to exclude from the ring. Instances in excluded zones will be filtered out from the ring.") | ||||
f.IntVar(&cfg.AutoForgetUnhealthyPeriods, prefix+"ring.auto-forget-unhealthy-periods", -1, "The number of heartbeat periods to occur after an ingester becomes unhealthy before it is forgotten from the ring. -1 to disable") | ||||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Is this flag needed? Is it used to auto forget Ingesters? I don't know if this is what we want to do in this PR at least |
||||
} | ||||
|
||||
type instanceInfo struct { | ||||
|
@@ -213,6 +215,7 @@ type Ring struct { | |||
numTokensGaugeVec *prometheus.GaugeVec | ||||
oldestTimestampGaugeVec *prometheus.GaugeVec | ||||
reportedOwners map[string]struct{} | ||||
unhealthyPeriodsCount map[string]int | ||||
|
||||
logger log.Logger | ||||
} | ||||
|
@@ -277,7 +280,8 @@ func NewWithStoreClientAndStrategy(cfg Config, name, key string, store kv.Client | |||
Help: "Timestamp of the oldest member in the ring.", | ||||
ConstLabels: map[string]string{"name": name}}, | ||||
[]string{"state"}), | ||||
logger: logger, | ||||
logger: logger, | ||||
unhealthyPeriodsCount: make(map[string]int), | ||||
} | ||||
|
||||
r.Service = services.NewBasicService(r.starting, r.loop, nil).WithName(fmt.Sprintf("%s ring client", name)) | ||||
|
@@ -655,6 +659,43 @@ func (r *Ring) countTokens() (map[string]uint32, map[string]int64) { | |||
return numTokens, owned | ||||
} | ||||
|
||||
// autoForgetUnhealthy forgets unhealthy ingesters from the ring. | ||||
func (r *Ring) AutoForgetUnhealthy(ctx context.Context) { | ||||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Instead of adding There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. +1 it should be similar to how cortex/pkg/ring/basic_lifecycler_delegates.go Line 143 in de5cfe1
works There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. After talking to @eeldaly offline I realized that the parameter is a bit different so probably hard to add the same handler. This seems cleaner to me. We can switch
|
||||
if r.cfg.AutoForgetUnhealthyPeriods < 0 { | ||||
return | ||||
} | ||||
|
||||
//remove counts for ingesters no longer in the ring | ||||
for id := range r.unhealthyPeriodsCount { | ||||
if _, ok := r.ringDesc.Ingesters[id]; !ok { | ||||
delete(r.unhealthyPeriodsCount, id) | ||||
} | ||||
} | ||||
|
||||
//update counts for ingesters in the ring and forget ingesters | ||||
for id, instance := range r.ringDesc.Ingesters { | ||||
if r.IsHealthy(&instance, Reporting, r.KVClient.LastUpdateTime(r.key)) { | ||||
delete(r.unhealthyPeriodsCount, id) | ||||
} else { | ||||
if _, ok := r.unhealthyPeriodsCount[id]; !ok { | ||||
r.unhealthyPeriodsCount[id] = 0 | ||||
} | ||||
r.unhealthyPeriodsCount[id] += 1 | ||||
|
||||
if r.unhealthyPeriodsCount[id] > r.cfg.AutoForgetUnhealthyPeriods { | ||||
unregister := func(in interface{}) (out interface{}, retry bool, err error) { | ||||
ringDesc := in.(*Desc) | ||||
ringDesc.RemoveIngester(id) | ||||
level.Info(r.logger).Log("msg", "Forgetting from ring", id) | ||||
return ringDesc, true, nil | ||||
} | ||||
r.KVClient.CAS(ctx, r.key, unregister) | ||||
delete(r.unhealthyPeriodsCount, id) | ||||
} | ||||
} | ||||
} | ||||
} | ||||
|
||||
// updateRingMetrics updates ring metrics. Caller must be holding the Write lock! | ||||
func (r *Ring) updateRingMetrics(compareResult CompareResult) { | ||||
if compareResult == Equal { | ||||
|
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
This needs to add under
## master / unreleased