Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

planner: fix column pruner will clear handleCols and lead filling _tidb_rowid when datasource's schema length is 0 for a pkIsHandled table (#45217) #45257

Merged
Merged
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: 1 addition & 1 deletion planner/core/handle_cols.go
Original file line number Diff line number Diff line change
Expand Up @@ -46,7 +46,7 @@ type HandleCols interface {
BuildPartitionHandleFromIndexRow(row chunk.Row) (kv.PartitionHandle, error)
// ResolveIndices resolves handle column indices.
ResolveIndices(schema *expression.Schema) (HandleCols, error)
// IsInt returns if the HandleCols is a single tnt column.
// IsInt returns if the HandleCols is a single int column.
IsInt() bool
// String implements the fmt.Stringer interface.
String() string
Expand Down
1 change: 1 addition & 0 deletions planner/core/logical_plan_builder.go
Original file line number Diff line number Diff line change
Expand Up @@ -4789,6 +4789,7 @@ 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}
b.handleHelper.pushMap(handleMap)
Expand Down
3 changes: 2 additions & 1 deletion planner/core/logical_plans.go
Original file line number Diff line number Diff line change
Expand Up @@ -1210,7 +1210,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
handleCols HandleCols
unMutableHandleCols HandleCols
// TblCols contains the original columns of table before being pruned, and it
// is used for estimating table scan cost.
TblCols []*expression.Column
Expand Down
10 changes: 10 additions & 0 deletions planner/core/rule_column_pruning.go
Original file line number Diff line number Diff line change
Expand Up @@ -343,6 +343,11 @@ func (ds *DataSource) PruneColumns(parentUsedCols []*expression.Column, opt *log
ds.Columns = append(ds.Columns, handleColInfo)
ds.schema.Append(handleCol)
}
// ref: https://github.com/pingcap/tidb/issues/44579
// when first entering columnPruner, we kept a column-a in datasource since upper agg function count(a) is used.
// then we mark the handleCols as nil here.
// when second entering columnPruner, the count(a) is eliminated since it always not null. we should fill another
// extra col, in this way, handle col is useful again, otherwise, _tidb_rowid will be filled.
if ds.handleCols != nil && ds.handleCols.IsInt() && ds.schema.ColumnIndex(ds.handleCols.GetCol(0)) == -1 {
ds.handleCols = nil
}
Expand Down Expand Up @@ -659,6 +664,11 @@ func preferKeyColumnFromTable(dataSource *DataSource, originColumns []*expressio
if dataSource.handleCols != nil {
resultColumn = dataSource.handleCols.GetCol(0)
resultColumnInfo = resultColumn.ToInfo()
} else if dataSource.table.Meta().PKIsHandle {
// dataSource.handleCols = nil doesn't mean datasource doesn't have a intPk handle.
// since datasource.handleCols will be cleared in the first columnPruner.
resultColumn = dataSource.unMutableHandleCols.GetCol(0)
resultColumnInfo = resultColumn.ToInfo()
} else {
resultColumn = dataSource.newExtraHandleSchemaCol()
resultColumnInfo = model.NewExtraHandleColInfo()
Expand Down