Skip to content

Commit e94d8dc

Browse files
authored
statistics: GCStats should not wrongly remove record of an existing table (#58108) (#58796)
close #57983
1 parent b313166 commit e94d8dc

File tree

3 files changed

+51
-11
lines changed

3 files changed

+51
-11
lines changed

pkg/statistics/handle/storage/BUILD.bazel

Lines changed: 2 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -58,7 +58,7 @@ go_test(
5858
"stats_read_writer_test.go",
5959
],
6060
flaky = True,
61-
shard_count = 23,
61+
shard_count = 24,
6262
deps = [
6363
":storage",
6464
"//pkg/domain",
@@ -75,6 +75,7 @@ go_test(
7575
"//pkg/testkit/analyzehelper",
7676
"//pkg/types",
7777
"//pkg/util",
78+
"@com_github_pingcap_failpoint//:failpoint",
7879
"@com_github_stretchr_testify//require",
7980
],
8081
)

pkg/statistics/handle/storage/gc.go

Lines changed: 22 additions & 10 deletions
Original file line numberDiff line numberDiff line change
@@ -22,6 +22,7 @@ import (
2222
"time"
2323

2424
"github.com/pingcap/errors"
25+
"github.com/pingcap/failpoint"
2526
"github.com/pingcap/tidb/pkg/infoschema"
2627
"github.com/pingcap/tidb/pkg/parser/terror"
2728
"github.com/pingcap/tidb/pkg/sessionctx"
@@ -91,6 +92,10 @@ func GCStats(
9192
return nil
9293
}
9394

95+
failpoint.Inject("injectGCStatsLastTSOffset", func(val failpoint.Value) {
96+
offset = uint64(val.(int))
97+
})
98+
9499
// Get the last gc time.
95100
gcVer := now - offset
96101
lastGC, err := getLastGCTimestamp(sctx)
@@ -266,29 +271,36 @@ func removeDeletedExtendedStats(sctx sessionctx.Context, version uint64) (err er
266271
}
267272

268273
// gcTableStats GC this table's stats.
274+
// The GC of a table will be a two-phase process:
275+
// 1. Delete the column/index's stats from storage. Then other TiDB nodes will be aware that those stats are deleted.
276+
// 2. Then delete the record in stats_meta.
269277
func gcTableStats(sctx sessionctx.Context,
270278
statsHandler types.StatsHandle,
271279
is infoschema.InfoSchema, physicalID int64) error {
280+
tbl, ok := statsHandler.TableInfoByID(is, physicalID)
272281
rows, _, err := util.ExecRows(sctx, "select is_index, hist_id from mysql.stats_histograms where table_id = %?", physicalID)
273282
if err != nil {
274283
return errors.Trace(err)
275284
}
276-
// The table has already been deleted in stats and acknowledged to all tidb,
277-
// we can safely remove the meta info now.
278-
if len(rows) == 0 {
285+
if !ok {
286+
if len(rows) > 0 {
287+
// It's the first time to run into it. Delete column/index stats to notify other TiDB nodes.
288+
logutil.BgLogger().Info("remove stats in GC due to dropped table", zap.Int64("tableID", physicalID))
289+
return util.WrapTxn(sctx, func(sctx sessionctx.Context) error {
290+
return errors.Trace(DeleteTableStatsFromKV(sctx, []int64{physicalID}))
291+
})
292+
}
293+
// len(rows) == 0 => The table's stats is empty.
294+
// The table has already been deleted in stats and acknowledged to all tidb,
295+
// We can safely remove the meta info now.
279296
_, _, err = util.ExecRows(sctx, "delete from mysql.stats_meta where table_id = %?", physicalID)
280297
if err != nil {
281298
return errors.Trace(err)
282299
}
283300
cache.TableRowStatsCache.Invalidate(physicalID)
301+
return nil
284302
}
285-
tbl, ok := statsHandler.TableInfoByID(is, physicalID)
286-
if !ok {
287-
logutil.BgLogger().Info("remove stats in GC due to dropped table", zap.Int64("table_id", physicalID))
288-
return util.WrapTxn(sctx, func(sctx sessionctx.Context) error {
289-
return errors.Trace(DeleteTableStatsFromKV(sctx, []int64{physicalID}))
290-
})
291-
}
303+
292304
tblInfo := tbl.Meta()
293305
for _, row := range rows {
294306
isIndex, histID := row.GetInt64(0), row.GetInt64(1)

pkg/statistics/handle/storage/gc_test.go

Lines changed: 27 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -15,9 +15,12 @@
1515
package storage_test
1616

1717
import (
18+
"context"
1819
"testing"
1920
"time"
2021

22+
"github.com/pingcap/failpoint"
23+
"github.com/pingcap/tidb/pkg/parser/model"
2124
"github.com/pingcap/tidb/pkg/sessionctx/variable"
2225
statstestutil "github.com/pingcap/tidb/pkg/statistics/handle/ddl/testutil"
2326
"github.com/pingcap/tidb/pkg/testkit"
@@ -177,3 +180,27 @@ func TestDeleteAnalyzeJobs(t *testing.T) {
177180
rows = testKit.MustQuery("show analyze status").Rows()
178181
require.Equal(t, 0, len(rows))
179182
}
183+
184+
func TestExtremCaseOfGC(t *testing.T) {
185+
// This case tests that there's no records in mysql.stats_histograms but this table is not deleted in fact.
186+
// We should not delete the record in mysql.stats_meta.
187+
store, dom := testkit.CreateMockStoreAndDomain(t)
188+
testKit := testkit.NewTestKit(t, store)
189+
testKit.MustExec("use test")
190+
testKit.MustExec("create table t(a int, b int)")
191+
testKit.MustExec("insert into t values (1,2),(3,4)")
192+
testKit.MustExec("analyze table t")
193+
tbl, err := dom.InfoSchema().TableByName(context.TODO(), model.NewCIStr("test"), model.NewCIStr("t"))
194+
require.NoError(t, err)
195+
tid := tbl.Meta().ID
196+
rs := testKit.MustQuery("select * from mysql.stats_meta where table_id = ?", tid)
197+
require.Len(t, rs.Rows(), 1)
198+
rs = testKit.MustQuery("select * from mysql.stats_histograms where table_id = ?", tid)
199+
require.Len(t, rs.Rows(), 0)
200+
h := dom.StatsHandle()
201+
failpoint.Enable("github.com/pingcap/tidb/pkg/statistics/handle/storage/injectGCStatsLastTSOffset", `return(0)`)
202+
h.GCStats(dom.InfoSchema(), time.Second*3)
203+
rs = testKit.MustQuery("select * from mysql.stats_meta where table_id = ?", tid)
204+
require.Len(t, rs.Rows(), 1)
205+
failpoint.Disable("github.com/pingcap/tidb/pkg/statistics/handle/storage/injectGCStatsLastTSOffset")
206+
}

0 commit comments

Comments
 (0)