diff --git a/planner/core/stats.go b/planner/core/stats.go index b91a9266340ed..0e466add5c7a6 100644 --- a/planner/core/stats.go +++ b/planner/core/stats.go @@ -67,7 +67,7 @@ func (p *LogicalMemTable) DeriveStats(_ []*property.StatsInfo, selfSchema *expre stats := &property.StatsInfo{ RowCount: float64(statsTable.Count), ColNDVs: make(map[int64]float64, len(p.TableInfo.Columns)), - HistColl: statsTable.GenerateHistCollFromColumnInfo(p.TableInfo.Columns, p.schema.Columns), + HistColl: statsTable.GenerateHistCollFromColumnInfo(p.TableInfo, p.schema.Columns), StatsVersion: statistics.PseudoVersion, } for _, col := range selfSchema.Columns { @@ -256,7 +256,7 @@ func (ds *DataSource) initStats(colGroups [][]*expression.Column) { tableStats := &property.StatsInfo{ RowCount: float64(ds.statisticTable.Count), ColNDVs: make(map[int64]float64, ds.schema.Len()), - HistColl: ds.statisticTable.GenerateHistCollFromColumnInfo(ds.Columns, ds.schema.Columns), + HistColl: ds.statisticTable.GenerateHistCollFromColumnInfo(ds.tableInfo, ds.schema.Columns), StatsVersion: ds.statisticTable.Version, } if ds.statisticTable.Pseudo { diff --git a/statistics/integration_test.go b/statistics/integration_test.go index 494d5204191f0..05c8ed9ac98a2 100644 --- a/statistics/integration_test.go +++ b/statistics/integration_test.go @@ -771,3 +771,19 @@ func TestIndexJoinInnerRowCountUpperBound(t *testing.T) { " └─TableRowIDScan 1000000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", )) } + +func TestIssue44369(t *testing.T) { + store, dom := testkit.CreateMockStoreAndDomain(t) + h := dom.StatsHandle() + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("create table t(a int, b int, index iab(a,b));") + require.NoError(t, h.HandleDDLEvent(<-h.DDLEventCh())) + tk.MustExec("insert into t value(1,1);") + require.NoError(t, h.DumpStatsDeltaToKV(handle.DumpAll)) + tk.MustExec("analyze table t;") + is := dom.InfoSchema() + require.NoError(t, h.Update(is)) + tk.MustExec("alter table t rename column b to bb;") + tk.MustExec("select * from t where a = 10 and bb > 20;") +} diff --git a/statistics/selectivity_test.go b/statistics/selectivity_test.go index 3cd5211fc1c8d..fca2f9261c632 100644 --- a/statistics/selectivity_test.go +++ b/statistics/selectivity_test.go @@ -574,7 +574,7 @@ func TestSelectivity(t *testing.T) { sel := p.(plannercore.LogicalPlan).Children()[0].(*plannercore.LogicalSelection) ds := sel.Children()[0].(*plannercore.DataSource) - histColl := statsTbl.GenerateHistCollFromColumnInfo(ds.Columns, ds.Schema().Columns) + histColl := statsTbl.GenerateHistCollFromColumnInfo(ds.TableInfo(), ds.Schema().Columns) ratio, _, err := histColl.Selectivity(sctx, sel.Conditions, nil) require.NoErrorf(t, err, "for %s", tt.exprs) @@ -687,7 +687,7 @@ func TestDNFCondSelectivity(t *testing.T) { sel := p.(plannercore.LogicalPlan).Children()[0].(*plannercore.LogicalSelection) ds := sel.Children()[0].(*plannercore.DataSource) - histColl := statsTbl.GenerateHistCollFromColumnInfo(ds.Columns, ds.Schema().Columns) + histColl := statsTbl.GenerateHistCollFromColumnInfo(ds.TableInfo(), ds.Schema().Columns) ratio, _, err := histColl.Selectivity(sctx, sel.Conditions, nil) require.NoErrorf(t, err, "error %v, for expr %s", err, tt) diff --git a/statistics/table.go b/statistics/table.go index d1461c290819e..15bcd6af535fc 100644 --- a/statistics/table.go +++ b/statistics/table.go @@ -845,19 +845,13 @@ func (coll *HistColl) ID2UniqueID(columns []*expression.Column) *HistColl { } // GenerateHistCollFromColumnInfo generates a new HistColl whose ColID2IdxIDs and IdxID2ColIDs is built from the given parameter. -func (coll *HistColl) GenerateHistCollFromColumnInfo(infos []*model.ColumnInfo, columns []*expression.Column) *HistColl { +func (coll *HistColl) GenerateHistCollFromColumnInfo(tblInfo *model.TableInfo, columns []*expression.Column) *HistColl { newColHistMap := make(map[int64]*Column) colInfoID2UniqueID := make(map[int64]int64, len(columns)) - colNames2UniqueID := make(map[string]int64) + idxID2idxInfo := make(map[int64]*model.IndexInfo) for _, col := range columns { colInfoID2UniqueID[col.ID] = col.UniqueID } - for _, colInfo := range infos { - uniqueID, ok := colInfoID2UniqueID[colInfo.ID] - if ok { - colNames2UniqueID[colInfo.Name.L] = uniqueID - } - } for id, colHist := range coll.Columns { uniqueID, ok := colInfoID2UniqueID[id] // Collect the statistics by the given columns. @@ -865,13 +859,20 @@ func (coll *HistColl) GenerateHistCollFromColumnInfo(infos []*model.ColumnInfo, newColHistMap[uniqueID] = colHist } } + for _, idxInfo := range tblInfo.Indices { + idxID2idxInfo[idxInfo.ID] = idxInfo + } newIdxHistMap := make(map[int64]*Index) idx2Columns := make(map[int64][]int64) colID2IdxIDs := make(map[int64][]int64) - for _, idxHist := range coll.Indices { - ids := make([]int64, 0, len(idxHist.Info.Columns)) - for _, idxCol := range idxHist.Info.Columns { - uniqueID, ok := colNames2UniqueID[idxCol.Name.L] + for id, idxHist := range coll.Indices { + idxInfo := idxID2idxInfo[id] + if idxInfo == nil { + continue + } + ids := make([]int64, 0, len(idxInfo.Columns)) + for _, idxCol := range idxInfo.Columns { + uniqueID, ok := colInfoID2UniqueID[tblInfo.Columns[idxCol.Offset].ID] if !ok { break } diff --git a/statistics/trace_test.go b/statistics/trace_test.go index 90a7e3015eab2..649d31b78ee47 100644 --- a/statistics/trace_test.go +++ b/statistics/trace_test.go @@ -123,3 +123,160 @@ func TestTraceCEPartitionTable(t *testing.T) { require.Equal(t, "t", r.TableName) } } +<<<<<<< HEAD +======= + +func TestTraceDebugSelectivity(t *testing.T) { + store, dom := testkit.CreateMockStoreAndDomain(t) + tk := testkit.NewTestKit(t, store) + statsHandle := dom.StatsHandle() + + // Make the result of v1 analyze result stable + // 1. make sure all rows are always collect as samples + originalSampleSize := executor.MaxRegionSampleSize + executor.MaxRegionSampleSize = 10000 + defer func() { + executor.MaxRegionSampleSize = originalSampleSize + }() + // 2. make the order of samples for building TopN stable + // (the earlier TopN entry will modify the CMSketch, therefore influence later TopN entry's row count, + // see (*SampleCollector).ExtractTopN() for details) + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/statistics/StabilizeV1AnalyzeTopN", `return(true)`)) + defer func() { + require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/statistics/StabilizeV1AnalyzeTopN")) + }() + + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("create table t(a int, b int, index iab(a, b), index ib(b))") + require.NoError(t, statsHandle.HandleDDLEvent(<-statsHandle.DDLEventCh())) + + // Prepare the data. + + // For column a, from -1000 to 999, each value appears 1 time, + // but if it's dividable by 100, make this value appear 50 times. + // For column b, it's always a+500. + start := -1000 + for i := 0; i < 2000; i += 50 { + sql := "insert into t values " + // 50 rows as a batch + values := make([]string, 0, 50) + for j := 0; j < 50; j++ { + values = append(values, fmt.Sprintf("(%d,%d)", start+i+j, start+i+j+500)) + } + sql = sql + strings.Join(values, ",") + tk.MustExec(sql) + + if i%100 == 0 { + sql := "insert into t values " + topNValue := fmt.Sprintf("(%d,%d) ,", start+i, start+i+500) + sql = sql + strings.Repeat(topNValue, 49) + sql = sql[0 : len(sql)-1] + tk.MustExec(sql) + } + } + require.Nil(t, statsHandle.DumpStatsDeltaToKV(handle.DumpAll)) + tk.MustExec("analyze table t with 1 samplerate, 20 topn") + require.Nil(t, statsHandle.Update(dom.InfoSchema())) + // Add 100 modify count + sql := "insert into t values " + topNValue := fmt.Sprintf("(%d,%d) ,", 5000, 5000) + sql = sql + strings.Repeat(topNValue, 100) + sql = sql[0 : len(sql)-1] + tk.MustExec(sql) + require.Nil(t, statsHandle.DumpStatsDeltaToKV(handle.DumpAll)) + require.Nil(t, statsHandle.Update(dom.InfoSchema())) + + var ( + in []string + out []struct { + ResultForV1 interface{} + ResultForV2 interface{} + } + ) + traceSuiteData := statistics.GetTraceSuiteData() + traceSuiteData.LoadTestCases(t, &in, &out) + + // Trigger loading needed statistics. + for _, tt := range in { + sql := "explain " + tt + tk.MustExec(sql) + } + err := statsHandle.LoadNeededHistograms() + require.NoError(t, err) + + sctx := tk.Session().(sessionctx.Context) + tb, err := dom.InfoSchema().TableByName(model.NewCIStr("test"), model.NewCIStr("t")) + require.NoError(t, err) + tblInfo := tb.Meta() + statsTbl := statsHandle.GetTableStats(tblInfo) + stmtCtx := sctx.GetSessionVars().StmtCtx + stmtCtx.EnableOptimizerDebugTrace = true + + // Collect common information for the following tests. + p := parser.New() + dsSchemaCols := make([][]*expression.Column, 0, len(in)) + selConditions := make([][]expression.Expression, 0, len(in)) + tblInfos := make([]*model.TableInfo, 0, len(in)) + for _, sql := range in { + stmt, err := p.ParseOneStmt(sql, "", "") + require.NoError(t, err) + ret := &plannercore.PreprocessorReturn{} + err = plannercore.Preprocess(context.Background(), sctx, stmt, plannercore.WithPreprocessorReturn(ret)) + require.NoError(t, err) + p, _, err := plannercore.BuildLogicalPlanForTest(context.Background(), sctx, stmt, ret.InfoSchema) + require.NoError(t, err) + + sel := p.(plannercore.LogicalPlan).Children()[0].(*plannercore.LogicalSelection) + ds := sel.Children()[0].(*plannercore.DataSource) + + dsSchemaCols = append(dsSchemaCols, ds.Schema().Columns) + selConditions = append(selConditions, sel.Conditions) + tblInfos = append(tblInfos, ds.TableInfo()) + } + var buf bytes.Buffer + encoder := json.NewEncoder(&buf) + encoder.SetEscapeHTML(false) + + // Test using ver2 stats. + for i, sql := range in { + stmtCtx.OptimizerDebugTrace = nil + histColl := statsTbl.GenerateHistCollFromColumnInfo(tblInfos[i], dsSchemaCols[i]) + _, _, err = histColl.Selectivity(sctx, selConditions[i], nil) + require.NoError(t, err, sql, "For ver2") + traceInfo := stmtCtx.OptimizerDebugTrace + buf.Reset() + require.NoError(t, encoder.Encode(traceInfo), sql, "For ver2") + var res interface{} + require.NoError(t, json.Unmarshal(buf.Bytes(), &res), sql, "For ver2") + testdata.OnRecord(func() { + out[i].ResultForV2 = res + }) + require.Equal(t, out[i].ResultForV2, res, sql, "For ver2") + } + + tk.MustExec("set tidb_analyze_version = 1") + tk.MustExec("analyze table t with 20 topn") + require.Nil(t, statsHandle.Update(dom.InfoSchema())) + statsTbl = statsHandle.GetTableStats(tblInfo) + + // Test using ver1 stats. + stmtCtx = sctx.GetSessionVars().StmtCtx + stmtCtx.EnableOptimizerDebugTrace = true + for i, sql := range in { + stmtCtx.OptimizerDebugTrace = nil + histColl := statsTbl.GenerateHistCollFromColumnInfo(tblInfos[i], dsSchemaCols[i]) + _, _, err = histColl.Selectivity(sctx, selConditions[i], nil) + require.NoError(t, err, sql, "For ver1") + traceInfo := stmtCtx.OptimizerDebugTrace + buf.Reset() + require.NoError(t, encoder.Encode(traceInfo), sql, "For ver1") + var res interface{} + require.NoError(t, json.Unmarshal(buf.Bytes(), &res), sql, "For ver1") + testdata.OnRecord(func() { + out[i].ResultForV1 = res + }) + require.Equal(t, out[i].ResultForV1, res, sql, "For ver1") + } +} +>>>>>>> 282c753cfbc (statistics, planner: use the correct `IndexInfo` in `GenerateHistCollFromColumnInfo()` (#44441))