Skip to content

Commit 51f1a82

Browse files
authored
statistics: record last gc ts to avoid huge read on stats table (#46138)
close #31778, close #45966
1 parent 789d6d0 commit 51f1a82

File tree

8 files changed

+107
-9
lines changed

8 files changed

+107
-9
lines changed

build/BUILD.bazel

Lines changed: 0 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -174,7 +174,6 @@ nogo(
174174
}) +
175175
select({
176176
"//build:without_rbe": [
177-
"//build/linter/filepermission",
178177
],
179178
"//conditions:default": [],
180179
}),

ddl/partition.go

Lines changed: 2 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -2062,6 +2062,7 @@ func (w *worker) onDropTablePartition(d *ddlCtx, t *meta.Meta, job *model.Job) (
20622062
if tblInfo.TiFlashReplica != nil {
20632063
removeTiFlashAvailablePartitionIDs(tblInfo, physicalTableIDs)
20642064
}
2065+
droppedDefs := tblInfo.Partition.DroppingDefinitions
20652066
tblInfo.Partition.DroppingDefinitions = nil
20662067
// used by ApplyDiff in updateSchemaVersion
20672068
job.CtxVars = []interface{}{physicalTableIDs}
@@ -2071,7 +2072,7 @@ func (w *worker) onDropTablePartition(d *ddlCtx, t *meta.Meta, job *model.Job) (
20712072
}
20722073
job.SchemaState = model.StateNone
20732074
job.FinishTableJob(model.JobStateDone, model.StateNone, ver, tblInfo)
2074-
asyncNotifyEvent(d, &util.Event{Tp: model.ActionDropTablePartition, TableInfo: tblInfo, PartInfo: &model.PartitionInfo{Definitions: tblInfo.Partition.Definitions}})
2075+
asyncNotifyEvent(d, &util.Event{Tp: model.ActionDropTablePartition, TableInfo: tblInfo, PartInfo: &model.PartitionInfo{Definitions: droppedDefs}})
20752076
// A background job will be created to delete old partition data.
20762077
job.Args = []interface{}{physicalTableIDs}
20772078
default:

ddl/table.go

Lines changed: 5 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -388,6 +388,11 @@ func onDropTableOrView(d *ddlCtx, t *meta.Meta, job *model.Job) (ver int64, _ er
388388
job.FinishTableJob(model.JobStateDone, model.StateNone, ver, tblInfo)
389389
startKey := tablecodec.EncodeTablePrefix(job.TableID)
390390
job.Args = append(job.Args, startKey, oldIDs, ruleIDs)
391+
if tblInfo.IsSequence() {
392+
asyncNotifyEvent(d, &util.Event{Tp: model.ActionDropSequence, TableInfo: tblInfo})
393+
} else if !tblInfo.IsView() {
394+
asyncNotifyEvent(d, &util.Event{Tp: model.ActionDropTable, TableInfo: tblInfo})
395+
}
391396
default:
392397
return ver, errors.Trace(dbterror.ErrInvalidDDLState.GenWithStackByArgs("table", tblInfo.State))
393398
}

statistics/handle/ddl.go

Lines changed: 42 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -43,6 +43,13 @@ func (h *Handle) HandleDDLEvent(t *util.Event) error {
4343
return err
4444
}
4545
}
46+
case model.ActionDropTable:
47+
ids := h.getInitStateTableIDs(t.TableInfo)
48+
for _, id := range ids {
49+
if err := h.resetTableStats2KVForDrop(id); err != nil {
50+
return err
51+
}
52+
}
4653
case model.ActionAddColumn, model.ActionModifyColumn:
4754
ids := h.getInitStateTableIDs(t.TableInfo)
4855
for _, id := range ids {
@@ -63,6 +70,11 @@ func (h *Handle) HandleDDLEvent(t *util.Event) error {
6370
return err
6471
}
6572
}
73+
for _, def := range t.PartInfo.Definitions {
74+
if err := h.resetTableStats2KVForDrop(def.ID); err != nil {
75+
return err
76+
}
77+
}
6678
case model.ActionReorganizePartition:
6779
for _, def := range t.PartInfo.Definitions {
6880
// TODO: Should we trigger analyze instead of adding 0s?
@@ -320,6 +332,36 @@ func (h *Handle) insertTableStats2KV(info *model.TableInfo, physicalID int64) (e
320332
return nil
321333
}
322334

335+
// resetTableStats2KV resets the count to 0.
336+
func (h *Handle) resetTableStats2KVForDrop(physicalID int64) (err error) {
337+
statsVer := uint64(0)
338+
defer func() {
339+
if err == nil && statsVer != 0 {
340+
h.recordHistoricalStatsMeta(physicalID, statsVer, StatsMetaHistorySourceSchemaChange)
341+
}
342+
}()
343+
h.mu.Lock()
344+
defer h.mu.Unlock()
345+
ctx := kv.WithInternalSourceType(context.Background(), kv.InternalTxnStats)
346+
exec := h.mu.ctx.(sqlexec.SQLExecutor)
347+
_, err = exec.ExecuteInternal(ctx, "begin")
348+
if err != nil {
349+
return errors.Trace(err)
350+
}
351+
defer func() {
352+
err = finishTransaction(ctx, exec, err)
353+
}()
354+
txn, err := h.mu.ctx.Txn(true)
355+
if err != nil {
356+
return errors.Trace(err)
357+
}
358+
startTS := txn.StartTS()
359+
if _, err := exec.ExecuteInternal(ctx, "update mysql.stats_meta set version=%? where table_id =%?", startTS, physicalID); err != nil {
360+
return err
361+
}
362+
return nil
363+
}
364+
323365
// insertColStats2KV insert a record to stats_histograms with distinct_count 1 and insert a bucket to stats_buckets with default value.
324366
// This operation also updates version.
325367
func (h *Handle) insertColStats2KV(physicalID int64, colInfos []*model.ColumnInfo) (err error) {

statistics/handle/ddl_test.go

Lines changed: 7 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -190,11 +190,16 @@ func TestDDLHistogram(t *testing.T) {
190190
func TestDDLPartition(t *testing.T) {
191191
store, do := testkit.CreateMockStoreAndDomain(t)
192192
testKit := testkit.NewTestKit(t, store)
193-
for _, pruneMode := range []string{"static", "dynamic"} {
193+
for i, pruneMode := range []string{"static", "dynamic"} {
194194
testKit.MustExec("set @@tidb_partition_prune_mode=`" + pruneMode + "`")
195195
testKit.MustExec("set global tidb_partition_prune_mode=`" + pruneMode + "`")
196196
testKit.MustExec("use test")
197197
testKit.MustExec("drop table if exists t")
198+
h := do.StatsHandle()
199+
if i == 1 {
200+
err := h.HandleDDLEvent(<-h.DDLEventCh())
201+
require.NoError(t, err)
202+
}
198203
createTable := `CREATE TABLE t (a int, b int, primary key(a), index idx(b))
199204
PARTITION BY RANGE ( a ) (
200205
PARTITION p0 VALUES LESS THAN (6),
@@ -207,14 +212,13 @@ PARTITION BY RANGE ( a ) (
207212
tbl, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t"))
208213
require.NoError(t, err)
209214
tableInfo := tbl.Meta()
210-
h := do.StatsHandle()
211215
err = h.HandleDDLEvent(<-h.DDLEventCh())
212216
require.NoError(t, err)
213217
require.Nil(t, h.Update(is))
214218
pi := tableInfo.GetPartitionInfo()
215219
for _, def := range pi.Definitions {
216220
statsTbl := h.GetPartitionStats(tableInfo, def.ID)
217-
require.False(t, statsTbl.Pseudo)
221+
require.False(t, statsTbl.Pseudo, "for %v", pruneMode)
218222
}
219223

220224
testKit.MustExec("insert into t values (1,2),(6,2),(11,2),(16,2)")

statistics/handle/gc.go

Lines changed: 42 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -17,6 +17,7 @@ package handle
1717
import (
1818
"context"
1919
"encoding/json"
20+
"strconv"
2021
"time"
2122

2223
"github.com/pingcap/errors"
@@ -34,9 +35,11 @@ import (
3435
"go.uber.org/zap"
3536
)
3637

38+
const gcLastTSVarName = "tidb_stats_gc_last_ts"
39+
3740
// GCStats will garbage collect the useless stats info. For dropped tables, we will first update their version so that
3841
// other tidb could know that table is deleted.
39-
func (h *Handle) GCStats(is infoschema.InfoSchema, ddlLease time.Duration) error {
42+
func (h *Handle) GCStats(is infoschema.InfoSchema, ddlLease time.Duration) (err error) {
4043
ctx := context.Background()
4144
// To make sure that all the deleted tables' schema and stats info have been acknowledged to all tidb,
4245
// we only garbage collect version before 10 lease.
@@ -47,7 +50,17 @@ func (h *Handle) GCStats(is infoschema.InfoSchema, ddlLease time.Duration) error
4750
return nil
4851
}
4952
gcVer := now - offset
50-
rows, _, err := h.execRestrictedSQL(ctx, "select table_id from mysql.stats_meta where version < %?", gcVer)
53+
lastGC, err := h.GetLastGCTimestamp(ctx)
54+
if err != nil {
55+
return err
56+
}
57+
defer func() {
58+
if err != nil {
59+
return
60+
}
61+
err = h.writeGCTimestampToKV(ctx, gcVer)
62+
}()
63+
rows, _, err := h.execRestrictedSQL(ctx, "select table_id from mysql.stats_meta where version >= %? and version < %?", lastGC, gcVer)
5164
if err != nil {
5265
return errors.Trace(err)
5366
}
@@ -70,6 +83,33 @@ func (h *Handle) GCStats(is infoschema.InfoSchema, ddlLease time.Duration) error
7083
return h.removeDeletedExtendedStats(gcVer)
7184
}
7285

86+
// GetLastGCTimestamp loads the last gc time from mysql.tidb.
87+
func (h *Handle) GetLastGCTimestamp(ctx context.Context) (uint64, error) {
88+
rows, _, err := h.execRestrictedSQL(ctx, "SELECT HIGH_PRIORITY variable_value FROM mysql.tidb WHERE variable_name=%?", gcLastTSVarName)
89+
if err != nil {
90+
return 0, errors.Trace(err)
91+
}
92+
if len(rows) == 0 {
93+
return 0, nil
94+
}
95+
lastGcTSString := rows[0].GetString(0)
96+
lastGcTS, err := strconv.ParseUint(lastGcTSString, 10, 64)
97+
if err != nil {
98+
return 0, errors.Trace(err)
99+
}
100+
return lastGcTS, nil
101+
}
102+
103+
func (h *Handle) writeGCTimestampToKV(ctx context.Context, newTS uint64) error {
104+
_, _, err := h.execRestrictedSQL(ctx,
105+
"insert into mysql.tidb (variable_name, variable_value) values (%?, %?) on duplicate key update variable_value = %?",
106+
gcLastTSVarName,
107+
newTS,
108+
newTS,
109+
)
110+
return err
111+
}
112+
73113
func (h *Handle) gcTableStats(is infoschema.InfoSchema, physicalID int64) error {
74114
ctx := context.Background()
75115
rows, _, err := h.execRestrictedSQL(ctx, "select is_index, hist_id from mysql.stats_histograms where table_id = %?", physicalID)

statistics/handle/gc_test.go

Lines changed: 3 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -103,6 +103,8 @@ func TestGCExtendedStats(t *testing.T) {
103103
testKit.MustExec("insert into t values (1,1,1),(2,2,2),(3,3,3)")
104104
testKit.MustExec("alter table t add stats_extended s1 correlation(a,b)")
105105
testKit.MustExec("alter table t add stats_extended s2 correlation(b,c)")
106+
h := dom.StatsHandle()
107+
require.Nil(t, h.HandleDDLEvent(<-h.DDLEventCh()))
106108
testKit.MustExec("analyze table t")
107109

108110
testKit.MustQuery("select name, type, column_ids, stats, status from mysql.stats_extended").Sort().Check(testkit.Rows(
@@ -114,7 +116,6 @@ func TestGCExtendedStats(t *testing.T) {
114116
"s1 2 [1,2] 1.000000 1",
115117
"s2 2 [2,3] 1.000000 1",
116118
))
117-
h := dom.StatsHandle()
118119
ddlLease := time.Duration(0)
119120
require.Nil(t, h.GCStats(dom.InfoSchema(), ddlLease))
120121
testKit.MustQuery("select name, type, column_ids, stats, status from mysql.stats_extended").Sort().Check(testkit.Rows(
@@ -130,6 +131,7 @@ func TestGCExtendedStats(t *testing.T) {
130131
testKit.MustQuery("select name, type, column_ids, stats, status from mysql.stats_extended").Sort().Check(testkit.Rows(
131132
"s2 2 [2,3] 1.000000 1",
132133
))
134+
require.Nil(t, h.HandleDDLEvent(<-h.DDLEventCh()))
133135
require.Nil(t, h.GCStats(dom.InfoSchema(), ddlLease))
134136
testKit.MustQuery("select name, type, column_ids, stats, status from mysql.stats_extended").Sort().Check(testkit.Rows(
135137
"s2 2 [2,3] 1.000000 2",

statistics/integration_test.go

Lines changed: 6 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -754,11 +754,14 @@ func TestGlobalIndexStatistics(t *testing.T) {
754754
tk := testkit.NewTestKit(t, store)
755755
tk.MustExec("use test")
756756

757-
for _, version := range []string{"1", "2"} {
757+
for i, version := range []string{"1", "2"} {
758758
tk.MustExec("set @@session.tidb_analyze_version = " + version)
759759

760760
// analyze table t
761761
tk.MustExec("drop table if exists t")
762+
if i != 0 {
763+
require.Nil(t, h.HandleDDLEvent(<-h.DDLEventCh()))
764+
}
762765
tk.MustExec("CREATE TABLE t ( a int, b int, c int default 0, key(a) )" +
763766
"PARTITION BY RANGE (a) (" +
764767
"PARTITION p0 VALUES LESS THAN (10)," +
@@ -779,6 +782,7 @@ func TestGlobalIndexStatistics(t *testing.T) {
779782

780783
// analyze table t index idx
781784
tk.MustExec("drop table if exists t")
785+
require.Nil(t, h.HandleDDLEvent(<-h.DDLEventCh()))
782786
tk.MustExec("CREATE TABLE t ( a int, b int, c int default 0, primary key(b, a) clustered )" +
783787
"PARTITION BY RANGE (a) (" +
784788
"PARTITION p0 VALUES LESS THAN (10)," +
@@ -796,6 +800,7 @@ func TestGlobalIndexStatistics(t *testing.T) {
796800

797801
// analyze table t index
798802
tk.MustExec("drop table if exists t")
803+
require.Nil(t, h.HandleDDLEvent(<-h.DDLEventCh()))
799804
tk.MustExec("CREATE TABLE t ( a int, b int, c int default 0, primary key(b, a) clustered )" +
800805
"PARTITION BY RANGE (a) (" +
801806
"PARTITION p0 VALUES LESS THAN (10)," +

0 commit comments

Comments
 (0)