From a70a56b14e303b7ce43f763b5d4fd7b201ca23c1 Mon Sep 17 00:00:00 2001 From: Yuanjia Zhang Date: Wed, 12 Feb 2020 20:25:24 +0800 Subject: [PATCH 01/21] config: support to dynamically update some config items read from PD --- config/config.go | 130 +++++++++++---------------- config/config_handler.go | 24 ++++- config/config_test.go | 25 ------ server/http_handler.go | 13 --- server/http_status.go | 1 - sessionctx/variable/session.go | 11 --- sessionctx/variable/sysvar.go | 6 -- sessionctx/variable/tidb_vars.go | 15 ---- sessionctx/variable/varsutil.go | 16 +--- sessionctx/variable/varsutil_test.go | 13 --- tidb-server/main.go | 78 +++------------- 11 files changed, 84 insertions(+), 248 deletions(-) diff --git a/config/config.go b/config/config.go index 89e4dd27293d0..b430e0745b0e4 100644 --- a/config/config.go +++ b/config/config.go @@ -14,7 +14,6 @@ package config import ( - "bytes" "crypto/tls" "crypto/x509" "encoding/json" @@ -22,18 +21,16 @@ import ( "io/ioutil" "net/url" "os" - "reflect" "strings" - "sync" "time" "github.com/BurntSushi/toml" "github.com/pingcap/errors" zaplog "github.com/pingcap/log" "github.com/pingcap/parser/mysql" + "github.com/pingcap/parser/terror" "github.com/pingcap/tidb/util/logutil" tracing "github.com/uber/jaeger-client-go/config" - "go.uber.org/atomic" "go.uber.org/zap" ) @@ -614,12 +611,7 @@ var defaultConf = Config{ } var ( - globalConf = atomic.Value{} - reloadConfPath = "" - confReloader func(nc, c *Config) - confReloadLock sync.Mutex - supportedReloadConfigs = make(map[string]struct{}, 32) - supportedReloadConfList = make([]string, 0, 32) + globalConfHandler ConfHandler ) // NewConfig creates a new config instance with default value. @@ -628,94 +620,71 @@ func NewConfig() *Config { return &conf } -// SetConfReloader sets reload config path and a reloader. -// It should be called only once at start time. -func SetConfReloader(cpath string, reloader func(nc, c *Config), confItems ...string) { - reloadConfPath = cpath - confReloader = reloader - for _, item := range confItems { - supportedReloadConfigs[item] = struct{}{} - supportedReloadConfList = append(supportedReloadConfList, item) - } -} - // GetGlobalConfig returns the global configuration for this server. // It should store configuration from command line and configuration file. // Other parts of the system can read the global configuration use this function. func GetGlobalConfig() *Config { - return globalConf.Load().(*Config) + return globalConfHandler.GetConfig() } // StoreGlobalConfig stores a new config to the globalConf. It mostly uses in the test to avoid some data races. func StoreGlobalConfig(config *Config) { - globalConf.Store(config) + if err := globalConfHandler.SetConfig(config); err != nil { + logutil.BgLogger().Error("update the global config error", zap.Error(err)) + } } -// ReloadGlobalConfig reloads global configuration for this server. -func ReloadGlobalConfig() error { - confReloadLock.Lock() - defer confReloadLock.Unlock() - - nc := NewConfig() - if err := nc.Load(reloadConfPath); err != nil { - return err - } - if err := nc.Valid(); err != nil { - return err - } - c := GetGlobalConfig() +var deprecatedConfig = map[string]struct{}{ + "pessimistic-txn.ttl": {}, + "log.rotate": {}, +} - diffs := collectsDiff(*nc, *c, "") - if len(diffs) == 0 { - return nil - } - var formattedDiff bytes.Buffer - for k, vs := range diffs { - formattedDiff.WriteString(fmt.Sprintf(", %v:%v->%v", k, vs[1], vs[0])) - } - unsupported := make([]string, 0, 2) - for k := range diffs { - if _, ok := supportedReloadConfigs[k]; !ok { - unsupported = append(unsupported, k) +func isDeprecatedConfigItem(items []string) bool { + for _, item := range items { + if _, ok := deprecatedConfig[item]; !ok { + return false } } - if len(unsupported) > 0 { - return fmt.Errorf("reloading config %v is not supported, only %v are supported now, "+ - "your changes%s", unsupported, supportedReloadConfList, formattedDiff.String()) - } - - confReloader(nc, c) - globalConf.Store(nc) - logutil.BgLogger().Info("reload config changes" + formattedDiff.String()) - return nil + return true } -// collectsDiff collects different config items. -// map[string][]string -> map[field path][]{new value, old value} -func collectsDiff(i1, i2 interface{}, fieldPath string) map[string][]interface{} { - diff := make(map[string][]interface{}) - t := reflect.TypeOf(i1) - if t.Kind() != reflect.Struct { - if reflect.DeepEqual(i1, i2) { - return diff +// InitializeConfig initialize the global config handler. +func InitializeConfig(confPath string, configCheck, configStrict bool, reloadFunc ConfReloadFunc, overwriteFunc OverwriteFunc) { + cfg := GetGlobalConfig() + var err error + if confPath != "" { + err = cfg.Load(confPath) + if err == nil { + return } - diff[fieldPath] = []interface{}{i1, i2} - return diff - } - - v1 := reflect.ValueOf(i1) - v2 := reflect.ValueOf(i2) - for i := 0; i < v1.NumField(); i++ { - p := t.Field(i).Name - if fieldPath != "" { - p = fieldPath + "." + p + // Unused config item erro turns to warnings. + if tmp, ok := err.(*ErrConfigValidationFailed); ok { + if isDeprecatedConfigItem(tmp.UndecodedItems) { + fmt.Fprintln(os.Stderr, err.Error()) + err = nil + } + // This block is to accommodate an interim situation where strict config checking + // is not the default behavior of TiDB. The warning message must be deferred until + // logging has been set up. After strict config checking is the default behavior, + // This should all be removed. + if !configCheck && !configStrict { + fmt.Fprintln(os.Stderr, err.Error()) + err = nil + } } - m := collectsDiff(v1.Field(i).Interface(), v2.Field(i).Interface(), p) - for k, v := range m { - diff[k] = v + + terror.MustNil(err) + } else { + // configCheck should have the config file specified. + if configCheck { + fmt.Fprintln(os.Stderr, "config check failed", errors.New("no config file specified for config-check")) + os.Exit(1) } } - return diff + overwriteFunc(cfg) + globalConfHandler, err = NewConfHandler(cfg, reloadFunc, overwriteFunc) + terror.MustNil(err) + globalConfHandler.Start() } // Load loads config options from a toml file. @@ -862,7 +831,8 @@ func (t *OpenTracing) ToTracingConfig() *tracing.Configuration { } func init() { - globalConf.Store(&defaultConf) + conf := defaultConf + globalConfHandler = &constantConfHandler{&conf} if checkBeforeDropLDFlag == "1" { CheckTableBeforeDrop = true } diff --git a/config/config_handler.go b/config/config_handler.go index e34b8626155d6..f7d1b7938480f 100644 --- a/config/config_handler.go +++ b/config/config_handler.go @@ -35,16 +35,21 @@ type ConfHandler interface { Start() Close() GetConfig() *Config // read only + SetConfig(conf *Config) error } // ConfReloadFunc is used to reload the config to make it work. type ConfReloadFunc func(oldConf, newConf *Config) +// OverwriteFunc is used to overwrite some config items which are initialized from commend and +// shouldn't be updated during runtime. +type OverwriteFunc func(conf *Config) + // NewConfHandler creates a new ConfHandler according to the local config. -func NewConfHandler(localConf *Config, reloadFunc ConfReloadFunc) (ConfHandler, error) { +func NewConfHandler(localConf *Config, reloadFunc ConfReloadFunc, overwriteFunc OverwriteFunc) (ConfHandler, error) { switch defaultConf.Store { case "tikv": - return newPDConfHandler(localConf, reloadFunc, nil) + return newPDConfHandler(localConf, reloadFunc, overwriteFunc, nil) default: return &constantConfHandler{localConf}, nil } @@ -62,6 +67,11 @@ func (cch *constantConfHandler) Close() {} func (cch *constantConfHandler) GetConfig() *Config { return cch.conf } +func (cch *constantConfHandler) SetConfig(conf *Config) error { + cch.conf = conf + return nil +} + const ( pdConfHandlerRefreshInterval = 30 * time.Second tidbComponentName = "tidb" @@ -76,9 +86,11 @@ type pdConfHandler struct { exit chan struct{} pdConfCli pd.ConfigClient reloadFunc func(oldConf, newConf *Config) + + overwriteFunc OverwriteFunc } -func newPDConfHandler(localConf *Config, reloadFunc ConfReloadFunc, +func newPDConfHandler(localConf *Config, reloadFunc ConfReloadFunc, overwriteFunc OverwriteFunc, newPDCliFunc func([]string, pd.SecurityOption) (pd.ConfigClient, error), // for test ) (*pdConfHandler, error) { addresses, _, err := ParsePath(localConf.Path) @@ -138,6 +150,8 @@ func newPDConfHandler(localConf *Config, reloadFunc ConfReloadFunc, exit: make(chan struct{}), pdConfCli: pdCli, reloadFunc: reloadFunc, + + overwriteFunc: overwriteFunc, } ch.curConf.Store(newConf) return ch, nil @@ -158,6 +172,10 @@ func (ch *pdConfHandler) GetConfig() *Config { return ch.curConf.Load().(*Config) } +func (ch *pdConfHandler) SetConfig(conf *Config) error { + return errors.New("PDConfHandler only support to update the config from PD whereas forbid to modify it locally") +} + func (ch *pdConfHandler) run() { defer func() { if r := recover(); r != nil { diff --git a/config/config_test.go b/config/config_test.go index 454ad5428fe8f..d93a8dc785ca5 100644 --- a/config/config_test.go +++ b/config/config_test.go @@ -328,31 +328,6 @@ xkNuJ2BlEGkwWLiRbKy1lNBBFUXKuhh3L/EIY10WTnr3TQzeL6H1 c.Assert(os.Remove(keyFile), IsNil) } -func (s *testConfigSuite) TestConfigDiff(c *C) { - c1 := NewConfig() - c2 := &Config{} - *c2 = *c1 - c1.OOMAction = "c1" - c2.OOMAction = "c2" - c1.MemQuotaQuery = 2333 - c2.MemQuotaQuery = 3222 - c1.Performance.CrossJoin = true - c2.Performance.CrossJoin = false - c1.Performance.FeedbackProbability = 2333 - c2.Performance.FeedbackProbability = 23.33 - - diffs := collectsDiff(*c1, *c2, "") - c.Assert(len(diffs), Equals, 4) - c.Assert(diffs["OOMAction"][0], Equals, "c1") - c.Assert(diffs["OOMAction"][1], Equals, "c2") - c.Assert(diffs["MemQuotaQuery"][0], Equals, int64(2333)) - c.Assert(diffs["MemQuotaQuery"][1], Equals, int64(3222)) - c.Assert(diffs["Performance.CrossJoin"][0], Equals, true) - c.Assert(diffs["Performance.CrossJoin"][1], Equals, false) - c.Assert(diffs["Performance.FeedbackProbability"][0], Equals, float64(2333)) - c.Assert(diffs["Performance.FeedbackProbability"][1], Equals, float64(23.33)) -} - func (s *testConfigSuite) TestOOMActionValid(c *C) { c1 := NewConfig() tests := []struct { diff --git a/server/http_handler.go b/server/http_handler.go index 04534377117ce..65f9fd360f59b 100644 --- a/server/http_handler.go +++ b/server/http_handler.go @@ -674,19 +674,6 @@ func (h settingsHandler) ServeHTTP(w http.ResponseWriter, req *http.Request) { } } -// configReloadHandler is the handler for reloading config online. -type configReloadHandler struct { -} - -// ServeHTTP handles request of reloading config for this server. -func (h configReloadHandler) ServeHTTP(w http.ResponseWriter, req *http.Request) { - if err := config.ReloadGlobalConfig(); err != nil { - writeError(w, err) - } else { - writeData(w, "success!") - } -} - // ServeHTTP recovers binlog service. func (h binlogRecover) ServeHTTP(w http.ResponseWriter, req *http.Request) { op := req.FormValue(qOperation) diff --git a/server/http_status.go b/server/http_status.go index 1a428c3f49880..0f7ea9acf4c66 100644 --- a/server/http_status.go +++ b/server/http_status.go @@ -80,7 +80,6 @@ func (s *Server) startHTTPServer() { router.Handle("/stats/dump/{db}/{table}/{snapshot}", s.newStatsHistoryHandler()).Name("StatsHistoryDump") router.Handle("/settings", settingsHandler{}).Name("Settings") - router.Handle("/reload-config", configReloadHandler{}).Name("ConfigReload") router.Handle("/binlog/recover", binlogRecover{}).Name("BinlogRecover") tikvHandlerTool := s.newTikvHandlerTool() diff --git a/sessionctx/variable/session.go b/sessionctx/variable/session.go index 26de64cead630..508e68098e372 100644 --- a/sessionctx/variable/session.go +++ b/sessionctx/variable/session.go @@ -41,7 +41,6 @@ import ( "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/execdetails" - "github.com/pingcap/tidb/util/logutil" "github.com/pingcap/tidb/util/rowcodec" "github.com/pingcap/tidb/util/storeutil" "github.com/pingcap/tidb/util/timeutil" @@ -1051,16 +1050,8 @@ func (s *SessionVars) SetSystemVar(name string, val string) error { atomic.StoreUint32(&ProcessGeneralLog, uint32(tidbOptPositiveInt32(val, DefTiDBGeneralLog))) case TiDBPProfSQLCPU: EnablePProfSQLCPU.Store(uint32(tidbOptPositiveInt32(val, DefTiDBPProfSQLCPU)) > 0) - case TiDBSlowLogThreshold: - atomic.StoreUint64(&config.GetGlobalConfig().Log.SlowThreshold, uint64(tidbOptInt64(val, logutil.DefaultSlowThreshold))) - case TiDBRecordPlanInSlowLog: - atomic.StoreUint32(&config.GetGlobalConfig().Log.RecordPlanInSlowLog, uint32(tidbOptInt64(val, logutil.DefaultRecordPlanInSlowLog))) - case TiDBEnableSlowLog: - atomic.StoreUint32(&config.GetGlobalConfig().Log.EnableSlowLog, uint32(tidbOptInt64(val, logutil.DefaultTiDBEnableSlowLog))) case TiDBDDLSlowOprThreshold: atomic.StoreUint32(&DDLSlowOprThreshold, uint32(tidbOptPositiveInt32(val, DefTiDBDDLSlowOprThreshold))) - case TiDBQueryLogMaxLen: - atomic.StoreUint64(&config.GetGlobalConfig().Log.QueryLogMaxLen, uint64(tidbOptInt64(val, logutil.DefaultQueryLogMaxLen))) case TiDBRetryLimit: s.RetryLimit = tidbOptInt64(val, DefTiDBRetryLimit) case TiDBDisableTxnAutoRetry: @@ -1087,8 +1078,6 @@ func (s *SessionVars) SetSystemVar(name string, val string) error { s.EnableVectorizedExpression = TiDBOptOn(val) case TiDBOptJoinReorderThreshold: s.TiDBOptJoinReorderThreshold = tidbOptPositiveInt32(val, DefTiDBOptJoinReorderThreshold) - case TiDBCheckMb4ValueInUTF8: - config.GetGlobalConfig().CheckMb4ValueInUTF8 = TiDBOptOn(val) case TiDBSlowQueryFile: s.SlowQueryFile = val case TiDBEnableFastAnalyze: diff --git a/sessionctx/variable/sysvar.go b/sessionctx/variable/sysvar.go index e342861f7880c..56f180f2a4eba 100644 --- a/sessionctx/variable/sysvar.go +++ b/sessionctx/variable/sysvar.go @@ -22,7 +22,6 @@ import ( "github.com/pingcap/parser/terror" "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/kv" - "github.com/pingcap/tidb/util/logutil" ) // ScopeFlag is for system variable whether can be changed in global/session dynamically or not. @@ -690,11 +689,7 @@ var defaultSysVars = []*SysVar{ /* The following variable is defined as session scope but is actually server scope. */ {ScopeSession, TiDBGeneralLog, strconv.Itoa(DefTiDBGeneralLog)}, {ScopeSession, TiDBPProfSQLCPU, strconv.Itoa(DefTiDBPProfSQLCPU)}, - {ScopeSession, TiDBSlowLogThreshold, strconv.Itoa(logutil.DefaultSlowThreshold)}, - {ScopeSession, TiDBRecordPlanInSlowLog, strconv.Itoa(logutil.DefaultRecordPlanInSlowLog)}, - {ScopeSession, TiDBEnableSlowLog, strconv.Itoa(logutil.DefaultTiDBEnableSlowLog)}, {ScopeSession, TiDBDDLSlowOprThreshold, strconv.Itoa(DefTiDBDDLSlowOprThreshold)}, - {ScopeSession, TiDBQueryLogMaxLen, strconv.Itoa(logutil.DefaultQueryLogMaxLen)}, {ScopeSession, TiDBConfig, ""}, {ScopeGlobal, TiDBDDLReorgWorkerCount, strconv.Itoa(DefTiDBDDLReorgWorkerCount)}, {ScopeGlobal, TiDBDDLReorgBatchSize, strconv.Itoa(DefTiDBDDLReorgBatchSize)}, @@ -704,7 +699,6 @@ var defaultSysVars = []*SysVar{ {ScopeSession, TiDBForcePriority, mysql.Priority2Str[DefTiDBForcePriority]}, {ScopeSession, TiDBEnableRadixJoin, BoolToIntStr(DefTiDBUseRadixJoin)}, {ScopeGlobal | ScopeSession, TiDBOptJoinReorderThreshold, strconv.Itoa(DefTiDBOptJoinReorderThreshold)}, - {ScopeSession, TiDBCheckMb4ValueInUTF8, BoolToIntStr(config.GetGlobalConfig().CheckMb4ValueInUTF8)}, {ScopeSession, TiDBSlowQueryFile, ""}, {ScopeGlobal, TiDBScatterRegion, BoolToIntStr(DefTiDBScatterRegion)}, {ScopeSession, TiDBWaitSplitRegionFinish, BoolToIntStr(DefTiDBWaitSplitRegionFinish)}, diff --git a/sessionctx/variable/tidb_vars.go b/sessionctx/variable/tidb_vars.go index 0fe4e910e2162..e324a10d43a2e 100644 --- a/sessionctx/variable/tidb_vars.go +++ b/sessionctx/variable/tidb_vars.go @@ -110,18 +110,6 @@ const ( // tidb_pprof_sql_cpu is used to add label sql label to pprof result. TiDBPProfSQLCPU = "tidb_pprof_sql_cpu" - // tidb_slow_log_threshold is used to set the slow log threshold in the server. - TiDBSlowLogThreshold = "tidb_slow_log_threshold" - - // tidb_record_plan_in_slow_log is used to log the plan of the slow query. - TiDBRecordPlanInSlowLog = "tidb_record_plan_in_slow_log" - - // tidb_enable_slow_log enables TiDB to log slow queries. - TiDBEnableSlowLog = "tidb_enable_slow_log" - - // tidb_query_log_max_len is used to set the max length of the query in the log. - TiDBQueryLogMaxLen = "tidb_query_log_max_len" - // tidb_retry_limit is the maximum number of retries when committing a transaction. TiDBRetryLimit = "tidb_retry_limit" @@ -150,9 +138,6 @@ const ( // off: always disable table partition. TiDBEnableTablePartition = "tidb_enable_table_partition" - // TiDBCheckMb4ValueInUTF8 is used to control whether to enable the check wrong utf8 value. - TiDBCheckMb4ValueInUTF8 = "tidb_check_mb4_value_in_utf8" - // tidb_skip_isolation_level_check is used to control whether to return error when set unsupported transaction // isolation level. TiDBSkipIsolationLevelCheck = "tidb_skip_isolation_level_check" diff --git a/sessionctx/variable/varsutil.go b/sessionctx/variable/varsutil.go index 692f89303208d..a72f0daff6fb5 100644 --- a/sessionctx/variable/varsutil.go +++ b/sessionctx/variable/varsutil.go @@ -133,22 +133,12 @@ func GetSessionOnlySysVars(s *SessionVars, key string) (string, bool, error) { return string(j), true, nil case TiDBForcePriority: return mysql.Priority2Str[mysql.PriorityEnum(atomic.LoadInt32(&ForcePriority))], true, nil - case TiDBSlowLogThreshold: - return strconv.FormatUint(atomic.LoadUint64(&config.GetGlobalConfig().Log.SlowThreshold), 10), true, nil - case TiDBRecordPlanInSlowLog: - return strconv.FormatUint(uint64(atomic.LoadUint32(&config.GetGlobalConfig().Log.RecordPlanInSlowLog)), 10), true, nil - case TiDBEnableSlowLog: - return strconv.FormatUint(uint64(atomic.LoadUint32(&config.GetGlobalConfig().Log.EnableSlowLog)), 10), true, nil case TiDBDDLSlowOprThreshold: return strconv.FormatUint(uint64(atomic.LoadUint32(&DDLSlowOprThreshold)), 10), true, nil - case TiDBQueryLogMaxLen: - return strconv.FormatUint(atomic.LoadUint64(&config.GetGlobalConfig().Log.QueryLogMaxLen), 10), true, nil case PluginDir: return config.GetGlobalConfig().Plugin.Dir, true, nil case PluginLoad: return config.GetGlobalConfig().Plugin.Load, true, nil - case TiDBCheckMb4ValueInUTF8: - return BoolToIntStr(config.GetGlobalConfig().CheckMb4ValueInUTF8), true, nil } sVal, ok := s.GetSystemVar(key) if ok { @@ -405,8 +395,8 @@ func ValidateSetSystemVar(vars *SessionVars, name string, value string) (string, TiDBOptInSubqToJoinAndAgg, TiDBEnableFastAnalyze, TiDBBatchInsert, TiDBDisableTxnAutoRetry, TiDBEnableStreaming, TiDBEnableChunkRPC, TiDBBatchDelete, TiDBBatchCommit, TiDBEnableCascadesPlanner, TiDBEnableWindowFunction, TiDBPProfSQLCPU, - TiDBCheckMb4ValueInUTF8, TiDBLowResolutionTSO, TiDBEnableIndexMerge, TiDBEnableNoopFuncs, TiDBEnableSlowLog, - TiDBScatterRegion, TiDBGeneralLog, TiDBConstraintCheckInPlace, TiDBEnableVectorizedExpression, TiDBRecordPlanInSlowLog: + TiDBLowResolutionTSO, TiDBEnableIndexMerge, TiDBEnableNoopFuncs, + TiDBScatterRegion, TiDBGeneralLog, TiDBConstraintCheckInPlace, TiDBEnableVectorizedExpression: fallthrough case GeneralLog, AvoidTemporalUpgrade, BigTables, CheckProxyUsers, LogBin, CoreFile, EndMakersInJSON, SQLLogBin, OfflineMode, PseudoSlaveMode, LowPriorityUpdates, @@ -532,8 +522,6 @@ func ValidateSetSystemVar(vars *SessionVars, name string, value string) (string, TIDBMemQuotaIndexLookupJoin, TIDBMemQuotaNestedLoopApply, TiDBRetryLimit, - TiDBSlowLogThreshold, - TiDBQueryLogMaxLen, TiDBEvolvePlanTaskMaxTime: _, err := strconv.ParseInt(value, 10, 64) if err != nil { diff --git a/sessionctx/variable/varsutil_test.go b/sessionctx/variable/varsutil_test.go index c2791a2439582..204a85fadfc39 100644 --- a/sessionctx/variable/varsutil_test.go +++ b/sessionctx/variable/varsutil_test.go @@ -265,19 +265,6 @@ func (s *testVarsutilSuite) TestVarsutil(c *C) { c.Assert(val, Equals, "5") c.Assert(v.TiDBOptJoinReorderThreshold, Equals, 5) - err = SetSessionSystemVar(v, TiDBCheckMb4ValueInUTF8, types.NewStringDatum("1")) - c.Assert(err, IsNil) - val, err = GetSessionSystemVar(v, TiDBCheckMb4ValueInUTF8) - c.Assert(err, IsNil) - c.Assert(val, Equals, "1") - c.Assert(config.GetGlobalConfig().CheckMb4ValueInUTF8, Equals, true) - err = SetSessionSystemVar(v, TiDBCheckMb4ValueInUTF8, types.NewStringDatum("0")) - c.Assert(err, IsNil) - val, err = GetSessionSystemVar(v, TiDBCheckMb4ValueInUTF8) - c.Assert(err, IsNil) - c.Assert(val, Equals, "0") - c.Assert(config.GetGlobalConfig().CheckMb4ValueInUTF8, Equals, false) - SetSessionSystemVar(v, TiDBLowResolutionTSO, types.NewStringDatum("1")) val, err = GetSessionSystemVar(v, TiDBLowResolutionTSO) c.Assert(err, IsNil) diff --git a/tidb-server/main.go b/tidb-server/main.go index 9b56415605a9d..9f2ce61606844 100644 --- a/tidb-server/main.go +++ b/tidb-server/main.go @@ -143,7 +143,6 @@ var ( ) var ( - cfg *config.Config storage kv.Storage dom *domain.Domain svr *server.Server @@ -158,9 +157,8 @@ func main() { } registerStores() registerMetrics() - configWarning := loadConfig() - overrideConfig() - if err := cfg.Valid(); err != nil { + config.InitializeConfig(*configPath, *configCheck, *configStrict, reloadConfig, overrideConfig) + if err := config.GetGlobalConfig().Valid(); err != nil { fmt.Fprintln(os.Stderr, "invalid config", err) os.Exit(1) } @@ -171,12 +169,6 @@ func main() { setGlobalVars() setCPUAffinity() setupLog() - // If configStrict had been specified, and there had been an error, the server would already - // have exited by now. If configWarning is not an empty string, write it to the log now that - // it's been properly set up. - if configWarning != "" { - log.Warn(configWarning) - } setupTracing() // Should before createServer and after setup config. printInfo() setupBinlogClient() @@ -238,6 +230,7 @@ func registerMetrics() { } func createStoreAndDomain() { + cfg := config.GetGlobalConfig() fullPath := fmt.Sprintf("%s://%s", cfg.Store, cfg.Path) var err error storage, err = kvstore.New(fullPath) @@ -248,6 +241,7 @@ func createStoreAndDomain() { } func setupBinlogClient() { + cfg := config.GetGlobalConfig() if !cfg.Binlog.Enable { return } @@ -312,6 +306,7 @@ func prometheusPushClient(addr string, interval time.Duration) { } func instanceName() string { + cfg := config.GetGlobalConfig() hostname, err := os.Hostname() if err != nil { return "unknown" @@ -340,62 +335,6 @@ func flagBoolean(name string, defaultVal bool, usage string) *bool { return flag.Bool(name, defaultVal, usage) } -var deprecatedConfig = map[string]struct{}{ - "pessimistic-txn.ttl": {}, - "log.rotate": {}, -} - -func isDeprecatedConfigItem(items []string) bool { - for _, item := range items { - if _, ok := deprecatedConfig[item]; !ok { - return false - } - } - return true -} - -func loadConfig() string { - cfg = config.GetGlobalConfig() - if *configPath != "" { - // Not all config items are supported now. - config.SetConfReloader(*configPath, reloadConfig, hotReloadConfigItems...) - - err := cfg.Load(*configPath) - if err == nil { - return "" - } - - // Unused config item erro turns to warnings. - if tmp, ok := err.(*config.ErrConfigValidationFailed); ok { - if isDeprecatedConfigItem(tmp.UndecodedItems) { - return err.Error() - } - // This block is to accommodate an interim situation where strict config checking - // is not the default behavior of TiDB. The warning message must be deferred until - // logging has been set up. After strict config checking is the default behavior, - // This should all be removed. - if !*configCheck && !*configStrict { - return err.Error() - } - } - - terror.MustNil(err) - } else { - // configCheck should have the config file specified. - if *configCheck { - fmt.Fprintln(os.Stderr, "config check failed", errors.New("no config file specified for config-check")) - os.Exit(1) - } - } - return "" -} - -// hotReloadConfigItems lists all config items which support hot-reload. -var hotReloadConfigItems = []string{"Performance.MaxProcs", "Performance.MaxMemory", "Performance.CrossJoin", - "Performance.FeedbackProbability", "Performance.QueryFeedbackLimit", "Performance.PseudoEstimateRatio", - "OOMUseTmpStorage", "OOMAction", "MemQuotaQuery", "StmtSummary.MaxStmtCount", "StmtSummary.MaxSQLLength", "Log.QueryLogMaxLen", - "TiKVClient.EnableChunkRPC", "TiKVClient.StoreLimit"} - func reloadConfig(nc, c *config.Config) { // Just a part of config items need to be reload explicitly. // Some of them like OOMAction are always used by getting from global config directly @@ -422,7 +361,7 @@ func reloadConfig(nc, c *config.Config) { } } -func overrideConfig() { +func overrideConfig(cfg *config.Config) { actualFlags := make(map[string]bool) flag.Visit(func(f *flag.Flag) { actualFlags[f.Name] = true @@ -526,6 +465,7 @@ func overrideConfig() { } func setGlobalVars() { + cfg := config.GetGlobalConfig() ddlLeaseDuration := parseDuration(cfg.Lease) session.SetSchemaLease(ddlLeaseDuration) runtime.GOMAXPROCS(int(cfg.Performance.MaxProcs)) @@ -580,6 +520,7 @@ func setGlobalVars() { } func setupLog() { + cfg := config.GetGlobalConfig() err := logutil.InitZapLogger(cfg.Log.ToLogConfig()) terror.MustNil(err) @@ -606,6 +547,7 @@ func printInfo() { } func createServer() { + cfg := config.GetGlobalConfig() driver := server.NewTiDBDriver(storage) var err error svr, err = server.NewServer(cfg, driver) @@ -624,6 +566,7 @@ func serverShutdown(isgraceful bool) { } func setupMetrics() { + cfg := config.GetGlobalConfig() // Enable the mutex profile, 1/10 of mutex blocking event sampling. runtime.SetMutexProfileFraction(10) systimeErrHandler := func() { @@ -644,6 +587,7 @@ func setupMetrics() { } func setupTracing() { + cfg := config.GetGlobalConfig() tracingCfg := cfg.OpenTracing.ToTracingConfig() tracingCfg.ServiceName = "TiDB" tracer, _, err := tracingCfg.NewTracer() From 94234afe5075028c7d071e7ae89f8c846b4c3307 Mon Sep 17 00:00:00 2001 From: Yuanjia Zhang Date: Wed, 12 Feb 2020 20:55:07 +0800 Subject: [PATCH 02/21] fix CI --- config/config_handler_test.go | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/config/config_handler_test.go b/config/config_handler_test.go index 1c70dde2061f5..0c0c554297523 100644 --- a/config/config_handler_test.go +++ b/config/config_handler_test.go @@ -64,7 +64,7 @@ func (mc *mockPDConfigClient) Close() {} func (s *testConfigSuite) TestConstantConfHandler(c *C) { conf := defaultConf conf.Store = "mock" - ch, err := NewConfHandler(&conf, nil) + ch, err := NewConfHandler(&conf, nil, nil) c.Assert(err, IsNil) _, ok := ch.(*constantConfHandler) c.Assert(ok, IsTrue) @@ -77,27 +77,27 @@ func (s *testConfigSuite) TestPDConfHandler(c *C) { // wrong path conf.Store = "tikv" conf.Path = "WRONGPATH" - _, err := newPDConfHandler(&conf, nil, newMockPDConfigClient) + _, err := newPDConfHandler(&conf, nil, nil, newMockPDConfigClient) c.Assert(err, NotNil) // error when creating PD config client conf.Path = "tikv://node1:2379" newMockPDConfigClientErr = fmt.Errorf("") - _, err = newPDConfHandler(&conf, nil, newMockPDConfigClient) + _, err = newPDConfHandler(&conf, nil, nil, newMockPDConfigClient) c.Assert(err, NotNil) // error when registering newMockPDConfigClientErr = nil mockPDConfigClient0.err = fmt.Errorf("") mockPDConfigClient0.confContent.Store("") - ch, err := newPDConfHandler(&conf, nil, newMockPDConfigClient) + ch, err := newPDConfHandler(&conf, nil, nil, newMockPDConfigClient) c.Assert(err, IsNil) // the local config will be used ch.Close() // wrong response when registering mockPDConfigClient0.err = nil mockPDConfigClient0.status = &configpb.Status{Code: configpb.StatusCode_UNKNOWN} - ch, err = newPDConfHandler(&conf, nil, newMockPDConfigClient) + ch, err = newPDConfHandler(&conf, nil, nil, newMockPDConfigClient) c.Assert(err, IsNil) ch.Close() @@ -105,7 +105,7 @@ func (s *testConfigSuite) TestPDConfHandler(c *C) { mockPDConfigClient0.status.Code = configpb.StatusCode_WRONG_VERSION content, _ := encodeConfig(&conf) mockPDConfigClient0.confContent.Store(content) - ch, err = newPDConfHandler(&conf, nil, newMockPDConfigClient) + ch, err = newPDConfHandler(&conf, nil, nil, newMockPDConfigClient) c.Assert(err, IsNil) ch.Close() @@ -117,7 +117,7 @@ func (s *testConfigSuite) TestPDConfHandler(c *C) { c.Assert(oldConf.Log.Level, Equals, "info") c.Assert(newConf.Log.Level, Equals, "debug") } - ch, err = newPDConfHandler(&conf, mockReloadFunc, newMockPDConfigClient) + ch, err = newPDConfHandler(&conf, mockReloadFunc, nil, newMockPDConfigClient) c.Assert(err, IsNil) ch.interval = time.Second ch.Start() From a9aaeecf554489f885d48596fc2e9500a61c60c1 Mon Sep 17 00:00:00 2001 From: Yuanjia Zhang Date: Wed, 12 Feb 2020 21:02:49 +0800 Subject: [PATCH 03/21] fixup --- executor/set_test.go | 19 ------------------- 1 file changed, 19 deletions(-) diff --git a/executor/set_test.go b/executor/set_test.go index e75dfcc3adc30..6857d8ee94afc 100644 --- a/executor/set_test.go +++ b/executor/set_test.go @@ -268,20 +268,6 @@ func (s *testSuite5) TestSetVar(c *C) { tk.MustExec("set global tidb_constraint_check_in_place = 0") tk.MustQuery(`select @@global.tidb_constraint_check_in_place;`).Check(testkit.Rows("0")) - tk.MustExec("set tidb_slow_log_threshold = 0") - tk.MustQuery("select @@session.tidb_slow_log_threshold;").Check(testkit.Rows("0")) - tk.MustExec("set tidb_slow_log_threshold = 30000") - tk.MustQuery("select @@session.tidb_slow_log_threshold;").Check(testkit.Rows("30000")) - _, err = tk.Exec("set global tidb_slow_log_threshold = 0") - c.Assert(err, NotNil) - - tk.MustExec("set tidb_query_log_max_len = 0") - tk.MustQuery("select @@session.tidb_query_log_max_len;").Check(testkit.Rows("0")) - tk.MustExec("set tidb_query_log_max_len = 20") - tk.MustQuery("select @@session.tidb_query_log_max_len;").Check(testkit.Rows("20")) - _, err = tk.Exec("set global tidb_query_log_max_len = 20") - c.Assert(err, NotNil) - tk.MustExec("set tidb_batch_commit = 0") tk.MustQuery("select @@session.tidb_batch_commit;").Check(testkit.Rows("0")) tk.MustExec("set tidb_batch_commit = 1") @@ -387,11 +373,6 @@ func (s *testSuite5) TestSetVar(c *C) { tk.MustExec("set @@tidb_expensive_query_time_threshold=70") tk.MustQuery("select @@tidb_expensive_query_time_threshold;").Check(testkit.Rows("70")) - tk.MustExec("set @@tidb_record_plan_in_slow_log = 1") - tk.MustQuery("select @@tidb_record_plan_in_slow_log;").Check(testkit.Rows("1")) - tk.MustExec("set @@tidb_record_plan_in_slow_log = 0") - tk.MustQuery("select @@tidb_record_plan_in_slow_log;").Check(testkit.Rows("0")) - tk.MustQuery("select @@tidb_store_limit;").Check(testkit.Rows("0")) tk.MustExec("set @@tidb_store_limit = 100") tk.MustQuery("select @@tidb_store_limit;").Check(testkit.Rows("100")) From e3ef54f80ea3ac7deb6a2c139a905b6f223c52a1 Mon Sep 17 00:00:00 2001 From: Yuanjia Zhang Date: Wed, 12 Feb 2020 21:14:11 +0800 Subject: [PATCH 04/21] fix CI --- executor/adapter_test.go | 12 ------------ 1 file changed, 12 deletions(-) diff --git a/executor/adapter_test.go b/executor/adapter_test.go index 6013f3a95cb64..bf33a3bf4f1be 100644 --- a/executor/adapter_test.go +++ b/executor/adapter_test.go @@ -17,8 +17,6 @@ import ( "time" . "github.com/pingcap/check" - "github.com/pingcap/tidb/config" - "github.com/pingcap/tidb/util/logutil" "github.com/pingcap/tidb/util/testkit" ) @@ -37,13 +35,3 @@ func (s *testSuiteP2) TestQueryTime(c *C) { costTime = time.Since(tk.Se.GetSessionVars().StartTime) c.Assert(costTime < 1*time.Second, IsTrue) } - -func (s testSuiteP2) TestTurnOffSlowLog(c *C) { - tk := testkit.NewTestKit(c, s.store) - - c.Assert(config.GetGlobalConfig().Log.EnableSlowLog, Equals, uint32(logutil.DefaultTiDBEnableSlowLog)) - tk.MustExec("set @@tidb_enable_slow_log=0") - c.Assert(config.GetGlobalConfig().Log.EnableSlowLog, Equals, uint32(0)) - tk.MustExec("set @@tidb_enable_slow_log=1") - c.Assert(config.GetGlobalConfig().Log.EnableSlowLog, Equals, uint32(logutil.DefaultTiDBEnableSlowLog)) -} From 2a4160eab77abe79485b34920fd0c642c62bd755 Mon Sep 17 00:00:00 2001 From: Yuanjia Zhang Date: Thu, 13 Feb 2020 18:28:58 +0800 Subject: [PATCH 05/21] address comments --- config/config.go | 11 +++++++---- config/config_handler.go | 14 +++----------- config/config_handler_test.go | 14 +++++++------- 3 files changed, 17 insertions(+), 22 deletions(-) diff --git a/config/config.go b/config/config.go index 7912a88f0cf96..f0fc570d3eb8d 100644 --- a/config/config.go +++ b/config/config.go @@ -649,7 +649,10 @@ func isDeprecatedConfigItem(items []string) bool { } // InitializeConfig initialize the global config handler. -func InitializeConfig(confPath string, configCheck, configStrict bool, reloadFunc ConfReloadFunc, overwriteFunc OverwriteFunc) { +// The function mergeCmdArgs is used to merge the config file with command arguments: +// For example, if you start TiDB by the command "./tidb-server --port=3000", the port number should be +// overwritten to 3000 and ignore the port number in the config file. +func InitializeConfig(confPath string, configCheck, configStrict bool, reloadFunc ConfReloadFunc, mergeCmdArgs func(*Config)) { cfg := GetGlobalConfig() var err error if confPath != "" { @@ -657,7 +660,7 @@ func InitializeConfig(confPath string, configCheck, configStrict bool, reloadFun if err == nil { return } - // Unused config item erro turns to warnings. + // Unused config item error turns to warnings. if tmp, ok := err.(*ErrConfigValidationFailed); ok { if isDeprecatedConfigItem(tmp.UndecodedItems) { fmt.Fprintln(os.Stderr, err.Error()) @@ -681,8 +684,8 @@ func InitializeConfig(confPath string, configCheck, configStrict bool, reloadFun os.Exit(1) } } - overwriteFunc(cfg) - globalConfHandler, err = NewConfHandler(cfg, reloadFunc, overwriteFunc) + mergeCmdArgs(cfg) + globalConfHandler, err = NewConfHandler(cfg, reloadFunc) terror.MustNil(err) globalConfHandler.Start() } diff --git a/config/config_handler.go b/config/config_handler.go index f7d1b7938480f..ba8c4aa964821 100644 --- a/config/config_handler.go +++ b/config/config_handler.go @@ -41,15 +41,11 @@ type ConfHandler interface { // ConfReloadFunc is used to reload the config to make it work. type ConfReloadFunc func(oldConf, newConf *Config) -// OverwriteFunc is used to overwrite some config items which are initialized from commend and -// shouldn't be updated during runtime. -type OverwriteFunc func(conf *Config) - // NewConfHandler creates a new ConfHandler according to the local config. -func NewConfHandler(localConf *Config, reloadFunc ConfReloadFunc, overwriteFunc OverwriteFunc) (ConfHandler, error) { +func NewConfHandler(localConf *Config, reloadFunc ConfReloadFunc) (ConfHandler, error) { switch defaultConf.Store { case "tikv": - return newPDConfHandler(localConf, reloadFunc, overwriteFunc, nil) + return newPDConfHandler(localConf, reloadFunc, nil) default: return &constantConfHandler{localConf}, nil } @@ -86,11 +82,9 @@ type pdConfHandler struct { exit chan struct{} pdConfCli pd.ConfigClient reloadFunc func(oldConf, newConf *Config) - - overwriteFunc OverwriteFunc } -func newPDConfHandler(localConf *Config, reloadFunc ConfReloadFunc, overwriteFunc OverwriteFunc, +func newPDConfHandler(localConf *Config, reloadFunc ConfReloadFunc, newPDCliFunc func([]string, pd.SecurityOption) (pd.ConfigClient, error), // for test ) (*pdConfHandler, error) { addresses, _, err := ParsePath(localConf.Path) @@ -150,8 +144,6 @@ func newPDConfHandler(localConf *Config, reloadFunc ConfReloadFunc, overwriteFun exit: make(chan struct{}), pdConfCli: pdCli, reloadFunc: reloadFunc, - - overwriteFunc: overwriteFunc, } ch.curConf.Store(newConf) return ch, nil diff --git a/config/config_handler_test.go b/config/config_handler_test.go index 0c0c554297523..1c70dde2061f5 100644 --- a/config/config_handler_test.go +++ b/config/config_handler_test.go @@ -64,7 +64,7 @@ func (mc *mockPDConfigClient) Close() {} func (s *testConfigSuite) TestConstantConfHandler(c *C) { conf := defaultConf conf.Store = "mock" - ch, err := NewConfHandler(&conf, nil, nil) + ch, err := NewConfHandler(&conf, nil) c.Assert(err, IsNil) _, ok := ch.(*constantConfHandler) c.Assert(ok, IsTrue) @@ -77,27 +77,27 @@ func (s *testConfigSuite) TestPDConfHandler(c *C) { // wrong path conf.Store = "tikv" conf.Path = "WRONGPATH" - _, err := newPDConfHandler(&conf, nil, nil, newMockPDConfigClient) + _, err := newPDConfHandler(&conf, nil, newMockPDConfigClient) c.Assert(err, NotNil) // error when creating PD config client conf.Path = "tikv://node1:2379" newMockPDConfigClientErr = fmt.Errorf("") - _, err = newPDConfHandler(&conf, nil, nil, newMockPDConfigClient) + _, err = newPDConfHandler(&conf, nil, newMockPDConfigClient) c.Assert(err, NotNil) // error when registering newMockPDConfigClientErr = nil mockPDConfigClient0.err = fmt.Errorf("") mockPDConfigClient0.confContent.Store("") - ch, err := newPDConfHandler(&conf, nil, nil, newMockPDConfigClient) + ch, err := newPDConfHandler(&conf, nil, newMockPDConfigClient) c.Assert(err, IsNil) // the local config will be used ch.Close() // wrong response when registering mockPDConfigClient0.err = nil mockPDConfigClient0.status = &configpb.Status{Code: configpb.StatusCode_UNKNOWN} - ch, err = newPDConfHandler(&conf, nil, nil, newMockPDConfigClient) + ch, err = newPDConfHandler(&conf, nil, newMockPDConfigClient) c.Assert(err, IsNil) ch.Close() @@ -105,7 +105,7 @@ func (s *testConfigSuite) TestPDConfHandler(c *C) { mockPDConfigClient0.status.Code = configpb.StatusCode_WRONG_VERSION content, _ := encodeConfig(&conf) mockPDConfigClient0.confContent.Store(content) - ch, err = newPDConfHandler(&conf, nil, nil, newMockPDConfigClient) + ch, err = newPDConfHandler(&conf, nil, newMockPDConfigClient) c.Assert(err, IsNil) ch.Close() @@ -117,7 +117,7 @@ func (s *testConfigSuite) TestPDConfHandler(c *C) { c.Assert(oldConf.Log.Level, Equals, "info") c.Assert(newConf.Log.Level, Equals, "debug") } - ch, err = newPDConfHandler(&conf, mockReloadFunc, nil, newMockPDConfigClient) + ch, err = newPDConfHandler(&conf, mockReloadFunc, newMockPDConfigClient) c.Assert(err, IsNil) ch.interval = time.Second ch.Start() From 89e0182eac3f09989caaf2e9a7dfb50ae00a8dec Mon Sep 17 00:00:00 2001 From: Yuanjia Zhang Date: Thu, 13 Feb 2020 18:34:21 +0800 Subject: [PATCH 06/21] address comments --- config/config.go | 4 ++-- config/config_handler.go | 3 +-- tidb-server/main.go | 1 + 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/config/config.go b/config/config.go index f0fc570d3eb8d..12c0cc6b53624 100644 --- a/config/config.go +++ b/config/config.go @@ -639,7 +639,7 @@ var deprecatedConfig = map[string]struct{}{ "log.rotate": {}, } -func isDeprecatedConfigItem(items []string) bool { +func isAllDeprecatedConfigItems(items []string) bool { for _, item := range items { if _, ok := deprecatedConfig[item]; !ok { return false @@ -662,7 +662,7 @@ func InitializeConfig(confPath string, configCheck, configStrict bool, reloadFun } // Unused config item error turns to warnings. if tmp, ok := err.(*ErrConfigValidationFailed); ok { - if isDeprecatedConfigItem(tmp.UndecodedItems) { + if isAllDeprecatedConfigItems(tmp.UndecodedItems) { fmt.Fprintln(os.Stderr, err.Error()) err = nil } diff --git a/config/config_handler.go b/config/config_handler.go index ba8c4aa964821..006652b48b2a6 100644 --- a/config/config_handler.go +++ b/config/config_handler.go @@ -43,7 +43,7 @@ type ConfReloadFunc func(oldConf, newConf *Config) // NewConfHandler creates a new ConfHandler according to the local config. func NewConfHandler(localConf *Config, reloadFunc ConfReloadFunc) (ConfHandler, error) { - switch defaultConf.Store { + switch localConf.Store { case "tikv": return newPDConfHandler(localConf, reloadFunc, nil) default: @@ -52,7 +52,6 @@ func NewConfHandler(localConf *Config, reloadFunc ConfReloadFunc) (ConfHandler, } // constantConfHandler is used in local or debug environment. -// It always returns the constant config initialized at the beginning. type constantConfHandler struct { conf *Config } diff --git a/tidb-server/main.go b/tidb-server/main.go index bada11f08c8cb..de5d6c6dc2a2e 100644 --- a/tidb-server/main.go +++ b/tidb-server/main.go @@ -364,6 +364,7 @@ func reloadConfig(nc, c *config.Config) { } } +// overrideConfig considers command arguments and overrides some config items in the Config. func overrideConfig(cfg *config.Config) { actualFlags := make(map[string]bool) flag.Visit(func(f *flag.Flag) { From 20babaf773c0935b847f2fbd742723ca18e07f36 Mon Sep 17 00:00:00 2001 From: Yuanjia Zhang Date: Thu, 13 Feb 2020 18:36:32 +0800 Subject: [PATCH 07/21] address comments --- config/config.go | 32 +++++++++++++++----------------- 1 file changed, 15 insertions(+), 17 deletions(-) diff --git a/config/config.go b/config/config.go index 12c0cc6b53624..b1dfcf2ba3eaa 100644 --- a/config/config.go +++ b/config/config.go @@ -656,23 +656,21 @@ func InitializeConfig(confPath string, configCheck, configStrict bool, reloadFun cfg := GetGlobalConfig() var err error if confPath != "" { - err = cfg.Load(confPath) - if err == nil { - return - } - // Unused config item error turns to warnings. - if tmp, ok := err.(*ErrConfigValidationFailed); ok { - if isAllDeprecatedConfigItems(tmp.UndecodedItems) { - fmt.Fprintln(os.Stderr, err.Error()) - err = nil - } - // This block is to accommodate an interim situation where strict config checking - // is not the default behavior of TiDB. The warning message must be deferred until - // logging has been set up. After strict config checking is the default behavior, - // This should all be removed. - if !configCheck && !configStrict { - fmt.Fprintln(os.Stderr, err.Error()) - err = nil + if err = cfg.Load(confPath); err != nil { + // Unused config item error turns to warnings. + if tmp, ok := err.(*ErrConfigValidationFailed); ok { + if isAllDeprecatedConfigItems(tmp.UndecodedItems) { + fmt.Fprintln(os.Stderr, err.Error()) + err = nil + } + // This block is to accommodate an interim situation where strict config checking + // is not the default behavior of TiDB. The warning message must be deferred until + // logging has been set up. After strict config checking is the default behavior, + // This should all be removed. + if !configCheck && !configStrict { + fmt.Fprintln(os.Stderr, err.Error()) + err = nil + } } } From cc3cf4877ed99d6e9aade7f69e888c1c613ffd5e Mon Sep 17 00:00:00 2001 From: Yuanjia Zhang Date: Fri, 14 Feb 2020 15:40:31 +0800 Subject: [PATCH 08/21] fixup --- config/config_handler.go | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/config/config_handler.go b/config/config_handler.go index 006652b48b2a6..2b9ddb3ae6538 100644 --- a/config/config_handler.go +++ b/config/config_handler.go @@ -86,7 +86,8 @@ type pdConfHandler struct { func newPDConfHandler(localConf *Config, reloadFunc ConfReloadFunc, newPDCliFunc func([]string, pd.SecurityOption) (pd.ConfigClient, error), // for test ) (*pdConfHandler, error) { - addresses, _, err := ParsePath(localConf.Path) + fullPath := fmt.Sprintf("%s://%s", localConf.Store, localConf.Path) + addresses, _, err := ParsePath(fullPath) if err != nil { return nil, err } From 870d5d44c2a269bd44fc033898560be301c0a242 Mon Sep 17 00:00:00 2001 From: Yuanjia Zhang Date: Fri, 14 Feb 2020 16:22:56 +0800 Subject: [PATCH 09/21] fixup --- config/config_handler.go | 73 +++++++++++++++++++++++----------------- 1 file changed, 42 insertions(+), 31 deletions(-) diff --git a/config/config_handler.go b/config/config_handler.go index 2b9ddb3ae6538..328dd5482d85a 100644 --- a/config/config_handler.go +++ b/config/config_handler.go @@ -81,6 +81,7 @@ type pdConfHandler struct { exit chan struct{} pdConfCli pd.ConfigClient reloadFunc func(oldConf, newConf *Config) + registered bool } func newPDConfHandler(localConf *Config, reloadFunc ConfReloadFunc, @@ -108,44 +109,17 @@ func newPDConfHandler(localConf *Config, reloadFunc ConfReloadFunc, if err != nil { return nil, err } - confContent, err := encodeConfig(localConf) - if err != nil { - return nil, err - } - - // register to PD and get the new default config. - // see https://github.com/pingcap/tidb/pull/13660 for more details. - // suppose port and security config items cannot be change online. - status, version, conf, err := pdCli.Create(context.Background(), new(configpb.Version), tidbComponentName, id, confContent) - if err != nil { - logutil.Logger(context.Background()).Warn("register the config to PD error, local config will be used", zap.Error(err)) - } else if status.Code != configpb.StatusCode_OK && status.Code != configpb.StatusCode_WRONG_VERSION { - logutil.Logger(context.Background()).Warn("invalid status when registering the config to PD", zap.String("code", status.Code.String()), zap.String("errmsg", status.Message)) - conf = "" - } - - tmpConf := *localConf // use the local config if the remote config is invalid - newConf := &tmpConf - if conf != "" { - newConf, err = decodeConfig(conf) - if err != nil { - logutil.Logger(context.Background()).Warn("decode remote config error", zap.Error(err)) - newConf = &tmpConf - } else if err := newConf.Valid(); err != nil { - logutil.Logger(context.Background()).Warn("invalid remote config", zap.Error(err)) - newConf = &tmpConf - } - } - ch := &pdConfHandler{ id: id, - version: version, + version: new(configpb.Version), interval: pdConfHandlerRefreshInterval, exit: make(chan struct{}), pdConfCli: pdCli, reloadFunc: reloadFunc, + registered: false, } - ch.curConf.Store(newConf) + ch.curConf.Store(localConf) // use the local config at first + ch.register() return ch, nil } @@ -168,6 +142,38 @@ func (ch *pdConfHandler) SetConfig(conf *Config) error { return errors.New("PDConfHandler only support to update the config from PD whereas forbid to modify it locally") } +func (ch *pdConfHandler) register() { + // register to PD and get the new default config. + // see https://github.com/pingcap/tidb/pull/13660 for more details. + // suppose port and security config items cannot be change online. + confContent, err := encodeConfig(ch.curConf.Load().(*Config)) + if err != nil { + logutil.Logger(context.Background()).Warn("encode config error when registering", zap.Error(err)) + return + } + + status, version, conf, err := ch.pdConfCli.Create(context.Background(), new(configpb.Version), tidbComponentName, ch.id, confContent) + if err != nil { + logutil.Logger(context.Background()).Warn("RPC to PD error when registering", zap.Error(err)) + return + } else if status.Code != configpb.StatusCode_OK && status.Code != configpb.StatusCode_WRONG_VERSION { + logutil.Logger(context.Background()).Warn("invalid status when registering", zap.String("code", status.Code.String()), zap.String("errmsg", status.Message)) + return + } + + newConf, err := decodeConfig(conf) + if err != nil { + logutil.Logger(context.Background()).Warn("decode config error when registering", zap.Error(err)) + return + } else if err := newConf.Valid(); err != nil { + logutil.Logger(context.Background()).Warn("invalid remote config when registering", zap.Error(err)) + return + } + + ch.registered = true + ch.version = version +} + func (ch *pdConfHandler) run() { defer func() { if r := recover(); r != nil { @@ -181,6 +187,11 @@ func (ch *pdConfHandler) run() { for { select { case <-time.After(ch.interval): + if !ch.registered { + ch.register() + continue + } + // fetch new config from PD status, version, newConfContent, err := ch.pdConfCli.Get(context.Background(), ch.version, tidbComponentName, ch.id) if err != nil { From 943f4e63b1c08289e7d70d34d6314d5d6fad008c Mon Sep 17 00:00:00 2001 From: Yuanjia Zhang Date: Fri, 14 Feb 2020 16:32:18 +0800 Subject: [PATCH 10/21] update log --- config/config_handler.go | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/config/config_handler.go b/config/config_handler.go index 328dd5482d85a..54f24c9914a04 100644 --- a/config/config_handler.go +++ b/config/config_handler.go @@ -219,9 +219,10 @@ func (ch *pdConfHandler) run() { ch.reloadFunc(ch.curConf.Load().(*Config), newConf) ch.curConf.Store(newConf) - logutil.Logger(context.Background()).Info("PDConfHandler update config successfully", - zap.String("fromVersion", ch.version.String()), zap.String("toVersion", version.String())) ch.version = version + logutil.Logger(context.Background()).Info("PDConfHandler update config successfully", + zap.String("fromVersion", ch.version.String()), zap.String("toVersion", version.String()), + zap.String("new_config", newConfContent)) case <-ch.exit: return } From ab5f70a6ba0fb81510812124cfdacc6a8227c22e Mon Sep 17 00:00:00 2001 From: Yuanjia Zhang Date: Fri, 14 Feb 2020 16:37:44 +0800 Subject: [PATCH 11/21] add a switch --- config/config.go | 4 ++++ config/config.toml.example | 4 ++++ config/config_handler.go | 7 +++---- 3 files changed, 11 insertions(+), 4 deletions(-) diff --git a/config/config.go b/config/config.go index b1dfcf2ba3eaa..ee5914bb12e2e 100644 --- a/config/config.go +++ b/config/config.go @@ -109,6 +109,9 @@ type Config struct { MaxServerConnections uint32 `toml:"max-server-connections" json:"max-server-connections"` Experimental Experimental `toml:"experimental" json:"experimental"` + // EnableDynamicConfig enables the TiDB to fetch configs from PD and update itself during runtime. + // see https://github.com/pingcap/tidb/pull/13660 for more details. + EnableDynamicConfig bool `toml:"enable-dynamic-config" json:"enable-dynamic-config"` } // nullableBool defaults unset bool options to unset instead of false, which enables us to know if the user has set 2 @@ -608,6 +611,7 @@ var defaultConf = Config{ Experimental: Experimental{ AllowAutoRandom: false, }, + EnableDynamicConfig: false, } var ( diff --git a/config/config.toml.example b/config/config.toml.example index 3c5c567747a5f..9481c42b668b5 100644 --- a/config/config.toml.example +++ b/config/config.toml.example @@ -372,6 +372,10 @@ history-size = 24 # enable column attribute `auto_random` to be defined on the primary key column. allow-auto-random = false +# enable the TiDB to fetch configs from PD and update itself during runtime. +# see https://github.com/pingcap/tidb/pull/13660 for more details. +enable-dynamic-config = false + # server level isolation read by engines and labels [isolation-read] # engines means allow the tidb server read data from which types of engines. options: "tikv", "tiflash", "tidb". diff --git a/config/config_handler.go b/config/config_handler.go index 54f24c9914a04..873f1ad0be80f 100644 --- a/config/config_handler.go +++ b/config/config_handler.go @@ -17,6 +17,7 @@ import ( "bytes" "context" "fmt" + "strings" "sync" "sync/atomic" "time" @@ -43,12 +44,10 @@ type ConfReloadFunc func(oldConf, newConf *Config) // NewConfHandler creates a new ConfHandler according to the local config. func NewConfHandler(localConf *Config, reloadFunc ConfReloadFunc) (ConfHandler, error) { - switch localConf.Store { - case "tikv": + if strings.ToLower(localConf.Store) == "tikv" && localConf.EnableDynamicConfig { return newPDConfHandler(localConf, reloadFunc, nil) - default: - return &constantConfHandler{localConf}, nil } + return &constantConfHandler{localConf}, nil } // constantConfHandler is used in local or debug environment. From 2febee71c5cd1ed3f1871a427fde80a46e43c1d7 Mon Sep 17 00:00:00 2001 From: Yuanjia Zhang Date: Fri, 14 Feb 2020 16:39:48 +0800 Subject: [PATCH 12/21] add log --- config/config_handler.go | 1 + 1 file changed, 1 insertion(+) diff --git a/config/config_handler.go b/config/config_handler.go index 873f1ad0be80f..4a4b2aece6e62 100644 --- a/config/config_handler.go +++ b/config/config_handler.go @@ -171,6 +171,7 @@ func (ch *pdConfHandler) register() { ch.registered = true ch.version = version + logutil.Logger(context.Background()).Info("register the config to PD successful", zap.String("new_config", conf)) } func (ch *pdConfHandler) run() { From 1928ae4d7198f84600d240bd2d5f1b53168ae788 Mon Sep 17 00:00:00 2001 From: Yuanjia Zhang Date: Fri, 14 Feb 2020 16:44:05 +0800 Subject: [PATCH 13/21] update config --- config/config.toml.example | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/config/config.toml.example b/config/config.toml.example index 9481c42b668b5..cf12770715bd8 100644 --- a/config/config.toml.example +++ b/config/config.toml.example @@ -69,6 +69,10 @@ delay-clean-table-lock = 0 # Maximum number of the splitting region, which is used by the split region statement. split-region-max-num = 1000 +# enable the TiDB to fetch configs from PD and update itself during runtime. +# see https://github.com/pingcap/tidb/pull/13660 for more details. +enable-dynamic-config = false + # alter-primary-key is used to control alter primary key feature. Default is false, indicate the alter primary key feature is disabled. # If it is true, we can add the primary key by "alter table", but we may not be able to drop the primary key. # In order to support "drop primary key" operation , this flag must be true and the table does not have the pkIsHandle flag. @@ -372,10 +376,6 @@ history-size = 24 # enable column attribute `auto_random` to be defined on the primary key column. allow-auto-random = false -# enable the TiDB to fetch configs from PD and update itself during runtime. -# see https://github.com/pingcap/tidb/pull/13660 for more details. -enable-dynamic-config = false - # server level isolation read by engines and labels [isolation-read] # engines means allow the tidb server read data from which types of engines. options: "tikv", "tiflash", "tidb". From 5c725460f68512866fcfcd618cb82ae8e4bf753b Mon Sep 17 00:00:00 2001 From: Yuanjia Zhang Date: Fri, 14 Feb 2020 17:03:23 +0800 Subject: [PATCH 14/21] fix CI --- config/config_handler_test.go | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/config/config_handler_test.go b/config/config_handler_test.go index 1c70dde2061f5..e1976ad17466e 100644 --- a/config/config_handler_test.go +++ b/config/config_handler_test.go @@ -75,13 +75,14 @@ func (s *testConfigSuite) TestPDConfHandler(c *C) { conf := defaultConf // wrong path - conf.Store = "tikv" + conf.Store = "WRONGPATH" conf.Path = "WRONGPATH" _, err := newPDConfHandler(&conf, nil, newMockPDConfigClient) c.Assert(err, NotNil) // error when creating PD config client - conf.Path = "tikv://node1:2379" + conf.Store = "tikv" + conf.Path = "node1:2379" newMockPDConfigClientErr = fmt.Errorf("") _, err = newPDConfHandler(&conf, nil, newMockPDConfigClient) c.Assert(err, NotNil) From 5a5b4747ff55e7748411c6f6a5cabf8cbd02cd2c Mon Sep 17 00:00:00 2001 From: Yuanjia Zhang Date: Mon, 17 Feb 2020 14:16:18 +0800 Subject: [PATCH 15/21] address comments --- config/config.go | 6 +++--- config/config_handler.go | 17 +++++++++++++---- 2 files changed, 16 insertions(+), 7 deletions(-) diff --git a/config/config.go b/config/config.go index 681422c254c87..fe5050e9357b5 100644 --- a/config/config.go +++ b/config/config.go @@ -653,10 +653,10 @@ func isAllDeprecatedConfigItems(items []string) bool { } // InitializeConfig initialize the global config handler. -// The function mergeCmdArgs is used to merge the config file with command arguments: +// The function enforceCmdArgs is used to merge the config file with command arguments: // For example, if you start TiDB by the command "./tidb-server --port=3000", the port number should be // overwritten to 3000 and ignore the port number in the config file. -func InitializeConfig(confPath string, configCheck, configStrict bool, reloadFunc ConfReloadFunc, mergeCmdArgs func(*Config)) { +func InitializeConfig(confPath string, configCheck, configStrict bool, reloadFunc ConfReloadFunc, enforceCmdArgs func(*Config)) { cfg := GetGlobalConfig() var err error if confPath != "" { @@ -686,7 +686,7 @@ func InitializeConfig(confPath string, configCheck, configStrict bool, reloadFun os.Exit(1) } } - mergeCmdArgs(cfg) + enforceCmdArgs(cfg) globalConfHandler, err = NewConfHandler(cfg, reloadFunc) terror.MustNil(err) globalConfHandler.Start() diff --git a/config/config_handler.go b/config/config_handler.go index 4a4b2aece6e62..28477a7e5d920 100644 --- a/config/config_handler.go +++ b/config/config_handler.go @@ -50,7 +50,8 @@ func NewConfHandler(localConf *Config, reloadFunc ConfReloadFunc) (ConfHandler, return &constantConfHandler{localConf}, nil } -// constantConfHandler is used in local or debug environment. +// constantConfHandler is used when EnableDynamicConfig is false.a +// The conf in it will always be the configuration that initialized when TiDB is started. type constantConfHandler struct { conf *Config } @@ -66,6 +67,10 @@ func (cch *constantConfHandler) SetConfig(conf *Config) error { return nil } +var ( + unspecifiedVersion = new(configpb.Version) +) + const ( pdConfHandlerRefreshInterval = 30 * time.Second tidbComponentName = "tidb" @@ -110,7 +115,7 @@ func newPDConfHandler(localConf *Config, reloadFunc ConfReloadFunc, } ch := &pdConfHandler{ id: id, - version: new(configpb.Version), + version: unspecifiedVersion, interval: pdConfHandlerRefreshInterval, exit: make(chan struct{}), pdConfCli: pdCli, @@ -170,8 +175,12 @@ func (ch *pdConfHandler) register() { } ch.registered = true + ch.reloadFunc(ch.curConf.Load().(*Config), newConf) + ch.curConf.Store(newConf) ch.version = version - logutil.Logger(context.Background()).Info("register the config to PD successful", zap.String("new_config", conf)) + logutil.Logger(context.Background()).Info("PDConfHandler register config successfully", + zap.String("version", version.String()), + zap.Any("new_config", newConf)) } func (ch *pdConfHandler) run() { @@ -222,7 +231,7 @@ func (ch *pdConfHandler) run() { ch.version = version logutil.Logger(context.Background()).Info("PDConfHandler update config successfully", zap.String("fromVersion", ch.version.String()), zap.String("toVersion", version.String()), - zap.String("new_config", newConfContent)) + zap.Any("new_config", newConf)) case <-ch.exit: return } From 8918ea99668175071ee60714a5d87ec369028852 Mon Sep 17 00:00:00 2001 From: Yuanjia Zhang Date: Mon, 17 Feb 2020 14:51:08 +0800 Subject: [PATCH 16/21] fix CI --- config/config_handler.go | 2 +- config/config_handler_test.go | 11 +++++------ 2 files changed, 6 insertions(+), 7 deletions(-) diff --git a/config/config_handler.go b/config/config_handler.go index 28477a7e5d920..66742e8ab05b2 100644 --- a/config/config_handler.go +++ b/config/config_handler.go @@ -123,7 +123,6 @@ func newPDConfHandler(localConf *Config, reloadFunc ConfReloadFunc, registered: false, } ch.curConf.Store(localConf) // use the local config at first - ch.register() return ch, nil } @@ -193,6 +192,7 @@ func (ch *pdConfHandler) run() { ch.wg.Done() }() + ch.register() // the first time to register for { select { case <-time.After(ch.interval): diff --git a/config/config_handler_test.go b/config/config_handler_test.go index e1976ad17466e..b0420afbe1ee9 100644 --- a/config/config_handler_test.go +++ b/config/config_handler_test.go @@ -106,7 +106,7 @@ func (s *testConfigSuite) TestPDConfHandler(c *C) { mockPDConfigClient0.status.Code = configpb.StatusCode_WRONG_VERSION content, _ := encodeConfig(&conf) mockPDConfigClient0.confContent.Store(content) - ch, err = newPDConfHandler(&conf, nil, newMockPDConfigClient) + ch, err = newPDConfHandler(&conf, func(oldC, newC *Config) {}, newMockPDConfigClient) c.Assert(err, IsNil) ch.Close() @@ -114,19 +114,18 @@ func (s *testConfigSuite) TestPDConfHandler(c *C) { wg := sync.WaitGroup{} wg.Add(1) mockReloadFunc := func(oldConf, newConf *Config) { - wg.Done() - c.Assert(oldConf.Log.Level, Equals, "info") - c.Assert(newConf.Log.Level, Equals, "debug") + c.Assert(oldConf.Log.Level, Equals, "info") + c.Assert(newConf.Log.Level, Equals, "debug") + wg.Done() } ch, err = newPDConfHandler(&conf, mockReloadFunc, newMockPDConfigClient) c.Assert(err, IsNil) ch.interval = time.Second - ch.Start() - c.Assert(ch.GetConfig().Log.Level, Equals, "info") newConf := conf newConf.Log.Level = "debug" newContent, _ := encodeConfig(&newConf) mockPDConfigClient0.confContent.Store(newContent) + ch.Start() wg.Wait() c.Assert(ch.GetConfig().Log.Level, Equals, "debug") ch.Close() From 2e94c9b05a6f046ea421f6b3af98c7ff4ef94688 Mon Sep 17 00:00:00 2001 From: Yuanjia Zhang Date: Mon, 17 Feb 2020 15:02:19 +0800 Subject: [PATCH 17/21] add tests --- config/config.go | 2 +- config/config_handler.go | 6 ++++-- config/config_handler_test.go | 28 +++++++++++++++++++++++----- 3 files changed, 28 insertions(+), 8 deletions(-) diff --git a/config/config.go b/config/config.go index fe5050e9357b5..91265d16c56ba 100644 --- a/config/config.go +++ b/config/config.go @@ -687,7 +687,7 @@ func InitializeConfig(confPath string, configCheck, configStrict bool, reloadFun } } enforceCmdArgs(cfg) - globalConfHandler, err = NewConfHandler(cfg, reloadFunc) + globalConfHandler, err = NewConfHandler(cfg, reloadFunc, nil) terror.MustNil(err) globalConfHandler.Start() } diff --git a/config/config_handler.go b/config/config_handler.go index 66742e8ab05b2..6532c57bf4f99 100644 --- a/config/config_handler.go +++ b/config/config_handler.go @@ -43,9 +43,11 @@ type ConfHandler interface { type ConfReloadFunc func(oldConf, newConf *Config) // NewConfHandler creates a new ConfHandler according to the local config. -func NewConfHandler(localConf *Config, reloadFunc ConfReloadFunc) (ConfHandler, error) { +func NewConfHandler(localConf *Config, reloadFunc ConfReloadFunc, + newPDCliFunc func([]string, pd.SecurityOption) (pd.ConfigClient, error), // for test +) (ConfHandler, error) { if strings.ToLower(localConf.Store) == "tikv" && localConf.EnableDynamicConfig { - return newPDConfHandler(localConf, reloadFunc, nil) + return newPDConfHandler(localConf, reloadFunc, newPDCliFunc) } return &constantConfHandler{localConf}, nil } diff --git a/config/config_handler_test.go b/config/config_handler_test.go index b0420afbe1ee9..02137bb40c02e 100644 --- a/config/config_handler_test.go +++ b/config/config_handler_test.go @@ -16,6 +16,7 @@ package config import ( "context" "fmt" + "reflect" "sync" "sync/atomic" "time" @@ -64,7 +65,7 @@ func (mc *mockPDConfigClient) Close() {} func (s *testConfigSuite) TestConstantConfHandler(c *C) { conf := defaultConf conf.Store = "mock" - ch, err := NewConfHandler(&conf, nil) + ch, err := NewConfHandler(&conf, nil, nil) c.Assert(err, IsNil) _, ok := ch.(*constantConfHandler) c.Assert(ok, IsTrue) @@ -106,7 +107,7 @@ func (s *testConfigSuite) TestPDConfHandler(c *C) { mockPDConfigClient0.status.Code = configpb.StatusCode_WRONG_VERSION content, _ := encodeConfig(&conf) mockPDConfigClient0.confContent.Store(content) - ch, err = newPDConfHandler(&conf, func(oldC, newC *Config) {}, newMockPDConfigClient) + ch, err = newPDConfHandler(&conf, nil, newMockPDConfigClient) c.Assert(err, IsNil) ch.Close() @@ -114,9 +115,9 @@ func (s *testConfigSuite) TestPDConfHandler(c *C) { wg := sync.WaitGroup{} wg.Add(1) mockReloadFunc := func(oldConf, newConf *Config) { - c.Assert(oldConf.Log.Level, Equals, "info") - c.Assert(newConf.Log.Level, Equals, "debug") - wg.Done() + c.Assert(oldConf.Log.Level, Equals, "info") + c.Assert(newConf.Log.Level, Equals, "debug") + wg.Done() } ch, err = newPDConfHandler(&conf, mockReloadFunc, newMockPDConfigClient) c.Assert(err, IsNil) @@ -130,3 +131,20 @@ func (s *testConfigSuite) TestPDConfHandler(c *C) { c.Assert(ch.GetConfig().Log.Level, Equals, "debug") ch.Close() } + +func (s *testConfigSuite) TestEnableDynamicConfig(c *C) { + conf := &defaultConf + for _, store := range []string{"tikv", "mocktikv"} { + for _, enable := range []bool{true, false} { + conf.Store = store + conf.EnableDynamicConfig = enable + ch, err := NewConfHandler(conf, nil, newMockPDConfigClient) + c.Assert(err, IsNil) + if store == "tikv" && enable == true { + c.Assert(fmt.Sprintf("%v", reflect.TypeOf(ch)), Equals, "*config.pdConfHandler") + } else { + c.Assert(fmt.Sprintf("%v", reflect.TypeOf(ch)), Equals, "*config.constantConfHandler") + } + } + } +} From 45febfcb9619f603f18db7586cb9af3322395955 Mon Sep 17 00:00:00 2001 From: Yuanjia Zhang Date: Mon, 17 Feb 2020 15:39:42 +0800 Subject: [PATCH 18/21] address comments --- config/config_handler.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/config/config_handler.go b/config/config_handler.go index 6532c57bf4f99..dad63b34bc1b2 100644 --- a/config/config_handler.go +++ b/config/config_handler.go @@ -52,7 +52,7 @@ func NewConfHandler(localConf *Config, reloadFunc ConfReloadFunc, return &constantConfHandler{localConf}, nil } -// constantConfHandler is used when EnableDynamicConfig is false.a +// constantConfHandler is used when EnableDynamicConfig is false. // The conf in it will always be the configuration that initialized when TiDB is started. type constantConfHandler struct { conf *Config @@ -230,10 +230,10 @@ func (ch *pdConfHandler) run() { ch.reloadFunc(ch.curConf.Load().(*Config), newConf) ch.curConf.Store(newConf) - ch.version = version logutil.Logger(context.Background()).Info("PDConfHandler update config successfully", zap.String("fromVersion", ch.version.String()), zap.String("toVersion", version.String()), zap.Any("new_config", newConf)) + ch.version = version case <-ch.exit: return } From 8b199ebd9c72c67f20ee275ed459dceb29c286ae Mon Sep 17 00:00:00 2001 From: Yuanjia Zhang Date: Mon, 17 Feb 2020 17:48:47 +0800 Subject: [PATCH 19/21] fix DATA RACE --- config/config.go | 3 ++- config/config_handler.go | 10 ++++++---- 2 files changed, 8 insertions(+), 5 deletions(-) diff --git a/config/config.go b/config/config.go index 9cc43ab635321..f09854dcc3ff5 100644 --- a/config/config.go +++ b/config/config.go @@ -839,7 +839,8 @@ func (t *OpenTracing) ToTracingConfig() *tracing.Configuration { func init() { conf := defaultConf - globalConfHandler = &constantConfHandler{&conf} + globalConfHandler = new(constantConfHandler) + globalConfHandler.SetConfig(&conf) if checkBeforeDropLDFlag == "1" { CheckTableBeforeDrop = true } diff --git a/config/config_handler.go b/config/config_handler.go index dad63b34bc1b2..37e927dd160a8 100644 --- a/config/config_handler.go +++ b/config/config_handler.go @@ -49,23 +49,25 @@ func NewConfHandler(localConf *Config, reloadFunc ConfReloadFunc, if strings.ToLower(localConf.Store) == "tikv" && localConf.EnableDynamicConfig { return newPDConfHandler(localConf, reloadFunc, newPDCliFunc) } - return &constantConfHandler{localConf}, nil + cch := new(constantConfHandler) + cch.curConf.Store(localConf) + return cch, nil } // constantConfHandler is used when EnableDynamicConfig is false. // The conf in it will always be the configuration that initialized when TiDB is started. type constantConfHandler struct { - conf *Config + curConf atomic.Value } func (cch *constantConfHandler) Start() {} func (cch *constantConfHandler) Close() {} -func (cch *constantConfHandler) GetConfig() *Config { return cch.conf } +func (cch *constantConfHandler) GetConfig() *Config { return cch.curConf.Load().(*Config) } func (cch *constantConfHandler) SetConfig(conf *Config) error { - cch.conf = conf + cch.curConf.Store(conf) return nil } From 34aa1c856ae29f4191ec6487a86f31c9b88de8c4 Mon Sep 17 00:00:00 2001 From: Yuanjia Zhang Date: Mon, 17 Feb 2020 18:10:12 +0800 Subject: [PATCH 20/21] make linter happy --- config/config.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/config/config.go b/config/config.go index f09854dcc3ff5..5b887c6f5a02b 100644 --- a/config/config.go +++ b/config/config.go @@ -840,7 +840,7 @@ func (t *OpenTracing) ToTracingConfig() *tracing.Configuration { func init() { conf := defaultConf globalConfHandler = new(constantConfHandler) - globalConfHandler.SetConfig(&conf) + _ = globalConfHandler.SetConfig(&conf) if checkBeforeDropLDFlag == "1" { CheckTableBeforeDrop = true } From 1700c4b06fcce02c8dc343c457aa121974966180 Mon Sep 17 00:00:00 2001 From: Yuanjia Zhang Date: Mon, 17 Feb 2020 18:15:15 +0800 Subject: [PATCH 21/21] make linter happy --- config/config.go | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/config/config.go b/config/config.go index 5b887c6f5a02b..4d4b0bf39cdd0 100644 --- a/config/config.go +++ b/config/config.go @@ -839,8 +839,9 @@ func (t *OpenTracing) ToTracingConfig() *tracing.Configuration { func init() { conf := defaultConf - globalConfHandler = new(constantConfHandler) - _ = globalConfHandler.SetConfig(&conf) + cch := new(constantConfHandler) + cch.curConf.Store(&conf) + globalConfHandler = cch if checkBeforeDropLDFlag == "1" { CheckTableBeforeDrop = true }