Skip to content

Commit

Permalink
update
Browse files Browse the repository at this point in the history
  • Loading branch information
hawkingrei committed Jul 10, 2024
1 parent 9b95889 commit 35a2237
Show file tree
Hide file tree
Showing 6 changed files with 56 additions and 40 deletions.
32 changes: 17 additions & 15 deletions pkg/planner/cardinality/selectivity_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -1313,31 +1313,33 @@ func TestBuiltinInEstWithoutStats(t *testing.T) {
h := dom.StatsHandle()

tk.MustExec("use test")
tk.MustExec("create table t(a int)")
tk.MustExec("create table t(a int, b int)")
require.NoError(t, h.HandleDDLEvent(<-h.DDLEventCh()))
tk.MustExec("insert into t values(1), (2), (3), (4), (5), (6), (7), (8), (9), (10)")
tk.MustExec("insert into t values(1,1), (2,2), (3,3), (4,4), (5,5), (6,6), (7,7), (8,8), (9,9), (10,10)")
require.NoError(t, h.DumpStatsDeltaToKV(true))
is := dom.InfoSchema()
require.NoError(t, h.Update(is))

tk.MustQuery("explain format='brief' select * from t where a in (1, 2, 3, 4, 5, 6, 7, 8)").Check(testkit.Rows(
expectedA := testkit.Rows(
"TableReader 0.08 root data:Selection",
"└─Selection 0.08 cop[tikv] in(test.t.a, 1, 2, 3, 4, 5, 6, 7, 8)",
" └─TableFullScan 10.00 cop[tikv] table:t keep order:false, stats:pseudo",
))
)
expectedB := testkit.Rows(
"TableReader 0.08 root data:Selection",
"└─Selection 0.08 cop[tikv] in(test.t.b, 1, 2, 3, 4, 5, 6, 7, 8)",
" └─TableFullScan 10.00 cop[tikv] table:t keep order:false, stats:pseudo",
)
tk.MustQuery("explain format='brief' select * from t where a in (1, 2, 3, 4, 5, 6, 7, 8)").Check(expectedA)
// try again with other column
tk.MustQuery("explain format='brief' select * from t where b in (1, 2, 3, 4, 5, 6, 7, 8)").Check(expectedB)

h.Clear()
require.NoError(t, h.InitStatsLite(is))
tk.MustQuery("explain format='brief' select * from t where a in (1, 2, 3, 4, 5, 6, 7, 8)").Check(testkit.Rows(
"TableReader 0.08 root data:Selection",
"└─Selection 0.08 cop[tikv] in(test.t.a, 1, 2, 3, 4, 5, 6, 7, 8)",
" └─TableFullScan 10.00 cop[tikv] table:t keep order:false, stats:pseudo",
))
tk.MustQuery("explain format='brief' select * from t where a in (1, 2, 3, 4, 5, 6, 7, 8)").Check(expectedA)
tk.MustQuery("explain format='brief' select * from t where b in (1, 2, 3, 4, 5, 6, 7, 8)").Check(expectedB)

h.Clear()
require.NoError(t, h.InitStats(is))
tk.MustQuery("explain format='brief' select * from t where a in (1, 2, 3, 4, 5, 6, 7, 8)").Check(testkit.Rows(
"TableReader 0.08 root data:Selection",
"└─Selection 0.08 cop[tikv] in(test.t.a, 1, 2, 3, 4, 5, 6, 7, 8)",
" └─TableFullScan 10.00 cop[tikv] table:t keep order:false, stats:pseudo",
))
tk.MustQuery("explain format='brief' select * from t where a in (1, 2, 3, 4, 5, 6, 7, 8)").Check(expectedA)
tk.MustQuery("explain format='brief' select * from t where b in (1, 2, 3, 4, 5, 6, 7, 8)").Check(expectedB)
}
6 changes: 0 additions & 6 deletions pkg/statistics/handle/bootstrap.go
Original file line number Diff line number Diff line change
Expand Up @@ -194,12 +194,6 @@ func (h *Handle) initStatsHistograms4Chunk(is infoschema.InfoSchema, cache stats
id, ndv, nullCount, version, totColSize := row.GetInt64(2), row.GetInt64(3), row.GetInt64(5), row.GetUint64(4), row.GetInt64(7)
lastAnalyzePos := row.GetDatum(11, types.NewFieldType(mysql.TypeBlob))
tbl, _ := h.TableInfoByID(is, table.PhysicalID)
for _, col := range tbl.Meta().Columns {
table.ColAndIdxExistenceMap.InsertCol(col.ID, col, statsVer != statistics.Version0)
}
for _, col := range tbl.Meta().Indices {
table.ColAndIdxExistenceMap.InsertIndex(col.ID, col, statsVer != statistics.Version0)
}
if row.GetInt64(1) > 0 {
var idxInfo *model.IndexInfo
for _, idx := range tbl.Meta().Indices {
Expand Down
32 changes: 25 additions & 7 deletions pkg/statistics/handle/storage/read.go
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,7 @@ import (
"github.com/pingcap/errors"
"github.com/pingcap/failpoint"
"github.com/pingcap/tidb/pkg/config"
"github.com/pingcap/tidb/pkg/infoschema"
"github.com/pingcap/tidb/pkg/kv"
"github.com/pingcap/tidb/pkg/parser/ast"
"github.com/pingcap/tidb/pkg/parser/model"
Expand Down Expand Up @@ -559,14 +560,14 @@ func LoadHistogram(sctx sessionctx.Context, tableID int64, isIndex int, histID i
}

// LoadNeededHistograms will load histograms for those needed columns/indices.
func LoadNeededHistograms(sctx sessionctx.Context, statsCache statstypes.StatsCache, loadFMSketch bool) (err error) {
func LoadNeededHistograms(sctx sessionctx.Context, statsHandle statstypes.StatsHandle, loadFMSketch bool) (err error) {
items := asyncload.AsyncLoadHistogramNeededItems.AllItems()
for _, item := range items {
if !item.IsIndex {
err = loadNeededColumnHistograms(sctx, statsCache, item.TableItemID, loadFMSketch, item.FullLoad)
err = loadNeededColumnHistograms(sctx, statsHandle, item.TableItemID, loadFMSketch, item.FullLoad)
} else {
// Index is always full load.
err = loadNeededIndexHistograms(sctx, statsCache, item.TableItemID, loadFMSketch)
err = loadNeededIndexHistograms(sctx, statsHandle, item.TableItemID, loadFMSketch)
}
if err != nil {
return err
Expand Down Expand Up @@ -602,8 +603,8 @@ func CleanFakeItemsForShowHistInFlights(statsCache statstypes.StatsCache) int {
return reallyNeeded
}

func loadNeededColumnHistograms(sctx sessionctx.Context, statsCache statstypes.StatsCache, col model.TableItemID, loadFMSketch bool, fullLoad bool) (err error) {
tbl, ok := statsCache.Get(col.TableID)
func loadNeededColumnHistograms(sctx sessionctx.Context, statsHandle statstypes.StatsHandle, col model.TableItemID, loadFMSketch bool, fullLoad bool) (err error) {
tbl, ok := statsHandle.Get(col.TableID)
if !ok {
return nil
}
Expand All @@ -613,7 +614,19 @@ func loadNeededColumnHistograms(sctx sessionctx.Context, statsCache statstypes.S
asyncload.AsyncLoadHistogramNeededItems.Delete(col)
return nil
}
isUpdateColAndIdxExistenceMap := false
colInfo = tbl.ColAndIdxExistenceMap.GetCol(col.ID)
if colInfo == nil {
// Now, we cannot init the column info in the ColAndIdxExistenceMap when to disable lite-init-stats.
// so we have to get the column info from the domain.
is := sctx.GetDomainInfoSchema().(infoschema.InfoSchema)
tblInfo, ok := statsHandle.TableInfoByID(is, col.TableID)
if !ok {
return nil
}
colInfo = tblInfo.Meta().GetColumnByID(col.ID)
isUpdateColAndIdxExistenceMap = true
}
hg, _, statsVer, _, err := HistMetaFromStorageWithHighPriority(sctx, &col, colInfo)
if hg == nil || err != nil {
asyncload.AsyncLoadHistogramNeededItems.Delete(col)
Expand Down Expand Up @@ -652,7 +665,7 @@ func loadNeededColumnHistograms(sctx sessionctx.Context, statsCache statstypes.S
}
// Reload the latest stats cache, otherwise the `updateStatsCache` may fail with high probability, because functions
// like `GetPartitionStats` called in `fmSketchFromStorage` would have modified the stats cache already.
tbl, ok = statsCache.Get(col.TableID)
tbl, ok = statsHandle.Get(col.TableID)
if !ok {
return nil
}
Expand All @@ -667,9 +680,14 @@ func loadNeededColumnHistograms(sctx sessionctx.Context, statsCache statstypes.S
if statsVer != statistics.Version0 {
tbl.StatsVer = int(statsVer)
}
if isUpdateColAndIdxExistenceMap {
tbl.ColAndIdxExistenceMap.InsertCol(col.ID, colInfo, true)
}
} else if isUpdateColAndIdxExistenceMap {
tbl.ColAndIdxExistenceMap.InsertCol(col.ID, colInfo, false)
}
tbl.SetCol(col.ID, colHist)
statsCache.UpdateStatsCache([]*statistics.Table{tbl}, nil)
statsHandle.UpdateStatsCache([]*statistics.Table{tbl}, nil)
asyncload.AsyncLoadHistogramNeededItems.Delete(col)
if col.IsSyncLoadFailed {
logutil.BgLogger().Warn("Hist for column should already be loaded as sync but not found.",
Expand Down
7 changes: 5 additions & 2 deletions pkg/statistics/handle/syncload/stats_syncload.go
Original file line number Diff line number Diff line change
Expand Up @@ -323,12 +323,13 @@ func (s *statsSyncLoad) handleOneItemTask(task *statstypes.NeededItemTask) (err
if col != nil {
wrapper.colInfo = col.Info
} else if colInfo := tbl.ColAndIdxExistenceMap.GetCol(item.ID); colInfo != nil {
wrapper.colInfo = tbl.ColAndIdxExistenceMap.GetCol(item.ID)
wrapper.colInfo = colInfo
} else {
// Now, we cannot init the column info in the ColAndIdxExistenceMap when to disable lite-init-stats.
// so we have to get the column info from the domain.
is := sctx.GetDomainInfoSchema().(infoschema.InfoSchema)
tblInfo, ok := s.statsHandle.TableInfoByID(is, item.TableID)
if !ok {
fmt.Println("fuck")
return nil
}
wrapper.colInfo = tblInfo.Meta().GetColumnByID(item.ID)
Expand Down Expand Up @@ -557,6 +558,8 @@ func (s *statsSyncLoad) updateCachedItem(item model.TableItemID, colHist *statis
// If the column is analyzed we refresh the map for the possible change.
if colHist.StatsAvailable() {
tbl.ColAndIdxExistenceMap.InsertCol(item.ID, colHist.Info, true)
} else {
tbl.ColAndIdxExistenceMap.InsertCol(item.ID, colHist.Info, false)
}
// All the objects shares the same stats version. Update it here.
if colHist.StatsVer != statistics.Version0 {
Expand Down
17 changes: 8 additions & 9 deletions pkg/statistics/table.go
Original file line number Diff line number Diff line change
Expand Up @@ -787,23 +787,22 @@ func (t *Table) ColumnIsLoadNeeded(id int64, fullLoad bool) (*Column, bool, bool
if t.Pseudo {
return nil, false, false
}
if len(t.columns) == 0 {
// when we use non-lite init stats, it cannot init the stats for common columns.
// so we need to foce to load the stats.
col, ok := t.columns[id]
if !ok {
return nil, true, true
}
col, ok := t.columns[id]
hasAnalyzed := t.ColAndIdxExistenceMap.HasAnalyzed(id, false)

// If it's not analyzed yet.
if !hasAnalyzed {
// If we don't have it in memory, we create a fake hist for pseudo estimation (see handleOneItemTask()).
if !ok {
// If we don't have this column. We skip it.
// It's something ridiculous. But it's possible that the stats don't have some ColumnInfo.
// We need to find a way to maintain it more correctly.
return nil, false, true
}
// If we don't have this column. We skip it.
// It's something ridiculous. But it's possible that the stats don't have some ColumnInfo.
// We need to find a way to maintain it more correctly.
// Otherwise we don't need to load it.
return nil, false, false
return nil, t.ColAndIdxExistenceMap.Has(id, false), false
}

// Restore the condition from the simplified form:
Expand Down
2 changes: 1 addition & 1 deletion tests/integrationtest/r/executor/show.result
Original file line number Diff line number Diff line change
Expand Up @@ -147,7 +147,7 @@ t CREATE TABLE `t` (
) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin /*T![ttl] TTL=`created_at` + INTERVAL 100 YEAR */ /*T![ttl] TTL_ENABLE='ON' */ /*T![ttl] TTL_JOB_INTERVAL='1d' */
show histograms_in_flight;
HistogramsInFlight
0
2
show open tables;
Database Table In_use Name_locked
show open tables in executor__show;
Expand Down

0 comments on commit 35a2237

Please sign in to comment.