diff --git a/executor/benchmark_test.go b/executor/benchmark_test.go index 3260ce2cac..fdcdc9c614 100644 --- a/executor/benchmark_test.go +++ b/executor/benchmark_test.go @@ -535,8 +535,8 @@ type hashJoinTestCase struct { concurrency int ctx sessionctx.Context keyIdx []int - disk bool joinType core.JoinType + disk bool useOuterToBuild bool } diff --git a/executor/cluster_reader.go b/executor/cluster_reader.go index c96b4b59ca..8b4caaa28c 100644 --- a/executor/cluster_reader.go +++ b/executor/cluster_reader.go @@ -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 diff --git a/executor/insert_common.go b/executor/insert_common.go index 3f917c4701..30ba752ea2 100644 --- a/executor/insert_common.go +++ b/executor/insert_common.go @@ -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. diff --git a/executor/join.go b/executor/join.go index 8e6c95f339..1697e8dae0 100644 --- a/executor/join.go +++ b/executor/join.go @@ -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. } diff --git a/executor/load_data.go b/executor/load_data.go index f922a87711..71bcb2110d 100644 --- a/executor/load_data.go +++ b/executor/load_data.go @@ -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 diff --git a/executor/projection.go b/executor/projection.go index bf27a651ec..ca50159045 100644 --- a/executor/projection.go +++ b/executor/projection.go @@ -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. diff --git a/executor/update.go b/executor/update.go index 4b36d90aa7..aa2491292e 100644 --- a/executor/update.go +++ b/executor/update.go @@ -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) {