diff --git a/pkg/executor/BUILD.bazel b/pkg/executor/BUILD.bazel index 39dd946d59307..9d1f970de35ad 100644 --- a/pkg/executor/BUILD.bazel +++ b/pkg/executor/BUILD.bazel @@ -157,6 +157,7 @@ go_library( "//pkg/planner/core/base", "//pkg/planner/util", "//pkg/planner/util/fixcontrol", + "//pkg/planner/util/handlecol", "//pkg/plugin", "//pkg/privilege", "//pkg/privilege/privileges", @@ -415,6 +416,7 @@ go_test( "//pkg/planner/core/base", "//pkg/planner/property", "//pkg/planner/util", + "//pkg/planner/util/handlecol", "//pkg/server", "//pkg/session", "//pkg/session/types", diff --git a/pkg/executor/admin.go b/pkg/executor/admin.go index a859de119e8e2..9e52a7e7d2744 100644 --- a/pkg/executor/admin.go +++ b/pkg/executor/admin.go @@ -28,7 +28,7 @@ import ( "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/parser/terror" - plannercore "github.com/pingcap/tidb/pkg/planner/core" + "github.com/pingcap/tidb/pkg/planner/util/handlecol" "github.com/pingcap/tidb/pkg/table" "github.com/pingcap/tidb/pkg/table/tables" "github.com/pingcap/tidb/pkg/tablecodec" @@ -189,7 +189,7 @@ type RecoverIndexExec struct { columns []*model.ColumnInfo colFieldTypes []*types.FieldType srcChunk *chunk.Chunk - handleCols plannercore.HandleCols + handleCols handlecol.HandleCols containsGenedCol bool cols []*expression.Column @@ -386,7 +386,7 @@ func (e *RecoverIndexExec) fetchRecoverRows(ctx context.Context, srcResult dists return nil, err } e.idxValsBufs[result.scanRowCount] = idxVals - rsData := tables.TryGetHandleRestoredDataWrapper(e.table.Meta(), plannercore.GetCommonHandleDatum(e.handleCols, row), nil, e.index.Meta()) + rsData := tables.TryGetHandleRestoredDataWrapper(e.table.Meta(), handlecol.GetCommonHandleDatum(e.handleCols, row), nil, e.index.Meta()) e.recoverRows = append(e.recoverRows, recoverRows{handle: handle, idxVals: idxVals, rsData: rsData, skip: true}) result.scanRowCount++ result.currentHandle = handle @@ -591,7 +591,7 @@ type CleanupIndexExec struct { columns []*model.ColumnInfo idxColFieldTypes []*types.FieldType idxChunk *chunk.Chunk - handleCols plannercore.HandleCols + handleCols handlecol.HandleCols idxValues *kv.HandleMap // kv.Handle -> [][]types.Datum batchSize uint64 diff --git a/pkg/executor/analyze_col.go b/pkg/executor/analyze_col.go index 2ba5272041ac7..760a2d9e1e3ac 100644 --- a/pkg/executor/analyze_col.go +++ b/pkg/executor/analyze_col.go @@ -30,6 +30,7 @@ import ( "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/planner/core" + "github.com/pingcap/tidb/pkg/planner/util/handlecol" "github.com/pingcap/tidb/pkg/statistics" "github.com/pingcap/tidb/pkg/tablecodec" "github.com/pingcap/tidb/pkg/types" @@ -46,7 +47,7 @@ type AnalyzeColumnsExec struct { tableInfo *model.TableInfo colsInfo []*model.ColumnInfo - handleCols core.HandleCols + handleCols handlecol.HandleCols commonHandle *model.IndexInfo resultHandler *tableResultHandler indexes []*model.IndexInfo @@ -379,7 +380,7 @@ func (e *AnalyzeColumnsExecV1) analyzeColumnsPushDownV1() *statistics.AnalyzeRes } } -func hasPkHist(handleCols core.HandleCols) bool { +func hasPkHist(handleCols handlecol.HandleCols) bool { return handleCols != nil && handleCols.IsInt() } diff --git a/pkg/executor/builder.go b/pkg/executor/builder.go index 42448f1189b51..1dfd61339dd1c 100644 --- a/pkg/executor/builder.go +++ b/pkg/executor/builder.go @@ -61,6 +61,7 @@ import ( plannercore "github.com/pingcap/tidb/pkg/planner/core" "github.com/pingcap/tidb/pkg/planner/core/base" plannerutil "github.com/pingcap/tidb/pkg/planner/util" + "github.com/pingcap/tidb/pkg/planner/util/handlecol" "github.com/pingcap/tidb/pkg/sessionctx" "github.com/pingcap/tidb/pkg/sessionctx/stmtctx" "github.com/pingcap/tidb/pkg/sessionctx/variable" @@ -612,14 +613,14 @@ func (b *executorBuilder) buildRecoverIndex(v *plannercore.RecoverIndex) exec.Ex } func buildHandleColsForExec(sctx *stmtctx.StatementContext, tblInfo *model.TableInfo, - allColInfo []*model.ColumnInfo) plannercore.HandleCols { + allColInfo []*model.ColumnInfo) handlecol.HandleCols { if !tblInfo.IsCommonHandle { extraColPos := len(allColInfo) - 1 intCol := &expression.Column{ Index: extraColPos, RetType: types.NewFieldType(mysql.TypeLonglong), } - return plannercore.NewIntHandleCols(intCol) + return handlecol.NewIntHandleCols(intCol) } tblCols := make([]*expression.Column, len(tblInfo.Columns)) for i := 0; i < len(tblInfo.Columns); i++ { @@ -637,7 +638,7 @@ func buildHandleColsForExec(sctx *stmtctx.StatementContext, tblInfo *model.Table } } } - return plannercore.NewCommonHandleCols(sctx, tblInfo, pkIdx, tblCols) + return handlecol.NewCommonHandleCols(sctx, tblInfo, pkIdx, tblCols) } func (b *executorBuilder) buildCleanupIndex(v *plannercore.CleanupIndex) exec.Executor { @@ -2280,7 +2281,7 @@ func (b *executorBuilder) buildUnionAll(v *plannercore.PhysicalUnionAll) exec.Ex return e } -func buildHandleColsForSplit(sc *stmtctx.StatementContext, tbInfo *model.TableInfo) plannercore.HandleCols { +func buildHandleColsForSplit(sc *stmtctx.StatementContext, tbInfo *model.TableInfo) handlecol.HandleCols { if tbInfo.IsCommonHandle { primaryIdx := tables.FindPrimaryIndex(tbInfo) tableCols := make([]*expression.Column, len(tbInfo.Columns)) @@ -2293,12 +2294,12 @@ func buildHandleColsForSplit(sc *stmtctx.StatementContext, tbInfo *model.TableIn for i, pkCol := range primaryIdx.Columns { tableCols[pkCol.Offset].Index = i } - return plannercore.NewCommonHandleCols(sc, tbInfo, primaryIdx, tableCols) + return handlecol.NewCommonHandleCols(sc, tbInfo, primaryIdx, tableCols) } intCol := &expression.Column{ RetType: types.NewFieldType(mysql.TypeLonglong), } - return plannercore.NewIntHandleCols(intCol) + return handlecol.NewIntHandleCols(intCol) } func (b *executorBuilder) buildSplitRegion(v *plannercore.SplitRegion) exec.Executor { diff --git a/pkg/executor/delete.go b/pkg/executor/delete.go index cbd328e14dafe..7dd1cd286a5cf 100644 --- a/pkg/executor/delete.go +++ b/pkg/executor/delete.go @@ -22,6 +22,7 @@ import ( "github.com/pingcap/tidb/pkg/kv" "github.com/pingcap/tidb/pkg/parser/model" plannercore "github.com/pingcap/tidb/pkg/planner/core" + "github.com/pingcap/tidb/pkg/planner/util/handlecol" "github.com/pingcap/tidb/pkg/sessionctx" "github.com/pingcap/tidb/pkg/sessionctx/variable" "github.com/pingcap/tidb/pkg/sessiontxn" @@ -61,7 +62,7 @@ func (e *DeleteExec) Next(ctx context.Context, req *chunk.Chunk) error { return e.deleteSingleTableByChunk(ctx) } -func (e *DeleteExec) deleteOneRow(tbl table.Table, handleCols plannercore.HandleCols, isExtraHandle bool, row []types.Datum) error { +func (e *DeleteExec) deleteOneRow(tbl table.Table, handleCols handlecol.HandleCols, isExtraHandle bool, row []types.Datum) error { end := len(row) if isExtraHandle { end-- @@ -81,7 +82,7 @@ func (e *DeleteExec) deleteSingleTableByChunk(ctx context.Context) error { var ( tbl table.Table isExtrahandle bool - handleCols plannercore.HandleCols + handleCols handlecol.HandleCols rowCount int ) for _, info := range e.tblColPosInfos { diff --git a/pkg/executor/executor.go b/pkg/executor/executor.go index e30200b1fdc53..55e6c8710077c 100644 --- a/pkg/executor/executor.go +++ b/pkg/executor/executor.go @@ -56,6 +56,7 @@ import ( plannercore "github.com/pingcap/tidb/pkg/planner/core" "github.com/pingcap/tidb/pkg/planner/core/base" "github.com/pingcap/tidb/pkg/planner/util/fixcontrol" + "github.com/pingcap/tidb/pkg/planner/util/handlecol" "github.com/pingcap/tidb/pkg/privilege" "github.com/pingcap/tidb/pkg/resourcemanager/pool/workerpool" poolutil "github.com/pingcap/tidb/pkg/resourcemanager/util" @@ -1108,7 +1109,7 @@ type SelectLockExec struct { keys []kv.Key // The children may be a join of multiple tables, so we need a map. - tblID2Handle map[int64][]plannercore.HandleCols + tblID2Handle map[int64][]handlecol.HandleCols // When SelectLock work on a partition table, we need the partition ID // (Physical Table ID) instead of the 'logical' table ID to calculate diff --git a/pkg/executor/index_merge_reader.go b/pkg/executor/index_merge_reader.go index b5013cf60f25f..09d64f56992d2 100644 --- a/pkg/executor/index_merge_reader.go +++ b/pkg/executor/index_merge_reader.go @@ -41,6 +41,7 @@ import ( plannercore "github.com/pingcap/tidb/pkg/planner/core" "github.com/pingcap/tidb/pkg/planner/core/base" plannerutil "github.com/pingcap/tidb/pkg/planner/util" + "github.com/pingcap/tidb/pkg/planner/util/handlecol" "github.com/pingcap/tidb/pkg/sessionctx" "github.com/pingcap/tidb/pkg/table" "github.com/pingcap/tidb/pkg/types" @@ -135,7 +136,7 @@ type IndexMergeReaderExecutor struct { partialNetDataSizes []float64 dataAvgRowSize float64 - handleCols plannercore.HandleCols + handleCols handlecol.HandleCols stats *IndexMergeRuntimeStat // Indicates whether there is correlated column in filter or table/index range. @@ -633,7 +634,7 @@ func (w *partialTableWorker) needPartitionHandle() (bool, error) { } func (w *partialTableWorker) fetchHandles(ctx context.Context, exitCh <-chan struct{}, fetchCh chan<- *indexMergeTableTask, - finished <-chan struct{}, handleCols plannercore.HandleCols, parTblIdx int, partialPlanIndex int) (count int64, err error) { + finished <-chan struct{}, handleCols handlecol.HandleCols, parTblIdx int, partialPlanIndex int) (count int64, err error) { chk := w.tableReader.NewChunkWithCapacity(w.getRetTpsForTableScan(), w.maxChunkSize, w.maxBatchSize) for { start := time.Now() @@ -666,7 +667,7 @@ func (w *partialTableWorker) getRetTpsForTableScan() []*types.FieldType { return exec.RetTypes(w.tableReader) } -func (w *partialTableWorker) extractTaskHandles(ctx context.Context, chk *chunk.Chunk, handleCols plannercore.HandleCols) ( +func (w *partialTableWorker) extractTaskHandles(ctx context.Context, chk *chunk.Chunk, handleCols handlecol.HandleCols) ( handles []kv.Handle, retChk *chunk.Chunk, err error) { handles = make([]kv.Handle, 0, w.batchSize) if len(w.byItems) != 0 { @@ -1701,7 +1702,7 @@ func (w *partialIndexWorker) fetchHandles( exitCh <-chan struct{}, fetchCh chan<- *indexMergeTableTask, finished <-chan struct{}, - handleCols plannercore.HandleCols, + handleCols handlecol.HandleCols, partialPlanIndex int) (count int64, err error) { tps := w.getRetTpsForIndexScan(handleCols) chk := chunk.NewChunkWithCapacity(tps, w.maxChunkSize) @@ -1734,7 +1735,7 @@ func (w *partialIndexWorker) fetchHandles( return count, nil } -func (w *partialIndexWorker) getRetTpsForIndexScan(handleCols plannercore.HandleCols) []*types.FieldType { +func (w *partialIndexWorker) getRetTpsForIndexScan(handleCols handlecol.HandleCols) []*types.FieldType { var tps []*types.FieldType if len(w.byItems) != 0 { for _, item := range w.byItems { @@ -1748,7 +1749,7 @@ func (w *partialIndexWorker) getRetTpsForIndexScan(handleCols plannercore.Handle return tps } -func (w *partialIndexWorker) extractTaskHandles(ctx context.Context, chk *chunk.Chunk, idxResult distsql.SelectResult, handleCols plannercore.HandleCols) ( +func (w *partialIndexWorker) extractTaskHandles(ctx context.Context, chk *chunk.Chunk, idxResult distsql.SelectResult, handleCols handlecol.HandleCols) ( handles []kv.Handle, retChk *chunk.Chunk, err error) { handles = make([]kv.Handle, 0, w.batchSize) if len(w.byItems) != 0 { diff --git a/pkg/executor/split.go b/pkg/executor/split.go index 9582f0bdaec64..44f07fcb8dd91 100644 --- a/pkg/executor/split.go +++ b/pkg/executor/split.go @@ -27,7 +27,7 @@ import ( "github.com/pingcap/tidb/pkg/kv" "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/parser/mysql" - "github.com/pingcap/tidb/pkg/planner/core" + "github.com/pingcap/tidb/pkg/planner/util/handlecol" "github.com/pingcap/tidb/pkg/sessionctx" "github.com/pingcap/tidb/pkg/store/helper" "github.com/pingcap/tidb/pkg/table/tables" @@ -330,7 +330,7 @@ type SplitTableRegionExec struct { lower []types.Datum upper []types.Datum num int - handleCols core.HandleCols + handleCols handlecol.HandleCols valueLists [][]types.Datum splitKeys [][]byte diff --git a/pkg/executor/split_test.go b/pkg/executor/split_test.go index b65009502ecd5..412bcc1543db7 100644 --- a/pkg/executor/split_test.go +++ b/pkg/executor/split_test.go @@ -28,7 +28,7 @@ import ( "github.com/pingcap/tidb/pkg/kv" "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/parser/mysql" - "github.com/pingcap/tidb/pkg/planner/core" + "github.com/pingcap/tidb/pkg/planner/util/handlecol" "github.com/pingcap/tidb/pkg/sessionctx/stmtctx" "github.com/pingcap/tidb/pkg/table/tables" "github.com/pingcap/tidb/pkg/tablecodec" @@ -320,7 +320,7 @@ func TestSplitTable(t *testing.T) { e := &SplitTableRegionExec{ BaseExecutor: exec.NewBaseExecutor(ctx, nil, 0), tableInfo: tbInfo, - handleCols: core.NewIntHandleCols(&expression.Column{RetType: types.NewFieldType(mysql.TypeLonglong)}), + handleCols: handlecol.NewIntHandleCols(&expression.Column{RetType: types.NewFieldType(mysql.TypeLonglong)}), lower: []types.Datum{types.NewDatum(0)}, upper: []types.Datum{types.NewDatum(100)}, num: 10, @@ -380,7 +380,7 @@ func TestStepShouldLargeThanMinStep(t *testing.T) { e1 := &SplitTableRegionExec{ BaseExecutor: exec.NewBaseExecutor(ctx, nil, 0), tableInfo: tbInfo, - handleCols: core.NewIntHandleCols(&expression.Column{RetType: types.NewFieldType(mysql.TypeLonglong)}), + handleCols: handlecol.NewIntHandleCols(&expression.Column{RetType: types.NewFieldType(mysql.TypeLonglong)}), lower: []types.Datum{types.NewDatum(0)}, upper: []types.Datum{types.NewDatum(1000)}, num: 10, diff --git a/pkg/executor/union_scan.go b/pkg/executor/union_scan.go index a23450764fd4a..c792d9a7df155 100644 --- a/pkg/executor/union_scan.go +++ b/pkg/executor/union_scan.go @@ -24,7 +24,7 @@ import ( "github.com/pingcap/tidb/pkg/kv" "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/parser/mysql" - plannercore "github.com/pingcap/tidb/pkg/planner/core" + "github.com/pingcap/tidb/pkg/planner/util/handlecol" "github.com/pingcap/tidb/pkg/sessionctx/stmtctx" "github.com/pingcap/tidb/pkg/table" "github.com/pingcap/tidb/pkg/tablecodec" @@ -291,7 +291,7 @@ type compareExec struct { usedIndex []int desc bool // handleCols is the handle's position of the below scan plan. - handleCols plannercore.HandleCols + handleCols handlecol.HandleCols } func (ce compareExec) compare(sctx *stmtctx.StatementContext, a, b []types.Datum) (ret int, err error) { diff --git a/pkg/planner/core/BUILD.bazel b/pkg/planner/core/BUILD.bazel index c30e812a32cb6..fc077daff982e 100644 --- a/pkg/planner/core/BUILD.bazel +++ b/pkg/planner/core/BUILD.bazel @@ -6,7 +6,6 @@ go_library( "access_object.go", "collect_column_stats_usage.go", "common_plans.go", - "debugtrace.go", "encode.go", "exhaust_physical_plans.go", "explain.go", @@ -15,7 +14,6 @@ go_library( "flat_plan.go", "foreign_key.go", "fragment.go", - "handle_cols.go", "hashcode.go", "hint_utils.go", "indexmerge_path.go", @@ -121,6 +119,7 @@ go_library( "//pkg/planner/util/coreusage", "//pkg/planner/util/debugtrace", "//pkg/planner/util/fixcontrol", + "//pkg/planner/util/handlecol", "//pkg/privilege", "//pkg/sessionctx", "//pkg/sessionctx/stmtctx", diff --git a/pkg/planner/core/common_plans.go b/pkg/planner/core/common_plans.go index 392755ecfb9f1..552c8a23edbe1 100644 --- a/pkg/planner/core/common_plans.go +++ b/pkg/planner/core/common_plans.go @@ -30,6 +30,7 @@ import ( "github.com/pingcap/tidb/pkg/planner/core/base" "github.com/pingcap/tidb/pkg/planner/property" "github.com/pingcap/tidb/pkg/planner/util/coreusage" + "github.com/pingcap/tidb/pkg/planner/util/handlecol" "github.com/pingcap/tidb/pkg/sessionctx/variable" "github.com/pingcap/tidb/pkg/statistics" "github.com/pingcap/tidb/pkg/table" @@ -519,7 +520,7 @@ type V2AnalyzeOptions struct { // AnalyzeColumnsTask is used for analyze columns. type AnalyzeColumnsTask struct { - HandleCols HandleCols + HandleCols handlecol.HandleCols CommonHandleInfo *model.IndexInfo ColsInfo []*model.ColumnInfo TblInfo *model.TableInfo diff --git a/pkg/planner/core/exhaust_physical_plans.go b/pkg/planner/core/exhaust_physical_plans.go index fa77ed8d8a414..5301f0af46e73 100644 --- a/pkg/planner/core/exhaust_physical_plans.go +++ b/pkg/planner/core/exhaust_physical_plans.go @@ -35,6 +35,7 @@ import ( "github.com/pingcap/tidb/pkg/planner/property" "github.com/pingcap/tidb/pkg/planner/util" "github.com/pingcap/tidb/pkg/planner/util/fixcontrol" + "github.com/pingcap/tidb/pkg/planner/util/handlecol" "github.com/pingcap/tidb/pkg/statistics" "github.com/pingcap/tidb/pkg/types" "github.com/pingcap/tidb/pkg/util/chunk" @@ -1264,8 +1265,9 @@ func (p *LogicalJoin) constructInnerIndexScanTask( }.Init(ds.SCtx(), ds.QueryBlockOffset()) ts.schema = is.dataSourceSchema.Clone() if ds.tableInfo.IsCommonHandle { - commonHandle := ds.handleCols.(*CommonHandleCols) - for _, col := range commonHandle.columns { + commonHandle := ds.handleCols.(*handlecol.CommonHandleCols) + for i := 0; i < commonHandle.NumCols(); i++ { + col := commonHandle.GetCol(i) if ts.schema.ColumnIndex(col) == -1 { ts.Schema().Append(col) ts.Columns = append(ts.Columns, col.ToInfo()) diff --git a/pkg/planner/core/find_best_task.go b/pkg/planner/core/find_best_task.go index 81ce43388c1d5..9c1b2fe7aba05 100644 --- a/pkg/planner/core/find_best_task.go +++ b/pkg/planner/core/find_best_task.go @@ -35,6 +35,7 @@ import ( "github.com/pingcap/tidb/pkg/planner/util" "github.com/pingcap/tidb/pkg/planner/util/coreusage" "github.com/pingcap/tidb/pkg/planner/util/fixcontrol" + "github.com/pingcap/tidb/pkg/planner/util/handlecol" "github.com/pingcap/tidb/pkg/statistics" "github.com/pingcap/tidb/pkg/types" tidbutil "github.com/pingcap/tidb/pkg/util" @@ -1754,7 +1755,7 @@ func (ds *DataSource) convertToPartialTableScan(prop *property.PhysicalProperty, func overwritePartialTableScanSchema(ds *DataSource, ts *PhysicalTableScan) { handleCols := ds.handleCols if handleCols == nil { - handleCols = NewIntHandleCols(ds.newExtraHandleSchemaCol()) + handleCols = handlecol.NewIntHandleCols(ds.newExtraHandleSchemaCol()) } hdColNum := handleCols.NumCols() exprCols := make([]*expression.Column, 0, hdColNum) @@ -1776,7 +1777,7 @@ func overwritePartialTableScanSchema(ds *DataSource, ts *PhysicalTableScan) { func setIndexMergeTableScanHandleCols(ds *DataSource, ts *PhysicalTableScan) (err error) { handleCols := ds.handleCols if handleCols == nil { - handleCols = NewIntHandleCols(ds.newExtraHandleSchemaCol()) + handleCols = handlecol.NewIntHandleCols(ds.newExtraHandleSchemaCol()) } hdColNum := handleCols.NumCols() exprCols := make([]*expression.Column, 0, hdColNum) @@ -2058,8 +2059,9 @@ func (ds *DataSource) convertToIndexScan(prop *property.PhysicalProperty, task = cop if cop.tablePlan != nil && ds.tableInfo.IsCommonHandle { cop.commonHandleCols = ds.commonHandleCols - commonHandle := ds.handleCols.(*CommonHandleCols) - for _, col := range commonHandle.columns { + commonHandle := ds.handleCols.(*handlecol.CommonHandleCols) + for i := 0; i < commonHandle.NumCols(); i++ { + col := commonHandle.GetCol(i) if ds.schema.ColumnIndex(col) == -1 { ts := cop.tablePlan.(*PhysicalTableScan) ts.Schema().Append(col) diff --git a/pkg/planner/core/logical_plan_builder.go b/pkg/planner/core/logical_plan_builder.go index c62139bdb47f6..59cd3a373792a 100644 --- a/pkg/planner/core/logical_plan_builder.go +++ b/pkg/planner/core/logical_plan_builder.go @@ -47,6 +47,7 @@ import ( "github.com/pingcap/tidb/pkg/planner/util" "github.com/pingcap/tidb/pkg/planner/util/debugtrace" "github.com/pingcap/tidb/pkg/planner/util/fixcontrol" + "github.com/pingcap/tidb/pkg/planner/util/handlecol" "github.com/pingcap/tidb/pkg/privilege" "github.com/pingcap/tidb/pkg/sessionctx" "github.com/pingcap/tidb/pkg/sessionctx/variable" @@ -4507,6 +4508,37 @@ func getStatsTable(ctx base.PlanContext, tblInfo *model.TableInfo, pid int64) *s return statsTbl } +func debugTraceGetStatsTbl( + s base.PlanContext, + tblInfo *model.TableInfo, + pid int64, + handleIsNil, + usePartitionStats, + countIsZero, + uninitialized, + outdated bool, + statsTbl *statistics.Table, +) { + root := debugtrace.GetOrInitDebugTraceRoot(s) + traceInfo := &statistics.GetStatsTblInfo{ + TableName: tblInfo.Name.O, + TblInfoID: tblInfo.ID, + InputPhysicalID: pid, + HandleIsNil: handleIsNil, + UsePartitionStats: usePartitionStats, + CountIsZero: countIsZero, + Uninitialized: uninitialized, + Outdated: outdated, + StatsTblInfo: statistics.TraceStatsTbl(statsTbl), + } + failpoint.Inject("DebugTraceStableStatsTbl", func(val failpoint.Value) { + if val.(bool) { + statistics.StabilizeGetStatsTblInfo(traceInfo) + } + }) + root.AppendStepToCurrentContext(traceInfo) +} + // getLatestVersionFromStatsTable gets statistics information for a table specified by "tableID", and get the max // LastUpdateVersion among all Columns and Indices in it. // Its overall logic is quite similar to getStatsTable(). During plan cache matching, only the latest version is needed. @@ -4963,7 +4995,7 @@ func (b *PlanBuilder) buildDataSource(ctx context.Context, tn *ast.TableName, as is: b.is, isForUpdateRead: b.isForUpdateRead, }.Init(b.ctx, b.getSelectOffset()) - var handleCols HandleCols + var handleCols handlecol.HandleCols schema := expression.NewSchema(make([]*expression.Column, 0, len(columns))...) names := make([]*types.FieldName, 0, len(columns)) for i, col := range columns { @@ -4985,7 +5017,7 @@ func (b *PlanBuilder) buildDataSource(ctx context.Context, tn *ast.TableName, as IsHidden: col.Hidden, } if col.IsPKHandleColumn(tableInfo) { - handleCols = &IntHandleCols{col: newCol} + handleCols = handlecol.NewIntHandleCols(newCol) } schema.Append(newCol) ds.TblCols = append(ds.TblCols, newCol) @@ -4995,10 +5027,10 @@ func (b *PlanBuilder) buildDataSource(ctx context.Context, tn *ast.TableName, as if handleCols == nil { if tableInfo.IsCommonHandle { primaryIdx := tables.FindPrimaryIndex(tableInfo) - handleCols = NewCommonHandleCols(b.ctx.GetSessionVars().StmtCtx, tableInfo, primaryIdx, ds.TblCols) + handleCols = handlecol.NewCommonHandleCols(b.ctx.GetSessionVars().StmtCtx, tableInfo, primaryIdx, ds.TblCols) } else { extraCol := ds.newExtraHandleSchemaCol() - handleCols = &IntHandleCols{col: extraCol} + handleCols = handlecol.NewIntHandleCols(extraCol) ds.Columns = append(ds.Columns, model.NewExtraHandleColInfo()) schema.Append(extraCol) names = append(names, &types.FieldName{ @@ -5012,8 +5044,8 @@ func (b *PlanBuilder) buildDataSource(ctx context.Context, tn *ast.TableName, as } ds.handleCols = handleCols ds.unMutableHandleCols = handleCols - handleMap := make(map[int64][]HandleCols) - handleMap[tableInfo.ID] = []HandleCols{handleCols} + handleMap := make(map[int64][]handlecol.HandleCols) + handleMap[tableInfo.ID] = []handlecol.HandleCols{handleCols} b.handleHelper.pushMap(handleMap) ds.SetSchema(schema) ds.names = names @@ -5248,7 +5280,7 @@ func (b *PlanBuilder) buildMemTable(_ context.Context, dbName model.CIStr, table // a stable schema and there is no online DDL on the memory table. schema := expression.NewSchema(make([]*expression.Column, 0, len(tableInfo.Columns))...) names := make([]*types.FieldName, 0, len(tableInfo.Columns)) - var handleCols HandleCols + var handleCols handlecol.HandleCols for _, col := range tableInfo.Columns { names = append(names, &types.FieldName{ DBName: dbName, @@ -5264,14 +5296,14 @@ func (b *PlanBuilder) buildMemTable(_ context.Context, dbName model.CIStr, table RetType: &col.FieldType, } if tableInfo.PKIsHandle && mysql.HasPriKeyFlag(col.GetFlag()) { - handleCols = &IntHandleCols{col: newCol} + handleCols = handlecol.NewIntHandleCols(newCol) } schema.Append(newCol) } if handleCols != nil { - handleMap := make(map[int64][]HandleCols) - handleMap[tableInfo.ID] = []HandleCols{handleCols} + handleMap := make(map[int64][]handlecol.HandleCols) + handleMap[tableInfo.ID] = []handlecol.HandleCols{handleCols} b.handleHelper.pushMap(handleMap) } else { b.handleHelper.pushMap(nil) @@ -5649,7 +5681,7 @@ type TblColPosInfo struct { // Start and End represent the ordinal range [Start, End) of the consecutive columns. Start, End int // HandleOrdinal represents the ordinal of the handle column. - HandleCols HandleCols + HandleCols handlecol.HandleCols } // MemoryUsage return the memory usage of TblColPosInfo @@ -5700,7 +5732,7 @@ func (c TblColPosInfoSlice) FindTblIdx(colOrdinal int) (int, bool) { // buildColumns2Handle builds columns to handle mapping. func buildColumns2Handle( names []*types.FieldName, - tblID2Handle map[int64][]HandleCols, + tblID2Handle map[int64][]handlecol.HandleCols, tblID2Table map[int64]table.Table, onlyWritableCol bool, ) (TblColPosInfoSlice, error) { @@ -6328,8 +6360,8 @@ func (b *PlanBuilder) buildDelete(ctx context.Context, ds *ast.DeleteStmt) (base return del, err } -func resolveIndicesForTblID2Handle(tblID2Handle map[int64][]HandleCols, schema *expression.Schema) (map[int64][]HandleCols, error) { - newMap := make(map[int64][]HandleCols, len(tblID2Handle)) +func resolveIndicesForTblID2Handle(tblID2Handle map[int64][]handlecol.HandleCols, schema *expression.Schema) (map[int64][]handlecol.HandleCols, error) { + newMap := make(map[int64][]handlecol.HandleCols, len(tblID2Handle)) for i, cols := range tblID2Handle { for _, col := range cols { resolvedCol, err := col.ResolveIndices(schema) @@ -6344,9 +6376,9 @@ func resolveIndicesForTblID2Handle(tblID2Handle map[int64][]HandleCols, schema * func (p *Delete) cleanTblID2HandleMap( tablesToDelete map[int64][]*ast.TableName, - tblID2Handle map[int64][]HandleCols, + tblID2Handle map[int64][]handlecol.HandleCols, outputNames []*types.FieldName, -) map[int64][]HandleCols { +) map[int64][]handlecol.HandleCols { for id, cols := range tblID2Handle { names, ok := tablesToDelete[id] if !ok { diff --git a/pkg/planner/core/logical_plans.go b/pkg/planner/core/logical_plans.go index 3def077d74f0d..9fd272cd441d0 100644 --- a/pkg/planner/core/logical_plans.go +++ b/pkg/planner/core/logical_plans.go @@ -33,6 +33,7 @@ import ( "github.com/pingcap/tidb/pkg/planner/property" "github.com/pingcap/tidb/pkg/planner/util" "github.com/pingcap/tidb/pkg/planner/util/debugtrace" + "github.com/pingcap/tidb/pkg/planner/util/handlecol" "github.com/pingcap/tidb/pkg/sessionctx" "github.com/pingcap/tidb/pkg/statistics" "github.com/pingcap/tidb/pkg/table" @@ -1412,7 +1413,7 @@ type LogicalUnionScan struct { conditions []expression.Expression - handleCols HandleCols + handleCols handlecol.HandleCols } // GetAllConds Exported for unit test. @@ -1454,8 +1455,8 @@ type DataSource struct { // handleCol represents the handle column for the datasource, either the // int primary key column or extra handle column. // handleCol *expression.Column - handleCols HandleCols - unMutableHandleCols HandleCols + handleCols handlecol.HandleCols + unMutableHandleCols handlecol.HandleCols // TblCols contains the original columns of table before being pruned, and it // is used for estimating table scan cost. TblCols []*expression.Column @@ -1515,7 +1516,7 @@ type TiKVSingleGather struct { type LogicalTableScan struct { logicalSchemaProducer Source *DataSource - HandleCols HandleCols + HandleCols handlecol.HandleCols AccessConds expression.CNFExprs Ranges []*ranger.Range } @@ -1972,7 +1973,7 @@ type LogicalLock struct { baseLogicalPlan Lock *ast.SelectLockInfo - tblID2Handle map[int64][]HandleCols + tblID2Handle map[int64][]handlecol.HandleCols // tblID2phyTblIDCol is used for partitioned tables, // the child executor need to return an extra column containing diff --git a/pkg/planner/core/physical_plans.go b/pkg/planner/core/physical_plans.go index 3f74d7b54b5ac..37bed6c52451f 100644 --- a/pkg/planner/core/physical_plans.go +++ b/pkg/planner/core/physical_plans.go @@ -32,6 +32,7 @@ import ( "github.com/pingcap/tidb/pkg/planner/property" "github.com/pingcap/tidb/pkg/planner/util" "github.com/pingcap/tidb/pkg/planner/util/coreusage" + "github.com/pingcap/tidb/pkg/planner/util/handlecol" "github.com/pingcap/tidb/pkg/sessionctx" "github.com/pingcap/tidb/pkg/sessionctx/stmtctx" "github.com/pingcap/tidb/pkg/statistics" @@ -603,7 +604,7 @@ type PhysicalIndexMergeReader struct { KeepOrder bool - HandleCols HandleCols + HandleCols handlecol.HandleCols } // GetAvgTableRowSize return the average row size of table plan. @@ -877,7 +878,7 @@ type PhysicalTableScan struct { // HandleIdx is the index of handle, which is only used for admin check table. HandleIdx []int - HandleCols HandleCols + HandleCols handlecol.HandleCols StoreType kv.StoreType @@ -1759,7 +1760,7 @@ type PhysicalLock struct { Lock *ast.SelectLockInfo - TblID2Handle map[int64][]HandleCols + TblID2Handle map[int64][]handlecol.HandleCols TblID2PhysTblIDCol map[int64]*expression.Column } @@ -2124,7 +2125,7 @@ type PhysicalUnionScan struct { Conditions []expression.Expression - HandleCols HandleCols + HandleCols handlecol.HandleCols } // ExtractCorrelatedCols implements op.PhysicalPlan interface. diff --git a/pkg/planner/core/plan_cache_utils.go b/pkg/planner/core/plan_cache_utils.go index 04ae7a2b8dcbb..850e32f9991da 100644 --- a/pkg/planner/core/plan_cache_utils.go +++ b/pkg/planner/core/plan_cache_utils.go @@ -34,6 +34,7 @@ import ( "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/planner/core/base" "github.com/pingcap/tidb/pkg/planner/util" + "github.com/pingcap/tidb/pkg/planner/util/debugtrace" "github.com/pingcap/tidb/pkg/planner/util/fixcontrol" "github.com/pingcap/tidb/pkg/sessionctx" "github.com/pingcap/tidb/pkg/sessionctx/variable" @@ -502,6 +503,29 @@ func GetPreparedStmt(stmt *ast.ExecuteStmt, vars *variable.SessionVars) (*PlanCa return nil, plannererrors.ErrStmtNotFound } +func init() { + debugtrace.GetPreparedStmt = GetPreparedStmt4DebugTrace +} + +// GetPreparedStmt4DebugTrace is util function to avoid util/pkg depend back on core/pkg +func GetPreparedStmt4DebugTrace(stmt ast.StmtNode) (planCacheStmtIsNil bool, notNilText string, binaryParams []expression.Expression) { + var planCacheStmt *PlanCacheStmt + if execStmt, ok := stmt.(*ast.ExecuteStmt); ok { + if execStmt.PrepStmt != nil { + planCacheStmt, _ = execStmt.PrepStmt.(*PlanCacheStmt) + } + if execStmt.BinaryArgs != nil { + binaryParams, _ = execStmt.BinaryArgs.([]expression.Expression) + } + } + if planCacheStmt == nil { + planCacheStmtIsNil = true + } else { + notNilText = planCacheStmt.StmtText + } + return planCacheStmtIsNil, notNilText, binaryParams +} + // GetMatchOpts get options to fetch plan or generate new plan // we can add more options here func GetMatchOpts(sctx sessionctx.Context, is infoschema.InfoSchema, stmt *PlanCacheStmt, params []expression.Expression) (*utilpc.PlanCacheMatchOpts, error) { diff --git a/pkg/planner/core/planbuilder.go b/pkg/planner/core/planbuilder.go index 527ed8f6977bc..f8aceae2495ed 100644 --- a/pkg/planner/core/planbuilder.go +++ b/pkg/planner/core/planbuilder.go @@ -44,6 +44,7 @@ import ( "github.com/pingcap/tidb/pkg/planner/core/base" "github.com/pingcap/tidb/pkg/planner/property" "github.com/pingcap/tidb/pkg/planner/util" + "github.com/pingcap/tidb/pkg/planner/util/handlecol" "github.com/pingcap/tidb/pkg/privilege" "github.com/pingcap/tidb/pkg/sessionctx" "github.com/pingcap/tidb/pkg/sessionctx/stmtctx" @@ -301,7 +302,7 @@ type PlanBuilder struct { } type handleColHelper struct { - id2HandleMapStack []map[int64][]HandleCols + id2HandleMapStack []map[int64][]handlecol.HandleCols stackTail int } @@ -311,36 +312,36 @@ func (hch *handleColHelper) resetForReuse() { } } -func (hch *handleColHelper) popMap() map[int64][]HandleCols { +func (hch *handleColHelper) popMap() map[int64][]handlecol.HandleCols { ret := hch.id2HandleMapStack[hch.stackTail-1] hch.stackTail-- hch.id2HandleMapStack = hch.id2HandleMapStack[:hch.stackTail] return ret } -func (hch *handleColHelper) pushMap(m map[int64][]HandleCols) { +func (hch *handleColHelper) pushMap(m map[int64][]handlecol.HandleCols) { hch.id2HandleMapStack = append(hch.id2HandleMapStack, m) hch.stackTail++ } -func (hch *handleColHelper) mergeAndPush(m1, m2 map[int64][]HandleCols) { - newMap := make(map[int64][]HandleCols, max(len(m1), len(m2))) +func (hch *handleColHelper) mergeAndPush(m1, m2 map[int64][]handlecol.HandleCols) { + newMap := make(map[int64][]handlecol.HandleCols, max(len(m1), len(m2))) for k, v := range m1 { - newMap[k] = make([]HandleCols, len(v)) + newMap[k] = make([]handlecol.HandleCols, len(v)) copy(newMap[k], v) } for k, v := range m2 { if _, ok := newMap[k]; ok { newMap[k] = append(newMap[k], v...) } else { - newMap[k] = make([]HandleCols, len(v)) + newMap[k] = make([]handlecol.HandleCols, len(v)) copy(newMap[k], v) } } hch.pushMap(newMap) } -func (hch *handleColHelper) tailMap() map[int64][]HandleCols { +func (hch *handleColHelper) tailMap() map[int64][]handlecol.HandleCols { return hch.id2HandleMapStack[hch.stackTail-1] } @@ -431,7 +432,7 @@ func NewPlanBuilder(opts ...PlanBuilderOpt) *PlanBuilder { builder := &PlanBuilder{ outerCTEs: make([]*cteInfo, 0), colMapper: make(map[*ast.ColumnNameExpr]int), - handleHelper: &handleColHelper{id2HandleMapStack: make([]map[int64][]HandleCols, 0)}, + handleHelper: &handleColHelper{id2HandleMapStack: make([]map[int64][]handlecol.HandleCols, 0)}, correlatedAggMapper: make(map[*ast.AggregateFuncExpr]*expression.CorrelatedColumn), } for _, opt := range opts { @@ -1814,8 +1815,8 @@ func getColsInfo(tn *ast.TableName) (indicesInfo []*model.IndexInfo, colsInfo [] } // BuildHandleColsForAnalyze returns HandleCols for ANALYZE. -func BuildHandleColsForAnalyze(ctx base.PlanContext, tblInfo *model.TableInfo, allColumns bool, colsInfo []*model.ColumnInfo) HandleCols { - var handleCols HandleCols +func BuildHandleColsForAnalyze(ctx base.PlanContext, tblInfo *model.TableInfo, allColumns bool, colsInfo []*model.ColumnInfo) handlecol.HandleCols { + var handleCols handlecol.HandleCols switch { case tblInfo.PKIsHandle: pkCol := tblInfo.GetPkColInfo() @@ -1827,11 +1828,11 @@ func BuildHandleColsForAnalyze(ctx base.PlanContext, tblInfo *model.TableInfo, a // If only a part of the columns need to be analyzed, we need to set index according to colsInfo. index = getColOffsetForAnalyze(colsInfo, pkCol.ID) } - handleCols = &IntHandleCols{col: &expression.Column{ + handleCols = handlecol.NewIntHandleCols(&expression.Column{ ID: pkCol.ID, RetType: &pkCol.FieldType, Index: index, - }} + }) case tblInfo.IsCommonHandle: pkIdx := tables.FindPrimaryIndex(tblInfo) pkColLen := len(pkIdx.Columns) @@ -1858,12 +1859,7 @@ func BuildHandleColsForAnalyze(ctx base.PlanContext, tblInfo *model.TableInfo, a // The second reason is that in (cb *CommonHandleCols).BuildHandleByDatums, tablecodec.TruncateIndexValues(cb.tblInfo, cb.idxInfo, datumBuf) // is called, which asks that IndexColumn.Offset of cb.idxInfo must be according to cb,tblInfo. // TODO: find a better way to find handle columns in ANALYZE rather than use Column.Index - handleCols = &CommonHandleCols{ - tblInfo: tblInfo, - idxInfo: pkIdx, - columns: columns, - sc: ctx.GetSessionVars().StmtCtx, - } + handleCols = handlecol.NewCommonHandleColsWithoutOffsetAlign(ctx.GetSessionVars().StmtCtx, tblInfo, pkIdx, columns) } return handleCols } @@ -2283,7 +2279,7 @@ func (b *PlanBuilder) buildAnalyzeFullSamplingTask( extraCol := model.NewExtraHandleColInfo() // Always place _tidb_rowid at the end of colsInfo, this is corresponding to logics in `analyzeColumnsPushdown`. newTask.ColsInfo = append(newTask.ColsInfo, extraCol) - newTask.HandleCols = &IntHandleCols{col: colInfoToColumn(extraCol, len(newTask.ColsInfo)-1)} + newTask.HandleCols = handlecol.NewIntHandleCols(colInfoToColumn(extraCol, len(newTask.ColsInfo)-1)) } analyzePlan.ColTasks = append(analyzePlan.ColTasks, newTask) for _, indexInfo := range independentIndexes { diff --git a/pkg/planner/core/point_get_plan.go b/pkg/planner/core/point_get_plan.go index 5abcf89e8d389..0e9f5c6b337bb 100644 --- a/pkg/planner/core/point_get_plan.go +++ b/pkg/planner/core/point_get_plan.go @@ -37,6 +37,7 @@ import ( "github.com/pingcap/tidb/pkg/planner/core/operator/baseimpl" "github.com/pingcap/tidb/pkg/planner/property" "github.com/pingcap/tidb/pkg/planner/util/coreusage" + "github.com/pingcap/tidb/pkg/planner/util/handlecol" "github.com/pingcap/tidb/pkg/privilege" "github.com/pingcap/tidb/pkg/sessionctx" "github.com/pingcap/tidb/pkg/sessionctx/stmtctx" @@ -2094,24 +2095,24 @@ func colInfoToColumn(col *model.ColumnInfo, idx int) *expression.Column { } } -func buildHandleCols(ctx base.PlanContext, tbl *model.TableInfo, schema *expression.Schema) HandleCols { +func buildHandleCols(ctx base.PlanContext, tbl *model.TableInfo, schema *expression.Schema) handlecol.HandleCols { // fields len is 0 for update and delete. if tbl.PKIsHandle { for i, col := range tbl.Columns { if mysql.HasPriKeyFlag(col.GetFlag()) { - return &IntHandleCols{col: schema.Columns[i]} + return handlecol.NewIntHandleCols(schema.Columns[i]) } } } if tbl.IsCommonHandle { pkIdx := tables.FindPrimaryIndex(tbl) - return NewCommonHandleCols(ctx.GetSessionVars().StmtCtx, tbl, pkIdx, schema.Columns) + return handlecol.NewCommonHandleCols(ctx.GetSessionVars().StmtCtx, tbl, pkIdx, schema.Columns) } handleCol := colInfoToColumn(model.NewExtraHandleColInfo(), schema.Len()) schema.Append(handleCol) - return &IntHandleCols{col: handleCol} + return handlecol.NewIntHandleCols(handleCol) } // TODO: Remove this, by enabling all types of partitioning diff --git a/pkg/planner/core/stats.go b/pkg/planner/core/stats.go index 5c49479c2821f..ea3faf8576669 100644 --- a/pkg/planner/core/stats.go +++ b/pkg/planner/core/stats.go @@ -499,7 +499,7 @@ func (ds *DataSource) DeriveStats(_ []*property.StatsInfo, _ *expression.Schema, } if ds.SCtx().GetSessionVars().StmtCtx.EnableOptimizerDebugTrace { - debugTraceAccessPaths(ds.SCtx(), ds.possibleAccessPaths) + debugtrace.DebugTraceAccessPaths(ds.SCtx(), ds.possibleAccessPaths) } ds.accessPathMinSelectivity = getMinSelectivityFromPaths(ds.possibleAccessPaths, float64(ds.TblColHists.RealtimeCount)) diff --git a/pkg/planner/optimize.go b/pkg/planner/optimize.go index 23946a250e400..033d8a0a2c5d5 100644 --- a/pkg/planner/optimize.go +++ b/pkg/planner/optimize.go @@ -279,7 +279,7 @@ func Optimize(ctx context.Context, sctx sessionctx.Context, node ast.Node, is in continue } if sessVars.StmtCtx.EnableOptimizerDebugTrace { - core.DebugTraceTryBinding(pctx, binding.Hint) + debugtrace.DebugTraceTryBinding(pctx, binding.Hint) } hint.BindHint(stmtNode, binding.Hint) curStmtHints, _, curWarns := hint.ParseStmtHints(binding.Hint.GetFirstTableHints(), setVarHintChecker, byte(kv.ReplicaReadFollower)) @@ -326,7 +326,7 @@ func Optimize(ctx context.Context, sctx sessionctx.Context, node ast.Node, is in } if sessVars.StmtCtx.EnableOptimizerDebugTrace && bestPlanFromBind != nil { - core.DebugTraceBestBinding(pctx, chosenBinding.Hint) + debugtrace.DebugTraceBestBinding(pctx, chosenBinding.Hint) } // No plan found from the bindings, or the bindings are ignored. if bestPlan == nil { diff --git a/pkg/planner/util/debugtrace/BUILD.bazel b/pkg/planner/util/debugtrace/BUILD.bazel index 13deb51fd2a5d..f4f4af46e1b53 100644 --- a/pkg/planner/util/debugtrace/BUILD.bazel +++ b/pkg/planner/util/debugtrace/BUILD.bazel @@ -2,8 +2,18 @@ load("@io_bazel_rules_go//go:def.bzl", "go_library") go_library( name = "debugtrace", - srcs = ["base.go"], + srcs = [ + "base.go", + "debugtrace.go", + ], importpath = "github.com/pingcap/tidb/pkg/planner/util/debugtrace", visibility = ["//visibility:public"], - deps = ["//pkg/planner/context"], + deps = [ + "//pkg/expression", + "//pkg/parser/ast", + "//pkg/parser/mysql", + "//pkg/planner/context", + "//pkg/planner/util", + "//pkg/util/hint", + ], ) diff --git a/pkg/planner/core/debugtrace.go b/pkg/planner/util/debugtrace/debugtrace.go similarity index 63% rename from pkg/planner/core/debugtrace.go rename to pkg/planner/util/debugtrace/debugtrace.go index 19843e1824f3f..5079104e1d5a2 100644 --- a/pkg/planner/core/debugtrace.go +++ b/pkg/planner/util/debugtrace/debugtrace.go @@ -12,22 +12,17 @@ // See the License for the specific language governing permissions and // limitations under the License. -package core +package debugtrace import ( "strconv" "strings" - "github.com/pingcap/failpoint" "github.com/pingcap/tidb/pkg/expression" "github.com/pingcap/tidb/pkg/parser/ast" - "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/planner/context" - "github.com/pingcap/tidb/pkg/planner/core/base" "github.com/pingcap/tidb/pkg/planner/util" - "github.com/pingcap/tidb/pkg/planner/util/debugtrace" - "github.com/pingcap/tidb/pkg/statistics" "github.com/pingcap/tidb/pkg/util/hint" ) @@ -69,39 +64,41 @@ func (info *binaryParamInfo) MarshalJSON() ([]byte, error) { strconv.Quote(info.Value), quote), quote) - return debugtrace.EncodeJSONCommon(infoForMarshal) + return EncodeJSONCommon(infoForMarshal) } +// GetPreparedStmt will be pre-init by core/pkg to avoid import cycle. +var GetPreparedStmt func(stmt ast.StmtNode) (planCacheStmtIsNil bool, + NotNilText string, binaryParams []expression.Expression) + // DebugTraceReceivedCommand records the received command from the client to the debug trace. -func DebugTraceReceivedCommand(s base.PlanContext, cmd byte, stmtNode ast.StmtNode) { +// +//revive:disable:exported +func DebugTraceReceivedCommand(s context.PlanContext, cmd byte, stmtNode ast.StmtNode) { sessionVars := s.GetSessionVars() - trace := debugtrace.GetOrInitDebugTraceRoot(s) + trace := GetOrInitDebugTraceRoot(s) traceInfo := new(receivedCmdInfo) trace.AppendStepWithNameToCurrentContext(traceInfo, "Received Command") traceInfo.Command = mysql.Command2Str[cmd] traceInfo.ExecutedASTText = stmtNode.Text() // Collect information for execute stmt, and record it in executeInfo. - var binaryParams []expression.Expression - var planCacheStmt *PlanCacheStmt - if execStmt, ok := stmtNode.(*ast.ExecuteStmt); ok { - if execStmt.PrepStmt != nil { - planCacheStmt, _ = execStmt.PrepStmt.(*PlanCacheStmt) - } - if execStmt.BinaryArgs != nil { - binaryParams, _ = execStmt.BinaryArgs.([]expression.Expression) - } - } + var ( + planCacheStmtIsNil bool + planCacheStmtText string + binaryParams []expression.Expression + ) + planCacheStmtIsNil, planCacheStmtText, binaryParams = GetPreparedStmt(stmtNode) useCursor := sessionVars.HasStatusFlag(mysql.ServerStatusCursorExists) // If none of them needs record, we don't need a executeInfo. - if binaryParams == nil && planCacheStmt == nil && !useCursor { + if binaryParams == nil && planCacheStmtIsNil && !useCursor { return } execInfo := &executeInfo{} traceInfo.ExecuteStmtInfo = execInfo execInfo.UseCursor = useCursor - if planCacheStmt != nil { - execInfo.PreparedSQL = planCacheStmt.StmtText + if !planCacheStmtIsNil { + execInfo.PreparedSQL = planCacheStmtText } if len(binaryParams) > 0 { execInfo.BinaryParamsInfo = make([]binaryParamInfo, len(binaryParams)) @@ -125,19 +122,21 @@ func (b *bindingHint) MarshalJSON() ([]byte, error) { tmp := make(map[string]string, 1) hintStr, err := b.Hint.Restore() if err != nil { - return debugtrace.EncodeJSONCommon(err) + return EncodeJSONCommon(err) } if b.trying { tmp["Trying Hint"] = hintStr } else { tmp["Best Hint"] = hintStr } - return debugtrace.EncodeJSONCommon(tmp) + return EncodeJSONCommon(tmp) } // DebugTraceTryBinding records the hint that might be chosen to the debug trace. +// +//revive:disable-line:exported func DebugTraceTryBinding(s context.PlanContext, binding *hint.HintsSet) { - root := debugtrace.GetOrInitDebugTraceRoot(s) + root := GetOrInitDebugTraceRoot(s) traceInfo := &bindingHint{ Hint: binding, trying: true, @@ -146,8 +145,10 @@ func DebugTraceTryBinding(s context.PlanContext, binding *hint.HintsSet) { } // DebugTraceBestBinding records the chosen hint to the debug trace. +// +//revive:disable-line:exported func DebugTraceBestBinding(s context.PlanContext, binding *hint.HintsSet) { - root := debugtrace.GetOrInitDebugTraceRoot(s) + root := GetOrInitDebugTraceRoot(s) traceInfo := &bindingHint{ Hint: binding, trying: false, @@ -155,72 +156,6 @@ func DebugTraceBestBinding(s context.PlanContext, binding *hint.HintsSet) { root.AppendStepToCurrentContext(traceInfo) } -/* - Below is debug trace for getStatsTable(). - Part of the logic for collecting information is in statistics/debug_trace.go. -*/ - -type getStatsTblInfo struct { - TableName string - TblInfoID int64 - InputPhysicalID int64 - HandleIsNil bool - UsePartitionStats bool - CountIsZero bool - Uninitialized bool - Outdated bool - StatsTblInfo *statistics.StatsTblTraceInfo -} - -func debugTraceGetStatsTbl( - s base.PlanContext, - tblInfo *model.TableInfo, - pid int64, - handleIsNil, - usePartitionStats, - countIsZero, - uninitialized, - outdated bool, - statsTbl *statistics.Table, -) { - root := debugtrace.GetOrInitDebugTraceRoot(s) - traceInfo := &getStatsTblInfo{ - TableName: tblInfo.Name.O, - TblInfoID: tblInfo.ID, - InputPhysicalID: pid, - HandleIsNil: handleIsNil, - UsePartitionStats: usePartitionStats, - CountIsZero: countIsZero, - Uninitialized: uninitialized, - Outdated: outdated, - StatsTblInfo: statistics.TraceStatsTbl(statsTbl), - } - failpoint.Inject("DebugTraceStableStatsTbl", func(val failpoint.Value) { - if val.(bool) { - stabilizeGetStatsTblInfo(traceInfo) - } - }) - root.AppendStepToCurrentContext(traceInfo) -} - -// Only for test. -func stabilizeGetStatsTblInfo(info *getStatsTblInfo) { - info.TblInfoID = 100 - info.InputPhysicalID = 100 - tbl := info.StatsTblInfo - if tbl == nil { - return - } - tbl.PhysicalID = 100 - tbl.Version = 440930000000000000 - for _, col := range tbl.Columns { - col.LastUpdateVersion = 440930000000000000 - } - for _, idx := range tbl.Indexes { - idx.LastUpdateVersion = 440930000000000000 - } -} - /* Below is debug trace for AccessPath. */ @@ -250,8 +185,11 @@ func convertAccessPathForDebugTrace(path *util.AccessPath, out *accessPathForDeb } } -func debugTraceAccessPaths(s base.PlanContext, paths []*util.AccessPath) { - root := debugtrace.GetOrInitDebugTraceRoot(s) +// DebugTraceAccessPaths trace the access paths in debug. +// +//revive:disable-line:exported +func DebugTraceAccessPaths(s context.PlanContext, paths []*util.AccessPath) { + root := GetOrInitDebugTraceRoot(s) traceInfo := make([]accessPathForDebugTrace, len(paths)) for i, partialPath := range paths { convertAccessPathForDebugTrace(partialPath, &traceInfo[i]) diff --git a/pkg/planner/util/handlecol/BUILD.bazel b/pkg/planner/util/handlecol/BUILD.bazel new file mode 100644 index 0000000000000..c09731feb9b14 --- /dev/null +++ b/pkg/planner/util/handlecol/BUILD.bazel @@ -0,0 +1,21 @@ +load("@io_bazel_rules_go//go:def.bzl", "go_library") + +go_library( + name = "handlecol", + srcs = ["handle_cols.go"], + importpath = "github.com/pingcap/tidb/pkg/planner/util/handlecol", + visibility = ["//visibility:public"], + deps = [ + "//pkg/expression", + "//pkg/kv", + "//pkg/parser/model", + "//pkg/parser/mysql", + "//pkg/sessionctx/stmtctx", + "//pkg/tablecodec", + "//pkg/types", + "//pkg/util/chunk", + "//pkg/util/codec", + "//pkg/util/collate", + "//pkg/util/size", + ], +) diff --git a/pkg/planner/core/handle_cols.go b/pkg/planner/util/handlecol/handle_cols.go similarity index 95% rename from pkg/planner/core/handle_cols.go rename to pkg/planner/util/handlecol/handle_cols.go index 9c36ab5843da8..9b841c37925f9 100644 --- a/pkg/planner/core/handle_cols.go +++ b/pkg/planner/util/handlecol/handle_cols.go @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -package core +package handlecol import ( "strings" @@ -225,6 +225,18 @@ func NewCommonHandleCols(sc *stmtctx.StatementContext, tblInfo *model.TableInfo, return cols } +// NewCommonHandleColsWithoutOffsetAlign NewCommonHandleCols creates a new CommonHandleCols without offset align. +// It's a substitution of &CommonHandleCols{...} before. +func NewCommonHandleColsWithoutOffsetAlign(sc *stmtctx.StatementContext, tblInfo *model.TableInfo, + idxInfo *model.IndexInfo, alignedTableCols []*expression.Column) *CommonHandleCols { + return &CommonHandleCols{ + tblInfo: tblInfo, + idxInfo: idxInfo, + sc: sc, + columns: alignedTableCols, + } +} + // IntHandleCols implements the kv.HandleCols interface. type IntHandleCols struct { col *expression.Column diff --git a/pkg/session/BUILD.bazel b/pkg/session/BUILD.bazel index 142dc508662a8..d88fe2fd6b168 100644 --- a/pkg/session/BUILD.bazel +++ b/pkg/session/BUILD.bazel @@ -60,6 +60,7 @@ go_library( "//pkg/planner/contextimpl", "//pkg/planner/core", "//pkg/planner/core/base", + "//pkg/planner/util/debugtrace", "//pkg/plugin", "//pkg/privilege", "//pkg/privilege/conn", diff --git a/pkg/session/session.go b/pkg/session/session.go index c761b0399f9bf..df99095c70c0c 100644 --- a/pkg/session/session.go +++ b/pkg/session/session.go @@ -74,6 +74,7 @@ import ( planctx "github.com/pingcap/tidb/pkg/planner/context" plannercore "github.com/pingcap/tidb/pkg/planner/core" "github.com/pingcap/tidb/pkg/planner/core/base" + "github.com/pingcap/tidb/pkg/planner/util/debugtrace" "github.com/pingcap/tidb/pkg/plugin" "github.com/pingcap/tidb/pkg/privilege" "github.com/pingcap/tidb/pkg/privilege/conn" @@ -2039,7 +2040,7 @@ func (s *session) ExecuteStmt(ctx context.Context, stmtNode ast.StmtNode) (sqlex } } if sessVars.StmtCtx.EnableOptimizerDebugTrace { - plannercore.DebugTraceReceivedCommand(s.pctx, cmdByte, stmtNode) + debugtrace.DebugTraceReceivedCommand(s.pctx, cmdByte, stmtNode) } if err := s.validateStatementInTxn(stmtNode); err != nil { diff --git a/pkg/statistics/debugtrace.go b/pkg/statistics/debugtrace.go index 3936feafe2fd2..df094a4966d7c 100644 --- a/pkg/statistics/debugtrace.go +++ b/pkg/statistics/debugtrace.go @@ -265,3 +265,39 @@ func debugTraceTopNRange(s context.PlanContext, t *TopN, startIdx, endIdx int) { traceInfo.Count = cnts root.AppendStepWithNameToCurrentContext(traceInfo, "Related TopN Range") } + +/* + Below is debug trace for getStatsTable(). + Part of the logic for collecting information is in statistics/debug_trace.go. +*/ + +// GetStatsTblInfo is used for debug trace of getStatsTable +type GetStatsTblInfo struct { + StatsTblInfo *StatsTblTraceInfo + TableName string + TblInfoID int64 + InputPhysicalID int64 + HandleIsNil bool + UsePartitionStats bool + CountIsZero bool + Uninitialized bool + Outdated bool +} + +// StabilizeGetStatsTblInfo is only used for test +func StabilizeGetStatsTblInfo(info *GetStatsTblInfo) { + info.TblInfoID = 100 + info.InputPhysicalID = 100 + tbl := info.StatsTblInfo + if tbl == nil { + return + } + tbl.PhysicalID = 100 + tbl.Version = 440930000000000000 + for _, col := range tbl.Columns { + col.LastUpdateVersion = 440930000000000000 + } + for _, idx := range tbl.Indexes { + idx.LastUpdateVersion = 440930000000000000 + } +}