Skip to content

Commit

Permalink
*: Deprecate tidb_enable_table_partition and always set to ON (#56190)
Browse files Browse the repository at this point in the history
close #56189
  • Loading branch information
mjonss authored Sep 25, 2024
1 parent 8c12f12 commit 9129323
Show file tree
Hide file tree
Showing 23 changed files with 18 additions and 179 deletions.
4 changes: 1 addition & 3 deletions pkg/ddl/index_modify_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -186,9 +186,7 @@ func testAddIndex(t *testing.T, tp testAddIndexType, createTableSQL, idxTp strin
tk.MustExec("set global tidb_scatter_region = 0")
}()
}
if isTestPartition {
tk.MustExec("set @@session.tidb_enable_table_partition = '1';")
} else if (testClusteredIndex & tp) > 0 {
if (testClusteredIndex & tp) > 0 {
tk.Session().GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeOn
}
tk.MustExec("drop table if exists test_add_index")
Expand Down
5 changes: 0 additions & 5 deletions pkg/ddl/partition.go
Original file line number Diff line number Diff line change
Expand Up @@ -508,11 +508,6 @@ func buildTablePartitionInfo(ctx sessionctx.Context, s *ast.PartitionOptions, tb
return nil
}

if strings.EqualFold(ctx.GetSessionVars().EnableTablePartition, "OFF") {
ctx.GetSessionVars().StmtCtx.AppendWarning(dbterror.ErrTablePartitionDisabled)
return nil
}

var enable bool
switch s.Tp {
case pmodel.PartitionTypeRange:
Expand Down
7 changes: 0 additions & 7 deletions pkg/ddl/tests/partition/db_partition_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -275,7 +275,6 @@ func TestCreateTableWithPartition(t *testing.T) {
partition p4 values less than (18446744073709551000 + 10)
);`)

tk.MustExec("set @@tidb_enable_table_partition = 1")
tk.MustExec(`create table t30 (
a int,
b varchar(20),
Expand Down Expand Up @@ -349,7 +348,6 @@ func TestCreateTableWithHashPartition(t *testing.T) {
tk := testkit.NewTestKit(t, store)
tk.MustExec("use test;")
tk.MustExec("drop table if exists employees;")
tk.MustExec("set @@session.tidb_enable_table_partition = 1")
tk.MustExec(`
create table employees (
id int not null,
Expand Down Expand Up @@ -2251,7 +2249,6 @@ func TestTruncatePartitionAndDropTable(t *testing.T) {

// Test truncate table partition reassigns new partitionIDs.
tk.MustExec("drop table if exists t5;")
tk.MustExec("set @@session.tidb_enable_table_partition=1;")
tk.MustExec(`create table t5(
id int, name varchar(50),
purchased date
Expand All @@ -2276,7 +2273,6 @@ func TestTruncatePartitionAndDropTable(t *testing.T) {
newPID := newTblInfo.Meta().Partition.Definitions[0].ID
require.True(t, oldPID != newPID)

tk.MustExec("set @@session.tidb_enable_table_partition = 1;")
tk.MustExec("drop table if exists clients;")
tk.MustExec(`create table clients (
id int,
Expand Down Expand Up @@ -2392,7 +2388,6 @@ func testPartitionAddIndexOrPK(t *testing.T, tk *testkit.TestKit, key string) {
testPartitionAddIndex(tk, t, key)

// test hash partition table.
tk.MustExec("set @@session.tidb_enable_table_partition = '1';")
tk.MustExec("drop table if exists partition_add_idx")
tk.MustExec(`create table partition_add_idx (
id int not null,
Expand All @@ -2403,7 +2398,6 @@ func testPartitionAddIndexOrPK(t *testing.T, tk *testkit.TestKit, key string) {
// Test hash partition for pr 10475.
tk.MustExec("drop table if exists t1")
defer tk.MustExec("drop table if exists t1")
tk.MustExec("set @@session.tidb_enable_table_partition = '1';")
tk.MustExec("create table t1 (a int, b int, unique key(a)) partition by hash(a) partitions 5;")
tk.MustExec("insert into t1 values (0,0),(1,1),(2,2),(3,3);")
tk.MustExec(fmt.Sprintf("alter table t1 add %s idx(a)", key))
Expand Down Expand Up @@ -2549,7 +2543,6 @@ func TestPartitionErrorCode(t *testing.T) {
tk1 := testkit.NewTestKit(t, store)

// add partition
tk.MustExec("set @@session.tidb_enable_table_partition = 1")
tk.MustExec("drop database if exists test_db_with_partition")
tk.MustExec("create database test_db_with_partition")
tk.MustExec("use test_db_with_partition")
Expand Down
1 change: 0 additions & 1 deletion pkg/executor/show_stats_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -235,7 +235,6 @@ func TestShowPartitionStats(t *testing.T) {

tk := testkit.NewTestKit(t, store)
testkit.WithPruneMode(tk, variable.Static, func() {
tk.MustExec("set @@session.tidb_enable_table_partition=1")
// Version2 is tested in TestGlobalStatsData1/2/3 and TestAnalyzeGlobalStatsWithOpts.
tk.MustExec("set @@session.tidb_analyze_version=1")
tk.MustExec("use test")
Expand Down
1 change: 0 additions & 1 deletion pkg/executor/test/splittest/split_table_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -215,7 +215,6 @@ func TestShowTableRegion(t *testing.T) {
// Test show table regions for partition table when disable split region when create table.
atomic.StoreUint32(&ddl.EnableSplitTableRegion, 0)
tk.MustExec("drop table if exists partition_t;")
tk.MustExec("set @@session.tidb_enable_table_partition = '1';")
tk.MustExec("create table partition_t (a int, b int,index(a)) partition by hash (a) partitions 3")
re = tk.MustQuery("show table partition_t regions")
rows = re.Rows()
Expand Down
2 changes: 0 additions & 2 deletions pkg/planner/core/binary_plan_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -143,7 +143,6 @@ func TestTooLongBinaryPlan(t *testing.T) {

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.Static) + `'`)
tk.MustExec("create table th (i int, a int,b int, c int, index (a)) partition by hash (a) partitions 8192;")
tk.MustQuery("select count(*) from th t1 join th t2 join th t3 join th t4 join th t5 join th t6 where t1.i=t2.a and t1.i=t3.i and t3.i=t4.i and t4.i=t5.i and t5.i=t6.i")
Expand Down Expand Up @@ -205,7 +204,6 @@ func TestLongBinaryPlan(t *testing.T) {
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.Static) + `'`)
tk.MustExec("create table th (i int, a int,b int, c int, index (a)) partition by hash (a) partitions 1000;")
tk.MustQuery("select count(*) from th t1 join th t2 join th t3 join th t4 join th t5 join th t6 where t1.i=t2.a and t1.i=t3.i and t3.i=t4.i and t4.i=t5.i and t5.i=t6.i")
Expand Down
2 changes: 0 additions & 2 deletions pkg/planner/core/plan_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -427,7 +427,6 @@ func BenchmarkEncodePlan(b *testing.B) {
tk := testkit.NewTestKit(b, store)
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.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 All @@ -451,7 +450,6 @@ func BenchmarkEncodeFlatPlan(b *testing.B) {
tk := testkit.NewTestKit(b, store)
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.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
3 changes: 0 additions & 3 deletions pkg/sessionctx/variable/session.go
Original file line number Diff line number Diff line change
Expand Up @@ -1106,9 +1106,6 @@ type SessionVars struct {
// OptimizerEnableNAAJ enables TiDB to use null-aware anti join.
OptimizerEnableNAAJ bool

// EnableTablePartition enables table partition feature.
EnableTablePartition string

// EnableCascadesPlanner enables the cascades planner.
EnableCascadesPlanner bool

Expand Down
8 changes: 5 additions & 3 deletions pkg/sessionctx/variable/sysvar.go
Original file line number Diff line number Diff line change
Expand Up @@ -2075,9 +2075,11 @@ var defaultSysVars = []*SysVar{
s.SetEnableIndexMerge(TiDBOptOn(val))
return nil
}},
{Scope: ScopeGlobal | ScopeSession, Name: TiDBEnableTablePartition, Value: On, Type: TypeEnum, PossibleValues: []string{Off, On, "AUTO"}, SetSession: func(s *SessionVars, val string) error {
s.EnableTablePartition = val
return nil
{Scope: ScopeGlobal | ScopeSession, Name: TiDBEnableTablePartition, Value: On, Type: TypeEnum, PossibleValues: []string{Off, On, "AUTO"}, Validation: func(vars *SessionVars, s string, s2 string, flag ScopeFlag) (string, error) {
if s == Off {
vars.StmtCtx.AppendWarning(errors.NewNoStackError("tidb_enable_table_partition is always turned on. This variable has been deprecated and will be removed in the future releases"))
}
return On, nil
}},
// Keeping tidb_enable_list_partition here, to give errors if setting it to anything other than ON
{Scope: ScopeGlobal | ScopeSession, Name: TiDBEnableListTablePartition, Value: On, Type: TypeBool, Validation: func(vars *SessionVars, normalizedValue, _ string, _ ScopeFlag) (string, error) {
Expand Down
12 changes: 0 additions & 12 deletions pkg/sessionctx/variable/varsutil_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -258,14 +258,6 @@ func TestVarsutil(t *testing.T) {
require.Equal(t, "3", val)
require.Equal(t, int64(3), v.RetryLimit)

require.Equal(t, "", v.EnableTablePartition)
err = v.SetSystemVar(TiDBEnableTablePartition, "on")
require.NoError(t, err)
val, err = v.GetSessionOrGlobalSystemVar(context.Background(), TiDBEnableTablePartition)
require.NoError(t, err)
require.Equal(t, "ON", val)
require.Equal(t, "ON", v.EnableTablePartition)

require.Equal(t, DefTiDBOptJoinReorderThreshold, v.TiDBOptJoinReorderThreshold)
err = v.SetSystemVar(TiDBOptJoinReorderThreshold, "5")
require.NoError(t, err)
Expand Down Expand Up @@ -493,10 +485,6 @@ func TestValidate(t *testing.T) {
{SecureAuth, "3", true},
{MyISAMUseMmap, "ON", false},
{MyISAMUseMmap, "OFF", false},
{TiDBEnableTablePartition, "ON", false},
{TiDBEnableTablePartition, "OFF", false},
{TiDBEnableTablePartition, "AUTO", false},
{TiDBEnableTablePartition, "UN", true},
{TiDBOptCorrelationExpFactor, "a", true},
{TiDBOptCorrelationExpFactor, "-10", false},
{TiDBOptCorrelationThreshold, "a", true},
Expand Down
1 change: 0 additions & 1 deletion pkg/statistics/handle/storage/gc_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -63,7 +63,6 @@ func TestGCPartition(t *testing.T) {
testKit.MustExec("set @@tidb_analyze_version = 1")
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))
partition by range (a) (
partition p0 values less than (3),
Expand Down
2 changes: 0 additions & 2 deletions pkg/table/tables/test/partition/partition_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -154,8 +154,6 @@ func TestHashPartitionAddRecord(t *testing.T) {
require.NoError(t, err)
_, err = tk.Session().Execute(context.Background(), "drop table if exists t1;")
require.NoError(t, err)
_, err = tk.Session().Execute(context.Background(), "set @@session.tidb_enable_table_partition = '1';")
require.NoError(t, err)
_, err = tk.Session().Execute(context.Background(), `CREATE TABLE test.t1 (id int(11), index(id)) PARTITION BY HASH (id) partitions 4;`)
require.NoError(t, err)
tb, err := dom.InfoSchema().TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t1"))
Expand Down
2 changes: 0 additions & 2 deletions pkg/util/dbterror/ddl_terror.go
Original file line number Diff line number Diff line change
Expand Up @@ -168,8 +168,6 @@ var (
ErrFunctionalIndexRowValueIsNotAllowed = ClassDDL.NewStd(mysql.ErrFunctionalIndexRowValueIsNotAllowed)
// ErrUnsupportedCreatePartition returns for does not support create partitions.
ErrUnsupportedCreatePartition = ClassDDL.NewStdErr(mysql.ErrUnsupportedDDLOperation, parser_mysql.Message(fmt.Sprintf(mysql.MySQLErrName[mysql.ErrUnsupportedDDLOperation].Raw, "partition type, treat as normal table"), nil))
// ErrTablePartitionDisabled returns for table partition is disabled.
ErrTablePartitionDisabled = ClassDDL.NewStdErr(mysql.ErrUnsupportedDDLOperation, parser_mysql.Message("Partitions are ignored because Table Partition is disabled, please set 'tidb_enable_table_partition' if you need to need to enable it", nil))
// ErrUnsupportedIndexType returns for unsupported index type.
ErrUnsupportedIndexType = ClassDDL.NewStdErr(mysql.ErrUnsupportedDDLOperation, parser_mysql.Message(fmt.Sprintf(mysql.MySQLErrName[mysql.ErrUnsupportedDDLOperation].Raw, "index type"), nil))
// ErrWindowInvalidWindowFuncUse returns for invalid window function use.
Expand Down
65 changes: 0 additions & 65 deletions tests/integrationtest/r/ddl/db_partition.result
Original file line number Diff line number Diff line change
@@ -1,64 +1,3 @@
set @@session.tidb_enable_table_partition = 'ON';
drop table if exists t;
create table t (id int) partition by list (id) (
partition p0 values in (1,2),partition p1 values in (3,4));
show create table t;
Table Create Table
t CREATE TABLE `t` (
`id` int(11) DEFAULT NULL
) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin
PARTITION BY LIST (`id`)
(PARTITION `p0` VALUES IN (1,2),
PARTITION `p1` VALUES IN (3,4))
alter table t add partition (
partition p4 values in (7),
partition p5 values in (8,9));
insert into t values (1),(3),(7),(9);
set @@session.tidb_enable_table_partition = 'OFF';
drop table if exists t;
create table t (id int) partition by list (id) (
partition p0 values in (1,2),partition p1 values in (3,4));
show create table t;
Table Create Table
t CREATE TABLE `t` (
`id` int(11) DEFAULT NULL
) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin
alter table t add partition (
partition p4 values in (7),
partition p5 values in (8,9));
Error 1505 (HY000): Partition management on a not partitioned table is not possible
insert into t values (1),(3),(7),(9);
set @@session.tidb_enable_table_partition = 0;
drop table if exists t;
create table t (id int) partition by list (id) (
partition p0 values in (1,2),partition p1 values in (3,4));
show create table t;
Table Create Table
t CREATE TABLE `t` (
`id` int(11) DEFAULT NULL
) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin
alter table t add partition (
partition p4 values in (7),
partition p5 values in (8,9));
Error 1505 (HY000): Partition management on a not partitioned table is not possible
insert into t values (1),(3),(7),(9);
set @@session.tidb_enable_table_partition = 'ON';
drop table if exists t;
create table t (id int) partition by list (id) (
partition p0 values in (1,2),partition p1 values in (3,4));
show create table t;
Table Create Table
t CREATE TABLE `t` (
`id` int(11) DEFAULT NULL
) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin
PARTITION BY LIST (`id`)
(PARTITION `p0` VALUES IN (1,2),
PARTITION `p1` VALUES IN (3,4))
alter table t add partition (
partition p4 values in (7),
partition p5 values in (8,9));
insert into t values (1),(3),(7),(9);
set @@session.tidb_enable_table_partition = default;
drop table if exists t;
create table t (a varchar(255) charset utf8mb4 collate utf8mb4_bin) partition by range columns (a)
(partition p0A values less than ("A"),
Expand Down Expand Up @@ -1840,7 +1779,6 @@ t CREATE TABLE `t` (
`store_id` int(11) DEFAULT NULL
) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin
PARTITION BY KEY (`hired`) PARTITIONS 11
set @@session.tidb_enable_table_partition = 1;
drop database if exists test_db_with_partition_const;
create database test_db_with_partition_const;
use test_db_with_partition_const;
Expand Down Expand Up @@ -1909,7 +1847,6 @@ partition by range(day(time_recorded)) (
partition p0 values less than (1)
);
Error 1486 (HY000): Constant, random or timezone-dependent expressions in (sub)partitioning function are not allowed
set @@session.tidb_enable_table_partition = default;
drop table if exists test_1465;
create table test_1465 (a int)
partition by range(a) (
Expand Down Expand Up @@ -3100,7 +3037,6 @@ Error 1526 (HY000): Table has no partition for value from column_list
insert into t values ('2023-08-31', 'c', "FAIL");
alter table rcp EXCHANGE PARTITION p20230829 WITH TABLE t;
Error 1737 (HY000): Found a row that does not match the partition
set @@session.tidb_enable_table_partition = 1;
drop table if exists t1, t2, t3, t4, t5;
create table t1 ( time_recorded datetime )
partition by range(TO_DAYS(time_recorded)) (
Expand All @@ -3118,7 +3054,6 @@ create table t5 ( time_recorded timestamp )
partition by range(unix_timestamp(time_recorded)) (
partition p1 values less than (1559192604)
);
set @@session.tidb_enable_table_partition = default;
set @@tidb_enable_exchange_partition=1;
drop table if exists pt1;
create table pt1(a int, b int, c int) PARTITION BY hash (a) partitions 1;
Expand Down
2 changes: 0 additions & 2 deletions tests/integrationtest/r/executor/executor.result
Original file line number Diff line number Diff line change
Expand Up @@ -2474,7 +2474,6 @@ select a from (select 1 a), (select 2 a);
Error 1052 (23000): Column 'a' in field list is ambiguous
set sql_mode = default;
drop table if exists th;
set @@session.tidb_enable_table_partition = '1';
create table th (a int, b int) partition by hash(a) partitions 3;
insert into th values (0,0),(1,1),(2,2),(3,3),(4,4),(5,5),(6,6),(7,7),(8,8);
insert into th values (-1,-1),(-2,-2),(-3,-3),(-4,-4),(-5,-5),(-6,-6),(-7,-7),(-8,-8);
Expand Down Expand Up @@ -2504,7 +2503,6 @@ select * from th where a=5;
a b
5 5
drop table if exists th;
set @@session.tidb_enable_table_partition = default;
drop table if exists view_t;
create table view_t (a int,b int);
insert into view_t values(1,2);
Expand Down
2 changes: 0 additions & 2 deletions tests/integrationtest/r/executor/partition/write.result
Original file line number Diff line number Diff line change
Expand Up @@ -718,7 +718,6 @@ affected rows: 1
info:
drop table t1;
set tidb_opt_fix_control=default;
set @@session.tidb_enable_table_partition = '1';
drop table if exists replace_test;
create table replace_test (id int PRIMARY KEY AUTO_INCREMENT, c1 int, c2 int, c3 int default 1)
partition by hash(id) partitions 4;
Expand Down Expand Up @@ -807,7 +806,6 @@ replace into tIssue989(a, b) values (111, 2);
select * from tIssue989;
a b
111 2
set @@session.tidb_enable_table_partition = default;
drop table if exists insert_update_ignore_test;
create table insert_update_ignore_test (a int) partition by range (a) (partition p0 values less than (100), partition p1 values less than (200));
insert ignore into insert_update_ignore_test values(1000);
Expand Down
1 change: 0 additions & 1 deletion tests/integrationtest/r/select.result
Original file line number Diff line number Diff line change
Expand Up @@ -356,7 +356,6 @@ id estRows task access object operator info
Projection_3 1.00 root sysdate()->Column#1, sleep(1)->Column#2, sysdate()->Column#3
└─TableDual_4 1.00 root rows:1
drop table if exists th;
set @@session.tidb_enable_table_partition = '1';
set @@session.tidb_partition_prune_mode = 'static';
create table th (a int, b int) partition by hash(a) partitions 3;
insert into th values (0,0),(1,1),(2,2),(3,3),(4,4),(5,5),(6,6),(7,7),(8,8);
Expand Down
17 changes: 8 additions & 9 deletions tests/integrationtest/r/session/variable.result
Original file line number Diff line number Diff line change
Expand Up @@ -80,13 +80,16 @@ id
DROP TABLE t_sel_in_share;
set @@tidb_enable_noop_functions = default;
set tidb_enable_table_partition=off;
show warnings;
Level Code Message
Warning 1105 tidb_enable_table_partition is always turned on. This variable has been deprecated and will be removed in the future releases
set global tidb_enable_table_partition=off;
show warnings;
Level Code Message
Warning 1105 tidb_enable_table_partition is always turned on. This variable has been deprecated and will be removed in the future releases
show variables like 'tidb_enable_table_partition';
Variable_name Value
tidb_enable_table_partition OFF
set global tidb_enable_table_partition = on;
show variables like 'tidb_enable_table_partition';
Variable_name Value
tidb_enable_table_partition OFF
tidb_enable_table_partition ON
show global variables like 'tidb_enable_table_partition';
Variable_name Value
tidb_enable_table_partition ON
Expand All @@ -111,10 +114,6 @@ set tidb_enable_list_partition=1;
show variables like 'tidb_enable_list_partition';
Variable_name Value
tidb_enable_list_partition ON
show variables like 'tidb_enable_table_partition';
Variable_name Value
tidb_enable_table_partition ON
set global tidb_enable_table_partition = default;
set @@foreign_key_checks=0;
CREATE TABLE address (
id bigint(20) NOT NULL AUTO_INCREMENT,
Expand Down
Loading

0 comments on commit 9129323

Please sign in to comment.