diff --git a/cmd/explaintest/r/generated_columns.result b/cmd/explaintest/r/generated_columns.result index eea618c0d20dd..761dfc6053354 100644 --- a/cmd/explaintest/r/generated_columns.result +++ b/cmd/explaintest/r/generated_columns.result @@ -1,4 +1,4 @@ -set @@tidb_partition_prune_mode='dynamic-only'; +set @@tidb_partition_prune_mode='dynamic'; DROP TABLE IF EXISTS person; CREATE TABLE person ( id INT NOT NULL AUTO_INCREMENT PRIMARY KEY, diff --git a/cmd/explaintest/t/generated_columns.test b/cmd/explaintest/t/generated_columns.test index a2f4a207fd824..82dfcf4d1d8c8 100644 --- a/cmd/explaintest/t/generated_columns.test +++ b/cmd/explaintest/t/generated_columns.test @@ -2,7 +2,7 @@ -- Most of the cases are ported from other tests to make sure generated columns behaves the same. -- Stored generated columns as indices -set @@tidb_partition_prune_mode='dynamic-only'; +set @@tidb_partition_prune_mode='dynamic'; DROP TABLE IF EXISTS person; CREATE TABLE person ( diff --git a/executor/analyze.go b/executor/analyze.go index 22021bc49fa18..1bb8dc00b9270 100644 --- a/executor/analyze.go +++ b/executor/analyze.go @@ -99,7 +99,7 @@ func (e *AnalyzeExec) Next(ctx context.Context, req *chunk.Chunk) error { pruneMode := variable.PartitionPruneMode(e.ctx.GetSessionVars().PartitionPruneMode.Load()) // needGlobalStats used to indicate whether we should merge the partition-level stats to global-level stats. - needGlobalStats := pruneMode == variable.DynamicOnly || pruneMode == variable.StaticButPrepareDynamic + needGlobalStats := pruneMode == variable.Dynamic type globalStatsKey struct { tableID int64 indexID int64 @@ -810,7 +810,7 @@ func (e *AnalyzeFastExec) calculateEstimateSampleStep() (err error) { sqlexec.MustFormatSQL(sql, "select count(*) from %n.%n", dbInfo.Name.L, e.tblInfo.Name.L) pruneMode := variable.PartitionPruneMode(e.ctx.GetSessionVars().PartitionPruneMode.Load()) - if pruneMode != variable.DynamicOnly && e.tblInfo.ID != e.tableID.GetStatisticsID() { + if pruneMode != variable.Dynamic && e.tblInfo.ID != e.tableID.GetStatisticsID() { for _, definition := range e.tblInfo.Partition.Definitions { if definition.ID == e.tableID.GetStatisticsID() { sqlexec.MustFormatSQL(sql, " partition(%n)", definition.Name.L) diff --git a/executor/analyze_test.go b/executor/analyze_test.go index 60bdc0ac9972c..5dc3403e7967e 100644 --- a/executor/analyze_test.go +++ b/executor/analyze_test.go @@ -52,7 +52,7 @@ var _ = Suite(&testFastAnalyze{}) func (s *testSuite1) TestAnalyzePartition(c *C) { tk := testkit.NewTestKit(c, s.store) - testkit.WithPruneMode(tk, variable.StaticOnly, func() { + testkit.WithPruneMode(tk, variable.Static, func() { tk.MustExec("use test") tk.MustExec("drop table if exists t") createTable := `CREATE TABLE t (a int, b int, c varchar(10), primary key(a), index idx(b)) @@ -464,7 +464,7 @@ func (s *testFastAnalyze) TestFastAnalyze(c *C) { "test t2 a 0 0 1 1 0 0 0", "test t2 a 0 1 2 1 18446744073709551615 18446744073709551615 0")) - tk.MustExec(`set @@tidb_partition_prune_mode='` + string(variable.StaticOnly) + `'`) + tk.MustExec(`set @@tidb_partition_prune_mode='` + string(variable.Static) + `'`) tk.MustExec(`create table t3 (id int, v int, primary key(id), index k(v)) partition by hash (id) partitions 4`) tk.MustExec(`insert into t3 values(1, 1), (2, 2), (5, 1), (9, 3), (13, 3), (17, 5), (3, 0)`) tk.MustExec(`analyze table t3`) @@ -472,7 +472,7 @@ func (s *testFastAnalyze) TestFastAnalyze(c *C) { "IndexReader 2.00 root index:IndexRangeScan", "└─IndexRangeScan 2.00 cop[tikv] table:t3, partition:p1, index:k(v) range:[3,3], keep order:false", )) - tk.MustExec(`set @@tidb_partition_prune_mode='` + string(variable.DynamicOnly) + `'`) + tk.MustExec(`set @@tidb_partition_prune_mode='` + string(variable.Dynamic) + `'`) } func (s *testSuite1) TestIssue15993(c *C) { diff --git a/executor/index_lookup_merge_join_test.go b/executor/index_lookup_merge_join_test.go index 298599ad8e39a..4f2bd6e1488b9 100644 --- a/executor/index_lookup_merge_join_test.go +++ b/executor/index_lookup_merge_join_test.go @@ -109,7 +109,7 @@ func (s *testSuite9) TestIssue20137(c *C) { func (s *testSuiteWithData) TestIndexJoinOnSinglePartitionTable(c *C) { // For issue 19145 tk := testkit.NewTestKitWithInit(c, s.store) - for _, val := range []string{string(variable.StaticOnly), string(variable.DynamicOnly)} { + for _, val := range []string{string(variable.Static), string(variable.Dynamic)} { tk.MustExec("set @@tidb_partition_prune_mode= '" + val + "'") tk.MustExec("drop table if exists t1, t2") tk.MustExec("create table t1 (c_int int, c_str varchar(40), primary key (c_int) ) partition by range (c_int) ( partition p0 values less than (10), partition p1 values less than maxvalue )") diff --git a/executor/infoschema_reader_test.go b/executor/infoschema_reader_test.go index 085686b15c9f3..8b1861d65d34e 100644 --- a/executor/infoschema_reader_test.go +++ b/executor/infoschema_reader_test.go @@ -418,7 +418,7 @@ func (s *testInfoschemaTableSerialSuite) TestPartitionsTable(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("USE test;") - testkit.WithPruneMode(tk, variable.StaticOnly, func() { + testkit.WithPruneMode(tk, variable.Static, func() { c.Assert(h.RefreshVars(), IsNil) 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));`) diff --git a/executor/partition_table_test.go b/executor/partition_table_test.go index 3f6f896757082..c2deef9a6f632 100644 --- a/executor/partition_table_test.go +++ b/executor/partition_table_test.go @@ -151,7 +151,7 @@ func (s *partitionTableSuite) TestPartitionReaderUnderApply(c *C) { "5 naughty swartz 9.524000")) // For issue 19450 release-4.0 - tk.MustExec(`set @@tidb_partition_prune_mode='` + string(variable.StaticOnly) + `'`) + tk.MustExec(`set @@tidb_partition_prune_mode='` + string(variable.Static) + `'`) tk.MustQuery("select * from t1 where c_decimal in (select c_decimal from t2 where t1.c_int = t2.c_int or t1.c_int = t2.c_int and t1.c_str > t2.c_str)").Check(testkit.Rows( "1 romantic robinson 4.436000", "2 stoic chaplygin 9.826000", @@ -171,7 +171,7 @@ PRIMARY KEY (pk1,pk2)) partition by hash(pk2) partitions 4;`) tk.MustExec("create table coverage_dt (pk1 varchar(35), pk2 int)") tk.MustExec("insert into coverage_rr values ('ios', 3, 2),('android', 4, 7),('linux',5,1)") tk.MustExec("insert into coverage_dt values ('apple',3),('ios',3),('linux',5)") - tk.MustExec("set @@tidb_partition_prune_mode = 'dynamic-only'") + tk.MustExec("set @@tidb_partition_prune_mode = 'dynamic'") tk.MustQuery("select /*+ INL_JOIN(dt, rr) */ * from coverage_dt dt join coverage_rr rr on (dt.pk1 = rr.pk1 and dt.pk2 = rr.pk2);").Sort().Check(testkit.Rows("ios 3 ios 3 2", "linux 5 linux 5 1")) tk.MustQuery("select /*+ INL_MERGE_JOIN(dt, rr) */ * from coverage_dt dt join coverage_rr rr on (dt.pk1 = rr.pk1 and dt.pk2 = rr.pk2);").Sort().Check(testkit.Rows("ios 3 ios 3 2", "linux 5 linux 5 1")) } diff --git a/executor/show_stats_test.go b/executor/show_stats_test.go index f21ada8ea2b1c..9540324931d25 100644 --- a/executor/show_stats_test.go +++ b/executor/show_stats_test.go @@ -175,7 +175,7 @@ func (s *testShowStatsSuite) TestShowStatsHasNullValue(c *C) { func (s *testShowStatsSuite) TestShowPartitionStats(c *C) { tk := testkit.NewTestKit(c, s.store) - testkit.WithPruneMode(tk, variable.StaticOnly, func() { + testkit.WithPruneMode(tk, variable.Static, func() { tk.MustExec("set @@session.tidb_enable_table_partition=1") tk.MustExec("use test") tk.MustExec("drop table if exists t") diff --git a/go.sum b/go.sum index 2c8cc824d8869..542ac0049dc22 100644 --- a/go.sum +++ b/go.sum @@ -463,7 +463,6 @@ github.com/russross/blackfriday v1.5.2/go.mod h1:JO/DiYxRf+HjHt06OyowR9PTA263kcR github.com/russross/blackfriday/v2 v2.0.1/go.mod h1:+Rmxgy9KzJVeS9/2gXHxylqXiyQDYRxCVz55jmeOWTM= github.com/sasha-s/go-deadlock v0.2.0/go.mod h1:StQn567HiB1fF2yJ44N9au7wOhrPS3iZqiDbRupzT10= github.com/satori/go.uuid v1.2.0/go.mod h1:dA0hQrYB0VpLJoorglMZABFdXlWrHn1NEOzdhQKdks0= -github.com/sergi/go-diff v1.0.1-0.20180205163309-da645544ed44 h1:tB9NOR21++IjLyVx3/PCPhWMwqGNCMQEH96A6dMZ/gc= github.com/sergi/go-diff v1.0.1-0.20180205163309-da645544ed44/go.mod h1:0CfEIISq7TuYL3j771MWULgwwjU+GofnZX9QAmXWZgo= github.com/shirou/gopsutil v2.19.10+incompatible/go.mod h1:5b4v6he4MtMOwMlS0TUMTu2PcXUg8+E1lC7eC3UO/RA= github.com/shirou/gopsutil v3.20.12+incompatible h1:6VEGkOXP/eP4o2Ilk8cSsX0PhOEfX6leqAnD+urrp9M= diff --git a/planner/core/cbo_test.go b/planner/core/cbo_test.go index 06b5f2a76da18..0eeb8cf2f9465 100644 --- a/planner/core/cbo_test.go +++ b/planner/core/cbo_test.go @@ -355,7 +355,7 @@ func (s *testAnalyzeSuite) TestAnalyze(c *C) { testKit.MustExec("create table t3 (a int, b int)") testKit.MustExec("create index a on t3 (a)") - testKit.MustExec("set @@tidb_partition_prune_mode = 'static-only';") + testKit.MustExec("set @@tidb_partition_prune_mode = 'static';") testKit.MustExec("create table t4 (a int, b int) partition by range (a) (partition p1 values less than (2), partition p2 values less than (3))") testKit.MustExec("create index a on t4 (a)") testKit.MustExec("create index b on t4 (b)") diff --git a/planner/core/integration_test.go b/planner/core/integration_test.go index db274684b830e..82018d67c9591 100644 --- a/planner/core/integration_test.go +++ b/planner/core/integration_test.go @@ -806,7 +806,7 @@ func (s *testIntegrationSerialSuite) TestIsolationReadDoNotFilterSystemDB(c *C) func (s *testIntegrationSuite) TestPartitionTableStats(c *C) { tk := testkit.NewTestKit(c, s.store) { - tk.MustExec(`set @@tidb_partition_prune_mode='` + string(variable.StaticOnly) + `'`) + tk.MustExec(`set @@tidb_partition_prune_mode='` + string(variable.Static) + `'`) tk.MustExec("use test") tk.MustExec("drop table if exists t") tk.MustExec("create table t(a int, b int)partition by range columns(a)(partition p0 values less than (10), partition p1 values less than(20), partition p2 values less than(30));") @@ -1147,7 +1147,7 @@ func (s *testIntegrationSuite) TestApproxCountDistinctInPartitionTable(c *C) { tk.MustExec("create table t(a int(11), b int) partition by range (a) (partition p0 values less than (3), partition p1 values less than maxvalue);") tk.MustExec("insert into t values(1, 1), (2, 1), (3, 1), (4, 2), (4, 2)") tk.MustExec("set session tidb_opt_agg_push_down=1") - tk.MustExec(`set @@tidb_partition_prune_mode='` + string(variable.StaticOnly) + `'`) + tk.MustExec(`set @@tidb_partition_prune_mode='` + string(variable.Static) + `'`) tk.MustQuery("explain format = 'brief' select approx_count_distinct(a), b from t group by b order by b desc").Check(testkit.Rows("Sort 16000.00 root test.t.b:desc", "└─HashAgg 16000.00 root group by:test.t.b, funcs:approx_count_distinct(Column#5)->Column#4, funcs:firstrow(Column#6)->test.t.b", " └─PartitionUnion 16000.00 root ", @@ -1522,7 +1522,7 @@ func (s *testIntegrationSuite) TestOptimizeHintOnPartitionTable(c *C) { } } - tk.MustExec(`set @@tidb_partition_prune_mode='` + string(variable.StaticOnly) + `'`) + tk.MustExec(`set @@tidb_partition_prune_mode='` + string(variable.Static) + `'`) var input []string var output []struct { diff --git a/planner/core/physical_plan_test.go b/planner/core/physical_plan_test.go index 8f5007b2889ca..c33199ef377ec 100644 --- a/planner/core/physical_plan_test.go +++ b/planner/core/physical_plan_test.go @@ -536,7 +536,7 @@ func (s *testPlanSuite) TestIndexJoinUnionScan(c *C) { tk.MustExec("create table t (a int primary key, b int, index idx(a))") tk.MustExec("create table tt (a int primary key) partition by range (a) (partition p0 values less than (100), partition p1 values less than (200))") - tk.MustExec(`set @@tidb_partition_prune_mode='` + string(variable.StaticOnly) + `'`) + tk.MustExec(`set @@tidb_partition_prune_mode='` + string(variable.Static) + `'`) s.testData.GetTestCases(c, &input, &output) for i, ts := range input { @@ -1101,7 +1101,7 @@ func (s *testPlanSuite) doTestPushdownDistinct(c *C, vars, input []string, outpu tk.MustExec(fmt.Sprintf("set session %s=1", variable.TiDBHashAggPartialConcurrency)) tk.MustExec(fmt.Sprintf("set session %s=1", variable.TiDBHashAggFinalConcurrency)) - tk.MustExec(`set @@tidb_partition_prune_mode='` + string(variable.StaticOnly) + `'`) + tk.MustExec(`set @@tidb_partition_prune_mode='` + string(variable.Static) + `'`) for _, v := range vars { tk.MustExec(v) @@ -1671,7 +1671,7 @@ func (s *testPlanSuite) TestNthPlanHintWithExplain(c *C) { _, err = se.Execute(ctx, "insert into tt values (1, 1), (2, 2), (3, 4)") c.Assert(err, IsNil) - tk.MustExec(`set @@tidb_partition_prune_mode='` + string(variable.StaticOnly) + `'`) + tk.MustExec(`set @@tidb_partition_prune_mode='` + string(variable.Static) + `'`) var input []string var output []struct { diff --git a/planner/core/plan_test.go b/planner/core/plan_test.go index c7dc88607892b..43b599ac7eaa3 100644 --- a/planner/core/plan_test.go +++ b/planner/core/plan_test.go @@ -109,7 +109,7 @@ func (s *testPlanNormalize) TestPreferRangeScan(c *C) { func (s *testPlanNormalize) TestNormalizedPlan(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") - tk.MustExec("set @@tidb_partition_prune_mode='static-only';") + tk.MustExec("set @@tidb_partition_prune_mode='static';") tk.MustExec("drop table if exists t1,t2,t3,t4") tk.MustExec("create table t1 (a int key,b int,c int, index (b));") tk.MustExec("create table t2 (a int key,b int,c int, index (b));") @@ -528,7 +528,7 @@ func (s *testPlanNormalize) BenchmarkEncodePlan(c *C) { tk.MustExec("use test") tk.MustExec("drop table if exists th") tk.MustExec("set @@session.tidb_enable_table_partition = 1") - tk.MustExec(`set @@tidb_partition_prune_mode='` + string(variable.StaticOnly) + `'`) + tk.MustExec(`set @@tidb_partition_prune_mode='` + string(variable.Static) + `'`) tk.MustExec("create table th (i int, a int,b int, c int, index (a)) partition by hash (a) partitions 8192;") tk.MustExec("set @@tidb_slow_log_threshold=200000") diff --git a/planner/core/planbuilder.go b/planner/core/planbuilder.go index 8eaf92de0e1a5..b470b7985071c 100644 --- a/planner/core/planbuilder.go +++ b/planner/core/planbuilder.go @@ -1637,7 +1637,7 @@ func getPhysicalIDsAndPartitionNames(tblInfo *model.TableInfo, partitionNames [] func (b *PlanBuilder) buildAnalyzeTable(as *ast.AnalyzeTableStmt, opts map[ast.AnalyzeOptionType]uint64, version int) (Plan, error) { p := &Analyze{Opts: opts} pruneMode := variable.PartitionPruneMode(b.ctx.GetSessionVars().PartitionPruneMode.Load()) - if len(as.PartitionNames) > 0 && pruneMode == variable.DynamicOnly { + if len(as.PartitionNames) > 0 && pruneMode == variable.Dynamic { logutil.BgLogger().Info("analyze partition didn't affect in dynamic-prune-mode", zap.String("partitions", as.PartitionNames[0].L)) return p, nil } @@ -1714,7 +1714,7 @@ func (b *PlanBuilder) buildAnalyzeIndex(as *ast.AnalyzeTableStmt, opts map[ast.A p := &Analyze{Opts: opts} tblInfo := as.TableNames[0].TableInfo pruneMode := variable.PartitionPruneMode(b.ctx.GetSessionVars().PartitionPruneMode.Load()) - if len(as.PartitionNames) > 0 && pruneMode == variable.DynamicOnly { + if len(as.PartitionNames) > 0 && pruneMode == variable.Dynamic { logutil.BgLogger().Info("analyze partition didn't affect in dynamic-prune-mode", zap.String("table", tblInfo.Name.L), zap.String("partitions", as.PartitionNames[0].L)) return p, nil } @@ -1779,7 +1779,7 @@ func (b *PlanBuilder) buildAnalyzeAllIndex(as *ast.AnalyzeTableStmt, opts map[as p := &Analyze{Opts: opts} tblInfo := as.TableNames[0].TableInfo pruneMode := variable.PartitionPruneMode(b.ctx.GetSessionVars().PartitionPruneMode.Load()) - if len(as.PartitionNames) > 0 && pruneMode == variable.DynamicOnly { + if len(as.PartitionNames) > 0 && pruneMode == variable.Dynamic { logutil.BgLogger().Info("analyze partition didn't affect in dynamic-prune-mode", zap.String("table", tblInfo.Name.L), zap.String("partitions", as.PartitionNames[0].L)) return p, nil } diff --git a/planner/core/prepare_test.go b/planner/core/prepare_test.go index d49db9b693046..fc71286daeefe 100644 --- a/planner/core/prepare_test.go +++ b/planner/core/prepare_test.go @@ -445,7 +445,7 @@ func (s *testPrepareSerialSuite) TestPrepareCacheForPartition(c *C) { c.Assert(err, IsNil) tk.MustExec("use test") - for _, val := range []string{string(variable.StaticOnly), string(variable.DynamicOnly)} { + for _, val := range []string{string(variable.Static), string(variable.Dynamic)} { tk.MustExec("set @@tidb_partition_prune_mode = '" + val + "'") // Test for PointGet and IndexRead. tk.MustExec("drop table if exists t_index_read") diff --git a/session/bootstrap.go b/session/bootstrap.go index 284acb75f8001..7672d3d6c8c10 100644 --- a/session/bootstrap.go +++ b/session/bootstrap.go @@ -1549,10 +1549,10 @@ func doDMLWorks(s Session) { vVal = strconv.Itoa(variable.DefTiDBRowFormatV2) } if v.Name == variable.TiDBPartitionPruneMode { - vVal = string(variable.StaticOnly) + vVal = string(variable.Static) if flag.Lookup("test.v") != nil || flag.Lookup("check.v") != nil || config.CheckTableBeforeDrop { // enable Dynamic Prune by default in test case. - vVal = string(variable.DynamicOnly) + vVal = string(variable.Dynamic) } } if v.Name == variable.TiDBEnableChangeMultiSchema { diff --git a/session/session.go b/session/session.go index c578fa8bb1337..0edc6ec4dc9c9 100644 --- a/session/session.go +++ b/session/session.go @@ -1083,12 +1083,6 @@ func (s *session) SetGlobalSysVar(name, value string) error { return err } } - if name == variable.TiDBPartitionPruneMode && value == string(variable.DynamicOnly) { - err := s.ensureFullGlobalStats() - if err != nil { - return err - } - } var sVal string var err error sVal, err = variable.ValidateSetSystemVar(s.sessionVars, name, value, variable.ScopeGlobal) @@ -1187,24 +1181,6 @@ func (s *session) getTiDBTableValue(name, val string) (string, error) { return validatedVal, nil } -func (s *session) ensureFullGlobalStats() error { - stmt, err := s.ParseWithParams(context.TODO(), `select count(1) from information_schema.tables t where t.create_options = 'partitioned' - and not exists (select 1 from mysql.stats_meta m where m.table_id = t.tidb_table_id)`) - if err != nil { - return err - } - rows, _, err := s.ExecRestrictedStmt(context.TODO(), stmt) - if err != nil { - return err - } - row := rows[0] - count := row.GetInt64(0) - if count > 0 { - return errors.New("need analyze all partition table in 'static-collect-dynamic' mode before switch to 'dynamic-only'") - } - return nil -} - func (s *session) ParseSQL(ctx context.Context, sql, charset, collation string) ([]ast.StmtNode, []error, error) { if span := opentracing.SpanFromContext(ctx); span != nil && span.Tracer() != nil { span1 := span.Tracer().StartSpan("session.ParseSQL", opentracing.ChildOf(span.Context())) diff --git a/sessionctx/variable/session.go b/sessionctx/variable/session.go index 605a68cd9b575..8ddc65f3df91d 100644 --- a/sessionctx/variable/session.go +++ b/sessionctx/variable/session.go @@ -835,7 +835,7 @@ func (s *SessionVars) CheckAndGetTxnScope() string { // UseDynamicPartitionPrune indicates whether use new dynamic partition prune. func (s *SessionVars) UseDynamicPartitionPrune() bool { - return PartitionPruneMode(s.PartitionPruneMode.Load()) == DynamicOnly + return PartitionPruneMode(s.PartitionPruneMode.Load()) == Dynamic } // BuildParserConfig generate parser.ParserConfig for initial parser @@ -850,24 +850,43 @@ func (s *SessionVars) BuildParserConfig() parser.ParserConfig { type PartitionPruneMode string const ( - // StaticOnly indicates only prune at plan phase. + // Static indicates only prune at plan phase. + Static PartitionPruneMode = "static" + // Dynamic indicates only prune at execute phase. + Dynamic PartitionPruneMode = "dynamic" + + // Don't use out-of-date mode. + + // StaticOnly is out-of-date. StaticOnly PartitionPruneMode = "static-only" - // DynamicOnly indicates only prune at execute phase. + // DynamicOnly is out-of-date. DynamicOnly PartitionPruneMode = "dynamic-only" - // StaticButPrepareDynamic indicates prune at plan phase but collect stats need for dynamic prune. + // StaticButPrepareDynamic is out-of-date. StaticButPrepareDynamic PartitionPruneMode = "static-collect-dynamic" ) // Valid indicate PruneMode is validated. func (p PartitionPruneMode) Valid() bool { switch p { - case StaticOnly, StaticButPrepareDynamic, DynamicOnly: + case Static, Dynamic, StaticOnly, DynamicOnly: return true default: return false } } +// Update updates out-of-date PruneMode. +func (p PartitionPruneMode) Update() PartitionPruneMode { + switch p { + case StaticOnly, StaticButPrepareDynamic: + return Static + case DynamicOnly: + return Dynamic + default: + return p + } +} + // PreparedParams contains the parameters of the current prepared statement when executing it. type PreparedParams []types.Datum diff --git a/sessionctx/variable/sysvar.go b/sessionctx/variable/sysvar.go index 6e10e7678cda7..1121ab9ab3833 100644 --- a/sessionctx/variable/sysvar.go +++ b/sessionctx/variable/sysvar.go @@ -745,11 +745,12 @@ var defaultSysVars = []*SysVar{ {Scope: ScopeSession, Name: TiDBEnableCollectExecutionInfo, Value: BoolToOnOff(DefTiDBEnableCollectExecutionInfo), Type: TypeBool}, {Scope: ScopeGlobal | ScopeSession, Name: TiDBAllowAutoRandExplicitInsert, Value: BoolToOnOff(DefTiDBAllowAutoRandExplicitInsert), Type: TypeBool}, {Scope: ScopeGlobal | ScopeSession, Name: TiDBEnableClusteredIndex, Value: BoolToOnOff(DefTiDBEnableClusteredIndex), Type: TypeBool}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBPartitionPruneMode, Value: string(StaticOnly), Type: TypeStr, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { - if !PartitionPruneMode(normalizedValue).Valid() { + {Scope: ScopeGlobal | ScopeSession, Name: TiDBPartitionPruneMode, Value: string(Static), Type: TypeStr, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { + mode := PartitionPruneMode(normalizedValue).Update() + if !mode.Valid() { return normalizedValue, ErrWrongTypeForVar.GenWithStackByArgs(TiDBPartitionPruneMode) } - return normalizedValue, nil + return string(mode), nil }}, {Scope: ScopeGlobal | ScopeSession, Name: TiDBSlowLogMasking, Value: BoolToOnOff(DefTiDBRedactLog), Type: TypeBool}, {Scope: ScopeGlobal | ScopeSession, Name: TiDBRedactLog, Value: BoolToOnOff(DefTiDBRedactLog), Type: TypeBool}, diff --git a/sessionctx/variable/tidb_vars.go b/sessionctx/variable/tidb_vars.go index f05f48d651799..c6e0ddeba52c2 100644 --- a/sessionctx/variable/tidb_vars.go +++ b/sessionctx/variable/tidb_vars.go @@ -659,7 +659,7 @@ const ( DefTiDBEnableTelemetry = true DefTiDBEnableParallelApply = false DefTiDBEnableAmendPessimisticTxn = false - DefTiDBPartitionPruneMode = "static-only" + DefTiDBPartitionPruneMode = "static" DefTiDBEnableRateLimitAction = true DefTiDBEnableAsyncCommit = false DefTiDBEnable1PC = false diff --git a/statistics/handle/ddl.go b/statistics/handle/ddl.go index 7140c7b2a8231..f671067cced38 100644 --- a/statistics/handle/ddl.go +++ b/statistics/handle/ddl.go @@ -45,19 +45,19 @@ func (h *Handle) HandleDDLEvent(t *util.Event) error { } case model.ActionAddTablePartition, model.ActionTruncateTablePartition: pruneMode := h.CurrentPruneMode() - if pruneMode == variable.StaticOnly || pruneMode == variable.StaticButPrepareDynamic { + if pruneMode == variable.Static { for _, def := range t.PartInfo.Definitions { if err := h.insertTableStats2KV(t.TableInfo, def.ID); err != nil { return err } } } - if pruneMode == variable.DynamicOnly || pruneMode == variable.StaticButPrepareDynamic { + if pruneMode == variable.Dynamic { // TODO: need trigger full analyze } case model.ActionDropTablePartition: pruneMode := h.CurrentPruneMode() - if pruneMode == variable.DynamicOnly || pruneMode == variable.StaticButPrepareDynamic { + if pruneMode == variable.Dynamic { // TODO: need trigger full analyze } } @@ -71,12 +71,12 @@ func (h *Handle) getInitStateTableIDs(tblInfo *model.TableInfo) (ids []int64) { } ids = make([]int64, 0, len(pi.Definitions)+1) pruneMode := h.CurrentPruneMode() - if pruneMode == variable.StaticOnly || pruneMode == variable.StaticButPrepareDynamic { + if pruneMode == variable.Static { for _, def := range pi.Definitions { ids = append(ids, def.ID) } } - if pruneMode == variable.DynamicOnly || pruneMode == variable.StaticButPrepareDynamic { + if pruneMode == variable.Dynamic { ids = append(ids, tblInfo.ID) } return ids diff --git a/statistics/handle/ddl_test.go b/statistics/handle/ddl_test.go index ee9055d3832c8..c62e80d372766 100644 --- a/statistics/handle/ddl_test.go +++ b/statistics/handle/ddl_test.go @@ -185,7 +185,7 @@ func (s *testStatsSuite) TestDDLHistogram(c *C) { func (s *testStatsSuite) TestDDLPartition(c *C) { defer cleanEnv(c, s.store, s.do) testKit := testkit.NewTestKit(c, s.store) - testkit.WithPruneMode(testKit, variable.StaticOnly, func() { + testkit.WithPruneMode(testKit, variable.Static, func() { testKit.MustExec("use test") testKit.MustExec("drop table if exists t") createTable := `CREATE TABLE t (a int, b int, primary key(a), index idx(b)) diff --git a/statistics/handle/dump_test.go b/statistics/handle/dump_test.go index c8c94220232ce..9ec8448b0816d 100644 --- a/statistics/handle/dump_test.go +++ b/statistics/handle/dump_test.go @@ -80,7 +80,7 @@ func (s *testStatsSuite) TestDumpGlobalStats(c *C) { defer cleanEnv(c, s.store, s.do) tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") - tk.MustExec("set @@tidb_partition_prune_mode = 'static-only'") + tk.MustExec("set @@tidb_partition_prune_mode = 'static'") tk.MustExec("drop table if exists t") tk.MustExec("create table t (a int, key(a)) partition by hash(a) partitions 2") tk.MustExec("insert into t values (1), (2)") @@ -93,7 +93,7 @@ func (s *testStatsSuite) TestDumpGlobalStats(c *C) { c.Assert(stats.Partitions["global"], IsNil) // global-stats is existed - tk.MustExec("set @@tidb_partition_prune_mode = 'dynamic-only'") + tk.MustExec("set @@tidb_partition_prune_mode = 'dynamic'") tk.MustExec("analyze table t") stats = s.getStatsJSON(c, "test", "t") c.Assert(stats.Partitions["p0"], NotNil) @@ -105,7 +105,7 @@ func (s *testStatsSuite) TestLoadGlobalStats(c *C) { defer cleanEnv(c, s.store, s.do) tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") - tk.MustExec("set @@tidb_partition_prune_mode = 'dynamic-only'") + tk.MustExec("set @@tidb_partition_prune_mode = 'dynamic'") tk.MustExec("drop table if exists t") tk.MustExec("create table t (a int, key(a)) partition by hash(a) partitions 2") tk.MustExec("insert into t values (1), (2)") diff --git a/statistics/handle/gc_test.go b/statistics/handle/gc_test.go index ffbeb71003059..2bcd01e318442 100644 --- a/statistics/handle/gc_test.go +++ b/statistics/handle/gc_test.go @@ -58,7 +58,7 @@ func (s *testStatsSuite) TestGCStats(c *C) { func (s *testStatsSuite) TestGCPartition(c *C) { defer cleanEnv(c, s.store, s.do) testKit := testkit.NewTestKit(c, s.store) - testkit.WithPruneMode(testKit, variable.StaticOnly, func() { + testkit.WithPruneMode(testKit, variable.Static, func() { testKit.MustExec("use test") testKit.MustExec("set @@session.tidb_enable_table_partition=1") testKit.MustExec(`create table t (a bigint(64), b bigint(64), index idx(a, b)) diff --git a/statistics/handle/handle_test.go b/statistics/handle/handle_test.go index 5a8064aaeb32c..f87c13a08c931 100644 --- a/statistics/handle/handle_test.go +++ b/statistics/handle/handle_test.go @@ -681,7 +681,7 @@ func (s *testStatsSuite) TestShowGlobalStats(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") tk.MustExec("drop table if exists t") - tk.MustExec("set @@tidb_partition_prune_mode = 'static-only'") + tk.MustExec("set @@tidb_partition_prune_mode = 'static'") tk.MustExec("create table t (a int, key(a)) partition by hash(a) partitions 2") tk.MustExec("insert into t values (1), (2), (3), (4)") tk.MustExec("analyze table t with 1 buckets") @@ -694,7 +694,7 @@ func (s *testStatsSuite) TestShowGlobalStats(c *C) { c.Assert(len(tk.MustQuery("show stats_healthy").Rows()), Equals, 2) c.Assert(len(tk.MustQuery("show stats_healthy where partition_name='global'").Rows()), Equals, 0) - tk.MustExec("set @@tidb_partition_prune_mode = 'dynamic-only'") + tk.MustExec("set @@tidb_partition_prune_mode = 'dynamic'") tk.MustExec("analyze table t with 1 buckets") c.Assert(len(tk.MustQuery("show stats_meta").Rows()), Equals, 3) c.Assert(len(tk.MustQuery("show stats_meta where partition_name='global'").Rows()), Equals, 1) @@ -711,7 +711,7 @@ func (s *testStatsSuite) TestBuildGlobalLevelStats(c *C) { testKit := testkit.NewTestKit(c, s.store) testKit.MustExec("use test") testKit.MustExec("drop table if exists t, t1;") - testKit.MustExec("set @@tidb_partition_prune_mode = 'static-only';") + testKit.MustExec("set @@tidb_partition_prune_mode = 'static';") testKit.MustExec("create table t(a int, b int, c int) PARTITION BY HASH(a) PARTITIONS 3;") testKit.MustExec("create table t1(a int);") testKit.MustExec("insert into t values(1,1,1),(3,12,3),(4,20,4),(2,7,2),(5,21,5);") @@ -733,8 +733,8 @@ func (s *testStatsSuite) TestBuildGlobalLevelStats(c *C) { result = testKit.MustQuery("show stats_histograms where table_name = 't1';").Sort() c.Assert(len(result.Rows()), Equals, 1) - // Test the 'dynamic-only' mode - testKit.MustExec("set @@tidb_partition_prune_mode = 'dynamic-only';") + // Test the 'dynamic' mode + testKit.MustExec("set @@tidb_partition_prune_mode = 'dynamic';") testKit.MustExec("analyze table t, t1;") result = testKit.MustQuery("show stats_meta where table_name = 't'").Sort() c.Assert(len(result.Rows()), Equals, 4) @@ -776,7 +776,7 @@ partition by range (a) ( partition p0 values less than (10), partition p1 values less than (20) )`) - tk.MustExec("set @@tidb_partition_prune_mode='dynamic-only'") + tk.MustExec("set @@tidb_partition_prune_mode='dynamic'") tk.MustExec("insert into t values (1), (5), (null), (11), (15)") c.Assert(s.do.StatsHandle().DumpStatsDeltaToKV(handle.DumpAll), IsNil) tk.MustExec("analyze table t") diff --git a/statistics/handle/update.go b/statistics/handle/update.go index 2014b1284cd61..379e101ad1ab3 100644 --- a/statistics/handle/update.go +++ b/statistics/handle/update.go @@ -720,7 +720,7 @@ func (h *Handle) handleSingleHistogramUpdate(is infoschema.InfoSchema, rows []ch return nil } var tbl *statistics.Table - if table.Meta().GetPartitionInfo() == nil || h.CurrentPruneMode() == variable.DynamicOnly { + if table.Meta().GetPartitionInfo() == nil || h.CurrentPruneMode() == variable.Dynamic { tbl = h.GetTableStats(table.Meta()) } else { tbl = h.GetPartitionStats(table.Meta(), physicalTableID) @@ -905,7 +905,7 @@ func (h *Handle) HandleAutoAnalyze(is infoschema.InfoSchema) { for _, tbl := range tbls { tblInfo := tbl.Meta() pi := tblInfo.GetPartitionInfo() - if pi == nil || pruneMode == variable.DynamicOnly || pruneMode == variable.StaticButPrepareDynamic { + if pi == nil || pruneMode == variable.Dynamic { statsTbl := h.GetTableStats(tblInfo) sql := "analyze table %n.%n" analyzed := h.autoAnalyzeTable(tblInfo, statsTbl, start, end, autoAnalyzeRatio, sql, db, tblInfo.Name.O) @@ -914,7 +914,7 @@ func (h *Handle) HandleAutoAnalyze(is infoschema.InfoSchema) { } continue } - if pruneMode == variable.StaticOnly || pruneMode == variable.StaticButPrepareDynamic { + if pruneMode == variable.Static { for _, def := range pi.Definitions { sql := "analyze table %n.%n partition %n" statsTbl := h.GetPartitionStats(tblInfo, def.ID) diff --git a/statistics/handle/update_test.go b/statistics/handle/update_test.go index abf795be382b6..1fa609e5bb934 100644 --- a/statistics/handle/update_test.go +++ b/statistics/handle/update_test.go @@ -350,7 +350,7 @@ func (s *testStatsSuite) TestUpdatePartition(c *C) { testKit := testkit.NewTestKit(c, s.store) testKit.MustQuery("select @@tidb_partition_prune_mode").Check(testkit.Rows(string(s.do.StatsHandle().CurrentPruneMode()))) testKit.MustExec("use test") - testkit.WithPruneMode(testKit, variable.StaticOnly, func() { + testkit.WithPruneMode(testKit, variable.Static, func() { s.do.StatsHandle().RefreshVars() testKit.MustExec("drop table if exists t") createTable := `CREATE TABLE t (a int, b char(5)) PARTITION BY RANGE (a) (PARTITION p0 VALUES LESS THAN (6),PARTITION p1 VALUES LESS THAN (11))` @@ -402,7 +402,7 @@ func (s *testStatsSuite) TestUpdatePartition(c *C) { func (s *testStatsSuite) TestAutoUpdate(c *C) { defer cleanEnv(c, s.store, s.do) testKit := testkit.NewTestKit(c, s.store) - testkit.WithPruneMode(testKit, variable.StaticOnly, func() { + testkit.WithPruneMode(testKit, variable.Static, func() { testKit.MustExec("use test") testKit.MustExec("create table t (a varchar(20))") @@ -501,7 +501,7 @@ func (s *testStatsSuite) TestAutoUpdate(c *C) { func (s *testStatsSuite) TestAutoUpdatePartition(c *C) { defer cleanEnv(c, s.store, s.do) testKit := testkit.NewTestKit(c, s.store) - testkit.WithPruneMode(testKit, variable.StaticOnly, func() { + testkit.WithPruneMode(testKit, variable.Static, func() { testKit.MustExec("use test") testKit.MustExec("drop table if exists t") testKit.MustExec("create table t (a int) PARTITION BY RANGE (a) (PARTITION p0 VALUES LESS THAN (6))") @@ -738,7 +738,7 @@ func (s *testStatsSuite) TestUpdatePartitionErrorRate(c *C) { testKit := testkit.NewTestKit(c, s.store) testKit.MustExec("use test") - testKit.MustExec(`set @@tidb_partition_prune_mode='` + string(variable.StaticOnly) + `'`) + testKit.MustExec(`set @@tidb_partition_prune_mode='` + string(variable.Static) + `'`) testKit.MustExec("create table t (a bigint(64), primary key(a)) partition by range (a) (partition p0 values less than (30))") h.HandleDDLEvent(<-h.DDLEventCh()) @@ -958,7 +958,7 @@ func (s *testStatsSuite) TestQueryFeedbackForPartition(c *C) { defer cleanEnv(c, s.store, s.do) testKit := testkit.NewTestKit(c, s.store) testKit.MustExec("use test") - testKit.MustExec(`set @@tidb_partition_prune_mode='` + string(variable.StaticOnly) + `'`) + testKit.MustExec(`set @@tidb_partition_prune_mode='` + string(variable.Static) + `'`) testKit.MustExec(`create table t (a bigint(64), b bigint(64), primary key(a), index idx(b)) partition by range (a) ( partition p0 values less than (3), @@ -1090,7 +1090,7 @@ func (s *testStatsSuite) TestUpdateStatsByLocalFeedback(c *C) { defer cleanEnv(c, s.store, s.do) testKit := testkit.NewTestKit(c, s.store) testKit.MustExec("use test") - testKit.MustExec(`set @@tidb_partition_prune_mode='` + string(variable.StaticOnly) + `'`) + testKit.MustExec(`set @@tidb_partition_prune_mode='` + string(variable.Static) + `'`) testKit.MustExec("create table t (a bigint(64), b bigint(64), primary key(a), index idx(b))") testKit.MustExec("insert into t values (1,2),(2,2),(4,5)") testKit.MustExec("analyze table t with 0 topn") @@ -1150,7 +1150,7 @@ func (s *testStatsSuite) TestUpdatePartitionStatsByLocalFeedback(c *C) { defer cleanEnv(c, s.store, s.do) testKit := testkit.NewTestKit(c, s.store) testKit.MustExec("use test") - testKit.MustExec(`set @@tidb_partition_prune_mode='` + string(variable.StaticOnly) + `'`) + testKit.MustExec(`set @@tidb_partition_prune_mode='` + string(variable.Static) + `'`) testKit.MustExec("create table t (a bigint(64), b bigint(64), primary key(a)) partition by range (a) (partition p0 values less than (6))") testKit.MustExec("insert into t values (1,2),(2,2),(4,5)") testKit.MustExec("analyze table t") diff --git a/util/ranger/ranger_test.go b/util/ranger/ranger_test.go index 1ae8bc3355424..e26624f752381 100644 --- a/util/ranger/ranger_test.go +++ b/util/ranger/ranger_test.go @@ -1443,7 +1443,7 @@ func (s *testRangerSuite) TestIndexRangeForBit(c *C) { c.Assert(err, IsNil) testKit := testkit.NewTestKit(c, store) testKit.MustExec("use test;") - testKit.MustExec("set @@tidb_partition_prune_mode = 'static-only';") + testKit.MustExec("set @@tidb_partition_prune_mode = 'static';") testKit.MustExec("set @@tidb_executor_concurrency = 1;") testKit.MustExec("drop table if exists t;") testKit.MustExec("CREATE TABLE `t` (" +