Skip to content

Commit 27b5392

Browse files
authored
*: use DDL subscriber updating stats meta (#57872)
close #57871
1 parent 033b175 commit 27b5392

Some content is hidden

Large Commits have some content hidden by default. Use the searchbox below for content that may be hidden.

58 files changed

+369
-559
lines changed

pkg/ddl/ddl.go

Lines changed: 7 additions & 11 deletions
Original file line numberDiff line numberDiff line change
@@ -588,18 +588,14 @@ func asyncNotifyEvent(jobCtx *jobContext, e *notifier.SchemaChangeEvent, job *mo
588588
return nil
589589
}
590590

591-
ch := jobCtx.oldDDLCtx.ddlEventCh
592-
if ch != nil {
593-
forLoop:
594-
for i := 0; i < 10; i++ {
595-
select {
596-
case ch <- e:
597-
break forLoop
598-
default:
599-
time.Sleep(time.Microsecond * 10)
600-
}
591+
// In test environments, we use a channel-based approach to handle DDL events.
592+
// This maintains compatibility with existing test cases that expect events to be delivered through channels.
593+
// In production, DDL events are handled by the notifier system instead.
594+
if intest.InTest {
595+
ch := jobCtx.oldDDLCtx.ddlEventCh
596+
if ch != nil {
597+
ch <- e
601598
}
602-
logutil.DDLLogger().Warn("fail to notify DDL event", zap.Stringer("event", e))
603599
}
604600

605601
intest.Assert(jobCtx.eventPublishStore != nil, "eventPublishStore should not be nil")

pkg/ddl/tests/partition/BUILD.bazel

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -33,6 +33,7 @@ go_test(
3333
"//pkg/sessionctx",
3434
"//pkg/sessionctx/variable",
3535
"//pkg/sessiontxn",
36+
"//pkg/statistics/handle/ddl/testutil",
3637
"//pkg/store/gcworker",
3738
"//pkg/store/mockstore",
3839
"//pkg/table",

pkg/ddl/tests/partition/db_partition_test.go

Lines changed: 17 additions & 8 deletions
Original file line numberDiff line numberDiff line change
@@ -42,6 +42,7 @@ import (
4242
"github.com/pingcap/tidb/pkg/sessionctx"
4343
"github.com/pingcap/tidb/pkg/sessionctx/variable"
4444
"github.com/pingcap/tidb/pkg/sessiontxn"
45+
statstestutil "github.com/pingcap/tidb/pkg/statistics/handle/ddl/testutil"
4546
"github.com/pingcap/tidb/pkg/store/mockstore"
4647
"github.com/pingcap/tidb/pkg/table"
4748
"github.com/pingcap/tidb/pkg/table/tables"
@@ -2947,7 +2948,8 @@ func TestRemoveKeyPartitioning(t *testing.T) {
29472948
tk.MustExec("create database RemovePartitioning")
29482949
tk.MustExec("use RemovePartitioning")
29492950
tk.MustExec(`create table t (a varchar(255), b varchar(255), key (a,b), key (b)) partition by key (a) partitions 7`)
2950-
require.NoError(t, h.HandleDDLEvent(<-h.DDLEventCh()))
2951+
err := statstestutil.HandleNextDDLEventWithTxn(h)
2952+
require.NoError(t, err)
29512953
// Fill the data with ascii strings
29522954
for i := 32; i <= 126; i++ {
29532955
tk.MustExec(fmt.Sprintf(`insert into t values (char(%d,%d,%d),char(%d,%d,%d,%d))`, i, i, i, i, i, i, i))
@@ -2979,7 +2981,8 @@ func TestRemoveKeyPartitioning(t *testing.T) {
29792981
" KEY `b` (`b`)\n" +
29802982
") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin"))
29812983
// Statistics are updated asynchronously
2982-
require.NoError(t, h.HandleDDLEvent(<-h.DDLEventCh()))
2984+
err = statstestutil.HandleNextDDLEventWithTxn(h)
2985+
require.NoError(t, err)
29832986
// And also cached and lazy loaded
29842987
h.Clear()
29852988
require.NoError(t, h.Update(context.Background(), dom.InfoSchema()))
@@ -2997,7 +3000,8 @@ func TestRemoveListPartitioning(t *testing.T) {
29973000
tk.MustExec("create database RemoveListPartitioning")
29983001
tk.MustExec("use RemoveListPartitioning")
29993002
tk.MustExec(`create table t (a int, b varchar(255), key (a,b), key (b)) partition by list (a) (partition p0 values in (0), partition p1 values in (1), partition p2 values in (2), partition p3 values in (3), partition p4 values in (4))`)
3000-
require.NoError(t, h.HandleDDLEvent(<-h.DDLEventCh()))
3003+
err := statstestutil.HandleNextDDLEventWithTxn(h)
3004+
require.NoError(t, err)
30013005
// Fill the data with ascii strings
30023006
for i := 32; i <= 126; i++ {
30033007
tk.MustExec(fmt.Sprintf(`insert into t values (%d,char(%d,%d,%d,%d))`, i%5, i, i, i, i))
@@ -3025,7 +3029,8 @@ func TestRemoveListPartitioning(t *testing.T) {
30253029
" KEY `b` (`b`)\n" +
30263030
") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin"))
30273031
// Statistics are updated asynchronously
3028-
require.NoError(t, h.HandleDDLEvent(<-h.DDLEventCh()))
3032+
err = statstestutil.HandleNextDDLEventWithTxn(h)
3033+
require.NoError(t, err)
30293034
// And also cached and lazy loaded
30303035
h.Clear()
30313036
require.NoError(t, h.Update(context.Background(), dom.InfoSchema()))
@@ -3043,7 +3048,8 @@ func TestRemoveListColumnPartitioning(t *testing.T) {
30433048
tk.MustExec("create database RemoveListPartitioning")
30443049
tk.MustExec("use RemoveListPartitioning")
30453050
tk.MustExec(`create table t (a varchar(255), b varchar(255), key (a,b), key (b)) partition by list columns (a) (partition p0 values in ("0"), partition p1 values in ("1"), partition p2 values in ("2"), partition p3 values in ("3"), partition p4 values in ("4"))`)
3046-
require.NoError(t, h.HandleDDLEvent(<-h.DDLEventCh()))
3051+
err := statstestutil.HandleNextDDLEventWithTxn(h)
3052+
require.NoError(t, err)
30473053
// Fill the data with ascii strings
30483054
for i := 32; i <= 126; i++ {
30493055
tk.MustExec(fmt.Sprintf(`insert into t values ("%d",char(%d,%d,%d,%d))`, i%5, i, i, i, i))
@@ -3071,7 +3077,8 @@ func TestRemoveListColumnPartitioning(t *testing.T) {
30713077
" KEY `b` (`b`)\n" +
30723078
") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin"))
30733079
// Statistics are updated asynchronously
3074-
require.NoError(t, h.HandleDDLEvent(<-h.DDLEventCh()))
3080+
err = statstestutil.HandleNextDDLEventWithTxn(h)
3081+
require.NoError(t, err)
30753082
// And also cached and lazy loaded
30763083
h.Clear()
30773084
require.NoError(t, h.Update(context.Background(), dom.InfoSchema()))
@@ -3089,7 +3096,8 @@ func TestRemoveListColumnsPartitioning(t *testing.T) {
30893096
tk.MustExec("create database RemoveListPartitioning")
30903097
tk.MustExec("use RemoveListPartitioning")
30913098
tk.MustExec(`create table t (a int, b varchar(255), key (a,b), key (b)) partition by list columns (a,b) (partition p0 values in ((0,"0")), partition p1 values in ((1,"1")), partition p2 values in ((2,"2")), partition p3 values in ((3,"3")), partition p4 values in ((4,"4")))`)
3092-
require.NoError(t, h.HandleDDLEvent(<-h.DDLEventCh()))
3099+
err := statstestutil.HandleNextDDLEventWithTxn(h)
3100+
require.NoError(t, err)
30933101
// Fill the data
30943102
for i := 32; i <= 126; i++ {
30953103
tk.MustExec(fmt.Sprintf(`insert into t values (%d,"%d")`, i%5, i%5))
@@ -3117,7 +3125,8 @@ func TestRemoveListColumnsPartitioning(t *testing.T) {
31173125
" KEY `b` (`b`)\n" +
31183126
") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin"))
31193127
// Statistics are updated asynchronously
3120-
require.NoError(t, h.HandleDDLEvent(<-h.DDLEventCh()))
3128+
err = statstestutil.HandleNextDDLEventWithTxn(h)
3129+
require.NoError(t, err)
31213130
// And also cached and lazy loaded
31223131
h.Clear()
31233132
require.NoError(t, h.Update(context.Background(), dom.InfoSchema()))

pkg/domain/domain.go

Lines changed: 0 additions & 21 deletions
Original file line numberDiff line numberDiff line change
@@ -2449,9 +2449,6 @@ func (do *Domain) UpdateTableStatsLoop(ctx, initStatsCtx sessionctx.Context) err
24492449
// This is because the updated worker's primary responsibilities are to update the change delta and handle DDL operations.
24502450
// These tasks do not interfere with or depend on the initialization process.
24512451
do.wg.Run(func() { do.updateStatsWorker(ctx) }, "updateStatsWorker")
2452-
do.wg.Run(func() {
2453-
do.handleDDLEvent()
2454-
}, "handleDDLEvent")
24552452
// Wait for the stats worker to finish the initialization.
24562453
// Otherwise, we may start the auto analyze worker before the stats cache is initialized.
24572454
do.wg.Run(
@@ -2651,24 +2648,6 @@ func (do *Domain) updateStatsWorkerExitPreprocessing(statsHandle *handle.Handle)
26512648
}
26522649
}
26532650

2654-
func (do *Domain) handleDDLEvent() {
2655-
logutil.BgLogger().Info("handleDDLEvent started.")
2656-
defer util.Recover(metrics.LabelDomain, "handleDDLEvent", nil, false)
2657-
statsHandle := do.StatsHandle()
2658-
for {
2659-
select {
2660-
case <-do.exit:
2661-
return
2662-
// This channel is sent only by ddl owner.
2663-
case t := <-statsHandle.DDLEventCh():
2664-
err := statsHandle.HandleDDLEvent(t)
2665-
if err != nil {
2666-
logutil.BgLogger().Error("handle ddl event failed", zap.String("event", t.String()), zap.Error(err))
2667-
}
2668-
}
2669-
}
2670-
}
2671-
26722651
func (do *Domain) updateStatsWorker(_ sessionctx.Context) {
26732652
defer util.Recover(metrics.LabelDomain, "updateStatsWorker", nil, false)
26742653
logutil.BgLogger().Info("updateStatsWorker started.")

pkg/executor/BUILD.bazel

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -438,6 +438,7 @@ go_test(
438438
"//pkg/sessiontxn",
439439
"//pkg/sessiontxn/staleread",
440440
"//pkg/statistics",
441+
"//pkg/statistics/handle/ddl/testutil",
441442
"//pkg/statistics/handle/storage",
442443
"//pkg/statistics/util",
443444
"//pkg/store/copr",

pkg/executor/infoschema_reader_test.go

Lines changed: 9 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -31,6 +31,7 @@ import (
3131
"github.com/pingcap/tidb/pkg/parser/auth"
3232
"github.com/pingcap/tidb/pkg/parser/mysql"
3333
"github.com/pingcap/tidb/pkg/sessionctx/variable"
34+
statstestutil "github.com/pingcap/tidb/pkg/statistics/handle/ddl/testutil"
3435
"github.com/pingcap/tidb/pkg/store/mockstore"
3536
"github.com/pingcap/tidb/pkg/testkit"
3637
"github.com/pingcap/tidb/pkg/testkit/testfailpoint"
@@ -173,7 +174,8 @@ func TestDataForTableStatsField(t *testing.T) {
173174
tk.MustExec("use test")
174175
tk.MustExec("drop table if exists t")
175176
tk.MustExec("create table t (c int, d int, e char(5), index idx(e))")
176-
require.NoError(t, h.HandleDDLEvent(<-h.DDLEventCh()))
177+
err := statstestutil.HandleNextDDLEventWithTxn(h)
178+
require.NoError(t, err)
177179
tk.MustQuery("select table_rows, avg_row_length, data_length, index_length from information_schema.tables where table_name='t'").Check(
178180
testkit.Rows("0 0 0 0"))
179181
tk.MustExec(`insert into t(c, d, e) values(1, 2, "c"), (2, 3, "d"), (3, 4, "e")`)
@@ -200,7 +202,8 @@ func TestDataForTableStatsField(t *testing.T) {
200202
// Test partition table.
201203
tk.MustExec("drop table if exists t")
202204
tk.MustExec(`CREATE TABLE t (a int, b int, c varchar(5), primary key(a), index idx(c)) PARTITION BY RANGE (a) (PARTITION p0 VALUES LESS THAN (6), PARTITION p1 VALUES LESS THAN (11), PARTITION p2 VALUES LESS THAN (16))`)
203-
require.NoError(t, h.HandleDDLEvent(<-h.DDLEventCh()))
205+
err = statstestutil.HandleNextDDLEventWithTxn(h)
206+
require.NoError(t, err)
204207
tk.MustExec(`insert into t(a, b, c) values(1, 2, "c"), (7, 3, "d"), (12, 4, "e")`)
205208
require.NoError(t, h.DumpStatsDeltaToKV(true))
206209
require.NoError(t, h.Update(context.Background(), is))
@@ -219,7 +222,8 @@ func TestPartitionsTable(t *testing.T) {
219222
testkit.WithPruneMode(tk, variable.Static, func() {
220223
tk.MustExec("DROP TABLE IF EXISTS `test_partitions`;")
221224
tk.MustExec(`CREATE TABLE test_partitions (a int, b int, c varchar(5), primary key(a), index idx(c)) PARTITION BY RANGE (a) (PARTITION p0 VALUES LESS THAN (6), PARTITION p1 VALUES LESS THAN (11), PARTITION p2 VALUES LESS THAN (16));`)
222-
require.NoError(t, h.HandleDDLEvent(<-h.DDLEventCh()))
225+
err := statstestutil.HandleNextDDLEventWithTxn(h)
226+
require.NoError(t, err)
223227
tk.MustExec(`insert into test_partitions(a, b, c) values(1, 2, "c"), (7, 3, "d"), (12, 4, "e");`)
224228

225229
tk.MustQuery("select PARTITION_NAME, PARTITION_DESCRIPTION from information_schema.PARTITIONS where table_name='test_partitions';").Check(
@@ -245,7 +249,8 @@ func TestPartitionsTable(t *testing.T) {
245249
// Test for table has no partitions.
246250
tk.MustExec("DROP TABLE IF EXISTS `test_partitions_1`;")
247251
tk.MustExec(`CREATE TABLE test_partitions_1 (a int, b int, c varchar(5), primary key(a), index idx(c));`)
248-
require.NoError(t, h.HandleDDLEvent(<-h.DDLEventCh()))
252+
err := statstestutil.HandleNextDDLEventWithTxn(h)
253+
require.NoError(t, err)
249254
tk.MustExec(`insert into test_partitions_1(a, b, c) values(1, 2, "c"), (7, 3, "d"), (12, 4, "e");`)
250255
require.NoError(t, h.DumpStatsDeltaToKV(true))
251256
require.NoError(t, h.Update(context.Background(), is))

pkg/executor/test/analyzetest/BUILD.bazel

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -27,6 +27,7 @@ go_test(
2727
"//pkg/sessionctx",
2828
"//pkg/sessionctx/variable",
2929
"//pkg/statistics",
30+
"//pkg/statistics/handle/ddl/testutil",
3031
"//pkg/testkit",
3132
"//pkg/testkit/analyzehelper",
3233
"//pkg/util/dbterror/exeerrors",

pkg/executor/test/analyzetest/analyze_test.go

Lines changed: 7 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -39,6 +39,7 @@ import (
3939
"github.com/pingcap/tidb/pkg/sessionctx"
4040
"github.com/pingcap/tidb/pkg/sessionctx/variable"
4141
"github.com/pingcap/tidb/pkg/statistics"
42+
statstestutil "github.com/pingcap/tidb/pkg/statistics/handle/ddl/testutil"
4243
"github.com/pingcap/tidb/pkg/testkit"
4344
"github.com/pingcap/tidb/pkg/testkit/analyzehelper"
4445
"github.com/pingcap/tidb/pkg/util/dbterror/exeerrors"
@@ -496,7 +497,8 @@ func TestAdjustSampleRateNote(t *testing.T) {
496497
statsHandle := domain.GetDomain(tk.Session().(sessionctx.Context)).StatsHandle()
497498
tk.MustExec("drop table if exists t")
498499
tk.MustExec("create table t(a int, index index_a(a))")
499-
require.NoError(t, statsHandle.HandleDDLEvent(<-statsHandle.DDLEventCh()))
500+
err := statstestutil.HandleNextDDLEventWithTxn(statsHandle)
501+
require.NoError(t, err)
500502
is := tk.Session().(sessionctx.Context).GetInfoSchema().(infoschema.InfoSchema)
501503
tbl, err := is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t"))
502504
require.NoError(t, err)
@@ -2719,7 +2721,8 @@ func TestAutoAnalyzeAwareGlobalVariableChange(t *testing.T) {
27192721
tk.MustExec("create table t(a int)")
27202722
analyzehelper.TriggerPredicateColumnsCollection(t, tk, store, "t", "a")
27212723
h := dom.StatsHandle()
2722-
require.NoError(t, h.HandleDDLEvent(<-h.DDLEventCh()))
2724+
err := statstestutil.HandleNextDDLEventWithTxn(h)
2725+
require.NoError(t, err)
27232726
tbl, err := dom.InfoSchema().TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t"))
27242727
require.NoError(t, err)
27252728
tid := tbl.Meta().ID
@@ -2836,7 +2839,8 @@ func TestAnalyzeMVIndex(t *testing.T) {
28362839
"index ij_binary((cast(j->'$.bin' as binary(50) array)))," +
28372840
"index ij_char((cast(j->'$.char' as char(50) array)))" +
28382841
")")
2839-
require.NoError(t, h.HandleDDLEvent(<-h.DDLEventCh()))
2842+
err := statstestutil.HandleNextDDLEventWithTxn(h)
2843+
require.NoError(t, err)
28402844
jsonData := []map[string]any{
28412845
{
28422846
"signed": []int64{1, 2, 300, 300, 0, 4, 5, -40000},

pkg/planner/cardinality/BUILD.bazel

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -80,6 +80,7 @@ go_test(
8080
"//pkg/sessionctx/stmtctx",
8181
"//pkg/sessionctx/variable",
8282
"//pkg/statistics",
83+
"//pkg/statistics/handle/ddl/testutil",
8384
"//pkg/testkit",
8485
"//pkg/testkit/testdata",
8586
"//pkg/testkit/testmain",

pkg/planner/cardinality/selectivity_test.go

Lines changed: 17 additions & 8 deletions
Original file line numberDiff line numberDiff line change
@@ -41,6 +41,7 @@ import (
4141
"github.com/pingcap/tidb/pkg/sessionctx/stmtctx"
4242
"github.com/pingcap/tidb/pkg/sessionctx/variable"
4343
"github.com/pingcap/tidb/pkg/statistics"
44+
statstestutil "github.com/pingcap/tidb/pkg/statistics/handle/ddl/testutil"
4445
"github.com/pingcap/tidb/pkg/testkit"
4546
"github.com/pingcap/tidb/pkg/testkit/testdata"
4647
"github.com/pingcap/tidb/pkg/types"
@@ -175,7 +176,8 @@ func TestOutOfRangeEstimationAfterDelete(t *testing.T) {
175176
testKit.MustExec("use test")
176177
testKit.MustExec("drop table if exists t")
177178
testKit.MustExec("create table t(a int unsigned)")
178-
require.NoError(t, h.HandleDDLEvent(<-h.DDLEventCh()))
179+
err := statstestutil.HandleNextDDLEventWithTxn(h)
180+
require.NoError(t, err)
179181
// [300, 900)
180182
// 5 rows for each value, 3000 rows in total.
181183
for i := 0; i < 3000; i++ {
@@ -898,7 +900,8 @@ func TestGlobalStatsOutOfRangeEstimationAfterDelete(t *testing.T) {
898900
"partition p2 values less than (800)," +
899901
"partition p3 values less than (1000)," +
900902
"partition p4 values less than (1200))")
901-
require.NoError(t, h.HandleDDLEvent(<-h.DDLEventCh()))
903+
err := statstestutil.HandleNextDDLEventWithTxn(h)
904+
require.NoError(t, err)
902905
for i := 0; i < 3000; i++ {
903906
testKit.MustExec(fmt.Sprintf("insert into t values (%v)", i/5+300)) // [300, 900)
904907
}
@@ -1005,7 +1008,8 @@ func TestIndexJoinInnerRowCountUpperBound(t *testing.T) {
10051008
testKit.MustExec("use test")
10061009
testKit.MustExec("drop table if exists t")
10071010
testKit.MustExec("create table t(a int, b int, index idx(b))")
1008-
require.NoError(t, h.HandleDDLEvent(<-h.DDLEventCh()))
1011+
err := statstestutil.HandleNextDDLEventWithTxn(h)
1012+
require.NoError(t, err)
10091013
is := dom.InfoSchema()
10101014
tb, err := is.TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t"))
10111015
require.NoError(t, err)
@@ -1076,7 +1080,8 @@ func TestOrderingIdxSelectivityThreshold(t *testing.T) {
10761080
testKit.MustExec("use test")
10771081
testKit.MustExec("drop table if exists t")
10781082
testKit.MustExec("create table t(a int primary key , b int, c int, index ib(b), index ic(c))")
1079-
require.NoError(t, h.HandleDDLEvent(<-h.DDLEventCh()))
1083+
err := statstestutil.HandleNextDDLEventWithTxn(h)
1084+
require.NoError(t, err)
10801085
is := dom.InfoSchema()
10811086
tb, err := is.TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t"))
10821087
require.NoError(t, err)
@@ -1159,7 +1164,8 @@ func TestOrderingIdxSelectivityRatio(t *testing.T) {
11591164
testKit.MustExec("use test")
11601165
testKit.MustExec("drop table if exists t")
11611166
testKit.MustExec("create table t(a int primary key, b int, c int, index ib(b), index ic(c))")
1162-
require.NoError(t, h.HandleDDLEvent(<-h.DDLEventCh()))
1167+
err := statstestutil.HandleNextDDLEventWithTxn(h)
1168+
require.NoError(t, err)
11631169
is := dom.InfoSchema()
11641170
tb, err := is.TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t"))
11651171
require.NoError(t, err)
@@ -1240,7 +1246,8 @@ func TestCrossValidationSelectivity(t *testing.T) {
12401246
tk.MustExec("drop table if exists t")
12411247
tk.MustExec("set @@tidb_analyze_version = 1")
12421248
tk.MustExec("create table t (a int, b int, c int, primary key (a, b) clustered)")
1243-
require.NoError(t, h.HandleDDLEvent(<-h.DDLEventCh()))
1249+
err := statstestutil.HandleNextDDLEventWithTxn(h)
1250+
require.NoError(t, err)
12441251
tk.MustExec("insert into t values (1,2,3), (1,4,5)")
12451252
require.NoError(t, h.DumpStatsDeltaToKV(true))
12461253
tk.MustExec("analyze table t")
@@ -1257,7 +1264,8 @@ func TestIgnoreRealtimeStats(t *testing.T) {
12571264
testKit.MustExec("drop table if exists t")
12581265
testKit.MustExec("create table t(a int, b int)")
12591266
h := dom.StatsHandle()
1260-
require.NoError(t, h.HandleDDLEvent(<-h.DDLEventCh()))
1267+
err := statstestutil.HandleNextDDLEventWithTxn(h)
1268+
require.NoError(t, err)
12611269

12621270
// 1. Insert 11 rows of data without ANALYZE.
12631271
testKit.MustExec("insert into t values(1,1),(1,2),(1,3),(1,4),(1,5),(2,1),(2,2),(2,3),(2,4),(2,5),(3,1)")
@@ -1371,7 +1379,8 @@ func TestBuiltinInEstWithoutStats(t *testing.T) {
13711379

13721380
tk.MustExec("use test")
13731381
tk.MustExec("create table t(a int, b int)")
1374-
require.NoError(t, h.HandleDDLEvent(<-h.DDLEventCh()))
1382+
err := statstestutil.HandleNextDDLEventWithTxn(h)
1383+
require.NoError(t, err)
13751384
tk.MustExec("insert into t values(1,1), (2,2), (3,3), (4,4), (5,5), (6,6), (7,7), (8,8), (9,9), (10,10)")
13761385
require.NoError(t, h.DumpStatsDeltaToKV(true))
13771386
is := dom.InfoSchema()

0 commit comments

Comments
 (0)