From 04b470f89308f48ca8e640de1bb84a22c81a4acb Mon Sep 17 00:00:00 2001 From: Alex Le Date: Thu, 17 Oct 2024 16:03:34 -0700 Subject: [PATCH 1/6] Make sure ingester is active when joining the ring Signed-off-by: Alex Le --- pkg/ingester/ingester.go | 16 ++++++++-------- pkg/ring/lifecycler.go | 3 +++ 2 files changed, 11 insertions(+), 8 deletions(-) diff --git a/pkg/ingester/ingester.go b/pkg/ingester/ingester.go index 7093138802b..6b7ca55a1ac 100644 --- a/pkg/ingester/ingester.go +++ b/pkg/ingester/ingester.go @@ -713,7 +713,7 @@ func New(cfg Config, limits *validation.Overrides, registerer prometheus.Registe }, i.getOldestUnshippedBlockMetric) } - i.lifecycler, err = ring.NewLifecycler(cfg.LifecyclerConfig, i, "ingester", RingKey, false, cfg.BlocksStorageConfig.TSDB.FlushBlocksOnShutdown, logger, prometheus.WrapRegistererWithPrefix("cortex_", registerer)) + i.lifecycler, err = ring.NewLifecycler(cfg.LifecyclerConfig, i, "ingester", RingKey, true, cfg.BlocksStorageConfig.TSDB.FlushBlocksOnShutdown, logger, prometheus.WrapRegistererWithPrefix("cortex_", registerer)) if err != nil { return nil, err } @@ -799,6 +799,13 @@ func (i *Ingester) startingV2ForFlusher(ctx context.Context) error { } func (i *Ingester) starting(ctx context.Context) error { + if err := i.openExistingTSDB(ctx); err != nil { + // Try to rollback and close opened TSDBs before halting the ingester. + i.closeAllTSDB() + + return errors.Wrap(err, "opening existing TSDBs") + } + // Important: we want to keep lifecycler running until we ask it to stop, so we need to give it independent context if err := i.lifecycler.StartAsync(context.Background()); err != nil { return errors.Wrap(err, "failed to start lifecycler") @@ -807,13 +814,6 @@ func (i *Ingester) starting(ctx context.Context) error { return errors.Wrap(err, "failed to start lifecycler") } - if err := i.openExistingTSDB(ctx); err != nil { - // Try to rollback and close opened TSDBs before halting the ingester. - i.closeAllTSDB() - - return errors.Wrap(err, "opening existing TSDBs") - } - i.lifecycler.Join() // let's start the rest of subservices via manager diff --git a/pkg/ring/lifecycler.go b/pkg/ring/lifecycler.go index e9b9970e1a5..ca4751d0099 100644 --- a/pkg/ring/lifecycler.go +++ b/pkg/ring/lifecycler.go @@ -453,6 +453,8 @@ func (i *Lifecycler) loop(ctx context.Context) error { return errors.Wrapf(err, "failed to join the ring %s", i.RingName) } + level.Info(i.logger).Log("msg", "finished init ring", "ring", i.RingName, "state", i.GetState()) + // We do various period tasks var autoJoinAfter <-chan time.Time var observeChan <-chan time.Time @@ -480,6 +482,7 @@ func (i *Lifecycler) loop(ctx context.Context) error { select { case <-i.autojoinChan: autoJoinAfter = time.After(i.cfg.JoinAfter) + level.Info(i.logger).Log("msg", "will do auto-joining after timeout", "timeout", i.cfg.JoinAfter, "state", i.GetState()) case <-autoJoinAfter: if joined { continue From d6c9cf3d6d0d52dbc51e530f85697e75e268d02a Mon Sep 17 00:00:00 2001 From: Alex Le Date: Tue, 3 Dec 2024 13:49:19 -0800 Subject: [PATCH 2/6] tokens from file would be ignored if instance was not in the ring when starting Signed-off-by: Alex Le --- pkg/ingester/ingester.go | 16 ++++++++-------- pkg/ring/lifecycler.go | 22 +++++++++++++--------- 2 files changed, 21 insertions(+), 17 deletions(-) diff --git a/pkg/ingester/ingester.go b/pkg/ingester/ingester.go index 4adec31e9d3..00dd1337ce1 100644 --- a/pkg/ingester/ingester.go +++ b/pkg/ingester/ingester.go @@ -730,7 +730,7 @@ func New(cfg Config, limits *validation.Overrides, registerer prometheus.Registe }, i.getOldestUnshippedBlockMetric) } - i.lifecycler, err = ring.NewLifecycler(cfg.LifecyclerConfig, i, "ingester", RingKey, true, cfg.BlocksStorageConfig.TSDB.FlushBlocksOnShutdown, logger, prometheus.WrapRegistererWithPrefix("cortex_", registerer)) + i.lifecycler, err = ring.NewLifecycler(cfg.LifecyclerConfig, i, "ingester", RingKey, false, cfg.BlocksStorageConfig.TSDB.FlushBlocksOnShutdown, logger, prometheus.WrapRegistererWithPrefix("cortex_", registerer)) if err != nil { return nil, err } @@ -817,13 +817,6 @@ func (i *Ingester) startingV2ForFlusher(ctx context.Context) error { } func (i *Ingester) starting(ctx context.Context) error { - if err := i.openExistingTSDB(ctx); err != nil { - // Try to rollback and close opened TSDBs before halting the ingester. - i.closeAllTSDB() - - return errors.Wrap(err, "opening existing TSDBs") - } - // Important: we want to keep lifecycler running until we ask it to stop, so we need to give it independent context if err := i.lifecycler.StartAsync(context.Background()); err != nil { return errors.Wrap(err, "failed to start lifecycler") @@ -832,6 +825,13 @@ func (i *Ingester) starting(ctx context.Context) error { return errors.Wrap(err, "failed to start lifecycler") } + if err := i.openExistingTSDB(ctx); err != nil { + // Try to rollback and close opened TSDBs before halting the ingester. + i.closeAllTSDB() + + return errors.Wrap(err, "opening existing TSDBs") + } + i.lifecycler.Join() // let's start the rest of subservices via manager diff --git a/pkg/ring/lifecycler.go b/pkg/ring/lifecycler.go index dd70e00a69f..1a4a826d6de 100644 --- a/pkg/ring/lifecycler.go +++ b/pkg/ring/lifecycler.go @@ -495,8 +495,6 @@ func (i *Lifecycler) loop(ctx context.Context) error { return errors.Wrapf(err, "failed to join the ring %s", i.RingName) } - level.Info(i.logger).Log("msg", "finished init ring", "ring", i.RingName, "state", i.GetState()) - // We do various period tasks var autoJoinAfter <-chan time.Time var observeChan <-chan time.Time @@ -524,7 +522,6 @@ func (i *Lifecycler) loop(ctx context.Context) error { select { case <-i.autojoinChan: autoJoinAfter = time.After(i.cfg.JoinAfter) - level.Info(i.logger).Log("msg", "will do auto-joining after timeout", "timeout", i.cfg.JoinAfter, "state", i.GetState()) case <-autoJoinAfter: if joined { continue @@ -692,13 +689,16 @@ func (i *Lifecycler) initRing(ctx context.Context) error { // We use the tokens from the file only if it does not exist in the ring yet. if len(tokensFromFile) > 0 { - level.Info(i.logger).Log("msg", "adding tokens from file", "num_tokens", len(tokensFromFile)) if len(tokensFromFile) >= i.cfg.NumTokens && i.autoJoinOnStartup { + level.Info(i.logger).Log("msg", "adding tokens from file", "num_tokens", len(tokensFromFile)) i.setState(i.getPreviousState()) + state := i.GetState() + ringDesc.AddIngester(i.ID, i.Addr, i.Zone, tokensFromFile, state, registeredAt) + level.Info(i.logger).Log("msg", "auto join on startup, adding with token and state", "ring", i.RingName, "state", state) + i.setTokens(tokensFromFile) + return ringDesc, true, nil } - ringDesc.AddIngester(i.ID, i.Addr, i.Zone, tokensFromFile, i.GetState(), registeredAt) - i.setTokens(tokensFromFile) - return ringDesc, true, nil + level.Info(i.logger).Log("msg", "ignore tokens from file since autoJoinOnStartup set to false") } // Either we are a new ingester, or consul must have restarted @@ -894,7 +894,9 @@ func (i *Lifecycler) autoJoin(ctx context.Context, targetState InstanceState) er if needTokens == 0 && myTokens.Equals(i.getTokens()) { // Tokens have been verified. No need to change them. - ringDesc.AddIngester(i.ID, i.Addr, i.Zone, i.getTokens(), i.GetState(), i.getRegisteredAt()) + state := i.GetState() + ringDesc.AddIngester(i.ID, i.Addr, i.Zone, i.getTokens(), state, i.getRegisteredAt()) + level.Info(i.logger).Log("msg", "auto joined with existing tokens", "ring", i.RingName, "state", state) return ringDesc, true, nil } @@ -908,7 +910,9 @@ func (i *Lifecycler) autoJoin(ctx context.Context, targetState InstanceState) er sort.Sort(myTokens) i.setTokens(myTokens) - ringDesc.AddIngester(i.ID, i.Addr, i.Zone, i.getTokens(), i.GetState(), i.getRegisteredAt()) + state := i.GetState() + ringDesc.AddIngester(i.ID, i.Addr, i.Zone, i.getTokens(), state, i.getRegisteredAt()) + level.Info(i.logger).Log("msg", "auto joined with new tokens", "ring", i.RingName, "state", state) return ringDesc, true, nil }) From fb1050448621e0bfbad594cba12280c207f5c56f Mon Sep 17 00:00:00 2001 From: Alex Le Date: Tue, 3 Dec 2024 21:11:18 -0800 Subject: [PATCH 3/6] Skip CAS if instance was not in the ring on start and delay heartbeat start time Signed-off-by: Alex Le --- pkg/ring/kv/dynamodb/client.go | 1 + pkg/ring/lifecycler.go | 53 ++++++++++++++++++++++------------ 2 files changed, 36 insertions(+), 18 deletions(-) diff --git a/pkg/ring/kv/dynamodb/client.go b/pkg/ring/kv/dynamodb/client.go index 75fef517b39..71de47f0e5c 100644 --- a/pkg/ring/kv/dynamodb/client.go +++ b/pkg/ring/kv/dynamodb/client.go @@ -199,6 +199,7 @@ func (c *Client) CAS(ctx context.Context, key string, f func(in interface{}) (ou if len(putRequests) == 0 && len(deleteRequests) == 0 { // no change detected, retry + level.Warn(c.logger).Log("msg", "no change detected in ring, retry CAS") bo.Wait() continue } diff --git a/pkg/ring/lifecycler.go b/pkg/ring/lifecycler.go index 1a4a826d6de..1500114781a 100644 --- a/pkg/ring/lifecycler.go +++ b/pkg/ring/lifecycler.go @@ -491,7 +491,8 @@ func (i *Lifecycler) loop(ctx context.Context) error { joined := false // First, see if we exist in the cluster, update our state to match if we do, // and add ourselves (without tokens) if we don't. - if err := i.initRing(context.Background()); err != nil { + addedInRing, err := i.initRing(context.Background()) + if err != nil { return errors.Wrapf(err, "failed to join the ring %s", i.RingName) } @@ -504,18 +505,23 @@ func (i *Lifecycler) loop(ctx context.Context) error { } var heartbeatTickerChan <-chan time.Time - if uint64(i.cfg.HeartbeatPeriod) > 0 { - heartbeatTicker := time.NewTicker(i.cfg.HeartbeatPeriod) - heartbeatTicker.Stop() - // We are jittering for at least half of the time and max the time of the heartbeat. - // If we jitter too soon, we can have problems of concurrency with autoJoin leaving the instance on ACTIVE without tokens - time.AfterFunc(time.Duration(uint64(i.cfg.HeartbeatPeriod/2)+uint64(mathrand.Int63())%uint64(i.cfg.HeartbeatPeriod/2)), func() { - i.heartbeat(ctx) - heartbeatTicker.Reset(i.cfg.HeartbeatPeriod) - }) - defer heartbeatTicker.Stop() - - heartbeatTickerChan = heartbeatTicker.C + startHeartbeat := func() { + if uint64(i.cfg.HeartbeatPeriod) > 0 { + heartbeatTicker := time.NewTicker(i.cfg.HeartbeatPeriod) + heartbeatTicker.Stop() + // We are jittering for at least half of the time and max the time of the heartbeat. + // If we jitter too soon, we can have problems of concurrency with autoJoin leaving the instance on ACTIVE without tokens + time.AfterFunc(time.Duration(uint64(i.cfg.HeartbeatPeriod/2)+uint64(mathrand.Int63())%uint64(i.cfg.HeartbeatPeriod/2)), func() { + i.heartbeat(ctx) + heartbeatTicker.Reset(i.cfg.HeartbeatPeriod) + }) + defer heartbeatTicker.Stop() + + heartbeatTickerChan = heartbeatTicker.C + } + } + if addedInRing { + startHeartbeat() } for { @@ -547,6 +553,10 @@ func (i *Lifecycler) loop(ctx context.Context) error { return errors.Wrapf(err, "failed to pick tokens in the KV store, ring: %s, state: %s", i.RingName, i.getPreviousState()) } } + + if !addedInRing { + startHeartbeat() + } } case <-observeChan: @@ -565,6 +575,10 @@ func (i *Lifecycler) loop(ctx context.Context) error { if err != nil { level.Error(i.logger).Log("msg", "failed to set state", "ring", i.RingName, "state", i.getPreviousState(), "err", err) } + + if !addedInRing { + startHeartbeat() + } } else { level.Info(i.logger).Log("msg", "token verification failed, observing", "ring", i.RingName) // keep observing @@ -653,12 +667,13 @@ heartbeatLoop: // initRing is the first thing we do when we start. It: // - add an ingester entry to the ring // - copies out our state and tokens if they exist -func (i *Lifecycler) initRing(ctx context.Context) error { +func (i *Lifecycler) initRing(ctx context.Context) (bool, error) { var ( ringDesc *Desc tokensFromFile Tokens err error ) + addedInRing := true if i.cfg.TokensFilePath != "" { tokenFile, err := i.loadTokenFile() @@ -689,16 +704,18 @@ func (i *Lifecycler) initRing(ctx context.Context) error { // We use the tokens from the file only if it does not exist in the ring yet. if len(tokensFromFile) > 0 { + level.Info(i.logger).Log("msg", "adding tokens from file", "num_tokens", len(tokensFromFile)) if len(tokensFromFile) >= i.cfg.NumTokens && i.autoJoinOnStartup { - level.Info(i.logger).Log("msg", "adding tokens from file", "num_tokens", len(tokensFromFile)) i.setState(i.getPreviousState()) state := i.GetState() ringDesc.AddIngester(i.ID, i.Addr, i.Zone, tokensFromFile, state, registeredAt) level.Info(i.logger).Log("msg", "auto join on startup, adding with token and state", "ring", i.RingName, "state", state) - i.setTokens(tokensFromFile) return ringDesc, true, nil } - level.Info(i.logger).Log("msg", "ignore tokens from file since autoJoinOnStartup set to false") + i.setTokens(tokensFromFile) + // Do not return ring to CAS call since instance has not been added to ring yet. + addedInRing = false + return nil, true, nil } // Either we are a new ingester, or consul must have restarted @@ -760,7 +777,7 @@ func (i *Lifecycler) initRing(ctx context.Context) error { i.updateCounters(ringDesc) } - return err + return addedInRing, err } func (i *Lifecycler) RenewTokens(ratio float64, ctx context.Context) { From 913e0564f9d841ffad6d70fc940e02e98ad70a0e Mon Sep 17 00:00:00 2001 From: Alex Le Date: Wed, 4 Dec 2024 10:01:55 -0800 Subject: [PATCH 4/6] Fix compactor tests Signed-off-by: Alex Le --- pkg/compactor/compactor_test.go | 2 ++ 1 file changed, 2 insertions(+) diff --git a/pkg/compactor/compactor_test.go b/pkg/compactor/compactor_test.go index 7da85dc8566..09a3929d644 100644 --- a/pkg/compactor/compactor_test.go +++ b/pkg/compactor/compactor_test.go @@ -1043,6 +1043,7 @@ func TestCompactor_ShouldCompactAllUsersOnShardingEnabledButOnlyOneInstanceRunni tsdbPlanner.AssertNumberOfCalls(t, "Plan", 2) assert.ElementsMatch(t, []string{ + `level=info component=compactor msg="auto joined with new tokens" ring=compactor state=ACTIVE`, `level=info component=compactor msg="waiting until compactor is ACTIVE in the ring"`, `level=info component=compactor msg="compactor is ACTIVE in the ring"`, `level=info component=compactor msg="compactor started"`, @@ -1836,6 +1837,7 @@ func TestCompactor_ShouldFailCompactionOnTimeout(t *testing.T) { assert.Equal(t, context.DeadlineExceeded, err) assert.ElementsMatch(t, []string{ + `level=info component=compactor msg="auto joined with new tokens" ring=compactor state=JOINING`, `level=info component=compactor msg="compactor started"`, `level=info component=compactor msg="waiting until compactor is ACTIVE in the ring"`, `level=error component=compactor msg="compactor failed to become ACTIVE in the ring" err="context deadline exceeded"`, From 1b19602cf6ec11b09a5291517344d8e2dd87e747 Mon Sep 17 00:00:00 2001 From: Alex Le Date: Wed, 4 Dec 2024 13:16:59 -0800 Subject: [PATCH 5/6] Fixed existing token not loaded issue and added unit test Signed-off-by: Alex Le --- pkg/ring/lifecycler.go | 9 ++-- pkg/ring/lifecycler_test.go | 102 ++++++++++++++++++++++++++++++++++++ 2 files changed, 108 insertions(+), 3 deletions(-) diff --git a/pkg/ring/lifecycler.go b/pkg/ring/lifecycler.go index 1500114781a..06eb05746a4 100644 --- a/pkg/ring/lifecycler.go +++ b/pkg/ring/lifecycler.go @@ -542,14 +542,14 @@ func (i *Lifecycler) loop(ctx context.Context) error { if i.cfg.ObservePeriod > 0 { // let's observe the ring. By using JOINING state, this ingester will be ignored by LEAVING // ingesters, but we also signal that it is not fully functional yet. - if err := i.autoJoin(context.Background(), JOINING); err != nil { + if err := i.autoJoin(context.Background(), JOINING, addedInRing); err != nil { return errors.Wrapf(err, "failed to pick tokens in the KV store, ring: %s", i.RingName) } level.Info(i.logger).Log("msg", "observing tokens before going ACTIVE", "ring", i.RingName) observeChan = time.After(i.cfg.ObservePeriod) } else { - if err := i.autoJoin(context.Background(), i.getPreviousState()); err != nil { + if err := i.autoJoin(context.Background(), i.getPreviousState(), addedInRing); err != nil { return errors.Wrapf(err, "failed to pick tokens in the KV store, ring: %s, state: %s", i.RingName, i.getPreviousState()) } } @@ -892,7 +892,7 @@ func (i *Lifecycler) compareTokens(fromRing Tokens) bool { } // autoJoin selects random tokens & moves state to targetState -func (i *Lifecycler) autoJoin(ctx context.Context, targetState InstanceState) error { +func (i *Lifecycler) autoJoin(ctx context.Context, targetState InstanceState, alreadyInRing bool) error { var ringDesc *Desc err := i.KVStore.CAS(ctx, i.RingKey, func(in interface{}) (out interface{}, retry bool, err error) { @@ -907,6 +907,9 @@ func (i *Lifecycler) autoJoin(ctx context.Context, targetState InstanceState) er // At this point, we should not have any tokens, and we should be in PENDING state. // Need to make sure we didn't change the num of tokens configured myTokens, _ := ringDesc.TokensFor(i.ID) + if !alreadyInRing { + myTokens = i.getTokens() + } needTokens := i.cfg.NumTokens - len(myTokens) if needTokens == 0 && myTokens.Equals(i.getTokens()) { diff --git a/pkg/ring/lifecycler_test.go b/pkg/ring/lifecycler_test.go index cdd684dd96a..d8d04904e3f 100644 --- a/pkg/ring/lifecycler_test.go +++ b/pkg/ring/lifecycler_test.go @@ -800,6 +800,108 @@ func TestTokenFileOnDisk(t *testing.T) { } } +func TestTokenFileOnDisk_WithoutAutoJoinOnStartup(t *testing.T) { + ringStore, closer := consul.NewInMemoryClient(GetCodec(), log.NewNopLogger(), nil) + t.Cleanup(func() { assert.NoError(t, closer.Close()) }) + + var ringConfig Config + flagext.DefaultValues(&ringConfig) + ringConfig.KVStore.Mock = ringStore + + r, err := New(ringConfig, "ingester", ringKey, log.NewNopLogger(), nil) + require.NoError(t, err) + require.NoError(t, services.StartAndAwaitRunning(context.Background(), r)) + defer services.StopAndAwaitTerminated(context.Background(), r) //nolint:errcheck + + tokenDir := t.TempDir() + + lifecyclerConfig := testLifecyclerConfig(ringConfig, "ing1") + lifecyclerConfig.NumTokens = 512 + lifecyclerConfig.TokensFilePath = tokenDir + "/tokens" + + // Start first ingester. + l1, err := NewLifecycler(lifecyclerConfig, &noopFlushTransferer{}, "ingester", ringKey, false, true, log.NewNopLogger(), nil) + require.NoError(t, err) + require.NoError(t, services.StartAndAwaitRunning(context.Background(), l1)) + + // First ingester joins the ring + l1.Join() + + // Check this ingester joined, is active, and has 512 token. + var expTokens []uint32 + test.Poll(t, 1000*time.Millisecond, true, func() interface{} { + d, err := r.KVClient.Get(context.Background(), ringKey) + require.NoError(t, err) + + desc, ok := d.(*Desc) + if ok { + expTokens = desc.Ingesters["ing1"].Tokens + } + return ok && + len(desc.Ingesters) == 1 && + desc.Ingesters["ing1"].State == ACTIVE && + len(desc.Ingesters["ing1"].Tokens) == 512 + }) + + // Change state from ACTIVE to READONLY + err = l1.ChangeState(context.Background(), READONLY) + require.NoError(t, err) + test.Poll(t, 1000*time.Millisecond, true, func() interface{} { + d, err := r.KVClient.Get(context.Background(), ringKey) + require.NoError(t, err) + + desc, ok := d.(*Desc) + return ok && + desc.Ingesters["ing1"].State == READONLY + }) + + require.NoError(t, services.StopAndAwaitTerminated(context.Background(), l1)) + + // Start new ingester at same token directory. + lifecyclerConfig.ID = "ing2" + l2, err := NewLifecycler(lifecyclerConfig, &noopFlushTransferer{}, "ingester", ringKey, false, true, log.NewNopLogger(), nil) + require.NoError(t, err) + require.NoError(t, services.StartAndAwaitRunning(context.Background(), l2)) + defer services.StopAndAwaitTerminated(context.Background(), l2) //nolint:errcheck + + // Check this ingester should not in the ring before calling Join + test.Poll(t, 1000*time.Millisecond, true, func() interface{} { + d, err := r.KVClient.Get(context.Background(), ringKey) + require.NoError(t, err) + desc, ok := d.(*Desc) + if ok { + _, ingesterInRing := desc.Ingesters["ing2"] + return !ingesterInRing + } + return ok + }) + + // New ingester joins the ring + l2.Join() + + // Check this ingester joined, is in readonly state, and has 512 token. + var actTokens []uint32 + test.Poll(t, 1000*time.Millisecond, true, func() interface{} { + d, err := r.KVClient.Get(context.Background(), ringKey) + require.NoError(t, err) + desc, ok := d.(*Desc) + if ok { + actTokens = desc.Ingesters["ing2"].Tokens + } + return ok && + len(desc.Ingesters) == 1 && + desc.Ingesters["ing2"].State == READONLY && + len(desc.Ingesters["ing2"].Tokens) == 512 + }) + + // Check for same tokens. + sort.Slice(expTokens, func(i, j int) bool { return expTokens[i] < expTokens[j] }) + sort.Slice(actTokens, func(i, j int) bool { return actTokens[i] < actTokens[j] }) + for i := 0; i < 512; i++ { + require.Equal(t, expTokens, actTokens) + } +} + // JoinInLeavingState ensures that if the lifecycler starts up and the ring already has it in a LEAVING state that it still is able to auto join func TestJoinInLeavingState(t *testing.T) { ringStore, closer := consul.NewInMemoryClient(GetCodec(), log.NewNopLogger(), nil) From 3a1145d1aebcf2e5613c0f6291a662cfee4993dd Mon Sep 17 00:00:00 2001 From: Alex Le Date: Thu, 5 Dec 2024 11:46:17 -0800 Subject: [PATCH 6/6] update changelog Signed-off-by: Alex Le --- CHANGELOG.md | 1 + 1 file changed, 1 insertion(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index 1832fc50a14..112aadb3263 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -34,6 +34,7 @@ * [ENHANCEMENT] Ingester: Introduce a new experimental feature for caching expanded postings on the ingester. #6296 * [ENHANCEMENT] Querier/Ruler: Expose `store_gateway_consistency_check_max_attempts` for max retries when querying store gateway in consistency check. #6276 * [ENHANCEMENT] StoreGateway: Add new `cortex_bucket_store_chunk_pool_inuse_bytes` metric to track the usage in chunk pool. #6310 +* [ENHANCEMENT] Ingester: Make sure unregistered ingester joining the ring after WAL replay #6277 * [BUGFIX] Runtime-config: Handle absolute file paths when working directory is not / #6224 ## 1.18.1 2024-10-14