executor: resort fields for reducing the memory allocated (#14208)
This commit is contained in:
@ -535,8 +535,8 @@ type hashJoinTestCase struct {
|
||||
concurrency int
|
||||
ctx sessionctx.Context
|
||||
keyIdx []int
|
||||
disk bool
|
||||
joinType core.JoinType
|
||||
disk bool
|
||||
useOuterToBuild bool
|
||||
}
|
||||
|
||||
|
||||
@ -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
|
||||
|
||||
@ -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.
|
||||
|
||||
@ -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.
|
||||
}
|
||||
|
||||
|
||||
@ -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
|
||||
|
||||
@ -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.
|
||||
|
||||
@ -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) {
|
||||
|
||||
Reference in New Issue
Block a user