Skip to content

Commit

Permalink
planner: move handle cols to planner util (#53093)
Browse files Browse the repository at this point in the history
ref #51664, ref #52714
  • Loading branch information
AilinKid authored May 8, 2024
1 parent fe7c324 commit 788d505
Show file tree
Hide file tree
Showing 20 changed files with 104 additions and 83 deletions.
2 changes: 1 addition & 1 deletion pkg/executor/admin.go
Original file line number Diff line number Diff line change
Expand Up @@ -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"
plannercore "github.com/pingcap/tidb/pkg/planner/util"
"github.com/pingcap/tidb/pkg/table"
"github.com/pingcap/tidb/pkg/table/tables"
"github.com/pingcap/tidb/pkg/tablecodec"
Expand Down
5 changes: 3 additions & 2 deletions pkg/executor/analyze_col.go
Original file line number Diff line number Diff line change
Expand Up @@ -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"
plannerutil "github.com/pingcap/tidb/pkg/planner/util"
"github.com/pingcap/tidb/pkg/statistics"
"github.com/pingcap/tidb/pkg/tablecodec"
"github.com/pingcap/tidb/pkg/types"
Expand All @@ -46,7 +47,7 @@ type AnalyzeColumnsExec struct {

tableInfo *model.TableInfo
colsInfo []*model.ColumnInfo
handleCols core.HandleCols
handleCols plannerutil.HandleCols
commonHandle *model.IndexInfo
resultHandler *tableResultHandler
indexes []*model.IndexInfo
Expand Down Expand Up @@ -379,7 +380,7 @@ func (e *AnalyzeColumnsExecV1) analyzeColumnsPushDownV1() *statistics.AnalyzeRes
}
}

func hasPkHist(handleCols core.HandleCols) bool {
func hasPkHist(handleCols plannerutil.HandleCols) bool {
return handleCols != nil && handleCols.IsInt()
}

Expand Down
12 changes: 6 additions & 6 deletions pkg/executor/builder.go
Original file line number Diff line number Diff line change
Expand Up @@ -615,14 +615,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) plannerutil.HandleCols {
if !tblInfo.IsCommonHandle {
extraColPos := len(allColInfo) - 1
intCol := &expression.Column{
Index: extraColPos,
RetType: types.NewFieldType(mysql.TypeLonglong),
}
return plannercore.NewIntHandleCols(intCol)
return plannerutil.NewIntHandleCols(intCol)
}
tblCols := make([]*expression.Column, len(tblInfo.Columns))
for i := 0; i < len(tblInfo.Columns); i++ {
Expand All @@ -640,7 +640,7 @@ func buildHandleColsForExec(sctx *stmtctx.StatementContext, tblInfo *model.Table
}
}
}
return plannercore.NewCommonHandleCols(sctx, tblInfo, pkIdx, tblCols)
return plannerutil.NewCommonHandleCols(sctx, tblInfo, pkIdx, tblCols)
}

func (b *executorBuilder) buildCleanupIndex(v *plannercore.CleanupIndex) exec.Executor {
Expand Down Expand Up @@ -2294,7 +2294,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) plannerutil.HandleCols {
if tbInfo.IsCommonHandle {
primaryIdx := tables.FindPrimaryIndex(tbInfo)
tableCols := make([]*expression.Column, len(tbInfo.Columns))
Expand All @@ -2307,12 +2307,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 plannerutil.NewCommonHandleCols(sc, tbInfo, primaryIdx, tableCols)
}
intCol := &expression.Column{
RetType: types.NewFieldType(mysql.TypeLonglong),
}
return plannercore.NewIntHandleCols(intCol)
return plannerutil.NewIntHandleCols(intCol)
}

func (b *executorBuilder) buildSplitRegion(v *plannercore.SplitRegion) exec.Executor {
Expand Down
5 changes: 3 additions & 2 deletions pkg/executor/delete.go
Original file line number Diff line number Diff line change
Expand Up @@ -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"
"github.com/pingcap/tidb/pkg/sessionctx"
"github.com/pingcap/tidb/pkg/sessionctx/variable"
"github.com/pingcap/tidb/pkg/sessiontxn"
Expand Down Expand Up @@ -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 util.HandleCols, isExtraHandle bool, row []types.Datum) error {
end := len(row)
if isExtraHandle {
end--
Expand All @@ -81,7 +82,7 @@ func (e *DeleteExec) deleteSingleTableByChunk(ctx context.Context) error {
var (
tbl table.Table
isExtrahandle bool
handleCols plannercore.HandleCols
handleCols util.HandleCols
rowCount int
)
for _, info := range e.tblColPosInfos {
Expand Down
3 changes: 2 additions & 1 deletion pkg/executor/executor.go
Original file line number Diff line number Diff line change
Expand Up @@ -55,6 +55,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"
plannerutil "github.com/pingcap/tidb/pkg/planner/util"
"github.com/pingcap/tidb/pkg/planner/util/fixcontrol"
"github.com/pingcap/tidb/pkg/privilege"
"github.com/pingcap/tidb/pkg/resourcemanager/pool/workerpool"
Expand Down Expand Up @@ -1099,7 +1100,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][]plannerutil.HandleCols

// When SelectLock work on a partition table, we need the partition ID
// (Physical Table ID) instead of the 'logical' table ID to calculate
Expand Down
12 changes: 6 additions & 6 deletions pkg/executor/index_merge_reader.go
Original file line number Diff line number Diff line change
Expand Up @@ -134,7 +134,7 @@ type IndexMergeReaderExecutor struct {
partialNetDataSizes []float64
dataAvgRowSize float64

handleCols plannercore.HandleCols
handleCols plannerutil.HandleCols
stats *IndexMergeRuntimeStat

// Indicates whether there is correlated column in filter or table/index range.
Expand Down Expand Up @@ -641,7 +641,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 plannerutil.HandleCols, parTblIdx int, partialPlanIndex int) (count int64, err error) {
chk := w.tableReader.NewChunkWithCapacity(w.getRetTpsForTableScan(), w.maxChunkSize, w.maxBatchSize)
for {
start := time.Now()
Expand Down Expand Up @@ -674,7 +674,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 plannerutil.HandleCols) (
handles []kv.Handle, retChk *chunk.Chunk, err error) {
handles = make([]kv.Handle, 0, w.batchSize)
if len(w.byItems) != 0 {
Expand Down Expand Up @@ -1724,7 +1724,7 @@ func (w *partialIndexWorker) fetchHandles(
exitCh <-chan struct{},
fetchCh chan<- *indexMergeTableTask,
finished <-chan struct{},
handleCols plannercore.HandleCols,
handleCols plannerutil.HandleCols,
partialPlanIndex int) (count int64, err error) {
tps := w.getRetTpsForIndexScan(handleCols)
chk := chunk.NewChunkWithCapacity(tps, w.maxChunkSize)
Expand Down Expand Up @@ -1757,7 +1757,7 @@ func (w *partialIndexWorker) fetchHandles(
return count, nil
}

func (w *partialIndexWorker) getRetTpsForIndexScan(handleCols plannercore.HandleCols) []*types.FieldType {
func (w *partialIndexWorker) getRetTpsForIndexScan(handleCols plannerutil.HandleCols) []*types.FieldType {
var tps []*types.FieldType
if len(w.byItems) != 0 {
for _, item := range w.byItems {
Expand All @@ -1771,7 +1771,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 plannerutil.HandleCols) (
handles []kv.Handle, retChk *chunk.Chunk, err error) {
handles = make([]kv.Handle, 0, w.batchSize)
if len(w.byItems) != 0 {
Expand Down
4 changes: 2 additions & 2 deletions pkg/executor/split.go
Original file line number Diff line number Diff line change
Expand Up @@ -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"
"github.com/pingcap/tidb/pkg/sessionctx"
"github.com/pingcap/tidb/pkg/store/helper"
"github.com/pingcap/tidb/pkg/table/tables"
Expand Down Expand Up @@ -330,7 +330,7 @@ type SplitTableRegionExec struct {
lower []types.Datum
upper []types.Datum
num int
handleCols core.HandleCols
handleCols util.HandleCols
valueLists [][]types.Datum
splitKeys [][]byte

Expand Down
6 changes: 3 additions & 3 deletions pkg/executor/split_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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"
"github.com/pingcap/tidb/pkg/sessionctx/stmtctx"
"github.com/pingcap/tidb/pkg/table/tables"
"github.com/pingcap/tidb/pkg/tablecodec"
Expand Down Expand Up @@ -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: util.NewIntHandleCols(&expression.Column{RetType: types.NewFieldType(mysql.TypeLonglong)}),
lower: []types.Datum{types.NewDatum(0)},
upper: []types.Datum{types.NewDatum(100)},
num: 10,
Expand Down Expand Up @@ -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: util.NewIntHandleCols(&expression.Column{RetType: types.NewFieldType(mysql.TypeLonglong)}),
lower: []types.Datum{types.NewDatum(0)},
upper: []types.Datum{types.NewDatum(1000)},
num: 10,
Expand Down
4 changes: 2 additions & 2 deletions pkg/executor/union_scan.go
Original file line number Diff line number Diff line change
Expand Up @@ -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"
plannerutil "github.com/pingcap/tidb/pkg/planner/util"
"github.com/pingcap/tidb/pkg/sessionctx/stmtctx"
"github.com/pingcap/tidb/pkg/table"
"github.com/pingcap/tidb/pkg/tablecodec"
Expand Down Expand Up @@ -294,7 +294,7 @@ type compareExec struct {
usedIndex []int
desc bool
// handleCols is the handle's position of the below scan plan.
handleCols plannercore.HandleCols
handleCols plannerutil.HandleCols
}

func (ce compareExec) compare(sctx *stmtctx.StatementContext, a, b []types.Datum) (ret int, err error) {
Expand Down
1 change: 0 additions & 1 deletion pkg/planner/core/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -15,7 +15,6 @@ go_library(
"flat_plan.go",
"foreign_key.go",
"fragment.go",
"handle_cols.go",
"hashcode.go",
"hint_utils.go",
"indexmerge_path.go",
Expand Down
3 changes: 2 additions & 1 deletion pkg/planner/core/common_plans.go
Original file line number Diff line number Diff line change
Expand Up @@ -29,6 +29,7 @@ import (
"github.com/pingcap/tidb/pkg/parser/mysql"
"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/costusage"
"github.com/pingcap/tidb/pkg/planner/util/optimizetrace"
"github.com/pingcap/tidb/pkg/sessionctx/variable"
Expand Down Expand Up @@ -520,7 +521,7 @@ type V2AnalyzeOptions struct {

// AnalyzeColumnsTask is used for analyze columns.
type AnalyzeColumnsTask struct {
HandleCols HandleCols
HandleCols util.HandleCols
CommonHandleInfo *model.IndexInfo
ColsInfo []*model.ColumnInfo
TblInfo *model.TableInfo
Expand Down
4 changes: 2 additions & 2 deletions pkg/planner/core/exhaust_physical_plans.go
Original file line number Diff line number Diff line change
Expand Up @@ -1291,8 +1291,8 @@ 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.(*util.CommonHandleCols)
for _, col := range commonHandle.GetColumns() {
if ts.schema.ColumnIndex(col) == -1 {
ts.Schema().Append(col)
ts.Columns = append(ts.Columns, col.ToInfo())
Expand Down
8 changes: 4 additions & 4 deletions pkg/planner/core/find_best_task.go
Original file line number Diff line number Diff line change
Expand Up @@ -1744,7 +1744,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 = util.NewIntHandleCols(ds.newExtraHandleSchemaCol())
}
hdColNum := handleCols.NumCols()
exprCols := make([]*expression.Column, 0, hdColNum)
Expand All @@ -1766,7 +1766,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 = util.NewIntHandleCols(ds.newExtraHandleSchemaCol())
}
hdColNum := handleCols.NumCols()
exprCols := make([]*expression.Column, 0, hdColNum)
Expand Down Expand Up @@ -2048,8 +2048,8 @@ 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.(*util.CommonHandleCols)
for _, col := range commonHandle.GetColumns() {
if ds.schema.ColumnIndex(col) == -1 {
ts := cop.tablePlan.(*PhysicalTableScan)
ts.Schema().Append(col)
Expand Down
Loading

0 comments on commit 788d505

Please sign in to comment.