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

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
10 changes: 10 additions & 0 deletions pkg/meta/model/column.go
Original file line number Diff line number Diff line change
Expand Up @@ -313,3 +313,13 @@ func NewExtraPhysTblIDColInfo() *ColumnInfo {
colInfo.SetCollate(charset.CollationBin)
return colInfo
}

// GetColIdxMapping builds a map of column names to their corresponding offsets
// into the given slice of column infos.
func GetColIdxMapping(colInfos []*ColumnInfo) map[string]int {
colIdxsByName := make(map[string]int, len(colInfos))
for i, info := range colInfos {
colIdxsByName[info.Name.L] = i
}
return colIdxsByName
}
23 changes: 23 additions & 0 deletions pkg/meta/model/column_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -104,3 +104,26 @@ func TestDefaultValue(t *testing.T) {
require.Equal(t, mysql.NotNullFlag, extraPhysTblIDCol.GetFlag())
require.Equal(t, mysql.TypeLonglong, extraPhysTblIDCol.GetType())
}

func TestColIdxMapping(t *testing.T) {
const numCols int64 = 10
colInfos := make([]*ColumnInfo, 0, numCols)
names := make([]string, 0, numCols)
for i := range numCols {
name := ast.NewCIStr(fmt.Sprintf("col%d", i))
colInfo := &ColumnInfo{
ID: i,
Name: name,
}
colInfos = append(colInfos, colInfo)
names = append(names, name.L)
}

colIdxsByName := GetColIdxMapping(colInfos)
for _, name := range names {
i, found := colIdxsByName[name]
require.True(t, found)
colInfo := colInfos[i]
require.Equal(t, name, colInfo.Name.L)
}
}
2 changes: 2 additions & 0 deletions pkg/planner/core/exhaust_physical_plans.go
Original file line number Diff line number Diff line change
Expand Up @@ -1216,6 +1216,7 @@ func constructDS2TableScanTask(
ts := physicalop.PhysicalTableScan{
Table: ds.TableInfo,
Columns: ds.Columns,
ColIdxsByName: ds.ColIdxsByName,
TableAsName: ds.TableAsName,
DBName: ds.DBName,
FilterCondition: ds.PushedDownConds,
Expand Down Expand Up @@ -1463,6 +1464,7 @@ func constructDS2IndexScanTask(
// On this way, it's double read case.
ts := physicalop.PhysicalTableScan{
Columns: ds.Columns,
ColIdxsByName: ds.ColIdxsByName,
Table: is.Table,
TableAsName: ds.TableAsName,
DBName: ds.DBName,
Expand Down
1 change: 1 addition & 0 deletions pkg/planner/core/find_best_task.go
Original file line number Diff line number Diff line change
Expand Up @@ -2060,6 +2060,7 @@ func convertToIndexScan(ds *logicalop.DataSource, prop *property.PhysicalPropert
// On this way, it's double read case.
ts := physicalop.PhysicalTableScan{
Columns: sliceutil.DeepClone(ds.Columns),
ColIdxsByName: model.GetColIdxMapping(ds.Columns),
Table: is.Table,
TableAsName: ds.TableAsName,
DBName: ds.DBName,
Expand Down
16 changes: 6 additions & 10 deletions pkg/planner/core/logical_plan_builder.go
Original file line number Diff line number Diff line change
Expand Up @@ -4114,9 +4114,7 @@ func addExtraPhysTblIDColumn4DS(ds *logicalop.DataSource) *expression.Column {
OrigName: fmt.Sprintf("%v.%v.%v", ds.DBName, ds.TableInfo.Name, model.ExtraPhysTblIDName),
}

ds.Columns = append(ds.Columns, model.NewExtraPhysTblIDColInfo())
schema := ds.Schema()
schema.Append(pidCol)
ds.AppendColumn(pidCol, model.NewExtraPhysTblIDColInfo())
ds.SetOutputNames(append(ds.OutputNames(), &types.FieldName{
DBName: ds.DBName,
TblName: ds.TableInfo.Name,
Expand Down Expand Up @@ -4598,18 +4596,18 @@ func (b *PlanBuilder) buildDataSource(ctx context.Context, tn *ast.TableName, as
PossibleAccessPaths: possiblePaths,
AllPossibleAccessPaths: allPaths,
Columns: make([]*model.ColumnInfo, 0, countCnt),
ColIdxsByName: make(map[string]int, countCnt),
PartitionNames: tn.PartitionNames,
TblCols: make([]*expression.Column, 0, countCnt),
TblColsByID: make(map[int64]*expression.Column, countCnt),
PreferPartitions: make(map[int][]ast.CIStr),
IS: b.is,
IsForUpdateRead: b.isForUpdateRead,
}.Init(b.ctx, b.getSelectOffset())
ds.SetSchema(expression.NewSchema(make([]*expression.Column, 0, countCnt)...))
var handleCols util.HandleCols
schema := expression.NewSchema(make([]*expression.Column, 0, countCnt)...)
names := make([]*types.FieldName, 0, countCnt)
for i, col := range columns {
ds.Columns = append(ds.Columns, col.ToInfo())
names = append(names, &types.FieldName{
DBName: dbName,
TblName: tableInfo.Name,
Expand All @@ -4629,7 +4627,7 @@ func (b *PlanBuilder) buildDataSource(ctx context.Context, tn *ast.TableName, as
if col.IsPKHandleColumn(tableInfo) {
handleCols = util.NewIntHandleCols(newCol)
}
schema.Append(newCol)
ds.AppendColumn(newCol, col.ToInfo())
ds.AppendTableCol(newCol)
}
// We append an extra handle column to the schema when the handle
Expand All @@ -4641,8 +4639,7 @@ func (b *PlanBuilder) buildDataSource(ctx context.Context, tn *ast.TableName, as
} else {
extraCol := ds.NewExtraHandleSchemaCol()
handleCols = util.NewIntHandleCols(extraCol)
ds.Columns = append(ds.Columns, model.NewExtraHandleColInfo())
schema.Append(extraCol)
ds.AppendColumn(extraCol, model.NewExtraHandleColInfo())
names = append(names, &types.FieldName{
DBName: dbName,
TblName: tableInfo.Name,
Expand All @@ -4657,13 +4654,12 @@ func (b *PlanBuilder) buildDataSource(ctx context.Context, tn *ast.TableName, as
handleMap := make(map[int64][]util.HandleCols)
handleMap[tableInfo.ID] = []util.HandleCols{handleCols}
b.handleHelper.pushMap(handleMap)
ds.SetSchema(schema)
ds.SetOutputNames(names)
// setPreferredStoreType will mark user preferred path, which should be shared by all ds alternative. Here
// we only mark it for the AllPossibleAccessPaths(since the element inside is shared by PossibleAccessPaths),
// and the following ds alternative will clone/inherit this mark from DS copying.
setPreferredStoreType(ds, b.TableHints())
ds.SampleInfo = tablesampler.NewTableSampleInfo(tn.TableSample, schema, b.partitionedTable)
ds.SampleInfo = tablesampler.NewTableSampleInfo(tn.TableSample, ds.Schema(), b.partitionedTable)
b.isSampling = ds.SampleInfo != nil

for i, colExpr := range ds.Schema().Columns {
Expand Down
21 changes: 19 additions & 2 deletions pkg/planner/core/operator/logicalop/logical_datasource.go
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,7 @@ package logicalop
import (
"bytes"
"fmt"
"maps"

"github.com/pingcap/tidb/pkg/expression"
"github.com/pingcap/tidb/pkg/infoschema"
Expand Down Expand Up @@ -52,6 +53,8 @@ type DataSource struct {
Columns []*model.ColumnInfo
DBName ast.CIStr

ColIdxsByName map[string]int

TableAsName *ast.CIStr `hash64-equals:"true"`
// IndexMergeHints are the hint for indexmerge.
IndexMergeHints []h.HintedIndex
Expand Down Expand Up @@ -195,6 +198,10 @@ func (ds *DataSource) PruneColumns(parentUsedCols []*expression.Column) (base.Lo
expression.GcColumnExprIsTidbShard(ds.Schema().Columns[i].VirtualExpr) {
continue
}
delete(ds.ColIdxsByName, ds.Columns[i].Name.L)
for j := i + 1; j < len(ds.Columns); j++ {
ds.ColIdxsByName[ds.Columns[j].Name.L] = j - 1
}
// TODO: investigate why we cannot use slices.Delete for these two:
ds.Schema().Columns = append(ds.Schema().Columns[:i], ds.Schema().Columns[i+1:]...)
ds.Columns = append(ds.Columns[:i], ds.Columns[i+1:]...)
Expand All @@ -207,8 +214,7 @@ func (ds *DataSource) PruneColumns(parentUsedCols []*expression.Column) (base.Lo
var handleCol *expression.Column
var handleColInfo *model.ColumnInfo
handleCol, handleColInfo = preferKeyColumnFromTable(ds, originSchemaColumns, originColumns)
ds.Columns = append(ds.Columns, handleColInfo)
ds.Schema().Append(handleCol)
ds.AppendColumn(handleCol, handleColInfo)
addOneHandle = true
}
// ref: https://github.com/pingcap/tidb/issues/44579
Expand Down Expand Up @@ -498,6 +504,8 @@ func (ds *DataSource) buildIndexGather(path *util.AccessPath) base.LogicalPlan {

is.Columns = make([]*model.ColumnInfo, len(ds.Columns))
copy(is.Columns, ds.Columns)
is.ColIdxsByName = make(map[string]int, len(ds.Columns))
maps.Copy(is.ColIdxsByName, ds.ColIdxsByName)
is.SetSchema(ds.Schema())
is.IdxCols, is.IdxColLens = expression.IndexInfo2PrefixCols(is.Columns, is.Schema().Columns, is.Index)

Expand Down Expand Up @@ -695,6 +703,15 @@ func isIndexColsCoveringCol(sctx expression.EvalContext, col *expression.Column,
return false
}

// AppendColumn appends the given column and info to the columns and schema's
// columns of this data source.
func (ds *DataSource) AppendColumn(col *expression.Column, colInfo *model.ColumnInfo) {
colIdx := len(ds.Columns)
ds.ColIdxsByName[colInfo.Name.L] = colIdx
ds.Columns = append(ds.Columns, colInfo)
ds.Schema().Append(col)
}

// AppendTableCol appends a column to the original columns of the table before pruning,
// accessed through ds.TblCols and ds.TblColsByID.
func (ds *DataSource) AppendTableCol(col *expression.Column) {
Expand Down
1 change: 1 addition & 0 deletions pkg/planner/core/operator/logicalop/logical_index_scan.go
Original file line number Diff line number Diff line change
Expand Up @@ -42,6 +42,7 @@ type LogicalIndexScan struct {

Index *model.IndexInfo
Columns []*model.ColumnInfo
ColIdxsByName map[string]int
FullIdxCols []*expression.Column
FullIdxColLens []int
IdxCols []*expression.Column
Expand Down
6 changes: 6 additions & 0 deletions pkg/planner/core/operator/physicalop/physical_table_scan.go
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,7 @@ package physicalop
import (
"bytes"
"fmt"
"maps"
"slices"
"strconv"
"strings"
Expand Down Expand Up @@ -97,6 +98,8 @@ type PhysicalTableScan struct {
DBName ast.CIStr `plan-cache-clone:"shallow"`
Ranges []*ranger.Range `plan-cache-clone:"shallow"`

ColIdxsByName map[string]int `plan-cache-clone:"shallow"`

TableAsName *ast.CIStr `plan-cache-clone:"shallow"`

PhysicalTableID int64
Expand Down Expand Up @@ -161,6 +164,7 @@ func GetPhysicalScan4LogicalTableScan(s *logicalop.LogicalTableScan, schema *exp
ts := PhysicalTableScan{
Table: ds.TableInfo,
Columns: ds.Columns,
ColIdxsByName: ds.ColIdxsByName,
TableAsName: ds.TableAsName,
DBName: ds.DBName,
isPartition: ds.PartitionDefIdx != nil,
Expand All @@ -180,6 +184,7 @@ func GetOriginalPhysicalTableScan(ds *logicalop.DataSource, prop *property.Physi
ts := PhysicalTableScan{
Table: ds.TableInfo,
Columns: slices.Clone(ds.Columns),
ColIdxsByName: maps.Clone(ds.ColIdxsByName),
TableAsName: ds.TableAsName,
DBName: ds.DBName,
isPartition: ds.PartitionDefIdx != nil,
Expand Down Expand Up @@ -767,6 +772,7 @@ func BuildIndexMergeTableScan(ds *logicalop.DataSource, tableFilters []expressio
ts := PhysicalTableScan{
Table: ds.TableInfo,
Columns: slices.Clone(ds.Columns),
ColIdxsByName: maps.Clone(ds.ColIdxsByName),
TableAsName: ds.TableAsName,
DBName: ds.DBName,
PhysicalTableID: ds.PhysicalTableID,
Expand Down
2 changes: 2 additions & 0 deletions pkg/planner/core/planbuilder.go
Original file line number Diff line number Diff line change
Expand Up @@ -1659,6 +1659,7 @@ func (b *PlanBuilder) buildPhysicalIndexLookUpReader(_ context.Context, dbName a
pkHandleInfo, pkHandleCol, hasPkIsHandle := tryGetPkHandleCol(tblInfo, fullExprCols)
commonInfos, commonCols, hasCommonCols := tryGetCommonHandleCols(tbl, fullExprCols)
idxColInfos := getIndexColumnInfos(tblInfo, idx)
colIdxsByName := model.GetColIdxMapping(idxColInfos)
idxColSchema := getIndexColsSchema(tblInfo, idx, fullExprCols)
idxCols, idxColLens := expression.IndexInfo2PrefixCols(idxColInfos, idxColSchema.Columns, idx)
pseudoHistColl := statistics.PseudoHistColl(physicalID, false)
Expand Down Expand Up @@ -1688,6 +1689,7 @@ func (b *PlanBuilder) buildPhysicalIndexLookUpReader(_ context.Context, dbName a
// It's double read case.
ts := physicalop.PhysicalTableScan{
Columns: idxColInfos,
ColIdxsByName: colIdxsByName,
Table: tblInfo,
TableAsName: &tblInfo.Name,
DBName: dbName,
Expand Down
42 changes: 42 additions & 0 deletions pkg/planner/core/planbuilder_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -37,6 +37,7 @@ import (
"github.com/pingcap/tidb/pkg/parser/ast"
"github.com/pingcap/tidb/pkg/parser/mysql"
"github.com/pingcap/tidb/pkg/planner/core/base"
"github.com/pingcap/tidb/pkg/planner/core/operator/logicalop"
"github.com/pingcap/tidb/pkg/planner/core/operator/physicalop"
"github.com/pingcap/tidb/pkg/planner/core/resolve"
"github.com/pingcap/tidb/pkg/planner/property"
Expand Down Expand Up @@ -1275,3 +1276,44 @@ func TestIndexLookUpReaderTryLookUpPushDown(t *testing.T) {
require.True(t, ok)
check(clonedForCache)
}

// TestDataSourceColumnMaps ensures that AppendColumn keeps the Columns slice
// coherent with the ColIdxsByName map, and that AppendTableCol keeps the
// TblCols slice coherent with the TblColsByID map.
func TestDataSourceColumnMaps(t *testing.T) {
ctx := coretestsdk.MockContext()
defer func() {
domain.GetDomain(ctx).StatsHandle().Close()
}()
b, _ := NewPlanBuilder().Init(ctx, nil, hint.NewQBHintHandler(nil))

colInfos := []*model.ColumnInfo{
{ID: 1, Name: ast.NewCIStr("id"), FieldType: *types.NewFieldType(mysql.TypeLonglong)},
{ID: 2, Name: ast.NewCIStr("name"), FieldType: *types.NewFieldType(mysql.TypeString)},
{ID: 3, Name: ast.NewCIStr("age"), FieldType: *types.NewFieldType(mysql.TypeLonglong)},
}
colCount := len(colInfos)

ds := logicalop.DataSource{
Columns: make([]*model.ColumnInfo, 0, colCount),
ColIdxsByName: make(map[string]int, colCount),
TblCols: make([]*expression.Column, 0, colCount),
TblColsByID: make(map[int64]*expression.Column, colCount),
}.Init(b.ctx, b.getSelectOffset())
ds.SetSchema(expression.NewSchema(make([]*expression.Column, 0, colCount)...))
for _, colInfo := range colInfos {
newCol := &expression.Column{ID: colInfo.ID}
ds.AppendColumn(newCol, colInfo)
ds.AppendTableCol(newCol)
}

for _, colInfo := range colInfos {
colIdx, found := ds.ColIdxsByName[colInfo.Name.L]
require.True(t, found)
require.Equal(t, ds.Columns[colIdx], colInfo)

tblCol, found := ds.TblColsByID[colInfo.ID]
require.True(t, found)
require.Equal(t, ds.TblCols[colIdx], tblCol)
}
}