diff --git a/pkg/statistics/handle/storage/BUILD.bazel b/pkg/statistics/handle/storage/BUILD.bazel index 5dffed12cb20f..72412b217a56d 100644 --- a/pkg/statistics/handle/storage/BUILD.bazel +++ b/pkg/statistics/handle/storage/BUILD.bazel @@ -59,7 +59,7 @@ go_test( "stats_read_writer_test.go", ], flaky = True, - shard_count = 23, + shard_count = 24, deps = [ ":storage", "//pkg/domain", @@ -76,6 +76,7 @@ go_test( "//pkg/testkit/analyzehelper", "//pkg/types", "//pkg/util", + "@com_github_pingcap_failpoint//:failpoint", "@com_github_stretchr_testify//require", ], ) diff --git a/pkg/statistics/handle/storage/gc.go b/pkg/statistics/handle/storage/gc.go index 0d12327857952..9b42a224ff02e 100644 --- a/pkg/statistics/handle/storage/gc.go +++ b/pkg/statistics/handle/storage/gc.go @@ -22,6 +22,7 @@ import ( "time" "github.com/pingcap/errors" + "github.com/pingcap/failpoint" "github.com/pingcap/tidb/pkg/infoschema" "github.com/pingcap/tidb/pkg/parser/terror" "github.com/pingcap/tidb/pkg/sessionctx" @@ -91,6 +92,10 @@ func GCStats( return nil } + failpoint.Inject("injectGCStatsLastTSOffset", func(val failpoint.Value) { + offset = uint64(val.(int)) + }) + // Get the last gc time. gcVer := now - offset lastGC, err := getLastGCTimestamp(sctx) @@ -266,29 +271,36 @@ func removeDeletedExtendedStats(sctx sessionctx.Context, version uint64) (err er } // gcTableStats GC this table's stats. +// The GC of a table will be a two-phase process: +// 1. Delete the column/index's stats from storage. Then other TiDB nodes will be aware that those stats are deleted. +// 2. Then delete the record in stats_meta. func gcTableStats(sctx sessionctx.Context, statsHandler types.StatsHandle, is infoschema.InfoSchema, physicalID int64) error { + tbl, ok := statsHandler.TableInfoByID(is, physicalID) rows, _, err := util.ExecRows(sctx, "select is_index, hist_id from mysql.stats_histograms where table_id = %?", physicalID) if err != nil { return errors.Trace(err) } - // The table has already been deleted in stats and acknowledged to all tidb, - // we can safely remove the meta info now. - if len(rows) == 0 { + if !ok { + if len(rows) > 0 { + // It's the first time to run into it. Delete column/index stats to notify other TiDB nodes. + logutil.BgLogger().Info("remove stats in GC due to dropped table", zap.Int64("tableID", physicalID)) + return util.WrapTxn(sctx, func(sctx sessionctx.Context) error { + return errors.Trace(DeleteTableStatsFromKV(sctx, []int64{physicalID})) + }) + } + // len(rows) == 0 => The table's stats is empty. + // The table has already been deleted in stats and acknowledged to all tidb, + // We can safely remove the meta info now. _, _, err = util.ExecRows(sctx, "delete from mysql.stats_meta where table_id = %?", physicalID) if err != nil { return errors.Trace(err) } cache.TableRowStatsCache.Invalidate(physicalID) + return nil } - tbl, ok := statsHandler.TableInfoByID(is, physicalID) - if !ok { - logutil.BgLogger().Info("remove stats in GC due to dropped table", zap.Int64("table_id", physicalID)) - return util.WrapTxn(sctx, func(sctx sessionctx.Context) error { - return errors.Trace(DeleteTableStatsFromKV(sctx, []int64{physicalID})) - }) - } + tblInfo := tbl.Meta() for _, row := range rows { isIndex, histID := row.GetInt64(0), row.GetInt64(1) diff --git a/pkg/statistics/handle/storage/gc_test.go b/pkg/statistics/handle/storage/gc_test.go index 42dd3d39a1137..3c480100e54fb 100644 --- a/pkg/statistics/handle/storage/gc_test.go +++ b/pkg/statistics/handle/storage/gc_test.go @@ -15,9 +15,12 @@ package storage_test import ( + "context" "testing" "time" + "github.com/pingcap/failpoint" + "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/sessionctx/variable" "github.com/pingcap/tidb/pkg/testkit" "github.com/pingcap/tidb/pkg/testkit/analyzehelper" @@ -174,3 +177,27 @@ func TestDeleteAnalyzeJobs(t *testing.T) { rows = testKit.MustQuery("show analyze status").Rows() require.Equal(t, 0, len(rows)) } + +func TestExtremCaseOfGC(t *testing.T) { + // This case tests that there's no records in mysql.stats_histograms but this table is not deleted in fact. + // We should not delete the record in mysql.stats_meta. + store, dom := testkit.CreateMockStoreAndDomain(t) + testKit := testkit.NewTestKit(t, store) + testKit.MustExec("use test") + testKit.MustExec("create table t(a int, b int)") + testKit.MustExec("insert into t values (1,2),(3,4)") + testKit.MustExec("analyze table t") + tbl, err := dom.InfoSchema().TableByName(context.TODO(), model.NewCIStr("test"), model.NewCIStr("t")) + require.NoError(t, err) + tid := tbl.Meta().ID + rs := testKit.MustQuery("select * from mysql.stats_meta where table_id = ?", tid) + require.Len(t, rs.Rows(), 1) + rs = testKit.MustQuery("select * from mysql.stats_histograms where table_id = ?", tid) + require.Len(t, rs.Rows(), 0) + h := dom.StatsHandle() + failpoint.Enable("github.com/pingcap/tidb/pkg/statistics/handle/storage/injectGCStatsLastTSOffset", `return(0)`) + h.GCStats(dom.InfoSchema(), time.Second*3) + rs = testKit.MustQuery("select * from mysql.stats_meta where table_id = ?", tid) + require.Len(t, rs.Rows(), 1) + failpoint.Disable("github.com/pingcap/tidb/pkg/statistics/handle/storage/injectGCStatsLastTSOffset") +}