Skip to content

Commit

Permalink
executor: add partition table testcase for historical stats (#40453)
Browse files Browse the repository at this point in the history
* add partition table testcase

* add partition table testcase

* fix lint

Co-authored-by: Ti Chi Robot <ti-community-prow-bot@tidb.io>
  • Loading branch information
Yisaer and ti-chi-bot committed Jan 11, 2023
1 parent d2d33df commit 668881f
Show file tree
Hide file tree
Showing 3 changed files with 75 additions and 4 deletions.
7 changes: 6 additions & 1 deletion domain/historical_stats.go
Original file line number Diff line number Diff line change
Expand Up @@ -77,5 +77,10 @@ func (w *HistoricalStatsWorker) DumpHistoricalStats(tableID int64, statsHandle *

// GetOneHistoricalStatsTable gets one tableID from channel, only used for test
func (w *HistoricalStatsWorker) GetOneHistoricalStatsTable() int64 {
return <-w.tblCH
select {
case tblID := <-w.tblCH:
return tblID
default:
return -1
}
}
63 changes: 63 additions & 0 deletions executor/historical_stats_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -26,6 +26,7 @@ import (
"github.com/pingcap/tidb/statistics/handle"
"github.com/pingcap/tidb/testkit"
"github.com/stretchr/testify/require"
"github.com/tikv/client-go/v2/oracle"
)

func TestRecordHistoryStatsAfterAnalyze(t *testing.T) {
Expand Down Expand Up @@ -243,3 +244,65 @@ PARTITION p0 VALUES LESS THAN (6)
require.NoError(t, err)
tk.MustQuery("select count(*) from mysql.stats_history").Check(testkit.Rows("2"))
}

func TestDumpHistoricalStatsByTable(t *testing.T) {
store, dom := testkit.CreateMockStoreAndDomain(t)
tk := testkit.NewTestKit(t, store)
tk.MustExec("set global tidb_enable_historical_stats = 1")
tk.MustExec("set @@tidb_partition_prune_mode='static'")
tk.MustExec("use test")
tk.MustExec("drop table if exists t")
tk.MustExec(`CREATE TABLE t (a int, b int, index idx(b))
PARTITION BY RANGE ( a ) (
PARTITION p0 VALUES LESS THAN (6)
)`)
// dump historical stats
h := dom.StatsHandle()

tk.MustExec("analyze table t")
is := dom.InfoSchema()
tbl, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t"))
require.NoError(t, err)
require.NotNil(t, tbl)

// dump historical stats
hsWorker := dom.GetHistoricalStatsWorker()
// only partition p0 stats will be dumped in static mode
tblID := hsWorker.GetOneHistoricalStatsTable()
require.NotEqual(t, tblID, -1)
err = hsWorker.DumpHistoricalStats(tblID, h)
require.NoError(t, err)
tblID = hsWorker.GetOneHistoricalStatsTable()
require.Equal(t, tblID, int64(-1))

time.Sleep(1 * time.Second)
snapshot := oracle.GoTimeToTS(time.Now())
jsTable, err := h.DumpHistoricalStatsBySnapshot("test", tbl.Meta(), snapshot)
require.NoError(t, err)
require.NotNil(t, jsTable)
// only has p0 stats
require.NotNil(t, jsTable.Partitions["p0"])
require.Nil(t, jsTable.Partitions["global"])

// change static to dynamic then assert
tk.MustExec("set @@tidb_partition_prune_mode='dynamic'")
tk.MustExec("analyze table t")
require.NoError(t, err)
// global and p0's stats will be dumped
tblID = hsWorker.GetOneHistoricalStatsTable()
require.NotEqual(t, tblID, -1)
err = hsWorker.DumpHistoricalStats(tblID, h)
require.NoError(t, err)
tblID = hsWorker.GetOneHistoricalStatsTable()
require.NotEqual(t, tblID, -1)
err = hsWorker.DumpHistoricalStats(tblID, h)
require.NoError(t, err)
time.Sleep(1 * time.Second)
snapshot = oracle.GoTimeToTS(time.Now())
jsTable, err = h.DumpHistoricalStatsBySnapshot("test", tbl.Meta(), snapshot)
require.NoError(t, err)
require.NotNil(t, jsTable)
// has both global and p0 stats
require.NotNil(t, jsTable.Partitions["p0"])
require.NotNil(t, jsTable.Partitions["global"])
}
9 changes: 6 additions & 3 deletions statistics/handle/dump.go
Original file line number Diff line number Diff line change
Expand Up @@ -32,8 +32,10 @@ import (
"github.com/pingcap/tidb/sessionctx/variable"
"github.com/pingcap/tidb/statistics"
"github.com/pingcap/tidb/types"
"github.com/pingcap/tidb/util/logutil"
"github.com/pingcap/tidb/util/sqlexec"
"github.com/pingcap/tipb/go-tipb"
"go.uber.org/zap"
)

// JSONTable is used for dumping statistics.
Expand Down Expand Up @@ -173,9 +175,10 @@ func (h *Handle) DumpHistoricalStatsBySnapshot(dbName string, tableInfo *model.T
if isDynamicMode {
tbl, err := h.tableHistoricalStatsToJSON(tableInfo.ID, snapshot)
if err != nil {
return nil, errors.Trace(err)
}
if tbl != nil {
logutil.BgLogger().Warn("dump global historical stats failed",
zap.Int64("table-id", tableInfo.ID),
zap.String("table-name", tableInfo.Name.String()))
} else if tbl != nil {
jsonTbl.Partitions["global"] = tbl
}
}
Expand Down

0 comments on commit 668881f

Please sign in to comment.