From 62af3aaec2d1df283296b543c5226e77aec451ba Mon Sep 17 00:00:00 2001 From: Weizhen Wang Date: Thu, 2 Dec 2021 17:21:49 +0800 Subject: [PATCH 1/5] config: make EnableSlowLog atomic Signed-off-by: Weizhen Wang --- config/config.go | 51 ++++++++++++++++++++++++++---- config/config_test.go | 2 +- config/config_util_test.go | 2 +- executor/adapter.go | 2 +- session/bench_test.go | 2 +- sessionctx/variable/sysvar.go | 9 ++---- sessionctx/variable/sysvar_test.go | 2 +- 7 files changed, 52 insertions(+), 18 deletions(-) diff --git a/config/config.go b/config/config.go index 43bac8c4c95d7..465e719c77e87 100644 --- a/config/config.go +++ b/config/config.go @@ -34,6 +34,7 @@ import ( "github.com/pingcap/tidb/util/versioninfo" tikvcfg "github.com/tikv/client-go/v2/config" tracing "github.com/uber/jaeger-client-go/config" + atomicutil "go.uber.org/atomic" "go.uber.org/zap" ) @@ -301,6 +302,42 @@ func (b *nullableBool) UnmarshalJSON(data []byte) error { return err } +// AtomicBool is a helper type for atomic operations on a boolean value. +type AtomicBool struct { + atomicutil.Bool +} + +// NewAtomicBool creates an AtomicBool. +func NewAtomicBool(v bool) *AtomicBool { + return &AtomicBool{*atomicutil.NewBool(v)} +} + +// MarshalText implements the encoding.TextMarshaler interface. +func (b AtomicBool) MarshalText() ([]byte, error) { + if b.Load() { + return []byte("true"), nil + } + return []byte("false"), nil +} + +// UnmarshalText implements the encoding.TextUnmarshaler interface. +func (b *AtomicBool) UnmarshalText(text []byte) error { + str := string(text) + switch str { + case "", "null": + *b = AtomicBool{*atomicutil.NewBool(false)} + return nil + case "true": + *b = AtomicBool{*atomicutil.NewBool(true)} + case "false": + *b = AtomicBool{*atomicutil.NewBool(false)} + default: + *b = AtomicBool{*atomicutil.NewBool(false)} + return errors.New("Invalid value for bool type: " + str) + } + return nil +} + // Log is the log section of config. type Log struct { // Log level. @@ -320,12 +357,12 @@ type Log struct { // File log config. File logutil.FileLogConfig `toml:"file" json:"file"` - EnableSlowLog bool `toml:"enable-slow-log" json:"enable-slow-log"` - SlowQueryFile string `toml:"slow-query-file" json:"slow-query-file"` - SlowThreshold uint64 `toml:"slow-threshold" json:"slow-threshold"` - ExpensiveThreshold uint `toml:"expensive-threshold" json:"expensive-threshold"` - QueryLogMaxLen uint64 `toml:"query-log-max-len" json:"query-log-max-len"` - RecordPlanInSlowLog uint32 `toml:"record-plan-in-slow-log" json:"record-plan-in-slow-log"` + EnableSlowLog AtomicBool `toml:"enable-slow-log" json:"enable-slow-log"` + SlowQueryFile string `toml:"slow-query-file" json:"slow-query-file"` + SlowThreshold uint64 `toml:"slow-threshold" json:"slow-threshold"` + ExpensiveThreshold uint `toml:"expensive-threshold" json:"expensive-threshold"` + QueryLogMaxLen uint64 `toml:"query-log-max-len" json:"query-log-max-len"` + RecordPlanInSlowLog uint32 `toml:"record-plan-in-slow-log" json:"record-plan-in-slow-log"` } func (l *Log) getDisableTimestamp() bool { @@ -619,7 +656,7 @@ var defaultConf = Config{ DisableTimestamp: nbUnset, // If both options are nbUnset, getDisableTimestamp() returns false QueryLogMaxLen: logutil.DefaultQueryLogMaxLen, RecordPlanInSlowLog: logutil.DefaultRecordPlanInSlowLog, - EnableSlowLog: logutil.DefaultTiDBEnableSlowLog, + EnableSlowLog: *NewAtomicBool(logutil.DefaultTiDBEnableSlowLog), }, Status: Status{ ReportStatus: true, diff --git a/config/config_test.go b/config/config_test.go index 8490d6dd786cb..c442fc4df6f76 100644 --- a/config/config_test.go +++ b/config/config_test.go @@ -153,7 +153,7 @@ func TestConfig(t *testing.T) { conf.Performance.TxnTotalSizeLimit = 1000 conf.TiKVClient.CommitTimeout = "10s" conf.TiKVClient.RegionCacheTTL = 600 - conf.Log.EnableSlowLog = logutil.DefaultTiDBEnableSlowLog + conf.Log.EnableSlowLog.Store(logutil.DefaultTiDBEnableSlowLog) configFile := "config.toml" _, localFile, _, _ := runtime.Caller(0) configFile = filepath.Join(filepath.Dir(localFile), configFile) diff --git a/config/config_util_test.go b/config/config_util_test.go index df3a9f0e6082d..baf06b4a35082 100644 --- a/config/config_util_test.go +++ b/config/config_util_test.go @@ -35,7 +35,7 @@ func TestCloneConf(t *testing.T) { c1.Store = "abc" c1.Port = 2333 - c1.Log.EnableSlowLog = !c1.Log.EnableSlowLog + c1.Log.EnableSlowLog.Store(!c1.Log.EnableSlowLog.Load()) c1.RepairTableList = append(c1.RepairTableList, "abc") require.NotEqual(t, c2.Store, c1.Store) require.NotEqual(t, c2.Port, c1.Port) diff --git a/executor/adapter.go b/executor/adapter.go index f8098751e5f08..b8610ffd10f60 100644 --- a/executor/adapter.go +++ b/executor/adapter.go @@ -939,7 +939,7 @@ func (a *ExecStmt) LogSlowQuery(txnTS uint64, succ bool, hasMoreResults bool) { cfg := config.GetGlobalConfig() costTime := time.Since(sessVars.StartTime) + sessVars.DurationParse threshold := time.Duration(atomic.LoadUint64(&cfg.Log.SlowThreshold)) * time.Millisecond - enable := cfg.Log.EnableSlowLog + enable := cfg.Log.EnableSlowLog.Load() // if the level is Debug, or trace is enabled, print slow logs anyway force := level <= zapcore.DebugLevel || trace.IsEnabled() if (!enable || costTime < threshold) && !force { diff --git a/session/bench_test.go b/session/bench_test.go index 6a91caadc1038..d34bb94501071 100644 --- a/session/bench_test.go +++ b/session/bench_test.go @@ -44,7 +44,7 @@ var bigCount = 10000 func prepareBenchSession() (Session, *domain.Domain, kv.Storage) { config.UpdateGlobal(func(cfg *config.Config) { - cfg.Log.EnableSlowLog = false + cfg.Log.EnableSlowLog.Store(false) }) store, err := mockstore.NewMockStore() diff --git a/sessionctx/variable/sysvar.go b/sessionctx/variable/sysvar.go index 5a47319c6580f..29ed3430fb289 100644 --- a/sessionctx/variable/sysvar.go +++ b/sessionctx/variable/sysvar.go @@ -1604,10 +1604,7 @@ var defaultSysVars = []*SysVar{ s.MetricSchemaRangeDuration = tidbOptInt64(val, DefTiDBMetricSchemaRangeDuration) return nil }}, - {Scope: ScopeSession, Name: TiDBSlowLogThreshold, Value: strconv.Itoa(logutil.DefaultSlowThreshold), skipInit: true, Type: TypeInt, MinValue: -1, MaxValue: math.MaxInt64, SetSession: func(s *SessionVars, val string) error { - atomic.StoreUint64(&config.GetGlobalConfig().Log.SlowThreshold, uint64(tidbOptInt64(val, logutil.DefaultSlowThreshold))) - return nil - }, GetSession: func(s *SessionVars) (string, error) { + {Scope: ScopeSession, Name: TiDBSlowLogThreshold, Value: strconv.Itoa(logutil.DefaultSlowThreshold), skipInit: true, Type: TypeInt, MinValue: -1, MaxValue: math.MaxInt64, GetSession: func(s *SessionVars) (string, error) { return strconv.FormatUint(atomic.LoadUint64(&config.GetGlobalConfig().Log.SlowThreshold), 10), nil }}, {Scope: ScopeSession, Name: TiDBRecordPlanInSlowLog, Value: int32ToBoolStr(logutil.DefaultRecordPlanInSlowLog), skipInit: true, Type: TypeBool, SetSession: func(s *SessionVars, val string) error { @@ -1618,10 +1615,10 @@ var defaultSysVars = []*SysVar{ return BoolToOnOff(enabled), nil }}, {Scope: ScopeSession, Name: TiDBEnableSlowLog, Value: BoolToOnOff(logutil.DefaultTiDBEnableSlowLog), Type: TypeBool, skipInit: true, SetSession: func(s *SessionVars, val string) error { - config.GetGlobalConfig().Log.EnableSlowLog = TiDBOptOn(val) + config.GetGlobalConfig().Log.EnableSlowLog.Store(TiDBOptOn(val)) return nil }, GetSession: func(s *SessionVars) (string, error) { - return BoolToOnOff(config.GetGlobalConfig().Log.EnableSlowLog), nil + return BoolToOnOff(config.GetGlobalConfig().Log.EnableSlowLog.Load()), nil }}, {Scope: ScopeSession, Name: TiDBQueryLogMaxLen, Value: strconv.Itoa(logutil.DefaultQueryLogMaxLen), Type: TypeInt, MinValue: -1, MaxValue: math.MaxInt64, skipInit: true, SetSession: func(s *SessionVars, val string) error { atomic.StoreUint64(&config.GetGlobalConfig().Log.QueryLogMaxLen, uint64(tidbOptInt64(val, logutil.DefaultQueryLogMaxLen))) diff --git a/sessionctx/variable/sysvar_test.go b/sessionctx/variable/sysvar_test.go index de53b352fd391..e15c9f92b92b8 100644 --- a/sessionctx/variable/sysvar_test.go +++ b/sessionctx/variable/sysvar_test.go @@ -608,7 +608,7 @@ func TestInstanceScopedVars(t *testing.T) { val, err = GetSessionOrGlobalSystemVar(vars, TiDBEnableSlowLog) require.NoError(t, err) - require.Equal(t, BoolToOnOff(config.GetGlobalConfig().Log.EnableSlowLog), val) + require.Equal(t, BoolToOnOff(config.GetGlobalConfig().Log.EnableSlowLog.Load()), val) val, err = GetSessionOrGlobalSystemVar(vars, TiDBQueryLogMaxLen) require.NoError(t, err) From 8c9fd0142004d34f36c793ffaecd019099162f17 Mon Sep 17 00:00:00 2001 From: Weizhen Wang Date: Thu, 2 Dec 2021 21:01:04 +0800 Subject: [PATCH 2/5] config: make EnableSlowLog atomic Signed-off-by: Weizhen Wang --- sessionctx/variable/sysvar.go | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/sessionctx/variable/sysvar.go b/sessionctx/variable/sysvar.go index 29ed3430fb289..848e380575582 100644 --- a/sessionctx/variable/sysvar.go +++ b/sessionctx/variable/sysvar.go @@ -1604,7 +1604,10 @@ var defaultSysVars = []*SysVar{ s.MetricSchemaRangeDuration = tidbOptInt64(val, DefTiDBMetricSchemaRangeDuration) return nil }}, - {Scope: ScopeSession, Name: TiDBSlowLogThreshold, Value: strconv.Itoa(logutil.DefaultSlowThreshold), skipInit: true, Type: TypeInt, MinValue: -1, MaxValue: math.MaxInt64, GetSession: func(s *SessionVars) (string, error) { + {Scope: ScopeSession, Name: TiDBSlowLogThreshold, Value: strconv.Itoa(logutil.DefaultSlowThreshold), skipInit: true, Type: TypeInt, MinValue: -1, MaxValue: math.MaxInt64, SetSession: func(s *SessionVars, val string) error { + atomic.StoreUint64(&config.GetGlobalConfig().Log.SlowThreshold, uint64(tidbOptInt64(val, logutil.DefaultSlowThreshold))) + return nil + }, GetSession: func(s *SessionVars) (string, error) { return strconv.FormatUint(atomic.LoadUint64(&config.GetGlobalConfig().Log.SlowThreshold), 10), nil }}, {Scope: ScopeSession, Name: TiDBRecordPlanInSlowLog, Value: int32ToBoolStr(logutil.DefaultRecordPlanInSlowLog), skipInit: true, Type: TypeBool, SetSession: func(s *SessionVars, val string) error { From 3dcb8ada973e1edb6781ea9f220737c989756f80 Mon Sep 17 00:00:00 2001 From: Weizhen Wang Date: Thu, 2 Dec 2021 21:37:23 +0800 Subject: [PATCH 3/5] config: make EnableSlowLog atomic Signed-off-by: Weizhen Wang --- config/config_test.go | 33 +++++++++++++++++++++++++++++++++ 1 file changed, 33 insertions(+) diff --git a/config/config_test.go b/config/config_test.go index c442fc4df6f76..77d02fbb03f9f 100644 --- a/config/config_test.go +++ b/config/config_test.go @@ -15,6 +15,7 @@ package config import ( + "bytes" "encoding/json" "os" "os/user" @@ -31,6 +32,38 @@ import ( tracing "github.com/uber/jaeger-client-go/config" ) +func TestAtomicBoolUnmarshal(t *testing.T) { + t.Parallel() + type data struct { + Ab AtomicBool `toml:"ab"` + } + var d data + var firstBuffer bytes.Buffer + _, err := toml.Decode("ab=true", &d) + require.NoError(t, err) + require.True(t, d.Ab.Load()) + err = toml.NewEncoder(&firstBuffer).Encode(d) + require.NoError(t, err) + require.Equal(t, "ab = \"true\"\n", firstBuffer.String()) + firstBuffer.Reset() + + d = data{} + _, err = toml.Decode("", &d) + require.NoError(t, err) + require.False(t, d.Ab.Load()) + + d = data{} + _, err = toml.Decode("ab=false", &d) + require.NoError(t, err) + require.False(t, d.Ab.Load()) + err = toml.NewEncoder(&firstBuffer).Encode(d) + require.NoError(t, err) + require.Equal(t, "ab = \"false\"\n", firstBuffer.String()) + + _, err = toml.Decode("ab = 1", &d) + require.EqualError(t, err, "Invalid value for bool type: 1") +} + func TestNullableBoolUnmarshal(t *testing.T) { t.Parallel() From 23f531e6cb7534efb2f2f13037d6368be1046527 Mon Sep 17 00:00:00 2001 From: Weizhen Wang Date: Thu, 2 Dec 2021 21:51:35 +0800 Subject: [PATCH 4/5] config: make EnableSlowLog atomic Signed-off-by: Weizhen Wang --- config/config.go | 1 - config/config_test.go | 6 ------ 2 files changed, 7 deletions(-) diff --git a/config/config.go b/config/config.go index 465e719c77e87..53141fadd093a 100644 --- a/config/config.go +++ b/config/config.go @@ -326,7 +326,6 @@ func (b *AtomicBool) UnmarshalText(text []byte) error { switch str { case "", "null": *b = AtomicBool{*atomicutil.NewBool(false)} - return nil case "true": *b = AtomicBool{*atomicutil.NewBool(true)} case "false": diff --git a/config/config_test.go b/config/config_test.go index 77d02fbb03f9f..eaa8e5ea16756 100644 --- a/config/config_test.go +++ b/config/config_test.go @@ -47,12 +47,6 @@ func TestAtomicBoolUnmarshal(t *testing.T) { require.Equal(t, "ab = \"true\"\n", firstBuffer.String()) firstBuffer.Reset() - d = data{} - _, err = toml.Decode("", &d) - require.NoError(t, err) - require.False(t, d.Ab.Load()) - - d = data{} _, err = toml.Decode("ab=false", &d) require.NoError(t, err) require.False(t, d.Ab.Load()) From 1449522c215065391cc3dd82f653172b127de1ec Mon Sep 17 00:00:00 2001 From: Weizhen Wang Date: Fri, 3 Dec 2021 00:17:41 +0800 Subject: [PATCH 5/5] config: make EnableSlowLog atomic Signed-off-by: Weizhen Wang --- config/config_util.go | 10 ++++++++++ 1 file changed, 10 insertions(+) diff --git a/config/config_util.go b/config/config_util.go index 7db47d7d40231..f8ae5221c2617 100644 --- a/config/config_util.go +++ b/config/config_util.go @@ -69,6 +69,16 @@ func MergeConfigItems(dstConf, newConf *Config) (acceptedItems, rejectedItems [] func mergeConfigItems(dstConf, newConf reflect.Value, fieldPath string) (acceptedItems, rejectedItems []string) { t := dstConf.Type() + if t.Name() == "AtomicBool" { + if reflect.DeepEqual(dstConf.Interface().(AtomicBool), newConf.Interface().(AtomicBool)) { + return + } + if _, ok := dynamicConfigItems[fieldPath]; ok { + dstConf.Set(newConf) + return []string{fieldPath}, nil + } + return nil, []string{fieldPath} + } if t.Kind() == reflect.Ptr { t = t.Elem() dstConf = dstConf.Elem()