Skip to content

Commit

Permalink
*: enable the predicate columns feature by default (#54440)
Browse files Browse the repository at this point in the history
ref #53567
  • Loading branch information
Rustin170506 authored Jul 12, 2024
1 parent 0b9cd2f commit e5a8a23
Show file tree
Hide file tree
Showing 121 changed files with 481 additions and 342 deletions.
4 changes: 2 additions & 2 deletions br/pkg/backup/schema_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -206,7 +206,7 @@ func TestBuildBackupRangeAndSchemaWithBrokenStats(t *testing.T) {
tk.MustExec("drop table if exists t3;")
tk.MustExec("create table t3 (a char(1));")
tk.MustExec("insert into t3 values ('1');")
tk.MustExec("analyze table t3;")
tk.MustExec("analyze table t3 all columns;")
// corrupt the statistics like pingcap/br#679.
tk.MustExec(`
update mysql.stats_buckets set upper_bound = 0xffffffff
Expand Down Expand Up @@ -251,7 +251,7 @@ func TestBuildBackupRangeAndSchemaWithBrokenStats(t *testing.T) {
require.NotNil(t, schemas[0].DB)

// recover the statistics.
tk.MustExec("analyze table t3;")
tk.MustExec("analyze table t3 all columns;")

_, backupSchemas, _, err = backup.BuildBackupRangeAndInitSchema(m.Storage, f, math.MaxUint64, false, true)
require.NoError(t, err)
Expand Down
2 changes: 1 addition & 1 deletion br/pkg/restore/snap_client/systable_restore_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -115,5 +115,5 @@ func TestCheckSysTableCompatibility(t *testing.T) {
//
// The above variables are in the file br/pkg/restore/systable_restore.go
func TestMonitorTheSystemTableIncremental(t *testing.T) {
require.Equal(t, int64(209), session.CurrentBootstrapVersion)
require.Equal(t, int64(210), session.CurrentBootstrapVersion)
}
5 changes: 5 additions & 0 deletions pkg/executor/analyze_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -108,6 +108,11 @@ func TestAnalyzePartitionTableByConcurrencyInDynamic(t *testing.T) {
tk.MustExec("set @@tidb_partition_prune_mode='dynamic'")
tk.MustExec("use test")
tk.MustExec("create table t(id int) partition by hash(id) partitions 4")
tk.MustExec("select * from t where id = 0")
do, err := session.GetDomain(store)
require.NoError(t, err)
statsHandle := do.StatsHandle()
require.NoError(t, statsHandle.DumpColStatsUsageToKV())
testcases := []struct {
concurrency string
}{
Expand Down
12 changes: 6 additions & 6 deletions pkg/executor/distsql_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -381,7 +381,7 @@ func TestAdaptiveClosestRead(t *testing.T) {

tk.MustExec("create table t(id int primary key, s varchar(8), p varchar(16));")
tk.MustExec("insert into t values (1, '00000001', '0000000000000001'), (2, '00000003', '0000000000000002'), (3, '00000011', '0000000000000003');")
tk.MustExec("analyze table t;")
tk.MustExec("analyze table t all columns;")

tk.MustExec("set @@tidb_partition_prune_mode ='static';")
tk.MustExec("set tidb_replica_read = 'closest-adaptive';")
Expand All @@ -404,7 +404,7 @@ func TestAdaptiveClosestRead(t *testing.T) {
tk.MustExec("drop table if exists t")
tk.MustExec("create table t(id int primary key, s varchar(8), p varchar(16)) " + partitionDef)
tk.MustExec("insert into t values (1, '00000001', '0000000000000001'), (2, '00000003', '0000000000000002'), (3, '00000011', '0000000000000003'), (4, '00000044', '0000000000000004');")
tk.MustExec("analyze table t;")
tk.MustExec("analyze table t all columns;")
// estimate cost is 38
checkMetrics("select s from t where id >= 1 and id < 3;", 1, 0)
// estimate cost is 39 with 2 cop request
Expand All @@ -414,7 +414,7 @@ func TestAdaptiveClosestRead(t *testing.T) {
tk.MustExec("drop table if exists t;")
tk.MustExec("create table t (id int, s varchar(8), p varchar(8), key `idx_s_p`(`s`, `p`));")
tk.MustExec("insert into t values (1, 'test1000', '11111111'), (2, 'test2000', '11111111');")
tk.MustExec("analyze table t;")
tk.MustExec("analyze table t all columns;")
// avg row size = 27.91
checkMetrics("select p from t where s >= 'test' and s < 'test11'", 0, 1)
checkMetrics("select p from t where s >= 'test' and s < 'test22'", 1, 0)
Expand All @@ -424,7 +424,7 @@ func TestAdaptiveClosestRead(t *testing.T) {
tk.MustExec("drop table if exists t;")
tk.MustExec("create table t (v int, id int, p varchar(8), key `idx_id_p`(`id`, `p`)) " + partitionDef)
tk.MustExec("insert into t values (1, 1, '11111111'), (2, 2, '22222222'), (3, 3, '33333333'), (4, 4, '44444444');")
tk.MustExec("analyze table t;")
tk.MustExec("analyze table t all columns;")
// avg row size = 19
checkMetrics("select p from t where id >= 1 and id < 3", 1, 0)
checkMetrics("select p from t where id >= 2 and id < 4", 0, 2)
Expand All @@ -435,7 +435,7 @@ func TestAdaptiveClosestRead(t *testing.T) {
tk.MustExec("create table t (id int, s varchar(8), p varchar(50), key `idx_s`(`s`));")
str := "this_is_a_string_with_length_of_50________________"
tk.MustExec(fmt.Sprintf("insert into t values (1, 'test1000', '%s'), (2, 'test2000', '%s');", str, str))
tk.MustExec("analyze table t;")
tk.MustExec("analyze table t all columns;")
tk.MustExec("set tidb_adaptive_closest_read_threshold = 80;")
// IndexReader cost is 22, TableReader cost (1 row) is 67
checkMetrics("select/*+ FORCE_INDEX(t, idx_s) */ p from t where s >= 'test' and s < 'test11'", 0, 2)
Expand All @@ -447,7 +447,7 @@ func TestAdaptiveClosestRead(t *testing.T) {
// use int field to avoid the planer estimation with big random fluctuation.
tk.MustExec("create table t (id int, v bigint not null, s1 int not null, s2 int not null, key `idx_v_s1`(`s1`, `v`), key `idx_s2`(`s2`));")
tk.MustExec("insert into t values (1, 1, 1, 1), (2, 2, 2, 2), (3, 3, 3, 3);")
tk.MustExec("analyze table t;")
tk.MustExec("analyze table t all columns;")
tk.MustExec("set tidb_adaptive_closest_read_threshold = 30;")
// 2 IndexScan with cost 19/56, 2 TableReader with cost 32.5/65.
checkMetrics("select/* +USE_INDEX_MERGE(t) */ id from t use index(`idx_v_s1`) use index(idx_s2) where (s1 < 3 and v > 0) or s2 = 3;", 3, 1)
Expand Down
8 changes: 4 additions & 4 deletions pkg/executor/historical_stats_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -42,7 +42,7 @@ func TestRecordHistoryStatsAfterAnalyze(t *testing.T) {
tk.MustExec("set global tidb_enable_historical_stats = 0")
tk.MustExec("use test")
tk.MustExec("drop table if exists t")
tk.MustExec("create table t(a int, b varchar(10))")
tk.MustExec("create table t(a int, b varchar(10), index idx(a, b))")

h := dom.StatsHandle()
is := dom.InfoSchema()
Expand Down Expand Up @@ -103,7 +103,7 @@ func TestRecordHistoryStatsMetaAfterAnalyze(t *testing.T) {
tk.MustExec("set global tidb_enable_historical_stats = 0")
tk.MustExec("use test")
tk.MustExec("drop table if exists t")
tk.MustExec("create table t(a int, b int)")
tk.MustExec("create table t(a int, b int, index idx(a, b))")
tk.MustExec("analyze table test.t")

h := dom.StatsHandle()
Expand Down Expand Up @@ -162,7 +162,7 @@ func TestGCHistoryStatsAfterDropTable(t *testing.T) {
tk.MustExec("set global tidb_enable_historical_stats = 1")
tk.MustExec("use test")
tk.MustExec("drop table if exists t")
tk.MustExec("create table t(a int, b varchar(10))")
tk.MustExec("create table t(a int, b varchar(10), index idx(a, b))")
tk.MustExec("analyze table test.t")
is := dom.InfoSchema()
tableInfo, err := is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t"))
Expand Down Expand Up @@ -248,7 +248,7 @@ func TestGCOutdatedHistoryStats(t *testing.T) {
tk.MustExec("set global tidb_enable_historical_stats = 1")
tk.MustExec("use test")
tk.MustExec("drop table if exists t")
tk.MustExec("create table t(a int, b varchar(10))")
tk.MustExec("create table t(a int, b varchar(10), index idx(a, b))")
tk.MustExec("analyze table test.t")
is := dom.InfoSchema()
tableInfo, err := is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t"))
Expand Down
4 changes: 2 additions & 2 deletions pkg/executor/infoschema_reader_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -309,7 +309,7 @@ func TestForAnalyzeStatus(t *testing.T) {
tk.MustExec("insert into analyze_test values (1,2),(3,4)")

tk.MustQuery("select distinct TABLE_NAME from information_schema.analyze_status where TABLE_NAME='analyze_test'").Check([][]any{})
tk.MustExec("analyze table analyze_test")
tk.MustExec("analyze table analyze_test all columns")
tk.MustQuery("select distinct TABLE_NAME from information_schema.analyze_status where TABLE_NAME='analyze_test'").Check(testkit.Rows("analyze_test"))

// test the privilege of new user for information_schema.analyze_status
Expand All @@ -326,7 +326,7 @@ func TestForAnalyzeStatus(t *testing.T) {
// test the privilege of user with privilege of test.t1 for information_schema.analyze_status
tk.MustExec("create table t1 (a int, b int, index idx(a))")
tk.MustExec("insert into t1 values (1,2),(3,4)")
tk.MustExec("analyze table t1")
tk.MustExec("analyze table t1 all columns")
tk.MustQuery("show warnings").Check(testkit.Rows("Note 1105 Analyze use auto adjusted sample rate 1.000000 for table test.t1, reason to use this rate is \"use min(1, 110000/10000) as the sample-rate=1\"")) // 1 note.
require.NoError(t, dom.StatsHandle().LoadNeededHistograms())
tk.MustExec("CREATE ROLE r_t1 ;")
Expand Down
6 changes: 3 additions & 3 deletions pkg/executor/set_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -682,11 +682,11 @@ func TestSetVar(t *testing.T) {
require.Error(t, tk.ExecToErr("set global tidb_enable_column_tracking = -1"))

// test for tidb_analyze_column_options
tk.MustQuery("select @@tidb_analyze_column_options").Check(testkit.Rows("ALL"))
tk.MustExec("set global tidb_analyze_column_options = 'PREDICATE'")
tk.MustQuery("select @@tidb_analyze_column_options").Check(testkit.Rows("PREDICATE"))
tk.MustExec("set global tidb_analyze_column_options = 'all'")
tk.MustExec("set global tidb_analyze_column_options = 'ALL'")
tk.MustQuery("select @@tidb_analyze_column_options").Check(testkit.Rows("ALL"))
tk.MustExec("set global tidb_analyze_column_options = 'predicate'")
tk.MustQuery("select @@tidb_analyze_column_options").Check(testkit.Rows("PREDICATE"))
require.Error(t, tk.ExecToErr("set global tidb_analyze_column_options = 'UNKNOWN'"))

// test for tidb_ignore_prepared_cache_close_stmt
Expand Down
6 changes: 3 additions & 3 deletions pkg/executor/show_stats_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -36,7 +36,7 @@ func TestShowStatsMeta(t *testing.T) {
tk.MustExec("drop table if exists t, t1")
tk.MustExec("create table t (a int, b int)")
tk.MustExec("create table t1 (a int, b int)")
tk.MustExec("analyze table t, t1")
tk.MustExec("analyze table t, t1 all columns")
result := tk.MustQuery("show stats_meta")
result = result.Sort()
require.Len(t, result.Rows(), 2)
Expand Down Expand Up @@ -78,11 +78,11 @@ func TestShowStatsHistograms(t *testing.T) {
tk.MustExec("use test")
tk.MustExec("drop table if exists t")
tk.MustExec("create table t (a int, b int)")
tk.MustExec("analyze table t")
tk.MustExec("analyze table t all columns")
result := tk.MustQuery("show stats_histograms")
require.Len(t, result.Rows(), 2)
tk.MustExec("insert into t values(1,1)")
tk.MustExec("analyze table t")
tk.MustExec("analyze table t all columns")
result = tk.MustQuery("show stats_histograms").Sort()
require.Len(t, result.Rows(), 2)
require.Equal(t, "a", result.Rows()[0][3])
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 @@ -28,6 +28,7 @@ go_test(
"//pkg/statistics",
"//pkg/statistics/handle/autoanalyze/exec",
"//pkg/testkit",
"//pkg/testkit/analyzehelper",
"//pkg/util/dbterror/exeerrors",
"@com_github_pingcap_errors//:errors",
"@com_github_pingcap_failpoint//:failpoint",
Expand Down
2 changes: 1 addition & 1 deletion pkg/executor/test/analyzetest/analyze_bench_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -49,6 +49,6 @@ func BenchmarkAnalyzePartition(b *testing.B) {
tk.MustExec(insertStr)
b.ResetTimer()
for i := 0; i < b.N; i++ {
tk.MustExec("analyze table t")
tk.MustExec("analyze table t all columns")
}
}
31 changes: 23 additions & 8 deletions pkg/executor/test/analyzetest/analyze_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -41,6 +41,7 @@ import (
"github.com/pingcap/tidb/pkg/statistics"
"github.com/pingcap/tidb/pkg/statistics/handle/autoanalyze/exec"
"github.com/pingcap/tidb/pkg/testkit"
"github.com/pingcap/tidb/pkg/testkit/analyzehelper"
"github.com/pingcap/tidb/pkg/util/dbterror/exeerrors"
"github.com/stretchr/testify/require"
)
Expand All @@ -63,6 +64,7 @@ PARTITION BY RANGE ( a ) (
for i := 1; i < 21; i++ {
tk.MustExec(fmt.Sprintf(`insert into t values (%d, %d, "hello")`, i, i))
}
analyzehelper.TriggerPredicateColumnsCollection(t, tk, store, "t", "c")
tk.MustExec("analyze table t")

is := tk.Session().(sessionctx.Context).GetInfoSchema().(infoschema.InfoSchema)
Expand Down Expand Up @@ -205,6 +207,7 @@ func TestAnalyzeTooLongColumns(t *testing.T) {
tk.MustExec(fmt.Sprintf("insert into t values ('%s')", value))

tk.MustExec("set @@session.tidb_analyze_skip_column_types = ''")
analyzehelper.TriggerPredicateColumnsCollection(t, tk, store, "t", "a")
tk.MustExec("analyze table t")
is := tk.Session().(sessionctx.Context).GetInfoSchema().(infoschema.InfoSchema)
table, err := is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t"))
Expand Down Expand Up @@ -505,14 +508,20 @@ func TestAdjustSampleRateNote(t *testing.T) {
result := tk.MustQuery("show stats_meta where table_name = 't'")
require.Equal(t, "220000", result.Rows()[0][5])
tk.MustExec("analyze table t")
tk.MustQuery("show warnings").Check(testkit.Rows("Note 1105 Analyze use auto adjusted sample rate 0.500000 for table test.t, reason to use this rate is \"use min(1, 110000/220000) as the sample-rate=0.5\""))
tk.MustQuery("show warnings").Check(testkit.Rows(
"Warning 1105 No predicate column has been collected yet for table test.t, so only indexes and the columns composing the indexes will be analyzed",
"Note 1105 Analyze use auto adjusted sample rate 0.500000 for table test.t, reason to use this rate is \"use min(1, 110000/220000) as the sample-rate=0.5\"",
))
tk.MustExec("insert into t values(1),(1),(1)")
require.NoError(t, statsHandle.DumpStatsDeltaToKV(true))
require.NoError(t, statsHandle.Update(is))
result = tk.MustQuery("show stats_meta where table_name = 't'")
require.Equal(t, "3", result.Rows()[0][5])
tk.MustExec("analyze table t")
tk.MustQuery("show warnings").Check(testkit.Rows("Note 1105 Analyze use auto adjusted sample rate 1.000000 for table test.t, reason to use this rate is \"use min(1, 110000/3) as the sample-rate=1\""))
tk.MustQuery("show warnings").Check(testkit.Rows(
"Warning 1105 No predicate column has been collected yet for table test.t, so only indexes and the columns composing the indexes will be analyzed",
"Note 1105 Analyze use auto adjusted sample rate 1.000000 for table test.t, reason to use this rate is \"use min(1, 110000/3) as the sample-rate=1\"",
))
}

func TestAnalyzeIndex(t *testing.T) {
Expand Down Expand Up @@ -620,7 +629,7 @@ func TestAnalyzeSamplingWorkPanic(t *testing.T) {
tk := testkit.NewTestKit(t, store)
tk.MustExec("use test")
tk.MustExec("set @@session.tidb_analyze_version = 2")
tk.MustExec("create table t(a int)")
tk.MustExec("create table t(a int, index idx(a))")
tk.MustExec("insert into t values(1), (2), (3), (4), (5), (6), (7), (8), (9), (10), (11), (12)")
tk.MustExec("split table t between (-9223372036854775808) and (9223372036854775807) regions 12")

Expand All @@ -644,6 +653,7 @@ func TestSmallTableAnalyzeV2(t *testing.T) {
tk.MustExec("use test")
tk.MustExec("set @@session.tidb_analyze_version = 2")
tk.MustExec("create table small_table_inject_pd(a int)")
analyzehelper.TriggerPredicateColumnsCollection(t, tk, store, "small_table_inject_pd", "a")
tk.MustExec("insert into small_table_inject_pd values(1), (2), (3), (4), (5)")
tk.MustExec("analyze table small_table_inject_pd")
tk.MustQuery("show warnings").Check(testkit.Rows("Note 1105 Analyze use auto adjusted sample rate 1.000000 for table test.small_table_inject_pd, reason to use this rate is \"use min(1, 110000/10000) as the sample-rate=1\""))
Expand All @@ -656,6 +666,7 @@ create table small_table_inject_pd_with_partition(
partition p2 values less than (15)
)`)
tk.MustExec("insert into small_table_inject_pd_with_partition values(1), (6), (11)")
analyzehelper.TriggerPredicateColumnsCollection(t, tk, store, "small_table_inject_pd_with_partition", "a")
tk.MustExec("analyze table small_table_inject_pd_with_partition")
tk.MustQuery("show warnings").Check(testkit.Rows(
"Note 1105 Analyze use auto adjusted sample rate 1.000000 for table test.small_table_inject_pd_with_partition's partition p0, reason to use this rate is \"use min(1, 110000/10000) as the sample-rate=1\"",
Expand Down Expand Up @@ -703,6 +714,7 @@ func TestSavedAnalyzeOptions(t *testing.T) {
tk.MustExec("set @@session.tidb_stats_load_sync_wait = 20000") // to stabilise test
tk.MustExec("create table t(a int, b int, c int, primary key(a), key idx(b))")
tk.MustExec("insert into t values (1,1,1),(2,1,2),(3,1,3),(4,1,4),(5,1,5),(6,1,6),(7,7,7),(8,8,8),(9,9,9)")
analyzehelper.TriggerPredicateColumnsCollection(t, tk, store, "t", "c")

h := dom.StatsHandle()
oriLease := h.Lease()
Expand Down Expand Up @@ -1726,7 +1738,7 @@ func TestAnalyzeColumnsAfterAnalyzeAll(t *testing.T) {
require.NoError(t, err)
tblID := tbl.Meta().ID

tk.MustExec("analyze table t with 2 topn, 2 buckets")
tk.MustExec("analyze table t all columns with 2 topn, 2 buckets")
tk.MustQuery(fmt.Sprintf("select modify_count, count from mysql.stats_meta where table_id = %d", tblID)).Sort().Check(
testkit.Rows("0 6"))
tk.MustQuery("show stats_topn where db_name = 'test' and table_name = 't'").Sort().Check(
Expand All @@ -1750,10 +1762,6 @@ func TestAnalyzeColumnsAfterAnalyzeAll(t *testing.T) {
case model.ColumnList:
tk.MustExec("analyze table t columns b with 2 topn, 2 buckets")
case model.PredicateColumns:
originalVal := tk.MustQuery("select @@tidb_enable_column_tracking").Rows()[0][0].(string)
defer func() {
tk.MustExec(fmt.Sprintf("set global tidb_enable_column_tracking = %v", originalVal))
}()
tk.MustExec("select * from t where b > 1")
require.NoError(t, h.DumpColStatsUsageToKV())
rows := tk.MustQuery("show column_stats_usage where db_name = 'test' and table_name = 't' and last_used_at is not null").Rows()
Expand Down Expand Up @@ -1792,6 +1800,7 @@ func TestAnalyzeSampleRateReason(t *testing.T) {
tk.MustExec("drop table if exists t")
tk.MustExec("create table t (a int, b int)")
require.NoError(t, dom.StatsHandle().DumpStatsDeltaToKV(true))
analyzehelper.TriggerPredicateColumnsCollection(t, tk, store, "t", "a", "b")

tk.MustExec(`analyze table t`)
tk.MustQuery(`show warnings`).Sort().Check(testkit.Rows(
Expand Down Expand Up @@ -1890,6 +1899,7 @@ func testKillAutoAnalyze(t *testing.T, ver int) {
tk.MustExec("drop table if exists t")
tk.MustExec("create table t (a int, b int)")
tk.MustExec("insert into t values (1,2), (3,4)")
analyzehelper.TriggerPredicateColumnsCollection(t, tk, store, "t", "a", "b")
is := dom.InfoSchema()
h := dom.StatsHandle()
require.NoError(t, h.DumpStatsDeltaToKV(true))
Expand Down Expand Up @@ -1973,6 +1983,7 @@ func TestKillAutoAnalyzeIndex(t *testing.T) {
tk.MustExec("drop table if exists t")
tk.MustExec("create table t (a int, b int)")
tk.MustExec("insert into t values (1,2), (3,4)")
analyzehelper.TriggerPredicateColumnsCollection(t, tk, store, "t", "a", "b")
is := dom.InfoSchema()
h := dom.StatsHandle()
require.NoError(t, h.DumpStatsDeltaToKV(true))
Expand Down Expand Up @@ -2420,6 +2431,7 @@ PARTITION BY RANGE ( a ) (
tk.MustExec(createTable)
tk.MustExec("insert into t values (1,1,1,1),(2,1,2,2),(3,1,3,3),(4,1,4,4),(5,1,5,5),(6,1,6,6),(7,7,7,7),(8,8,8,8),(9,9,9,9)")
tk.MustExec("insert into t values (10,10,10,10),(11,11,11,11),(12,12,12,12),(13,13,13,13),(14,14,14,14)")
analyzehelper.TriggerPredicateColumnsCollection(t, tk, store, "t", "a", "b", "c", "d")
h := dom.StatsHandle()
oriLease := h.Lease()
h.SetLease(1)
Expand Down Expand Up @@ -2476,6 +2488,7 @@ PARTITION BY RANGE ( a ) (
tk.MustExec(createTable)
tk.MustExec("insert into t values (1,1,1,1),(2,1,2,2),(3,1,3,3),(4,1,4,4),(5,1,5,5),(6,1,6,6),(7,7,7,7),(8,8,8,8),(9,9,9,9)")
tk.MustExec("insert into t values (10,10,10,10),(11,11,11,11),(12,12,12,12),(13,13,13,13),(14,14,14,14)")
analyzehelper.TriggerPredicateColumnsCollection(t, tk, store, "t", "a", "b", "c", "d")
h := dom.StatsHandle()
oriLease := h.Lease()
h.SetLease(1)
Expand Down Expand Up @@ -2705,6 +2718,7 @@ func TestAutoAnalyzeAwareGlobalVariableChange(t *testing.T) {
tk.MustExec("set @@global.tidb_enable_analyze_snapshot = 1")
tk.MustExec("set @@global.tidb_analyze_version = 2")
tk.MustExec("create table t(a int)")
analyzehelper.TriggerPredicateColumnsCollection(t, tk, store, "t", "a")
h := dom.StatsHandle()
require.NoError(t, h.HandleDDLEvent(<-h.DDLEventCh()))
tbl, err := dom.InfoSchema().TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t"))
Expand Down Expand Up @@ -3094,6 +3108,7 @@ func TestAnalyzePartitionVerify(t *testing.T) {
}
insertStr += ";"
tk.MustExec(insertStr)
analyzehelper.TriggerPredicateColumnsCollection(t, tk, store, "t", "a", "b", "c")
tk.MustExec("analyze table t")

result := tk.MustQuery("show stats_histograms where Db_name='test'").Sort()
Expand Down
Loading

0 comments on commit e5a8a23

Please sign in to comment.