Skip to content

Commit

Permalink
*: use DDL subscriber updating stats meta (#57872)
Browse files Browse the repository at this point in the history
close #57871
  • Loading branch information
Rustin170506 authored Dec 13, 2024
1 parent 033b175 commit 27b5392
Show file tree
Hide file tree
Showing 58 changed files with 369 additions and 559 deletions.
18 changes: 7 additions & 11 deletions pkg/ddl/ddl.go
Original file line number Diff line number Diff line change
Expand Up @@ -588,18 +588,14 @@ func asyncNotifyEvent(jobCtx *jobContext, e *notifier.SchemaChangeEvent, job *mo
return nil
}

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

intest.Assert(jobCtx.eventPublishStore != nil, "eventPublishStore should not be nil")
Expand Down
1 change: 1 addition & 0 deletions pkg/ddl/tests/partition/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -33,6 +33,7 @@ go_test(
"//pkg/sessionctx",
"//pkg/sessionctx/variable",
"//pkg/sessiontxn",
"//pkg/statistics/handle/ddl/testutil",
"//pkg/store/gcworker",
"//pkg/store/mockstore",
"//pkg/table",
Expand Down
25 changes: 17 additions & 8 deletions pkg/ddl/tests/partition/db_partition_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -42,6 +42,7 @@ import (
"github.com/pingcap/tidb/pkg/sessionctx"
"github.com/pingcap/tidb/pkg/sessionctx/variable"
"github.com/pingcap/tidb/pkg/sessiontxn"
statstestutil "github.com/pingcap/tidb/pkg/statistics/handle/ddl/testutil"
"github.com/pingcap/tidb/pkg/store/mockstore"
"github.com/pingcap/tidb/pkg/table"
"github.com/pingcap/tidb/pkg/table/tables"
Expand Down Expand Up @@ -2947,7 +2948,8 @@ func TestRemoveKeyPartitioning(t *testing.T) {
tk.MustExec("create database RemovePartitioning")
tk.MustExec("use RemovePartitioning")
tk.MustExec(`create table t (a varchar(255), b varchar(255), key (a,b), key (b)) partition by key (a) partitions 7`)
require.NoError(t, h.HandleDDLEvent(<-h.DDLEventCh()))
err := statstestutil.HandleNextDDLEventWithTxn(h)
require.NoError(t, err)
// Fill the data with ascii strings
for i := 32; i <= 126; i++ {
tk.MustExec(fmt.Sprintf(`insert into t values (char(%d,%d,%d),char(%d,%d,%d,%d))`, i, i, i, i, i, i, i))
Expand Down Expand Up @@ -2979,7 +2981,8 @@ func TestRemoveKeyPartitioning(t *testing.T) {
" KEY `b` (`b`)\n" +
") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin"))
// Statistics are updated asynchronously
require.NoError(t, h.HandleDDLEvent(<-h.DDLEventCh()))
err = statstestutil.HandleNextDDLEventWithTxn(h)
require.NoError(t, err)
// And also cached and lazy loaded
h.Clear()
require.NoError(t, h.Update(context.Background(), dom.InfoSchema()))
Expand All @@ -2997,7 +3000,8 @@ func TestRemoveListPartitioning(t *testing.T) {
tk.MustExec("create database RemoveListPartitioning")
tk.MustExec("use RemoveListPartitioning")
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))`)
require.NoError(t, h.HandleDDLEvent(<-h.DDLEventCh()))
err := statstestutil.HandleNextDDLEventWithTxn(h)
require.NoError(t, err)
// Fill the data with ascii strings
for i := 32; i <= 126; i++ {
tk.MustExec(fmt.Sprintf(`insert into t values (%d,char(%d,%d,%d,%d))`, i%5, i, i, i, i))
Expand Down Expand Up @@ -3025,7 +3029,8 @@ func TestRemoveListPartitioning(t *testing.T) {
" KEY `b` (`b`)\n" +
") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin"))
// Statistics are updated asynchronously
require.NoError(t, h.HandleDDLEvent(<-h.DDLEventCh()))
err = statstestutil.HandleNextDDLEventWithTxn(h)
require.NoError(t, err)
// And also cached and lazy loaded
h.Clear()
require.NoError(t, h.Update(context.Background(), dom.InfoSchema()))
Expand All @@ -3043,7 +3048,8 @@ func TestRemoveListColumnPartitioning(t *testing.T) {
tk.MustExec("create database RemoveListPartitioning")
tk.MustExec("use RemoveListPartitioning")
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"))`)
require.NoError(t, h.HandleDDLEvent(<-h.DDLEventCh()))
err := statstestutil.HandleNextDDLEventWithTxn(h)
require.NoError(t, err)
// Fill the data with ascii strings
for i := 32; i <= 126; i++ {
tk.MustExec(fmt.Sprintf(`insert into t values ("%d",char(%d,%d,%d,%d))`, i%5, i, i, i, i))
Expand Down Expand Up @@ -3071,7 +3077,8 @@ func TestRemoveListColumnPartitioning(t *testing.T) {
" KEY `b` (`b`)\n" +
") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin"))
// Statistics are updated asynchronously
require.NoError(t, h.HandleDDLEvent(<-h.DDLEventCh()))
err = statstestutil.HandleNextDDLEventWithTxn(h)
require.NoError(t, err)
// And also cached and lazy loaded
h.Clear()
require.NoError(t, h.Update(context.Background(), dom.InfoSchema()))
Expand All @@ -3089,7 +3096,8 @@ func TestRemoveListColumnsPartitioning(t *testing.T) {
tk.MustExec("create database RemoveListPartitioning")
tk.MustExec("use RemoveListPartitioning")
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")))`)
require.NoError(t, h.HandleDDLEvent(<-h.DDLEventCh()))
err := statstestutil.HandleNextDDLEventWithTxn(h)
require.NoError(t, err)
// Fill the data
for i := 32; i <= 126; i++ {
tk.MustExec(fmt.Sprintf(`insert into t values (%d,"%d")`, i%5, i%5))
Expand Down Expand Up @@ -3117,7 +3125,8 @@ func TestRemoveListColumnsPartitioning(t *testing.T) {
" KEY `b` (`b`)\n" +
") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin"))
// Statistics are updated asynchronously
require.NoError(t, h.HandleDDLEvent(<-h.DDLEventCh()))
err = statstestutil.HandleNextDDLEventWithTxn(h)
require.NoError(t, err)
// And also cached and lazy loaded
h.Clear()
require.NoError(t, h.Update(context.Background(), dom.InfoSchema()))
Expand Down
21 changes: 0 additions & 21 deletions pkg/domain/domain.go
Original file line number Diff line number Diff line change
Expand Up @@ -2449,9 +2449,6 @@ func (do *Domain) UpdateTableStatsLoop(ctx, initStatsCtx sessionctx.Context) err
// This is because the updated worker's primary responsibilities are to update the change delta and handle DDL operations.
// These tasks do not interfere with or depend on the initialization process.
do.wg.Run(func() { do.updateStatsWorker(ctx) }, "updateStatsWorker")
do.wg.Run(func() {
do.handleDDLEvent()
}, "handleDDLEvent")
// Wait for the stats worker to finish the initialization.
// Otherwise, we may start the auto analyze worker before the stats cache is initialized.
do.wg.Run(
Expand Down Expand Up @@ -2651,24 +2648,6 @@ func (do *Domain) updateStatsWorkerExitPreprocessing(statsHandle *handle.Handle)
}
}

func (do *Domain) handleDDLEvent() {
logutil.BgLogger().Info("handleDDLEvent started.")
defer util.Recover(metrics.LabelDomain, "handleDDLEvent", nil, false)
statsHandle := do.StatsHandle()
for {
select {
case <-do.exit:
return
// This channel is sent only by ddl owner.
case t := <-statsHandle.DDLEventCh():
err := statsHandle.HandleDDLEvent(t)
if err != nil {
logutil.BgLogger().Error("handle ddl event failed", zap.String("event", t.String()), zap.Error(err))
}
}
}
}

func (do *Domain) updateStatsWorker(_ sessionctx.Context) {
defer util.Recover(metrics.LabelDomain, "updateStatsWorker", nil, false)
logutil.BgLogger().Info("updateStatsWorker started.")
Expand Down
1 change: 1 addition & 0 deletions pkg/executor/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -438,6 +438,7 @@ go_test(
"//pkg/sessiontxn",
"//pkg/sessiontxn/staleread",
"//pkg/statistics",
"//pkg/statistics/handle/ddl/testutil",
"//pkg/statistics/handle/storage",
"//pkg/statistics/util",
"//pkg/store/copr",
Expand Down
13 changes: 9 additions & 4 deletions pkg/executor/infoschema_reader_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -31,6 +31,7 @@ import (
"github.com/pingcap/tidb/pkg/parser/auth"
"github.com/pingcap/tidb/pkg/parser/mysql"
"github.com/pingcap/tidb/pkg/sessionctx/variable"
statstestutil "github.com/pingcap/tidb/pkg/statistics/handle/ddl/testutil"
"github.com/pingcap/tidb/pkg/store/mockstore"
"github.com/pingcap/tidb/pkg/testkit"
"github.com/pingcap/tidb/pkg/testkit/testfailpoint"
Expand Down Expand Up @@ -173,7 +174,8 @@ func TestDataForTableStatsField(t *testing.T) {
tk.MustExec("use test")
tk.MustExec("drop table if exists t")
tk.MustExec("create table t (c int, d int, e char(5), index idx(e))")
require.NoError(t, h.HandleDDLEvent(<-h.DDLEventCh()))
err := statstestutil.HandleNextDDLEventWithTxn(h)
require.NoError(t, err)
tk.MustQuery("select table_rows, avg_row_length, data_length, index_length from information_schema.tables where table_name='t'").Check(
testkit.Rows("0 0 0 0"))
tk.MustExec(`insert into t(c, d, e) values(1, 2, "c"), (2, 3, "d"), (3, 4, "e")`)
Expand All @@ -200,7 +202,8 @@ func TestDataForTableStatsField(t *testing.T) {
// Test partition table.
tk.MustExec("drop table if exists t")
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))`)
require.NoError(t, h.HandleDDLEvent(<-h.DDLEventCh()))
err = statstestutil.HandleNextDDLEventWithTxn(h)
require.NoError(t, err)
tk.MustExec(`insert into t(a, b, c) values(1, 2, "c"), (7, 3, "d"), (12, 4, "e")`)
require.NoError(t, h.DumpStatsDeltaToKV(true))
require.NoError(t, h.Update(context.Background(), is))
Expand All @@ -219,7 +222,8 @@ func TestPartitionsTable(t *testing.T) {
testkit.WithPruneMode(tk, variable.Static, func() {
tk.MustExec("DROP TABLE IF EXISTS `test_partitions`;")
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));`)
require.NoError(t, h.HandleDDLEvent(<-h.DDLEventCh()))
err := statstestutil.HandleNextDDLEventWithTxn(h)
require.NoError(t, err)
tk.MustExec(`insert into test_partitions(a, b, c) values(1, 2, "c"), (7, 3, "d"), (12, 4, "e");`)

tk.MustQuery("select PARTITION_NAME, PARTITION_DESCRIPTION from information_schema.PARTITIONS where table_name='test_partitions';").Check(
Expand All @@ -245,7 +249,8 @@ func TestPartitionsTable(t *testing.T) {
// Test for table has no partitions.
tk.MustExec("DROP TABLE IF EXISTS `test_partitions_1`;")
tk.MustExec(`CREATE TABLE test_partitions_1 (a int, b int, c varchar(5), primary key(a), index idx(c));`)
require.NoError(t, h.HandleDDLEvent(<-h.DDLEventCh()))
err := statstestutil.HandleNextDDLEventWithTxn(h)
require.NoError(t, err)
tk.MustExec(`insert into test_partitions_1(a, b, c) values(1, 2, "c"), (7, 3, "d"), (12, 4, "e");`)
require.NoError(t, h.DumpStatsDeltaToKV(true))
require.NoError(t, h.Update(context.Background(), is))
Expand Down
1 change: 1 addition & 0 deletions pkg/executor/test/analyzetest/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -27,6 +27,7 @@ go_test(
"//pkg/sessionctx",
"//pkg/sessionctx/variable",
"//pkg/statistics",
"//pkg/statistics/handle/ddl/testutil",
"//pkg/testkit",
"//pkg/testkit/analyzehelper",
"//pkg/util/dbterror/exeerrors",
Expand Down
10 changes: 7 additions & 3 deletions pkg/executor/test/analyzetest/analyze_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -39,6 +39,7 @@ import (
"github.com/pingcap/tidb/pkg/sessionctx"
"github.com/pingcap/tidb/pkg/sessionctx/variable"
"github.com/pingcap/tidb/pkg/statistics"
statstestutil "github.com/pingcap/tidb/pkg/statistics/handle/ddl/testutil"
"github.com/pingcap/tidb/pkg/testkit"
"github.com/pingcap/tidb/pkg/testkit/analyzehelper"
"github.com/pingcap/tidb/pkg/util/dbterror/exeerrors"
Expand Down Expand Up @@ -496,7 +497,8 @@ func TestAdjustSampleRateNote(t *testing.T) {
statsHandle := domain.GetDomain(tk.Session().(sessionctx.Context)).StatsHandle()
tk.MustExec("drop table if exists t")
tk.MustExec("create table t(a int, index index_a(a))")
require.NoError(t, statsHandle.HandleDDLEvent(<-statsHandle.DDLEventCh()))
err := statstestutil.HandleNextDDLEventWithTxn(statsHandle)
require.NoError(t, err)
is := tk.Session().(sessionctx.Context).GetInfoSchema().(infoschema.InfoSchema)
tbl, err := is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t"))
require.NoError(t, err)
Expand Down Expand Up @@ -2719,7 +2721,8 @@ func TestAutoAnalyzeAwareGlobalVariableChange(t *testing.T) {
tk.MustExec("create table t(a int)")
analyzehelper.TriggerPredicateColumnsCollection(t, tk, store, "t", "a")
h := dom.StatsHandle()
require.NoError(t, h.HandleDDLEvent(<-h.DDLEventCh()))
err := statstestutil.HandleNextDDLEventWithTxn(h)
require.NoError(t, err)
tbl, err := dom.InfoSchema().TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t"))
require.NoError(t, err)
tid := tbl.Meta().ID
Expand Down Expand Up @@ -2836,7 +2839,8 @@ func TestAnalyzeMVIndex(t *testing.T) {
"index ij_binary((cast(j->'$.bin' as binary(50) array)))," +
"index ij_char((cast(j->'$.char' as char(50) array)))" +
")")
require.NoError(t, h.HandleDDLEvent(<-h.DDLEventCh()))
err := statstestutil.HandleNextDDLEventWithTxn(h)
require.NoError(t, err)
jsonData := []map[string]any{
{
"signed": []int64{1, 2, 300, 300, 0, 4, 5, -40000},
Expand Down
1 change: 1 addition & 0 deletions pkg/planner/cardinality/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -80,6 +80,7 @@ go_test(
"//pkg/sessionctx/stmtctx",
"//pkg/sessionctx/variable",
"//pkg/statistics",
"//pkg/statistics/handle/ddl/testutil",
"//pkg/testkit",
"//pkg/testkit/testdata",
"//pkg/testkit/testmain",
Expand Down
25 changes: 17 additions & 8 deletions pkg/planner/cardinality/selectivity_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -41,6 +41,7 @@ import (
"github.com/pingcap/tidb/pkg/sessionctx/stmtctx"
"github.com/pingcap/tidb/pkg/sessionctx/variable"
"github.com/pingcap/tidb/pkg/statistics"
statstestutil "github.com/pingcap/tidb/pkg/statistics/handle/ddl/testutil"
"github.com/pingcap/tidb/pkg/testkit"
"github.com/pingcap/tidb/pkg/testkit/testdata"
"github.com/pingcap/tidb/pkg/types"
Expand Down Expand Up @@ -175,7 +176,8 @@ func TestOutOfRangeEstimationAfterDelete(t *testing.T) {
testKit.MustExec("use test")
testKit.MustExec("drop table if exists t")
testKit.MustExec("create table t(a int unsigned)")
require.NoError(t, h.HandleDDLEvent(<-h.DDLEventCh()))
err := statstestutil.HandleNextDDLEventWithTxn(h)
require.NoError(t, err)
// [300, 900)
// 5 rows for each value, 3000 rows in total.
for i := 0; i < 3000; i++ {
Expand Down Expand Up @@ -898,7 +900,8 @@ func TestGlobalStatsOutOfRangeEstimationAfterDelete(t *testing.T) {
"partition p2 values less than (800)," +
"partition p3 values less than (1000)," +
"partition p4 values less than (1200))")
require.NoError(t, h.HandleDDLEvent(<-h.DDLEventCh()))
err := statstestutil.HandleNextDDLEventWithTxn(h)
require.NoError(t, err)
for i := 0; i < 3000; i++ {
testKit.MustExec(fmt.Sprintf("insert into t values (%v)", i/5+300)) // [300, 900)
}
Expand Down Expand Up @@ -1005,7 +1008,8 @@ func TestIndexJoinInnerRowCountUpperBound(t *testing.T) {
testKit.MustExec("use test")
testKit.MustExec("drop table if exists t")
testKit.MustExec("create table t(a int, b int, index idx(b))")
require.NoError(t, h.HandleDDLEvent(<-h.DDLEventCh()))
err := statstestutil.HandleNextDDLEventWithTxn(h)
require.NoError(t, err)
is := dom.InfoSchema()
tb, err := is.TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t"))
require.NoError(t, err)
Expand Down Expand Up @@ -1076,7 +1080,8 @@ func TestOrderingIdxSelectivityThreshold(t *testing.T) {
testKit.MustExec("use test")
testKit.MustExec("drop table if exists t")
testKit.MustExec("create table t(a int primary key , b int, c int, index ib(b), index ic(c))")
require.NoError(t, h.HandleDDLEvent(<-h.DDLEventCh()))
err := statstestutil.HandleNextDDLEventWithTxn(h)
require.NoError(t, err)
is := dom.InfoSchema()
tb, err := is.TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t"))
require.NoError(t, err)
Expand Down Expand Up @@ -1159,7 +1164,8 @@ func TestOrderingIdxSelectivityRatio(t *testing.T) {
testKit.MustExec("use test")
testKit.MustExec("drop table if exists t")
testKit.MustExec("create table t(a int primary key, b int, c int, index ib(b), index ic(c))")
require.NoError(t, h.HandleDDLEvent(<-h.DDLEventCh()))
err := statstestutil.HandleNextDDLEventWithTxn(h)
require.NoError(t, err)
is := dom.InfoSchema()
tb, err := is.TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t"))
require.NoError(t, err)
Expand Down Expand Up @@ -1240,7 +1246,8 @@ func TestCrossValidationSelectivity(t *testing.T) {
tk.MustExec("drop table if exists t")
tk.MustExec("set @@tidb_analyze_version = 1")
tk.MustExec("create table t (a int, b int, c int, primary key (a, b) clustered)")
require.NoError(t, h.HandleDDLEvent(<-h.DDLEventCh()))
err := statstestutil.HandleNextDDLEventWithTxn(h)
require.NoError(t, err)
tk.MustExec("insert into t values (1,2,3), (1,4,5)")
require.NoError(t, h.DumpStatsDeltaToKV(true))
tk.MustExec("analyze table t")
Expand All @@ -1257,7 +1264,8 @@ func TestIgnoreRealtimeStats(t *testing.T) {
testKit.MustExec("drop table if exists t")
testKit.MustExec("create table t(a int, b int)")
h := dom.StatsHandle()
require.NoError(t, h.HandleDDLEvent(<-h.DDLEventCh()))
err := statstestutil.HandleNextDDLEventWithTxn(h)
require.NoError(t, err)

// 1. Insert 11 rows of data without ANALYZE.
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)")
Expand Down Expand Up @@ -1371,7 +1379,8 @@ func TestBuiltinInEstWithoutStats(t *testing.T) {

tk.MustExec("use test")
tk.MustExec("create table t(a int, b int)")
require.NoError(t, h.HandleDDLEvent(<-h.DDLEventCh()))
err := statstestutil.HandleNextDDLEventWithTxn(h)
require.NoError(t, err)
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)")
require.NoError(t, h.DumpStatsDeltaToKV(true))
is := dom.InfoSchema()
Expand Down
6 changes: 4 additions & 2 deletions pkg/planner/cardinality/trace_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -35,6 +35,7 @@ import (
"github.com/pingcap/tidb/pkg/planner/core/operator/logicalop"
"github.com/pingcap/tidb/pkg/planner/core/resolve"
"github.com/pingcap/tidb/pkg/sessionctx"
statstestutil "github.com/pingcap/tidb/pkg/statistics/handle/ddl/testutil"
"github.com/pingcap/tidb/pkg/testkit"
"github.com/pingcap/tidb/pkg/testkit/testdata"
"github.com/pingcap/tidb/pkg/util/tracing"
Expand Down Expand Up @@ -135,7 +136,8 @@ func TestTraceDebugSelectivity(t *testing.T) {
tk.MustExec("use test")
tk.MustExec("drop table if exists t")
tk.MustExec("create table t(a int, b int, index iab(a, b), index ib(b))")
require.NoError(t, statsHandle.HandleDDLEvent(<-statsHandle.DDLEventCh()))
err := statstestutil.HandleNextDDLEventWithTxn(statsHandle)
require.NoError(t, err)

// Prepare the data.

Expand Down Expand Up @@ -188,7 +190,7 @@ func TestTraceDebugSelectivity(t *testing.T) {
sql := "explain " + tt
tk.MustExec(sql)
}
err := statsHandle.LoadNeededHistograms(dom.InfoSchema())
err = statsHandle.LoadNeededHistograms(dom.InfoSchema())
require.NoError(t, err)

sctx := tk.Session().(sessionctx.Context)
Expand Down
1 change: 1 addition & 0 deletions pkg/planner/core/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -300,6 +300,7 @@ go_test(
"//pkg/sessionctx/variable",
"//pkg/sessiontxn",
"//pkg/statistics",
"//pkg/statistics/handle/ddl/testutil",
"//pkg/store/mockstore",
"//pkg/table",
"//pkg/testkit",
Expand Down
1 change: 1 addition & 0 deletions pkg/planner/core/casetest/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -24,6 +24,7 @@ go_test(
"//pkg/planner/core/operator/logicalop",
"//pkg/planner/core/resolve",
"//pkg/planner/property",
"//pkg/statistics/handle/ddl/testutil",
"//pkg/testkit",
"//pkg/testkit/testdata",
"//pkg/testkit/testmain",
Expand Down
Loading

0 comments on commit 27b5392

Please sign in to comment.