Skip to content

Commit

Permalink
*: modify the switch to control global stats (#22866)
Browse files Browse the repository at this point in the history
  • Loading branch information
rebelice authored Mar 1, 2021
1 parent b1b7730 commit e1004a2
Show file tree
Hide file tree
Showing 28 changed files with 83 additions and 88 deletions.
2 changes: 1 addition & 1 deletion cmd/explaintest/r/generated_columns.result
Original file line number Diff line number Diff line change
@@ -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,
Expand Down
2 changes: 1 addition & 1 deletion cmd/explaintest/t/generated_columns.test
Original file line number Diff line number Diff line change
Expand Up @@ -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 (
Expand Down
4 changes: 2 additions & 2 deletions executor/analyze.go
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand Down Expand Up @@ -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)
Expand Down
6 changes: 3 additions & 3 deletions executor/analyze_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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))
Expand Down Expand Up @@ -464,15 +464,15 @@ 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`)
tk.MustQuery(`explain format = 'brief' select v from t3 partition(p1) where v = 3`).Check(testkit.Rows(
"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) {
Expand Down
2 changes: 1 addition & 1 deletion executor/index_lookup_merge_join_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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 )")
Expand Down
2 changes: 1 addition & 1 deletion executor/infoschema_reader_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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));`)
Expand Down
4 changes: 2 additions & 2 deletions executor/partition_table_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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",
Expand All @@ -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"))
}
Expand Down
2 changes: 1 addition & 1 deletion executor/show_stats_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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")
Expand Down
1 change: 0 additions & 1 deletion go.sum
Original file line number Diff line number Diff line change
Expand Up @@ -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=
Expand Down
2 changes: 1 addition & 1 deletion planner/core/cbo_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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)")
Expand Down
6 changes: 3 additions & 3 deletions planner/core/integration_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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));")
Expand Down Expand Up @@ -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 ",
Expand Down Expand Up @@ -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 {
Expand Down
6 changes: 3 additions & 3 deletions planner/core/physical_plan_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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 {
Expand Down Expand Up @@ -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)
Expand Down Expand Up @@ -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 {
Expand Down
4 changes: 2 additions & 2 deletions planner/core/plan_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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));")
Expand Down Expand Up @@ -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")

Expand Down
6 changes: 3 additions & 3 deletions planner/core/planbuilder.go
Original file line number Diff line number Diff line change
Expand Up @@ -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
}
Expand Down Expand Up @@ -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
}
Expand Down Expand Up @@ -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
}
Expand Down
2 changes: 1 addition & 1 deletion planner/core/prepare_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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")
Expand Down
4 changes: 2 additions & 2 deletions session/bootstrap.go
Original file line number Diff line number Diff line change
Expand Up @@ -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 {
Expand Down
24 changes: 0 additions & 24 deletions session/session.go
Original file line number Diff line number Diff line change
Expand Up @@ -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)
Expand Down Expand Up @@ -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()))
Expand Down
29 changes: 24 additions & 5 deletions sessionctx/variable/session.go
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand All @@ -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

Expand Down
7 changes: 4 additions & 3 deletions sessionctx/variable/sysvar.go
Original file line number Diff line number Diff line change
Expand Up @@ -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},
Expand Down
2 changes: 1 addition & 1 deletion sessionctx/variable/tidb_vars.go
Original file line number Diff line number Diff line change
Expand Up @@ -659,7 +659,7 @@ const (
DefTiDBEnableTelemetry = true
DefTiDBEnableParallelApply = false
DefTiDBEnableAmendPessimisticTxn = false
DefTiDBPartitionPruneMode = "static-only"
DefTiDBPartitionPruneMode = "static"
DefTiDBEnableRateLimitAction = true
DefTiDBEnableAsyncCommit = false
DefTiDBEnable1PC = false
Expand Down
Loading

0 comments on commit e1004a2

Please sign in to comment.