Skip to content

Commit 1c856d8

Browse files
committed
statistics: use DDL subscriber updating stats meta
Signed-off-by: Rustin170506 <[email protected]>
1 parent 6d74071 commit 1c856d8

File tree

5 files changed

+19
-202
lines changed

5 files changed

+19
-202
lines changed

pkg/domain/domain.go

Lines changed: 0 additions & 21 deletions
Original file line numberDiff line numberDiff line change
@@ -2398,9 +2398,6 @@ func (do *Domain) UpdateTableStatsLoop(ctx, initStatsCtx sessionctx.Context) err
23982398
// This is because the updated worker's primary responsibilities are to update the change delta and handle DDL operations.
23992399
// These tasks do not interfere with or depend on the initialization process.
24002400
do.wg.Run(func() { do.updateStatsWorker(ctx) }, "updateStatsWorker")
2401-
do.wg.Run(func() {
2402-
do.handleDDLEvent()
2403-
}, "handleDDLEvent")
24042401
// Wait for the stats worker to finish the initialization.
24052402
// Otherwise, we may start the auto analyze worker before the stats cache is initialized.
24062403
do.wg.Run(
@@ -2599,24 +2596,6 @@ func (do *Domain) updateStatsWorkerExitPreprocessing(statsHandle *handle.Handle)
25992596
}
26002597
}
26012598

2602-
func (do *Domain) handleDDLEvent() {
2603-
logutil.BgLogger().Info("handleDDLEvent started.")
2604-
defer util.Recover(metrics.LabelDomain, "handleDDLEvent", nil, false)
2605-
statsHandle := do.StatsHandle()
2606-
for {
2607-
select {
2608-
case <-do.exit:
2609-
return
2610-
// This channel is sent only by ddl owner.
2611-
case t := <-statsHandle.DDLEventCh():
2612-
err := statsHandle.HandleDDLEvent(t)
2613-
if err != nil {
2614-
logutil.BgLogger().Error("handle ddl event failed", zap.String("event", t.String()), zap.Error(err))
2615-
}
2616-
}
2617-
}
2618-
}
2619-
26202599
func (do *Domain) updateStatsWorker(_ sessionctx.Context) {
26212600
defer util.Recover(metrics.LabelDomain, "updateStatsWorker", nil, false)
26222601
logutil.BgLogger().Info("updateStatsWorker started.")

pkg/statistics/handle/ddl/ddl.go

Lines changed: 4 additions & 161 deletions
Original file line numberDiff line numberDiff line change
@@ -19,22 +19,18 @@ import (
1919

2020
"github.com/pingcap/errors"
2121
"github.com/pingcap/tidb/pkg/ddl/notifier"
22-
"github.com/pingcap/tidb/pkg/meta/model"
2322
"github.com/pingcap/tidb/pkg/sessionctx"
24-
"github.com/pingcap/tidb/pkg/sessionctx/variable"
2523
"github.com/pingcap/tidb/pkg/statistics/handle/lockstats"
26-
"github.com/pingcap/tidb/pkg/statistics/handle/logutil"
2724
"github.com/pingcap/tidb/pkg/statistics/handle/storage"
2825
"github.com/pingcap/tidb/pkg/statistics/handle/types"
2926
"github.com/pingcap/tidb/pkg/statistics/handle/util"
30-
"github.com/pingcap/tidb/pkg/util/intest"
31-
"go.uber.org/zap"
3227
)
3328

3429
type ddlHandlerImpl struct {
3530
ddlEventCh chan *notifier.SchemaChangeEvent
3631
statsWriter types.StatsReadWriter
3732
statsHandler types.StatsHandle
33+
sub *subscriber
3834
}
3935

4036
// NewDDLHandler creates a new ddl handler.
@@ -46,147 +42,13 @@ func NewDDLHandler(
4642
ddlEventCh: make(chan *notifier.SchemaChangeEvent, 1000),
4743
statsWriter: statsWriter,
4844
statsHandler: statsHandler,
45+
sub: NewSubscriber(statsHandler),
4946
}
5047
}
5148

5249
// HandleDDLEvent begins to process a ddl task.
53-
func (h *ddlHandlerImpl) HandleDDLEvent(s *notifier.SchemaChangeEvent) error {
54-
switch s.GetType() {
55-
case model.ActionCreateTable:
56-
newTableInfo := s.GetCreateTableInfo()
57-
ids, err := h.getTableIDs(newTableInfo)
58-
if err != nil {
59-
return err
60-
}
61-
for _, id := range ids {
62-
if err := h.statsWriter.InsertTableStats2KV(newTableInfo, id); err != nil {
63-
return err
64-
}
65-
}
66-
case model.ActionTruncateTable:
67-
newTableInfo, droppedTableInfo := s.GetTruncateTableInfo()
68-
ids, err := h.getTableIDs(newTableInfo)
69-
if err != nil {
70-
return err
71-
}
72-
for _, id := range ids {
73-
if err := h.statsWriter.InsertTableStats2KV(newTableInfo, id); err != nil {
74-
return err
75-
}
76-
}
77-
78-
// Remove the old table stats.
79-
droppedIDs, err := h.getTableIDs(droppedTableInfo)
80-
if err != nil {
81-
return err
82-
}
83-
for _, id := range droppedIDs {
84-
if err := h.statsWriter.UpdateStatsMetaVersionForGC(id); err != nil {
85-
return err
86-
}
87-
}
88-
case model.ActionDropTable:
89-
droppedTableInfo := s.GetDropTableInfo()
90-
ids, err := h.getTableIDs(droppedTableInfo)
91-
if err != nil {
92-
return err
93-
}
94-
for _, id := range ids {
95-
if err := h.statsWriter.UpdateStatsMetaVersionForGC(id); err != nil {
96-
return err
97-
}
98-
}
99-
case model.ActionAddColumn:
100-
newTableInfo, newColumnInfo := s.GetAddColumnInfo()
101-
ids, err := h.getTableIDs(newTableInfo)
102-
if err != nil {
103-
return err
104-
}
105-
for _, id := range ids {
106-
if err := h.statsWriter.InsertColStats2KV(id, newColumnInfo); err != nil {
107-
return err
108-
}
109-
}
110-
case model.ActionModifyColumn:
111-
newTableInfo, modifiedColumnInfo := s.GetModifyColumnInfo()
112-
ids, err := h.getTableIDs(newTableInfo)
113-
if err != nil {
114-
return err
115-
}
116-
for _, id := range ids {
117-
if err := h.statsWriter.InsertColStats2KV(id, modifiedColumnInfo); err != nil {
118-
return err
119-
}
120-
}
121-
case model.ActionAddTablePartition:
122-
globalTableInfo, addedPartitionInfo := s.GetAddPartitionInfo()
123-
for _, def := range addedPartitionInfo.Definitions {
124-
if err := h.statsWriter.InsertTableStats2KV(globalTableInfo, def.ID); err != nil {
125-
return err
126-
}
127-
}
128-
case model.ActionTruncateTablePartition:
129-
if err := h.onTruncatePartitions(s); err != nil {
130-
return err
131-
}
132-
case model.ActionDropTablePartition:
133-
if err := h.onDropPartitions(s); err != nil {
134-
return err
135-
}
136-
// EXCHANGE PARTITION EVENT NOTES:
137-
// 1. When a partition is exchanged with a system table, we need to adjust the global statistics
138-
// based on the count delta and modify count delta. However, due to the involvement of the system table,
139-
// a complete update of the global statistics is not feasible. Therefore, we bypass the statistics update
140-
// for the table in this scenario. Despite this, the table id still changes, so the statistics for the
141-
// system table will still be visible.
142-
// 2. If the system table is a partitioned table, we will update the global statistics for the partitioned table.
143-
// It is rare to exchange a partition from a system table, so we can ignore this case. In this case,
144-
// the system table will have statistics, but this is not a significant issue.
145-
// So we decided to completely ignore the system table event.
146-
case model.ActionExchangeTablePartition:
147-
if err := h.onExchangeAPartition(s); err != nil {
148-
return err
149-
}
150-
case model.ActionReorganizePartition:
151-
if err := h.onReorganizePartitions(s); err != nil {
152-
return err
153-
}
154-
case model.ActionAlterTablePartitioning:
155-
oldSingleTableID, globalTableInfo, addedPartInfo := s.GetAddPartitioningInfo()
156-
// Add new partition stats.
157-
for _, def := range addedPartInfo.Definitions {
158-
if err := h.statsWriter.InsertTableStats2KV(globalTableInfo, def.ID); err != nil {
159-
return err
160-
}
161-
}
162-
// Change id for global stats, since the data has not changed!
163-
// Note: This operation will update all tables related to statistics with the new ID.
164-
return h.statsWriter.ChangeGlobalStatsID(oldSingleTableID, globalTableInfo.ID)
165-
case model.ActionRemovePartitioning:
166-
// Change id for global stats, since the data has not changed!
167-
// Note: This operation will update all tables related to statistics with the new ID.
168-
oldTblID, newSingleTableInfo, droppedPartInfo := s.GetRemovePartitioningInfo()
169-
if err := h.statsWriter.ChangeGlobalStatsID(oldTblID, newSingleTableInfo.ID); err != nil {
170-
return err
171-
}
172-
173-
// Remove partition stats.
174-
for _, def := range droppedPartInfo.Definitions {
175-
if err := h.statsWriter.UpdateStatsMetaVersionForGC(def.ID); err != nil {
176-
return err
177-
}
178-
}
179-
case model.ActionFlashbackCluster:
180-
return h.statsWriter.UpdateStatsVersion()
181-
case model.ActionAddIndex:
182-
// No need to update the stats meta for the adding index event.
183-
case model.ActionDropSchema:
184-
// TODO: handle the drop schema event.
185-
default:
186-
intest.Assert(false)
187-
logutil.StatsLogger().Error("Unhandled schema change event", zap.Stringer("type", s))
188-
}
189-
return nil
50+
func (h *ddlHandlerImpl) HandleDDLEvent(ctx context.Context, sctx sessionctx.Context, s *notifier.SchemaChangeEvent) error {
51+
return h.sub.handle(ctx, sctx, s)
19052
}
19153

19254
// UpdateStatsWithCountDeltaAndModifyCountDeltaForTest updates the global stats with the given count delta and modify count delta.
@@ -282,25 +144,6 @@ func updateStatsWithCountDeltaAndModifyCountDelta(
282144
return err
283145
}
284146

285-
func (h *ddlHandlerImpl) getTableIDs(tblInfo *model.TableInfo) (ids []int64, err error) {
286-
pi := tblInfo.GetPartitionInfo()
287-
if pi == nil {
288-
return []int64{tblInfo.ID}, nil
289-
}
290-
ids = make([]int64, 0, len(pi.Definitions)+1)
291-
for _, def := range pi.Definitions {
292-
ids = append(ids, def.ID)
293-
}
294-
pruneMode, err := util.GetCurrentPruneMode(h.statsHandler.SPool())
295-
if err != nil {
296-
return nil, err
297-
}
298-
if variable.PartitionPruneMode(pruneMode) == variable.Dynamic {
299-
ids = append(ids, tblInfo.ID)
300-
}
301-
return ids, nil
302-
}
303-
304147
// DDLEventCh returns ddl events channel in handle.
305148
func (h *ddlHandlerImpl) DDLEventCh() chan *notifier.SchemaChangeEvent {
306149
return h.ddlEventCh

pkg/statistics/handle/ddl/subscriber.go

Lines changed: 11 additions & 13 deletions
Original file line numberDiff line numberDiff line change
@@ -31,21 +31,19 @@ import (
3131
"go.uber.org/zap"
3232
)
3333

34-
type handler struct {
34+
type subscriber struct {
3535
statsCache types.StatsCache
3636
}
3737

38-
// NewHandlerAndRegister creates a new handler and registers it to the DDL
39-
// notifier.
40-
func NewHandlerAndRegister(
38+
// NewSubscriber creates a new subscriber.
39+
func NewSubscriber(
4140
statsCache types.StatsCache,
42-
registry *notifier.DDLNotifier,
43-
) {
44-
h := handler{statsCache: statsCache}
45-
registry.RegisterHandler(notifier.StatsMetaHandlerID, h.handle)
41+
) *subscriber {
42+
h := subscriber{statsCache: statsCache}
43+
return &h
4644
}
4745

48-
func (h handler) handle(
46+
func (h subscriber) handle(
4947
ctx context.Context,
5048
sctx sessionctx.Context,
5149
change *notifier.SchemaChangeEvent,
@@ -243,7 +241,7 @@ func (h handler) handle(
243241
return nil
244242
}
245243

246-
func (h handler) insertStats4PhysicalID(
244+
func (h subscriber) insertStats4PhysicalID(
247245
ctx context.Context,
248246
sctx sessionctx.Context,
249247
info *model.TableInfo,
@@ -256,7 +254,7 @@ func (h handler) insertStats4PhysicalID(
256254
return errors.Trace(h.recordHistoricalStatsMeta(ctx, sctx, id, startTS))
257255
}
258256

259-
func (h handler) recordHistoricalStatsMeta(
257+
func (h subscriber) recordHistoricalStatsMeta(
260258
ctx context.Context,
261259
sctx sessionctx.Context,
262260
id int64,
@@ -287,7 +285,7 @@ func (h handler) recordHistoricalStatsMeta(
287285
)
288286
}
289287

290-
func (h handler) delayedDeleteStats4PhysicalID(
288+
func (h subscriber) delayedDeleteStats4PhysicalID(
291289
ctx context.Context,
292290
sctx sessionctx.Context,
293291
id int64,
@@ -299,7 +297,7 @@ func (h handler) delayedDeleteStats4PhysicalID(
299297
return errors.Trace(h.recordHistoricalStatsMeta(ctx, sctx, id, startTS))
300298
}
301299

302-
func (h handler) insertStats4Col(
300+
func (h subscriber) insertStats4Col(
303301
ctx context.Context,
304302
sctx sessionctx.Context,
305303
physicalID int64,

pkg/statistics/handle/handle.go

Lines changed: 3 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -146,6 +146,9 @@ func NewHandle(
146146
handle.StatsReadWriter,
147147
handle,
148148
)
149+
if ddlNotifier != nil {
150+
ddlNotifier.RegisterHandler(notifier.StatsMetaHandlerID, handle.DDL.HandleDDLEvent)
151+
}
149152
return handle, nil
150153
}
151154

@@ -195,12 +198,6 @@ func (h *Handle) getPartitionStats(tblInfo *model.TableInfo, pid int64, returnPs
195198

196199
// FlushStats flushes the cached stats update into store.
197200
func (h *Handle) FlushStats() {
198-
for len(h.DDLEventCh()) > 0 {
199-
e := <-h.DDLEventCh()
200-
if err := h.HandleDDLEvent(e); err != nil {
201-
statslogutil.StatsLogger().Error("handle ddl event fail", zap.Error(err))
202-
}
203-
}
204201
if err := h.DumpStatsDeltaToKV(true); err != nil {
205202
statslogutil.StatsLogger().Error("dump stats delta fail", zap.Error(err))
206203
}

pkg/statistics/handle/types/interfaces.go

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -496,7 +496,7 @@ type StatsGlobal interface {
496496
// DDL is used to handle ddl events.
497497
type DDL interface {
498498
// HandleDDLEvent handles ddl events.
499-
HandleDDLEvent(changeEvent *notifier.SchemaChangeEvent) error
499+
HandleDDLEvent(ctx context.Context, sctx sessionctx.Context, changeEvent *notifier.SchemaChangeEvent) error
500500
// DDLEventCh returns ddl events channel in handle.
501501
DDLEventCh() chan *notifier.SchemaChangeEvent
502502
}

0 commit comments

Comments
 (0)