You can not select more than 25 topics Topics must start with a letter or number, can include dashes ('-') and can be up to 35 characters long.
 
 

1016 lines
40 KiB

// 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 (
"crypto/tls"
"crypto/x509"
"encoding/base64"
"encoding/json"
"fmt"
"io/ioutil"
"net/url"
"os"
"os/user"
"path/filepath"
"strings"
"sync/atomic"
"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"
"github.com/pingcap/tidb/util/versioninfo"
tracing "github.com/uber/jaeger-client-go/config"
"go.uber.org/zap"
)
// Config number limitations
const (
MaxLogFileSize = 4096 // MB
// DefTxnTotalSizeLimit is the default value of TxnTxnTotalSizeLimit.
DefTxnTotalSizeLimit = 100 * 1024 * 1024
// DefMaxIndexLength is the maximum index length(in bytes). This value is consistent with MySQL.
DefMaxIndexLength = 3072
// DefMaxOfMaxIndexLength is the maximum index length(in bytes) for TiDB v3.0.7 and previous version.
DefMaxOfMaxIndexLength = 3072 * 4
// DefPort is the default port of TiDB
DefPort = 4000
// DefStatusPort is the default status port of TiDB
DefStatusPort = 10080
// DefHost is the default host of TiDB
DefHost = "0.0.0.0"
// DefStatusHost is the default status host of TiDB
DefStatusHost = "0.0.0.0"
// DefStoreLivenessTimeout is the default value for store liveness timeout.
DefStoreLivenessTimeout = "5s"
)
// Valid config maps
var (
ValidStorage = map[string]bool{
"mocktikv": true,
"tikv": true,
}
// checkTableBeforeDrop enable to execute `admin check table` before `drop table`.
CheckTableBeforeDrop = false
// checkBeforeDropLDFlag is a go build flag.
checkBeforeDropLDFlag = "None"
// tempStorageDirName is the default temporary storage dir name by base64 encoding a string `port/statusPort`
tempStorageDirName = encodeDefTempStorageDir(os.TempDir(), DefHost, DefStatusHost, DefPort, DefStatusPort)
)
// Config contains configuration options.
type Config struct {
Host string `toml:"host" json:"host"`
AdvertiseAddress string `toml:"advertise-address" json:"advertise-address"`
Port uint `toml:"port" json:"port"`
Cors string `toml:"cors" json:"cors"`
Store string `toml:"store" json:"store"`
Path string `toml:"path" json:"path"`
Socket string `toml:"socket" json:"socket"`
Lease string `toml:"lease" json:"lease"`
RunDDL bool `toml:"run-ddl" json:"run-ddl"`
SplitTable bool `toml:"split-table" json:"split-table"`
TokenLimit uint `toml:"token-limit" json:"token-limit"`
OOMUseTmpStorage bool `toml:"oom-use-tmp-storage" json:"oom-use-tmp-storage"`
TempStoragePath string `toml:"tmp-storage-path" json:"tmp-storage-path"`
OOMAction string `toml:"oom-action" json:"oom-action"`
MemQuotaQuery int64 `toml:"mem-quota-query" json:"mem-quota-query"`
// TempStorageQuota describe the temporary storage Quota during query exector when OOMUseTmpStorage is enabled
// If the quota exceed the capacity of the TempStoragePath, the tidb-server would exit with fatal error
TempStorageQuota int64 `toml:"tmp-storage-quota" json:"tmp-storage-quota"` // Bytes
EnableStreaming bool `toml:"enable-streaming" json:"enable-streaming"`
EnableBatchDML bool `toml:"enable-batch-dml" json:"enable-batch-dml"`
TxnLocalLatches TxnLocalLatches `toml:"-" json:"-"`
// Set sys variable lower-case-table-names, ref: https://dev.mysql.com/doc/refman/5.7/en/identifier-case-sensitivity.html.
// TODO: We actually only support mode 2, which keeps the original case, but the comparison is case-insensitive.
LowerCaseTableNames int `toml:"lower-case-table-names" json:"lower-case-table-names"`
ServerVersion string `toml:"server-version" json:"server-version"`
Log Log `toml:"log" json:"log"`
Security Security `toml:"security" json:"security"`
Status Status `toml:"status" json:"status"`
Performance Performance `toml:"performance" json:"performance"`
PreparedPlanCache PreparedPlanCache `toml:"prepared-plan-cache" json:"prepared-plan-cache"`
OpenTracing OpenTracing `toml:"opentracing" json:"opentracing"`
ProxyProtocol ProxyProtocol `toml:"proxy-protocol" json:"proxy-protocol"`
TiKVClient TiKVClient `toml:"tikv-client" json:"tikv-client"`
Binlog Binlog `toml:"binlog" json:"binlog"`
CompatibleKillQuery bool `toml:"compatible-kill-query" json:"compatible-kill-query"`
Plugin Plugin `toml:"plugin" json:"plugin"`
PessimisticTxn PessimisticTxn `toml:"pessimistic-txn" json:"pessimistic-txn"`
CheckMb4ValueInUTF8 bool `toml:"check-mb4-value-in-utf8" json:"check-mb4-value-in-utf8"`
MaxIndexLength int `toml:"max-index-length" json:"max-index-length"`
GracefulWaitBeforeShutdown int `toml:"graceful-wait-before-shutdown" json:"graceful-wait-before-shutdown"`
// AlterPrimaryKey is used to control alter primary key feature.
AlterPrimaryKey bool `toml:"alter-primary-key" json:"alter-primary-key"`
// TreatOldVersionUTF8AsUTF8MB4 is use to treat old version table/column UTF8 charset as UTF8MB4. This is for compatibility.
// Currently not support dynamic modify, because this need to reload all old version schema.
TreatOldVersionUTF8AsUTF8MB4 bool `toml:"treat-old-version-utf8-as-utf8mb4" json:"treat-old-version-utf8-as-utf8mb4"`
// EnableTableLock indicate whether enable table lock.
// TODO: remove this after table lock features stable.
EnableTableLock bool `toml:"enable-table-lock" json:"enable-table-lock"`
DelayCleanTableLock uint64 `toml:"delay-clean-table-lock" json:"delay-clean-table-lock"`
SplitRegionMaxNum uint64 `toml:"split-region-max-num" json:"split-region-max-num"`
StmtSummary StmtSummary `toml:"stmt-summary" json:"stmt-summary"`
// RepairMode indicates that the TiDB is in the repair mode for table meta.
RepairMode bool `toml:"repair-mode" json:"repair-mode"`
RepairTableList []string `toml:"repair-table-list" json:"repair-table-list"`
// IsolationRead indicates that the TiDB reads data from which isolation level(engine and label).
IsolationRead IsolationRead `toml:"isolation-read" json:"isolation-read"`
// MaxServerConnections is the maximum permitted number of simultaneous client connections.
MaxServerConnections uint32 `toml:"max-server-connections" json:"max-server-connections"`
// NewCollationsEnabledOnFirstBootstrap indicates if the new collations are enabled, it effects only when a TiDB cluster bootstrapped on the first time.
NewCollationsEnabledOnFirstBootstrap bool `toml:"new_collations_enabled_on_first_bootstrap" json:"new_collations_enabled_on_first_bootstrap"`
// Experimental contains parameters for experimental features.
Experimental Experimental `toml:"experimental" json:"experimental"`
// EnableCollectExecutionInfo enables the TiDB to collect execution info.
EnableCollectExecutionInfo bool `toml:"enable-collect-execution-info" json:"enable-collect-execution-info"`
// SkipRegisterToDashboard tells TiDB don't register itself to the dashboard.
SkipRegisterToDashboard bool `toml:"skip-register-to-dashboard" json:"skip-register-to-dashboard"`
// EnableTelemetry enables the usage data report to PingCAP.
EnableTelemetry bool `toml:"enable-telemetry" json:"enable-telemetry"`
}
// UpdateTempStoragePath is to update the `TempStoragePath` if port/statusPort was changed
// and the `tmp-storage-path` was not specified in the conf.toml or was specified the same as the default value.
func (c *Config) UpdateTempStoragePath() {
if c.TempStoragePath == tempStorageDirName {
c.TempStoragePath = encodeDefTempStorageDir(os.TempDir(), c.Host, c.Status.StatusHost, c.Port, c.Status.StatusPort)
} else {
c.TempStoragePath = encodeDefTempStorageDir(c.TempStoragePath, c.Host, c.Status.StatusHost, c.Port, c.Status.StatusPort)
}
}
func encodeDefTempStorageDir(tempDir string, host, statusHost string, port, statusPort uint) string {
dirName := base64.URLEncoding.EncodeToString([]byte(fmt.Sprintf("%v:%v/%v:%v", host, port, statusHost, statusPort)))
var osUID string
currentUser, err := user.Current()
if err != nil {
osUID = ""
} else {
osUID = currentUser.Uid
}
return filepath.Join(tempDir, osUID+"_tidb", dirName, "tmp-storage")
}
// nullableBool defaults unset bool options to unset instead of false, which enables us to know if the user has set 2
// conflict options at the same time.
type nullableBool struct {
IsValid bool
IsTrue bool
}
var (
nbUnset = nullableBool{false, false}
nbFalse = nullableBool{true, false}
nbTrue = nullableBool{true, true}
)
func (b *nullableBool) toBool() bool {
return b.IsValid && b.IsTrue
}
func (b nullableBool) MarshalJSON() ([]byte, error) {
switch b {
case nbTrue:
return json.Marshal(true)
case nbFalse:
return json.Marshal(false)
default:
return json.Marshal(nil)
}
}
func (b *nullableBool) UnmarshalText(text []byte) error {
str := string(text)
switch str {
case "", "null":
*b = nbUnset
return nil
case "true":
*b = nbTrue
case "false":
*b = nbFalse
default:
*b = nbUnset
return errors.New("Invalid value for bool type: " + str)
}
return nil
}
func (b nullableBool) MarshalText() ([]byte, error) {
if !b.IsValid {
return []byte(""), nil
}
if b.IsTrue {
return []byte("true"), nil
}
return []byte("false"), nil
}
func (b *nullableBool) UnmarshalJSON(data []byte) error {
var err error
var v interface{}
if err = json.Unmarshal(data, &v); err != nil {
return err
}
switch raw := v.(type) {
case bool:
*b = nullableBool{true, raw}
default:
*b = nbUnset
}
return err
}
// 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. Deprecated: use EnableTimestamp instead.
DisableTimestamp nullableBool `toml:"disable-timestamp" json:"disable-timestamp"`
// EnableTimestamp enables automatic timestamps in log output.
EnableTimestamp nullableBool `toml:"enable-timestamp" json:"enable-timestamp"`
// DisableErrorStack stops annotating logs with the full stack error
// message. Deprecated: use EnableErrorStack instead.
DisableErrorStack nullableBool `toml:"disable-error-stack" json:"disable-error-stack"`
// EnableErrorStack enables annotating logs with the full stack error
// message.
EnableErrorStack nullableBool `toml:"enable-error-stack" json:"enable-error-stack"`
// File log config.
File logutil.FileLogConfig `toml:"file" json:"file"`
EnableSlowLog bool `toml:"enable-slow-log" json:"enable-slow-log"`
SlowQueryFile string `toml:"slow-query-file" json:"slow-query-file"`
SlowThreshold uint64 `toml:"slow-threshold" json:"slow-threshold"`
ExpensiveThreshold uint `toml:"expensive-threshold" json:"expensive-threshold"`
QueryLogMaxLen uint64 `toml:"query-log-max-len" json:"query-log-max-len"`
RecordPlanInSlowLog uint32 `toml:"record-plan-in-slow-log" json:"record-plan-in-slow-log"`
}
func (l *Log) getDisableTimestamp() bool {
if l.EnableTimestamp == nbUnset && l.DisableTimestamp == nbUnset {
return false
}
if l.EnableTimestamp == nbUnset {
return l.DisableTimestamp.toBool()
}
return !l.EnableTimestamp.toBool()
}
func (l *Log) getDisableErrorStack() bool {
if l.EnableErrorStack == nbUnset && l.DisableErrorStack == nbUnset {
return true
}
if l.EnableErrorStack == nbUnset {
return l.DisableErrorStack.toBool()
}
return !l.EnableErrorStack.toBool()
}
// 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"`
RequireSecureTransport bool `toml:"require-secure-transport" json:"require-secure-transport"`
ClusterSSLCA string `toml:"cluster-ssl-ca" json:"cluster-ssl-ca"`
ClusterSSLCert string `toml:"cluster-ssl-cert" json:"cluster-ssl-cert"`
ClusterSSLKey string `toml:"cluster-ssl-key" json:"cluster-ssl-key"`
ClusterVerifyCN []string `toml:"cluster-verify-cn" json:"cluster-verify-cn"`
}
// The ErrConfigValidationFailed error is used so that external callers can do a type assertion
// to defer handling of this specific error when someone does not want strict type checking.
// This is needed only because logging hasn't been set up at the time we parse the config file.
// This should all be ripped out once strict config checking is made the default behavior.
type ErrConfigValidationFailed struct {
confFile string
UndecodedItems []string
}
func (e *ErrConfigValidationFailed) Error() string {
return fmt.Sprintf("config file %s contained unknown configuration options: %s", e.confFile, strings.Join(e.UndecodedItems, ", "))
}
// ToTLSConfig generates tls's config based on security section of the config.
func (s *Security) ToTLSConfig() (tlsConfig *tls.Config, err error) {
if len(s.ClusterSSLCA) != 0 {
certPool := x509.NewCertPool()
// Create a certificate pool from the certificate authority
var ca []byte
ca, err = ioutil.ReadFile(s.ClusterSSLCA)
if err != nil {
err = errors.Errorf("could not read ca certificate: %s", err)
return
}
// Append the certificates from the CA
if !certPool.AppendCertsFromPEM(ca) {
err = errors.New("failed to append ca certs")
return
}
tlsConfig = &tls.Config{
RootCAs: certPool,
ClientCAs: certPool,
}
if len(s.ClusterSSLCert) != 0 && len(s.ClusterSSLKey) != 0 {
getCert := func() (*tls.Certificate, error) {
// Load the client certificates from disk
cert, err := tls.LoadX509KeyPair(s.ClusterSSLCert, s.ClusterSSLKey)
if err != nil {
return nil, errors.Errorf("could not load client key pair: %s", err)
}
return &cert, nil
}
// pre-test cert's loading.
if _, err = getCert(); err != nil {
return
}
tlsConfig.GetClientCertificate = func(info *tls.CertificateRequestInfo) (certificate *tls.Certificate, err error) {
return getCert()
}
tlsConfig.GetCertificate = func(info *tls.ClientHelloInfo) (certificate *tls.Certificate, err error) {
return getCert()
}
}
}
return
}
// Status is the status section of the config.
type Status struct {
StatusHost string `toml:"status-host" json:"status-host"`
MetricsAddr string `toml:"metrics-addr" json:"metrics-addr"`
StatusPort uint `toml:"status-port" json:"status-port"`
MetricsInterval uint `toml:"metrics-interval" json:"metrics-interval"`
ReportStatus bool `toml:"report-status" json:"report-status"`
RecordQPSbyDB bool `toml:"record-db-qps" json:"record-db-qps"`
}
// Performance is the performance section of the config.
type Performance struct {
MaxProcs uint `toml:"max-procs" json:"max-procs"`
MaxMemory uint64 `toml:"max-memory" json:"max-memory"`
ServerMemoryQuota uint64 `toml:"server-memory-quota" json:"server-memory-quota"`
MemoryUsageAlarmRatio float64 `toml:"memory-usage-alarm-ratio" json:"memory-usage-alarm-ratio"`
StatsLease string `toml:"stats-lease" json:"stats-lease"`
StmtCountLimit uint `toml:"stmt-count-limit" json:"stmt-count-limit"`
FeedbackProbability float64 `toml:"feedback-probability" json:"feedback-probability"`
QueryFeedbackLimit uint `toml:"query-feedback-limit" json:"query-feedback-limit"`
PseudoEstimateRatio float64 `toml:"pseudo-estimate-ratio" json:"pseudo-estimate-ratio"`
ForcePriority string `toml:"force-priority" json:"force-priority"`
BindInfoLease string `toml:"bind-info-lease" json:"bind-info-lease"`
TxnTotalSizeLimit uint64 `toml:"txn-total-size-limit" json:"txn-total-size-limit"`
TCPKeepAlive bool `toml:"tcp-keep-alive" json:"tcp-keep-alive"`
CrossJoin bool `toml:"cross-join" json:"cross-join"`
RunAutoAnalyze bool `toml:"run-auto-analyze" json:"run-auto-analyze"`
DistinctAggPushDown bool `toml:"distinct-agg-push-down" json:"agg-push-down-join"`
CommitterConcurrency int `toml:"committer-concurrency" json:"committer-concurrency"`
MaxTxnTTL uint64 `toml:"max-txn-ttl" json:"max-txn-ttl"`
GOGC int `toml:"gogc" json:"gogc"`
}
// PlanCache is the PlanCache section of the config.
type PlanCache struct {
Enabled bool `toml:"enabled" json:"enabled"`
Capacity uint `toml:"capacity" json:"capacity"`
Shards uint `toml:"shards" json:"shards"`
}
// TxnLocalLatches is the TxnLocalLatches section of the config.
type TxnLocalLatches struct {
Enabled bool `toml:"-" json:"-"`
Capacity uint `toml:"-" json:"-"`
}
// PreparedPlanCache is the PreparedPlanCache section of the config.
type PreparedPlanCache struct {
Enabled bool `toml:"enabled" json:"enabled"`
Capacity uint `toml:"capacity" json:"capacity"`
MemoryGuardRatio float64 `toml:"memory-guard-ratio" json:"memory-guard-ratio"`
}
// OpenTracing is the opentracing section of the config.
type OpenTracing struct {
Enable bool `toml:"enable" json:"enable"`
RPCMetrics bool `toml:"rpc-metrics" json:"rpc-metrics"`
Sampler OpenTracingSampler `toml:"sampler" json:"sampler"`
Reporter OpenTracingReporter `toml:"reporter" json:"reporter"`
}
// OpenTracingSampler is the config for opentracing sampler.
// See https://godoc.org/github.com/uber/jaeger-client-go/config#SamplerConfig
type OpenTracingSampler struct {
Type string `toml:"type" json:"type"`
Param float64 `toml:"param" json:"param"`
SamplingServerURL string `toml:"sampling-server-url" json:"sampling-server-url"`
MaxOperations int `toml:"max-operations" json:"max-operations"`
SamplingRefreshInterval time.Duration `toml:"sampling-refresh-interval" json:"sampling-refresh-interval"`
}
// OpenTracingReporter is the config for opentracing reporter.
// See https://godoc.org/github.com/uber/jaeger-client-go/config#ReporterConfig
type OpenTracingReporter struct {
QueueSize int `toml:"queue-size" json:"queue-size"`
BufferFlushInterval time.Duration `toml:"buffer-flush-interval" json:"buffer-flush-interval"`
LogSpans bool `toml:"log-spans" json:"log-spans"`
LocalAgentHostPort string `toml:"local-agent-host-port" json:"local-agent-host-port"`
}
// ProxyProtocol is the PROXY protocol section of the config.
type ProxyProtocol struct {
// PROXY protocol acceptable client networks.
// Empty string means disable PROXY protocol,
// * means all networks.
Networks string `toml:"networks" json:"networks"`
// PROXY protocol header read timeout, Unit is second.
HeaderTimeout uint `toml:"header-timeout" json:"header-timeout"`
}
// TiKVClient is the config for tikv client.
type TiKVClient struct {
// GrpcConnectionCount is the max gRPC connections that will be established
// with each tikv-server.
GrpcConnectionCount uint `toml:"grpc-connection-count" json:"grpc-connection-count"`
// After a duration of this time in seconds if the client doesn't see any activity it pings
// the server to see if the transport is still alive.
GrpcKeepAliveTime uint `toml:"grpc-keepalive-time" json:"grpc-keepalive-time"`
// After having pinged for keepalive check, the client waits for a duration of Timeout in seconds
// and if no activity is seen even after that the connection is closed.
GrpcKeepAliveTimeout uint `toml:"grpc-keepalive-timeout" json:"grpc-keepalive-timeout"`
// CommitTimeout is the max time which command 'commit' will wait.
CommitTimeout string `toml:"commit-timeout" json:"commit-timeout"`
// MaxBatchSize is the max batch size when calling batch commands API.
MaxBatchSize uint `toml:"max-batch-size" json:"max-batch-size"`
// If TiKV load is greater than this, TiDB will wait for a while to avoid little batch.
OverloadThreshold uint `toml:"overload-threshold" json:"overload-threshold"`
// MaxBatchWaitTime in nanosecond is the max wait time for batch.
MaxBatchWaitTime time.Duration `toml:"max-batch-wait-time" json:"max-batch-wait-time"`
// BatchWaitSize is the max wait size for batch.
BatchWaitSize uint `toml:"batch-wait-size" json:"batch-wait-size"`
// EnableChunkRPC indicate the data encode in chunk format for coprocessor requests.
EnableChunkRPC bool `toml:"enable-chunk-rpc" json:"enable-chunk-rpc"`
// If a Region has not been accessed for more than the given duration (in seconds), it
// will be reloaded from the PD.
RegionCacheTTL uint `toml:"region-cache-ttl" json:"region-cache-ttl"`
// If a store has been up to the limit, it will return error for successive request to
// prevent the store occupying too much token in dispatching level.
StoreLimit int64 `toml:"store-limit" json:"store-limit"`
// StoreLivenessTimeout is the timeout for store liveness check request.
StoreLivenessTimeout string `toml:"store-liveness-timeout" json:"store-liveness-timeout"`
CoprCache CoprocessorCache `toml:"copr-cache" json:"copr-cache"`
}
// CoprocessorCache is the config for coprocessor cache.
type CoprocessorCache struct {
// Whether to enable the copr cache. The copr cache saves the result from TiKV Coprocessor in the memory and
// reuses the result when corresponding data in TiKV is unchanged, on a region basis.
Enable bool `toml:"enable" json:"enable"`
// The capacity in MB of the cache.
CapacityMB float64 `toml:"capacity-mb" json:"capacity-mb"`
// Only cache requests that containing small number of ranges. May to be changed in future.
AdmissionMaxRanges uint64 `toml:"admission-max-ranges" json:"admission-max-ranges"`
// Only cache requests whose result set is small.
AdmissionMaxResultMB float64 `toml:"admission-max-result-mb" json:"admission-max-result-mb"`
// Only cache requests takes notable time to process.
AdmissionMinProcessMs uint64 `toml:"admission-min-process-ms" json:"admission-min-process-ms"`
}
// Binlog is the config for binlog.
type Binlog struct {
Enable bool `toml:"enable" json:"enable"`
// If IgnoreError is true, when writing binlog meets error, TiDB would
// ignore the error.
IgnoreError bool `toml:"ignore-error" json:"ignore-error"`
WriteTimeout string `toml:"write-timeout" json:"write-timeout"`
// Use socket file to write binlog, for compatible with kafka version tidb-binlog.
BinlogSocket string `toml:"binlog-socket" json:"binlog-socket"`
// The strategy for sending binlog to pump, value can be "range" or "hash" now.
Strategy string `toml:"strategy" json:"strategy"`
}
// Plugin is the config for plugin
type Plugin struct {
Dir string `toml:"dir" json:"dir"`
Load string `toml:"load" json:"load"`
}
// PessimisticTxn is the config for pessimistic transaction.
type PessimisticTxn struct {
// Enable must be true for 'begin lock' or session variable to start a pessimistic transaction.
Enable bool `toml:"enable" json:"enable"`
// The max count of retry for a single statement in a pessimistic transaction.
MaxRetryCount uint `toml:"max-retry-count" json:"max-retry-count"`
}
// StmtSummary is the config for statement summary.
type StmtSummary struct {
// Enable statement summary or not.
Enable bool `toml:"enable" json:"enable"`
// Enable summary internal query.
EnableInternalQuery bool `toml:"enable-internal-query" json:"enable-internal-query"`
// The maximum number of statements kept in memory.
MaxStmtCount uint `toml:"max-stmt-count" json:"max-stmt-count"`
// The maximum length of displayed normalized SQL and sample SQL.
MaxSQLLength uint `toml:"max-sql-length" json:"max-sql-length"`
// The refresh interval of statement summary.
RefreshInterval int `toml:"refresh-interval" json:"refresh-interval"`
// The maximum history size of statement summary.
HistorySize int `toml:"history-size" json:"history-size"`
}
// IsolationRead is the config for isolation read.
type IsolationRead struct {
// Engines filters tidb-server access paths by engine type.
Engines []string `toml:"engines" json:"engines"`
}
// Experimental controls the features that are still experimental: their semantics, interfaces are subject to change.
// Using these features in the production environment is not recommended.
type Experimental struct {
// Whether enable creating expression index.
AllowsExpressionIndex bool `toml:"allow-expression-index" json:"allow-expression-index"`
}
var defaultConf = Config{
Host: DefHost,
AdvertiseAddress: "",
Port: DefPort,
Cors: "",
Store: "mocktikv",
Path: "/tmp/tidb",
RunDDL: true,
SplitTable: true,
Lease: "45s",
TokenLimit: 1000,
OOMUseTmpStorage: true,
TempStorageQuota: -1,
TempStoragePath: tempStorageDirName,
OOMAction: OOMActionLog,
MemQuotaQuery: 1 << 30,
EnableStreaming: false,
EnableBatchDML: false,
CheckMb4ValueInUTF8: true,
MaxIndexLength: 3072,
AlterPrimaryKey: false,
TreatOldVersionUTF8AsUTF8MB4: true,
EnableTableLock: false,
DelayCleanTableLock: 0,
SplitRegionMaxNum: 1000,
RepairMode: false,
RepairTableList: []string{},
MaxServerConnections: 0,
TxnLocalLatches: TxnLocalLatches{
Enabled: false,
Capacity: 0,
},
LowerCaseTableNames: 2,
GracefulWaitBeforeShutdown: 0,
ServerVersion: "",
Log: Log{
Level: "info",
Format: "text",
File: logutil.NewFileLogConfig(logutil.DefaultLogMaxSize),
SlowQueryFile: "tidb-slow.log",
SlowThreshold: logutil.DefaultSlowThreshold,
ExpensiveThreshold: 10000,
DisableErrorStack: nbUnset,
EnableErrorStack: nbUnset, // If both options are nbUnset, getDisableErrorStack() returns true
EnableTimestamp: nbUnset,
DisableTimestamp: nbUnset, // If both options are nbUnset, getDisableTimestamp() returns false
QueryLogMaxLen: logutil.DefaultQueryLogMaxLen,
RecordPlanInSlowLog: logutil.DefaultRecordPlanInSlowLog,
EnableSlowLog: logutil.DefaultTiDBEnableSlowLog,
},
Status: Status{
ReportStatus: true,
StatusHost: DefStatusHost,
StatusPort: DefStatusPort,
MetricsInterval: 15,
RecordQPSbyDB: false,
},
Performance: Performance{
MaxMemory: 0,
ServerMemoryQuota: 0,
MemoryUsageAlarmRatio: 0.8,
TCPKeepAlive: true,
CrossJoin: true,
StatsLease: "3s",
RunAutoAnalyze: true,
StmtCountLimit: 5000,
FeedbackProbability: 0.0,
QueryFeedbackLimit: 512,
PseudoEstimateRatio: 0.8,
ForcePriority: "NO_PRIORITY",
BindInfoLease: "3s",
TxnTotalSizeLimit: DefTxnTotalSizeLimit,
DistinctAggPushDown: false,
CommitterConcurrency: 16,
MaxTxnTTL: 10 * 60 * 1000, // 10min
GOGC: 100,
},
ProxyProtocol: ProxyProtocol{
Networks: "",
HeaderTimeout: 5,
},
PreparedPlanCache: PreparedPlanCache{
Enabled: false,
Capacity: 100,
MemoryGuardRatio: 0.1,
},
OpenTracing: OpenTracing{
Enable: false,
Sampler: OpenTracingSampler{
Type: "const",
Param: 1.0,
},
Reporter: OpenTracingReporter{},
},
TiKVClient: TiKVClient{
GrpcConnectionCount: 4,
GrpcKeepAliveTime: 10,
GrpcKeepAliveTimeout: 3,
CommitTimeout: "41s",
MaxBatchSize: 128,
OverloadThreshold: 200,
MaxBatchWaitTime: 0,
BatchWaitSize: 8,
EnableChunkRPC: true,
RegionCacheTTL: 600,
StoreLimit: 0,
StoreLivenessTimeout: DefStoreLivenessTimeout,
CoprCache: CoprocessorCache{
Enable: false,
CapacityMB: 1000,
AdmissionMaxRanges: 500,
AdmissionMaxResultMB: 10,
AdmissionMinProcessMs: 5,
},
},
Binlog: Binlog{
WriteTimeout: "15s",
Strategy: "range",
},
PessimisticTxn: PessimisticTxn{
Enable: true,
MaxRetryCount: 256,
},
StmtSummary: StmtSummary{
Enable: true,
EnableInternalQuery: false,
MaxStmtCount: 200,
MaxSQLLength: 4096,
RefreshInterval: 1800,
HistorySize: 24,
},
IsolationRead: IsolationRead{
Engines: []string{"tikv", "tiflash", "tidb"},
},
Experimental: Experimental{
AllowsExpressionIndex: false,
},
EnableCollectExecutionInfo: true,
EnableTelemetry: true,
}
var (
globalConf atomic.Value
)
// 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 {
return globalConf.Load().(*Config)
}
// 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)
}
var deprecatedConfig = map[string]struct{}{
"pessimistic-txn.ttl": {},
"log.file.log-rotate": {},
"log.log-slow-query": {},
"txn-local-latches": {},
"txn-local-latches.enabled": {},
"txn-local-latches.capacity": {},
"max-txn-time-use": {},
"experimental.allow-auto-random": {},
"enable-redact-log": {}, // use variable tidb_redact_log instead
}
func isAllDeprecatedConfigItems(items []string) bool {
for _, item := range items {
if _, ok := deprecatedConfig[item]; !ok {
return false
}
}
return true
}
// IsMemoryQuotaQuerySetByUser indicates whether the config item mem-quota-query
// is set by the user.
var IsMemoryQuotaQuerySetByUser bool
// InitializeConfig initialize the global config handler.
// 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, enforceCmdArgs func(*Config)) {
cfg := GetGlobalConfig()
var err error
if confPath != "" {
if err = cfg.Load(confPath); err != nil {
// Unused config item error turns to warnings.
if tmp, ok := err.(*ErrConfigValidationFailed); ok {
// 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) || isAllDeprecatedConfigItems(tmp.UndecodedItems) {
fmt.Fprintln(os.Stderr, err.Error())
err = nil
}
}
}
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)
}
}
enforceCmdArgs(cfg)
if err := cfg.Valid(); err != nil {
if !filepath.IsAbs(confPath) {
if tmp, err := filepath.Abs(confPath); err == nil {
confPath = tmp
}
}
fmt.Fprintln(os.Stderr, "load config file:", confPath)
fmt.Fprintln(os.Stderr, "invalid config", err)
os.Exit(1)
}
if configCheck {
fmt.Println("config check successful")
os.Exit(0)
}
StoreGlobalConfig(cfg)
}
// Load loads config options from a toml file.
func (c *Config) Load(confFile string) error {
metaData, err := toml.DecodeFile(confFile, c)
if c.TokenLimit == 0 {
c.TokenLimit = 1000
}
if metaData.IsDefined("mem-quota-query") {
IsMemoryQuotaQuerySetByUser = true
}
if len(c.ServerVersion) > 0 {
mysql.ServerVersion = c.ServerVersion
}
// If any items in confFile file are not mapped into the Config struct, issue
// an error and stop the server from starting.
undecoded := metaData.Undecoded()
if len(undecoded) > 0 && err == nil {
var undecodedItems []string
for _, item := range undecoded {
undecodedItems = append(undecodedItems, item.String())
}
err = &ErrConfigValidationFailed{confFile, undecodedItems}
}
return err
}
// Valid checks if this config is valid.
func (c *Config) Valid() error {
if c.Log.EnableErrorStack == c.Log.DisableErrorStack && c.Log.EnableErrorStack != nbUnset {
logutil.BgLogger().Warn(fmt.Sprintf("\"enable-error-stack\" (%v) conflicts \"disable-error-stack\" (%v). \"disable-error-stack\" is deprecated, please use \"enable-error-stack\" instead. disable-error-stack is ignored.", c.Log.EnableErrorStack, c.Log.DisableErrorStack))
// if two options conflict, we will use the value of EnableErrorStack
c.Log.DisableErrorStack = nbUnset
}
if c.Log.EnableTimestamp == c.Log.DisableTimestamp && c.Log.EnableTimestamp != nbUnset {
logutil.BgLogger().Warn(fmt.Sprintf("\"enable-timestamp\" (%v) conflicts \"disable-timestamp\" (%v). \"disable-timestamp\" is deprecated, please use \"enable-timestamp\" instead", c.Log.EnableTimestamp, c.Log.DisableTimestamp))
// if two options conflict, we will use the value of EnableTimestamp
c.Log.DisableTimestamp = nbUnset
}
if c.Security.SkipGrantTable && !hasRootPrivilege() {
return fmt.Errorf("TiDB run with skip-grant-table need root privilege")
}
if !ValidStorage[c.Store] {
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.MaxIndexLength < DefMaxIndexLength || c.MaxIndexLength > DefMaxOfMaxIndexLength {
return fmt.Errorf("max-index-length should be [%d, %d]", DefMaxIndexLength, DefMaxOfMaxIndexLength)
}
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)
if c.OOMAction != OOMActionLog && c.OOMAction != OOMActionCancel {
return fmt.Errorf("unsupported OOMAction %v, TiDB only supports [%v, %v]", c.OOMAction, OOMActionLog, OOMActionCancel)
}
// 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.Performance.TxnTotalSizeLimit > 10<<30 {
return fmt.Errorf("txn-total-size-limit should be less than %d", 10<<30)
}
if c.Performance.MemoryUsageAlarmRatio > 1 || c.Performance.MemoryUsageAlarmRatio < 0 {
return fmt.Errorf("memory-usage-alarm-ratio in [Performance] must be greater than or equal to 0 and less than or equal to 1")
}
if c.StmtSummary.MaxStmtCount <= 0 {
return fmt.Errorf("max-stmt-count in [stmt-summary] should be greater than 0")
}
if c.StmtSummary.HistorySize < 0 {
return fmt.Errorf("history-size in [stmt-summary] should be greater than or equal to 0")
}
if c.StmtSummary.RefreshInterval <= 0 {
return fmt.Errorf("refresh-interval in [stmt-summary] should be greater than 0")
}
if c.PreparedPlanCache.Capacity < 1 {
return fmt.Errorf("capacity in [prepared-plan-cache] should be at least 1")
}
if c.PreparedPlanCache.MemoryGuardRatio < 0 || c.PreparedPlanCache.MemoryGuardRatio > 1 {
return fmt.Errorf("memory-guard-ratio in [prepared-plan-cache] must be NOT less than 0 and more than 1")
}
if len(c.IsolationRead.Engines) < 1 {
return fmt.Errorf("the number of [isolation-read]engines for isolation read should be at least 1")
}
for _, engine := range c.IsolationRead.Engines {
if engine != "tidb" && engine != "tikv" && engine != "tiflash" {
return fmt.Errorf("type of [isolation-read]engines can't be %v should be one of tidb or tikv or tiflash", engine)
}
}
// test log level
l := zap.NewAtomicLevel()
return l.UnmarshalText([]byte(c.Log.Level))
}
// UpdateGlobal updates the global config, and provide a restore function that can be used to restore to the original.
func UpdateGlobal(f func(conf *Config)) {
g := GetGlobalConfig()
newConf := *g
f(&newConf)
StoreGlobalConfig(&newConf)
}
// RestoreFunc gets a function that restore the config to the current value.
func RestoreFunc() (restore func()) {
g := GetGlobalConfig()
return func() {
StoreGlobalConfig(g)
}
}
func hasRootPrivilege() bool {
return os.Geteuid() == 0
}
// TableLockEnabled uses to check whether enabled the table lock feature.
func TableLockEnabled() bool {
return GetGlobalConfig().EnableTableLock
}
// TableLockDelayClean uses to get the time of delay clean table lock.
var TableLockDelayClean = func() uint64 {
return GetGlobalConfig().DelayCleanTableLock
}
// ToLogConfig converts *Log to *logutil.LogConfig.
func (l *Log) ToLogConfig() *logutil.LogConfig {
return logutil.NewLogConfig(l.Level, l.Format, l.SlowQueryFile, l.File, l.getDisableTimestamp(), func(config *zaplog.Config) { config.DisableErrorVerbose = l.getDisableErrorStack() })
}
// ToTracingConfig converts *OpenTracing to *tracing.Configuration.
func (t *OpenTracing) ToTracingConfig() *tracing.Configuration {
ret := &tracing.Configuration{
Disabled: !t.Enable,
RPCMetrics: t.RPCMetrics,
Reporter: &tracing.ReporterConfig{},
Sampler: &tracing.SamplerConfig{},
}
ret.Reporter.QueueSize = t.Reporter.QueueSize
ret.Reporter.BufferFlushInterval = t.Reporter.BufferFlushInterval
ret.Reporter.LogSpans = t.Reporter.LogSpans
ret.Reporter.LocalAgentHostPort = t.Reporter.LocalAgentHostPort
ret.Sampler.Type = t.Sampler.Type
ret.Sampler.Param = t.Sampler.Param
ret.Sampler.SamplingServerURL = t.Sampler.SamplingServerURL
ret.Sampler.MaxOperations = t.Sampler.MaxOperations
ret.Sampler.SamplingRefreshInterval = t.Sampler.SamplingRefreshInterval
return ret
}
func init() {
initByLDFlags(versioninfo.TiDBEdition, checkBeforeDropLDFlag)
}
func initByLDFlags(edition, checkBeforeDropLDFlag string) {
if edition != versioninfo.CommunityEdition {
defaultConf.EnableTelemetry = false
}
conf := defaultConf
StoreGlobalConfig(&conf)
if checkBeforeDropLDFlag == "1" {
CheckTableBeforeDrop = true
}
}
// The following constants represents the valid action configurations for OOMAction.
// NOTE: Although the values is case insensitive, we should use lower-case
// strings because the configuration value will be transformed to lower-case
// string and compared with these constants in the further usage.
const (
OOMActionCancel = "cancel"
OOMActionLog = "log"
)
// ParsePath parses this path.
func ParsePath(path string) (etcdAddrs []string, disableGC bool, err error) {
var u *url.URL
u, err = url.Parse(path)
if err != nil {
err = errors.Trace(err)
return
}
if strings.ToLower(u.Scheme) != "tikv" {
err = errors.Errorf("Uri scheme expected [tikv] but found [%s]", u.Scheme)
logutil.BgLogger().Error("parsePath error", zap.Error(err))
return
}
switch strings.ToLower(u.Query().Get("disableGC")) {
case "true":
disableGC = true
case "false", "":
default:
err = errors.New("disableGC flag should be true/false")
return
}
etcdAddrs = strings.Split(u.Host, ",")
return
}