Skip to content

Commit c56694c

Browse files
authored
config: make tidb_enable_stats_owner controlling the stats owner (#55592)
close #55989
1 parent c7fde05 commit c56694c

File tree

7 files changed

+83
-24
lines changed

7 files changed

+83
-24
lines changed

pkg/config/config.go

Lines changed: 5 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -551,9 +551,10 @@ type Instance struct {
551551
PluginDir string `toml:"plugin_dir" json:"plugin_dir"`
552552
PluginLoad string `toml:"plugin_load" json:"plugin_load"`
553553
// MaxConnections is the maximum permitted number of simultaneous client connections.
554-
MaxConnections uint32 `toml:"max_connections" json:"max_connections"`
555-
TiDBEnableDDL AtomicBool `toml:"tidb_enable_ddl" json:"tidb_enable_ddl"`
556-
TiDBRCReadCheckTS bool `toml:"tidb_rc_read_check_ts" json:"tidb_rc_read_check_ts"`
554+
MaxConnections uint32 `toml:"max_connections" json:"max_connections"`
555+
TiDBEnableDDL AtomicBool `toml:"tidb_enable_ddl" json:"tidb_enable_ddl"`
556+
TiDBEnableStatsOwner AtomicBool `toml:"tidb_enable_stats_owner" json:"tidb_enable_stats_owner"`
557+
TiDBRCReadCheckTS bool `toml:"tidb_rc_read_check_ts" json:"tidb_rc_read_check_ts"`
557558
// TiDBServiceScope indicates the role for tidb for distributed task framework.
558559
TiDBServiceScope string `toml:"tidb_service_scope" json:"tidb_service_scope"`
559560
}
@@ -964,6 +965,7 @@ var defaultConf = Config{
964965
PluginLoad: "",
965966
MaxConnections: 0,
966967
TiDBEnableDDL: *NewAtomicBool(true),
968+
TiDBEnableStatsOwner: *NewAtomicBool(true),
967969
TiDBRCReadCheckTS: false,
968970
TiDBServiceScope: "",
969971
},

pkg/config/config_test.go

Lines changed: 2 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -1066,7 +1066,7 @@ func TestConflictInstanceConfig(t *testing.T) {
10661066
_, err = f.WriteString("check-mb4-value-in-utf8 = true \nrun-ddl = true \n" +
10671067
"[log] \nenable-slow-log = true \n" +
10681068
"[performance] \nforce-priority = \"NO_PRIORITY\"\n" +
1069-
"[instance] \ntidb_check_mb4_value_in_utf8 = false \ntidb_enable_slow_log = false \ntidb_force_priority = \"LOW_PRIORITY\"\ntidb_enable_ddl = false")
1069+
"[instance] \ntidb_check_mb4_value_in_utf8 = false \ntidb_enable_slow_log = false \ntidb_force_priority = \"LOW_PRIORITY\"\ntidb_enable_ddl = false\ntidb_enable_stats_owner = false")
10701070
require.NoError(t, err)
10711071
require.NoError(t, f.Sync())
10721072
err = conf.Load(configFile)
@@ -1080,6 +1080,7 @@ func TestConflictInstanceConfig(t *testing.T) {
10801080
require.Equal(t, "LOW_PRIORITY", conf.Instance.ForcePriority)
10811081
require.Equal(t, true, conf.RunDDL)
10821082
require.Equal(t, false, conf.Instance.TiDBEnableDDL.Load())
1083+
require.Equal(t, false, conf.Instance.TiDBEnableStatsOwner.Load())
10831084
require.Equal(t, 0, len(DeprecatedOptions))
10841085
for _, conflictOption := range ConflictOptions {
10851086
expectedConflictOption, ok := expectedConflictOptions[conflictOption.SectionName]

pkg/domain/domain.go

Lines changed: 42 additions & 18 deletions
Original file line numberDiff line numberDiff line change
@@ -215,6 +215,7 @@ type Domain struct {
215215

216216
instancePlanCache sessionctx.InstancePlanCache // the instance level plan cache
217217

218+
statsOwner owner.Manager
218219
// deferFn is used to release infoschema object lazily during v1 and v2 switch
219220
deferFn
220221
}
@@ -2313,20 +2314,22 @@ func (do *Domain) UpdateTableStatsLoop(ctx, initStatsCtx sessionctx.Context) err
23132314
if do.statsLease >= 0 {
23142315
do.wg.Run(do.loadStatsWorker, "loadStatsWorker")
23152316
}
2316-
owner := do.newOwnerManager(handle.StatsPrompt, handle.StatsOwnerKey)
2317+
variable.EnableStatsOwner = do.enableStatsOwner
2318+
variable.DisableStatsOwner = do.disableStatsOwner
2319+
do.statsOwner = do.newOwnerManager(handle.StatsPrompt, handle.StatsOwnerKey)
23172320
do.wg.Run(func() {
23182321
do.indexUsageWorker()
23192322
}, "indexUsageWorker")
23202323
if do.statsLease <= 0 {
23212324
// For statsLease > 0, `updateStatsWorker` handles the quit of stats owner.
2322-
do.wg.Run(func() { quitStatsOwner(do, owner) }, "quitStatsOwner")
2325+
do.wg.Run(func() { quitStatsOwner(do, do.statsOwner) }, "quitStatsOwner")
23232326
return nil
23242327
}
23252328
do.SetStatsUpdating(true)
23262329
// The stats updated worker doesn't require the stats initialization to be completed.
23272330
// This is because the updated worker's primary responsibilities are to update the change delta and handle DDL operations.
23282331
// These tasks do not interfere with or depend on the initialization process.
2329-
do.wg.Run(func() { do.updateStatsWorker(ctx, owner) }, "updateStatsWorker")
2332+
do.wg.Run(func() { do.updateStatsWorker(ctx) }, "updateStatsWorker")
23302333
do.wg.Run(func() {
23312334
do.handleDDLEvent()
23322335
}, "handleDDLEvent")
@@ -2339,7 +2342,7 @@ func (do *Domain) UpdateTableStatsLoop(ctx, initStatsCtx sessionctx.Context) err
23392342
case <-do.exit: // It may happen that before initStatsDone, tidb receive Ctrl+C
23402343
return
23412344
}
2342-
do.autoAnalyzeWorker(owner)
2345+
do.autoAnalyzeWorker()
23432346
},
23442347
"autoAnalyzeWorker",
23452348
)
@@ -2350,7 +2353,7 @@ func (do *Domain) UpdateTableStatsLoop(ctx, initStatsCtx sessionctx.Context) err
23502353
case <-do.exit: // It may happen that before initStatsDone, tidb receive Ctrl+C
23512354
return
23522355
}
2353-
do.analyzeJobsCleanupWorker(owner)
2356+
do.analyzeJobsCleanupWorker()
23542357
},
23552358
"analyzeJobsCleanupWorker",
23562359
)
@@ -2380,6 +2383,25 @@ func (do *Domain) UpdateTableStatsLoop(ctx, initStatsCtx sessionctx.Context) err
23802383
return nil
23812384
}
23822385

2386+
// enableStatsOwner enables this node to execute stats owner jobs.
2387+
// Since ownerManager.CampaignOwner will start a new goroutine to run ownerManager.campaignLoop,
2388+
// we should make sure that before invoking enableStatsOwner(), stats owner is DISABLE.
2389+
func (do *Domain) enableStatsOwner() error {
2390+
if !do.statsOwner.IsOwner() {
2391+
err := do.statsOwner.CampaignOwner()
2392+
return errors.Trace(err)
2393+
}
2394+
return nil
2395+
}
2396+
2397+
// disableStatsOwner disable this node to execute stats owner.
2398+
// We should make sure that before invoking disableStatsOwner(), stats owner is ENABLE.
2399+
func (do *Domain) disableStatsOwner() error {
2400+
// disable campaign by interrupting campaignLoop
2401+
do.statsOwner.CampaignCancel()
2402+
return nil
2403+
}
2404+
23832405
func quitStatsOwner(do *Domain, mgr owner.Manager) {
23842406
<-do.exit
23852407
mgr.Cancel()
@@ -2404,9 +2426,11 @@ func (do *Domain) newOwnerManager(prompt, ownerKey string) owner.Manager {
24042426
statsOwner = owner.NewOwnerManager(context.Background(), do.etcdClient, prompt, id, ownerKey)
24052427
}
24062428
// TODO: Need to do something when err is not nil.
2407-
err := statsOwner.CampaignOwner()
2408-
if err != nil {
2409-
logutil.BgLogger().Warn("campaign owner failed", zap.Error(err))
2429+
if ownerKey == handle.StatsOwnerKey && config.GetGlobalConfig().Instance.TiDBEnableStatsOwner.Load() {
2430+
err := statsOwner.CampaignOwner()
2431+
if err != nil {
2432+
logutil.BgLogger().Warn("campaign owner failed", zap.Error(err))
2433+
}
24102434
}
24112435
return statsOwner
24122436
}
@@ -2493,15 +2517,15 @@ func (do *Domain) indexUsageWorker() {
24932517
}
24942518
}
24952519

2496-
func (*Domain) updateStatsWorkerExitPreprocessing(statsHandle *handle.Handle, owner owner.Manager) {
2520+
func (do *Domain) updateStatsWorkerExitPreprocessing(statsHandle *handle.Handle) {
24972521
ch := make(chan struct{}, 1)
24982522
timeout, cancel := context.WithTimeout(context.Background(), 10*time.Second)
24992523
defer cancel()
25002524
go func() {
25012525
logutil.BgLogger().Info("updateStatsWorker is going to exit, start to flush stats")
25022526
statsHandle.FlushStats()
25032527
logutil.BgLogger().Info("updateStatsWorker ready to release owner")
2504-
owner.Cancel()
2528+
do.statsOwner.Cancel()
25052529
ch <- struct{}{}
25062530
}()
25072531
select {
@@ -2532,7 +2556,7 @@ func (do *Domain) handleDDLEvent() {
25322556
}
25332557
}
25342558

2535-
func (do *Domain) updateStatsWorker(_ sessionctx.Context, owner owner.Manager) {
2559+
func (do *Domain) updateStatsWorker(_ sessionctx.Context) {
25362560
defer util.Recover(metrics.LabelDomain, "updateStatsWorker", nil, false)
25372561
logutil.BgLogger().Info("updateStatsWorker started.")
25382562
lease := do.statsLease
@@ -2558,15 +2582,15 @@ func (do *Domain) updateStatsWorker(_ sessionctx.Context, owner owner.Manager) {
25582582
for {
25592583
select {
25602584
case <-do.exit:
2561-
do.updateStatsWorkerExitPreprocessing(statsHandle, owner)
2585+
do.updateStatsWorkerExitPreprocessing(statsHandle)
25622586
return
25632587
case <-deltaUpdateTicker.C:
25642588
err := statsHandle.DumpStatsDeltaToKV(false)
25652589
if err != nil {
25662590
logutil.BgLogger().Debug("dump stats delta failed", zap.Error(err))
25672591
}
25682592
case <-gcStatsTicker.C:
2569-
if !owner.IsOwner() {
2593+
if !do.statsOwner.IsOwner() {
25702594
continue
25712595
}
25722596
err := statsHandle.GCStats(do.InfoSchema(), do.GetSchemaLease())
@@ -2587,7 +2611,7 @@ func (do *Domain) updateStatsWorker(_ sessionctx.Context, owner owner.Manager) {
25872611
}
25882612
}
25892613

2590-
func (do *Domain) autoAnalyzeWorker(owner owner.Manager) {
2614+
func (do *Domain) autoAnalyzeWorker() {
25912615
defer util.Recover(metrics.LabelDomain, "autoAnalyzeWorker", nil, false)
25922616
statsHandle := do.StatsHandle()
25932617
analyzeTicker := time.NewTicker(do.statsLease)
@@ -2598,7 +2622,7 @@ func (do *Domain) autoAnalyzeWorker(owner owner.Manager) {
25982622
for {
25992623
select {
26002624
case <-analyzeTicker.C:
2601-
if variable.RunAutoAnalyze.Load() && !do.stopAutoAnalyze.Load() && owner.IsOwner() {
2625+
if variable.RunAutoAnalyze.Load() && !do.stopAutoAnalyze.Load() && do.statsOwner.IsOwner() {
26022626
statsHandle.HandleAutoAnalyze()
26032627
}
26042628
case <-do.exit:
@@ -2621,7 +2645,7 @@ func (do *Domain) autoAnalyzeWorker(owner owner.Manager) {
26212645
// It first retrieves the list of current analyze processes, then removes any analyze job
26222646
// that is not associated with a current process. Additionally, if the current instance is the owner,
26232647
// it also cleans up corrupted analyze jobs on dead instances.
2624-
func (do *Domain) analyzeJobsCleanupWorker(owner owner.Manager) {
2648+
func (do *Domain) analyzeJobsCleanupWorker() {
26252649
defer util.Recover(metrics.LabelDomain, "analyzeJobsCleanupWorker", nil, false)
26262650
// For GC.
26272651
const gcInterval = time.Hour
@@ -2642,7 +2666,7 @@ func (do *Domain) analyzeJobsCleanupWorker(owner owner.Manager) {
26422666
select {
26432667
case <-gcTicker.C:
26442668
// Only the owner should perform this operation.
2645-
if owner.IsOwner() {
2669+
if do.statsOwner.IsOwner() {
26462670
updateTime := time.Now().AddDate(0, 0, -daysToKeep)
26472671
err := statsHandle.DeleteAnalyzeJobs(updateTime)
26482672
if err != nil {
@@ -2666,7 +2690,7 @@ func (do *Domain) analyzeJobsCleanupWorker(owner owner.Manager) {
26662690
logutil.BgLogger().Warn("cleanup analyze jobs on current instance failed", zap.Error(err))
26672691
}
26682692

2669-
if owner.IsOwner() {
2693+
if do.statsOwner.IsOwner() {
26702694
err = statsHandle.CleanupCorruptedAnalyzeJobsOnDeadInstances()
26712695
if err != nil {
26722696
logutil.BgLogger().Warn("cleanup analyze jobs on dead instances failed", zap.Error(err))

pkg/domain/domain_test.go

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -181,8 +181,8 @@ func TestStatWorkRecoverFromPanic(t *testing.T) {
181181
metrics.PanicCounter.Reset()
182182
// Since the stats lease is 0 now, so create a new ticker will panic.
183183
// Test that they can recover from panic correctly.
184-
dom.updateStatsWorker(mock.NewContext(), nil)
185-
dom.autoAnalyzeWorker(nil)
184+
dom.updateStatsWorker(mock.NewContext())
185+
dom.autoAnalyzeWorker()
186186
counter := metrics.PanicCounter.WithLabelValues(metrics.LabelDomain)
187187
pb := &dto.Metric{}
188188
err = counter.Write(pb)

pkg/sessionctx/variable/sysvar.go

Lines changed: 18 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -558,6 +558,22 @@ var defaultSysVars = []*SysVar{
558558
return BoolToOnOff(config.GetGlobalConfig().Instance.TiDBEnableDDL.Load()), nil
559559
},
560560
},
561+
{Scope: ScopeInstance, Name: TiDBEnableStatsOwner, Value: BoolToOnOff(config.GetGlobalConfig().Instance.TiDBEnableStatsOwner.Load()), Type: TypeBool,
562+
SetGlobal: func(_ context.Context, s *SessionVars, val string) error {
563+
oldVal, newVal := config.GetGlobalConfig().Instance.TiDBEnableStatsOwner.Load(), TiDBOptOn(val)
564+
if oldVal != newVal {
565+
err := switchStats(newVal)
566+
if err != nil {
567+
return err
568+
}
569+
config.GetGlobalConfig().Instance.TiDBEnableStatsOwner.Store(newVal)
570+
}
571+
return nil
572+
},
573+
GetGlobal: func(_ context.Context, s *SessionVars) (string, error) {
574+
return BoolToOnOff(config.GetGlobalConfig().Instance.TiDBEnableStatsOwner.Load()), nil
575+
},
576+
},
561577
{Scope: ScopeInstance, Name: TiDBRCReadCheckTS, Value: BoolToOnOff(DefRCReadCheckTS), Type: TypeBool, SetGlobal: func(_ context.Context, s *SessionVars, val string) error {
562578
EnableRCReadCheckTS.Store(TiDBOptOn(val))
563579
return nil
@@ -3576,6 +3592,8 @@ const (
35763592
PluginLoad = "plugin_load"
35773593
// TiDBEnableDDL indicates whether the tidb-server campaigns the DDL owner,
35783594
TiDBEnableDDL = "tidb_enable_ddl"
3595+
// TiDBEnableStatsOwner indicates whether the tidb-server campaigns the Stats owner,
3596+
TiDBEnableStatsOwner = "tidb_enable_stats_owner"
35793597
// Port is the name for 'port' system variable.
35803598
Port = "port"
35813599
// DataDir is the name for 'datadir' system variable.

pkg/sessionctx/variable/tidb_vars.go

Lines changed: 4 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -1716,6 +1716,10 @@ var (
17161716
SetLowResolutionTSOUpdateInterval func(interval time.Duration) error = nil
17171717
// ChangeSchemaCacheSize is called when tidb_schema_cache_size is changed.
17181718
ChangeSchemaCacheSize func(ctx context.Context, size uint64) error
1719+
// EnableStatsOwner is the func registered by stats to enable running stats in this instance.
1720+
EnableStatsOwner func() error = nil
1721+
// DisableStatsOwner is the func registered by stats to disable running stats in this instance.
1722+
DisableStatsOwner func() error = nil
17191723
)
17201724

17211725
// Hooks functions for Cluster Resource Control.

pkg/sessionctx/variable/varsutil.go

Lines changed: 10 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -511,6 +511,16 @@ func switchDDL(on bool) error {
511511
return nil
512512
}
513513

514+
// switchStats turns on/off stats owner in an instance
515+
func switchStats(on bool) error {
516+
if on && EnableStatsOwner != nil {
517+
return EnableStatsOwner()
518+
} else if !on && DisableStatsOwner != nil {
519+
return DisableStatsOwner()
520+
}
521+
return nil
522+
}
523+
514524
func collectAllowFuncName4ExpressionIndex() string {
515525
str := make([]string, 0, len(GAFunction4ExpressionIndex))
516526
for funcName := range GAFunction4ExpressionIndex {

0 commit comments

Comments
 (0)