Skip to content

Commit

Permalink
executor: resort fields for reducing the memory allocated (pingcap#14208
Browse files Browse the repository at this point in the history
)
  • Loading branch information
hsqlu authored and ngaut committed Dec 31, 2019
1 parent ae106f2 commit ebc4fa9
Show file tree
Hide file tree
Showing 7 changed files with 27 additions and 20 deletions.
2 changes: 1 addition & 1 deletion executor/benchmark_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -535,8 +535,8 @@ type hashJoinTestCase struct {
concurrency int
ctx sessionctx.Context
keyIdx []int
disk bool
joinType core.JoinType
disk bool
useOuterToBuild bool
}

Expand Down
2 changes: 1 addition & 1 deletion executor/cluster_reader.go
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand Down
9 changes: 5 additions & 4 deletions executor/insert_common.go
Original file line number Diff line number Diff line change
Expand Up @@ -42,8 +42,6 @@ type InsertValues struct {
curBatchCnt uint64
maxRowsInBatch uint64
lastInsertID uint64
hasRefCols bool
hasExtraHandle bool

SelectExec Executor

Expand All @@ -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.
Expand Down
16 changes: 10 additions & 6 deletions executor/join.go
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand All @@ -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,
Expand Down Expand Up @@ -725,7 +728,6 @@ type NestedLoopApplyExec struct {
outerExec Executor
innerFilter expression.CNFExprs
outerFilter expression.CNFExprs
outer bool

joiner joiner

Expand All @@ -742,6 +744,8 @@ type NestedLoopApplyExec struct {
hasMatch bool
hasNull bool

outer bool

memTracker *memory.Tracker // track memory usage.
}

Expand Down
2 changes: 1 addition & 1 deletion executor/load_data.go
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand Down
13 changes: 7 additions & 6 deletions executor/projection.go
Original file line number Diff line number Diff line change
Expand Up @@ -55,26 +55,27 @@ 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
numWorkers int64
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.
Expand Down
3 changes: 2 additions & 1 deletion executor/update.go
Original file line number Diff line number Diff line change
Expand Up @@ -42,14 +42,15 @@ 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.
// the columns ordinals is present in ordinal range format, @see plannercore.TblColPosInfos
tblColPosInfos plannercore.TblColPosInfoSlice
evalBuffer chunk.MutRow
allAssignmentsAreConstant bool

fetched bool
}

func (e *UpdateExec) exec(ctx context.Context, schema *expression.Schema) ([]types.Datum, error) {
Expand Down

0 comments on commit ebc4fa9

Please sign in to comment.