// Copyright 2017 PingCAP, Inc. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. // You may obtain a copy of the License at // // http://www.apache.org/licenses/LICENSE-2.0 // // Unless required by applicable law or agreed to in writing, software // distributed under the License is distributed on an "AS IS" BASIS, // See the License for the specific language governing permissions and // limitations under the License. package executor import ( "bytes" "context" "fmt" "math" "runtime" "runtime/trace" "sort" "sync" "sync/atomic" "time" "unsafe" "github.com/pingcap/errors" "github.com/pingcap/parser/model" "github.com/pingcap/parser/mysql" "github.com/pingcap/parser/terror" "github.com/pingcap/tidb/distsql" "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/kv" plannercore "github.com/pingcap/tidb/planner/core" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/statistics" "github.com/pingcap/tidb/table" "github.com/pingcap/tidb/tablecodec" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util" "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/execdetails" "github.com/pingcap/tidb/util/logutil" "github.com/pingcap/tidb/util/memory" "github.com/pingcap/tidb/util/ranger" "github.com/pingcap/tipb/go-tipb" "go.uber.org/zap" ) var ( _ Executor = &TableReaderExecutor{} _ Executor = &IndexReaderExecutor{} _ Executor = &IndexLookUpExecutor{} ) // LookupTableTaskChannelSize represents the channel size of the index double read taskChan. var LookupTableTaskChannelSize int32 = 50 // lookupTableTask is created from a partial result of an index request which // contains the handles in those index keys. type lookupTableTask struct { handles []int64 rowIdx []int // rowIdx represents the handle index for every row. Only used when keep order. rows []chunk.Row idxRows *chunk.Chunk cursor int doneCh chan error // indexOrder map is used to save the original index order for the handles. // Without this map, the original index order might be lost. // The handles fetched from index is originally ordered by index, but we need handles to be ordered by itself // to do table request. indexOrder map[int64]int // duplicatedIndexOrder map likes indexOrder. But it's used when checkIndexValue isn't nil and // the same handle of index has multiple values. duplicatedIndexOrder map[int64]int // memUsage records the memory usage of this task calculated by table worker. // memTracker is used to release memUsage after task is done and unused. // // The sequence of function calls are: // 1. calculate task.memUsage. // 2. task.memTracker = tableWorker.memTracker // 3. task.memTracker.Consume(task.memUsage) // 4. task.memTracker.Consume(-task.memUsage) // // Step 1~3 are completed in "tableWorker.executeTask". // Step 4 is completed in "IndexLookUpExecutor.Next". memUsage int64 memTracker *memory.Tracker } func (task *lookupTableTask) Len() int { return len(task.rows) } func (task *lookupTableTask) Less(i, j int) bool { return task.rowIdx[i] < task.rowIdx[j] } func (task *lookupTableTask) Swap(i, j int) { task.rowIdx[i], task.rowIdx[j] = task.rowIdx[j], task.rowIdx[i] task.rows[i], task.rows[j] = task.rows[j], task.rows[i] } // Closeable is a interface for closeable structures. type Closeable interface { // Close closes the object. Close() error } // closeAll closes all objects even if an object returns an error. // If multiple objects returns error, the first error will be returned. func closeAll(objs ...Closeable) error { var err error for _, obj := range objs { if obj != nil { err1 := obj.Close() if err == nil && err1 != nil { err = err1 } } } if err != nil { return errors.Trace(err) } return nil } // handleIsExtra checks whether this column is a extra handle column generated during plan building phase. func handleIsExtra(col *expression.Column) bool { if col != nil && col.ID == model.ExtraHandleID { return true } return false } func splitRanges(ranges []*ranger.Range, keepOrder bool, desc bool) ([]*ranger.Range, []*ranger.Range) { if len(ranges) == 0 || ranges[0].LowVal[0].Kind() == types.KindInt64 { return ranges, nil } idx := sort.Search(len(ranges), func(i int) bool { return ranges[i].HighVal[0].GetUint64() > math.MaxInt64 }) if idx == len(ranges) { return ranges, nil } if ranges[idx].LowVal[0].GetUint64() > math.MaxInt64 { signedRanges := ranges[0:idx] unsignedRanges := ranges[idx:] if !keepOrder { return append(unsignedRanges, signedRanges...), nil } if desc { return unsignedRanges, signedRanges } return signedRanges, unsignedRanges } signedRanges := make([]*ranger.Range, 0, idx+1) unsignedRanges := make([]*ranger.Range, 0, len(ranges)-idx) signedRanges = append(signedRanges, ranges[0:idx]...) if !(ranges[idx].LowVal[0].GetUint64() == math.MaxInt64 && ranges[idx].LowExclude) { signedRanges = append(signedRanges, &ranger.Range{ LowVal: ranges[idx].LowVal, LowExclude: ranges[idx].LowExclude, HighVal: []types.Datum{types.NewUintDatum(math.MaxInt64)}, }) } if !(ranges[idx].HighVal[0].GetUint64() == math.MaxInt64+1 && ranges[idx].HighExclude) { unsignedRanges = append(unsignedRanges, &ranger.Range{ LowVal: []types.Datum{types.NewUintDatum(math.MaxInt64 + 1)}, HighVal: ranges[idx].HighVal, HighExclude: ranges[idx].HighExclude, }) } if idx < len(ranges) { unsignedRanges = append(unsignedRanges, ranges[idx+1:]...) } if !keepOrder { return append(unsignedRanges, signedRanges...), nil } if desc { return unsignedRanges, signedRanges } return signedRanges, unsignedRanges } // rebuildIndexRanges will be called if there's correlated column in access conditions. We will rebuild the range // by substitute correlated column with the constant. func rebuildIndexRanges(ctx sessionctx.Context, is *plannercore.PhysicalIndexScan, idxCols []*expression.Column, colLens []int) (ranges []*ranger.Range, err error) { access := make([]expression.Expression, 0, len(is.AccessCondition)) for _, cond := range is.AccessCondition { newCond, err1 := expression.SubstituteCorCol2Constant(cond) if err1 != nil { return nil, err1 } access = append(access, newCond) } ranges, _, err = ranger.DetachSimpleCondAndBuildRangeForIndex(ctx, access, idxCols, colLens) return ranges, err } // IndexReaderExecutor sends dag request and reads index data from kv layer. type IndexReaderExecutor struct { baseExecutor // For a partitioned table, the IndexReaderExecutor works on a partition, so // the type of this table field is actually `table.PhysicalTable`. table table.Table index *model.IndexInfo physicalTableID int64 ranges []*ranger.Range // kvRanges are only used for union scan. kvRanges []kv.KeyRange dagPB *tipb.DAGRequest startTS uint64 // result returns one or more distsql.PartialResult and each PartialResult is returned by one region. result distsql.SelectResult // columns are only required by union scan. columns []*model.ColumnInfo // outputColumns are only required by union scan. outputColumns []*expression.Column feedback *statistics.QueryFeedback streaming bool keepOrder bool desc bool corColInFilter bool corColInAccess bool idxCols []*expression.Column colLens []int plans []plannercore.PhysicalPlan memTracker *memory.Tracker selectResultHook // for testing } // Close clears all resources hold by current object. func (e *IndexReaderExecutor) Close() error { err := e.result.Close() e.result = nil e.ctx.StoreQueryFeedback(e.feedback) return err } // Next implements the Executor Next interface. func (e *IndexReaderExecutor) Next(ctx context.Context, req *chunk.Chunk) error { err := e.result.Next(ctx, req) if err != nil { e.feedback.Invalidate() } return err } // Open implements the Executor Open interface. func (e *IndexReaderExecutor) Open(ctx context.Context) error { var err error if e.corColInAccess { e.ranges, err = rebuildIndexRanges(e.ctx, e.plans[0].(*plannercore.PhysicalIndexScan), e.idxCols, e.colLens) if err != nil { return err } } kvRanges, err := distsql.IndexRangesToKVRanges(e.ctx.GetSessionVars().StmtCtx, e.physicalTableID, e.index.ID, e.ranges, e.feedback) if err != nil { e.feedback.Invalidate() return err } return e.open(ctx, kvRanges) } func (e *IndexReaderExecutor) open(ctx context.Context, kvRanges []kv.KeyRange) error { var err error if e.corColInFilter { e.dagPB.Executors, _, err = constructDistExec(e.ctx, e.plans) if err != nil { return err } } if e.runtimeStats != nil { collExec := true e.dagPB.CollectExecutionSummaries = &collExec } e.kvRanges = kvRanges e.memTracker = memory.NewTracker(e.id, -1) e.memTracker.AttachTo(e.ctx.GetSessionVars().StmtCtx.MemTracker) var builder distsql.RequestBuilder kvReq, err := builder.SetKeyRanges(kvRanges). SetDAGRequest(e.dagPB). SetStartTS(e.startTS). SetDesc(e.desc). SetKeepOrder(e.keepOrder). SetStreaming(e.streaming). SetFromSessionVars(e.ctx.GetSessionVars()). SetMemTracker(e.memTracker). Build() if err != nil { e.feedback.Invalidate() return err } e.result, err = e.SelectResult(ctx, e.ctx, kvReq, retTypes(e), e.feedback, getPhysicalPlanIDs(e.plans), e.id) if err != nil { e.feedback.Invalidate() return err } e.result.Fetch(ctx) return nil } // IndexLookUpExecutor implements double read for index scan. type IndexLookUpExecutor struct { baseExecutor table table.Table index *model.IndexInfo ranges []*ranger.Range dagPB *tipb.DAGRequest startTS uint64 // handleIdx is the index of handle, which is only used for case of keeping order. handleIdx int tableRequest *tipb.DAGRequest // columns are only required by union scan. columns []*model.ColumnInfo *dataReaderBuilder // All fields above are immutable. idxWorkerWg sync.WaitGroup tblWorkerWg sync.WaitGroup finished chan struct{} resultCh chan *lookupTableTask resultCurr *lookupTableTask feedback *statistics.QueryFeedback // memTracker is used to track the memory usage of this executor. memTracker *memory.Tracker // checkIndexValue is used to check the consistency of the index data. *checkIndexValue kvRanges []kv.KeyRange workerStarted bool keepOrder bool desc bool indexStreaming bool tableStreaming bool corColInIdxSide bool corColInTblSide bool corColInAccess bool idxPlans []plannercore.PhysicalPlan tblPlans []plannercore.PhysicalPlan idxCols []*expression.Column colLens []int // PushedLimit is used to skip the preceding and tailing handles when Limit is sunk into IndexLookUpReader. PushedLimit *plannercore.PushedDownLimit stats *IndexLookUpRunTimeStats } type checkIndexValue struct { idxColTps []*types.FieldType idxTblCols []*table.Column } // Open implements the Executor Open interface. func (e *IndexLookUpExecutor) Open(ctx context.Context) error { var err error if e.corColInAccess { e.ranges, err = rebuildIndexRanges(e.ctx, e.idxPlans[0].(*plannercore.PhysicalIndexScan), e.idxCols, e.colLens) if err != nil { return err } } e.kvRanges, err = distsql.IndexRangesToKVRanges(e.ctx.GetSessionVars().StmtCtx, getPhysicalTableID(e.table), e.index.ID, e.ranges, e.feedback) if err != nil { e.feedback.Invalidate() return err } e.initRuntimeStats() err = e.open(ctx) if err != nil { e.feedback.Invalidate() } return err } func (e *IndexLookUpExecutor) open(ctx context.Context) error { // We have to initialize "memTracker" and other execution resources in here // instead of in function "Open", because this "IndexLookUpExecutor" may be // constructed by a "IndexLookUpJoin" and "Open" will not be called in that // situation. e.memTracker = memory.NewTracker(e.id, -1) e.memTracker.AttachTo(e.ctx.GetSessionVars().StmtCtx.MemTracker) e.finished = make(chan struct{}) e.resultCh = make(chan *lookupTableTask, atomic.LoadInt32(&LookupTableTaskChannelSize)) var err error if e.corColInIdxSide { e.dagPB.Executors, _, err = constructDistExec(e.ctx, e.idxPlans) if err != nil { return err } } if e.corColInTblSide { e.tableRequest.Executors, _, err = constructDistExec(e.ctx, e.tblPlans) if err != nil { return err } } return nil } func (e *IndexLookUpExecutor) startWorkers(ctx context.Context, initBatchSize int) error { // indexWorker will write to workCh and tableWorker will read from workCh, // so fetching index and getting table data can run concurrently. workCh := make(chan *lookupTableTask, 1) if err := e.startIndexWorker(ctx, e.kvRanges, workCh, initBatchSize); err != nil { return err } e.startTableWorker(ctx, workCh) e.workerStarted = true return nil } // startIndexWorker launch a background goroutine to fetch handles, send the results to workCh. func (e *IndexLookUpExecutor) startIndexWorker(ctx context.Context, kvRanges []kv.KeyRange, workCh chan<- *lookupTableTask, initBatchSize int) error { if e.runtimeStats != nil { collExec := true e.dagPB.CollectExecutionSummaries = &collExec } tracker := memory.NewTracker(memory.LabelForIndexWorker, -1) tracker.AttachTo(e.memTracker) var builder distsql.RequestBuilder kvReq, err := builder.SetKeyRanges(kvRanges). SetDAGRequest(e.dagPB). SetStartTS(e.startTS). SetDesc(e.desc). SetKeepOrder(e.keepOrder). SetStreaming(e.indexStreaming). SetFromSessionVars(e.ctx.GetSessionVars()). SetMemTracker(tracker). Build() if err != nil { return err } tps := []*types.FieldType{types.NewFieldType(mysql.TypeLonglong)} if e.checkIndexValue != nil { tps = e.idxColTps } idxID := e.getIndexPlanRootID() // Since the first read only need handle information. So its returned col is only 1. result, err := distsql.SelectWithRuntimeStats(ctx, e.ctx, kvReq, tps, e.feedback, getPhysicalPlanIDs(e.idxPlans), idxID) if err != nil { return err } result.Fetch(ctx) worker := &indexWorker{ idxLookup: e, workCh: workCh, finished: e.finished, resultCh: e.resultCh, keepOrder: e.keepOrder, batchSize: initBatchSize, checkIndexValue: e.checkIndexValue, maxBatchSize: e.ctx.GetSessionVars().IndexLookupSize, maxChunkSize: e.maxChunkSize, PushedLimit: e.PushedLimit, } if worker.batchSize > worker.maxBatchSize { worker.batchSize = worker.maxBatchSize } e.idxWorkerWg.Add(1) go func() { defer trace.StartRegion(ctx, "IndexLookUpIndexWorker").End() ctx1, cancel := context.WithCancel(ctx) _, err := worker.fetchHandles(ctx1, result) if err != nil { e.feedback.Invalidate() } cancel() if err := result.Close(); err != nil { logutil.Logger(ctx).Error("close Select result failed", zap.Error(err)) } e.ctx.StoreQueryFeedback(e.feedback) close(workCh) close(e.resultCh) e.idxWorkerWg.Done() }() return nil } // startTableWorker launchs some background goroutines which pick tasks from workCh and execute the task. func (e *IndexLookUpExecutor) startTableWorker(ctx context.Context, workCh <-chan *lookupTableTask) { lookupConcurrencyLimit := e.ctx.GetSessionVars().IndexLookupConcurrency e.tblWorkerWg.Add(lookupConcurrencyLimit) for i := 0; i < lookupConcurrencyLimit; i++ { workerID := i worker := &tableWorker{ idxLookup: e, workCh: workCh, finished: e.finished, buildTblReader: e.buildTableReader, keepOrder: e.keepOrder, handleIdx: e.handleIdx, checkIndexValue: e.checkIndexValue, memTracker: memory.NewTracker(workerID, -1), } worker.memTracker.AttachTo(e.memTracker) ctx1, cancel := context.WithCancel(ctx) go func() { defer trace.StartRegion(ctx1, "IndexLookUpTableWorker").End() startTime := time.Now() worker.pickAndExecTask(ctx1) if e.stats != nil { atomic.AddInt64(&e.stats.TableRowScan, int64(time.Since(startTime))) } cancel() e.tblWorkerWg.Done() }() } } func (e *IndexLookUpExecutor) buildTableReader(ctx context.Context, handles []int64) (Executor, error) { tableReaderExec := &TableReaderExecutor{ baseExecutor: newBaseExecutor(e.ctx, e.schema, e.getTableRootPlanID()), table: e.table, dagPB: e.tableRequest, startTS: e.startTS, columns: e.columns, streaming: e.tableStreaming, feedback: statistics.NewQueryFeedback(0, nil, 0, false), corColInFilter: e.corColInTblSide, plans: e.tblPlans, } tableReaderExec.buildVirtualColumnInfo() tableReader, err := e.dataReaderBuilder.buildTableReaderFromHandles(ctx, tableReaderExec, handles, true) if err != nil { logutil.Logger(ctx).Error("build table reader from handles failed", zap.Error(err)) return nil, err } return tableReader, nil } // Close implements Exec Close interface. func (e *IndexLookUpExecutor) Close() error { if !e.workerStarted || e.finished == nil { return nil } close(e.finished) // Drain the resultCh and discard the result, in case that Next() doesn't fully // consume the data, background worker still writing to resultCh and block forever. for range e.resultCh { } e.idxWorkerWg.Wait() e.tblWorkerWg.Wait() e.finished = nil e.workerStarted = false e.memTracker = nil e.resultCurr = nil return nil } // Next implements Exec Next interface. func (e *IndexLookUpExecutor) Next(ctx context.Context, req *chunk.Chunk) error { if !e.workerStarted { if err := e.startWorkers(ctx, req.RequiredRows()); err != nil { return err } } req.Reset() for { resultTask, err := e.getResultTask() if err != nil { return err } if resultTask == nil { return nil } for resultTask.cursor < len(resultTask.rows) { req.AppendRow(resultTask.rows[resultTask.cursor]) resultTask.cursor++ if req.IsFull() { return nil } } } } func (e *IndexLookUpExecutor) getResultTask() (*lookupTableTask, error) { if e.resultCurr != nil && e.resultCurr.cursor < len(e.resultCurr.rows) { return e.resultCurr, nil } task, ok := <-e.resultCh if !ok { return nil, nil } if err := <-task.doneCh; err != nil { return nil, err } // Release the memory usage of last task before we handle a new task. if e.resultCurr != nil { e.resultCurr.memTracker.Consume(-e.resultCurr.memUsage) } e.resultCurr = task return e.resultCurr, nil } func (e *IndexLookUpExecutor) initRuntimeStats() { if e.runtimeStats != nil { if e.stats == nil { e.stats = &IndexLookUpRunTimeStats{ IndexScan: 0, TableRowScan: 0, TableTaskNum: 0, Concurrency: e.ctx.GetSessionVars().IndexLookupConcurrency, } e.ctx.GetSessionVars().StmtCtx.RuntimeStatsColl.RegisterStats(e.id, e.stats) } } } func (e *IndexLookUpExecutor) getIndexPlanRootID() int { if len(e.idxPlans) > 0 { return e.idxPlans[len(e.idxPlans)-1].ID() } return e.id } func (e *IndexLookUpExecutor) getTableRootPlanID() int { if len(e.tblPlans) > 0 { return e.tblPlans[len(e.tblPlans)-1].ID() } return e.id } // indexWorker is used by IndexLookUpExecutor to maintain index lookup background goroutines. type indexWorker struct { idxLookup *IndexLookUpExecutor workCh chan<- *lookupTableTask finished <-chan struct{} resultCh chan<- *lookupTableTask keepOrder bool // batchSize is for lightweight startup. It will be increased exponentially until reaches the max batch size value. batchSize int maxBatchSize int maxChunkSize int // checkIndexValue is used to check the consistency of the index data. *checkIndexValue // PushedLimit is used to skip the preceding and tailing handles when Limit is sunk into IndexLookUpReader. PushedLimit *plannercore.PushedDownLimit } // fetchHandles fetches a batch of handles from index data and builds the index lookup tasks. // The tasks are sent to workCh to be further processed by tableWorker, and sent to e.resultCh // at the same time to keep data ordered. func (w *indexWorker) fetchHandles(ctx context.Context, result distsql.SelectResult) (count uint64, err error) { defer func() { if r := recover(); r != nil { buf := make([]byte, 4096) stackSize := runtime.Stack(buf, false) buf = buf[:stackSize] logutil.Logger(ctx).Error("indexWorker in IndexLookupExecutor panicked", zap.String("stack", string(buf))) err4Panic := errors.Errorf("%v", r) doneCh := make(chan error, 1) doneCh <- err4Panic w.resultCh <- &lookupTableTask{ doneCh: doneCh, } if err != nil { err = errors.Trace(err4Panic) } } }() var chk *chunk.Chunk if w.checkIndexValue != nil { chk = chunk.NewChunkWithCapacity(w.idxColTps, w.maxChunkSize) } else { chk = chunk.NewChunkWithCapacity([]*types.FieldType{types.NewFieldType(mysql.TypeLonglong)}, w.idxLookup.maxChunkSize) } idxID := w.idxLookup.getIndexPlanRootID() var basicStats *execdetails.BasicRuntimeStats if w.idxLookup.ctx.GetSessionVars().StmtCtx.RuntimeStatsColl != nil { if idxID != w.idxLookup.id { basicStats = &execdetails.BasicRuntimeStats{} w.idxLookup.ctx.GetSessionVars().StmtCtx.RuntimeStatsColl.RegisterStats(idxID, basicStats) } } for { startTime := time.Now() handles, retChunk, scannedKeys, err := w.extractTaskHandles(ctx, chk, result, count) if err != nil { doneCh := make(chan error, 1) doneCh <- err w.resultCh <- &lookupTableTask{ doneCh: doneCh, } return count, err } count += scannedKeys if len(handles) == 0 { return count, nil } task := w.buildTableTask(handles, retChunk) if w.idxLookup.stats != nil { atomic.AddInt64(&w.idxLookup.stats.IndexScan, int64(time.Since(startTime))) } select { case <-ctx.Done(): return count, nil case <-w.finished: return count, nil case w.workCh <- task: w.resultCh <- task } if basicStats != nil { basicStats.Record(time.Since(startTime), chk.NumRows()) } } } func (w *indexWorker) extractTaskHandles(ctx context.Context, chk *chunk.Chunk, idxResult distsql.SelectResult, count uint64) ( handles []int64, retChk *chunk.Chunk, scannedKeys uint64, err error) { handleOffset := chk.NumCols() - 1 handles = make([]int64, 0, w.batchSize) // PushedLimit would always be nil for CheckIndex or CheckTable, we add this check just for insurance. checkLimit := (w.PushedLimit != nil) && (w.checkIndexValue == nil) for len(handles) < w.batchSize { requiredRows := w.batchSize - len(handles) if checkLimit { if w.PushedLimit.Offset+w.PushedLimit.Count <= scannedKeys+count { return handles, nil, scannedKeys, nil } leftCnt := w.PushedLimit.Offset + w.PushedLimit.Count - scannedKeys - count if uint64(requiredRows) > leftCnt { requiredRows = int(leftCnt) } } chk.SetRequiredRows(requiredRows, w.maxChunkSize) err = errors.Trace(idxResult.Next(ctx, chk)) if err != nil { return handles, nil, scannedKeys, err } if chk.NumRows() == 0 { return handles, retChk, scannedKeys, nil } for i := 0; i < chk.NumRows(); i++ { scannedKeys++ if checkLimit { if (count + scannedKeys) <= w.PushedLimit.Offset { // Skip the preceding Offset handles. continue } if (count + scannedKeys) > (w.PushedLimit.Offset + w.PushedLimit.Count) { // Skip the handles after Offset+Count. return handles, nil, scannedKeys, nil } } h := chk.GetRow(i).GetInt64(handleOffset) handles = append(handles, h) } if w.checkIndexValue != nil { if retChk == nil { retChk = chunk.NewChunkWithCapacity(w.idxColTps, w.batchSize) } retChk.Append(chk, 0, chk.NumRows()) } } w.batchSize *= 2 if w.batchSize > w.maxBatchSize { w.batchSize = w.maxBatchSize } return handles, retChk, scannedKeys, nil } func (w *indexWorker) buildTableTask(handles []int64, retChk *chunk.Chunk) *lookupTableTask { var indexOrder map[int64]int var duplicatedIndexOrder map[int64]int if w.keepOrder { // Save the index order. indexOrder = make(map[int64]int, len(handles)) for i, h := range handles { indexOrder[h] = i } } if w.checkIndexValue != nil { // Save the index order. indexOrder = make(map[int64]int, len(handles)) duplicatedIndexOrder = make(map[int64]int) for i, h := range handles { if _, ok := indexOrder[h]; ok { duplicatedIndexOrder[h] = i } else { indexOrder[h] = i } } } task := &lookupTableTask{ handles: handles, indexOrder: indexOrder, duplicatedIndexOrder: duplicatedIndexOrder, idxRows: retChk, } task.doneCh = make(chan error, 1) return task } // tableWorker is used by IndexLookUpExecutor to maintain table lookup background goroutines. type tableWorker struct { idxLookup *IndexLookUpExecutor workCh <-chan *lookupTableTask finished <-chan struct{} buildTblReader func(ctx context.Context, handles []int64) (Executor, error) keepOrder bool handleIdx int // memTracker is used to track the memory usage of this executor. memTracker *memory.Tracker // checkIndexValue is used to check the consistency of the index data. *checkIndexValue } // pickAndExecTask picks tasks from workCh, and execute them. func (w *tableWorker) pickAndExecTask(ctx context.Context) { var task *lookupTableTask var ok bool defer func() { if r := recover(); r != nil { buf := make([]byte, 4096) stackSize := runtime.Stack(buf, false) buf = buf[:stackSize] logutil.Logger(ctx).Error("tableWorker in IndexLookUpExecutor panicked", zap.String("stack", string(buf))) task.doneCh <- errors.Errorf("%v", r) } }() for { // Don't check ctx.Done() on purpose. If background worker get the signal and all // exit immediately, session's goroutine doesn't know this and still calling Next(), // it may block reading task.doneCh forever. select { case task, ok = <-w.workCh: if !ok { return } case <-w.finished: return } err := w.executeTask(ctx, task) if w.idxLookup.stats != nil { atomic.AddInt64(&w.idxLookup.stats.TableTaskNum, 1) } task.doneCh <- err } } // IndexLookUpRunTimeStats record the indexlookup runtime stat type IndexLookUpRunTimeStats struct { IndexScan int64 TableRowScan int64 TableTaskNum int64 Concurrency int } func (e *IndexLookUpRunTimeStats) String() string { var buf bytes.Buffer indexScan := atomic.LoadInt64(&e.IndexScan) tableScan := atomic.LoadInt64(&e.TableRowScan) tableTaskNum := atomic.LoadInt64(&e.TableTaskNum) concurrency := e.Concurrency if indexScan != 0 { buf.WriteString(fmt.Sprintf("index_task:%s", time.Duration(indexScan))) } if tableScan != 0 { if buf.Len() > 0 { buf.WriteByte(',') } buf.WriteString(fmt.Sprintf(" table_task:{num:%d, concurrency:%d, time:%s}", tableTaskNum, concurrency, time.Duration(tableScan))) } return buf.String() } // Clone implements the RuntimeStats interface. func (e *IndexLookUpRunTimeStats) Clone() execdetails.RuntimeStats { newRs := &IndexLookUpRunTimeStats{ IndexScan: e.IndexScan, TableRowScan: e.TableRowScan, TableTaskNum: e.TableTaskNum, Concurrency: e.Concurrency, } return newRs } // Merge implements the RuntimeStats interface. func (e *IndexLookUpRunTimeStats) Merge(other execdetails.RuntimeStats) { tmp, ok := other.(*IndexLookUpRunTimeStats) if !ok { return } e.IndexScan += tmp.IndexScan e.TableRowScan += tmp.TableRowScan e.TableTaskNum += tmp.TableTaskNum e.Concurrency += tmp.Concurrency } // Tp implements the RuntimeStats interface. func (e *IndexLookUpRunTimeStats) Tp() int { return execdetails.TpIndexLookUpRunTimeStats } func (w *tableWorker) compareData(ctx context.Context, task *lookupTableTask, tableReader Executor) error { chk := newFirstChunk(tableReader) tblInfo := w.idxLookup.table.Meta() vals := make([]types.Datum, 0, len(w.idxTblCols)) for { err := Next(ctx, tableReader, chk) if err != nil { return errors.Trace(err) } if chk.NumRows() == 0 { for h := range task.indexOrder { idxRow := task.idxRows.GetRow(task.indexOrder[h]) return errors.Errorf("handle %#v, index:%#v != record:%#v", h, idxRow.GetDatum(0, w.idxColTps[0]), nil) } break } iter := chunk.NewIterator4Chunk(chk) for row := iter.Begin(); row != iter.End(); row = iter.Next() { handle := row.GetInt64(w.handleIdx) offset, ok := task.indexOrder[handle] if !ok { offset = task.duplicatedIndexOrder[handle] } delete(task.indexOrder, handle) idxRow := task.idxRows.GetRow(offset) vals = vals[:0] for i, col := range w.idxTblCols { vals = append(vals, row.GetDatum(i, &col.FieldType)) } vals = tablecodec.TruncateIndexValuesIfNeeded(tblInfo, w.idxLookup.index, vals) for i, val := range vals { col := w.idxTblCols[i] tp := &col.FieldType ret := chunk.Compare(idxRow, i, &val) if ret != 0 { return errors.Errorf("col %s, handle %#v, index:%#v != record:%#v", col.Name, handle, idxRow.GetDatum(i, tp), val) } } } } return nil } // executeTask executes the table look up tasks. We will construct a table reader and send request by handles. // Then we hold the returning rows and finish this task. func (w *tableWorker) executeTask(ctx context.Context, task *lookupTableTask) error { tableReader, err := w.buildTblReader(ctx, task.handles) if err != nil { logutil.Logger(ctx).Error("build table reader failed", zap.Error(err)) return err } defer terror.Call(tableReader.Close) if w.checkIndexValue != nil { return w.compareData(ctx, task, tableReader) } task.memTracker = w.memTracker memUsage := int64(cap(task.handles) * 8) task.memUsage = memUsage task.memTracker.Consume(memUsage) handleCnt := len(task.handles) task.rows = make([]chunk.Row, 0, handleCnt) for { chk := newFirstChunk(tableReader) err = Next(ctx, tableReader, chk) if err != nil { logutil.Logger(ctx).Error("table reader fetch next chunk failed", zap.Error(err)) return err } if chk.NumRows() == 0 { break } memUsage = chk.MemoryUsage() task.memUsage += memUsage task.memTracker.Consume(memUsage) iter := chunk.NewIterator4Chunk(chk) for row := iter.Begin(); row != iter.End(); row = iter.Next() { task.rows = append(task.rows, row) } } defer trace.StartRegion(ctx, "IndexLookUpTableCompute").End() memUsage = int64(cap(task.rows)) * int64(unsafe.Sizeof(chunk.Row{})) task.memUsage += memUsage task.memTracker.Consume(memUsage) if w.keepOrder { task.rowIdx = make([]int, 0, len(task.rows)) for i := range task.rows { handle := task.rows[i].GetInt64(w.handleIdx) task.rowIdx = append(task.rowIdx, task.indexOrder[handle]) } memUsage = int64(cap(task.rowIdx) * 4) task.memUsage += memUsage task.memTracker.Consume(memUsage) sort.Sort(task) } if handleCnt != len(task.rows) && !util.HasCancelled(ctx) { if len(w.idxLookup.tblPlans) == 1 { obtainedHandlesMap := make(map[int64]struct{}, len(task.rows)) for _, row := range task.rows { handle := row.GetInt64(w.handleIdx) obtainedHandlesMap[handle] = struct{}{} } logutil.Logger(ctx).Error("inconsistent index handles", zap.String("index", w.idxLookup.index.Name.O), zap.Int("index_cnt", handleCnt), zap.Int("table_cnt", len(task.rows)), zap.Int64s("missing_handles", GetLackHandles(task.handles, obtainedHandlesMap)), zap.Int64s("total_handles", task.handles)) // table scan in double read can never has conditions according to convertToIndexScan. // if this table scan has no condition, the number of rows it returns must equal to the length of handles. return errors.Errorf("inconsistent index %s handle count %d isn't equal to value count %d", w.idxLookup.index.Name.O, handleCnt, len(task.rows)) } } return nil } // GetLackHandles gets the handles in expectedHandles but not in obtainedHandlesMap. func GetLackHandles(expectedHandles []int64, obtainedHandlesMap map[int64]struct{}) []int64 { diffCnt := len(expectedHandles) - len(obtainedHandlesMap) diffHandles := make([]int64, 0, diffCnt) var cnt int for _, handle := range expectedHandles { isExist := false if _, ok := obtainedHandlesMap[handle]; ok { delete(obtainedHandlesMap, handle) isExist = true } if !isExist { diffHandles = append(diffHandles, handle) cnt++ if cnt == diffCnt { break } } } return diffHandles } func getPhysicalPlanIDs(plans []plannercore.PhysicalPlan) []int { planIDs := make([]int, 0, len(plans)) for _, p := range plans { planIDs = append(planIDs, p.ID()) } return planIDs }