Skip to content

Commit

Permalink
planner: fix column pruner will clear handleCols and lead filling _ti…
Browse files Browse the repository at this point in the history
…db_rowid when datasource's schema length is 0 for a pkIsHandled table (#45217) (#46134)

close #44579
  • Loading branch information
ti-chi-bot authored Aug 16, 2023
1 parent 2bcc671 commit e650558
Show file tree
Hide file tree
Showing 4 changed files with 14 additions and 2 deletions.
2 changes: 1 addition & 1 deletion planner/core/handle_cols.go
Original file line number Diff line number Diff line change
Expand Up @@ -42,7 +42,7 @@ type HandleCols interface {
BuildHandleFromIndexRow(row chunk.Row) (kv.Handle, 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 @@ -4816,6 +4816,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 @@ -1213,7 +1213,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

0 comments on commit e650558

Please sign in to comment.