Skip to content

Commit

Permalink
This is an automated cherry-pick of pingcap#44441
Browse files Browse the repository at this point in the history
Signed-off-by: ti-chi-bot <ti-community-prow-bot@tidb.io>
  • Loading branch information
time-and-fate authored and ti-chi-bot committed Sep 6, 2023
1 parent 4350f8d commit 32419dd
Show file tree
Hide file tree
Showing 5 changed files with 190 additions and 16 deletions.
4 changes: 2 additions & 2 deletions planner/core/stats.go
Original file line number Diff line number Diff line change
Expand Up @@ -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 {
Expand Down Expand Up @@ -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 {
Expand Down
16 changes: 16 additions & 0 deletions statistics/integration_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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;")
}
4 changes: 2 additions & 2 deletions statistics/selectivity_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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)
Expand Down Expand Up @@ -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)
Expand Down
25 changes: 13 additions & 12 deletions statistics/table.go
Original file line number Diff line number Diff line change
Expand Up @@ -845,33 +845,34 @@ 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.
if ok {
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
}
Expand Down
157 changes: 157 additions & 0 deletions statistics/trace_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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))

0 comments on commit 32419dd

Please sign in to comment.