From 6d51ad33fd861c39b64a2f5d99db045d1fa0fb1d Mon Sep 17 00:00:00 2001 From: Maxwell Date: Tue, 13 Aug 2019 17:20:53 +0800 Subject: [PATCH] *: Optimize struct memory usage by adjust field order (#11629) --- cmd/importer/data.go | 2 +- config/config.go | 18 +++++++++--------- ddl/ddl_worker_test.go | 4 ++-- ddl/delete_range.go | 14 +++++++------- domain/domain.go | 4 ++-- domain/topn_slow_query.go | 4 ++-- executor/adapter.go | 10 +++++----- executor/aggregate.go | 13 ++++++------- executor/executor.go | 4 ++-- executor/grant.go | 6 +++--- executor/join.go | 11 +++++------ executor/load_data.go | 4 ++-- executor/prepared.go | 2 +- executor/show.go | 25 ++++++++++++------------- executor/table_reader.go | 19 +++++++++---------- expression/aggregation/concat.go | 2 +- expression/column.go | 8 ++++---- infoschema/slow_log.go | 4 ++-- kv/kv.go | 13 +++++++------ planner/core/common_plans.go | 6 +++--- planner/core/physical_plans.go | 25 +++++++++++++------------ planner/core/planbuilder.go | 5 ++--- planner/core/point_get_plan.go | 2 +- plugin/plugin.go | 2 +- plugin/spi.go | 5 +++-- server/conn.go | 4 ++-- sessionctx/variable/session.go | 6 +++--- statistics/table.go | 13 +++++++------ store/tikv/2pc.go | 26 ++++++++++++++------------ store/tikv/client.go | 8 ++++---- store/tikv/client_batch.go | 5 +++-- store/tikv/coprocessor.go | 11 ++++++----- store/tikv/scan.go | 7 ++++--- store/tikv/txn.go | 5 +++-- util/filesort/filesort.go | 19 ++++++++++--------- util/processinfo.go | 27 ++++++++++++++------------- 36 files changed, 175 insertions(+), 168 deletions(-) diff --git a/cmd/importer/data.go b/cmd/importer/data.go index 2e14db6ccd..826d0c0776 100644 --- a/cmd/importer/data.go +++ b/cmd/importer/data.go @@ -31,8 +31,8 @@ type datum struct { timeValue time.Time remains uint64 repeats uint64 - probability uint32 step int64 + probability uint32 init bool useRange bool diff --git a/config/config.go b/config/config.go index 242eb659be..13c6ab8bd7 100644 --- a/config/config.go +++ b/config/config.go @@ -174,11 +174,11 @@ func (s *Security) ToTLSConfig() (*tls.Config, error) { // Status is the status section of the config. type Status struct { - ReportStatus bool `toml:"report-status" json:"report-status"` StatusHost string `toml:"status-host" json:"status-host"` - StatusPort uint `toml:"status-port" json:"status-port"` 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"` } @@ -186,10 +186,7 @@ type Status struct { type Performance struct { MaxProcs uint `toml:"max-procs" json:"max-procs"` MaxMemory uint64 `toml:"max-memory" json:"max-memory"` - TCPKeepAlive bool `toml:"tcp-keep-alive" json:"tcp-keep-alive"` - CrossJoin bool `toml:"cross-join" json:"cross-join"` StatsLease string `toml:"stats-lease" json:"stats-lease"` - RunAutoAnalyze bool `toml:"run-auto-analyze" json:"run-auto-analyze"` 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"` @@ -197,6 +194,9 @@ type Performance struct { 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"` } // PlanCache is the PlanCache section of the config. @@ -222,9 +222,9 @@ type PreparedPlanCache struct { // 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"` - RPCMetrics bool `toml:"rpc-metrics" json:"rpc-metrics"` } // OpenTracingSampler is the config for opentracing sampler. @@ -285,11 +285,11 @@ type TiKVClient struct { // Binlog is the config for binlog. type Binlog struct { - Enable bool `toml:"enable" json:"enable"` - WriteTimeout string `toml:"write-timeout" json:"write-timeout"` + 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"` + 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. diff --git a/ddl/ddl_worker_test.go b/ddl/ddl_worker_test.go index 16663c6199..7b28a292f4 100644 --- a/ddl/ddl_worker_test.go +++ b/ddl/ddl_worker_test.go @@ -353,9 +353,9 @@ func checkCancelState(txn kv.Transaction, job *model.Job, test *testCancelJob) e } type testCancelJob struct { - act model.ActionType // act is the job action. jobIDs []int64 - cancelRetErrs []error // cancelRetErrs is the first return value of CancelJobs. + cancelRetErrs []error // cancelRetErrs is the first return value of CancelJobs. + act model.ActionType // act is the job action. cancelState model.SchemaState } diff --git a/ddl/delete_range.go b/ddl/delete_range.go index 2db0e6145f..d274c887de 100644 --- a/ddl/delete_range.go +++ b/ddl/delete_range.go @@ -55,14 +55,14 @@ type delRangeManager interface { } type delRange struct { - store kv.Storage - sessPool *sessionPool - storeSupport bool - emulatorCh chan struct{} - keys []kv.Key - quitCh chan struct{} + store kv.Storage + sessPool *sessionPool + emulatorCh chan struct{} + keys []kv.Key + quitCh chan struct{} - wait sync.WaitGroup // wait is only used when storeSupport is false. + wait sync.WaitGroup // wait is only used when storeSupport is false. + storeSupport bool } // newDelRangeManager returns a delRangeManager. diff --git a/domain/domain.go b/domain/domain.go index 533e7ab84f..6c66b666ea 100644 --- a/domain/domain.go +++ b/domain/domain.go @@ -61,7 +61,6 @@ type Domain struct { bindHandle *bindinfo.BindHandle statsHandle unsafe.Pointer statsLease time.Duration - statsUpdating sync2.AtomicInt32 ddl ddl.DDL info *InfoSyncer m sync.Mutex @@ -69,10 +68,11 @@ type Domain struct { sysSessionPool *sessionPool exit chan struct{} etcdClient *clientv3.Client - wg sync.WaitGroup gvc GlobalVariableCache slowQuery *topNSlowQueries expensiveQueryHandle *expensivequery.Handle + wg sync.WaitGroup + statsUpdating sync2.AtomicInt32 } // loadInfoSchema loads infoschema at startTS into handle, usedSchemaVersion is the currently used diff --git a/domain/topn_slow_query.go b/domain/topn_slow_query.go index ce1da7b885..f280a6c43a 100644 --- a/domain/topn_slow_query.go +++ b/domain/topn_slow_query.go @@ -217,13 +217,13 @@ type SlowQueryInfo struct { Start time.Time Duration time.Duration Detail execdetails.ExecDetails - Succ bool ConnID uint64 TxnTS uint64 User string DB string TableIDs string IndexIDs string - Internal bool Digest string + Internal bool + Succ bool } diff --git a/executor/adapter.go b/executor/adapter.go index c48a607f69..add0aad327 100644 --- a/executor/adapter.go +++ b/executor/adapter.go @@ -146,10 +146,6 @@ type ExecStmt struct { InfoSchema infoschema.InfoSchema // Plan stores a reference to the final physical plan. Plan plannercore.Plan - // LowerPriority represents whether to lower the execution priority of a query. - LowerPriority bool - // Cacheable represents whether the physical plan can be cached. - Cacheable bool // Text represents the origin query text. Text string @@ -157,7 +153,11 @@ type ExecStmt struct { Ctx sessionctx.Context // StartTime stands for the starting time when executing the statement. - StartTime time.Time + StartTime time.Time + // LowerPriority represents whether to lower the execution priority of a query. + LowerPriority bool + // Cacheable represents whether the physical plan can be cached. + Cacheable bool isPreparedStmt bool isSelectForUpdate bool retryCount uint diff --git a/executor/aggregate.go b/executor/aggregate.go index 346e7cca68..1486d25952 100644 --- a/executor/aggregate.go +++ b/executor/aggregate.go @@ -132,7 +132,6 @@ type AfFinalResult struct { type HashAggExec struct { baseExecutor - prepared bool sc *stmtctx.StatementContext PartialAggFuncs []aggfuncs.AggFunc FinalAggFuncs []aggfuncs.AggFunc @@ -144,10 +143,6 @@ type HashAggExec struct { groupKeyBuffer []byte groupValDatums []types.Datum - // After we support parallel execution for aggregation functions with distinct, - // we can remove this attribute. - isUnparallelExec bool - finishCh chan struct{} finalOutputCh chan *AfFinalResult finalInputCh chan *chunk.Chunk @@ -157,10 +152,14 @@ type HashAggExec struct { partialWorkers []HashAggPartialWorker finalWorkers []HashAggFinalWorker defaultVal *chunk.Chunk + childResult *chunk.Chunk + // isChildReturnEmpty indicates whether the child executor only returns an empty input. isChildReturnEmpty bool - - childResult *chunk.Chunk + // After we support parallel execution for aggregation functions with distinct, + // we can remove this attribute. + isUnparallelExec bool + prepared bool } // HashAggInput indicates the input of hash agg exec. diff --git a/executor/executor.go b/executor/executor.go index f6050b219a..a8a093f5ad 100644 --- a/executor/executor.go +++ b/executor/executor.go @@ -1240,14 +1240,14 @@ type UnionExec struct { baseExecutor stopFetchData atomic.Value - wg sync.WaitGroup finished chan struct{} resourcePools []chan *chunk.Chunk resultPool chan *unionWorkerResult - initialized bool childrenResults []*chunk.Chunk + wg sync.WaitGroup + initialized bool } // unionWorkerResult stores the result for a union worker. diff --git a/executor/grant.go b/executor/grant.go index 8b99bb8ffd..eda999972c 100644 --- a/executor/grant.go +++ b/executor/grant.go @@ -46,10 +46,10 @@ type GrantExec struct { ObjectType ast.ObjectTypeType Level *ast.GrantLevel Users []*ast.UserSpec - WithGrant bool - is infoschema.InfoSchema - done bool + is infoschema.InfoSchema + WithGrant bool + done bool } // Next implements the Executor Next interface. diff --git a/executor/join.go b/executor/join.go index 50f444ad82..d9469d32a9 100644 --- a/executor/join.go +++ b/executor/join.go @@ -48,7 +48,6 @@ type HashJoinExec struct { outerKeys []*expression.Column innerKeys []*expression.Column - prepared bool // concurrency is the number of partition, build and join workers. concurrency uint hashTable *mvmap.MVMap @@ -58,10 +57,8 @@ type HashJoinExec struct { joinWorkerWaitGroup sync.WaitGroup finished atomic.Value // closeCh add a lock for closing executor. - closeCh chan struct{} - joinType plannercore.JoinType - - isOuterJoin bool + closeCh chan struct{} + joinType plannercore.JoinType requiredRows int64 // We build individual joiner for each join worker when use chunk-based @@ -77,7 +74,9 @@ type HashJoinExec struct { joinResultCh chan *hashjoinWorkerResult hashTableValBufs [][][]byte - memTracker *memory.Tracker // track memory usage. + memTracker *memory.Tracker // track memory usage. + prepared bool + isOuterJoin bool } // outerChkResource stores the result of the join outer fetch worker, diff --git a/executor/load_data.go b/executor/load_data.go index 4b020a5ff4..3120e7543f 100644 --- a/executor/load_data.go +++ b/executor/load_data.go @@ -351,14 +351,14 @@ type field struct { type fieldWriter struct { pos int + ReadBuf []byte + OutputBuf []byte enclosedChar byte fieldTermChar byte term string isEnclosed bool isLineStart bool isFieldStart bool - ReadBuf []byte - OutputBuf []byte } func (w *fieldWriter) Init(enclosedChar byte, fieldTermChar byte, readBuf []byte, term string) { diff --git a/executor/prepared.go b/executor/prepared.go index 03e039c83a..f774fc210e 100644 --- a/executor/prepared.go +++ b/executor/prepared.go @@ -205,10 +205,10 @@ type ExecuteExec struct { is infoschema.InfoSchema name string usingVars []expression.Expression - id uint32 stmtExec Executor stmt ast.StmtNode plan plannercore.Plan + id uint32 lowerPriority bool } diff --git a/executor/show.go b/executor/show.go index d28fbbf1e7..27969047ef 100644 --- a/executor/show.go +++ b/executor/show.go @@ -58,24 +58,23 @@ var etcdDialTimeout = 5 * time.Second type ShowExec struct { baseExecutor - Tp ast.ShowStmtType // Databases/Tables/Columns/.... - DBName model.CIStr - Table *ast.TableName // Used for showing columns. - Column *ast.ColumnName // Used for `desc table column`. - IndexName model.CIStr // Used for show table regions. - Flag int // Some flag parsed from sql, such as FULL. - Full bool - User *auth.UserIdentity // Used by show grants, show create user. - Roles []*auth.RoleIdentity // Used for show grants. - IfNotExists bool // Used for `show create database if not exists` - - // GlobalScope is used by show variables - GlobalScope bool + Tp ast.ShowStmtType // Databases/Tables/Columns/.... + DBName model.CIStr + Table *ast.TableName // Used for showing columns. + Column *ast.ColumnName // Used for `desc table column`. + IndexName model.CIStr // Used for show table regions. + Flag int // Some flag parsed from sql, such as FULL. + Roles []*auth.RoleIdentity // Used for show grants. + User *auth.UserIdentity // Used by show grants, show create user. is infoschema.InfoSchema result *chunk.Chunk cursor int + + Full bool + IfNotExists bool // Used for `show create database if not exists` + GlobalScope bool // GlobalScope is used by show variables } // Next implements the Executor Next interface. diff --git a/executor/table_reader.go b/executor/table_reader.go index ec61cbf30d..1426c47fb1 100644 --- a/executor/table_reader.go +++ b/executor/table_reader.go @@ -55,10 +55,8 @@ func (sr selectResultHook) SelectResult(ctx context.Context, sctx sessionctx.Con type TableReaderExecutor struct { baseExecutor - table table.Table - keepOrder bool - desc bool - ranges []*ranger.Range + table table.Table + ranges []*ranger.Range // kvRanges are only use for union scan. kvRanges []kv.KeyRange dagPB *tipb.DAGRequest @@ -68,18 +66,19 @@ type TableReaderExecutor struct { // resultHandler handles the order of the result. Since (MAXInt64, MAXUint64] stores before [0, MaxInt64] physically // for unsigned int. resultHandler *tableResultHandler - streaming bool feedback *statistics.QueryFeedback + plans []plannercore.PhysicalPlan + memTracker *memory.Tracker + selectResultHook // for testing + + keepOrder bool + desc bool + streaming bool // corColInFilter tells whether there's correlated column in filter. corColInFilter bool // corColInAccess tells whether there's correlated column in access conditions. corColInAccess bool - plans []plannercore.PhysicalPlan - - memTracker *memory.Tracker - - selectResultHook // for testing } // Open initialzes necessary variables for using this executor. diff --git a/expression/aggregation/concat.go b/expression/aggregation/concat.go index 68ab1ccaa4..f456d659ae 100644 --- a/expression/aggregation/concat.go +++ b/expression/aggregation/concat.go @@ -28,8 +28,8 @@ import ( type concatFunction struct { aggFunction separator string - sepInited bool maxLen uint64 + sepInited bool // truncated according to MySQL, a 'group_concat' function generates exactly one 'truncated' warning during its life time, no matter // how many group actually truncated. 'truncated' acts as a sentinel to indicate whether this warning has already been // generated. diff --git a/expression/column.go b/expression/column.go index 50d53c0d34..de3354c5ae 100644 --- a/expression/column.go +++ b/expression/column.go @@ -163,16 +163,16 @@ type Column struct { ID int64 // UniqueID is the unique id of this column. UniqueID int64 - // IsReferenced means if this column is referenced to an Aggregation column, or a Subquery column, - // or an argument column of function IfNull. - // If so, this column's name will be the plain sql text. - IsReferenced bool // Index is used for execution, to tell the column's position in the given row. Index int hashcode []byte + // IsReferenced means if this column is referenced to an Aggregation column, or a Subquery column, + // or an argument column of function IfNull. + // If so, this column's name will be the plain sql text. + IsReferenced bool // InOperand indicates whether this column is the inner operand of column equal condition converted // from `[not] in (subq)`. InOperand bool diff --git a/infoschema/slow_log.go b/infoschema/slow_log.go index 7c85f30fa3..6259942062 100644 --- a/infoschema/slow_log.go +++ b/infoschema/slow_log.go @@ -184,7 +184,6 @@ type slowQueryTuple struct { processKeys uint64 db string indexIDs string - isInternal bool digest string statsInfo string avgProcessTime float64 @@ -196,8 +195,9 @@ type slowQueryTuple struct { maxWaitTime float64 maxWaitAddress string memMax int64 - succ bool sql string + isInternal bool + succ bool } func (st *slowQueryTuple) setFieldValue(tz *time.Location, field, value string) error { diff --git a/kv/kv.go b/kv/kv.go index 21cc6cc3f2..895d8ebed1 100644 --- a/kv/kv.go +++ b/kv/kv.go @@ -199,10 +199,7 @@ type Request struct { StartTs uint64 Data []byte KeyRanges []KeyRange - // KeepOrder is true, if the response should be returned in order. - KeepOrder bool - // Desc is true, if the request is sent in descending order. - Desc bool + // Concurrency is 1, if it only sends the request to a single storage unit when // ResponseIterator.Next is called. If concurrency is greater than 1, the request will be // sent to multiple storage units concurrently. @@ -211,6 +208,12 @@ type Request struct { IsolationLevel IsoLevel // Priority is the priority of this KV request, its value may be PriorityNormal/PriorityLow/PriorityHigh. Priority int + // MemTracker is used to trace and control memory usage in co-processor layer. + MemTracker *memory.Tracker + // KeepOrder is true, if the response should be returned in order. + KeepOrder bool + // Desc is true, if the request is sent in descending order. + Desc bool // NotFillCache makes this request do not touch the LRU cache of the underlying storage. NotFillCache bool // SyncLog decides whether the WAL(write-ahead log) of this request should be synchronized. @@ -218,8 +221,6 @@ type Request struct { // Streaming indicates using streaming API for this request, result in that one Next() // call would not corresponds to a whole region result. Streaming bool - // MemTracker is used to trace and control memory usage in co-processor layer. - MemTracker *memory.Tracker } // ResultSubset represents a result subset from a single storage unit. diff --git a/planner/core/common_plans.go b/planner/core/common_plans.go index 529f9c55a1..1c2ca0b46d 100644 --- a/planner/core/common_plans.go +++ b/planner/core/common_plans.go @@ -371,11 +371,11 @@ type Show struct { Table *ast.TableName // Used for showing columns. Column *ast.ColumnName // Used for `desc table column`. IndexName model.CIStr - Flag int // Some flag parsed from sql, such as FULL. - Full bool + Flag int // Some flag parsed from sql, such as FULL. User *auth.UserIdentity // Used for show grants. Roles []*auth.RoleIdentity // Used for show grants. - IfNotExists bool // Used for `show create database if not exists` + Full bool + IfNotExists bool // Used for `show create database if not exists` GlobalScope bool // Used by show variables } diff --git a/planner/core/physical_plans.go b/planner/core/physical_plans.go index dcda164e98..fb328d6994 100644 --- a/planner/core/physical_plans.go +++ b/planner/core/physical_plans.go @@ -98,11 +98,6 @@ type PhysicalIndexScan struct { Ranges []*ranger.Range Columns []*model.ColumnInfo DBName model.CIStr - Desc bool - KeepOrder bool - // DoubleRead means if the index executor will read kv two times. - // If the query requires the columns that don't belong to index, DoubleRead will be true. - DoubleRead bool TableAsName *model.CIStr @@ -117,10 +112,16 @@ type PhysicalIndexScan struct { rangeInfo string // The index scan may be on a partition. - isPartition bool physicalTableID int64 GenExprs map[model.TableColumnID]expression.Expression + + isPartition bool + Desc bool + KeepOrder bool + // DoubleRead means if the index executor will read kv two times. + // If the query requires the columns that don't belong to index, DoubleRead will be true. + DoubleRead bool } // PhysicalMemTable reads memory table. @@ -144,27 +145,27 @@ type PhysicalTableScan struct { Table *model.TableInfo Columns []*model.ColumnInfo DBName model.CIStr - Desc bool Ranges []*ranger.Range pkCol *expression.Column TableAsName *model.CIStr - // KeepOrder is true, if sort data by scanning pkcol, - KeepOrder bool - // Hist is the histogram when the query was issued. // It is used for query feedback. Hist *statistics.Histogram - // The table scan may be a partition, rather than a real table. - isPartition bool physicalTableID int64 rangeDecidedBy []*expression.Column // HandleIdx is the index of handle, which is only used for admin check table. HandleIdx int + + // The table scan may be a partition, rather than a real table. + isPartition bool + // KeepOrder is true, if sort data by scanning pkcol, + KeepOrder bool + Desc bool } // IsPartition returns true and partition ID if it's actually a partition. diff --git a/planner/core/planbuilder.go b/planner/core/planbuilder.go index 099b330b59..1e0c147811 100644 --- a/planner/core/planbuilder.go +++ b/planner/core/planbuilder.go @@ -173,7 +173,6 @@ type PlanBuilder struct { ctx sessionctx.Context is infoschema.InfoSchema outerSchemas []*expression.Schema - inUpdateStmt bool // colMapper stores the column that must be pre-resolved. colMapper map[*ast.ColumnNameExpr]int // visitInfo is used for privilege check. @@ -188,11 +187,11 @@ type PlanBuilder struct { rewriterPool []*expressionRewriter rewriterCounter int + windowSpecs map[string]*ast.WindowSpec + inUpdateStmt bool // inStraightJoin represents whether the current "SELECT" statement has // "STRAIGHT_JOIN" option. inStraightJoin bool - - windowSpecs map[string]*ast.WindowSpec } // GetVisitInfo gets the visitInfo of the PlanBuilder. diff --git a/planner/core/point_get_plan.go b/planner/core/point_get_plan.go index 5d48f282fb..0277acf45a 100644 --- a/planner/core/point_get_plan.go +++ b/planner/core/point_get_plan.go @@ -43,11 +43,11 @@ type PointGetPlan struct { IndexInfo *model.IndexInfo Handle int64 HandleParam *driver.ParamMarkerExpr - UnsignedHandle bool IndexValues []types.Datum IndexValueParams []*driver.ParamMarkerExpr expr expression.Expression ctx sessionctx.Context + UnsignedHandle bool IsTableDual bool Lock bool IsForUpdate bool diff --git a/plugin/plugin.go b/plugin/plugin.go index 06c9fe5201..6840a04fb3 100644 --- a/plugin/plugin.go +++ b/plugin/plugin.go @@ -96,9 +96,9 @@ type Config struct { type Plugin struct { *Manifest library *gplugin.Plugin - State State Path string Disabled uint32 + State State } // StateValue returns readable state string. diff --git a/plugin/spi.go b/plugin/spi.go index 8be7f6253f..1efab55ac3 100644 --- a/plugin/spi.go +++ b/plugin/spi.go @@ -31,10 +31,8 @@ const ( // Manifest describes plugin info and how it can do by plugin itself. type Manifest struct { - Kind Kind Name string Description string - Version uint16 RequireVersion map[string]uint16 License string BuildTime string @@ -54,6 +52,9 @@ type Manifest struct { // return error will write log and continue watch following flush. OnFlush func(ctx context.Context, manifest *Manifest) error flushWatcher *flushWatcher + + Version uint16 + Kind Kind } // ExportManifest exports a manifest to TiDB as a known format. diff --git a/server/conn.go b/server/conn.go index c3d9da6e0a..e59ed5538d 100644 --- a/server/conn.go +++ b/server/conn.go @@ -139,7 +139,6 @@ type clientConn struct { server *Server // a reference of server instance. capability uint32 // client capability affects the way server handles client request. connectionID uint32 // atomically allocated by a global variable, unique in process scope. - collation uint8 // collation used by client, may be different from the collation used by database. user string // user of the client. dbname string // default database name. salt []byte // random bytes used for authentication. @@ -147,10 +146,11 @@ type clientConn struct { lastCmd string // latest sql query string, currently used for logging error. ctx QueryCtx // an interface to execute sql statements. attrs map[string]string // attributes parsed from client handshake response, not used for now. - status int32 // dispatching/reading/shutdown/waitshutdown peerHost string // peer host peerPort string // peer port + status int32 // dispatching/reading/shutdown/waitshutdown lastCode uint16 // last error code + collation uint8 // collation used by client, may be different from the collation used by database. } func (cc *clientConn) String() string { diff --git a/sessionctx/variable/session.go b/sessionctx/variable/session.go index 715cede8bc..cbc015eecc 100644 --- a/sessionctx/variable/session.go +++ b/sessionctx/variable/session.go @@ -99,22 +99,22 @@ func (r *RetryInfo) GetCurrAutoIncrementID() (int64, error) { // TransactionContext is used to store variables that has transaction scope. type TransactionContext struct { - ForUpdate bool forUpdateTS uint64 DirtyDB interface{} Binlog interface{} InfoSchema interface{} - CouldRetry bool History interface{} SchemaVersion int64 StartTS uint64 Shard *int64 TableDeltaMap map[int64]TableDelta - IsPessimistic bool // CreateTime For metrics. CreateTime time.Time StatementCount int + ForUpdate bool + CouldRetry bool + IsPessimistic bool } // UpdateDeltaForTable updates the delta info for some table. diff --git a/statistics/table.go b/statistics/table.go index b2ba11a762..99669a7926 100644 --- a/statistics/table.go +++ b/statistics/table.go @@ -57,18 +57,19 @@ type Table struct { // HistColl is a collection of histogram. It collects enough information for plan to calculate the selectivity. type HistColl struct { PhysicalID int64 - // HavePhysicalID is true means this HistColl is from single table and have its ID's information. - // The physical id is used when try to load column stats from storage. - HavePhysicalID bool - Columns map[int64]*Column - Indices map[int64]*Index + Columns map[int64]*Column + Indices map[int64]*Index // Idx2ColumnIDs maps the index id to its column ids. It's used to calculate the selectivity in planner. Idx2ColumnIDs map[int64][]int64 // ColID2IdxID maps the column id to index id whose first column is it. It's used to calculate the selectivity in planner. ColID2IdxID map[int64]int64 - Pseudo bool Count int64 ModifyCount int64 // Total modify count in a table. + + // HavePhysicalID is true means this HistColl is from single table and have its ID's information. + // The physical id is used when try to load column stats from storage. + HavePhysicalID bool + Pseudo bool } // Copy copies the current table. diff --git a/store/tikv/2pc.go b/store/tikv/2pc.go index 5f6ecd7953..ff35649eab 100644 --- a/store/tikv/2pc.go +++ b/store/tikv/2pc.go @@ -89,26 +89,28 @@ type twoPhaseCommitter struct { mutations map[string]*mutationEx lockTTL uint64 commitTS uint64 - mu struct { - sync.RWMutex - committed bool - undeterminedErr error // undeterminedErr saves the rpc error we encounter when commit primary key. - } - priority pb.CommandPri - syncLog bool - connID uint64 // connID is used for log. - cleanWg sync.WaitGroup + priority pb.CommandPri + connID uint64 // connID is used for log. + cleanWg sync.WaitGroup // maxTxnTimeUse represents max time a Txn may use (in ms) from its startTS to commitTS. // We use it to guarantee GC worker will not influence any active txn. The value // should be less than GC life time. maxTxnTimeUse uint64 detail *execdetails.CommitDetails - // For pessimistic transaction - isPessimistic bool + primaryKey []byte forUpdateTS uint64 - isFirstLock bool pessimisticTTL uint64 + + mu struct { + sync.RWMutex + undeterminedErr error // undeterminedErr saves the rpc error we encounter when commit primary key. + committed bool + } + syncLog bool + // For pessimistic transaction + isPessimistic bool + isFirstLock bool } type mutationEx struct { diff --git a/store/tikv/client.go b/store/tikv/client.go index c1585e0a24..08d75ca9ed 100644 --- a/store/tikv/client.go +++ b/store/tikv/client.go @@ -199,15 +199,15 @@ func (a *connArray) Close() { // that there are too many concurrent requests which overload the service of TiKV. type rpcClient struct { sync.RWMutex - isClosed bool - done chan struct{} + done chan struct{} conns map[string]*connArray security config.Security - // Implement background cleanup. - // Periodically check whether there is any connection that is idle and then close and remove these idle connections. idleNotify uint32 + // Periodically check whether there is any connection that is idle and then close and remove these idle connections. + // Implement background cleanup. + isClosed bool } func newRPCClient(security config.Security) *rpcClient { diff --git a/store/tikv/client_batch.go b/store/tikv/client_batch.go index 4fda6bb208..f55c32ea76 100644 --- a/store/tikv/client_batch.go +++ b/store/tikv/client_batch.go @@ -35,7 +35,6 @@ import ( ) type batchConn struct { - index uint32 // batchCommandsCh used for batch commands. batchCommandsCh chan *batchCommandsEntry batchCommandsClients []*batchCommandsClient @@ -44,10 +43,12 @@ type batchConn struct { // Notify rpcClient to check the idle flag idleNotify *uint32 - idle bool idleDetect *time.Timer pendingRequests prometheus.Gauge + + index uint32 + idle bool } func newBatchConn(connCount, maxBatchSize uint, idleNotify *uint32) *batchConn { diff --git a/store/tikv/coprocessor.go b/store/tikv/coprocessor.go index 10769da96c..e49815f47f 100644 --- a/store/tikv/coprocessor.go +++ b/store/tikv/coprocessor.go @@ -330,10 +330,6 @@ type copIterator struct { req *kv.Request concurrency int finishCh chan struct{} - // closed represents when the Close is called. - // There are two cases we need to close the `finishCh` channel, one is when context is done, the other one is - // when the Close is called. we use atomic.CompareAndSwap `closed` to to make sure the channel is not closed twice. - closed uint32 // If keepOrder, results are stored in copTask.respChan, read them out one by one. tasks []*copTask @@ -344,11 +340,16 @@ type copIterator struct { // Otherwise, results are stored in respChan. respChan chan *copResponse - wg sync.WaitGroup vars *kv.Variables memTracker *memory.Tracker + + wg sync.WaitGroup + // closed represents when the Close is called. + // There are two cases we need to close the `finishCh` channel, one is when context is done, the other one is + // when the Close is called. we use atomic.CompareAndSwap `closed` to to make sure the channel is not closed twice. + closed uint32 } // copIteratorWorker receives tasks from copIteratorTaskSender, handles tasks and sends the copResponse to respChan. diff --git a/store/tikv/scan.go b/store/tikv/scan.go index 5804c5bf6d..645ecd93f0 100644 --- a/store/tikv/scan.go +++ b/store/tikv/scan.go @@ -29,16 +29,17 @@ import ( type Scanner struct { snapshot *tikvSnapshot batchSize int - valid bool cache []*pb.KvPair idx int nextStartKey []byte endKey []byte - eof bool // Use for reverse scan. - reverse bool nextEndKey []byte + reverse bool + + valid bool + eof bool } func newScanner(snapshot *tikvSnapshot, startKey []byte, endKey []byte, batchSize int, reverse bool) (*Scanner, error) { diff --git a/store/tikv/txn.go b/store/tikv/txn.go index 4c9ad48742..ec7a266e91 100644 --- a/store/tikv/txn.go +++ b/store/tikv/txn.go @@ -58,11 +58,9 @@ type tikvTxn struct { startTS uint64 startTime time.Time // Monotonic timestamp for recording txn time consuming. commitTS uint64 - valid bool lockKeys [][]byte lockedMap map[string]struct{} mu sync.Mutex // For thread-safe LockKeys function. - dirty bool setCnt int64 vars *kv.Variables committer *twoPhaseCommitter @@ -73,6 +71,9 @@ type tikvTxn struct { // StmtCommit/StmtRollback may change the confirmed position. assertions []assertionPair confirmed int + + valid bool + dirty bool } func newTiKVTxn(store *tikvStore) (*tikvTxn, error) { diff --git a/util/filesort/filesort.go b/util/filesort/filesort.go index e87d9430ee..9527b14c16 100644 --- a/util/filesort/filesort.go +++ b/util/filesort/filesort.go @@ -117,15 +117,11 @@ type FileSorter struct { nWorkers int // number of workers used in async sorting cWorker int // the next worker to which the sorting job is sent - mu sync.Mutex - wg sync.WaitGroup - tmpDir string - files []string - nFiles int - closed bool - fetched bool - external bool // mark the necessity of performing external file sort - cursor int // required when performing full in-memory sort + mu sync.Mutex + tmpDir string + files []string + nFiles int + cursor int // required when performing full in-memory sort rowHeap *rowHeap fds []*os.File @@ -135,6 +131,11 @@ type FileSorter struct { keySize int valSize int maxRowSize int + + wg sync.WaitGroup + closed bool + fetched bool + external bool // mark the necessity of performing external file sort } // Worker sorts file asynchronously. diff --git a/util/processinfo.go b/util/processinfo.go index 6338375de5..5c403413d6 100644 --- a/util/processinfo.go +++ b/util/processinfo.go @@ -24,22 +24,23 @@ import ( // ProcessInfo is a struct used for show processlist statement. type ProcessInfo struct { - ID uint64 - User string - Host string - DB string - Command byte - Plan interface{} - Time time.Time - State uint16 - Info string - CurTxnStartTS uint64 - StmtCtx *stmtctx.StatementContext - StatsInfo func(interface{}) map[string]uint64 - ExceedExpensiveTimeThresh bool + ID uint64 + User string + Host string + DB string + Plan interface{} + Time time.Time + Info string + CurTxnStartTS uint64 + StmtCtx *stmtctx.StatementContext + StatsInfo func(interface{}) map[string]uint64 // MaxExecutionTime is the timeout for select statement, in milliseconds. // If the query takes too long, kill it. MaxExecutionTime uint64 + + State uint16 + Command byte + ExceedExpensiveTimeThresh bool } // ToRowForShow returns []interface{} for the row data of "SHOW [FULL] PROCESSLIST".