*: Optimize struct memory usage by adjust field order (#11629)

This commit is contained in:
Maxwell
2019-08-13 17:20:53 +08:00
committed by winkyao
parent 3d6c561952
commit 6d51ad33fd
36 changed files with 175 additions and 168 deletions

View File

@ -31,8 +31,8 @@ type datum struct {
timeValue time.Time
remains uint64
repeats uint64
probability uint32
step int64
probability uint32
init bool
useRange bool

View File

@ -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.

View File

@ -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
}

View File

@ -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.

View File

@ -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

View File

@ -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
}

View File

@ -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

View File

@ -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.

View File

@ -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.

View File

@ -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.

View File

@ -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,

View File

@ -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) {

View File

@ -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
}

View File

@ -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.

View File

@ -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.

View File

@ -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.

View File

@ -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

View File

@ -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 {

View File

@ -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.

View File

@ -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
}

View File

@ -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.

View File

@ -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.

View File

@ -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

View File

@ -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.

View File

@ -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.

View File

@ -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 {

View File

@ -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.

View File

@ -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.

View File

@ -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 {

View File

@ -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 {

View File

@ -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 {

View File

@ -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.

View File

@ -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) {

View File

@ -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) {

View File

@ -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.

View File

@ -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".