// Copyright 2018 PingCAP, Inc. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. // You may obtain a copy of the License at // // http://www.apache.org/licenses/LICENSE-2.0 // // Unless required by applicable law or agreed to in writing, software // distributed under the License is distributed on an "AS IS" BASIS, // See the License for the specific language governing permissions and // limitations under the License. package ddl_test import ( "context" "fmt" "math" "math/rand" "strings" "sync/atomic" "time" . "github.com/pingcap/check" "github.com/pingcap/errors" "github.com/pingcap/parser/ast" "github.com/pingcap/parser/model" "github.com/pingcap/parser/mysql" "github.com/pingcap/parser/terror" "github.com/pingcap/tidb/ddl" "github.com/pingcap/tidb/ddl/testutil" "github.com/pingcap/tidb/domain" tmysql "github.com/pingcap/tidb/errno" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/meta" "github.com/pingcap/tidb/session" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/table" "github.com/pingcap/tidb/table/tables" "github.com/pingcap/tidb/tablecodec" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/admin" "github.com/pingcap/tidb/util/collate" "github.com/pingcap/tidb/util/mock" "github.com/pingcap/tidb/util/testkit" ) func (s *testIntegrationSuite3) TestCreateTableWithPartition(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test;") tk.MustExec("drop table if exists tp;") tk.MustExec(`CREATE TABLE tp (a int) PARTITION BY RANGE(a) ( PARTITION p0 VALUES LESS THAN (10), PARTITION p1 VALUES LESS THAN (20), PARTITION p2 VALUES LESS THAN (MAXVALUE) );`) ctx := tk.Se.(sessionctx.Context) is := domain.GetDomain(ctx).InfoSchema() tbl, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("tp")) c.Assert(err, IsNil) c.Assert(tbl.Meta().Partition, NotNil) part := tbl.Meta().Partition c.Assert(part.Type, Equals, model.PartitionTypeRange) c.Assert(part.Expr, Equals, "`a`") for _, pdef := range part.Definitions { c.Assert(pdef.ID, Greater, int64(0)) } c.Assert(part.Definitions, HasLen, 3) c.Assert(part.Definitions[0].LessThan[0], Equals, "10") c.Assert(part.Definitions[0].Name.L, Equals, "p0") c.Assert(part.Definitions[1].LessThan[0], Equals, "20") c.Assert(part.Definitions[1].Name.L, Equals, "p1") c.Assert(part.Definitions[2].LessThan[0], Equals, "MAXVALUE") c.Assert(part.Definitions[2].Name.L, Equals, "p2") tk.MustExec("drop table if exists employees;") sql1 := `create table employees ( id int not null, hired int not null ) partition by range( hired ) ( partition p1 values less than (1991), partition p2 values less than (1996), partition p2 values less than (2001) );` tk.MustGetErrCode(sql1, tmysql.ErrSameNamePartition) sql2 := `create table employees ( id int not null, hired int not null ) partition by range( hired ) ( partition p1 values less than (1998), partition p2 values less than (1996), partition p3 values less than (2001) );` tk.MustGetErrCode(sql2, tmysql.ErrRangeNotIncreasing) sql3 := `create table employees ( id int not null, hired int not null ) partition by range( hired ) ( partition p1 values less than (1998), partition p2 values less than maxvalue, partition p3 values less than (2001) );` tk.MustGetErrCode(sql3, tmysql.ErrPartitionMaxvalue) sql4 := `create table t4 ( a int not null, b int not null ) partition by range( a ) ( partition p1 values less than maxvalue, partition p2 values less than (1991), partition p3 values less than (1995) );` tk.MustGetErrCode(sql4, tmysql.ErrPartitionMaxvalue) _, err = tk.Exec(`CREATE TABLE rc ( a INT NOT NULL, b INT NOT NULL, c INT NOT NULL ) partition by range columns(a,b,c) ( partition p0 values less than (10,5,1), partition p2 values less than (50,maxvalue,10), partition p3 values less than (65,30,13), partition p4 values less than (maxvalue,30,40) );`) c.Assert(err, IsNil) sql6 := `create table employees ( id int not null, hired int not null ) partition by range( hired ) ( partition p0 values less than (6 , 10) );` tk.MustGetErrCode(sql6, tmysql.ErrTooManyValues) sql7 := `create table t7 ( a int not null, b int not null ) partition by range( a ) ( partition p1 values less than (1991), partition p2 values less than maxvalue, partition p3 values less than maxvalue, partition p4 values less than (1995), partition p5 values less than maxvalue );` tk.MustGetErrCode(sql7, tmysql.ErrPartitionMaxvalue) sql18 := `create table t8 ( a int not null, b int not null ) partition by range( a ) ( partition p1 values less than (19xx91), partition p2 values less than maxvalue );` tk.MustGetErrCode(sql18, mysql.ErrBadField) sql9 := `create TABLE t9 ( col1 int ) partition by range( case when col1 > 0 then 10 else 20 end ) ( partition p0 values less than (2), partition p1 values less than (6) );` tk.MustGetErrCode(sql9, tmysql.ErrPartitionFunctionIsNotAllowed) _, err = tk.Exec(`CREATE TABLE t9 ( a INT NOT NULL, b INT NOT NULL, c INT NOT NULL ) partition by range columns(a) ( partition p0 values less than (10), partition p2 values less than (20), partition p3 values less than (20) );`) c.Assert(ddl.ErrRangeNotIncreasing.Equal(err), IsTrue) tk.MustGetErrCode(`create TABLE t10 (c1 int,c2 int) partition by range(c1 / c2 ) (partition p0 values less than (2));`, tmysql.ErrPartitionFunctionIsNotAllowed) tk.MustExec(`create TABLE t11 (c1 int,c2 int) partition by range(c1 div c2 ) (partition p0 values less than (2));`) tk.MustExec(`create TABLE t12 (c1 int,c2 int) partition by range(c1 + c2 ) (partition p0 values less than (2));`) tk.MustExec(`create TABLE t13 (c1 int,c2 int) partition by range(c1 - c2 ) (partition p0 values less than (2));`) tk.MustExec(`create TABLE t14 (c1 int,c2 int) partition by range(c1 * c2 ) (partition p0 values less than (2));`) tk.MustExec(`create TABLE t15 (c1 int,c2 int) partition by range( abs(c1) ) (partition p0 values less than (2));`) tk.MustExec(`create TABLE t16 (c1 int) partition by range( c1) (partition p0 values less than (10));`) tk.MustGetErrCode(`create TABLE t17 (c1 int,c2 float) partition by range(c1 + c2 ) (partition p0 values less than (2));`, tmysql.ErrPartitionFuncNotAllowed) tk.MustGetErrCode(`create TABLE t18 (c1 int,c2 float) partition by range( floor(c2) ) (partition p0 values less than (2));`, tmysql.ErrPartitionFuncNotAllowed) tk.MustExec(`create TABLE t19 (c1 int,c2 float) partition by range( floor(c1) ) (partition p0 values less than (2));`) tk.MustExec(`create TABLE t20 (c1 int,c2 bit(10)) partition by range(c2) (partition p0 values less than (10));`) tk.MustExec(`create TABLE t21 (c1 int,c2 year) partition by range( c2 ) (partition p0 values less than (2000));`) tk.MustGetErrCode(`create TABLE t24 (c1 float) partition by range( c1 ) (partition p0 values less than (2000));`, tmysql.ErrFieldTypeNotAllowedAsPartitionField) // test check order. The sql below have 2 problem: 1. ErrFieldTypeNotAllowedAsPartitionField 2. ErrPartitionMaxvalue , mysql will return ErrPartitionMaxvalue. tk.MustGetErrCode(`create TABLE t25 (c1 float) partition by range( c1 ) (partition p1 values less than maxvalue,partition p0 values less than (2000));`, tmysql.ErrPartitionMaxvalue) // Fix issue 7362. tk.MustExec("create table test_partition(id bigint, name varchar(255), primary key(id)) ENGINE=InnoDB DEFAULT CHARSET=utf8 PARTITION BY RANGE COLUMNS(id) (PARTITION p1 VALUES LESS THAN (10) ENGINE = InnoDB);") // 'Less than' in partition expression could be a constant expression, notice that // the SHOW result changed. tk.MustExec(`create table t26 (a date) partition by range(to_seconds(a))( partition p0 values less than (to_seconds('2004-01-01')), partition p1 values less than (to_seconds('2005-01-01')));`) tk.MustQuery("show create table t26").Check( testkit.Rows("t26 CREATE TABLE `t26` (\n `a` date DEFAULT NULL\n) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin\nPARTITION BY RANGE ( TO_SECONDS(`a`) ) (\n PARTITION `p0` VALUES LESS THAN (63240134400),\n PARTITION `p1` VALUES LESS THAN (63271756800)\n)")) tk.MustExec(`create table t27 (a bigint unsigned not null) partition by range(a) ( partition p0 values less than (10), partition p1 values less than (100), partition p2 values less than (1000), partition p3 values less than (18446744073709551000), partition p4 values less than (18446744073709551614) );`) tk.MustExec(`create table t28 (a bigint unsigned not null) partition by range(a) ( partition p0 values less than (10), partition p1 values less than (100), partition p2 values less than (1000), partition p3 values less than (18446744073709551000 + 1), partition p4 values less than (18446744073709551000 + 10) );`) tk.MustExec("set @@tidb_enable_table_partition = 1") tk.MustExec("set @@tidb_enable_table_partition = 1") tk.MustExec(`create table t30 ( a int, b float, c varchar(30)) partition by range columns (a, b) (partition p0 values less than (10, 10.0))`) tk.MustQuery("show warnings").Check(testkit.Rows("Warning 8200 Unsupported partition type, treat as normal table")) tk.MustGetErrCode(`create table t31 (a int not null) partition by range( a );`, tmysql.ErrPartitionsMustBeDefined) tk.MustGetErrCode(`create table t32 (a int not null) partition by range columns( a );`, tmysql.ErrPartitionsMustBeDefined) tk.MustGetErrCode(`create table t33 (a int, b int) partition by hash(a) partitions 0;`, tmysql.ErrNoParts) tk.MustGetErrCode(`create table t33 (a timestamp, b int) partition by hash(a) partitions 30;`, tmysql.ErrFieldTypeNotAllowedAsPartitionField) tk.MustGetErrCode(`CREATE TABLE t34 (c0 INT) PARTITION BY HASH((CASE WHEN 0 THEN 0 ELSE c0 END )) PARTITIONS 1;`, tmysql.ErrPartitionFunctionIsNotAllowed) tk.MustGetErrCode(`CREATE TABLE t0(c0 INT) PARTITION BY HASH((c0= 10 { break } step := 10 rand.Seed(time.Now().Unix()) // delete some rows, and add some data for i := count; i < count+step; i++ { n := rand.Intn(count) tk.MustExec("delete from t1 where c1 = ?", n) tk.MustExec("insert into t1 values (?, ?, ?)", i+10, i, i) } count += step times++ } } t := testGetTableByName(c, ctx, "test_db", "t1") // Only one partition id test is taken here. pid := t.Meta().Partition.Definitions[0].ID for _, tidx := range t.Indices() { c.Assert(strings.EqualFold(tidx.Meta().Name.L, "c3_index"), IsFalse) } idx := tables.NewIndex(pid, t.Meta(), c3IdxInfo) checkDelRangeDone(c, ctx, idx) tk.MustExec("drop table t1") } func backgroundExecOnJobUpdatedExported(c *C, store kv.Storage, ctx sessionctx.Context, hook *ddl.TestDDLCallback, idxName string) ( func(*model.Job), *model.IndexInfo, error) { var checkErr error first := true c3IdxInfo := &model.IndexInfo{} hook.OnJobUpdatedExported = func(job *model.Job) { addIndexNotFirstReorg := (job.Type == model.ActionAddIndex || job.Type == model.ActionAddPrimaryKey) && job.SchemaState == model.StateWriteReorganization && job.SnapshotVer != 0 // If the action is adding index and the state is writing reorganization, it want to test the case of cancelling the job when backfilling indexes. // When the job satisfies this case of addIndexNotFirstReorg, the worker will start to backfill indexes. if !addIndexNotFirstReorg { // Get the index's meta. if c3IdxInfo != nil { return } t := testGetTableByName(c, ctx, "test_db", "t1") for _, index := range t.WritableIndices() { if index.Meta().Name.L == idxName { c3IdxInfo = index.Meta() } } return } // The job satisfies the case of addIndexNotFirst for the first time, the worker hasn't finished a batch of backfill indexes. if first { first = false return } if checkErr != nil { return } hookCtx := mock.NewContext() hookCtx.Store = store err := hookCtx.NewTxn(context.Background()) if err != nil { checkErr = errors.Trace(err) return } jobIDs := []int64{job.ID} txn, err := hookCtx.Txn(true) if err != nil { checkErr = errors.Trace(err) return } errs, err := admin.CancelJobs(txn, jobIDs) if err != nil { checkErr = errors.Trace(err) return } // It only tests cancel one DDL job. if errs[0] != nil { checkErr = errors.Trace(errs[0]) return } txn, err = hookCtx.Txn(true) if err != nil { checkErr = errors.Trace(err) return } err = txn.Commit(context.Background()) if err != nil { checkErr = errors.Trace(err) } } return hook.OnJobUpdatedExported, c3IdxInfo, checkErr } func (s *testIntegrationSuite5) TestPartitionAddPrimaryKey(c *C) { tk := testkit.NewTestKit(c, s.store) testPartitionAddIndexOrPK(c, tk, "primary key") } func (s *testIntegrationSuite1) TestPartitionAddIndex(c *C) { tk := testkit.NewTestKit(c, s.store) testPartitionAddIndexOrPK(c, tk, "index") } func testPartitionAddIndexOrPK(c *C, tk *testkit.TestKit, key string) { tk.MustExec("use test") tk.MustExec(`create table partition_add_idx ( id int not null, hired date not null ) partition by range( year(hired) ) ( partition p1 values less than (1991), partition p3 values less than (2001), partition p4 values less than (2004), partition p5 values less than (2008), partition p6 values less than (2012), partition p7 values less than (2018) );`) testPartitionAddIndex(tk, c, 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, hired date not null ) partition by hash( year(hired) ) partitions 4;`) testPartitionAddIndex(tk, c, key) // 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)) tk.MustExec("admin check table t1;") // Test range partition for pr 10475. tk.MustExec("drop table t1") tk.MustExec("create table t1 (a int, b int, unique key(a)) partition by range (a) (partition p0 values less than (10), partition p1 values less than (20));") tk.MustExec("insert into t1 values (0,0);") tk.MustExec(fmt.Sprintf("alter table t1 add %s idx(a)", key)) tk.MustExec("admin check table t1;") } func testPartitionAddIndex(tk *testkit.TestKit, c *C, key string) { idxName1 := "idx1" f := func(end int, isPK bool) string { dml := fmt.Sprintf("insert into partition_add_idx values") for i := 0; i < end; i++ { dVal := 1988 + rand.Intn(30) if isPK { dVal = 1518 + i } dml += fmt.Sprintf("(%d, '%d-01-01')", i, dVal) if i != end-1 { dml += "," } } return dml } var dml string if key == "primary key" { idxName1 = "primary" // For the primary key, hired must be unique. dml = f(500, true) } else { dml = f(500, false) } tk.MustExec(dml) tk.MustExec(fmt.Sprintf("alter table partition_add_idx add %s idx1 (hired)", key)) tk.MustExec("alter table partition_add_idx add index idx2 (id, hired)") ctx := tk.Se.(sessionctx.Context) is := domain.GetDomain(ctx).InfoSchema() t, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("partition_add_idx")) c.Assert(err, IsNil) var idx1 table.Index for _, idx := range t.Indices() { if idx.Meta().Name.L == idxName1 { idx1 = idx break } } c.Assert(idx1, NotNil) tk.MustQuery(fmt.Sprintf("select count(hired) from partition_add_idx use index(%s)", idxName1)).Check(testkit.Rows("500")) tk.MustQuery("select count(id) from partition_add_idx use index(idx2)").Check(testkit.Rows("500")) tk.MustExec("admin check table partition_add_idx") tk.MustExec("drop table partition_add_idx") } func (s *testIntegrationSuite5) TestDropSchemaWithPartitionTable(c *C) { tk := testkit.NewTestKit(c, s.store) 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") tk.MustExec(`create table t_part (a int key) partition by range(a) ( partition p0 values less than (10), partition p1 values less than (20) );`) tk.MustExec("insert into t_part values (1),(2),(11),(12);") ctx := s.ctx tbl := testGetTableByName(c, ctx, "test_db_with_partition", "t_part") // check records num before drop database. recordsNum := getPartitionTableRecordsNum(c, ctx, tbl.(table.PartitionedTable)) c.Assert(recordsNum, Equals, 4) tk.MustExec("drop database if exists test_db_with_partition") // check job args. rs, err := tk.Exec("admin show ddl jobs") c.Assert(err, IsNil) rows, err := session.GetRows4Test(context.Background(), tk.Se, rs) c.Assert(err, IsNil) row := rows[0] c.Assert(row.GetString(3), Equals, "drop schema") jobID := row.GetInt64(0) kv.RunInNewTxn(s.store, false, func(txn kv.Transaction) error { t := meta.NewMeta(txn) historyJob, err := t.GetHistoryDDLJob(jobID) c.Assert(err, IsNil) var tableIDs []int64 err = historyJob.DecodeArgs(&tableIDs) c.Assert(err, IsNil) // There is 2 partitions. c.Assert(len(tableIDs), Equals, 3) return nil }) // check records num after drop database. for i := 0; i < waitForCleanDataRound; i++ { recordsNum = getPartitionTableRecordsNum(c, ctx, tbl.(table.PartitionedTable)) if recordsNum != 0 { time.Sleep(waitForCleanDataInterval) } else { break } } c.Assert(recordsNum, Equals, 0) } func getPartitionTableRecordsNum(c *C, ctx sessionctx.Context, tbl table.PartitionedTable) int { num := 0 info := tbl.Meta().GetPartitionInfo() for _, def := range info.Definitions { pid := def.ID partition := tbl.(table.PartitionedTable).GetPartition(pid) startKey := partition.RecordKey(math.MinInt64) c.Assert(ctx.NewTxn(context.Background()), IsNil) err := partition.IterRecords(ctx, startKey, partition.Cols(), func(h int64, data []types.Datum, cols []*table.Column) (bool, error) { num++ return true, nil }) c.Assert(err, IsNil) } return num } func (s *testIntegrationSuite3) TestPartitionErrorCode(c *C) { tk := testkit.NewTestKit(c, s.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") tk.MustExec(`create table employees ( id int not null, fname varchar(30), lname varchar(30), hired date not null default '1970-01-01', separated date not null default '9999-12-31', job_code int, store_id int ) partition by hash(store_id) partitions 4;`) _, err := tk.Exec("alter table employees add partition partitions 8;") c.Assert(ddl.ErrUnsupportedAddPartition.Equal(err), IsTrue) _, err = tk.Exec("alter table employees add partition (partition p5 values less than (42));") c.Assert(ddl.ErrUnsupportedAddPartition.Equal(err), IsTrue) // coalesce partition tk.MustExec(`create table clients ( id int, fname varchar(30), lname varchar(30), signed date ) partition by hash( month(signed) ) partitions 12;`) _, err = tk.Exec("alter table clients coalesce partition 4;") c.Assert(ddl.ErrUnsupportedCoalescePartition.Equal(err), IsTrue) tk.MustExec(`create table t_part (a int key) partition by range(a) ( partition p0 values less than (10), partition p1 values less than (20) );`) _, err = tk.Exec("alter table t_part coalesce partition 4;") c.Assert(ddl.ErrCoalesceOnlyOnHashPartition.Equal(err), IsTrue) tk.MustGetErrCode(`alter table t_part reorganize partition p0, p1 into ( partition p0 values less than (1980));`, tmysql.ErrUnsupportedDDLOperation) tk.MustGetErrCode("alter table t_part check partition p0, p1;", tmysql.ErrUnsupportedDDLOperation) tk.MustGetErrCode("alter table t_part optimize partition p0,p1;", tmysql.ErrUnsupportedDDLOperation) tk.MustGetErrCode("alter table t_part rebuild partition p0,p1;", tmysql.ErrUnsupportedDDLOperation) tk.MustGetErrCode("alter table t_part remove partitioning;", tmysql.ErrUnsupportedDDLOperation) tk.MustExec("create table t_part2 like t_part") tk.MustGetErrCode("alter table t_part exchange partition p0 with table t_part2", tmysql.ErrUnsupportedDDLOperation) } func (s *testIntegrationSuite5) TestConstAndTimezoneDepent(c *C) { tk := testkit.NewTestKit(c, s.store) // add partition tk.MustExec("set @@session.tidb_enable_table_partition = 1") tk.MustExec("drop database if exists test_db_with_partition_const") tk.MustExec("create database test_db_with_partition_const") tk.MustExec("use test_db_with_partition_const") sql1 := `create table t1 ( id int ) partition by range(4) ( partition p1 values less than (10) );` tk.MustGetErrCode(sql1, tmysql.ErrWrongExprInPartitionFunc) sql2 := `create table t2 ( time_recorded timestamp ) partition by range(TO_DAYS(time_recorded)) ( partition p1 values less than (1559192604) );` tk.MustGetErrCode(sql2, tmysql.ErrWrongExprInPartitionFunc) sql3 := `create table t3 ( id int ) partition by range(DAY(id)) ( partition p1 values less than (1) );` tk.MustGetErrCode(sql3, tmysql.ErrWrongExprInPartitionFunc) sql4 := `create table t4 ( id int ) partition by hash(4) partitions 4 ;` tk.MustGetErrCode(sql4, tmysql.ErrWrongExprInPartitionFunc) sql5 := `create table t5 ( time_recorded timestamp ) partition by range(to_seconds(time_recorded)) ( partition p1 values less than (1559192604) );` tk.MustGetErrCode(sql5, tmysql.ErrWrongExprInPartitionFunc) sql6 := `create table t6 ( id int ) partition by range(to_seconds(id)) ( partition p1 values less than (1559192604) );` tk.MustGetErrCode(sql6, tmysql.ErrWrongExprInPartitionFunc) sql7 := `create table t7 ( time_recorded timestamp ) partition by range(abs(time_recorded)) ( partition p1 values less than (1559192604) );` tk.MustGetErrCode(sql7, tmysql.ErrWrongExprInPartitionFunc) sql8 := `create table t2332 ( time_recorded time ) partition by range(TO_DAYS(time_recorded)) ( partition p0 values less than (1) );` tk.MustGetErrCode(sql8, tmysql.ErrWrongExprInPartitionFunc) sql9 := `create table t1 ( id int ) partition by hash(4) partitions 4;` tk.MustGetErrCode(sql9, tmysql.ErrWrongExprInPartitionFunc) sql10 := `create table t1 ( id int ) partition by hash(ed) partitions 4;` tk.MustGetErrCode(sql10, tmysql.ErrBadField) sql11 := `create table t2332 ( time_recorded time ) partition by range(TO_SECONDS(time_recorded)) ( partition p0 values less than (1) );` tk.MustGetErrCode(sql11, tmysql.ErrWrongExprInPartitionFunc) sql12 := `create table t2332 ( time_recorded time ) partition by range(TO_SECONDS(time_recorded)) ( partition p0 values less than (1) );` tk.MustGetErrCode(sql12, tmysql.ErrWrongExprInPartitionFunc) sql13 := `create table t2332 ( time_recorded time ) partition by range(day(time_recorded)) ( partition p0 values less than (1) );` tk.MustGetErrCode(sql13, tmysql.ErrWrongExprInPartitionFunc) sql14 := `create table t2332 ( time_recorded timestamp ) partition by range(day(time_recorded)) ( partition p0 values less than (1) );` tk.MustGetErrCode(sql14, tmysql.ErrWrongExprInPartitionFunc) } func (s *testIntegrationSuite5) TestConstAndTimezoneDepent2(c *C) { tk := testkit.NewTestKit(c, s.store) // add partition tk.MustExec("set @@session.tidb_enable_table_partition = 1") tk.MustExec("drop database if exists test_db_with_partition_const") tk.MustExec("create database test_db_with_partition_const") tk.MustExec("use test_db_with_partition_const") tk.MustExec(`create table t1 ( time_recorded datetime ) partition by range(TO_DAYS(time_recorded)) ( partition p0 values less than (1));`) tk.MustExec(`create table t2 ( time_recorded date ) partition by range(TO_DAYS(time_recorded)) ( partition p0 values less than (1));`) tk.MustExec(`create table t3 ( time_recorded date ) partition by range(TO_SECONDS(time_recorded)) ( partition p0 values less than (1));`) tk.MustExec(`create table t4 ( time_recorded date ) partition by range(TO_SECONDS(time_recorded)) ( partition p0 values less than (1));`) tk.MustExec(`create table t5 ( time_recorded timestamp ) partition by range(unix_timestamp(time_recorded)) ( partition p1 values less than (1559192604) );`) } func (s *testIntegrationSuite3) TestUnsupportedPartitionManagementDDLs(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test;") tk.MustExec("drop table if exists test_1465;") tk.MustExec(` create table test_1465 (a int) partition by range(a) ( partition p1 values less than (10), partition p2 values less than (20), partition p3 values less than (30) ); `) _, err := tk.Exec("alter table test_1465 truncate partition p1, p2") c.Assert(err, ErrorMatches, ".*Unsupported multi schema change") _, err = tk.Exec("alter table test_1465 drop partition p1, p2") c.Assert(err, ErrorMatches, ".*Unsupported multi schema change") _, err = tk.Exec("alter table test_1465 partition by hash(a)") c.Assert(err, ErrorMatches, ".*alter table partition is unsupported") } func (s *testIntegrationSuite3) TestCommitWhenSchemaChange(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") tk.MustExec(`create table schema_change (a int, b timestamp) partition by range(a) ( partition p0 values less than (4), partition p1 values less than (7), partition p2 values less than (11) )`) tk2 := testkit.NewTestKit(c, s.store) tk2.MustExec("use test") tk.MustExec("begin") tk.MustExec("insert into schema_change values (1, '2019-12-25 13:27:42')") tk.MustExec("insert into schema_change values (3, '2019-12-25 13:27:43')") tk2.MustExec("alter table schema_change add index idx(b)") tk.MustExec("insert into schema_change values (5, '2019-12-25 13:27:43')") tk.MustExec("insert into schema_change values (9, '2019-12-25 13:27:44')") atomic.StoreUint32(&session.SchemaChangedWithoutRetry, 1) _, err := tk.Se.Execute(context.Background(), "commit") atomic.StoreUint32(&session.SchemaChangedWithoutRetry, 0) c.Assert(domain.ErrInfoSchemaChanged.Equal(err), IsTrue) // Cover a bug that schema validator does not prevent transaction commit when // the schema has changeed on the partitioned table. // That bug will cause data and index inconsistency! tk.MustExec("admin check table schema_change") tk.MustQuery("select * from schema_change").Check(testkit.Rows()) } func (s *testIntegrationSuite7) TestCreatePartitionTableWithWrongType(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") tk.MustExec("drop table if exists t") var err error _, err = tk.Exec(`create table t( b int(10) ) partition by range columns (b) ( partition p0 values less than (0x10), partition p3 values less than (0x20) )`) c.Assert(err, NotNil) c.Assert(ddl.ErrWrongTypeColumnValue.Equal(err), IsTrue) _, err = tk.Exec(`create table t( b int(10) ) partition by range columns (b) ( partition p0 values less than ('g'), partition p3 values less than ('k') )`) c.Assert(err, NotNil) c.Assert(ddl.ErrWrongTypeColumnValue.Equal(err), IsTrue) _, err = tk.Exec(`create table t( b char(10) ) partition by range columns (b) ( partition p0 values less than (30), partition p3 values less than (60) )`) c.Assert(err, NotNil) c.Assert(ddl.ErrWrongTypeColumnValue.Equal(err), IsTrue) _, err = tk.Exec(`create table t( b datetime ) partition by range columns (b) ( partition p0 values less than ('g'), partition p3 values less than ('m') )`) c.Assert(err, NotNil) } func (s *testIntegrationSuite7) TestAddPartitionForTableWithWrongType(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") tk.MustExec("drop tables if exists t_int, t_char, t_date") tk.MustExec(`create table t_int(b int(10)) partition by range columns (b) ( partition p0 values less than (10) )`) tk.MustExec(`create table t_char(b char(10)) partition by range columns (b) ( partition p0 values less than ('a') )`) tk.MustExec(`create table t_date(b datetime) partition by range columns (b) ( partition p0 values less than ('2020-09-01') )`) var err error _, err = tk.Exec("alter table t_int add partition (partition p1 values less than ('g'))") c.Assert(err, NotNil) c.Assert(ddl.ErrWrongTypeColumnValue.Equal(err), IsTrue) _, err = tk.Exec("alter table t_int add partition (partition p1 values less than (0x20))") c.Assert(err, NotNil) c.Assert(ddl.ErrWrongTypeColumnValue.Equal(err), IsTrue) _, err = tk.Exec("alter table t_char add partition (partition p1 values less than (0x20))") c.Assert(err, NotNil) c.Assert(ddl.ErrWrongTypeColumnValue.Equal(err), IsTrue) _, err = tk.Exec("alter table t_char add partition (partition p1 values less than (10))") c.Assert(err, NotNil) c.Assert(ddl.ErrWrongTypeColumnValue.Equal(err), IsTrue) _, err = tk.Exec("alter table t_date add partition (partition p1 values less than ('m'))") c.Assert(err, NotNil) c.Assert(ddl.ErrWrongTypeColumnValue.Equal(err), IsTrue) _, err = tk.Exec("alter table t_date add partition (partition p1 values less than (0x20))") c.Assert(err, NotNil) c.Assert(ddl.ErrWrongTypeColumnValue.Equal(err), IsTrue) _, err = tk.Exec("alter table t_date add partition (partition p1 values less than (20))") c.Assert(err, NotNil) c.Assert(ddl.ErrWrongTypeColumnValue.Equal(err), IsTrue) }