From 97242d362a97187320363b086a849d5f1ee13ee2 Mon Sep 17 00:00:00 2001 From: Ewan Chou Date: Thu, 14 Sep 2017 19:38:49 +0800 Subject: [PATCH] config: add config file support. (#4509) --- config/config.go | 154 ++++++++++-- config/config.toml.example | 113 +++++++++ config/config_test.go | 54 +++++ executor/adapter.go | 6 +- server/http_status.go | 5 +- server/region_handler_test.go | 11 +- server/server.go | 27 ++- server/server_test.go | 4 +- server/tidb_test.go | 63 +++-- tidb-server/main.go | 430 +++++++++++++++++++++------------- x-server/config.go | 1 - 11 files changed, 630 insertions(+), 238 deletions(-) create mode 100644 config/config.toml.example create mode 100644 config/config_test.go diff --git a/config/config.go b/config/config.go index aeb0e165ee778..e75e62ef12cb6 100644 --- a/config/config.go +++ b/config/config.go @@ -13,38 +13,140 @@ package config -import "sync" +import ( + "github.com/BurntSushi/toml" + "github.com/juju/errors" + "github.com/pingcap/pd/pkg/logutil" +) // Config contains configuration options. type Config struct { - Addr string `json:"addr" toml:"addr"` - LogLevel string `json:"log_level" toml:"log_level"` - SkipAuth bool `json:"skip_auth" toml:"skip_auth"` - StatusAddr string `json:"status_addr" toml:"status_addr"` - Socket string `json:"socket" toml:"socket"` - ReportStatus bool `json:"report_status" toml:"report_status"` - StorePath string `json:"store_path" toml:"store_path"` - Store string `json:"store" toml:"store"` - SlowThreshold int `json:"slow_threshold" toml:"slow_threshold"` - QueryLogMaxlen int `json:"query_log_max_len" toml:"query_log_max_len"` - TCPKeepAlive bool `json:"tcp_keep_alive" toml:"tcp_keep_alive"` - SSLCAPath string `json:"ssl_ca_path" toml:"ssl_ca_path"` - SSLCertPath string `json:"ssl_cert_path" toml:"ssl_cert_path"` - SSLKeyPath string `json:"ssl_key_path" toml:"ssl_key_path"` -} - -var cfg *Config -var once sync.Once + Host string `toml:"host" json:"host"` + Port int `toml:"port" json:"port"` + Store string `toml:"store" json:"store"` + Path string `toml:"path" json:"path"` + Socket string `toml:"socket" json:"socket"` + BinlogSocket string `toml:"binlog-socket" json:"binlog-socket"` + Lease string `toml:"lease" json:"lease"` + RunDDL bool `toml:"run-ddl" json:"run-ddl"` + + Log Log `toml:"log" json:"log"` + Security Security `toml:"security" json:"security"` + Status Status `toml:"status" json:"status"` + Performance Performance `toml:"performance" json:"performance"` + XProtocol XProtocol `toml:"xprotocol" json:"xprotocol"` +} + +// Log is the log section of config. +type Log struct { + // Log level. + Level string `toml:"level" json:"level"` + // Log format. one of json, text, or console. + Format string `toml:"format" json:"format"` + // Disable automatic timestamps in output. + DisableTimestamp bool `toml:"disable-timestamp" json:"disable-timestamp"` + // File log config. + File logutil.FileLogConfig `toml:"file" json:"file"` + + SlowThreshold int `toml:"slow-threshold" json:"slow-threshold"` + + QueryLogMaxLen int `toml:"query-log-max-len" json:"query-log-max-len"` +} + +// Security is the security section of the config. +type Security struct { + SkipGrantTable bool `toml:"skip-grant-table" json:"skip-grant-table"` + SSLCA string `toml:"ssl-ca" json:"ssl-ca"` + SSLCert string `toml:"ssl-cert" json:"ssl-cert"` + SSLKey string `toml:"ssl-key" json:"ssl-key"` +} + +// Status is the status section of the config. +type Status struct { + ReportStatus bool `toml:"report-status" json:"report-status"` + StatusPort int `toml:"status-port" json:"status-port"` + MetricsAddr string `toml:"metrics-addr" json:"metrics-addr"` + MetricsInterval int `toml:"metrics-interval" json:"metrics-interval"` +} + +// Performance is the performance section of the config. +type Performance struct { + TCPKeepAlive bool `toml:"tcp-keep-alive" json:"tcp-keep-alive"` + RetryLimit int `toml:"retry-limit" json:"retry-limit"` + JoinConcurrency int `toml:"join-concurrency" json:"join-concurrency"` + CrossJoin bool `toml:"cross-join" json:"cross-join"` + StatsLease string `toml:"stats-lease" json:"stats-lease"` +} + +// XProtocol is the XProtocol section of the config. +type XProtocol struct { + XServer bool `toml:"xserver" json:"xserver"` + XHost string `toml:"xhost" json:"xhost"` + XPort int `toml:"xport" json:"xport"` + XSocket string `toml:"xsocket" json:"xsocket"` +} + +var defaultConf = Config{ + Host: "0.0.0.0", + Port: 4000, + Store: "mocktikv", + Path: "/tmp/tidb", + RunDDL: true, + Lease: "10s", + Log: Log{ + Level: "info", + Format: "text", + File: logutil.FileLogConfig{ + LogRotate: true, + }, + SlowThreshold: 300, + QueryLogMaxLen: 2048, + }, + Status: Status{ + ReportStatus: true, + StatusPort: 10080, + MetricsInterval: 15, + }, + Performance: Performance{ + TCPKeepAlive: true, + RetryLimit: 10, + JoinConcurrency: 5, + CrossJoin: true, + StatsLease: "3s", + }, + XProtocol: XProtocol{ + XHost: "0.0.0.0", + XPort: 14000, + }, +} + +var globalConf = defaultConf + +// NewConfig creates a new config instance with default value. +func NewConfig() *Config { + conf := defaultConf + return &conf +} // 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 { - once.Do(func() { - cfg = &Config{ - SlowThreshold: 300, - QueryLogMaxlen: 2048, - } - }) - return cfg + return &globalConf +} + +// Load loads config options from a toml file. +func (c *Config) Load(confFile string) error { + _, err := toml.DecodeFile(confFile, c) + return errors.Trace(err) +} + +// ToLogConfig converts *Log to *logutil.LogConfig. +func (l *Log) ToLogConfig() *logutil.LogConfig { + return &logutil.LogConfig{ + Level: l.Level, + Format: l.Format, + DisableTimestamp: l.DisableTimestamp, + File: l.File, + } } diff --git a/config/config.toml.example b/config/config.toml.example new file mode 100644 index 0000000000000..ff4cd3579d0f0 --- /dev/null +++ b/config/config.toml.example @@ -0,0 +1,113 @@ +# TiDB Configuration. + +# TiDB server host. +host = "0.0.0.0" + +# TiDB server port. +port = 4000 + +# Registered store name, [memory, goleveldb, boltdb, tikv, mocktikv] +store = "mocktikv" + +# TiDB storage path. +path = "/tmp/tidb" + +# The socket file to use for connection. +#socket = "" + +# Socket file to write binlog. +#binlog-socket = "" + +# Run ddl worker on this tidb-server. +run-ddl = true + +# Schema lease duration, very dangerous to change only if you know what you do. +lease = "10s" + +[log] +# Log level: info, debug, warn, error, fatal. +level = "info" + +# Log format, one of json, text, console. +format = "text" + +# Disable automatic timestamps in output +disable-timestamp = false + +# Queries with execution time greater than this value will be logged. (Milliseconds) +slow-threshold = 300 + +# Maximum query length recorded in log. +query-log-max-len = 2048 + +# File logging. +[log.file] +# Log file name. +filename = "" + +# Max log file size in MB. +#max-size = 300 + +# Max log file keep days. +#max-days = 28 + +# Maximum number of old log files to retain. +#max-backups = 7 + +# Rotate log by day +log-rotate = true + +[security] +# This option causes the server to start without using the privilege system at all. +skip-grant-table = false + +# Path of file that contains list of trusted SSL CAs. +ssl-ca = "" + +# Path of file that contains X509 certificate in PEM format. +ssl-cert = "" + +# Path of file that contains X509 key in PEM format. +ssl-key = "" + +[status] +# If enable status report HTTP service. +report-status = true + +# TiDB status port. +status-port = 10080 + +# Prometheus pushgateway address, leaves it empty will disable prometheus push. +metrics-addr = "" + +# Prometheus client push interval in second, set \"0\" to disable prometheus push. +metrics-interval = 15 + +[performance] +# Set keep alive option for tcp connection. +tcp-keep-alive = true + +# The maximum number of retries when commit a transaction. +retry-limit = 10 + +# The number of goroutines that participate joining. +join-concurrency = 5 + +# Whether support cartesian product. +cross-join = true + +# Stats lease duration, which inflences the time of analyze and stats load. +stats-lease = "3s" + +[xprotocol] +# Start TiDB x server. +xserver = false + +# TiDB x protocol server host. +xhost = "0.0.0.0" + +# TiDB x protocol server port. +xport = 14000 + +# The socket file to use for x protocol connection. +xsocket = "" diff --git a/config/config_test.go b/config/config_test.go new file mode 100644 index 0000000000000..d1bdb389fcf89 --- /dev/null +++ b/config/config_test.go @@ -0,0 +1,54 @@ +// Copyright 2017 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package config + +import ( + "path" + "runtime" + "testing" + + . "github.com/pingcap/check" +) + +var _ = Suite(&testConfigSuite{}) + +type testConfigSuite struct{} + +func TestT(t *testing.T) { + CustomVerboseFlag = true + TestingT(t) +} + +func (s *testConfigSuite) TestConfig(c *C) { + conf := new(Config) + conf.BinlogSocket = "/tmp/socket" + conf.Performance.RetryLimit = 20 + + _, filename, _, _ := runtime.Caller(0) + configFile := path.Join(path.Dir(filename), "config.toml.example") + err := conf.Load(configFile) + c.Assert(err, IsNil) + + // Test that the original value will not be clear by load the config file that does not contain the option. + c.Assert(conf.BinlogSocket, Equals, "/tmp/socket") + + // Test that the value will be overwritten by the config file. + c.Assert(conf.Performance.RetryLimit, Equals, 10) + + // Reset + conf.BinlogSocket = "" + + // Make sure the example config is the same as default config. + c.Assert(conf, DeepEquals, GetGlobalConfig()) +} diff --git a/executor/adapter.go b/executor/adapter.go index da5e8addbd234..310a0f713ad86 100644 --- a/executor/adapter.go +++ b/executor/adapter.go @@ -259,11 +259,11 @@ func (a *statement) logSlowQuery() { cfg := config.GetGlobalConfig() costTime := time.Since(a.startTime) sql := a.text - if len(sql) > cfg.QueryLogMaxlen { - sql = sql[:cfg.QueryLogMaxlen] + fmt.Sprintf("(len:%d)", len(sql)) + if len(sql) > cfg.Log.QueryLogMaxLen { + sql = sql[:cfg.Log.QueryLogMaxLen] + fmt.Sprintf("(len:%d)", len(sql)) } connID := a.ctx.GetSessionVars().ConnectionID - if costTime < time.Duration(cfg.SlowThreshold)*time.Millisecond { + if costTime < time.Duration(cfg.Log.SlowThreshold)*time.Millisecond { log.Debugf("[%d][TIME_QUERY] %v %s", connID, costTime, sql) } else { log.Warnf("[%d][TIME_QUERY] %v %s", connID, costTime, sql) diff --git a/server/http_status.go b/server/http_status.go index 6675b4b68d7f3..d3ae2c32f55a5 100644 --- a/server/http_status.go +++ b/server/http_status.go @@ -15,6 +15,7 @@ package server import ( "encoding/json" + "fmt" "net/http" "sync" @@ -50,8 +51,8 @@ func (s *Server) startHTTPServer() { router.Handle("/mvcc/txn/{startTS}/{db}/{table}", mvccTxnHandler{tikvHandler, opMvccGetByTxn}) router.Handle("/mvcc/txn/{startTS}", mvccTxnHandler{tikvHandler, opMvccGetByTxn}) } - addr := s.cfg.StatusAddr - if len(addr) == 0 { + addr := fmt.Sprintf(":%d", s.cfg.Status.StatusPort) + if s.cfg.Status.StatusPort == 0 { addr = defaultStatusAddr } log.Infof("Listening on %v for status and metrics report.", addr) diff --git a/server/region_handler_test.go b/server/region_handler_test.go index 2fd7bae50479f..e21d49cea9490 100644 --- a/server/region_handler_test.go +++ b/server/region_handler_test.go @@ -178,18 +178,17 @@ func (ts *TidbRegionHandlerTestSuite) startServer(c *C) { tidbdrv := NewTiDBDriver(store) cfg := &config.Config{ - Addr: ":4001", - LogLevel: "debug", - StatusAddr: ":10090", - ReportStatus: true, - Store: "tikv", + Port: 4001, + Store: "tikv", } + cfg.Status.StatusPort = 10090 + cfg.Status.ReportStatus = true server, err := NewServer(cfg, tidbdrv) c.Assert(err, IsNil) ts.server = server go server.Run() - waitUntilServerOnline(cfg.StatusAddr) + waitUntilServerOnline(cfg.Status.StatusPort) } func (ts *TidbRegionHandlerTestSuite) stopServer(c *C) { diff --git a/server/server.go b/server/server.go index c6ce181dbff40..28c407592fb52 100644 --- a/server/server.go +++ b/server/server.go @@ -31,6 +31,7 @@ package server import ( "crypto/tls" "crypto/x509" + "fmt" "io/ioutil" "math/rand" "net" @@ -115,7 +116,7 @@ func (s *Server) newConn(conn net.Conn) *clientConn { alloc: arena.NewAllocator(32 * 1024), } log.Infof("[%d] new connection %s", cc.connectionID, conn.RemoteAddr().String()) - if s.cfg.TCPKeepAlive { + if s.cfg.Performance.TCPKeepAlive { if tcpConn, ok := conn.(*net.TCPConn); ok { if err := tcpConn.SetKeepAlive(true); err != nil { log.Error("failed to set tcp keep alive option:", err) @@ -128,7 +129,7 @@ func (s *Server) newConn(conn net.Conn) *clientConn { } func (s *Server) skipAuth() bool { - return s.cfg.SkipAuth + return s.cfg.Socket != "" } const tokenLimit = 1000 @@ -152,13 +153,13 @@ func NewServer(cfg *config.Config, driver IDriver) (*Server, error) { var err error if cfg.Socket != "" { - cfg.SkipAuth = true if s.listener, err = net.Listen("unix", cfg.Socket); err == nil { log.Infof("Server is running MySQL Protocol through Socket [%s]", cfg.Socket) } } else { - if s.listener, err = net.Listen("tcp", s.cfg.Addr); err == nil { - log.Infof("Server is running MySQL Protocol at [%s]", s.cfg.Addr) + addr := fmt.Sprintf("%s:%d", s.cfg.Host, s.cfg.Port) + if s.listener, err = net.Listen("tcp", addr); err == nil { + log.Infof("Server is running MySQL Protocol at [%s]", addr) } } if err != nil { @@ -176,19 +177,19 @@ func (s *Server) loadTLSCertificates() { log.Infof("Secure connection is enabled (client verification enabled = %v)", len(variable.SysVars["ssl_ca"].Value) > 0) variable.SysVars["have_openssl"].Value = "YES" variable.SysVars["have_ssl"].Value = "YES" - variable.SysVars["ssl_cert"].Value = s.cfg.SSLCertPath - variable.SysVars["ssl_key"].Value = s.cfg.SSLKeyPath + variable.SysVars["ssl_cert"].Value = s.cfg.Security.SSLCert + variable.SysVars["ssl_key"].Value = s.cfg.Security.SSLKey } else { log.Warn("Secure connection is NOT ENABLED") } }() - if len(s.cfg.SSLCertPath) == 0 || len(s.cfg.SSLKeyPath) == 0 { + if len(s.cfg.Security.SSLCert) == 0 || len(s.cfg.Security.SSLKey) == 0 { s.tlsConfig = nil return } - tlsCert, err := tls.LoadX509KeyPair(s.cfg.SSLCertPath, s.cfg.SSLKeyPath) + tlsCert, err := tls.LoadX509KeyPair(s.cfg.Security.SSLCert, s.cfg.Security.SSLKey) if err != nil { log.Warn(errors.ErrorStack(err)) s.tlsConfig = nil @@ -198,8 +199,8 @@ func (s *Server) loadTLSCertificates() { // Try loading CA cert. clientAuthPolicy := tls.NoClientCert var certPool *x509.CertPool - if len(s.cfg.SSLCAPath) > 0 { - caCert, err := ioutil.ReadFile(s.cfg.SSLCAPath) + if len(s.cfg.Security.SSLCA) > 0 { + caCert, err := ioutil.ReadFile(s.cfg.Security.SSLCA) if err != nil { log.Warn(errors.ErrorStack(err)) } else { @@ -207,7 +208,7 @@ func (s *Server) loadTLSCertificates() { if certPool.AppendCertsFromPEM(caCert) { clientAuthPolicy = tls.VerifyClientCertIfGiven } - variable.SysVars["ssl_ca"].Value = s.cfg.SSLCAPath + variable.SysVars["ssl_ca"].Value = s.cfg.Security.SSLCA } } s.tlsConfig = &tls.Config{ @@ -221,7 +222,7 @@ func (s *Server) loadTLSCertificates() { // Run runs the server. func (s *Server) Run() error { // Start HTTP API to report tidb info such as TPS. - if s.cfg.ReportStatus { + if s.cfg.Status.ReportStatus { s.startStatusHTTP() } for { diff --git a/server/server_test.go b/server/server_test.go index 2d1a6aec9f928..30327e91900bb 100644 --- a/server/server_test.go +++ b/server/server_test.go @@ -740,7 +740,7 @@ func getStmtCnt(content string) (stmtCnt map[string]int) { const retryTime = 100 -func waitUntilServerOnline(statusAddr string) { +func waitUntilServerOnline(statusPort int) { // connect server retry := 0 for ; retry < retryTime; retry++ { @@ -755,7 +755,7 @@ func waitUntilServerOnline(statusAddr string) { log.Fatalf("Failed to connect db for %d retries in every 10 ms", retryTime) } // connect http status - statusURL := fmt.Sprintf("http://127.0.0.1%s/status", statusAddr) + statusURL := fmt.Sprintf("http://127.0.0.1:%d/status", statusPort) for retry = 0; retry < retryTime; retry++ { resp, err := http.Get(statusURL) if err == nil { diff --git a/server/tidb_test.go b/server/tidb_test.go index 77a084003a3f2..9cad6a5ae4a52 100644 --- a/server/tidb_test.go +++ b/server/tidb_test.go @@ -50,18 +50,21 @@ func (ts *TidbTestSuite) SetUpSuite(c *C) { c.Assert(err, IsNil) ts.tidbdrv = NewTiDBDriver(store) cfg := &config.Config{ - Addr: ":4001", - LogLevel: "debug", - StatusAddr: ":10090", - ReportStatus: true, - TCPKeepAlive: true, + Port: 4001, + Status: config.Status{ + ReportStatus: true, + StatusPort: 10090, + }, + Performance: config.Performance{ + TCPKeepAlive: true, + }, } server, err := NewServer(cfg, ts.tidbdrv) c.Assert(err, IsNil) ts.server = server go ts.server.Run() - waitUntilServerOnline(cfg.StatusAddr) + waitUntilServerOnline(cfg.Status.StatusPort) // Run this test here because parallel would affect the result of it. runTestStmtCount(c) @@ -144,9 +147,10 @@ func (ts *TidbTestSuite) TestMultiStatements(c *C) { func (ts *TidbTestSuite) TestSocket(c *C) { cfg := &config.Config{ - LogLevel: "debug", - StatusAddr: ":10091", - Socket: "/tmp/tidbtest.sock", + Socket: "/tmp/tidbtest.sock", + Status: config.Status{ + StatusPort: 10091, + }, } server, err := NewServer(cfg, ts.tidbdrv) @@ -278,10 +282,11 @@ func (ts *TidbTestSuite) TestTLS(c *C) { config.Addr = "localhost:4002" } cfg := &config.Config{ - Addr: ":4002", - LogLevel: "debug", - StatusAddr: ":10091", - ReportStatus: true, + Port: 4002, + Status: config.Status{ + ReportStatus: true, + StatusPort: 10091, + }, } server, err := NewServer(cfg, ts.tidbdrv) c.Assert(err, IsNil) @@ -298,12 +303,15 @@ func (ts *TidbTestSuite) TestTLS(c *C) { config.Addr = "localhost:4003" } cfg = &config.Config{ - Addr: ":4003", - LogLevel: "debug", - StatusAddr: ":10091", - ReportStatus: true, - SSLCertPath: "/tmp/server-cert.pem", - SSLKeyPath: "/tmp/server-key.pem", + Port: 4003, + Status: config.Status{ + ReportStatus: true, + StatusPort: 10091, + }, + Security: config.Security{ + SSLCert: "/tmp/server-cert.pem", + SSLKey: "/tmp/server-key.pem", + }, } server, err = NewServer(cfg, ts.tidbdrv) c.Assert(err, IsNil) @@ -328,13 +336,16 @@ func (ts *TidbTestSuite) TestTLS(c *C) { config.Addr = "localhost:4004" } cfg = &config.Config{ - Addr: ":4004", - LogLevel: "debug", - StatusAddr: ":10091", - ReportStatus: true, - SSLCertPath: "/tmp/server-cert.pem", - SSLKeyPath: "/tmp/server-key.pem", - SSLCAPath: "/tmp/ca-cert.pem", + Port: 4004, + Status: config.Status{ + ReportStatus: true, + StatusPort: 10091, + }, + Security: config.Security{ + SSLCA: "/tmp/ca-cert.pem", + SSLCert: "/tmp/server-cert.pem", + SSLKey: "/tmp/server-key.pem", + }, } server, err = NewServer(cfg, ts.tidbdrv) c.Assert(err, IsNil) diff --git a/tidb-server/main.go b/tidb-server/main.go index 3b9b019c103d7..cbedaf0dabdf4 100644 --- a/tidb-server/main.go +++ b/tidb-server/main.go @@ -20,6 +20,7 @@ import ( "os" "os/signal" "runtime" + "strconv" "syscall" "time" @@ -29,6 +30,7 @@ import ( "github.com/pingcap/tidb" "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/ddl" + "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/plan" "github.com/pingcap/tidb/privilege/privileges" @@ -45,38 +47,58 @@ import ( "google.golang.org/grpc" ) +// Flag Names +const ( + nmVersion = "V" + nmConfig = "config" + nmStore = "store" + nmStorePath = "path" + nmHost = "host" + nmPort = "P" + nmSocket = "socket" + nmBinlogSocket = "binlog-socket" + nmRunDDL = "run-ddl" + nmLogLevel = "L" + nmLogFile = "log-file" + nmReportStatus = "report-status" + nmStatusPort = "status" + nmMetricsAddr = "metrics-addr" + nmMetricsInterval = "metrics-interval" + nmDdlLease = "lease" + nmJoinConcurrency = "join-concurrency" + nmStatsLease = "statsLease" +) + var ( - version = flagBoolean("V", false, "print version information and exit") - store = flag.String("store", "mocktikv", "registered store name, [memory, goleveldb, boltdb, tikv, mocktikv]") - storePath = flag.String("path", "/tmp/tidb", "tidb storage path") - logLevel = flag.String("L", "info", "log level: info, debug, warn, error, fatal") - host = flag.String("host", "0.0.0.0", "tidb server host") - port = flag.String("P", "4000", "tidb server port") - xhost = flag.String("xhost", "0.0.0.0", "tidb x protocol server host") - xport = flag.String("xP", "14000", "tidb x protocol server port") - statusPort = flag.String("status", "10080", "tidb server status port") - ddlLease = flag.String("lease", "10s", "schema lease duration, very dangerous to change only if you know what you do") - statsLease = flag.String("statsLease", "3s", "stats lease duration, which inflences the time of analyze and stats load.") - socket = flag.String("socket", "", "The socket file to use for connection.") - xsocket = flag.String("xsocket", "", "The socket file to use for x protocol connection.") + version = flagBoolean(nmVersion, false, "print version information and exit") + configPath = flag.String(nmConfig, "", "config file path") + + // Base + store = flag.String(nmStore, "mocktikv", "registered store name, [memory, goleveldb, boltdb, tikv, mocktikv]") + storePath = flag.String(nmStorePath, "/tmp/tidb", "tidb storage path") + host = flag.String(nmHost, "0.0.0.0", "tidb server host") + port = flag.String(nmPort, "4000", "tidb server port") + socket = flag.String(nmSocket, "", "The socket file to use for connection.") + binlogSocket = flag.String(nmBinlogSocket, "", "socket file to write binlog") + runDDL = flagBoolean(nmRunDDL, true, "run ddl worker on this tidb-server") + ddlLease = flag.String(nmDdlLease, "10s", "schema lease duration, very dangerous to change only if you know what you do") + + // Performance + joinCon = flag.Int(nmJoinConcurrency, 5, "the number of goroutines that participate joining.") + statsLease = flag.String(nmStatsLease, "3s", "stats lease duration, which inflences the time of analyze and stats load.") + + // Log + logLevel = flag.String(nmLogLevel, "info", "log level: info, debug, warn, error, fatal") + logFile = flag.String(nmLogFile, "", "log file path") + + // Status + reportStatus = flagBoolean(nmReportStatus, true, "If enable status report HTTP service.") + statusPort = flag.String(nmStatusPort, "10080", "tidb server status port") + metricsAddr = flag.String(nmMetricsAddr, "", "prometheus pushgateway address, leaves it empty will disable prometheus push.") + metricsInterval = flag.Int(nmMetricsInterval, 15, "prometheus client push interval in second, set \"0\" to disable prometheus push.") + + // To be removed. enablePrivilege = flagBoolean("privilege", true, "If enable privilege check feature. This flag will be removed in the future.") - reportStatus = flagBoolean("report-status", true, "If enable status report HTTP service.") - logFile = flag.String("log-file", "", "log file path") - joinCon = flag.Int("join-concurrency", 5, "the number of goroutines that participate joining.") - crossJoin = flagBoolean("cross-join", true, "whether support cartesian product or not.") - metricsAddr = flag.String("metrics-addr", "", "prometheus pushgateway address, leaves it empty will disable prometheus push.") - metricsInterval = flag.Int("metrics-interval", 15, "prometheus client push interval in second, set \"0\" to disable prometheus push.") - binlogSocket = flag.String("binlog-socket", "", "socket file to write binlog") - runDDL = flagBoolean("run-ddl", true, "run ddl worker on this tidb-server") - retryLimit = flag.Int("retry-limit", 10, "the maximum number of retries when commit a transaction") - skipGrantTable = flagBoolean("skip-grant-table", false, "This option causes the server to start without using the privilege system at all.") - slowThreshold = flag.Int("slow-threshold", 300, "Queries with execution time greater than this value will be logged. (Milliseconds)") - queryLogMaxlen = flag.Int("query-log-max-len", 2048, "Maximum query length recorded in log") - startXServer = flagBoolean("xserver", false, "start tidb x protocol server") - tcpKeepAlive = flagBoolean("tcp-keep-alive", false, "set keep alive option for tcp connection.") - sslCAPath = flag.String("ssl-ca", "", "Path of file that contains list of trusted SSL CAs") - sslCertPath = flag.String("ssl-cert", "", "Path of file that contains X509 certificate in PEM format") - sslKeyPath = flag.String("ssl-key", "", "Path of file that contains X509 key in PEM format") timeJumpBackCounter = prometheus.NewCounter( prometheus.CounterOpts{ @@ -87,160 +109,73 @@ var ( }) ) -func main() { - tidb.RegisterLocalStore("boltdb", boltdb.Driver{}) - tidb.RegisterStore("tikv", tikv.Driver{}) - tidb.RegisterStore("mocktikv", tikv.MockDriver{}) - - runtime.GOMAXPROCS(runtime.NumCPU()) +var ( + cfg *config.Config + storage kv.Storage + dom *domain.Domain + svr *server.Server + xsvr *xserver.Server +) +func main() { flag.Parse() if *version { printer.PrintRawTiDBInfo() os.Exit(0) } - if *skipGrantTable && !hasRootPrivilege() { - log.Error("TiDB run with skip-grant-table need root privilege.") - os.Exit(-1) - } - - ddlLeaseDuration := parseLease(*ddlLease) - tidb.SetSchemaLease(ddlLeaseDuration) - statsLeaseDuration := parseLease(*statsLease) - tidb.SetStatsLease(statsLeaseDuration) - ddl.RunWorker = *runDDL - tidb.SetCommitRetryLimit(*retryLimit) - - cfg := config.GetGlobalConfig() - cfg.Addr = fmt.Sprintf("%s:%s", *host, *port) - cfg.LogLevel = *logLevel - cfg.StatusAddr = fmt.Sprintf(":%s", *statusPort) - cfg.Socket = *socket - cfg.ReportStatus = *reportStatus - cfg.Store = *store - cfg.StorePath = *storePath - cfg.SlowThreshold = *slowThreshold - cfg.QueryLogMaxlen = *queryLogMaxlen - cfg.TCPKeepAlive = *tcpKeepAlive - cfg.SSLCAPath = *sslCAPath - cfg.SSLCertPath = *sslCertPath - cfg.SSLKeyPath = *sslKeyPath - - xcfg := &xserver.Config{ - Addr: fmt.Sprintf("%s:%s", *xhost, *xport), - Socket: *socket, - LogLevel: *logLevel, - } - - // set log options - logConf := &logutil.LogConfig{ - Level: *logLevel, - } - if len(*logFile) > 0 { - logConf.File = logutil.FileLogConfig{ - Filename: *logFile, - LogRotate: true, - } - } - err := logutil.InitLogger(logConf) - if err != nil { - log.Fatal(err) - } - - // Make sure the TiDB info is always printed. - level := log.GetLevel() - log.SetLevel(log.InfoLevel) - printer.PrintTiDBInfo() - log.SetLevel(level) - if joinCon != nil && *joinCon > 0 { - plan.JoinConcurrency = *joinCon - } - plan.AllowCartesianProduct = *crossJoin + runtime.GOMAXPROCS(runtime.NumCPU()) - store := createStore() + registerStores() + loadConfig() + overrideConfig() + validateConfig() + setGlobalVars() + setupLog() + printInfo() + createStoreAndDomain() + setupBinlogClient() + createServer() + setupSignalHandler() + setupMetrics() + runServer() + cleanup() + os.Exit(0) +} - privileges.Enable = *enablePrivilege - privileges.SkipWithGrant = *skipGrantTable - if *binlogSocket != "" { - createBinlogClient() - } +func registerStores() { + tidb.RegisterLocalStore("boltdb", boltdb.Driver{}) + tidb.RegisterStore("tikv", tikv.Driver{}) + tidb.RegisterStore("mocktikv", tikv.MockDriver{}) +} - // Bootstrap a session to load information schema. - domain, err := tidb.BootstrapSession(store) +func createStoreAndDomain() { + fullPath := fmt.Sprintf("%s://%s", cfg.Store, cfg.Path) + var err error + storage, err = tidb.NewStore(fullPath) if err != nil { log.Fatal(errors.ErrorStack(err)) } - - var driver server.IDriver - driver = server.NewTiDBDriver(store) - var svr *server.Server - svr, err = server.NewServer(cfg, driver) + // Bootstrap a session to load information schema. + dom, err = tidb.BootstrapSession(storage) if err != nil { log.Fatal(errors.ErrorStack(err)) } - var xsvr *xserver.Server - if *startXServer { - xsvr, err = xserver.NewServer(xcfg) - if err != nil { - log.Fatal(errors.ErrorStack(err)) - } - } - - sc := make(chan os.Signal, 1) - signal.Notify(sc, - syscall.SIGHUP, - syscall.SIGINT, - syscall.SIGTERM, - syscall.SIGQUIT) - - go func() { - sig := <-sc - log.Infof("Got signal [%d] to exit.", sig) - if *startXServer { - xsvr.Close() // Should close xserver before server. - } - svr.Close() - }() - - prometheus.MustRegister(timeJumpBackCounter) - go systimemon.StartMonitor(time.Now, func() { - timeJumpBackCounter.Inc() - }) - - pushMetric(*metricsAddr, time.Duration(*metricsInterval)*time.Second) - - if err := svr.Run(); err != nil { - log.Error(err) - } - if *startXServer { - if err := xsvr.Run(); err != nil { - log.Error(err) - } - } - domain.Close() - os.Exit(0) } -func createStore() kv.Storage { - fullPath := fmt.Sprintf("%s://%s", *store, *storePath) - store, err := tidb.NewStore(fullPath) - if err != nil { - log.Fatal(errors.ErrorStack(err)) +func setupBinlogClient() { + if cfg.BinlogSocket == "" { + return } - return store -} - -func createBinlogClient() { dialerOpt := grpc.WithDialer(func(addr string, timeout time.Duration) (net.Conn, error) { return net.DialTimeout("unix", addr, timeout) }) - clientCon, err := grpc.Dial(*binlogSocket, dialerOpt, grpc.WithInsecure()) + clientCon, err := grpc.Dial(cfg.BinlogSocket, dialerOpt, grpc.WithInsecure()) if err != nil { log.Fatal(errors.ErrorStack(err)) } binloginfo.SetPumpClient(binlog.NewPumpClient(clientCon)) - log.Infof("created binlog client at %s", *binlogSocket) + log.Infof("created binlog client at %s", cfg.BinlogSocket) } // Prometheus push. @@ -279,7 +214,7 @@ func instanceName() string { if err != nil { return "unknown" } - return fmt.Sprintf("%s_%s", hostname, *port) + return fmt.Sprintf("%s_%d", hostname, cfg.Port) } // parseLease parses lease argument string. @@ -306,3 +241,180 @@ func flagBoolean(name string, defaultVal bool, usage string) *bool { } return flag.Bool(name, defaultVal, usage) } + +func loadConfig() { + cfg = config.GetGlobalConfig() + if *configPath != "" { + err := cfg.Load(*configPath) + if err != nil { + log.Fatal(err) + } + } +} + +func overrideConfig() { + actualFlags := make(map[string]bool) + flag.Visit(func(f *flag.Flag) { + actualFlags[f.Name] = true + }) + + // Base + if actualFlags[nmHost] { + cfg.Host = *host + } + var err error + if actualFlags[nmPort] { + cfg.Port, err = strconv.Atoi(*port) + if err != nil { + log.Fatal(err) + } + } + if actualFlags[nmStore] { + cfg.Store = *store + } + if actualFlags[nmStorePath] { + cfg.Path = *storePath + } + if actualFlags[nmSocket] { + cfg.Socket = *socket + } + if actualFlags[nmBinlogSocket] { + cfg.BinlogSocket = *binlogSocket + } + if actualFlags[nmRunDDL] { + cfg.RunDDL = *runDDL + } + if actualFlags[nmDdlLease] { + cfg.Lease = *ddlLease + } + + // Log + if actualFlags[nmLogLevel] { + cfg.Log.Level = *logLevel + } + if actualFlags[nmLogFile] { + cfg.Log.File.Filename = *logFile + } + + // Status + if actualFlags[nmReportStatus] { + cfg.Status.ReportStatus = *reportStatus + } + if actualFlags[nmStatusPort] { + cfg.Status.StatusPort, err = strconv.Atoi(*statusPort) + if err != nil { + log.Fatal(err) + } + } + if actualFlags[nmMetricsAddr] { + cfg.Status.MetricsAddr = *metricsAddr + } + if actualFlags[nmMetricsInterval] { + cfg.Status.MetricsInterval = *metricsInterval + } + + // Performance + if actualFlags[nmStatsLease] { + cfg.Performance.StatsLease = *statsLease + } + if actualFlags[nmJoinConcurrency] { + cfg.Performance.JoinConcurrency = *joinCon + } +} + +func validateConfig() { + if cfg.Security.SkipGrantTable && !hasRootPrivilege() { + log.Error("TiDB run with skip-grant-table need root privilege.") + os.Exit(-1) + } +} + +func setGlobalVars() { + ddlLeaseDuration := parseLease(cfg.Lease) + tidb.SetSchemaLease(ddlLeaseDuration) + statsLeaseDuration := parseLease(cfg.Performance.StatsLease) + tidb.SetStatsLease(statsLeaseDuration) + ddl.RunWorker = cfg.RunDDL + tidb.SetCommitRetryLimit(cfg.Performance.RetryLimit) + plan.JoinConcurrency = cfg.Performance.JoinConcurrency + plan.AllowCartesianProduct = cfg.Performance.CrossJoin + privileges.SkipWithGrant = cfg.Security.SkipGrantTable +} + +func setupLog() { + err := logutil.InitLogger(cfg.Log.ToLogConfig()) + if err != nil { + log.Fatal(err) + } +} + +func printInfo() { + // Make sure the TiDB info is always printed. + level := log.GetLevel() + log.SetLevel(log.InfoLevel) + printer.PrintTiDBInfo() + log.SetLevel(level) +} + +func createServer() { + var driver server.IDriver + driver = server.NewTiDBDriver(storage) + var err error + svr, err = server.NewServer(cfg, driver) + if err != nil { + log.Fatal(errors.ErrorStack(err)) + } + if cfg.XProtocol.XServer { + xcfg := &xserver.Config{ + Addr: fmt.Sprintf("%s:%d", cfg.XProtocol.XHost, cfg.XProtocol.XPort), + Socket: cfg.XProtocol.XSocket, + } + xsvr, err = xserver.NewServer(xcfg) + if err != nil { + log.Fatal(errors.ErrorStack(err)) + } + } +} + +func setupSignalHandler() { + sc := make(chan os.Signal, 1) + signal.Notify(sc, + syscall.SIGHUP, + syscall.SIGINT, + syscall.SIGTERM, + syscall.SIGQUIT) + + go func() { + sig := <-sc + log.Infof("Got signal [%d] to exit.", sig) + if xsvr != nil { + xsvr.Close() // Should close xserver before server. + } + svr.Close() + }() +} + +func setupMetrics() { + prometheus.MustRegister(timeJumpBackCounter) + go systimemon.StartMonitor(time.Now, func() { + timeJumpBackCounter.Inc() + }) + + pushMetric(cfg.Status.MetricsAddr, time.Duration(cfg.Status.MetricsInterval)*time.Second) +} + +func runServer() { + if err := svr.Run(); err != nil { + log.Error(err) + } + if cfg.XProtocol.XServer { + if err := xsvr.Run(); err != nil { + log.Error(err) + } + } +} + +func cleanup() { + dom.Close() + storage.Close() +} diff --git a/x-server/config.go b/x-server/config.go index 4263efddfa77c..d6e560f970117 100644 --- a/x-server/config.go +++ b/x-server/config.go @@ -17,6 +17,5 @@ package xserver type Config struct { Addr string `json:"addr" toml:"addr"` Socket string `json:"socket" toml:"socket"` - LogLevel string `json:"log_level" toml:"log_level"` SkipAuth bool `json:"skip_auth" toml:"skip_auth"` }