executor: resort fields for reducing the memory allocated (#14208)

This commit is contained in:
Qiannan
2019-12-31 17:19:59 +08:00
committed by goroutine
parent ae106f2e3b
commit ebc4fa9dee
7 changed files with 27 additions and 20 deletions

View File

@ -535,8 +535,8 @@ type hashJoinTestCase struct {
concurrency int
ctx sessionctx.Context
keyIdx []int
disk bool
joinType core.JoinType
disk bool
useOuterToBuild bool
}

View File

@ -219,9 +219,9 @@ func (e *clusterConfigRetriever) retrieve(_ context.Context, sctx sessionctx.Con
type clusterServerInfoRetriever struct {
dummyCloser
retrieved bool
extractor *plannercore.ClusterTableExtractor
serverInfoType diagnosticspb.ServerInfoType
retrieved bool
}
// retrieve implements the clusterRetriever interface

View File

@ -42,8 +42,6 @@ type InsertValues struct {
curBatchCnt uint64
maxRowsInBatch uint64
lastInsertID uint64
hasRefCols bool
hasExtraHandle bool
SelectExec Executor
@ -56,14 +54,17 @@ type InsertValues struct {
insertColumns []*table.Column
allAssignmentsAreConstant bool
// colDefaultVals is used to store casted default value.
// Because not every insert statement needs colDefaultVals, so we will init the buffer lazily.
colDefaultVals []defaultVal
evalBuffer chunk.MutRow
evalBufferTypes []*types.FieldType
allAssignmentsAreConstant bool
hasRefCols bool
hasExtraHandle bool
// Fill the autoID lazily to datum. This is used for being compatible with JDBC using getGeneratedKeys().
// `insert|replace values` can guarantee consecutive autoID in a batch.
// Other statements like `insert select from` don't guarantee consecutive autoID.

View File

@ -53,9 +53,7 @@ type HashJoinExec struct {
concurrency uint
rowContainer *hashRowContainer
buildFinished chan error
// joinWorkerWaitGroup is for sync multiple join workers.
joinWorkerWaitGroup sync.WaitGroup
finished atomic.Value
// closeCh add a lock for closing executor.
closeCh chan struct{}
joinType plannercore.JoinType
@ -72,11 +70,16 @@ type HashJoinExec struct {
memTracker *memory.Tracker // track memory usage.
diskTracker *disk.Tracker // track disk usage.
prepared bool
isOuterJoin bool
outerMatchedStatus []*bitmap.ConcurrentBitmap
useOuterToBuild bool
prepared bool
isOuterJoin bool
// joinWorkerWaitGroup is for sync multiple join workers.
joinWorkerWaitGroup sync.WaitGroup
finished atomic.Value
}
// probeChkResource stores the result of the join probe side fetch worker,
@ -725,7 +728,6 @@ type NestedLoopApplyExec struct {
outerExec Executor
innerFilter expression.CNFExprs
outerFilter expression.CNFExprs
outer bool
joiner joiner
@ -742,6 +744,8 @@ type NestedLoopApplyExec struct {
hasMatch bool
hasNull bool
outer bool
memTracker *memory.Tracker // track memory usage.
}

View File

@ -507,9 +507,9 @@ type fieldWriter struct {
pos int
ReadBuf []byte
OutputBuf []byte
term string
enclosedChar byte
fieldTermChar byte
term string
isEnclosed bool
isLineStart bool
isFieldStart bool

View File

@ -55,10 +55,8 @@ type projectionOutput struct {
type ProjectionExec struct {
baseExecutor
evaluatorSuit *expression.EvaluatorSuite
calculateNoDelay bool
evaluatorSuit *expression.EvaluatorSuite
prepared bool
finishCh chan struct{}
outputCh chan *projectionOutput
fetcher projectionInputFetcher
@ -66,15 +64,18 @@ type ProjectionExec struct {
workers []*projectionWorker
childResult *chunk.Chunk
wg sync.WaitGroup
memTracker *memory.Tracker
// parentReqRows indicates how many rows the parent executor is
// requiring. It is set when parallelExecute() is called and used by the
// concurrent projectionInputFetcher.
//
// NOTE: It should be protected by atomic operations.
parentReqRows int64
memTracker *memory.Tracker
wg sync.WaitGroup
calculateNoDelay bool
prepared bool
}
// Open implements the Executor Open interface.

View File

@ -42,7 +42,6 @@ type UpdateExec struct {
rows [][]types.Datum // The rows fetched from TableExec.
newRowsData [][]types.Datum // The new values to be set.
fetched bool
cursor int
matched uint64 // a counter of matched rows during update
// tblColPosInfos stores relationship between column ordinal to its table handle.
@ -50,6 +49,8 @@ type UpdateExec struct {
tblColPosInfos plannercore.TblColPosInfoSlice
evalBuffer chunk.MutRow
allAssignmentsAreConstant bool
fetched bool
}
func (e *UpdateExec) exec(ctx context.Context, schema *expression.Schema) ([]types.Datum, error) {