Skip to content

Commit

Permalink
*: change @@tidb_enable_clustered_index to ON/OFF/INT_ONLY (#23529)
Browse files Browse the repository at this point in the history
  • Loading branch information
lysu authored Mar 26, 2021
1 parent 490c938 commit aee5819
Show file tree
Hide file tree
Showing 45 changed files with 259 additions and 116 deletions.
5 changes: 3 additions & 2 deletions cmd/ddltest/ddl_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -41,6 +41,7 @@ import (
"github.com/pingcap/tidb/kv"
"github.com/pingcap/tidb/session"
"github.com/pingcap/tidb/sessionctx"
"github.com/pingcap/tidb/sessionctx/variable"
"github.com/pingcap/tidb/store"
tidbdriver "github.com/pingcap/tidb/store/driver"
"github.com/pingcap/tidb/table"
Expand Down Expand Up @@ -559,7 +560,7 @@ func (s *TestDDLSuite) Bootstrap(c *C) {
tk.MustExec("create table test_mixed (c1 int, c2 int, primary key(c1))")
tk.MustExec("create table test_inc (c1 int, c2 int, primary key(c1))")

tk.Se.GetSessionVars().EnableClusteredIndex = true
tk.Se.GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeOn
tk.MustExec("drop table if exists test_insert_common, test_conflict_insert_common, " +
"test_update_common, test_conflict_update_common, test_delete_common, test_conflict_delete_common, " +
"test_mixed_common, test_inc_common")
Expand All @@ -571,7 +572,7 @@ func (s *TestDDLSuite) Bootstrap(c *C) {
tk.MustExec("create table test_conflict_delete_common (c1 int, c2 int, primary key(c1, c2))")
tk.MustExec("create table test_mixed_common (c1 int, c2 int, primary key(c1, c2))")
tk.MustExec("create table test_inc_common (c1 int, c2 int, primary key(c1, c2))")
tk.Se.GetSessionVars().EnableClusteredIndex = false
tk.Se.GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeIntOnly
}

func (s *TestDDLSuite) TestSimple(c *C) {
Expand Down
2 changes: 0 additions & 2 deletions cmd/explaintest/main.go
Original file line number Diff line number Diff line change
Expand Up @@ -94,7 +94,6 @@ func newTester(name string) *tester {
t.enableQueryLog = true
t.ctx = mock.NewContext()
t.ctx.GetSessionVars().EnableWindowFunction = true
t.ctx.GetSessionVars().IntPrimaryKeyDefaultAsClustered = true
return t
}

Expand Down Expand Up @@ -658,7 +657,6 @@ func main() {
"set @@tidb_projection_concurrency=4",
"set @@tidb_distsql_scan_concurrency=15",
"set @@global.tidb_enable_clustered_index=0;",
"set @@tidb_int_primary_key_default_as_clustered=1",
}
for _, sql := range resets {
if _, err = mdb.Exec(sql); err != nil {
Expand Down
3 changes: 3 additions & 0 deletions config/config.go
Original file line number Diff line number Diff line change
Expand Up @@ -126,6 +126,8 @@ type Config struct {
IndexLimit int `toml:"index-limit" json:"index-limit"`
TableColumnCountLimit uint32 `toml:"table-column-count-limit" json:"table-column-count-limit"`
GracefulWaitBeforeShutdown int `toml:"graceful-wait-before-shutdown" json:"graceful-wait-before-shutdown"`
// AlterPrimaryKey is used to control alter primary key feature.
AlterPrimaryKey bool `toml:"alter-primary-key" json:"alter-primary-key"`
// TreatOldVersionUTF8AsUTF8MB4 is use to treat old version table/column UTF8 charset as UTF8MB4. This is for compatibility.
// Currently not support dynamic modify, because this need to reload all old version schema.
TreatOldVersionUTF8AsUTF8MB4 bool `toml:"treat-old-version-utf8-as-utf8mb4" json:"treat-old-version-utf8-as-utf8mb4"`
Expand Down Expand Up @@ -560,6 +562,7 @@ var defaultConf = Config{
MaxIndexLength: 3072,
IndexLimit: 64,
TableColumnCountLimit: 1017,
AlterPrimaryKey: false,
TreatOldVersionUTF8AsUTF8MB4: true,
EnableTableLock: false,
DelayCleanTableLock: 0,
Expand Down
6 changes: 6 additions & 0 deletions config/config.toml.example
Original file line number Diff line number Diff line change
Expand Up @@ -86,6 +86,12 @@ delay-clean-table-lock = 0
# Maximum number of the splitting region, which is used by the split region statement.
split-region-max-num = 1000

# alter-primary-key is used to control whether the primary keys are clustered.
# Note that this config is deprecated. Only valid when @@global.tidb_enable_clustered_index = 'int_only'.
# Default is false, only the integer primary keys are clustered.
# If it is true, all types of primary keys are nonclustered.
alter-primary-key = false

# server-version is used to change the version string of TiDB in the following scenarios:
# 1. the server version returned by builtin-function `VERSION()`.
# 2. the server version filled in handshake packets of MySQL Connection Protocol, see https://dev.mysql.com/doc/internals/en/connection-phase-packets.html#packet-Protocol::Handshake for more details.
Expand Down
2 changes: 2 additions & 0 deletions config/config_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -184,6 +184,7 @@ unrecognized-option-test = true
_, err = f.WriteString(`
token-limit = 0
enable-table-lock = true
alter-primary-key = true
delay-clean-table-lock = 5
split-region-max-num=10000
enable-batch-dml = true
Expand Down Expand Up @@ -243,6 +244,7 @@ spilled-file-encryption-method = "plaintext"

// Test that the value will be overwritten by the config file.
c.Assert(conf.Performance.TxnTotalSizeLimit, Equals, uint64(2000))
c.Assert(conf.AlterPrimaryKey, Equals, true)
c.Assert(conf.Performance.TCPNoDelay, Equals, false)

c.Assert(conf.TiKVClient.CommitTimeout, Equals, "41s")
Expand Down
3 changes: 2 additions & 1 deletion ddl/db_integration_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -38,6 +38,7 @@ import (
"github.com/pingcap/tidb/session"
"github.com/pingcap/tidb/sessionctx"
"github.com/pingcap/tidb/sessionctx/stmtctx"
"github.com/pingcap/tidb/sessionctx/variable"
"github.com/pingcap/tidb/store/mockstore"
"github.com/pingcap/tidb/store/tikv/mockstore/cluster"
"github.com/pingcap/tidb/store/tikv/oracle"
Expand Down Expand Up @@ -2638,7 +2639,7 @@ func (s *testIntegrationSuite7) TestDuplicateErrorMessage(c *C) {
config.UpdateGlobal(func(conf *config.Config) {
conf.EnableGlobalIndex = globalIndex
})
for _, clusteredIndex := range []bool{false, true} {
for _, clusteredIndex := range []variable.ClusteredIndexDefMode{variable.ClusteredIndexDefModeOn, variable.ClusteredIndexDefModeOff, variable.ClusteredIndexDefModeIntOnly} {
tk.Se.GetSessionVars().EnableClusteredIndex = clusteredIndex
for _, t := range tests {
tk.MustExec("drop table if exists t;")
Expand Down
6 changes: 3 additions & 3 deletions ddl/db_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -178,7 +178,7 @@ func (s *testDBSuite7) TestAddIndexWithPK(c *C) {
tk.MustExec("use " + s.schemaName)

testAddIndexWithPK(tk)
tk.Se.GetSessionVars().EnableClusteredIndex = true
tk.Se.GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeOn
testAddIndexWithPK(tk)
}

Expand Down Expand Up @@ -1056,7 +1056,7 @@ func (s *testDBSuite6) TestAddMultiColumnsIndexClusterIndex(c *C) {
tk.MustExec("create database test_add_multi_col_index_clustered;")
tk.MustExec("use test_add_multi_col_index_clustered;")

tk.Se.GetSessionVars().EnableClusteredIndex = true
tk.Se.GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeOn
tk.MustExec("create table t (a int, b varchar(10), c int, primary key (a, b));")
tk.MustExec("insert into t values (1, '1', 1), (2, '2', NULL), (3, '3', 3);")
tk.MustExec("create index idx on t (a, c);")
Expand Down Expand Up @@ -1156,7 +1156,7 @@ func testAddIndex(c *C, store kv.Storage, lease time.Duration, tp testAddIndexTy
case testPartition:
tk.MustExec("set @@session.tidb_enable_table_partition = '1';")
case testClusteredIndex:
tk.Se.GetSessionVars().EnableClusteredIndex = true
tk.Se.GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeOn
}
tk.MustExec("drop table if exists test_add_index")
tk.MustExec(createTableSQL)
Expand Down
10 changes: 8 additions & 2 deletions ddl/ddl_api.go
Original file line number Diff line number Diff line change
Expand Up @@ -1523,8 +1523,14 @@ func isSingleIntPK(constr *ast.Constraint, lastCol *model.ColumnInfo) bool {
// ShouldBuildClusteredIndex is used to determine whether the CREATE TABLE statement should build a clustered index table.
func ShouldBuildClusteredIndex(ctx sessionctx.Context, opt *ast.IndexOption, isSingleIntPK bool) bool {
if opt == nil || opt.PrimaryKeyTp == model.PrimaryKeyTypeDefault {
return ctx.GetSessionVars().EnableClusteredIndex ||
(isSingleIntPK && ctx.GetSessionVars().IntPrimaryKeyDefaultAsClustered)
switch ctx.GetSessionVars().EnableClusteredIndex {
case variable.ClusteredIndexDefModeOn:
return true
case variable.ClusteredIndexDefModeIntOnly:
return !config.GetGlobalConfig().AlterPrimaryKey && isSingleIntPK
default:
return false
}
}
return opt.PrimaryKeyTp == model.PrimaryKeyTypeClustered
}
Expand Down
2 changes: 1 addition & 1 deletion ddl/failtest/fail_db_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -354,7 +354,7 @@ func (s *testFailDBSuite) TestAddIndexWorkerNum(c *C) {
tk.MustExec("use test_db")
tk.MustExec("drop table if exists test_add_index")
if s.IsCommonHandle {
tk.Se.GetSessionVars().EnableClusteredIndex = true
tk.Se.GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeOn
tk.MustExec("create table test_add_index (c1 bigint, c2 bigint, c3 bigint, primary key(c1, c3))")
} else {
tk.MustExec("create table test_add_index (c1 bigint, c2 bigint, c3 bigint, primary key(c1))")
Expand Down
10 changes: 5 additions & 5 deletions ddl/serial_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -117,7 +117,7 @@ func (s *testIntegrationSuite9) TestPrimaryKey(c *C) {
tk.MustExec("drop database if exists test_primary_key;")
tk.MustExec("create database test_primary_key;")
tk.MustExec("use test_primary_key;")
tk.Se.GetSessionVars().EnableClusteredIndex = false
tk.Se.GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeIntOnly

// Test add/drop primary key on a plain table.
tk.MustExec("drop table if exists t;")
Expand Down Expand Up @@ -325,7 +325,7 @@ func (s *testIntegrationSuite9) TestMultiRegionGetTableEndCommonHandle(c *C) {
tk.MustExec("drop database if exists test_get_endhandle")
tk.MustExec("create database test_get_endhandle")
tk.MustExec("use test_get_endhandle")
tk.Se.GetSessionVars().EnableClusteredIndex = true
tk.Se.GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeOn

tk.MustExec("create table t(a varchar(20), b int, c float, d bigint, primary key (a, b, c))")
var builder strings.Builder
Expand Down Expand Up @@ -369,7 +369,7 @@ func (s *testIntegrationSuite9) TestGetTableEndCommonHandle(c *C) {
tk.MustExec("drop database if exists test_get_endhandle")
tk.MustExec("create database test_get_endhandle")
tk.MustExec("use test_get_endhandle")
tk.Se.GetSessionVars().EnableClusteredIndex = true
tk.Se.GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeOn

tk.MustExec("create table t(a varchar(15), b bigint, c int, primary key (a, b))")
tk.MustExec("create table t1(a varchar(15), b bigint, c int, primary key (a(2), b))")
Expand Down Expand Up @@ -1406,7 +1406,7 @@ func (s *testIntegrationSuite9) TestInvisibleIndex(c *C) {

func (s *testIntegrationSuite9) TestCreateClusteredIndex(c *C) {
tk := testkit.NewTestKitWithInit(c, s.store)
tk.Se.GetSessionVars().EnableClusteredIndex = true
tk.Se.GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeOn
tk.MustExec("CREATE TABLE t1 (a int primary key, b int)")
tk.MustExec("CREATE TABLE t2 (a varchar(255) primary key, b int)")
tk.MustExec("CREATE TABLE t3 (a int, b int, c int, primary key (a, b))")
Expand Down Expand Up @@ -1447,7 +1447,7 @@ func (s *testIntegrationSuite9) TestCreateClusteredIndex(c *C) {
c.Assert(err, IsNil)
c.Assert(tbl.Meta().IsCommonHandle, IsTrue)

tk.Se.GetSessionVars().EnableClusteredIndex = false
tk.Se.GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeIntOnly
tk.MustExec("CREATE TABLE t7 (a varchar(255) primary key, b int)")
is = domain.GetDomain(ctx).InfoSchema()
tbl, err = is.TableByName(model.NewCIStr("test"), model.NewCIStr("t7"))
Expand Down
9 changes: 5 additions & 4 deletions executor/admin_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -24,6 +24,7 @@ import (
mysql "github.com/pingcap/tidb/errno"
"github.com/pingcap/tidb/executor"
"github.com/pingcap/tidb/kv"
"github.com/pingcap/tidb/sessionctx/variable"
"github.com/pingcap/tidb/table"
"github.com/pingcap/tidb/table/tables"
"github.com/pingcap/tidb/types"
Expand Down Expand Up @@ -194,7 +195,7 @@ func (s *testSuite5) TestClusteredIndexAdminRecoverIndex(c *C) {
tk.MustExec("drop database if exists test_cluster_index_admin_recover;")
tk.MustExec("create database test_cluster_index_admin_recover;")
tk.MustExec("use test_cluster_index_admin_recover;")
tk.Se.GetSessionVars().EnableClusteredIndex = true
tk.Se.GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeOn
dbName := model.NewCIStr("test_cluster_index_admin_recover")
tblName := model.NewCIStr("t")

Expand Down Expand Up @@ -310,7 +311,7 @@ func (s *testSuite5) TestAdminRecoverIndex1(c *C) {
sc := s.ctx.GetSessionVars().StmtCtx
tk.MustExec("use test")
tk.MustExec("drop table if exists admin_test")
tk.Se.GetSessionVars().EnableClusteredIndex = false
tk.Se.GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeIntOnly
tk.MustExec("create table admin_test (c1 varchar(255), c2 int, c3 int default 1, primary key(c1), unique key(c2))")
tk.MustExec("insert admin_test (c1, c2) values ('1', 1), ('2', 2), ('3', 3), ('10', 10), ('20', 20)")

Expand Down Expand Up @@ -515,7 +516,7 @@ func (s *testSuite5) TestAdminCleanupIndexPKNotHandle(c *C) {
tk := testkit.NewTestKit(c, s.store)
tk.MustExec("use test")
tk.MustExec("drop table if exists admin_test")
tk.Se.GetSessionVars().EnableClusteredIndex = false
tk.Se.GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeIntOnly
tk.MustExec("create table admin_test (c1 int, c2 int, c3 int, primary key (c1, c2))")
tk.MustExec("insert admin_test (c1, c2) values (1, 2), (3, 4), (-5, 5)")

Expand Down Expand Up @@ -627,7 +628,7 @@ func (s *testSuite5) TestClusteredAdminCleanupIndex(c *C) {
tk := testkit.NewTestKit(c, s.store)
tk.MustExec("use test")
tk.MustExec("drop table if exists admin_test")
tk.Se.GetSessionVars().EnableClusteredIndex = true
tk.Se.GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeOn
tk.MustExec("create table admin_test (c1 varchar(255), c2 int, c3 char(10) default 'c3', primary key (c1, c3), unique key(c2), key (c3))")
tk.MustExec("insert admin_test (c1, c2) values ('c1_1', 2), ('c1_2', 4), ('c1_3', NULL)")
tk.MustExec("insert admin_test (c1, c3) values ('c1_4', 'c3_4'), ('c1_5', 'c3_5'), ('c1_6', default)")
Expand Down
3 changes: 2 additions & 1 deletion executor/aggregate_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -30,6 +30,7 @@ import (
"github.com/pingcap/tidb/executor"
plannercore "github.com/pingcap/tidb/planner/core"
"github.com/pingcap/tidb/session"
"github.com/pingcap/tidb/sessionctx/variable"
"github.com/pingcap/tidb/util/sqlexec"
"github.com/pingcap/tidb/util/testkit"
"github.com/pingcap/tidb/util/testutil"
Expand Down Expand Up @@ -919,7 +920,7 @@ func (s *testSuiteAgg) TestAggEliminator(c *C) {
func (s *testSuiteAgg) TestClusterIndexMaxMinEliminator(c *C) {
tk := testkit.NewTestKitWithInit(c, s.store)
tk.MustExec("drop table if exists t;")
tk.Se.GetSessionVars().EnableClusteredIndex = true
tk.Se.GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeOn
tk.MustExec("create table t (a int, b int, c int, primary key(a, b));")
for i := 0; i < 10+1; i++ {
tk.MustExec("insert into t values (?, ?, ?)", i, i, i)
Expand Down
4 changes: 2 additions & 2 deletions executor/analyze_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -129,7 +129,7 @@ func (s *testSuite1) TestClusterIndexAnalyze(c *C) {
tk.MustExec("drop database if exists test_cluster_index_analyze;")
tk.MustExec("create database test_cluster_index_analyze;")
tk.MustExec("use test_cluster_index_analyze;")
tk.Se.GetSessionVars().EnableClusteredIndex = true
tk.Se.GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeOn

tk.MustExec("create table t (a int, b int, c int, primary key(a, b));")
for i := 0; i < 100; i++ {
Expand Down Expand Up @@ -832,7 +832,7 @@ func (s *testSuite1) TestNormalAnalyzeOnCommonHandle(c *C) {
tk := testkit.NewTestKit(c, s.store)
tk.MustExec("use test")
tk.MustExec("drop table if exists t1, t2, t3, t4")
tk.Se.GetSessionVars().EnableClusteredIndex = true
tk.Se.GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeOn
tk.MustExec("CREATE TABLE t1 (a int primary key, b int)")
tk.MustExec("insert into t1 values(1,1), (2,2), (3,3)")
tk.MustExec("CREATE TABLE t2 (a varchar(255) primary key, b int)")
Expand Down
3 changes: 2 additions & 1 deletion executor/batch_point_get_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -23,6 +23,7 @@ import (
"github.com/pingcap/tidb/domain"
"github.com/pingcap/tidb/kv"
"github.com/pingcap/tidb/session"
"github.com/pingcap/tidb/sessionctx/variable"
"github.com/pingcap/tidb/store/mockstore"
"github.com/pingcap/tidb/util/testkit"
)
Expand Down Expand Up @@ -177,7 +178,7 @@ func (s *testBatchPointGetSuite) TestBatchPointGetLockExistKey(c *C) {

errCh <- tk1.ExecToErr("use test")
errCh <- tk2.ExecToErr("use test")
tk1.Se.GetSessionVars().EnableClusteredIndex = false
tk1.Se.GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeIntOnly

errCh <- tk1.ExecToErr(fmt.Sprintf("drop table if exists %s", tableName))
errCh <- tk1.ExecToErr(fmt.Sprintf("create table %s(id int, v int, k int, %s key0(id, v))", tableName, key))
Expand Down
3 changes: 2 additions & 1 deletion executor/delete_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@ import (
"time"

. "github.com/pingcap/check"
"github.com/pingcap/tidb/sessionctx/variable"
"github.com/pingcap/tidb/util/testkit"
)

Expand Down Expand Up @@ -82,7 +83,7 @@ func (s *testSuite8) TestDeleteLockKey(c *C) {
tk1, tk2 := testkit.NewTestKit(c, s.store), testkit.NewTestKit(c, s.store)
tk1.MustExec("use test")
tk2.MustExec("use test")
tk1.Se.GetSessionVars().EnableClusteredIndex = false
tk1.Se.GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeIntOnly
tk1.MustExec(t.ddl)
tk1.MustExec(t.pre)
tk1.MustExec("begin pessimistic")
Expand Down
14 changes: 7 additions & 7 deletions executor/executor_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -2351,7 +2351,7 @@ func (s *testSuiteP2) TestClusteredIndexIsPointGet(c *C) {
tk.MustExec("create database test_cluster_index_is_point_get;")
tk.MustExec("use test_cluster_index_is_point_get;")

tk.Se.GetSessionVars().EnableClusteredIndex = true
tk.Se.GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeOn
tk.MustExec("drop table if exists t;")
tk.MustExec("create table t (a varchar(255), b int, c char(10), primary key (c, a));")
ctx := tk.Se.(sessionctx.Context)
Expand Down Expand Up @@ -3648,7 +3648,7 @@ func (s *testSuite) TestUnsignedPk(c *C) {
func (s *testSuite) TestSignedCommonHandle(c *C) {
tk := testkit.NewTestKitWithInit(c, s.store)

tk.Se.GetSessionVars().EnableClusteredIndex = true
tk.Se.GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeOn
tk.MustExec("drop table if exists t")
tk.MustExec("create table t(k1 int, k2 int, primary key(k1, k2))")
tk.MustExec("insert into t(k1, k2) value(-100, 1), (-50, 1), (0, 0), (1, 1), (3, 3)")
Expand Down Expand Up @@ -3816,7 +3816,7 @@ func (s *testSuite) TestCheckTableClusterIndex(c *C) {
tk := testkit.NewTestKit(c, s.store)

tk.MustExec("use test;")
tk.Se.GetSessionVars().EnableClusteredIndex = true
tk.Se.GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeOn
tk.MustExec("drop table if exists admin_test;")
tk.MustExec("create table admin_test (c1 int, c2 int, c3 int default 1, primary key (c1, c2), index (c1), unique key(c2));")
tk.MustExec("insert admin_test (c1, c2) values (1, 1), (2, 2), (3, 3);")
Expand Down Expand Up @@ -4260,7 +4260,7 @@ func (s *testSuite3) TestRowID(c *C) {
tk := testkit.NewTestKit(c, s.store)
tk.MustExec(`use test`)
tk.MustExec(`drop table if exists t`)
tk.Se.GetSessionVars().EnableClusteredIndex = false
tk.Se.GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeIntOnly
tk.MustExec(`create table t(a varchar(10), b varchar(10), c varchar(1), index idx(a, b, c));`)
tk.MustExec(`insert into t values('a', 'b', 'c');`)
tk.MustExec(`insert into t values('a', 'b', 'c');`)
Expand Down Expand Up @@ -4836,7 +4836,7 @@ func (s *testSplitTable) TestClusterIndexSplitTableIntegration(c *C) {
tk.MustExec("drop database if exists test_cluster_index_index_split_table_integration;")
tk.MustExec("create database test_cluster_index_index_split_table_integration;")
tk.MustExec("use test_cluster_index_index_split_table_integration;")
tk.Se.GetSessionVars().EnableClusteredIndex = true
tk.Se.GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeOn

tk.MustExec("create table t (a varchar(255), b double, c int, primary key (a, b));")

Expand Down Expand Up @@ -4891,7 +4891,7 @@ func (s *testSplitTable) TestClusterIndexShowTableRegion(c *C) {
tk.MustExec("drop database if exists cluster_index_regions;")
tk.MustExec("create database cluster_index_regions;")
tk.MustExec("use cluster_index_regions;")
tk.Se.GetSessionVars().EnableClusteredIndex = true
tk.Se.GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeOn
tk.MustExec("create table t (a int, b int, c int, primary key(a, b));")
tk.MustExec("insert t values (1, 1, 1), (2, 2, 2);")
tk.MustQuery("split table t between (1, 0) and (2, 3) regions 2;").Check(testkit.Rows("1 1"))
Expand All @@ -4914,7 +4914,7 @@ func (s *testSplitTable) TestClusterIndexShowTableRegion(c *C) {
func (s *testSuiteWithData) TestClusterIndexOuterJoinElimination(c *C) {
tk := testkit.NewTestKit(c, s.store)
tk.MustExec("use test")
tk.Se.GetSessionVars().EnableClusteredIndex = true
tk.Se.GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeOn
tk.MustExec("create table t (a int, b int, c int, primary key(a,b))")
rows := tk.MustQuery(`explain format = 'brief' select t1.a from t t1 left join t t2 on t1.a = t2.a and t1.b = t2.b`).Rows()
rowStrs := s.testData.ConvertRowsToStrings(rows)
Expand Down
Loading

0 comments on commit aee5819

Please sign in to comment.