*: Optimize struct memory usage by adjust field order (#11629)
This commit is contained in:
@ -31,8 +31,8 @@ type datum struct {
|
||||
timeValue time.Time
|
||||
remains uint64
|
||||
repeats uint64
|
||||
probability uint32
|
||||
step int64
|
||||
probability uint32
|
||||
|
||||
init bool
|
||||
useRange bool
|
||||
|
||||
@ -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.
|
||||
|
||||
@ -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
|
||||
}
|
||||
|
||||
|
||||
@ -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.
|
||||
|
||||
@ -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
|
||||
|
||||
@ -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
|
||||
}
|
||||
|
||||
@ -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
|
||||
|
||||
@ -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.
|
||||
|
||||
@ -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.
|
||||
|
||||
@ -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.
|
||||
|
||||
@ -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,
|
||||
|
||||
@ -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) {
|
||||
|
||||
@ -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
|
||||
}
|
||||
|
||||
|
||||
@ -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.
|
||||
|
||||
@ -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.
|
||||
|
||||
@ -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.
|
||||
|
||||
@ -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
|
||||
|
||||
@ -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 {
|
||||
|
||||
13
kv/kv.go
13
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.
|
||||
|
||||
@ -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
|
||||
}
|
||||
|
||||
@ -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.
|
||||
|
||||
@ -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.
|
||||
|
||||
@ -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
|
||||
|
||||
@ -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.
|
||||
|
||||
@ -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.
|
||||
|
||||
@ -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 {
|
||||
|
||||
@ -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.
|
||||
|
||||
@ -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.
|
||||
|
||||
@ -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 {
|
||||
|
||||
@ -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 {
|
||||
|
||||
@ -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 {
|
||||
|
||||
@ -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.
|
||||
|
||||
@ -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) {
|
||||
|
||||
@ -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) {
|
||||
|
||||
@ -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.
|
||||
|
||||
@ -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".
|
||||
|
||||
Reference in New Issue
Block a user