Skip to content

Commit

Permalink
*: make config items Performance.*, OOMAction and MemQuotaQuery
Browse files Browse the repository at this point in the history
… support hot-reloading (#10295)
  • Loading branch information
qw4990 authored and shenli committed Apr 29, 2019
1 parent e96190b commit 69b02a3
Show file tree
Hide file tree
Showing 19 changed files with 279 additions and 112 deletions.
141 changes: 139 additions & 2 deletions config/config.go
Original file line number Diff line number Diff line change
Expand Up @@ -14,17 +14,23 @@
package config

import (
"bytes"
"context"
"crypto/tls"
"crypto/x509"
"fmt"
"io/ioutil"
"os"
"reflect"
"strings"
"sync"
"time"

"github.com/BurntSushi/toml"
"github.com/pingcap/errors"
"github.com/pingcap/tidb/util/logutil"
tracing "github.com/uber/jaeger-client-go/config"
"go.uber.org/atomic"
)

// Config number limitations
Expand Down Expand Up @@ -370,19 +376,104 @@ var defaultConf = Config{
},
}

var globalConf = defaultConf
var (
globalConf = atomic.Value{}
reloadConfPath = ""
confReloader func(nc, c *Config)
confReloadLock sync.Mutex
supportedReloadConfigs = make(map[string]struct{}, 32)
supportedReloadConfList = make([]string, 0, 32)
)

// NewConfig creates a new config instance with default value.
func NewConfig() *Config {
conf := defaultConf
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
return globalConf.Load().(*Config)
}

// 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()

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)
}
}
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.Logger(context.Background()).Info("reload config changes" + formattedDiff.String())
return nil
}

// 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
}
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
}
m := collectsDiff(v1.Field(i).Interface(), v2.Field(i).Interface(), p)
for k, v := range m {
diff[k] = v
}
}
return diff
}

// Load loads config options from a toml file.
Expand All @@ -406,6 +497,51 @@ func (c *Config) Load(confFile string) error {
return err
}

// Valid checks if this config is valid.
func (c *Config) Valid() error {
if c.Security.SkipGrantTable && !hasRootPrivilege() {
return fmt.Errorf("TiDB run with skip-grant-table need root privilege")
}
if _, ok := ValidStorage[c.Store]; !ok {
nameList := make([]string, 0, len(ValidStorage))
for k, v := range ValidStorage {
if v {
nameList = append(nameList, k)
}
}
return fmt.Errorf("invalid store=%s, valid storages=%v", c.Store, nameList)
}
if c.Store == "mocktikv" && !c.RunDDL {
return fmt.Errorf("can't disable DDL on mocktikv")
}
if c.Log.File.MaxSize > MaxLogFileSize {
return fmt.Errorf("invalid max log file size=%v which is larger than max=%v", c.Log.File.MaxSize, MaxLogFileSize)
}
c.OOMAction = strings.ToLower(c.OOMAction)

// lower_case_table_names is allowed to be 0, 1, 2
if c.LowerCaseTableNames < 0 || c.LowerCaseTableNames > 2 {
return fmt.Errorf("lower-case-table-names should be 0 or 1 or 2")
}

if c.TxnLocalLatches.Enabled && c.TxnLocalLatches.Capacity == 0 {
return fmt.Errorf("txn-local-latches.capacity can not be 0")
}

// For tikvclient.
if c.TiKVClient.GrpcConnectionCount == 0 {
return fmt.Errorf("grpc-connection-count should be greater than 0")
}
if c.TiKVClient.MaxTxnTimeUse == 0 {
return fmt.Errorf("max-txn-time-use should be greater than 0")
}
return nil
}

func hasRootPrivilege() bool {
return os.Geteuid() == 0
}

// ToLogConfig converts *Log to *logutil.LogConfig.
func (l *Log) ToLogConfig() *logutil.LogConfig {
return logutil.NewLogConfig(l.Level, l.Format, l.SlowQueryFile, l.File, l.DisableTimestamp)
Expand Down Expand Up @@ -433,6 +569,7 @@ func (t *OpenTracing) ToTracingConfig() *tracing.Configuration {
}

func init() {
globalConf.Store(&defaultConf)
if checkBeforeDropLDFlag == "1" {
CheckTableBeforeDrop = true
}
Expand Down
25 changes: 25 additions & 0 deletions config/config_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -169,3 +169,28 @@ xkNuJ2BlEGkwWLiRbKy1lNBBFUXKuhh3L/EIY10WTnr3TQzeL6H1
c.Assert(os.Remove(certFile), IsNil)
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))
}
22 changes: 11 additions & 11 deletions executor/seqtest/prepared_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -45,7 +45,7 @@ func (s *seqTestSuite) TestPrepared(c *C) {
plannercore.PreparedPlanCacheMemoryGuardRatio = 0.1
// PreparedPlanCacheMaxMemory is set to MAX_UINT64 to make sure the cache
// behavior would not be effected by the uncertain memory utilization.
plannercore.PreparedPlanCacheMaxMemory = math.MaxUint64
plannercore.PreparedPlanCacheMaxMemory.Store(math.MaxUint64)
tk := testkit.NewTestKit(c, s.store)
tk.MustExec("use test")
tk.MustExec("drop table if exists prepare_test")
Expand Down Expand Up @@ -262,7 +262,7 @@ func (s *seqTestSuite) TestPreparedLimitOffset(c *C) {
plannercore.PreparedPlanCacheMemoryGuardRatio = 0.1
// PreparedPlanCacheMaxMemory is set to MAX_UINT64 to make sure the cache
// behavior would not be effected by the uncertain memory utilization.
plannercore.PreparedPlanCacheMaxMemory = math.MaxUint64
plannercore.PreparedPlanCacheMaxMemory.Store(math.MaxUint64)
tk := testkit.NewTestKit(c, s.store)
tk.MustExec("use test")
tk.MustExec("drop table if exists prepare_test")
Expand Down Expand Up @@ -305,7 +305,7 @@ func (s *seqTestSuite) TestPreparedNullParam(c *C) {
plannercore.PreparedPlanCacheMemoryGuardRatio = 0.1
// PreparedPlanCacheMaxMemory is set to MAX_UINT64 to make sure the cache
// behavior would not be effected by the uncertain memory utilization.
plannercore.PreparedPlanCacheMaxMemory = math.MaxUint64
plannercore.PreparedPlanCacheMaxMemory.Store(math.MaxUint64)
tk := testkit.NewTestKit(c, s.store)
tk.MustExec("use test")
tk.MustExec("drop table if exists t")
Expand Down Expand Up @@ -349,7 +349,7 @@ func (s *seqTestSuite) TestPrepareWithAggregation(c *C) {
plannercore.PreparedPlanCacheMemoryGuardRatio = 0.1
// PreparedPlanCacheMaxMemory is set to MAX_UINT64 to make sure the cache
// behavior would not be effected by the uncertain memory utilization.
plannercore.PreparedPlanCacheMaxMemory = math.MaxUint64
plannercore.PreparedPlanCacheMaxMemory.Store(math.MaxUint64)
tk := testkit.NewTestKit(c, s.store)
tk.MustExec("use test")
tk.MustExec("drop table if exists t")
Expand Down Expand Up @@ -384,7 +384,7 @@ func (s *seqTestSuite) TestPreparedIssue7579(c *C) {
plannercore.PreparedPlanCacheMemoryGuardRatio = 0.1
// PreparedPlanCacheMaxMemory is set to MAX_UINT64 to make sure the cache
// behavior would not be effected by the uncertain memory utilization.
plannercore.PreparedPlanCacheMaxMemory = math.MaxUint64
plannercore.PreparedPlanCacheMaxMemory.Store(math.MaxUint64)
tk := testkit.NewTestKit(c, s.store)
tk.MustExec("use test")
tk.MustExec("drop table if exists t")
Expand Down Expand Up @@ -439,7 +439,7 @@ func (s *seqTestSuite) TestPreparedInsert(c *C) {
plannercore.PreparedPlanCacheMemoryGuardRatio = 0.1
// PreparedPlanCacheMaxMemory is set to MAX_UINT64 to make sure the cache
// behavior would not be effected by the uncertain memory utilization.
plannercore.PreparedPlanCacheMaxMemory = math.MaxUint64
plannercore.PreparedPlanCacheMaxMemory.Store(math.MaxUint64)
tk := testkit.NewTestKit(c, s.store)
tk.MustExec("use test")
tk.MustExec("drop table if exists prepare_test")
Expand Down Expand Up @@ -522,7 +522,7 @@ func (s *seqTestSuite) TestPreparedUpdate(c *C) {
plannercore.PreparedPlanCacheMemoryGuardRatio = 0.1
// PreparedPlanCacheMaxMemory is set to MAX_UINT64 to make sure the cache
// behavior would not be effected by the uncertain memory utilization.
plannercore.PreparedPlanCacheMaxMemory = math.MaxUint64
plannercore.PreparedPlanCacheMaxMemory.Store(math.MaxUint64)
tk := testkit.NewTestKit(c, s.store)
tk.MustExec("use test")
tk.MustExec("drop table if exists prepare_test")
Expand Down Expand Up @@ -582,7 +582,7 @@ func (s *seqTestSuite) TestPreparedDelete(c *C) {
plannercore.PreparedPlanCacheMemoryGuardRatio = 0.1
// PreparedPlanCacheMaxMemory is set to MAX_UINT64 to make sure the cache
// behavior would not be effected by the uncertain memory utilization.
plannercore.PreparedPlanCacheMaxMemory = math.MaxUint64
plannercore.PreparedPlanCacheMaxMemory.Store(math.MaxUint64)
tk := testkit.NewTestKit(c, s.store)
tk.MustExec("use test")
tk.MustExec("drop table if exists prepare_test")
Expand Down Expand Up @@ -636,7 +636,7 @@ func (s *seqTestSuite) TestPrepareDealloc(c *C) {
plannercore.PreparedPlanCacheMemoryGuardRatio = 0.1
// PreparedPlanCacheMaxMemory is set to MAX_UINT64 to make sure the cache
// behavior would not be effected by the uncertain memory utilization.
plannercore.PreparedPlanCacheMaxMemory = math.MaxUint64
plannercore.PreparedPlanCacheMaxMemory.Store(math.MaxUint64)

tk := testkit.NewTestKit(c, s.store)
tk.MustExec("use test")
Expand Down Expand Up @@ -681,7 +681,7 @@ func (s *seqTestSuite) TestPreparedIssue8153(c *C) {
plannercore.PreparedPlanCacheMemoryGuardRatio = 0.1
// PreparedPlanCacheMaxMemory is set to MAX_UINT64 to make sure the cache
// behavior would not be effected by the uncertain memory utilization.
plannercore.PreparedPlanCacheMaxMemory = math.MaxUint64
plannercore.PreparedPlanCacheMaxMemory.Store(math.MaxUint64)
tk := testkit.NewTestKit(c, s.store)
tk.MustExec("use test")
tk.MustExec("drop table if exists t")
Expand Down Expand Up @@ -739,7 +739,7 @@ func (s *seqTestSuite) TestPreparedIssue8644(c *C) {
plannercore.PreparedPlanCacheMemoryGuardRatio = 0.1
// PreparedPlanCacheMaxMemory is set to MAX_UINT64 to make sure the cache
// behavior would not be effected by the uncertain memory utilization.
plannercore.PreparedPlanCacheMaxMemory = math.MaxUint64
plannercore.PreparedPlanCacheMaxMemory.Store(math.MaxUint64)
tk := testkit.NewTestKit(c, s.store)
tk.MustExec("use test")
tk.MustExec("drop table if exists t")
Expand Down
4 changes: 2 additions & 2 deletions executor/seqtest/seq_executor_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -777,14 +777,14 @@ func (s *seqTestSuite) TestCartesianProduct(c *C) {
tk.MustExec("use test")
tk.MustExec("drop table if exists t")
tk.MustExec("create table t(c1 int)")
plannercore.AllowCartesianProduct = false
plannercore.AllowCartesianProduct.Store(false)
err := tk.ExecToErr("select * from t t1, t t2")
c.Check(plannercore.ErrCartesianProductUnsupported.Equal(err), IsTrue)
err = tk.ExecToErr("select * from t t1 left join t t2 on 1")
c.Check(plannercore.ErrCartesianProductUnsupported.Equal(err), IsTrue)
err = tk.ExecToErr("select * from t t1 right join t t2 on 1")
c.Check(plannercore.ErrCartesianProductUnsupported.Equal(err), IsTrue)
plannercore.AllowCartesianProduct = true
plannercore.AllowCartesianProduct.Store(true)
}

type checkPrioClient struct {
Expand Down
1 change: 1 addition & 0 deletions go.mod
Original file line number Diff line number Diff line change
Expand Up @@ -61,6 +61,7 @@ require (
github.com/uber/jaeger-client-go v2.15.0+incompatible
github.com/uber/jaeger-lib v1.5.0 // indirect
github.com/unrolled/render v0.0.0-20180914162206-b9786414de4d // indirect
go.uber.org/atomic v1.3.2
go.uber.org/zap v1.9.1
golang.org/x/net v0.0.0-20190108225652-1e06a53dbb7e
golang.org/x/sys v0.0.0-20190109145017-48ac38b7c8cb // indirect
Expand Down
3 changes: 2 additions & 1 deletion planner/core/cache.go
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,7 @@ import (
"github.com/pingcap/tidb/util/codec"
"github.com/pingcap/tidb/util/hack"
"github.com/pingcap/tidb/util/kvcache"
atomic2 "go.uber.org/atomic"
)

var (
Expand All @@ -34,7 +35,7 @@ var (
// PreparedPlanCacheMemoryGuardRatio stores the global config "prepared-plan-cache-memory-guard-ratio".
PreparedPlanCacheMemoryGuardRatio float64
// PreparedPlanCacheMaxMemory stores the max memory size defined in the global config "performance-max-memory".
PreparedPlanCacheMaxMemory uint64
PreparedPlanCacheMaxMemory atomic2.Uint64
)

const (
Expand Down
8 changes: 4 additions & 4 deletions planner/core/cbo_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -216,9 +216,9 @@ func (s *testAnalyzeSuite) TestEstimation(c *C) {
defer func() {
dom.Close()
store.Close()
statistics.RatioOfPseudoEstimate = 0.7
statistics.RatioOfPseudoEstimate.Store(0.7)
}()
statistics.RatioOfPseudoEstimate = 10.0
statistics.RatioOfPseudoEstimate.Store(10.0)
testKit.MustExec("use test")
testKit.MustExec("create table t (a int)")
testKit.MustExec("insert into t values (1), (2), (3), (4), (5), (6), (7), (8), (9), (10)")
Expand Down Expand Up @@ -575,13 +575,13 @@ func (s *testAnalyzeSuite) TestOutdatedAnalyze(c *C) {
testKit.MustExec("insert into t select * from t")
h.DumpStatsDeltaToKV(handle.DumpAll)
c.Assert(h.Update(dom.InfoSchema()), IsNil)
statistics.RatioOfPseudoEstimate = 10.0
statistics.RatioOfPseudoEstimate.Store(10.0)
testKit.MustQuery("explain select * from t where a <= 5 and b <= 5").Check(testkit.Rows(
"TableReader_7 35.91 root data:Selection_6",
"└─Selection_6 35.91 cop le(test.t.a, 5), le(test.t.b, 5)",
" └─TableScan_5 80.00 cop table:t, range:[-inf,+inf], keep order:false",
))
statistics.RatioOfPseudoEstimate = 0.7
statistics.RatioOfPseudoEstimate.Store(0.7)
testKit.MustQuery("explain select * from t where a <= 5 and b <= 5").Check(testkit.Rows(
"IndexLookUp_11 8.84 root ",
"├─IndexScan_8 26.59 cop table:t, index:a, range:[-inf,5], keep order:false, stats:pseudo",
Expand Down
Loading

0 comments on commit 69b02a3

Please sign in to comment.