Skip to content

Commit

Permalink
planner: refactor getFullAnalyzeColumnsInfo (#46044)
Browse files Browse the repository at this point in the history
  • Loading branch information
Rustin170506 committed Aug 14, 2023
1 parent e9f4e31 commit d4a50aa
Show file tree
Hide file tree
Showing 3 changed files with 117 additions and 61 deletions.
4 changes: 2 additions & 2 deletions executor/test/analyzetest/analyze_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -2126,9 +2126,9 @@ func TestAnalyzeColumnsErrorAndWarning(t *testing.T) {
// analyze version 1 doesn't support `ANALYZE COLUMNS c1, ..., cn`/`ANALYZE PREDICATE COLUMNS` currently
tk.MustExec("set @@tidb_analyze_version = 1")
err := tk.ExecToErr("analyze table t columns a")
require.Equal(t, "Only the analyze version 2 supports analyzing the specified columns", err.Error())
require.Equal(t, "Only the version 2 of analyze supports analyzing the specified columns", err.Error())
err = tk.ExecToErr("analyze table t predicate columns")
require.Equal(t, "Only the analyze version 2 supports analyzing predicate columns", err.Error())
require.Equal(t, "Only the version 2 of analyze supports analyzing predicate columns", err.Error())

tk.MustExec("set @@tidb_analyze_version = 2")
// invalid column
Expand Down
152 changes: 99 additions & 53 deletions planner/core/planbuilder.go
Original file line number Diff line number Diff line change
Expand Up @@ -2325,15 +2325,7 @@ func (b *PlanBuilder) getFullAnalyzeColumnsInfo(
if mustAllColumns && warning && (columnChoice == model.PredicateColumns || columnChoice == model.ColumnList) {
b.ctx.GetSessionVars().StmtCtx.AppendWarning(errors.Errorf("Table %s.%s has version 1 statistics so all the columns must be analyzed to overwrite the current statistics", tbl.Schema.L, tbl.Name.L))
}
colSet2colList := func(colSet map[int64]struct{}) []*model.ColumnInfo {
colList := make([]*model.ColumnInfo, 0, len(colSet))
for _, colInfo := range tbl.TableInfo.Columns {
if _, ok := colSet[colInfo.ID]; ok {
colList = append(colList, colInfo)
}
}
return colList
}

switch columnChoice {
case model.DefaultChoice, model.AllColumns:
return tbl.TableInfo.Columns, nil, nil
Expand All @@ -2349,52 +2341,86 @@ func (b *PlanBuilder) getFullAnalyzeColumnsInfo(
if err != nil {
return nil, nil, err
}
colSet := make(map[int64]struct{}, len(predicate)+len(mustAnalyzed))
for colID := range predicate {
colSet[colID] = struct{}{}
}
for colID := range mustAnalyzed {
colSet[colID] = struct{}{}
}
return colSet2colList(colSet), nil, nil
colSet := combineColumnSets(predicate, mustAnalyzed)
return getColumnListFromSet(tbl.TableInfo.Columns, colSet), nil, nil
case model.ColumnList:
colSet := make(map[int64]struct{}, len(specifiedCols))
for _, colInfo := range specifiedCols {
colSet[colInfo.ID] = struct{}{}
}
colSet := getColumnSetFromSpecifiedCols(specifiedCols)
mustAnalyzed, err := b.getMustAnalyzedColumns(tbl, mustAnalyzedCols)
if err != nil {
return nil, nil, err
}
if warning {
missing := make(map[int64]struct{}, len(mustAnalyzed))
for colID := range mustAnalyzed {
if _, ok := colSet[colID]; !ok {
missing[colID] = struct{}{}
}
}
missing := getMissingColumns(colSet, mustAnalyzed)
if len(missing) > 0 {
missingNames := make([]string, 0, len(missing))
for _, col := range tbl.TableInfo.Columns {
if _, ok := missing[col.ID]; ok {
missingNames = append(missingNames, col.Name.O)
}
}
b.ctx.GetSessionVars().StmtCtx.AppendWarning(errors.Errorf("Columns %s are missing in ANALYZE but their stats are needed for calculating stats for indexes/primary key/extended stats", strings.Join(missingNames, ",")))
missingNames := getColumnNamesFromIDs(tbl.TableInfo.Columns, missing)
warningMsg := fmt.Sprintf("Columns %s are missing in ANALYZE but their stats are needed for calculating stats for indexes/primary key/extended stats", strings.Join(missingNames, ","))
b.ctx.GetSessionVars().StmtCtx.AppendWarning(errors.New(warningMsg))
}
}
for colID := range mustAnalyzed {
colSet[colID] = struct{}{}
}
colList := colSet2colList(colSet)
colSet = combineColumnSets(colSet, mustAnalyzed)
colList := getColumnListFromSet(tbl.TableInfo.Columns, colSet)
if mustAllColumns {
return tbl.TableInfo.Columns, colList, nil
}
return colList, colList, nil
}

return nil, nil, nil
}

// Helper function to combine two column sets.
func combineColumnSets(sets ...map[int64]struct{}) map[int64]struct{} {
result := make(map[int64]struct{})
for _, set := range sets {
for colID := range set {
result[colID] = struct{}{}
}
}
return result
}

// Helper function to extract column IDs from specified columns.
func getColumnSetFromSpecifiedCols(cols []*model.ColumnInfo) map[int64]struct{} {
colSet := make(map[int64]struct{}, len(cols))
for _, colInfo := range cols {
colSet[colInfo.ID] = struct{}{}
}
return colSet
}

// Helper function to get missing columns from a set.
func getMissingColumns(colSet, mustAnalyzed map[int64]struct{}) map[int64]struct{} {
missing := make(map[int64]struct{})
for colID := range mustAnalyzed {
if _, ok := colSet[colID]; !ok {
missing[colID] = struct{}{}
}
}
return missing
}

// Helper function to get column names from IDs.
func getColumnNamesFromIDs(columns []*model.ColumnInfo, colIDs map[int64]struct{}) []string {
var missingNames []string
for _, col := range columns {
if _, ok := colIDs[col.ID]; ok {
missingNames = append(missingNames, col.Name.O)
}
}
return missingNames
}

// Helper function to get a list of column infos from a set of column IDs.
func getColumnListFromSet(columns []*model.ColumnInfo, colSet map[int64]struct{}) []*model.ColumnInfo {
colList := make([]*model.ColumnInfo, 0, len(colSet))
for _, colInfo := range columns {
if _, ok := colSet[colInfo.ID]; ok {
colList = append(colList, colInfo)
}
}
return colList
}

func getColOffsetForAnalyze(colsInfo []*model.ColumnInfo, colID int64) int {
for i, col := range colsInfo {
if colID == col.ID {
Expand Down Expand Up @@ -2438,30 +2464,36 @@ func getModifiedIndexesInfoForAnalyze(sctx sessionctx.Context, tblInfo *model.Ta

func (b *PlanBuilder) buildAnalyzeFullSamplingTask(
as *ast.AnalyzeTableStmt,
taskSlice []AnalyzeColumnsTask,
tasks []AnalyzeColumnsTask,
physicalIDs []int64,
names []string,
partitionNames []string,
tbl *ast.TableName,
version int,
persistOpts bool,
rsOptionsMap map[int64]V2AnalyzeOptions,
) ([]AnalyzeColumnsTask, error) {
// Version 2 doesn't support incremental analyze.
// And incremental analyze will be deprecated in the future.
if as.Incremental {
b.ctx.GetSessionVars().StmtCtx.AppendWarning(errors.Errorf("The version 2 stats would ignore the INCREMENTAL keyword and do full sampling"))
}
astOpts, err := parseAnalyzeOptionsV2(as.AnalyzeOpts)

astOpts, err := handleAnalyzeOptionsV2(as.AnalyzeOpts)
if err != nil {
return nil, err
}
// Get all column info which need to be analyzed.
astColList, err := getAnalyzeColumnList(as.ColumnNames, tbl)
if err != nil {
return nil, err
}

var predicateCols, mustAnalyzedCols calcOnceMap
ver := version
statsHandle := domain.GetDomain(b.ctx).StatsHandle()
// If the statistics of the table is version 1, we must analyze all columns to overwrites all of old statistics.
mustAllColumns := !statsHandle.CheckAnalyzeVersion(tbl.TableInfo, physicalIDs, &ver)

astColsInfo, _, err := b.getFullAnalyzeColumnsInfo(tbl, as.ColumnChoice, astColList, &predicateCols, &mustAnalyzedCols, mustAllColumns, true)
if err != nil {
return nil, err
Expand All @@ -2474,6 +2506,7 @@ func (b *PlanBuilder) buildAnalyzeFullSamplingTask(
for physicalID, opts := range optionsMap {
rsOptionsMap[physicalID] = opts
}

for i, id := range physicalIDs {
physicalID := id
if id == tbl.TableInfo.ID {
Expand All @@ -2482,7 +2515,7 @@ func (b *PlanBuilder) buildAnalyzeFullSamplingTask(
info := AnalyzeInfo{
DBName: tbl.Schema.O,
TableName: tbl.Name.O,
PartitionName: names[i],
PartitionName: partitionNames[i],
TableID: statistics.AnalyzeTableID{TableID: tbl.TableInfo.ID, PartitionID: id},
Incremental: false,
StatsVersion: version,
Expand Down Expand Up @@ -2541,9 +2574,10 @@ func (b *PlanBuilder) buildAnalyzeFullSamplingTask(
newTask.ColsInfo = append(newTask.ColsInfo, extraCol)
newTask.HandleCols = &IntHandleCols{col: colInfoToColumn(extraCol, len(newTask.ColsInfo)-1)}
}
taskSlice = append(taskSlice, newTask)
tasks = append(tasks, newTask)
}
return taskSlice, nil

return tasks, nil
}

func (b *PlanBuilder) genV2AnalyzeOptions(
Expand Down Expand Up @@ -2708,26 +2742,30 @@ func mergeColumnList(choice1 model.ColumnChoice, list1 []*model.ColumnInfo, choi
return choice2, list2
}

// buildAnalyzeTable constructs anylyze tasks for each table.
func (b *PlanBuilder) buildAnalyzeTable(as *ast.AnalyzeTableStmt, opts map[ast.AnalyzeOptionType]uint64, version int) (Plan, error) {
p := &Analyze{Opts: opts}
p.OptionsMap = make(map[int64]V2AnalyzeOptions)
usePersistedOptions := variable.PersistAnalyzeOptions.Load()

// Construct tasks for each table.
for _, tbl := range as.TableNames {
if tbl.TableInfo.IsView() {
return nil, errors.Errorf("analyze view %s is not supported now", tbl.Name.O)
}
if tbl.TableInfo.IsSequence() {
return nil, errors.Errorf("analyze sequence %s is not supported now", tbl.Name.O)
}

idxInfo, colInfo := getColsInfo(tbl)
physicalIDs, names, err := GetPhysicalIDsAndPartitionNames(tbl.TableInfo, as.PartitionNames)
physicalIDs, partitionNames, err := GetPhysicalIDsAndPartitionNames(tbl.TableInfo, as.PartitionNames)
if err != nil {
return nil, err
}
var commonHandleInfo *model.IndexInfo
// If we want to analyze this table with analyze version 2 but the existing stats is version 1 and stats feedback is enabled,
// we will switch back to analyze version 1.
if statistics.FeedbackProbability.Load() > 0 && version == 2 {
if statistics.FeedbackProbability.Load() > 0 && version == statistics.Version2 {
statsHandle := domain.GetDomain(b.ctx).StatsHandle()
versionIsSame := statsHandle.CheckAnalyzeVersion(tbl.TableInfo, physicalIDs, &version)
if !versionIsSame {
Expand All @@ -2736,18 +2774,21 @@ func (b *PlanBuilder) buildAnalyzeTable(as *ast.AnalyzeTableStmt, opts map[ast.A
"If you want to switch to version 2 statistics, please first disable query feedback by setting feedback-probability to 0.0 in the config file.", tbl.Name))
}
}

if version == statistics.Version2 {
p.ColTasks, err = b.buildAnalyzeFullSamplingTask(as, p.ColTasks, physicalIDs, names, tbl, version, usePersistedOptions, p.OptionsMap)
p.ColTasks, err = b.buildAnalyzeFullSamplingTask(as, p.ColTasks, physicalIDs, partitionNames, tbl, version, usePersistedOptions, p.OptionsMap)
if err != nil {
return nil, err
}
continue
}

// Version 1 analyze.
if as.ColumnChoice == model.PredicateColumns {
return nil, errors.Errorf("Only the analyze version 2 supports analyzing predicate columns")
return nil, errors.Errorf("Only the version 2 of analyze supports analyzing predicate columns")
}
if as.ColumnChoice == model.ColumnList {
return nil, errors.Errorf("Only the analyze version 2 supports analyzing the specified columns")
return nil, errors.Errorf("Only the version 2 of analyze supports analyzing the specified columns")
}
for _, idx := range idxInfo {
// For prefix common handle. We don't use analyze mixed to handle it with columns. Because the full value
Expand All @@ -2767,7 +2808,7 @@ func (b *PlanBuilder) buildAnalyzeTable(as *ast.AnalyzeTableStmt, opts map[ast.A
info := AnalyzeInfo{
DBName: tbl.Schema.O,
TableName: tbl.Name.O,
PartitionName: names[i],
PartitionName: partitionNames[i],
TableID: statistics.AnalyzeTableID{TableID: tbl.TableInfo.ID, PartitionID: id},
Incremental: as.Incremental,
StatsVersion: version,
Expand All @@ -2788,7 +2829,7 @@ func (b *PlanBuilder) buildAnalyzeTable(as *ast.AnalyzeTableStmt, opts map[ast.A
info := AnalyzeInfo{
DBName: tbl.Schema.O,
TableName: tbl.Name.O,
PartitionName: names[i],
PartitionName: partitionNames[i],
TableID: statistics.AnalyzeTableID{TableID: tbl.TableInfo.ID, PartitionID: id},
Incremental: as.Incremental,
StatsVersion: version,
Expand All @@ -2803,6 +2844,7 @@ func (b *PlanBuilder) buildAnalyzeTable(as *ast.AnalyzeTableStmt, opts map[ast.A
}
}
}

return p, nil
}

Expand Down Expand Up @@ -2971,7 +3013,9 @@ var analyzeOptionDefaultV2 = map[ast.AnalyzeOptionType]uint64{
ast.AnalyzeOptSampleRate: math.Float64bits(-1),
}

func parseAnalyzeOptionsV2(opts []ast.AnalyzeOpt) (map[ast.AnalyzeOptionType]uint64, error) {
// This function very similar to handleAnalyzeOptions, but it's used for analyze version 2.
// Remove this function after we remove the support of analyze version 1.
func handleAnalyzeOptionsV2(opts []ast.AnalyzeOpt) (map[ast.AnalyzeOptionType]uint64, error) {
optMap := make(map[ast.AnalyzeOptionType]uint64, len(analyzeOptionDefault))
sampleNum, sampleRate := uint64(0), 0.0
for _, opt := range opts {
Expand Down Expand Up @@ -3009,6 +3053,7 @@ func parseAnalyzeOptionsV2(opts []ast.AnalyzeOpt) (map[ast.AnalyzeOptionType]uin
if sampleNum > 0 && sampleRate > 0 {
return nil, errors.Errorf("You can only either set the value of the sample num or set the value of the sample rate. Don't set both of them")
}

return optMap, nil
}

Expand Down Expand Up @@ -3074,7 +3119,8 @@ func handleAnalyzeOptions(opts []ast.AnalyzeOpt, statsVer int) (map[ast.AnalyzeO
if sampleNum > 0 && sampleRate > 0 {
return nil, errors.Errorf("You can only either set the value of the sample num or set the value of the sample rate. Don't set both of them")
}
if optMap[ast.AnalyzeOptCMSketchWidth]*optMap[ast.AnalyzeOptCMSketchDepth] > CMSketchSizeLimit {
// Only version 1 has cmsketch.
if statsVer == statistics.Version1 && optMap[ast.AnalyzeOptCMSketchWidth]*optMap[ast.AnalyzeOptCMSketchDepth] > CMSketchSizeLimit {
return nil, errors.Errorf("cm sketch size(depth * width) should not larger than %d", CMSketchSizeLimit)
}
return optMap, nil
Expand Down
22 changes: 16 additions & 6 deletions planner/core/planbuilder_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -542,7 +542,7 @@ func checkDeepClonedCore(v1, v2 reflect.Value, path string, whiteList []string,
return nil
}

func TestHandleAnalyzeOptions(t *testing.T) {
func TestHandleAnalyzeOptionsV1AndV2(t *testing.T) {
require.Equal(t, len(analyzeOptionDefault), len(analyzeOptionDefaultV2), "analyzeOptionDefault and analyzeOptionDefaultV2 should have the same length")

tests := []struct {
Expand All @@ -559,7 +559,7 @@ func TestHandleAnalyzeOptions(t *testing.T) {
Value: ast.NewValueExpr(16384+1, "", ""),
},
},
statsVer: 1,
statsVer: statistics.Version1,
ExpectedErr: "Value of analyze option TOPN should not be larger than 16384",
},
{
Expand All @@ -570,7 +570,7 @@ func TestHandleAnalyzeOptions(t *testing.T) {
Value: ast.NewValueExpr(1, "", ""),
},
},
statsVer: 1,
statsVer: statistics.Version1,
ExpectedErr: "Version 1's statistics doesn't support the SAMPLERATE option, please set tidb_analyze_version to 2",
},
{
Expand All @@ -581,7 +581,7 @@ func TestHandleAnalyzeOptions(t *testing.T) {
Value: ast.NewValueExpr(2, "", ""),
},
},
statsVer: 2,
statsVer: statistics.Version2,
ExpectedErr: "Value of analyze option SAMPLERATE should not larger than 1.000000, and should be greater than 0",
},
{
Expand All @@ -607,7 +607,7 @@ func TestHandleAnalyzeOptions(t *testing.T) {
Value: ast.NewValueExpr(0.1, "", ""),
},
},
statsVer: 2,
statsVer: statistics.Version2,
ExpectedErr: "ou can only either set the value of the sample num or set the value of the sample rate. Don't set both of them",
},
{
Expand All @@ -622,7 +622,7 @@ func TestHandleAnalyzeOptions(t *testing.T) {
Value: ast.NewValueExpr(2048, "", ""),
},
},
statsVer: 2,
statsVer: statistics.Version1,
ExpectedErr: "cm sketch size(depth * width) should not larger than 1258291",
},
}
Expand All @@ -636,6 +636,16 @@ func TestHandleAnalyzeOptions(t *testing.T) {
} else {
require.NoError(t, err)
}

if tt.statsVer == statistics.Version2 {
_, err := handleAnalyzeOptionsV2(tt.opts)
if tt.ExpectedErr != "" {
require.Error(t, err)
require.Contains(t, err.Error(), tt.ExpectedErr)
} else {
require.NoError(t, err)
}
}
})
}
}

0 comments on commit d4a50aa

Please sign in to comment.