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 503e7ed commit 3f9d927
Show file tree
Hide file tree
Showing 5 changed files with 354 additions and 11 deletions.
4 changes: 2 additions & 2 deletions planner/core/stats.go
Original file line number Diff line number Diff line change
Expand Up @@ -65,7 +65,7 @@ func (p *LogicalMemTable) DeriveStats(childStats []*property.StatsInfo, selfSche
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 @@ -232,7 +232,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
148 changes: 148 additions & 0 deletions statistics/integration_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -667,3 +667,151 @@ func TestUpdateNotLoadIndexFMSketch(t *testing.T) {
require.Nil(t, h.GetPartitionStats(tblInfo, p0.ID).Indices[idxInfo.ID].FMSketch)
require.Nil(t, h.GetPartitionStats(tblInfo, p1.ID).Indices[idxInfo.ID].FMSketch)
}
<<<<<<< HEAD
=======

func TestIndexJoinInnerRowCountUpperBound(t *testing.T) {
store, dom := testkit.CreateMockStoreAndDomain(t)
testKit := testkit.NewTestKit(t, store)
h := dom.StatsHandle()

testKit.MustExec("use test")
testKit.MustExec("drop table if exists t")
testKit.MustExec("create table t(a int, b int, index idx(b))")
require.NoError(t, h.HandleDDLEvent(<-h.DDLEventCh()))
is := dom.InfoSchema()
tb, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t"))
require.NoError(t, err)
tblInfo := tb.Meta()

// Mock the stats:
// The two columns are the same.
// From 0 to 499, each value has 1000 rows. Therefore, NDV is 500 and total row count is 500000.
mockStatsTbl := mockStatsTable(tblInfo, 500000)
colValues, err := generateIntDatum(1, 500)
require.NoError(t, err)
for i := 1; i <= 2; i++ {
mockStatsTbl.Columns[int64(i)] = &statistics.Column{
Histogram: *mockStatsHistogram(int64(i), colValues, 1000, types.NewFieldType(mysql.TypeLonglong)),
Info: tblInfo.Columns[i-1],
StatsLoadedStatus: statistics.NewStatsFullLoadStatus(),
StatsVer: 2,
}
}
generateMapsForMockStatsTbl(mockStatsTbl)
stat := h.GetTableStats(tblInfo)
stat.HistColl = mockStatsTbl.HistColl

testKit.MustQuery("explain format = 'brief' " +
"select /*+ inl_join(t2) */ * from (select * from t where t.a < 1) as t1 join t t2 where t2.a = 0 and t1.a = t2.b").
Check(testkit.Rows(
"IndexJoin 1000000.00 root inner join, inner:IndexLookUp, outer key:test.t.a, inner key:test.t.b, equal cond:eq(test.t.a, test.t.b)",
"├─TableReader(Build) 1000.00 root data:Selection",
"│ └─Selection 1000.00 cop[tikv] lt(test.t.a, 1), not(isnull(test.t.a))",
"│ └─TableFullScan 500000.00 cop[tikv] table:t keep order:false, stats:pseudo",
"└─IndexLookUp(Probe) 1000000.00 root ",
" ├─Selection(Build) 500000000.00 cop[tikv] not(isnull(test.t.b))",
" │ └─IndexRangeScan 500000000.00 cop[tikv] table:t2, index:idx(b) range: decided by [eq(test.t.b, test.t.a)], keep order:false, stats:pseudo",
" └─Selection(Probe) 1000000.00 cop[tikv] eq(test.t.a, 0)",
" └─TableRowIDScan 500000000.00 cop[tikv] table:t2 keep order:false, stats:pseudo",
))
}

func TestOrderingIdxSelectivityThreshold(t *testing.T) {
store, dom := testkit.CreateMockStoreAndDomain(t)
testKit := testkit.NewTestKit(t, store)
h := dom.StatsHandle()
sc := &stmtctx.StatementContext{TimeZone: time.UTC}

testKit.MustExec("use test")
testKit.MustExec("drop table if exists t")
testKit.MustExec("create table t(a int primary key , b int, c int, index ib(b), index ic(c))")
require.NoError(t, h.HandleDDLEvent(<-h.DDLEventCh()))
is := dom.InfoSchema()
tb, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t"))
require.NoError(t, err)
tblInfo := tb.Meta()

// Mock the stats:
// total row count 100000
// column a: PK, from 0 to 100000, NDV 100000
// column b, c: from 0 to 10000, each value has 10 rows, NDV 10000
// indexes are created on (b), (c) respectively
mockStatsTbl := mockStatsTable(tblInfo, 100000)
pkColValues, err := generateIntDatum(1, 100000)
require.NoError(t, err)
mockStatsTbl.Columns[1] = &statistics.Column{
Histogram: *mockStatsHistogram(1, pkColValues, 1, types.NewFieldType(mysql.TypeLonglong)),
Info: tblInfo.Columns[0],
StatsLoadedStatus: statistics.NewStatsFullLoadStatus(),
StatsVer: 2,
}
colValues, err := generateIntDatum(1, 10000)
require.NoError(t, err)
idxValues := make([]types.Datum, 0)
for _, val := range colValues {
b, err := codec.EncodeKey(sc, nil, val)
require.NoError(t, err)
idxValues = append(idxValues, types.NewBytesDatum(b))
}

for i := 2; i <= 3; i++ {
mockStatsTbl.Columns[int64(i)] = &statistics.Column{
Histogram: *mockStatsHistogram(int64(i), colValues, 10, types.NewFieldType(mysql.TypeLonglong)),
Info: tblInfo.Columns[i-1],
StatsLoadedStatus: statistics.NewStatsFullLoadStatus(),
StatsVer: 2,
}
}
for i := 1; i <= 2; i++ {
mockStatsTbl.Indices[int64(i)] = &statistics.Index{
Histogram: *mockStatsHistogram(int64(i), idxValues, 10, types.NewFieldType(mysql.TypeBlob)),
Info: tblInfo.Indices[i-1],
StatsLoadedStatus: statistics.NewStatsFullLoadStatus(),
StatsVer: 2,
}
}
generateMapsForMockStatsTbl(mockStatsTbl)
stat := h.GetTableStats(tblInfo)
stat.HistColl = mockStatsTbl.HistColl

var (
input []string
output []struct {
Query string
Result []string
}
)
integrationSuiteData := statistics.GetIntegrationSuiteData()
integrationSuiteData.LoadTestCases(t, &input, &output)
for i := 0; i < len(input); i++ {
testdata.OnRecord(func() {
output[i].Query = input[i]
})
if !strings.HasPrefix(input[i], "explain") {
testKit.MustExec(input[i])
continue
}
testdata.OnRecord(func() {
output[i].Result = testdata.ConvertRowsToStrings(testKit.MustQuery(input[i]).Rows())
})
testKit.MustQuery(input[i]).Check(testkit.Rows(output[i].Result...))
}
}

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;")
}
>>>>>>> 282c753cfbc (statistics, planner: use the correct `IndexInfo` in `GenerateHistCollFromColumnInfo()` (#44441))
4 changes: 2 additions & 2 deletions statistics/selectivity_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -582,7 +582,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 @@ -698,7 +698,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
28 changes: 21 additions & 7 deletions statistics/table.go
Original file line number Diff line number Diff line change
Expand Up @@ -598,34 +598,48 @@ func (coll *HistColl) ID2UniqueID(columns []*expression.Column) *HistColl {
return newColl
}

<<<<<<< HEAD
// GenerateHistCollFromColumnInfo generates a new HistColl whose ColID2IdxID and IdxID2ColIDs is built from the given parameter.
func (coll *HistColl) GenerateHistCollFromColumnInfo(infos []*model.ColumnInfo, columns []*expression.Column) *HistColl {
=======
// GenerateHistCollFromColumnInfo generates a new HistColl whose ColID2IdxIDs and IdxID2ColIDs is built from the given parameter.
func (coll *HistColl) GenerateHistCollFromColumnInfo(tblInfo *model.TableInfo, columns []*expression.Column) *HistColl {
>>>>>>> 282c753cfbc (statistics, planner: use the correct `IndexInfo` in `GenerateHistCollFromColumnInfo()` (#44441))
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)
<<<<<<< HEAD
colID2IdxID := 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]
=======
colID2IdxIDs := make(map[int64][]int64)
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]
>>>>>>> 282c753cfbc (statistics, planner: use the correct `IndexInfo` in `GenerateHistCollFromColumnInfo()` (#44441))
if !ok {
break
}
Expand Down
Loading

0 comments on commit 3f9d927

Please sign in to comment.