Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
2 changes: 2 additions & 0 deletions pkg/executor/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -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",
Expand Down Expand Up @@ -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",
Expand Down
8 changes: 4 additions & 4 deletions 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"
"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"
Expand Down Expand Up @@ -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
Expand Down Expand Up @@ -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
Expand Down Expand Up @@ -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
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"
"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"
Expand All @@ -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
Expand Down Expand Up @@ -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()
}

Expand Down
13 changes: 7 additions & 6 deletions pkg/executor/builder.go
Original file line number Diff line number Diff line change
Expand Up @@ -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"
Expand Down Expand Up @@ -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++ {
Expand All @@ -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 {
Expand Down Expand Up @@ -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))
Expand All @@ -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 {
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/handlecol"
"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 handlecol.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 handlecol.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 @@ -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"
Expand Down Expand Up @@ -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
Expand Down
13 changes: 7 additions & 6 deletions pkg/executor/index_merge_reader.go
Original file line number Diff line number Diff line change
Expand Up @@ -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"
Expand Down Expand Up @@ -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.
Expand Down Expand Up @@ -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()
Expand Down Expand Up @@ -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 {
Expand Down Expand Up @@ -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)
Expand Down Expand Up @@ -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 {
Expand All @@ -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 {
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/handlecol"
"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 handlecol.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/handlecol"
"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: handlecol.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: handlecol.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"
"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"
Expand Down Expand Up @@ -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) {
Expand Down
3 changes: 1 addition & 2 deletions pkg/planner/core/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -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",
Expand All @@ -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",
Expand Down Expand Up @@ -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",
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 @@ -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"
Expand Down Expand Up @@ -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
Expand Down
6 changes: 4 additions & 2 deletions pkg/planner/core/exhaust_physical_plans.go
Original file line number Diff line number Diff line change
Expand Up @@ -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"
Expand Down Expand Up @@ -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())
Expand Down
Loading