diff --git a/ddl/column_modify_test.go b/ddl/column_modify_test.go index 654329c43f3b3..3a86e6b9055b5 100644 --- a/ddl/column_modify_test.go +++ b/ddl/column_modify_test.go @@ -40,6 +40,7 @@ import ( "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/admin" "github.com/pingcap/tidb/util/mock" + "github.com/pingcap/tidb/util/testutil" "github.com/stretchr/testify/require" ) @@ -909,3 +910,152 @@ func TestColumnCheck(t *testing.T) { require.Equal(t, uint16(1), tk.Session().GetSessionVars().StmtCtx.WarningCount()) tk.MustQuery("show warnings").Check(testkit.RowsWithSep("|", "Warning|8231|CONSTRAINT CHECK is not supported")) } + +func TestModifyGeneratedColumn(t *testing.T) { + store, clean := testkit.CreateMockStoreWithSchemaLease(t, columnModifyLease) + defer clean() + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + modIdxColErrMsg := "[ddl:3106]'modifying an indexed column' is not supported for generated columns." + modStoredColErrMsg := "[ddl:3106]'modifying a stored column' is not supported for generated columns." + + // Modify column with single-col-index. + tk.MustExec("drop table if exists t1;") + tk.MustExec("create table t1 (a int, b int as (a+1), index idx(b));") + tk.MustExec("insert into t1 set a=1;") + tk.MustGetErrMsg("alter table t1 modify column b int as (a+2);", modIdxColErrMsg) + tk.MustExec("drop index idx on t1;") + tk.MustExec("alter table t1 modify b int as (a+2);") + tk.MustQuery("select * from t1").Check(testkit.Rows("1 3")) + + // Modify column with multi-col-index. + tk.MustExec("drop table t1;") + tk.MustExec("create table t1 (a int, b int as (a+1), index idx(a, b));") + tk.MustExec("insert into t1 set a=1;") + tk.MustGetErrMsg("alter table t1 modify column b int as (a+2);", modIdxColErrMsg) + tk.MustExec("drop index idx on t1;") + tk.MustExec("alter table t1 modify b int as (a+2);") + tk.MustQuery("select * from t1").Check(testkit.Rows("1 3")) + + // Modify column with stored status to a different expression. + tk.MustExec("drop table t1;") + tk.MustExec("create table t1 (a int, b int as (a+1) stored);") + tk.MustExec("insert into t1 set a=1;") + tk.MustGetErrMsg("alter table t1 modify column b int as (a+2) stored;", modStoredColErrMsg) + + // Modify column with stored status to the same expression. + tk.MustExec("drop table t1;") + tk.MustExec("create table t1 (a int, b int as (a+1) stored);") + tk.MustExec("insert into t1 set a=1;") + tk.MustExec("alter table t1 modify column b bigint as (a+1) stored;") + tk.MustExec("alter table t1 modify column b bigint as (a + 1) stored;") + tk.MustQuery("select * from t1").Check(testkit.Rows("1 2")) + + // Modify column with index to the same expression. + tk.MustExec("drop table t1;") + tk.MustExec("create table t1 (a int, b int as (a+1), index idx(b));") + tk.MustExec("insert into t1 set a=1;") + tk.MustExec("alter table t1 modify column b bigint as (a+1);") + tk.MustExec("alter table t1 modify column b bigint as (a + 1);") + tk.MustQuery("select * from t1").Check(testkit.Rows("1 2")) + + // Modify column from non-generated to stored generated. + tk.MustExec("drop table t1;") + tk.MustExec("create table t1 (a int, b int);") + tk.MustGetErrMsg("alter table t1 modify column b bigint as (a+1) stored;", modStoredColErrMsg) + + // Modify column from stored generated to non-generated. + tk.MustExec("drop table t1;") + tk.MustExec("create table t1 (a int, b int as (a+1) stored);") + tk.MustExec("insert into t1 set a=1;") + tk.MustExec("alter table t1 modify column b int;") + tk.MustQuery("select * from t1").Check(testkit.Rows("1 2")) +} + +func TestCheckColumnDefaultValue(t *testing.T) { + store, clean := testkit.CreateMockStoreWithSchemaLease(t, columnModifyLease) + defer clean() + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("drop table if exists text_default_text;") + tk.MustGetErrCode("create table text_default_text(c1 text not null default '');", errno.ErrBlobCantHaveDefault) + tk.MustGetErrCode("create table text_default_text(c1 text not null default 'scds');", errno.ErrBlobCantHaveDefault) + + tk.MustExec("drop table if exists text_default_json;") + tk.MustGetErrCode("create table text_default_json(c1 json not null default '');", errno.ErrBlobCantHaveDefault) + tk.MustGetErrCode("create table text_default_json(c1 json not null default 'dfew555');", errno.ErrBlobCantHaveDefault) + + tk.MustExec("drop table if exists text_default_blob;") + tk.MustGetErrCode("create table text_default_blob(c1 blob not null default '');", errno.ErrBlobCantHaveDefault) + tk.MustGetErrCode("create table text_default_blob(c1 blob not null default 'scds54');", errno.ErrBlobCantHaveDefault) + + tk.MustExec("set sql_mode='';") + tk.MustExec("create table text_default_text(c1 text not null default '');") + tk.MustQuery(`show create table text_default_text`).Check(testutil.RowsWithSep("|", + "text_default_text CREATE TABLE `text_default_text` (\n"+ + " `c1` text NOT NULL\n"+ + ") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin", + )) + is := domain.GetDomain(tk.Session()).InfoSchema() + tblInfo, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("text_default_text")) + require.NoError(t, err) + require.Empty(t, tblInfo.Meta().Columns[0].DefaultValue) + + tk.MustExec("create table text_default_blob(c1 blob not null default '');") + tk.MustQuery(`show create table text_default_blob`).Check(testutil.RowsWithSep("|", + "text_default_blob CREATE TABLE `text_default_blob` (\n"+ + " `c1` blob NOT NULL\n"+ + ") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin", + )) + is = domain.GetDomain(tk.Session()).InfoSchema() + tblInfo, err = is.TableByName(model.NewCIStr("test"), model.NewCIStr("text_default_blob")) + require.NoError(t, err) + require.Empty(t, tblInfo.Meta().Columns[0].DefaultValue) + + tk.MustExec("create table text_default_json(c1 json not null default '');") + tk.MustQuery(`show create table text_default_json`).Check(testutil.RowsWithSep("|", + "text_default_json CREATE TABLE `text_default_json` (\n"+ + " `c1` json NOT NULL DEFAULT 'null'\n"+ + ") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin", + )) + is = domain.GetDomain(tk.Session()).InfoSchema() + tblInfo, err = is.TableByName(model.NewCIStr("test"), model.NewCIStr("text_default_json")) + require.NoError(t, err) + require.Equal(t, "null", tblInfo.Meta().Columns[0].DefaultValue) +} + +func TestCheckConvertToCharacter(t *testing.T) { + store, clean := testkit.CreateMockStoreWithSchemaLease(t, columnModifyLease) + defer clean() + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("create table t(a varchar(10) charset binary);") + is := domain.GetDomain(tk.Session()).InfoSchema() + tbl, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) + require.NoError(t, err) + tk.MustGetErrCode("alter table t modify column a varchar(10) charset utf8 collate utf8_bin", errno.ErrUnsupportedDDLOperation) + tk.MustGetErrCode("alter table t modify column a varchar(10) charset utf8mb4 collate utf8mb4_bin", errno.ErrUnsupportedDDLOperation) + tk.MustGetErrCode("alter table t modify column a varchar(10) charset latin1 collate latin1_bin", errno.ErrUnsupportedDDLOperation) + require.Equal(t, "binary", tbl.Cols()[0].Charset) +} + +func TestAddMultiColumnsIndex(t *testing.T) { + store, clean := testkit.CreateMockStoreWithSchemaLease(t, columnModifyLease) + defer clean() + tk := testkit.NewTestKit(t, store) + tk.MustExec("drop database if exists tidb;") + tk.MustExec("create database tidb;") + tk.MustExec("use tidb;") + tk.MustExec("create table tidb.test (a int auto_increment primary key, b int);") + tk.MustExec("insert tidb.test values (1, 1);") + tk.MustExec("update tidb.test set b = b + 1 where a = 1;") + tk.MustExec("insert into tidb.test values (2, 2);") + // Test that the b value is nil. + tk.MustExec("insert into tidb.test (a) values (3);") + tk.MustExec("insert into tidb.test values (4, 4);") + // Test that the b value is nil again. + tk.MustExec("insert into tidb.test (a) values (5);") + tk.MustExec("insert tidb.test values (6, 6);") + tk.MustExec("alter table tidb.test add index idx1 (a, b);") + tk.MustExec("admin check table test") +} diff --git a/ddl/db_legacy_test.go b/ddl/db_legacy_test.go index 35ee20685b471..49459d5a8581e 100644 --- a/ddl/db_legacy_test.go +++ b/ddl/db_legacy_test.go @@ -30,7 +30,6 @@ import ( "github.com/pingcap/tidb/ddl" "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/errno" - "github.com/pingcap/tidb/executor" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/meta" "github.com/pingcap/tidb/meta/autoid" @@ -47,14 +46,11 @@ import ( "github.com/pingcap/tidb/store/mockstore" "github.com/pingcap/tidb/table" "github.com/pingcap/tidb/tablecodec" - ntestkit "github.com/pingcap/tidb/testkit" - "github.com/pingcap/tidb/util/admin" "github.com/pingcap/tidb/util/dbterror" "github.com/pingcap/tidb/util/mock" "github.com/pingcap/tidb/util/sqlexec" "github.com/pingcap/tidb/util/testkit" "github.com/pingcap/tidb/util/testleak" - "github.com/pingcap/tidb/util/testutil" "github.com/stretchr/testify/require" "github.com/tikv/client-go/v2/testutils" ) @@ -75,7 +71,6 @@ const ( waitForCleanDataInterval = time.Millisecond * 100 ) -var _ = Suite(&testDBSuite5{&testDBSuite{}}) var _ = SerialSuites(&testSerialDBSuite{&testDBSuite{}}) const defaultBatchSize = 1024 @@ -139,36 +134,8 @@ func (s *testDBSuite) TearDownSuite(c *C) { tearDownSuite(s, c) } -type testDBSuite5 struct{ *testDBSuite } type testSerialDBSuite struct{ *testDBSuite } -func (s *testDBSuite5) TestAddIndexWithDupIndex(c *C) { - tk := testkit.NewTestKit(c, s.store) - tk.MustExec("use " + s.schemaName) - - err1 := dbterror.ErrDupKeyName.GenWithStack("index already exist %s", "idx") - err2 := dbterror.ErrDupKeyName.GenWithStack("index already exist %s; "+ - "a background job is trying to add the same index, "+ - "please check by `ADMIN SHOW DDL JOBS`", "idx") - - // When there is already an duplicate index, show error message. - tk.MustExec("create table test_add_index_with_dup (a int, key idx (a))") - _, err := tk.Exec("alter table test_add_index_with_dup add index idx (a)") - c.Check(errors.Cause(err1).(*terror.Error).Equal(err), Equals, true) - c.Assert(errors.Cause(err1).Error() == err.Error(), IsTrue) - - // When there is another session adding duplicate index with state other than - // StatePublic, show explicit error message. - t := s.testGetTable(c, "test_add_index_with_dup") - indexInfo := t.Meta().FindIndexByName("idx") - indexInfo.State = model.StateNone - _, err = tk.Exec("alter table test_add_index_with_dup add index idx (a)") - c.Check(errors.Cause(err2).(*terror.Error).Equal(err), Equals, true) - c.Assert(errors.Cause(err2).Error() == err.Error(), IsTrue) - - tk.MustExec("drop table test_add_index_with_dup") -} - func testGetTableByName(c *C, ctx sessionctx.Context, db, table string) table.Table { dom := domain.GetDomain(ctx) // Make sure the table schema is the new schema. @@ -179,16 +146,6 @@ func testGetTableByName(c *C, ctx sessionctx.Context, db, table string) table.Ta return tbl } -func testGetSchemaByName(c *C, ctx sessionctx.Context, db string) *model.DBInfo { - dom := domain.GetDomain(ctx) - // Make sure the table schema is the new schema. - err := dom.Reload() - c.Assert(err, IsNil) - dbInfo, ok := dom.InfoSchema().SchemaByName(model.NewCIStr(db)) - c.Assert(ok, IsTrue) - return dbInfo -} - func (s *testDBSuite) testGetTable(c *C, name string) table.Table { ctx := s.s.(sessionctx.Context) return testGetTableByName(c, ctx, s.schemaName, name) @@ -363,234 +320,6 @@ func (s *testSerialDBSuite) TestDropTableOnTiKVDiskFull(c *C) { tk.MustExec("drop table test_disk_full_drop_table;") } -// TestCancelDropIndex tests cancel ddl job which type is drop primary key. -func (s *testDBSuite5) TestCancelDropPrimaryKey(c *C) { - idxName := "primary" - addIdxSQL := "alter table t add primary key idx_c2 (c2);" - dropIdxSQL := "alter table t drop primary key;" - testCancelDropIndex(c, s.store, s.dom.DDL(), idxName, addIdxSQL, dropIdxSQL, s.dom) -} - -// TestCancelDropIndex tests cancel ddl job which type is drop index. -func (s *testDBSuite5) TestCancelDropIndex(c *C) { - idxName := "idx_c2" - addIdxSQL := "alter table t add index idx_c2 (c2);" - dropIdxSQL := "alter table t drop index idx_c2;" - testCancelDropIndex(c, s.store, s.dom.DDL(), idxName, addIdxSQL, dropIdxSQL, s.dom) -} - -// testCancelDropIndex tests cancel ddl job which type is drop index. -func testCancelDropIndex(c *C, store kv.Storage, d ddl.DDL, idxName, addIdxSQL, dropIdxSQL string, dom *domain.Domain) { - tk := testkit.NewTestKit(c, store) - tk.MustExec("use test_db") - tk.MustExec("drop table if exists t") - tk.MustExec("create table t(c1 int, c2 int)") - defer tk.MustExec("drop table t;") - for i := 0; i < 5; i++ { - tk.MustExec("insert into t values (?, ?)", i, i) - } - testCases := []struct { - needAddIndex bool - jobState model.JobState - JobSchemaState model.SchemaState - cancelSucc bool - }{ - // model.JobStateNone means the jobs is canceled before the first run. - // if we cancel successfully, we need to set needAddIndex to false in the next test case. Otherwise, set needAddIndex to true. - {true, model.JobStateNone, model.StateNone, true}, - {false, model.JobStateRunning, model.StateWriteOnly, false}, - {true, model.JobStateRunning, model.StateDeleteOnly, false}, - {true, model.JobStateRunning, model.StateDeleteReorganization, false}, - } - var checkErr error - hook := &ddl.TestDDLCallback{Do: dom} - var jobID int64 - testCase := &testCases[0] - hook.OnJobRunBeforeExported = func(job *model.Job) { - if (job.Type == model.ActionDropIndex || job.Type == model.ActionDropPrimaryKey) && - job.State == testCase.jobState && job.SchemaState == testCase.JobSchemaState { - jobID = job.ID - jobIDs := []int64{job.ID} - hookCtx := mock.NewContext() - hookCtx.Store = store - err := hookCtx.NewTxn(context.TODO()) - if err != nil { - checkErr = errors.Trace(err) - return - } - 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 - } - if errs[0] != nil { - checkErr = errors.Trace(errs[0]) - return - } - checkErr = txn.Commit(context.Background()) - } - } - originalHook := d.GetHook() - d.SetHook(hook) - ctx := tk.Se.(sessionctx.Context) - for i := range testCases { - testCase = &testCases[i] - if testCase.needAddIndex { - tk.MustExec(addIdxSQL) - } - rs, err := tk.Exec(dropIdxSQL) - if rs != nil { - rs.Close() - } - t := testGetTableByName(c, ctx, "test_db", "t") - indexInfo := t.Meta().FindIndexByName(idxName) - if testCase.cancelSucc { - c.Assert(checkErr, IsNil) - c.Assert(err, NotNil) - c.Assert(err.Error(), Equals, "[ddl:8214]Cancelled DDL job") - c.Assert(indexInfo, NotNil) - c.Assert(indexInfo.State, Equals, model.StatePublic) - } else { - err1 := admin.ErrCannotCancelDDLJob.GenWithStackByArgs(jobID) - c.Assert(err, IsNil) - c.Assert(checkErr, NotNil) - c.Assert(checkErr.Error(), Equals, err1.Error()) - c.Assert(indexInfo, IsNil) - } - } - d.SetHook(originalHook) - tk.MustExec(addIdxSQL) - tk.MustExec(dropIdxSQL) -} - -// TestCancelTruncateTable tests cancel ddl job which type is truncate table. -func (s *testDBSuite5) TestCancelTruncateTable(c *C) { - tk := testkit.NewTestKit(c, s.store) - tk.MustExec("use test_db") - tk.MustExec("create database if not exists test_truncate_table") - tk.MustExec("drop table if exists t") - tk.MustExec("create table t(c1 int, c2 int)") - defer tk.MustExec("drop table t;") - var checkErr error - hook := &ddl.TestDDLCallback{Do: s.dom} - hook.OnJobRunBeforeExported = func(job *model.Job) { - if job.Type == model.ActionTruncateTable && job.State == model.JobStateNone { - jobIDs := []int64{job.ID} - hookCtx := mock.NewContext() - hookCtx.Store = s.store - err := hookCtx.NewTxn(context.Background()) - if err != nil { - checkErr = errors.Trace(err) - return - } - 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 - } - if errs[0] != nil { - checkErr = errors.Trace(errs[0]) - return - } - checkErr = txn.Commit(context.Background()) - } - } - originalHook := s.dom.DDL().GetHook() - s.dom.DDL().SetHook(hook) - _, err := tk.Exec("truncate table t") - c.Assert(checkErr, IsNil) - c.Assert(err, NotNil) - c.Assert(err.Error(), Equals, "[ddl:8214]Cancelled DDL job") - s.dom.DDL().SetHook(originalHook) -} - -func (s *testDBSuite5) TestParallelDropSchemaAndDropTable(c *C) { - tk := testkit.NewTestKit(c, s.store) - tk.MustExec("create database if not exists test_drop_schema_table") - tk.MustExec("use test_drop_schema_table") - tk.MustExec("create table t(c1 int, c2 int)") - var checkErr error - hook := &ddl.TestDDLCallback{Do: s.dom} - dbInfo := testGetSchemaByName(c, tk.Se, "test_drop_schema_table") - done := false - var wg sync.WaitGroup - tk2 := testkit.NewTestKit(c, s.store) - tk2.MustExec("use test_drop_schema_table") - hook.OnJobUpdatedExported = func(job *model.Job) { - if job.Type == model.ActionDropSchema && job.State == model.JobStateRunning && - job.SchemaState == model.StateWriteOnly && job.SchemaID == dbInfo.ID && done == false { - wg.Add(1) - done = true - go func() { - _, checkErr = tk2.Exec("drop table t") - wg.Done() - }() - time.Sleep(5 * time.Millisecond) - } - } - originalHook := s.dom.DDL().GetHook() - s.dom.DDL().SetHook(hook) - tk.MustExec("drop database test_drop_schema_table") - s.dom.DDL().SetHook(originalHook) - wg.Wait() - c.Assert(done, IsTrue) - c.Assert(checkErr, NotNil) - // There are two possible assert result because: - // 1: If drop-database is finished before drop-table being put into the ddl job queue, it will return "unknown table" error directly in the previous check. - // 2: If drop-table has passed the previous check and been put into the ddl job queue, then drop-database finished, it will return schema change error. - assertRes := checkErr.Error() == "[domain:8028]Information schema is changed during the execution of the"+ - " statement(for example, table definition may be updated by other DDL ran in parallel). "+ - "If you see this error often, try increasing `tidb_max_delta_schema_count`. [try again later]" || - checkErr.Error() == "[schema:1051]Unknown table 'test_drop_schema_table.t'" - - c.Assert(assertRes, Equals, true) - - // Below behaviour is use to mock query `curl "http://$IP:10080/tiflash/replica"` - fn := func(jobs []*model.Job) (bool, error) { - return executor.GetDropOrTruncateTableInfoFromJobs(jobs, 0, s.dom, func(job *model.Job, info *model.TableInfo) (bool, error) { - return false, nil - }) - } - err := tk.Se.NewTxn(context.Background()) - c.Assert(err, IsNil) - txn, err := tk.Se.Txn(true) - c.Assert(err, IsNil) - err = admin.IterHistoryDDLJobs(txn, fn) - c.Assert(err, IsNil) -} - -func (s *testDBSuite5) TestAddMultiColumnsIndex(c *C) { - tk := testkit.NewTestKit(c, s.store) - tk.MustExec("use " + s.schemaName) - - tk.MustExec("drop database if exists tidb;") - tk.MustExec("create database tidb;") - tk.MustExec("use tidb;") - tk.MustExec("create table tidb.test (a int auto_increment primary key, b int);") - tk.MustExec("insert tidb.test values (1, 1);") - tk.MustExec("update tidb.test set b = b + 1 where a = 1;") - tk.MustExec("insert into tidb.test values (2, 2);") - // Test that the b value is nil. - tk.MustExec("insert into tidb.test (a) values (3);") - tk.MustExec("insert into tidb.test values (4, 4);") - // Test that the b value is nil again. - tk.MustExec("insert into tidb.test (a) values (5);") - tk.MustExec("insert tidb.test values (6, 6);") - tk.MustExec("alter table tidb.test add index idx1 (a, b);") - tk.MustExec("admin check table test") -} - func testGetIndexID(t *testing.T, ctx sessionctx.Context, dbName, tblName, idxName string) int64 { is := domain.GetDomain(ctx).InfoSchema() tt, err := is.TableByName(model.NewCIStr(dbName), model.NewCIStr(tblName)) @@ -605,152 +334,6 @@ func testGetIndexID(t *testing.T, ctx sessionctx.Context, dbName, tblName, idxNa return -1 } -type testDDLJobIDCallback struct { - ddl.Callback - jobID int64 -} - -func (t *testDDLJobIDCallback) OnJobUpdated(job *model.Job) { - if t.jobID == 0 { - t.jobID = job.ID - } - if t.Callback != nil { - t.Callback.OnJobUpdated(job) - } -} - -func wrapJobIDExtCallback(oldCallback ddl.Callback) *testDDLJobIDCallback { - return &testDDLJobIDCallback{ - Callback: oldCallback, - jobID: 0, - } -} - -func setupJobIDExtCallback(ctx sessionctx.Context) (jobExt *testDDLJobIDCallback, tearDown func()) { - dom := domain.GetDomain(ctx) - originHook := dom.DDL().GetHook() - jobIDExt := wrapJobIDExtCallback(originHook) - dom.DDL().SetHook(jobIDExt) - return jobIDExt, func() { - dom.DDL().SetHook(originHook) - } -} - -func checkDelRangeAdded(tk *ntestkit.TestKit, jobID int64, elemID int64) { - query := `select sum(cnt) from - (select count(1) cnt from mysql.gc_delete_range where job_id = ? and element_id = ? union - select count(1) cnt from mysql.gc_delete_range_done where job_id = ? and element_id = ?) as gdr;` - tk.MustQuery(query, jobID, elemID, jobID, elemID).Check(testkit.Rows("1")) -} - -func (s *testDBSuite5) TestAlterPrimaryKey(c *C) { - tk := testkit.NewTestKitWithInit(c, s.store) - tk.MustExec("create table test_add_pk(a int, b int unsigned , c varchar(255) default 'abc', d int as (a+b), e int as (a+1) stored, index idx(b))") - defer tk.MustExec("drop table test_add_pk") - - // for generated columns - tk.MustGetErrCode("alter table test_add_pk add primary key(d);", errno.ErrUnsupportedOnGeneratedColumn) - // The primary key name is the same as the existing index name. - tk.MustExec("alter table test_add_pk add primary key idx(e)") - tk.MustExec("drop index `primary` on test_add_pk") - - // for describing table - tk.MustExec("create table test_add_pk1(a int, index idx(a))") - tk.MustQuery("desc test_add_pk1").Check(testutil.RowsWithSep(",", `a,int(11),YES,MUL,,`)) - tk.MustExec("alter table test_add_pk1 add primary key idx(a)") - tk.MustQuery("desc test_add_pk1").Check(testutil.RowsWithSep(",", `a,int(11),NO,PRI,,`)) - tk.MustExec("alter table test_add_pk1 drop primary key") - tk.MustQuery("desc test_add_pk1").Check(testutil.RowsWithSep(",", `a,int(11),NO,MUL,,`)) - tk.MustExec("create table test_add_pk2(a int, b int, index idx(a))") - tk.MustExec("alter table test_add_pk2 add primary key idx(a, b)") - tk.MustQuery("desc test_add_pk2").Check(testutil.RowsWithSep(",", ""+ - "a int(11) NO PRI ]\n"+ - "[b int(11) NO PRI ")) - tk.MustQuery("show create table test_add_pk2").Check(testutil.RowsWithSep("|", ""+ - "test_add_pk2 CREATE TABLE `test_add_pk2` (\n"+ - " `a` int(11) NOT NULL,\n"+ - " `b` int(11) NOT NULL,\n"+ - " KEY `idx` (`a`),\n"+ - " PRIMARY KEY (`a`,`b`) /*T![clustered_index] NONCLUSTERED */\n"+ - ") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin")) - tk.MustExec("alter table test_add_pk2 drop primary key") - tk.MustQuery("desc test_add_pk2").Check(testutil.RowsWithSep(",", ""+ - "a int(11) NO MUL ]\n"+ - "[b int(11) NO ")) - - // Check if the primary key exists before checking the table's pkIsHandle. - tk.MustGetErrCode("alter table test_add_pk drop primary key", errno.ErrCantDropFieldOrKey) - - // for the limit of name - validName := strings.Repeat("a", mysql.MaxIndexIdentifierLen) - invalidName := strings.Repeat("b", mysql.MaxIndexIdentifierLen+1) - tk.MustGetErrCode("alter table test_add_pk add primary key "+invalidName+"(a)", errno.ErrTooLongIdent) - // for valid name - tk.MustExec("alter table test_add_pk add primary key " + validName + "(a)") - // for multiple primary key - tk.MustGetErrCode("alter table test_add_pk add primary key (a)", errno.ErrMultiplePriKey) - tk.MustExec("alter table test_add_pk drop primary key") - // for not existing primary key - tk.MustGetErrCode("alter table test_add_pk drop primary key", errno.ErrCantDropFieldOrKey) - tk.MustGetErrCode("drop index `primary` on test_add_pk", errno.ErrCantDropFieldOrKey) - - // for too many key parts specified - tk.MustGetErrCode("alter table test_add_pk add primary key idx_test(f1,f2,f3,f4,f5,f6,f7,f8,f9,f10,f11,f12,f13,f14,f15,f16,f17);", - errno.ErrTooManyKeyParts) - - // for the limit of comment's length - validComment := "'" + strings.Repeat("a", ddl.MaxCommentLength) + "'" - invalidComment := "'" + strings.Repeat("b", ddl.MaxCommentLength+1) + "'" - tk.MustGetErrCode("alter table test_add_pk add primary key(a) comment "+invalidComment, errno.ErrTooLongIndexComment) - // for empty sql_mode - r := tk.MustQuery("select @@sql_mode") - sqlMode := r.Rows()[0][0].(string) - tk.MustExec("set @@sql_mode=''") - tk.MustExec("alter table test_add_pk add primary key(a) comment " + invalidComment) - c.Assert(tk.Se.GetSessionVars().StmtCtx.WarningCount(), Equals, uint16(1)) - tk.MustQuery("show warnings").Check(testutil.RowsWithSep("|", "Warning|1688|Comment for index 'PRIMARY' is too long (max = 1024)")) - tk.MustExec("set @@sql_mode= '" + sqlMode + "'") - tk.MustExec("alter table test_add_pk drop primary key") - // for valid comment - tk.MustExec("alter table test_add_pk add primary key(a, b, c) comment " + validComment) - ctx := tk.Se.(sessionctx.Context) - c.Assert(ctx.NewTxn(context.Background()), IsNil) - t := testGetTableByName(c, ctx, "test", "test_add_pk") - col1Flag := t.Cols()[0].Flag - col2Flag := t.Cols()[1].Flag - col3Flag := t.Cols()[2].Flag - c.Assert(mysql.HasNotNullFlag(col1Flag) && !mysql.HasPreventNullInsertFlag(col1Flag), IsTrue) - c.Assert(mysql.HasNotNullFlag(col2Flag) && !mysql.HasPreventNullInsertFlag(col2Flag) && mysql.HasUnsignedFlag(col2Flag), IsTrue) - c.Assert(mysql.HasNotNullFlag(col3Flag) && !mysql.HasPreventNullInsertFlag(col3Flag) && !mysql.HasNoDefaultValueFlag(col3Flag), IsTrue) - tk.MustExec("alter table test_add_pk drop primary key") - - // for null values in primary key - tk.MustExec("drop table test_add_pk") - tk.MustExec("create table test_add_pk(a int, b int unsigned , c varchar(255) default 'abc', index idx(b))") - tk.MustExec("insert into test_add_pk set a = 0, b = 0, c = 0") - tk.MustExec("insert into test_add_pk set a = 1") - tk.MustGetErrCode("alter table test_add_pk add primary key (b)", errno.ErrInvalidUseOfNull) - tk.MustExec("insert into test_add_pk set a = 2, b = 2") - tk.MustGetErrCode("alter table test_add_pk add primary key (a, b)", errno.ErrInvalidUseOfNull) - tk.MustExec("insert into test_add_pk set a = 3, c = 3") - tk.MustGetErrCode("alter table test_add_pk add primary key (c, b, a)", errno.ErrInvalidUseOfNull) -} - -func (s *testDBSuite5) TestCreateIndexType(c *C) { - tk := testkit.NewTestKit(c, s.store) - tk.MustExec("use " + s.schemaName) - sql := `CREATE TABLE test_index ( - price int(5) DEFAULT '0' NOT NULL, - area varchar(40) DEFAULT '' NOT NULL, - type varchar(40) DEFAULT '' NOT NULL, - transityes set('a','b'), - shopsyes enum('Y','N') DEFAULT 'Y' NOT NULL, - schoolsyes enum('Y','N') DEFAULT 'Y' NOT NULL, - petsyes enum('Y','N') DEFAULT 'Y' NOT NULL, - KEY price (price,area,type,transityes,shopsyes,schoolsyes,petsyes));` - tk.MustExec(sql) -} - func oldBackgroundExec(s kv.Storage, sql string, done chan error) { se, err := session.CreateSession4Test(s) if err != nil { @@ -1012,241 +595,6 @@ func (s *testSerialDBSuite) TestRebaseAutoID(c *C) { tk.MustGetErrCode("alter table tidb.test2 add column b int auto_increment key, auto_increment=10;", errno.ErrUnsupportedDDLOperation) } -func (s *testDBSuite5) TestCheckColumnDefaultValue(c *C) { - tk := testkit.NewTestKit(c, s.store) - tk.MustExec("use test;") - tk.MustExec("drop table if exists text_default_text;") - tk.MustGetErrCode("create table text_default_text(c1 text not null default '');", errno.ErrBlobCantHaveDefault) - tk.MustGetErrCode("create table text_default_text(c1 text not null default 'scds');", errno.ErrBlobCantHaveDefault) - - tk.MustExec("drop table if exists text_default_json;") - tk.MustGetErrCode("create table text_default_json(c1 json not null default '');", errno.ErrBlobCantHaveDefault) - tk.MustGetErrCode("create table text_default_json(c1 json not null default 'dfew555');", errno.ErrBlobCantHaveDefault) - - tk.MustExec("drop table if exists text_default_blob;") - tk.MustGetErrCode("create table text_default_blob(c1 blob not null default '');", errno.ErrBlobCantHaveDefault) - tk.MustGetErrCode("create table text_default_blob(c1 blob not null default 'scds54');", errno.ErrBlobCantHaveDefault) - - tk.MustExec("set sql_mode='';") - tk.MustExec("create table text_default_text(c1 text not null default '');") - tk.MustQuery(`show create table text_default_text`).Check(testutil.RowsWithSep("|", - "text_default_text CREATE TABLE `text_default_text` (\n"+ - " `c1` text NOT NULL\n"+ - ") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin", - )) - ctx := tk.Se.(sessionctx.Context) - is := domain.GetDomain(ctx).InfoSchema() - tblInfo, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("text_default_text")) - c.Assert(err, IsNil) - c.Assert(tblInfo.Meta().Columns[0].DefaultValue, Equals, "") - - tk.MustExec("create table text_default_blob(c1 blob not null default '');") - tk.MustQuery(`show create table text_default_blob`).Check(testutil.RowsWithSep("|", - "text_default_blob CREATE TABLE `text_default_blob` (\n"+ - " `c1` blob NOT NULL\n"+ - ") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin", - )) - is = domain.GetDomain(ctx).InfoSchema() - tblInfo, err = is.TableByName(model.NewCIStr("test"), model.NewCIStr("text_default_blob")) - c.Assert(err, IsNil) - c.Assert(tblInfo.Meta().Columns[0].DefaultValue, Equals, "") - - tk.MustExec("create table text_default_json(c1 json not null default '');") - tk.MustQuery(`show create table text_default_json`).Check(testutil.RowsWithSep("|", - "text_default_json CREATE TABLE `text_default_json` (\n"+ - " `c1` json NOT NULL DEFAULT 'null'\n"+ - ") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin", - )) - is = domain.GetDomain(ctx).InfoSchema() - tblInfo, err = is.TableByName(model.NewCIStr("test"), model.NewCIStr("text_default_json")) - c.Assert(err, IsNil) - c.Assert(tblInfo.Meta().Columns[0].DefaultValue, Equals, `null`) -} - -func (s *testDBSuite5) TestCheckConvertToCharacter(c *C) { - tk := testkit.NewTestKit(c, s.store) - tk.MustExec("use test") - tk.MustExec("drop table if exists t") - defer tk.MustExec("drop table t") - tk.MustExec("create table t(a varchar(10) charset binary);") - ctx := tk.Se.(sessionctx.Context) - is := domain.GetDomain(ctx).InfoSchema() - t, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) - c.Assert(err, IsNil) - tk.MustGetErrCode("alter table t modify column a varchar(10) charset utf8 collate utf8_bin", errno.ErrUnsupportedDDLOperation) - tk.MustGetErrCode("alter table t modify column a varchar(10) charset utf8mb4 collate utf8mb4_bin", errno.ErrUnsupportedDDLOperation) - tk.MustGetErrCode("alter table t modify column a varchar(10) charset latin1 collate latin1_bin", errno.ErrUnsupportedDDLOperation) - c.Assert(t.Cols()[0].Charset, Equals, "binary") -} - -func (s *testDBSuite5) TestModifyGeneratedColumn(c *C) { - tk := testkit.NewTestKit(c, s.store) - tk.MustExec("create database if not exists test;") - tk.MustExec("use test") - modIdxColErrMsg := "[ddl:3106]'modifying an indexed column' is not supported for generated columns." - modStoredColErrMsg := "[ddl:3106]'modifying a stored column' is not supported for generated columns." - - // Modify column with single-col-index. - tk.MustExec("drop table if exists t1;") - tk.MustExec("create table t1 (a int, b int as (a+1), index idx(b));") - tk.MustExec("insert into t1 set a=1;") - _, err := tk.Exec("alter table t1 modify column b int as (a+2);") - c.Assert(err, NotNil) - c.Assert(err.Error(), Equals, modIdxColErrMsg) - tk.MustExec("drop index idx on t1;") - tk.MustExec("alter table t1 modify b int as (a+2);") - tk.MustQuery("select * from t1").Check(testkit.Rows("1 3")) - - // Modify column with multi-col-index. - tk.MustExec("drop table t1;") - tk.MustExec("create table t1 (a int, b int as (a+1), index idx(a, b));") - tk.MustExec("insert into t1 set a=1;") - _, err = tk.Exec("alter table t1 modify column b int as (a+2);") - c.Assert(err, NotNil) - c.Assert(err.Error(), Equals, modIdxColErrMsg) - tk.MustExec("drop index idx on t1;") - tk.MustExec("alter table t1 modify b int as (a+2);") - tk.MustQuery("select * from t1").Check(testkit.Rows("1 3")) - - // Modify column with stored status to a different expression. - tk.MustExec("drop table t1;") - tk.MustExec("create table t1 (a int, b int as (a+1) stored);") - tk.MustExec("insert into t1 set a=1;") - _, err = tk.Exec("alter table t1 modify column b int as (a+2) stored;") - c.Assert(err, NotNil) - c.Assert(err.Error(), Equals, modStoredColErrMsg) - - // Modify column with stored status to the same expression. - tk.MustExec("drop table t1;") - tk.MustExec("create table t1 (a int, b int as (a+1) stored);") - tk.MustExec("insert into t1 set a=1;") - tk.MustExec("alter table t1 modify column b bigint as (a+1) stored;") - tk.MustExec("alter table t1 modify column b bigint as (a + 1) stored;") - tk.MustQuery("select * from t1").Check(testkit.Rows("1 2")) - - // Modify column with index to the same expression. - tk.MustExec("drop table t1;") - tk.MustExec("create table t1 (a int, b int as (a+1), index idx(b));") - tk.MustExec("insert into t1 set a=1;") - tk.MustExec("alter table t1 modify column b bigint as (a+1);") - tk.MustExec("alter table t1 modify column b bigint as (a + 1);") - tk.MustQuery("select * from t1").Check(testkit.Rows("1 2")) - - // Modify column from non-generated to stored generated. - tk.MustExec("drop table t1;") - tk.MustExec("create table t1 (a int, b int);") - _, err = tk.Exec("alter table t1 modify column b bigint as (a+1) stored;") - c.Assert(err, NotNil) - c.Assert(err.Error(), Equals, modStoredColErrMsg) - - // Modify column from stored generated to non-generated. - tk.MustExec("drop table t1;") - tk.MustExec("create table t1 (a int, b int as (a+1) stored);") - tk.MustExec("insert into t1 set a=1;") - tk.MustExec("alter table t1 modify column b int;") - tk.MustQuery("select * from t1").Check(testkit.Rows("1 2")) -} - -func (s *testDBSuite5) TestDefaultSQLFunction(c *C) { - tk := testkit.NewTestKit(c, s.store) - tk.MustExec("create database if not exists test;") - tk.MustExec("use test;") - tk.MustExec("drop table if exists t1, t2, t3, t4;") - - // For issue #13189 - // Use `DEFAULT()` in `INSERT` / `INSERT ON DUPLICATE KEY UPDATE` statement - tk.MustExec("create table t1 (a int primary key, b int default 20, c int default 30, d int default 40);") - tk.MustExec("SET @@time_zone = '+00:00'") - defer tk.MustExec("SET @@time_zone = DEFAULT") - tk.MustQuery("SELECT @@time_zone").Check(testkit.Rows("+00:00")) - tk.MustExec("create table t2 (a int primary key, b timestamp DEFAULT CURRENT_TIMESTAMP, c timestamp DEFAULT '2000-01-01 00:00:00')") - tk.MustExec("insert into t1 set a = 1, b = default(c);") - tk.MustQuery("select * from t1;").Check(testkit.Rows("1 30 30 40")) - tk.MustExec("insert into t1 set a = 2, b = default(c), c = default(d), d = default(b);") - tk.MustQuery("select * from t1;").Check(testkit.Rows("1 30 30 40", "2 30 40 20")) - tk.MustExec("insert into t1 values (2, 3, 4, 5) on duplicate key update b = default(d), c = default(b);") - tk.MustQuery("select * from t1;").Check(testkit.Rows("1 30 30 40", "2 40 20 20")) - tk.MustExec("delete from t1") - tk.MustExec("insert into t1 set a = default(b) + default(c) - default(d)") - tk.MustQuery("select * from t1;").Check(testkit.Rows("10 20 30 40")) - tk.MustExec("set @@timestamp = 1321009871") - defer tk.MustExec("set @@timestamp = DEFAULT") - tk.MustQuery("SELECT NOW()").Check(testkit.Rows("2011-11-11 11:11:11")) - tk.MustExec("insert into t2 set a = 1, b = default(c)") - tk.MustExec("insert into t2 set a = 2, c = default(b)") - tk.MustGetErrCode("insert into t2 set a = 3, b = default(a)", errno.ErrNoDefaultForField) - tk.MustExec("insert into t2 set a = 4, b = default(b), c = default(c)") - tk.MustExec("insert into t2 set a = 5, b = default, c = default") - tk.MustExec("insert into t2 set a = 6") - tk.MustQuery("select * from t2").Sort().Check(testkit.Rows( - "1 2000-01-01 00:00:00 2000-01-01 00:00:00", - "2 2011-11-11 11:11:11 2011-11-11 11:11:11", - "4 2011-11-11 11:11:11 2000-01-01 00:00:00", - "5 2011-11-11 11:11:11 2000-01-01 00:00:00", - "6 2011-11-11 11:11:11 2000-01-01 00:00:00")) - // Use `DEFAULT()` in `UPDATE` statement - tk.MustExec("delete from t1;") - tk.MustExec("insert into t1 value (1, 2, 3, 4);") - tk.MustExec("update t1 set a = 1, c = default(b);") - tk.MustQuery("select * from t1;").Check(testkit.Rows("1 2 20 4")) - tk.MustExec("insert into t1 value (2, 2, 3, 4);") - tk.MustExec("update t1 set c = default(b), b = default(c) where a = 2;") - tk.MustQuery("select * from t1;").Check(testkit.Rows("1 2 20 4", "2 30 20 4")) - tk.MustExec("delete from t1") - tk.MustExec("insert into t1 set a = 10") - tk.MustExec("update t1 set a = 10, b = default(c) + default(d)") - tk.MustQuery("select * from t1;").Check(testkit.Rows("10 70 30 40")) - tk.MustExec("set @@timestamp = 1671747742") - tk.MustExec("update t2 set b = default(c) WHERE a = 6") - tk.MustExec("update t2 set c = default(b) WHERE a = 5") - tk.MustGetErrCode("update t2 set b = default(a) WHERE a = 4", errno.ErrNoDefaultForField) - tk.MustExec("update t2 set b = default(b), c = default(c) WHERE a = 4") - // Non existing row! - tk.MustExec("update t2 set b = default(b), c = default(c) WHERE a = 3") - tk.MustExec("update t2 set b = default, c = default WHERE a = 2") - tk.MustExec("update t2 set b = default(b) WHERE a = 1") - tk.MustQuery("select * from t2;").Sort().Check(testkit.Rows( - "1 2022-12-22 22:22:22 2000-01-01 00:00:00", - "2 2022-12-22 22:22:22 2000-01-01 00:00:00", - "4 2022-12-22 22:22:22 2000-01-01 00:00:00", - "5 2011-11-11 11:11:11 2022-12-22 22:22:22", - "6 2000-01-01 00:00:00 2000-01-01 00:00:00")) - // Use `DEFAULT()` in `REPLACE` statement - tk.MustExec("delete from t1;") - tk.MustExec("insert into t1 value (1, 2, 3, 4);") - tk.MustExec("replace into t1 set a = 1, c = default(b);") - tk.MustQuery("select * from t1;").Check(testkit.Rows("1 20 20 40")) - tk.MustExec("insert into t1 value (2, 2, 3, 4);") - tk.MustExec("replace into t1 set a = 2, d = default(b), c = default(d);") - tk.MustQuery("select * from t1;").Check(testkit.Rows("1 20 20 40", "2 20 40 20")) - tk.MustExec("delete from t1") - tk.MustExec("insert into t1 set a = 10, c = 3") - tk.MustExec("replace into t1 set a = 10, b = default(c) + default(d)") - tk.MustQuery("select * from t1;").Check(testkit.Rows("10 70 30 40")) - tk.MustExec("replace into t1 set a = 20, d = default(c) + default(b)") - tk.MustQuery("select * from t1;").Check(testkit.Rows("10 70 30 40", "20 20 30 50")) - - // Use `DEFAULT()` in expression of generate columns, issue #12471 - tk.MustExec("DROP TABLE t2") - tk.MustExec("create table t2(a int default 9, b int as (1 + default(a)));") - tk.MustExec("insert into t2 values(1, default);") - tk.MustExec("insert into t2 values(2, default(b))") - tk.MustQuery("select * from t2").Sort().Check(testkit.Rows("1 10", "2 10")) - - // Use `DEFAULT()` with subquery, issue #13390 - tk.MustExec("create table t3(f1 int default 11);") - tk.MustExec("insert into t3 value ();") - tk.MustQuery("select default(f1) from (select * from t3) t1;").Check(testkit.Rows("11")) - tk.MustQuery("select default(f1) from (select * from (select * from t3) t1 ) t1;").Check(testkit.Rows("11")) - - tk.MustExec("create table t4(a int default 4);") - tk.MustExec("insert into t4 value (2);") - tk.MustQuery("select default(c) from (select b as c from (select a as b from t4) t3) t2;").Check(testkit.Rows("4")) - tk.MustGetErrCode("select default(a) from (select a from (select 1 as a) t4) t4;", errno.ErrNoDefaultForField) - - tk.MustExec("drop table t1, t2, t3, t4;") -} - func (s *testSerialDBSuite) TestProcessColumnFlags(c *C) { // check `processColumnFlags()` tk := testkit.NewTestKit(c, s.store) @@ -1564,17 +912,6 @@ func (s *testSerialDBSuite) TestSkipSchemaChecker(c *C) { c.Assert(terror.ErrorEqual(domain.ErrInfoSchemaChanged, err), IsTrue) } -func (s *testDBSuite5) TestAlterCheck(c *C) { - tk := testkit.NewTestKit(c, s.store) - tk.MustExec("use " + s.schemaName) - tk.MustExec("drop table if exists alter_check") - tk.MustExec("create table alter_check (pk int primary key)") - defer tk.MustExec("drop table if exists alter_check") - tk.MustExec("alter table alter_check alter check crcn ENFORCED") - c.Assert(tk.Se.GetSessionVars().StmtCtx.WarningCount(), Equals, uint16(1)) - tk.MustQuery("show warnings").Check(testutil.RowsWithSep("|", "Warning|8231|ALTER CHECK is not supported")) -} - func (s *testSerialDBSuite) TestDDLJobErrorCount(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") diff --git a/ddl/db_table_test.go b/ddl/db_table_test.go index 0b9aa35b0692c..1a2d51865580f 100644 --- a/ddl/db_table_test.go +++ b/ddl/db_table_test.go @@ -41,7 +41,6 @@ import ( "github.com/pingcap/tidb/util" "github.com/pingcap/tidb/util/admin" "github.com/pingcap/tidb/util/mock" - "github.com/pingcap/tidb/util/testutil" "github.com/stretchr/testify/require" ) @@ -526,12 +525,12 @@ func TestAlterTableWithValidation(t *testing.T) { // Test for alter table with validation. tk.MustExec("alter table t1 with validation") require.Equal(t, uint16(1), tk.Session().GetSessionVars().StmtCtx.WarningCount()) - tk.MustQuery("show warnings").Check(testutil.RowsWithSep("|", "Warning|8200|ALTER TABLE WITH VALIDATION is currently unsupported")) + tk.MustQuery("show warnings").Check(testkit.RowsWithSep("|", "Warning|8200|ALTER TABLE WITH VALIDATION is currently unsupported")) // Test for alter table without validation. tk.MustExec("alter table t1 without validation") require.Equal(t, uint16(1), tk.Session().GetSessionVars().StmtCtx.WarningCount()) - tk.MustQuery("show warnings").Check(testutil.RowsWithSep("|", "Warning|8200|ALTER TABLE WITHOUT VALIDATION is currently unsupported")) + tk.MustQuery("show warnings").Check(testkit.RowsWithSep("|", "Warning|8200|ALTER TABLE WITHOUT VALIDATION is currently unsupported")) } func TestBatchCreateTable(t *testing.T) { diff --git a/ddl/db_test.go b/ddl/db_test.go index 7ce3a4ea396af..945af055374b6 100644 --- a/ddl/db_test.go +++ b/ddl/db_test.go @@ -15,21 +15,29 @@ package ddl_test import ( + "context" "fmt" "strings" + "sync" "testing" "time" + "github.com/pingcap/errors" "github.com/pingcap/tidb/ddl" ddlutil "github.com/pingcap/tidb/ddl/util" "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/errno" + "github.com/pingcap/tidb/executor" + "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/parser/mysql" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/testkit" "github.com/pingcap/tidb/testkit/external" "github.com/pingcap/tidb/types" + "github.com/pingcap/tidb/util/admin" + "github.com/pingcap/tidb/util/mock" + "github.com/pingcap/tidb/util/testutil" "github.com/stretchr/testify/require" ) @@ -728,3 +736,431 @@ func TestCreateTableWithDecimalWithDoubleZero(t *testing.T) { tk.MustExec("alter table tt change column d d decimal(0, 0)") checkType("test", "tt", "d") } + +func TestAlterCheck(t *testing.T) { + store, clean := testkit.CreateMockStoreWithSchemaLease(t, dbTestLease) + defer clean() + + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("create table alter_check (pk int primary key)") + tk.MustExec("alter table alter_check alter check crcn ENFORCED") + require.Equal(t, uint16(1), tk.Session().GetSessionVars().StmtCtx.WarningCount()) + tk.MustQuery("show warnings").Check(testkit.RowsWithSep("|", "Warning|8231|ALTER CHECK is not supported")) +} + +func TestDefaultSQLFunction(t *testing.T) { + store, clean := testkit.CreateMockStoreWithSchemaLease(t, dbTestLease) + defer clean() + + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + + // For issue #13189 + // Use `DEFAULT()` in `INSERT` / `INSERT ON DUPLICATE KEY UPDATE` statement + tk.MustExec("create table t1 (a int primary key, b int default 20, c int default 30, d int default 40);") + tk.MustExec("SET @@time_zone = '+00:00'") + defer tk.MustExec("SET @@time_zone = DEFAULT") + tk.MustQuery("SELECT @@time_zone").Check(testkit.Rows("+00:00")) + tk.MustExec("create table t2 (a int primary key, b timestamp DEFAULT CURRENT_TIMESTAMP, c timestamp DEFAULT '2000-01-01 00:00:00')") + tk.MustExec("insert into t1 set a = 1, b = default(c);") + tk.MustQuery("select * from t1;").Check(testkit.Rows("1 30 30 40")) + tk.MustExec("insert into t1 set a = 2, b = default(c), c = default(d), d = default(b);") + tk.MustQuery("select * from t1;").Check(testkit.Rows("1 30 30 40", "2 30 40 20")) + tk.MustExec("insert into t1 values (2, 3, 4, 5) on duplicate key update b = default(d), c = default(b);") + tk.MustQuery("select * from t1;").Check(testkit.Rows("1 30 30 40", "2 40 20 20")) + tk.MustExec("delete from t1") + tk.MustExec("insert into t1 set a = default(b) + default(c) - default(d)") + tk.MustQuery("select * from t1;").Check(testkit.Rows("10 20 30 40")) + tk.MustExec("set @@timestamp = 1321009871") + defer tk.MustExec("set @@timestamp = DEFAULT") + tk.MustQuery("SELECT NOW()").Check(testkit.Rows("2011-11-11 11:11:11")) + tk.MustExec("insert into t2 set a = 1, b = default(c)") + tk.MustExec("insert into t2 set a = 2, c = default(b)") + tk.MustGetErrCode("insert into t2 set a = 3, b = default(a)", errno.ErrNoDefaultForField) + tk.MustExec("insert into t2 set a = 4, b = default(b), c = default(c)") + tk.MustExec("insert into t2 set a = 5, b = default, c = default") + tk.MustExec("insert into t2 set a = 6") + tk.MustQuery("select * from t2").Sort().Check(testkit.Rows( + "1 2000-01-01 00:00:00 2000-01-01 00:00:00", + "2 2011-11-11 11:11:11 2011-11-11 11:11:11", + "4 2011-11-11 11:11:11 2000-01-01 00:00:00", + "5 2011-11-11 11:11:11 2000-01-01 00:00:00", + "6 2011-11-11 11:11:11 2000-01-01 00:00:00")) + // Use `DEFAULT()` in `UPDATE` statement + tk.MustExec("delete from t1;") + tk.MustExec("insert into t1 value (1, 2, 3, 4);") + tk.MustExec("update t1 set a = 1, c = default(b);") + tk.MustQuery("select * from t1;").Check(testkit.Rows("1 2 20 4")) + tk.MustExec("insert into t1 value (2, 2, 3, 4);") + tk.MustExec("update t1 set c = default(b), b = default(c) where a = 2;") + tk.MustQuery("select * from t1;").Check(testkit.Rows("1 2 20 4", "2 30 20 4")) + tk.MustExec("delete from t1") + tk.MustExec("insert into t1 set a = 10") + tk.MustExec("update t1 set a = 10, b = default(c) + default(d)") + tk.MustQuery("select * from t1;").Check(testkit.Rows("10 70 30 40")) + tk.MustExec("set @@timestamp = 1671747742") + tk.MustExec("update t2 set b = default(c) WHERE a = 6") + tk.MustExec("update t2 set c = default(b) WHERE a = 5") + tk.MustGetErrCode("update t2 set b = default(a) WHERE a = 4", errno.ErrNoDefaultForField) + tk.MustExec("update t2 set b = default(b), c = default(c) WHERE a = 4") + // Non existing row! + tk.MustExec("update t2 set b = default(b), c = default(c) WHERE a = 3") + tk.MustExec("update t2 set b = default, c = default WHERE a = 2") + tk.MustExec("update t2 set b = default(b) WHERE a = 1") + tk.MustQuery("select * from t2;").Sort().Check(testkit.Rows( + "1 2022-12-22 22:22:22 2000-01-01 00:00:00", + "2 2022-12-22 22:22:22 2000-01-01 00:00:00", + "4 2022-12-22 22:22:22 2000-01-01 00:00:00", + "5 2011-11-11 11:11:11 2022-12-22 22:22:22", + "6 2000-01-01 00:00:00 2000-01-01 00:00:00")) + // Use `DEFAULT()` in `REPLACE` statement + tk.MustExec("delete from t1;") + tk.MustExec("insert into t1 value (1, 2, 3, 4);") + tk.MustExec("replace into t1 set a = 1, c = default(b);") + tk.MustQuery("select * from t1;").Check(testkit.Rows("1 20 20 40")) + tk.MustExec("insert into t1 value (2, 2, 3, 4);") + tk.MustExec("replace into t1 set a = 2, d = default(b), c = default(d);") + tk.MustQuery("select * from t1;").Check(testkit.Rows("1 20 20 40", "2 20 40 20")) + tk.MustExec("delete from t1") + tk.MustExec("insert into t1 set a = 10, c = 3") + tk.MustExec("replace into t1 set a = 10, b = default(c) + default(d)") + tk.MustQuery("select * from t1;").Check(testkit.Rows("10 70 30 40")) + tk.MustExec("replace into t1 set a = 20, d = default(c) + default(b)") + tk.MustQuery("select * from t1;").Check(testkit.Rows("10 70 30 40", "20 20 30 50")) + + // Use `DEFAULT()` in expression of generate columns, issue #12471 + tk.MustExec("DROP TABLE t2") + tk.MustExec("create table t2(a int default 9, b int as (1 + default(a)));") + tk.MustExec("insert into t2 values(1, default);") + tk.MustExec("insert into t2 values(2, default(b))") + tk.MustQuery("select * from t2").Sort().Check(testkit.Rows("1 10", "2 10")) + + // Use `DEFAULT()` with subquery, issue #13390 + tk.MustExec("create table t3(f1 int default 11);") + tk.MustExec("insert into t3 value ();") + tk.MustQuery("select default(f1) from (select * from t3) t1;").Check(testkit.Rows("11")) + tk.MustQuery("select default(f1) from (select * from (select * from t3) t1 ) t1;").Check(testkit.Rows("11")) + + tk.MustExec("create table t4(a int default 4);") + tk.MustExec("insert into t4 value (2);") + tk.MustQuery("select default(c) from (select b as c from (select a as b from t4) t3) t2;").Check(testkit.Rows("4")) + tk.MustGetErrCode("select default(a) from (select a from (select 1 as a) t4) t4;", errno.ErrNoDefaultForField) + + tk.MustExec("drop table t1, t2, t3, t4;") +} + +func TestCreateIndexType(t *testing.T) { + store, clean := testkit.CreateMockStoreWithSchemaLease(t, dbTestLease) + defer clean() + + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec(`CREATE TABLE test_index ( + price int(5) DEFAULT '0' NOT NULL, + area varchar(40) DEFAULT '' NOT NULL, + type varchar(40) DEFAULT '' NOT NULL, + transityes set('a','b'), + shopsyes enum('Y','N') DEFAULT 'Y' NOT NULL, + schoolsyes enum('Y','N') DEFAULT 'Y' NOT NULL, + petsyes enum('Y','N') DEFAULT 'Y' NOT NULL, + KEY price (price,area,type,transityes,shopsyes,schoolsyes,petsyes));`) +} + +func TestAlterPrimaryKey(t *testing.T) { + store, clean := testkit.CreateMockStoreWithSchemaLease(t, dbTestLease) + defer clean() + + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("create table test_add_pk(a int, b int unsigned , c varchar(255) default 'abc', d int as (a+b), e int as (a+1) stored, index idx(b))") + + // for generated columns + tk.MustGetErrCode("alter table test_add_pk add primary key(d);", errno.ErrUnsupportedOnGeneratedColumn) + // The primary key name is the same as the existing index name. + tk.MustExec("alter table test_add_pk add primary key idx(e)") + tk.MustExec("drop index `primary` on test_add_pk") + + // for describing table + tk.MustExec("create table test_add_pk1(a int, index idx(a))") + tk.MustQuery("desc test_add_pk1").Check(testutil.RowsWithSep(",", `a,int(11),YES,MUL,,`)) + tk.MustExec("alter table test_add_pk1 add primary key idx(a)") + tk.MustQuery("desc test_add_pk1").Check(testutil.RowsWithSep(",", `a,int(11),NO,PRI,,`)) + tk.MustExec("alter table test_add_pk1 drop primary key") + tk.MustQuery("desc test_add_pk1").Check(testutil.RowsWithSep(",", `a,int(11),NO,MUL,,`)) + tk.MustExec("create table test_add_pk2(a int, b int, index idx(a))") + tk.MustExec("alter table test_add_pk2 add primary key idx(a, b)") + tk.MustQuery("desc test_add_pk2").Check(testutil.RowsWithSep(",", ""+ + "a int(11) NO PRI ]\n"+ + "[b int(11) NO PRI ")) + tk.MustQuery("show create table test_add_pk2").Check(testutil.RowsWithSep("|", ""+ + "test_add_pk2 CREATE TABLE `test_add_pk2` (\n"+ + " `a` int(11) NOT NULL,\n"+ + " `b` int(11) NOT NULL,\n"+ + " KEY `idx` (`a`),\n"+ + " PRIMARY KEY (`a`,`b`) /*T![clustered_index] NONCLUSTERED */\n"+ + ") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin")) + tk.MustExec("alter table test_add_pk2 drop primary key") + tk.MustQuery("desc test_add_pk2").Check(testutil.RowsWithSep(",", ""+ + "a int(11) NO MUL ]\n"+ + "[b int(11) NO ")) + + // Check if the primary key exists before checking the table's pkIsHandle. + tk.MustGetErrCode("alter table test_add_pk drop primary key", errno.ErrCantDropFieldOrKey) + + // for the limit of name + validName := strings.Repeat("a", mysql.MaxIndexIdentifierLen) + invalidName := strings.Repeat("b", mysql.MaxIndexIdentifierLen+1) + tk.MustGetErrCode("alter table test_add_pk add primary key "+invalidName+"(a)", errno.ErrTooLongIdent) + // for valid name + tk.MustExec("alter table test_add_pk add primary key " + validName + "(a)") + // for multiple primary key + tk.MustGetErrCode("alter table test_add_pk add primary key (a)", errno.ErrMultiplePriKey) + tk.MustExec("alter table test_add_pk drop primary key") + // for not existing primary key + tk.MustGetErrCode("alter table test_add_pk drop primary key", errno.ErrCantDropFieldOrKey) + tk.MustGetErrCode("drop index `primary` on test_add_pk", errno.ErrCantDropFieldOrKey) + + // for too many key parts specified + tk.MustGetErrCode("alter table test_add_pk add primary key idx_test(f1,f2,f3,f4,f5,f6,f7,f8,f9,f10,f11,f12,f13,f14,f15,f16,f17);", + errno.ErrTooManyKeyParts) + + // for the limit of comment's length + validComment := "'" + strings.Repeat("a", ddl.MaxCommentLength) + "'" + invalidComment := "'" + strings.Repeat("b", ddl.MaxCommentLength+1) + "'" + tk.MustGetErrCode("alter table test_add_pk add primary key(a) comment "+invalidComment, errno.ErrTooLongIndexComment) + // for empty sql_mode + r := tk.MustQuery("select @@sql_mode") + sqlMode := r.Rows()[0][0].(string) + tk.MustExec("set @@sql_mode=''") + tk.MustExec("alter table test_add_pk add primary key(a) comment " + invalidComment) + require.Equal(t, uint16(1), tk.Session().GetSessionVars().StmtCtx.WarningCount()) + tk.MustQuery("show warnings").Check(testutil.RowsWithSep("|", "Warning|1688|Comment for index 'PRIMARY' is too long (max = 1024)")) + tk.MustExec("set @@sql_mode= '" + sqlMode + "'") + tk.MustExec("alter table test_add_pk drop primary key") + // for valid comment + tk.MustExec("alter table test_add_pk add primary key(a, b, c) comment " + validComment) + require.NoError(t, tk.Session().NewTxn(context.Background())) + tbl := external.GetTableByName(t, tk, "test", "test_add_pk") + col1Flag := tbl.Cols()[0].Flag + col2Flag := tbl.Cols()[1].Flag + col3Flag := tbl.Cols()[2].Flag + require.True(t, mysql.HasNotNullFlag(col1Flag) && !mysql.HasPreventNullInsertFlag(col1Flag)) + require.True(t, mysql.HasNotNullFlag(col2Flag) && !mysql.HasPreventNullInsertFlag(col2Flag) && mysql.HasUnsignedFlag(col2Flag)) + require.True(t, mysql.HasNotNullFlag(col3Flag) && !mysql.HasPreventNullInsertFlag(col3Flag) && !mysql.HasNoDefaultValueFlag(col3Flag)) + tk.MustExec("alter table test_add_pk drop primary key") + + // for null values in primary key + tk.MustExec("drop table test_add_pk") + tk.MustExec("create table test_add_pk(a int, b int unsigned , c varchar(255) default 'abc', index idx(b))") + tk.MustExec("insert into test_add_pk set a = 0, b = 0, c = 0") + tk.MustExec("insert into test_add_pk set a = 1") + tk.MustGetErrCode("alter table test_add_pk add primary key (b)", errno.ErrInvalidUseOfNull) + tk.MustExec("insert into test_add_pk set a = 2, b = 2") + tk.MustGetErrCode("alter table test_add_pk add primary key (a, b)", errno.ErrInvalidUseOfNull) + tk.MustExec("insert into test_add_pk set a = 3, c = 3") + tk.MustGetErrCode("alter table test_add_pk add primary key (c, b, a)", errno.ErrInvalidUseOfNull) +} + +func TestParallelDropSchemaAndDropTable(t *testing.T) { + store, dom, clean := testkit.CreateMockStoreAndDomainWithSchemaLease(t, dbTestLease) + defer clean() + + tk1 := testkit.NewTestKit(t, store) + tk1.MustExec("create database if not exists test_drop_schema_table") + tk1.MustExec("use test_drop_schema_table") + tk1.MustExec("create table t(c1 int, c2 int)") + var checkErr error + hook := &ddl.TestDDLCallback{Do: dom} + + dbInfo := external.GetSchemaByName(t, tk1, "test_drop_schema_table") + done := false + var wg sync.WaitGroup + tk2 := testkit.NewTestKit(t, store) + tk2.MustExec("use test_drop_schema_table") + hook.OnJobUpdatedExported = func(job *model.Job) { + if job.Type == model.ActionDropSchema && job.State == model.JobStateRunning && + job.SchemaState == model.StateWriteOnly && job.SchemaID == dbInfo.ID && done == false { + wg.Add(1) + done = true + go func() { + _, checkErr = tk2.Exec("drop table t") + wg.Done() + }() + time.Sleep(5 * time.Millisecond) + } + } + originalHook := dom.DDL().GetHook() + dom.DDL().SetHook(hook) + tk1.MustExec("drop database test_drop_schema_table") + dom.DDL().SetHook(originalHook) + wg.Wait() + require.True(t, done) + require.Error(t, checkErr) + // There are two possible assert result because: + // 1: If drop-database is finished before drop-table being put into the ddl job queue, it will return "unknown table" error directly in the previous check. + // 2: If drop-table has passed the previous check and been put into the ddl job queue, then drop-database finished, it will return schema change error. + assertRes := checkErr.Error() == "[domain:8028]Information schema is changed during the execution of the"+ + " statement(for example, table definition may be updated by other DDL ran in parallel). "+ + "If you see this error often, try increasing `tidb_max_delta_schema_count`. [try again later]" || + checkErr.Error() == "[schema:1051]Unknown table 'test_drop_schema_table.t'" + require.True(t, assertRes) + + // Below behaviour is use to mock query `curl "http://$IP:10080/tiflash/replica"` + fn := func(jobs []*model.Job) (bool, error) { + return executor.GetDropOrTruncateTableInfoFromJobs(jobs, 0, dom, func(job *model.Job, info *model.TableInfo) (bool, error) { + return false, nil + }) + } + require.NoError(t, tk1.Session().NewTxn(context.Background())) + txn, err := tk1.Session().Txn(true) + require.NoError(t, err) + require.NoError(t, admin.IterHistoryDDLJobs(txn, fn)) +} + +// TestCancelDropIndex tests cancel ddl job which type is drop primary key. +func TestCancelDropPrimaryKey(t *testing.T) { + store, dom, clean := testkit.CreateMockStoreAndDomainWithSchemaLease(t, dbTestLease) + defer clean() + idxName := "primary" + addIdxSQL := "alter table t add primary key idx_c2 (c2);" + dropIdxSQL := "alter table t drop primary key;" + testCancelDropIndex(t, store, dom.DDL(), idxName, addIdxSQL, dropIdxSQL, dom) +} + +// TestCancelDropIndex tests cancel ddl job which type is drop index. +func TestCancelDropIndex(t *testing.T) { + store, dom, clean := testkit.CreateMockStoreAndDomainWithSchemaLease(t, dbTestLease) + defer clean() + idxName := "idx_c2" + addIdxSQL := "alter table t add index idx_c2 (c2);" + dropIdxSQL := "alter table t drop index idx_c2;" + testCancelDropIndex(t, store, dom.DDL(), idxName, addIdxSQL, dropIdxSQL, dom) +} + +// testCancelDropIndex tests cancel ddl job which type is drop index. +func testCancelDropIndex(t *testing.T, store kv.Storage, d ddl.DDL, idxName, addIdxSQL, dropIdxSQL string, dom *domain.Domain) { + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("create table t(c1 int, c2 int)") + defer tk.MustExec("drop table t;") + for i := 0; i < 5; i++ { + tk.MustExec("insert into t values (?, ?)", i, i) + } + tests := []struct { + needAddIndex bool + jobState model.JobState + JobSchemaState model.SchemaState + cancelSucc bool + }{ + // model.JobStateNone means the jobs is canceled before the first run. + // if we cancel successfully, we need to set needAddIndex to false in the next test case. Otherwise, set needAddIndex to true. + {true, model.JobStateNone, model.StateNone, true}, + {false, model.JobStateRunning, model.StateWriteOnly, false}, + {true, model.JobStateRunning, model.StateDeleteOnly, false}, + {true, model.JobStateRunning, model.StateDeleteReorganization, false}, + } + var checkErr error + hook := &ddl.TestDDLCallback{Do: dom} + var jobID int64 + test := &tests[0] + hook.OnJobRunBeforeExported = func(job *model.Job) { + if (job.Type == model.ActionDropIndex || job.Type == model.ActionDropPrimaryKey) && + job.State == test.jobState && job.SchemaState == test.JobSchemaState { + jobID = job.ID + jobIDs := []int64{job.ID} + hookCtx := mock.NewContext() + hookCtx.Store = store + err := hookCtx.NewTxn(context.TODO()) + if err != nil { + checkErr = errors.Trace(err) + return + } + 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 + } + if errs[0] != nil { + checkErr = errors.Trace(errs[0]) + return + } + checkErr = txn.Commit(context.Background()) + } + } + originalHook := d.GetHook() + d.SetHook(hook) + for i := range tests { + test = &tests[i] + if test.needAddIndex { + tk.MustExec(addIdxSQL) + } + err := tk.ExecToErr(dropIdxSQL) + tbl := external.GetTableByName(t, tk, "test", "t") + indexInfo := tbl.Meta().FindIndexByName(idxName) + if test.cancelSucc { + require.NoError(t, checkErr) + require.EqualError(t, err, "[ddl:8214]Cancelled DDL job") + require.NotNil(t, indexInfo) + require.Equal(t, model.StatePublic, indexInfo.State) + } else { + err1 := admin.ErrCannotCancelDDLJob.GenWithStackByArgs(jobID) + require.NoError(t, err) + require.EqualError(t, checkErr, err1.Error()) + require.Nil(t, indexInfo) + } + } + d.SetHook(originalHook) + tk.MustExec(addIdxSQL) + tk.MustExec(dropIdxSQL) +} + +// TestCancelTruncateTable tests cancel ddl job which type is truncate table. +func TestCancelTruncateTable(t *testing.T) { + store, dom, clean := testkit.CreateMockStoreAndDomainWithSchemaLease(t, dbTestLease) + defer clean() + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("create table t(c1 int, c2 int)") + defer tk.MustExec("drop table t;") + var checkErr error + hook := &ddl.TestDDLCallback{Do: dom} + hook.OnJobRunBeforeExported = func(job *model.Job) { + if job.Type == model.ActionTruncateTable && job.State == model.JobStateNone { + jobIDs := []int64{job.ID} + hookCtx := mock.NewContext() + hookCtx.Store = store + err := hookCtx.NewTxn(context.Background()) + if err != nil { + checkErr = errors.Trace(err) + return + } + 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 + } + if errs[0] != nil { + checkErr = errors.Trace(errs[0]) + return + } + checkErr = txn.Commit(context.Background()) + } + } + dom.DDL().SetHook(hook) + err := tk.ExecToErr("truncate table t") + require.NoError(t, checkErr) + require.EqualError(t, err, "[ddl:8214]Cancelled DDL job") +} diff --git a/ddl/index_modify_test.go b/ddl/index_modify_test.go index 45410b3a4c4bc..688be10e06b2f 100644 --- a/ddl/index_modify_test.go +++ b/ddl/index_modify_test.go @@ -44,6 +44,7 @@ import ( "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/admin" "github.com/pingcap/tidb/util/codec" + "github.com/pingcap/tidb/util/dbterror" "github.com/pingcap/tidb/util/mock" "github.com/stretchr/testify/require" ) @@ -1346,3 +1347,28 @@ func TestAnonymousIndex(t *testing.T) { rows = tk.MustQuery("show index from t where key_name='bbbbbbbbbbbbbbbbbbbbbbbbbbbbbb_2'").Rows() require.Len(t, rows, 1) } + +func TestAddIndexWithDupIndex(t *testing.T) { + store, clean := testkit.CreateMockStoreWithSchemaLease(t, indexModifyLease) + defer clean() + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + + err1 := dbterror.ErrDupKeyName.GenWithStack("index already exist %s", "idx") + err2 := dbterror.ErrDupKeyName.GenWithStack("index already exist %s; "+ + "a background job is trying to add the same index, "+ + "please check by `ADMIN SHOW DDL JOBS`", "idx") + + // When there is already an duplicate index, show error message. + tk.MustExec("create table test_add_index_with_dup (a int, key idx (a))") + err := tk.ExecToErr("alter table test_add_index_with_dup add index idx (a)") + require.ErrorIs(t, err, errors.Cause(err1)) + + // When there is another session adding duplicate index with state other than + // StatePublic, show explicit error message. + tbl := external.GetTableByName(t, tk, "test", "test_add_index_with_dup") + indexInfo := tbl.Meta().FindIndexByName("idx") + indexInfo.State = model.StateNone + err = tk.ExecToErr("alter table test_add_index_with_dup add index idx (a)") + require.ErrorIs(t, err, errors.Cause(err2)) +} diff --git a/ddl/main_test.go b/ddl/main_test.go index bced6300f2c0c..70549f1c8f9b2 100644 --- a/ddl/main_test.go +++ b/ddl/main_test.go @@ -26,6 +26,9 @@ import ( "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/domain/infosync" "github.com/pingcap/tidb/meta/autoid" + "github.com/pingcap/tidb/parser/model" + "github.com/pingcap/tidb/sessionctx" + "github.com/pingcap/tidb/testkit" "github.com/pingcap/tidb/util/testbridge" "github.com/tikv/client-go/v2/tikv" "go.uber.org/goleak" @@ -64,3 +67,41 @@ func TestMain(m *testing.M) { goleak.VerifyTestMain(m, opts...) } + +func wrapJobIDExtCallback(oldCallback ddl.Callback) *testDDLJobIDCallback { + return &testDDLJobIDCallback{ + Callback: oldCallback, + jobID: 0, + } +} + +func setupJobIDExtCallback(ctx sessionctx.Context) (jobExt *testDDLJobIDCallback, tearDown func()) { + dom := domain.GetDomain(ctx) + originHook := dom.DDL().GetHook() + jobIDExt := wrapJobIDExtCallback(originHook) + dom.DDL().SetHook(jobIDExt) + return jobIDExt, func() { + dom.DDL().SetHook(originHook) + } +} + +func checkDelRangeAdded(tk *testkit.TestKit, jobID int64, elemID int64) { + query := `select sum(cnt) from + (select count(1) cnt from mysql.gc_delete_range where job_id = ? and element_id = ? union + select count(1) cnt from mysql.gc_delete_range_done where job_id = ? and element_id = ?) as gdr;` + tk.MustQuery(query, jobID, elemID, jobID, elemID).Check(testkit.Rows("1")) +} + +type testDDLJobIDCallback struct { + ddl.Callback + jobID int64 +} + +func (t *testDDLJobIDCallback) OnJobUpdated(job *model.Job) { + if t.jobID == 0 { + t.jobID = job.ID + } + if t.Callback != nil { + t.Callback.OnJobUpdated(job) + } +} diff --git a/expression/integration_test.go b/expression/integration_test.go index 92331c43f9b4f..6370346d7658a 100644 --- a/expression/integration_test.go +++ b/expression/integration_test.go @@ -49,7 +49,6 @@ import ( "github.com/pingcap/tidb/util/kvcache" "github.com/pingcap/tidb/util/sem" "github.com/pingcap/tidb/util/sqlexec" - "github.com/pingcap/tidb/util/testutil" "github.com/pingcap/tidb/util/versioninfo" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" @@ -520,16 +519,16 @@ func TestStringBuiltin(t *testing.T) { // for space result = tk.MustQuery(`select space(0), space(2), space(-1), space(1.1), space(1.9)`) - result.Check(testutil.RowsWithSep(",", ", ,, , ")) + result.Check(testkit.RowsWithSep(",", ", ,, , ")) result = tk.MustQuery(`select space("abc"), space("2"), space("1.1"), space(''), space(null)`) - result.Check(testutil.RowsWithSep(",", ", , ,,")) + result.Check(testkit.RowsWithSep(",", ", , ,,")) // for replace tk.MustExec("drop table if exists t") tk.MustExec("create table t(a char(20), b int, c double, d datetime, e time)") tk.MustExec(`insert into t values('www.mysql.com', 1234, 12.34, "2017-01-01 12:01:01", "12:01:01")`) result = tk.MustQuery(`select replace(a, 'mysql', 'pingcap'), replace(b, 2, 55), replace(c, 34, 0), replace(d, '-', '/'), replace(e, '01', '22') from t`) - result.Check(testutil.RowsWithSep(",", "www.pingcap.com,15534,12.0,2017/01/01 12:01:01,12:22:22")) + result.Check(testkit.RowsWithSep(",", "www.pingcap.com,15534,12.0,2017/01/01 12:01:01,12:22:22")) result = tk.MustQuery(`select replace('aaa', 'a', ''), replace(null, 'a', 'b'), replace('a', null, 'b'), replace('a', 'b', null)`) result.Check(testkit.Rows(" ")) @@ -553,7 +552,7 @@ func TestStringBuiltin(t *testing.T) { result = tk.MustQuery(`select substr(a, 3), substr(b, 2, 3), substr(c, -3), substr(d, -8), substr(e, -3, 100) from t`) result.Check(testkit.Rows("kila 234 .45 12:01:01 :01")) result = tk.MustQuery(`select substr('Sakila', 100), substr('Sakila', -100), substr('Sakila', -5, 3), substr('Sakila', 2, -1)`) - result.Check(testutil.RowsWithSep(",", ",,aki,")) + result.Check(testkit.RowsWithSep(",", ",,aki,")) result = tk.MustQuery(`select substr('foobarbar' from 4), substr('Sakila' from -4 for 2)`) result.Check(testkit.Rows("barbar ki")) result = tk.MustQuery(`select substr(null, 2, 3), substr('foo', null, 3), substr('foo', 2, null)`) @@ -584,7 +583,7 @@ func TestStringBuiltin(t *testing.T) { result = tk.MustQuery(`select substring_index('www.pingcap.com', '.', 0), substring_index('www.pingcap.com', '.', 100), substring_index('www.pingcap.com', '.', -100)`) result.Check(testkit.Rows(" www.pingcap.com www.pingcap.com")) result = tk.MustQuery(`select substring_index('www.pingcap.com', 'd', 1), substring_index('www.pingcap.com', '', 1), substring_index('', '.', 1)`) - result.Check(testutil.RowsWithSep(",", "www.pingcap.com,,")) + result.Check(testkit.RowsWithSep(",", "www.pingcap.com,,")) result = tk.MustQuery(`select substring_index(null, '.', 1), substring_index('www.pingcap.com', null, 1), substring_index('www.pingcap.com', '.', null)`) result.Check(testkit.Rows(" ")) @@ -625,13 +624,13 @@ func TestStringBuiltin(t *testing.T) { // for ltrim and rtrim result = tk.MustQuery(`select ltrim(' bar '), ltrim('bar'), ltrim(''), ltrim(null)`) - result.Check(testutil.RowsWithSep(",", "bar ,bar,,")) + result.Check(testkit.RowsWithSep(",", "bar ,bar,,")) result = tk.MustQuery(`select rtrim(' bar '), rtrim('bar'), rtrim(''), rtrim(null)`) - result.Check(testutil.RowsWithSep(",", " bar,bar,,")) + result.Check(testkit.RowsWithSep(",", " bar,bar,,")) result = tk.MustQuery(`select ltrim("\t bar "), ltrim(" \tbar"), ltrim("\n bar"), ltrim("\r bar")`) - result.Check(testutil.RowsWithSep(",", "\t bar ,\tbar,\n bar,\r bar")) + result.Check(testkit.RowsWithSep(",", "\t bar ,\tbar,\n bar,\r bar")) result = tk.MustQuery(`select rtrim(" bar \t"), rtrim("bar\t "), rtrim("bar \n"), rtrim("bar \r")`) - result.Check(testutil.RowsWithSep(",", " bar \t,bar\t,bar \n,bar \r")) + result.Check(testkit.RowsWithSep(",", " bar \t,bar\t,bar \n,bar \r")) // for reverse tk.MustExec(`DROP TABLE IF EXISTS t;`) @@ -646,11 +645,11 @@ func TestStringBuiltin(t *testing.T) { result = tk.MustQuery(`select trim(' bar '), trim(leading 'x' from 'xxxbarxxx'), trim(trailing 'xyz' from 'barxxyz'), trim(both 'x' from 'xxxbarxxx')`) result.Check(testkit.Rows("bar barxxx barx bar")) result = tk.MustQuery(`select trim('\t bar\n '), trim(' \rbar \t')`) - result.Check(testutil.RowsWithSep(",", "\t bar\n,\rbar \t")) + result.Check(testkit.RowsWithSep(",", "\t bar\n,\rbar \t")) result = tk.MustQuery(`select trim(leading from ' bar'), trim('x' from 'xxxbarxxx'), trim('x' from 'bar'), trim('' from ' bar ')`) - result.Check(testutil.RowsWithSep(",", "bar,bar,bar, bar ")) + result.Check(testkit.RowsWithSep(",", "bar,bar,bar, bar ")) result = tk.MustQuery(`select trim(''), trim('x' from '')`) - result.Check(testutil.RowsWithSep(",", ",")) + result.Check(testkit.RowsWithSep(",", ",")) result = tk.MustQuery(`select trim(null from 'bar'), trim('x' from null), trim(null), trim(leading null from 'bar')`) result.Check(testkit.Rows(" ")) @@ -903,7 +902,7 @@ func TestEncryptionBuiltin(t *testing.T) { result.Check(testkit.Rows(`45ABDD5C4802EFA6771A94C43F805208 45ABDD5C4802EFA6771A94C43F805208 791F1AEB6A6B796E6352BF381895CA0E D0147E2EB856186F146D9F6DE33F9546 `)) result = tk.MustQuery("select HEX(AES_ENCRYPT(a, 'key', 'iv')), HEX(AES_ENCRYPT(b, 'key', 'iv')) from t") result.Check(testkit.Rows("B3800B3A3CB4ECE2051A3E80FE373EAC B3800B3A3CB4ECE2051A3E80FE373EAC")) - tk.MustQuery("show warnings").Check(testutil.RowsWithSep("|", "Warning|1618| option ignored", "Warning|1618| option ignored")) + tk.MustQuery("show warnings").Check(testkit.RowsWithSep("|", "Warning|1618| option ignored", "Warning|1618| option ignored")) tk.MustExec("SET block_encryption_mode='aes-128-cbc';") result = tk.MustQuery("select HEX(AES_ENCRYPT(a, 'key', '1234567890123456')), HEX(AES_ENCRYPT(b, 'key', '1234567890123456')), HEX(AES_ENCRYPT(c, 'key', '1234567890123456')), HEX(AES_ENCRYPT(d, 'key', '1234567890123456')), HEX(AES_ENCRYPT(e, 'key', '1234567890123456')), HEX(AES_ENCRYPT(f, 'key', '1234567890123456')), HEX(AES_ENCRYPT(g, 'key', '1234567890123456')), HEX(AES_ENCRYPT(h, 'key', '1234567890123456')), HEX(AES_ENCRYPT(i, 'key', '1234567890123456')) from t") result.Check(testkit.Rows("341672829F84CB6B0BE690FEC4C4DAE9 341672829F84CB6B0BE690FEC4C4DAE9 D43734E147A12BB96C6897C4BBABA283 16F2C972411948DCEF3659B726D2CCB04AD1379A1A367FA64242058A50211B67 41E71D0C58967C1F50EEC074523946D1 1117D292E2D39C3EAA3B435371BE56FC 8ACB7ECC0883B672D7BD1CFAA9FA5FAF5B731ADE978244CD581F114D591C2E7E D2B13C30937E3251AEDA73859BA32E4B 2CF4A6051FF248A67598A17AA2C17267")) @@ -1495,7 +1494,7 @@ func TestArithmeticBuiltin(t *testing.T) { result.Check(testkit.Rows("2 2 1")) result = tk.MustQuery("SELECT 1.175494351E-37 div 1.7976931348623157E+308, 1.7976931348623157E+308 div -1.7976931348623157E+307, 1 div 1e-82;") result.Check(testkit.Rows("0 -1 ")) - tk.MustQuery("show warnings").Check(testutil.RowsWithSep("|", + tk.MustQuery("show warnings").Check(testkit.RowsWithSep("|", "Warning|1292|Truncated incorrect DECIMAL value: '1.7976931348623157e+308'", "Warning|1292|Truncated incorrect DECIMAL value: '1.7976931348623157e+308'", "Warning|1292|Truncated incorrect DECIMAL value: '-1.7976931348623158e+307'", @@ -1560,7 +1559,7 @@ func TestArithmeticBuiltin(t *testing.T) { tk.MustExec("insert into t value(1.2)") result = tk.MustQuery("select * from t where a/0 > 1") result.Check(testkit.Rows()) - tk.MustQuery("show warnings").Check(testutil.RowsWithSep("|", "Warning|1365|Division by 0")) + tk.MustQuery("show warnings").Check(testkit.RowsWithSep("|", "Warning|1365|Division by 0")) tk.MustExec("USE test;") tk.MustExec("DROP TABLE IF EXISTS t;") @@ -1771,14 +1770,14 @@ func TestCompareBuiltin(t *testing.T) { tk.MustQuery("show warnings").Check(testkit.Rows()) result = tk.MustQuery(`select greatest(cast("2017-01-01" as datetime), "123", "234", cast("2018-01-01" as date)), greatest(cast("2017-01-01" as date), "123", null)`) result.Check(testkit.Rows("234 ")) - tk.MustQuery("show warnings").Check(testutil.RowsWithSep("|", "Warning|1292|Incorrect time value: '123'", "Warning|1292|Incorrect time value: '234'", "Warning|1292|Incorrect time value: '123'")) + tk.MustQuery("show warnings").Check(testkit.RowsWithSep("|", "Warning|1292|Incorrect time value: '123'", "Warning|1292|Incorrect time value: '234'", "Warning|1292|Incorrect time value: '123'")) // for least result = tk.MustQuery(`select least(1, 2, 3), least("a", "b", "c"), least(1.1, 1.2, 1.3), least("123a", 1, 2)`) result.Check(testkit.Rows("1 a 1.1 1")) tk.MustQuery("show warnings").Check(testkit.Rows()) result = tk.MustQuery(`select least(cast("2017-01-01" as datetime), "123", "234", cast("2018-01-01" as date)), least(cast("2017-01-01" as date), "123", null)`) result.Check(testkit.Rows("123 ")) - tk.MustQuery("show warnings").Check(testutil.RowsWithSep("|", "Warning|1292|Incorrect time value: '123'", "Warning|1292|Incorrect time value: '234'", "Warning|1292|Incorrect time value: '123'")) + tk.MustQuery("show warnings").Check(testkit.RowsWithSep("|", "Warning|1292|Incorrect time value: '123'", "Warning|1292|Incorrect time value: '234'", "Warning|1292|Incorrect time value: '123'")) tk.MustQuery(`select 1 < 17666000000000000000, 1 > 17666000000000000000, 1 = 17666000000000000000`).Check(testkit.Rows("1 0 0")) tk.MustExec("drop table if exists t") @@ -1960,7 +1959,7 @@ func TestAggregationBuiltinGroupConcat(t *testing.T) { tk.MustExec("set @@group_concat_max_len=7") result = tk.MustQuery("select group_concat(a) from t") result.Check(testkit.Rows("hello,h")) - tk.MustQuery("show warnings").Check(testutil.RowsWithSep("|", "Warning 1260 Some rows were cut by GROUPCONCAT(test.t.a)")) + tk.MustQuery("show warnings").Check(testkit.RowsWithSep("|", "Warning 1260 Some rows were cut by GROUPCONCAT(test.t.a)")) _, err := tk.Exec("insert into d select group_concat(a) from t") require.Equal(t, errors.ErrCode(mysql.ErrCutValueGroupConcat), errors.Cause(err).(*terror.Error).Code()) @@ -1968,7 +1967,7 @@ func TestAggregationBuiltinGroupConcat(t *testing.T) { _, err = tk.Exec("set sql_mode=''") require.NoError(t, err) tk.MustExec("insert into d select group_concat(a) from t") - tk.MustQuery("show warnings").Check(testutil.RowsWithSep("|", "Warning 1260 Some rows were cut by GROUPCONCAT(test.t.a)")) + tk.MustQuery("show warnings").Check(testkit.RowsWithSep("|", "Warning 1260 Some rows were cut by GROUPCONCAT(test.t.a)")) tk.MustQuery("select * from d").Check(testkit.Rows("hello,h")) } @@ -2328,7 +2327,7 @@ func TestTimeLiteral(t *testing.T) { _, err = tk.Exec("select ADDDATE('2008-01-34', -1);") require.NoError(t, err) - tk.MustQuery("Show warnings;").Check(testutil.RowsWithSep("|", + tk.MustQuery("Show warnings;").Check(testkit.RowsWithSep("|", "Warning|1292|Incorrect datetime value: '2008-01-34'")) } @@ -2618,12 +2617,12 @@ func TestIssues(t *testing.T) { tk.MustExec("insert into t values('1e649'),('-1e649');") r = tk.MustQuery(`SELECT * FROM t where c < 1;`) r.Check(testkit.Rows("-1e649")) - tk.MustQuery("show warnings").Check(testutil.RowsWithSep("|", + tk.MustQuery("show warnings").Check(testkit.RowsWithSep("|", "Warning|1292|Truncated incorrect DOUBLE value: '1e649'", "Warning|1292|Truncated incorrect DOUBLE value: '-1e649'")) r = tk.MustQuery(`SELECT * FROM t where c > 1;`) r.Check(testkit.Rows("1e649")) - tk.MustQuery("show warnings").Check(testutil.RowsWithSep("|", + tk.MustQuery("show warnings").Check(testkit.RowsWithSep("|", "Warning|1292|Truncated incorrect DOUBLE value: '1e649'", "Warning|1292|Truncated incorrect DOUBLE value: '-1e649'")) diff --git a/parser/ast/ddl.go b/parser/ast/ddl.go index 6ab941275c14b..54b2734b8fde5 100644 --- a/parser/ast/ddl.go +++ b/parser/ast/ddl.go @@ -175,6 +175,9 @@ type AlterDatabaseStmt struct { // Restore implements Node interface. func (n *AlterDatabaseStmt) Restore(ctx *format.RestoreCtx) error { + if ctx.Flags.HasSkipPlacementRuleForRestoreFlag() && n.isAllPlacementOptions() { + return nil + } // If all options placement options and RestoreTiDBSpecialComment flag is on, // we should restore the whole node in special comment. For example, the restore result should be: // /*T![placement] ALTER DATABASE `db1` PLACEMENT POLICY = `p1` */ @@ -1936,6 +1939,9 @@ type PlacementOption struct { } func (n *PlacementOption) Restore(ctx *format.RestoreCtx) error { + if ctx.Flags.HasSkipPlacementRuleForRestoreFlag() { + return nil + } fn := func() error { switch n.Tp { case PlacementOptionPrimaryRegion: @@ -2324,6 +2330,9 @@ func (n *TableOption) Restore(ctx *format.RestoreCtx) error { ctx.WritePlain("= ") ctx.WriteString(n.StrValue) case TableOptionPlacementPolicy: + if ctx.Flags.HasSkipPlacementRuleForRestoreFlag() { + return nil + } placementOpt := PlacementOption{ Tp: PlacementOptionPolicy, UintValue: n.UintValue, @@ -2685,8 +2694,25 @@ func (n *AlterOrderItem) Restore(ctx *format.RestoreCtx) error { return nil } +func (n *AlterTableSpec) IsAllPlacementRule() bool { + switch n.Tp { + case AlterTablePartitionAttributes, AlterTablePartitionOptions, AlterTableOption, AlterTableAttributes: + for _, o := range n.Options { + if o.Tp != TableOptionPlacementPolicy { + return false + } + } + return true + default: + return false + } +} + // Restore implements Node interface. func (n *AlterTableSpec) Restore(ctx *format.RestoreCtx) error { + if n.IsAllPlacementRule() && ctx.Flags.HasSkipPlacementRuleForRestoreFlag() { + return nil + } switch n.Tp { case AlterTableSetTiFlashReplica: ctx.WriteKeyWord("SET TIFLASH REPLICA ") @@ -3275,13 +3301,36 @@ type AlterTableStmt struct { Specs []*AlterTableSpec } +func (n *AlterTableStmt) HaveOnlyPlacementOptions() bool { + for _, n := range n.Specs { + if n.Tp == AlterTablePartitionOptions { + if !n.IsAllPlacementRule() { + return false + } + } else { + return false + } + + } + return true +} + // Restore implements Node interface. func (n *AlterTableStmt) Restore(ctx *format.RestoreCtx) error { + if ctx.Flags.HasSkipPlacementRuleForRestoreFlag() && n.HaveOnlyPlacementOptions() { + return nil + } ctx.WriteKeyWord("ALTER TABLE ") if err := n.Table.Restore(ctx); err != nil { return errors.Annotate(err, "An error occurred while restore AlterTableStmt.Table") } - for i, spec := range n.Specs { + var specs []*AlterTableSpec + for _, spec := range n.Specs { + if !(spec.IsAllPlacementRule() && ctx.Flags.HasSkipPlacementRuleForRestoreFlag()) { + specs = append(specs, spec) + } + } + for i, spec := range specs { if i == 0 || spec.Tp == AlterTablePartition || spec.Tp == AlterTableRemovePartitioning || spec.Tp == AlterTableImportTablespace || spec.Tp == AlterTableDiscardTablespace { ctx.WritePlain(" ") } else { @@ -3991,6 +4040,9 @@ type AlterPlacementPolicyStmt struct { } func (n *AlterPlacementPolicyStmt) Restore(ctx *format.RestoreCtx) error { + if ctx.Flags.HasSkipPlacementRuleForRestoreFlag() { + return nil + } if ctx.Flags.HasTiDBSpecialCommentFlag() { return restorePlacementStmtInSpecialComment(ctx, n) } diff --git a/parser/ast/ddl_test.go b/parser/ast/ddl_test.go index 108c4c609f329..beb3a27b97370 100644 --- a/parser/ast/ddl_test.go +++ b/parser/ast/ddl_test.go @@ -754,3 +754,76 @@ func TestDropPlacementPolicyRestore(t *testing.T) { runNodeRestoreTestWithFlags(t, testCases, "%s", extractNodeFunc, ca.flags) } } + +func TestRemovePlacementRestore(t *testing.T) { + f := format.DefaultRestoreFlags | format.SkipPlacementRuleForRestore + cases := []struct { + sourceSQL string + expectSQL string + }{ + { + "CREATE TABLE t1 (id BIGINT NOT NULL PRIMARY KEY auto_increment, b varchar(255)) PLACEMENT POLICY=placement1;", + "CREATE TABLE `t1` (`id` BIGINT NOT NULL PRIMARY KEY AUTO_INCREMENT,`b` VARCHAR(255)) ", + }, + { + "CREATE TABLE `t1` (\n `a` int(11) DEFAULT NULL\n) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin /*T![placement] PLACEMENT POLICY=`p2` */", + "CREATE TABLE `t1` (`a` INT(11) DEFAULT NULL) ENGINE = InnoDB DEFAULT CHARACTER SET = UTF8MB4 DEFAULT COLLATE = UTF8MB4_BIN ", + }, + { + "CREATE TABLE t4 (firstname VARCHAR(25) NOT NULL,lastname VARCHAR(25) NOT NULL,username VARCHAR(16) NOT NULL,email VARCHAR(35),joined DATE NOT NULL) PARTITION BY RANGE( YEAR(joined) ) (PARTITION p0 VALUES LESS THAN (1960) PLACEMENT POLICY=p1,PARTITION p1 VALUES LESS THAN (1970),PARTITION p2 VALUES LESS THAN (1980),PARTITION p3 VALUES LESS THAN (1990),PARTITION p4 VALUES LESS THAN MAXVALUE);", + "CREATE TABLE `t4` (`firstname` VARCHAR(25) NOT NULL,`lastname` VARCHAR(25) NOT NULL,`username` VARCHAR(16) NOT NULL,`email` VARCHAR(35),`joined` DATE NOT NULL) PARTITION BY RANGE (YEAR(`joined`)) (PARTITION `p0` VALUES LESS THAN (1960) ,PARTITION `p1` VALUES LESS THAN (1970),PARTITION `p2` VALUES LESS THAN (1980),PARTITION `p3` VALUES LESS THAN (1990),PARTITION `p4` VALUES LESS THAN (MAXVALUE))", + }, + { + "ALTER TABLE t3 PLACEMENT POLICY=DEFAULT;", + "ALTER TABLE `t3`", + }, + { + "ALTER TABLE t1 PLACEMENT POLICY=p10", + "ALTER TABLE `t1`", + }, + { + "ALTER TABLE t1 PLACEMENT POLICY=p10, add d text(50)", + "ALTER TABLE `t1` ADD COLUMN `d` TEXT(50)", + }, + { + "alter table tp PARTITION p1 placement policy p2", + "", + }, + { + "alter table t add d text(50) PARTITION p1 placement policy p2", + "ALTER TABLE `t` ADD COLUMN `d` TEXT(50)", + }, + { + "alter table tp set tiflash replica 1 PARTITION p1 placement policy p2", + "ALTER TABLE `tp` SET TIFLASH REPLICA 1", + }, + { + "ALTER DATABASE TestResetPlacementDB PLACEMENT POLICY SET DEFAULT", + "", + }, + + { + "ALTER DATABASE TestResetPlacementDB PLACEMENT POLICY p1 charset utf8mb4", + "ALTER DATABASE `TestResetPlacementDB` CHARACTER SET = utf8mb4", + }, + { + "/*T![placement] ALTER DATABASE `db1` PLACEMENT POLICY = `p1` */", + "", + }, + { + "ALTER PLACEMENT POLICY p3 PRIMARY_REGION='us-east-1' REGIONS='us-east-1,us-east-2,us-west-1';", + "", + }, + } + + extractNodeFunc := func(node Node) Node { + return node + } + + for _, ca := range cases { + testCases := []NodeRestoreTestCase{ + {ca.sourceSQL, ca.expectSQL}, + } + runNodeRestoreTestWithFlagsStmtChange(t, testCases, "%s", extractNodeFunc, f) + } +} diff --git a/parser/ast/dml_test.go b/parser/ast/dml_test.go index 624717d2d7c54..f569543f701ea 100644 --- a/parser/ast/dml_test.go +++ b/parser/ast/dml_test.go @@ -17,6 +17,7 @@ import ( "testing" . "github.com/pingcap/tidb/parser/ast" + "github.com/pingcap/tidb/parser/format" "github.com/stretchr/testify/require" ) @@ -238,7 +239,7 @@ func TestJoinRestore(t *testing.T) { return node.(*SelectStmt).From.TableRefs } runNodeRestoreTest(t, testCases, "select * from %s", extractNodeFunc) - runNodeRestoreTestWithFlagsStmtChange(t, testChangedCases, "select * from %s", extractNodeFunc) + runNodeRestoreTestWithFlagsStmtChange(t, testChangedCases, "select * from %s", extractNodeFunc, format.DefaultRestoreFlags) } func TestTableRefsClauseRestore(t *testing.T) { diff --git a/parser/ast/util_test.go b/parser/ast/util_test.go index 29e813b1131b8..015f5dc5cc4eb 100644 --- a/parser/ast/util_test.go +++ b/parser/ast/util_test.go @@ -190,7 +190,7 @@ func runNodeRestoreTestWithFlags(t *testing.T, nodeTestCases []NodeRestoreTestCa // runNodeRestoreTestWithFlagsStmtChange likes runNodeRestoreTestWithFlags but not check if the ASTs are same. // Sometimes the AST are different and it's expected. -func runNodeRestoreTestWithFlagsStmtChange(t *testing.T, nodeTestCases []NodeRestoreTestCase, template string, extractNodeFunc func(node Node) Node) { +func runNodeRestoreTestWithFlagsStmtChange(t *testing.T, nodeTestCases []NodeRestoreTestCase, template string, extractNodeFunc func(node Node) Node, flags RestoreFlags) { p := parser.New() p.EnableWindowFunc(true) for _, testCase := range nodeTestCases { @@ -200,7 +200,7 @@ func runNodeRestoreTestWithFlagsStmtChange(t *testing.T, nodeTestCases []NodeRes comment := fmt.Sprintf("source %#v", testCase) require.NoError(t, err, comment) var sb strings.Builder - err = extractNodeFunc(stmt).Restore(NewRestoreCtx(DefaultRestoreFlags, &sb)) + err = extractNodeFunc(stmt).Restore(NewRestoreCtx(flags, &sb)) require.NoError(t, err, comment) restoreSql := fmt.Sprintf(template, sb.String()) comment = fmt.Sprintf("source %#v; restore %v", testCase, restoreSql) diff --git a/parser/format/format.go b/parser/format/format.go index 3147d992a18a4..4b9582f49fdd3 100644 --- a/parser/format/format.go +++ b/parser/format/format.go @@ -224,6 +224,7 @@ const ( RestoreStringWithoutDefaultCharset RestoreTiDBSpecialComment + SkipPlacementRuleForRestore ) const ( @@ -300,6 +301,10 @@ func (rf RestoreFlags) HasTiDBSpecialCommentFlag() bool { return rf.has(RestoreTiDBSpecialComment) } +func (rf RestoreFlags) HasSkipPlacementRuleForRestoreFlag() bool { + return rf.has(SkipPlacementRuleForRestore) +} + // RestoreCtx is `Restore` context to hold flags and writer. type RestoreCtx struct { Flags RestoreFlags diff --git a/telemetry/data_feature_usage.go b/telemetry/data_feature_usage.go index e05e93a7bbe54..e999269ce6a4e 100644 --- a/telemetry/data_feature_usage.go +++ b/telemetry/data_feature_usage.go @@ -149,9 +149,12 @@ type TemporaryOrCacheTableFeatureChecker interface { // TxnUsage records the usage info of transaction related features, including // async-commit, 1PC and counters of transactions committed with different protocols. type TxnUsage struct { - AsyncCommitUsed bool `json:"asyncCommitUsed"` - OnePCUsed bool `json:"onePCUsed"` - TxnCommitCounter metrics.TxnCommitCounter `json:"txnCommitCounter"` + AsyncCommitUsed bool `json:"asyncCommitUsed"` + OnePCUsed bool `json:"onePCUsed"` + TxnCommitCounter metrics.TxnCommitCounter `json:"txnCommitCounter"` + MutationCheckerUsed bool `json:"mutationCheckerUsed"` + AssertionLevel string `json:"assertionLevel"` + RcCheckTS bool `json:"rcCheckTS"` } var initialTxnCommitCounter metrics.TxnCommitCounter @@ -169,7 +172,19 @@ func getTxnUsageInfo(ctx sessionctx.Context) *TxnUsage { } curr := metrics.GetTxnCommitCounter() diff := curr.Sub(initialTxnCommitCounter) - return &TxnUsage{asyncCommitUsed, onePCUsed, diff} + mutationCheckerUsed := false + if val, err := variable.GetGlobalSystemVar(ctx.GetSessionVars(), variable.TiDBEnableMutationChecker); err == nil { + mutationCheckerUsed = val == variable.On + } + assertionUsed := "" + if val, err := variable.GetGlobalSystemVar(ctx.GetSessionVars(), variable.TiDBTxnAssertionLevel); err == nil { + assertionUsed = val + } + rcCheckTSUsed := false + if val, err := variable.GetGlobalSystemVar(ctx.GetSessionVars(), variable.TiDBRCReadCheckTS); err == nil { + rcCheckTSUsed = val == variable.On + } + return &TxnUsage{asyncCommitUsed, onePCUsed, diff, mutationCheckerUsed, assertionUsed, rcCheckTSUsed} } func postReportTxnUsage() { diff --git a/telemetry/data_feature_usage_test.go b/telemetry/data_feature_usage_test.go index a5ebdf2a06cef..5f885411dc486 100644 --- a/telemetry/data_feature_usage_test.go +++ b/telemetry/data_feature_usage_test.go @@ -43,6 +43,26 @@ func TestTxnUsageInfo(t *testing.T) { txnUsage = telemetry.GetTxnUsageInfo(tk.Session()) require.True(t, txnUsage.AsyncCommitUsed) require.True(t, txnUsage.OnePCUsed) + + tk.MustExec(fmt.Sprintf("set global %s = 0", variable.TiDBEnableMutationChecker)) + tk.MustExec(fmt.Sprintf("set global %s = off", variable.TiDBTxnAssertionLevel)) + txnUsage = telemetry.GetTxnUsageInfo(tk.Session()) + require.False(t, txnUsage.MutationCheckerUsed) + require.Equal(t, "OFF", txnUsage.AssertionLevel) + + tk.MustExec(fmt.Sprintf("set global %s = 1", variable.TiDBEnableMutationChecker)) + tk.MustExec(fmt.Sprintf("set global %s = strict", variable.TiDBTxnAssertionLevel)) + txnUsage = telemetry.GetTxnUsageInfo(tk.Session()) + require.True(t, txnUsage.MutationCheckerUsed) + require.Equal(t, "STRICT", txnUsage.AssertionLevel) + + tk.MustExec(fmt.Sprintf("set global %s = fast", variable.TiDBTxnAssertionLevel)) + txnUsage = telemetry.GetTxnUsageInfo(tk.Session()) + require.Equal(t, "FAST", txnUsage.AssertionLevel) + + tk.MustExec(fmt.Sprintf("set global %s = 1", variable.TiDBRCReadCheckTS)) + txnUsage = telemetry.GetTxnUsageInfo(tk.Session()) + require.True(t, txnUsage.RcCheckTS) }) t.Run("Count", func(t *testing.T) { diff --git a/testkit/external/util.go b/testkit/external/util.go index fd2278125078d..88776b9eeac79 100644 --- a/testkit/external/util.go +++ b/testkit/external/util.go @@ -26,6 +26,16 @@ import ( "github.com/stretchr/testify/require" ) +// GetSchemaByName gets schema by name for test. +func GetSchemaByName(t *testing.T, tk *testkit.TestKit, db string) *model.DBInfo { + dom := domain.GetDomain(tk.Session()) + // Make sure the table schema is the new schema. + require.NoError(t, dom.Reload()) + dbInfo, ok := dom.InfoSchema().SchemaByName(model.NewCIStr(db)) + require.True(t, ok) + return dbInfo +} + // GetTableByName gets table by name for test. func GetTableByName(t *testing.T, tk *testkit.TestKit, db, table string) table.Table { dom := domain.GetDomain(tk.Session())