From 2445f9d51bbe6dd925370e94a106dcea5a66676a Mon Sep 17 00:00:00 2001 From: jiyfhust Date: Sat, 14 Oct 2023 14:34:56 +0800 Subject: [PATCH 1/4] fix some check constrants ddl bugs --- pkg/ddl/constraint.go | 65 +++++++++++++++++++++++--------------- pkg/ddl/constraint_test.go | 18 +++++++++++ pkg/ddl/rollingback.go | 53 +++++++++++++++++++++++-------- 3 files changed, 97 insertions(+), 39 deletions(-) diff --git a/pkg/ddl/constraint.go b/pkg/ddl/constraint.go index 02b9843887caf..105eca8095cff 100644 --- a/pkg/ddl/constraint.go +++ b/pkg/ddl/constraint.go @@ -29,7 +29,6 @@ import ( "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/sessionctx" - "github.com/pingcap/tidb/pkg/table" "github.com/pingcap/tidb/pkg/util/dbterror" "github.com/pingcap/tidb/pkg/util/sqlexec" ) @@ -37,11 +36,7 @@ import ( func (w *worker) onAddCheckConstraint(d *ddlCtx, t *meta.Meta, job *model.Job) (ver int64, err error) { // Handle the rolling back job. if job.IsRollingback() { - ver, err = onDropCheckConstraint(d, t, job) - if err != nil { - return ver, errors.Trace(err) - } - return ver, nil + return rollingBackAddConstraint(d, t, job) } failpoint.Inject("errorBeforeDecodeArgs", func(val failpoint.Value) { @@ -84,23 +79,37 @@ func (w *worker) onAddCheckConstraint(d *ddlCtx, t *meta.Meta, job *model.Job) ( constraintInfoInMeta = constraintInfoInJob } - originalState := constraintInfoInMeta.State + // If not enforced, add it directly. + if !constraintInfoInMeta.Enforced { + constraintInfoInMeta.State = model.StatePublic + ver, err = updateVersionAndTableInfo(d, t, job, tblInfo, true) + if err != nil { + return ver, errors.Trace(err) + } + // Finish this job. + job.FinishTableJob(model.JobStateDone, model.StatePublic, ver, tblInfo) + return ver, nil + } + switch constraintInfoInMeta.State { case model.StateNone: job.SchemaState = model.StateWriteOnly constraintInfoInMeta.State = model.StateWriteOnly - ver, err = updateVersionAndTableInfoWithCheck(d, t, job, tblInfo, originalState != constraintInfoInMeta.State) + ver, err = updateVersionAndTableInfoWithCheck(d, t, job, tblInfo, true) case model.StateWriteOnly: job.SchemaState = model.StateWriteReorganization constraintInfoInMeta.State = model.StateWriteReorganization - ver, err = updateVersionAndTableInfoWithCheck(d, t, job, tblInfo, originalState != constraintInfoInMeta.State) + ver, err = updateVersionAndTableInfoWithCheck(d, t, job, tblInfo, true) case model.StateWriteReorganization: - err = w.verifyRemainRecordsForCheckConstraint(dbInfo, tblInfo, constraintInfoInMeta, job) + err = w.verifyRemainRecordsForCheckConstraint(dbInfo, tblInfo, constraintInfoInMeta) if err != nil { + if dbterror.ErrCheckConstraintIsViolated.Equal(err) { + job.State = model.JobStateRollingback + } return ver, errors.Trace(err) } constraintInfoInMeta.State = model.StatePublic - ver, err = updateVersionAndTableInfo(d, t, job, tblInfo, originalState != constraintInfoInMeta.State) + ver, err = updateVersionAndTableInfo(d, t, job, tblInfo, true) if err != nil { return ver, errors.Trace(err) } @@ -151,12 +160,11 @@ func onDropCheckConstraint(d *ddlCtx, t *meta.Meta, job *model.Job) (ver int64, return ver, errors.Trace(err) } - originalState := constraintInfo.State switch constraintInfo.State { case model.StatePublic: job.SchemaState = model.StateWriteOnly constraintInfo.State = model.StateWriteOnly - ver, err = updateVersionAndTableInfoWithCheck(d, t, job, tblInfo, originalState != constraintInfo.State) + ver, err = updateVersionAndTableInfoWithCheck(d, t, job, tblInfo, true) case model.StateWriteOnly: // write only state constraint will still take effect to check the newly inserted data. // So the dependent column shouldn't be dropped even in this intermediate state. @@ -167,7 +175,7 @@ func onDropCheckConstraint(d *ddlCtx, t *meta.Meta, job *model.Job) (ver int64, tblInfo.Constraints = append(tblInfo.Constraints[0:i], tblInfo.Constraints[i+1:]...) } } - ver, err = updateVersionAndTableInfo(d, t, job, tblInfo, originalState != constraintInfo.State) + ver, err = updateVersionAndTableInfo(d, t, job, tblInfo, true) if err != nil { return ver, errors.Trace(err) } @@ -212,29 +220,38 @@ func (w *worker) onAlterCheckConstraint(d *ddlCtx, t *meta.Meta, job *model.Job) return ver, errors.Trace(err) } + if job.IsRollingback() { + return rollingBackAlterConstraint(d, t, job) + } + + // Current State is desired. + if constraintInfo.State == model.StatePublic && constraintInfo.Enforced == enforced { + job.FinishTableJob(model.JobStateDone, model.StatePublic, ver, tblInfo) + return + } + // enforced will fetch table data and check the constraint. if enforced { - originalState := constraintInfo.State switch constraintInfo.State { case model.StatePublic: job.SchemaState = model.StateWriteReorganization constraintInfo.State = model.StateWriteReorganization constraintInfo.Enforced = enforced - ver, err = updateVersionAndTableInfoWithCheck(d, t, job, tblInfo, originalState != constraintInfo.State) + ver, err = updateVersionAndTableInfoWithCheck(d, t, job, tblInfo, true) case model.StateWriteReorganization: job.SchemaState = model.StateWriteOnly constraintInfo.State = model.StateWriteOnly - ver, err = updateVersionAndTableInfoWithCheck(d, t, job, tblInfo, originalState != constraintInfo.State) + ver, err = updateVersionAndTableInfoWithCheck(d, t, job, tblInfo, true) case model.StateWriteOnly: - err = w.verifyRemainRecordsForCheckConstraint(dbInfo, tblInfo, constraintInfo, job) + err = w.verifyRemainRecordsForCheckConstraint(dbInfo, tblInfo, constraintInfo) if err != nil { - if !table.ErrCheckConstraintViolated.Equal(err) { - return ver, errors.Trace(err) + if dbterror.ErrCheckConstraintIsViolated.Equal(err) { + job.State = model.JobStateRollingback } - constraintInfo.Enforced = !enforced + return ver, errors.Trace(err) } constraintInfo.State = model.StatePublic - ver, err = updateVersionAndTableInfoWithCheck(d, t, job, tblInfo, originalState != constraintInfo.State) + ver, err = updateVersionAndTableInfoWithCheck(d, t, job, tblInfo, true) if err != nil { return ver, errors.Trace(err) } @@ -336,7 +353,7 @@ func findDependentColsInExpr(expr ast.ExprNode) map[string]struct{} { return colsMap } -func (w *worker) verifyRemainRecordsForCheckConstraint(dbInfo *model.DBInfo, tableInfo *model.TableInfo, constr *model.ConstraintInfo, job *model.Job) error { +func (w *worker) verifyRemainRecordsForCheckConstraint(dbInfo *model.DBInfo, tableInfo *model.TableInfo, constr *model.ConstraintInfo) error { // Inject a fail-point to skip the remaining records check. failpoint.Inject("mockVerifyRemainDataSuccess", func(val failpoint.Value) { if val.(bool) { @@ -363,8 +380,6 @@ func (w *worker) verifyRemainRecordsForCheckConstraint(dbInfo *model.DBInfo, tab } rowCount := len(rows) if rowCount != 0 { - // If check constraint fail, the job state should be changed to canceled, otherwise it will tracked in. - job.State = model.JobStateCancelled return dbterror.ErrCheckConstraintIsViolated.GenWithStackByArgs(constr.Name.L) } return nil diff --git a/pkg/ddl/constraint_test.go b/pkg/ddl/constraint_test.go index c1837bcd54d6b..7e00827d4b750 100644 --- a/pkg/ddl/constraint_test.go +++ b/pkg/ddl/constraint_test.go @@ -287,3 +287,21 @@ func TestAlterEnforcedConstraintStateChange(t *testing.T) { tk.MustExec("alter table t alter constraint c1 enforced") tk.MustQuery("select * from t").Check(testkit.Rows("12")) } + +// Related issue TiDB#47567, #47631 and #47632. +func TestCheckConstraintIssue47567(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + + tk.MustExec("set @@global.tidb_enable_check_constraint = 1") + tk.MustExec("use test") + tk.MustExec("CREATE TABLE `t` (`a` int(11) DEFAULT NULL)") + tk.MustExec("insert t values(1)") + tk.MustGetErrMsg("alter table t ADD CONSTRAINT chk CHECK (a > 1) ENFORCED", "[ddl:3819]Check constraint 'chk' is violated.") + tk.MustGetErrMsg("alter table t ADD CONSTRAINT chk CHECK (a > 1) ENFORCED", "[ddl:3819]Check constraint 'chk' is violated.") + tk.MustExec("alter table t ADD CONSTRAINT chk CHECK (a > 1) NOT ENFORCED") + tk.MustGetErrMsg("ALTER TABLE t ALTER CONSTRAINT chk ENFORCED;", "[ddl:3819]Check constraint 'chk' is violated.") + tk.MustQuery("select constraint_name from information_schema.CHECK_CONSTRAINTS where constraint_schema = 'test'").Check(testkit.Rows("chk")) + tk.MustExec("alter table t drop CONSTRAINT chk") + tk.MustQuery("select constraint_name from information_schema.CHECK_CONSTRAINTS where constraint_schema = 'test'").Check(testkit.Rows()) +} diff --git a/pkg/ddl/rollingback.go b/pkg/ddl/rollingback.go index 3ea1082a28b35..abaeecf2a3396 100644 --- a/pkg/ddl/rollingback.go +++ b/pkg/ddl/rollingback.go @@ -498,8 +498,7 @@ func convertJob2RollbackJob(w *worker, d *ddlCtx, t *meta.Meta, job *model.Job) model.ActionModifyTableCharsetAndCollate, model.ActionModifySchemaCharsetAndCollate, model.ActionRepairTable, model.ActionModifyTableAutoIdCache, model.ActionAlterIndexVisibility, - model.ActionModifySchemaDefaultPlacement, - model.ActionRecoverSchema, model.ActionAlterCheckConstraint: + model.ActionModifySchemaDefaultPlacement, model.ActionRecoverSchema: ver, err = cancelOnlyNotHandledJob(job, model.StateNone) case model.ActionMultiSchemaChange: err = rollingBackMultiSchemaChange(job) @@ -507,6 +506,8 @@ func convertJob2RollbackJob(w *worker, d *ddlCtx, t *meta.Meta, job *model.Job) ver, err = rollingBackAddConstraint(d, t, job) case model.ActionDropCheckConstraint: ver, err = rollingBackDropConstraint(t, job) + case model.ActionAlterCheckConstraint: + ver, err = rollingBackAlterConstraint(d, t, job) default: job.State = model.JobStateCancelled err = dbterror.ErrCancelledDDLJob @@ -554,7 +555,6 @@ func convertJob2RollbackJob(w *worker, d *ddlCtx, t *meta.Meta, job *model.Job) } func rollingBackAddConstraint(d *ddlCtx, t *meta.Meta, job *model.Job) (ver int64, err error) { - job.State = model.JobStateRollingback _, tblInfo, constrInfoInMeta, _, err := checkAddCheckConstraint(t, job) if err != nil { return ver, errors.Trace(err) @@ -565,18 +565,21 @@ func rollingBackAddConstraint(d *ddlCtx, t *meta.Meta, job *model.Job) (ver int6 job.State = model.JobStateCancelled return ver, dbterror.ErrCancelledDDLJob } - // Add constraint has stored constraint info into meta, that means the job has at least - // arrived write only state. - originalState := constrInfoInMeta.State - constrInfoInMeta.State = model.StateWriteOnly - job.SchemaState = model.StateWriteOnly - - job.Args = []interface{}{constrInfoInMeta.Name} - ver, err = updateVersionAndTableInfo(d, t, job, tblInfo, originalState != constrInfoInMeta.State) - if err != nil { - return ver, errors.Trace(err) + // StatePublic means the check have already been added. + if constrInfoInMeta.State == model.StatePublic { + // Do we need handle it especially? } - return ver, dbterror.ErrCancelledDDLJob + for i, constr := range tblInfo.Constraints { + if constr.Name.L == constrInfoInMeta.Name.L { + tblInfo.Constraints = append(tblInfo.Constraints[0:i], tblInfo.Constraints[i+1:]...) + break + } + } + if job.IsRollingback() { + job.State = model.JobStateRollbackDone + } + ver, err = updateVersionAndTableInfo(d, t, job, tblInfo, true) + return ver, errors.Trace(err) } func rollingBackDropConstraint(t *meta.Meta, job *model.Job) (ver int64, err error) { @@ -594,3 +597,25 @@ func rollingBackDropConstraint(t *meta.Meta, job *model.Job) (ver int64, err err job.State = model.JobStateRunning return ver, nil } + +func rollingBackAlterConstraint(d *ddlCtx, t *meta.Meta, job *model.Job) (ver int64, err error) { + _, tblInfo, constraintInfo, enforced, err := checkAlterCheckConstraint(t, job) + if err != nil { + return ver, errors.Trace(err) + } + + // StatePublic means when the job is not running yet. + if constraintInfo.State == model.StatePublic { + job.State = model.JobStateCancelled + return ver, dbterror.ErrCancelledDDLJob + } + + // Only alter check constraints ENFORCED can get here. + constraintInfo.Enforced = !enforced + constraintInfo.State = model.StatePublic + if job.IsRollingback() { + job.State = model.JobStateRollbackDone + } + ver, err = updateVersionAndTableInfoWithCheck(d, t, job, tblInfo, true) + return ver, errors.Trace(err) +} From 2f9f36313c7208005ffbc748a9b952fd28d3a1a4 Mon Sep 17 00:00:00 2001 From: jiyfhust Date: Sun, 15 Oct 2023 19:45:15 +0800 Subject: [PATCH 2/4] fix --- pkg/ddl/constraint.go | 7 +------ 1 file changed, 1 insertion(+), 6 deletions(-) diff --git a/pkg/ddl/constraint.go b/pkg/ddl/constraint.go index 105eca8095cff..1eaee3c618fa0 100644 --- a/pkg/ddl/constraint.go +++ b/pkg/ddl/constraint.go @@ -179,12 +179,7 @@ func onDropCheckConstraint(d *ddlCtx, t *meta.Meta, job *model.Job) (ver int64, if err != nil { return ver, errors.Trace(err) } - // Finish this job. - if job.IsRollingback() { - job.FinishTableJob(model.JobStateRollbackDone, model.StateNone, ver, tblInfo) - } else { - job.FinishTableJob(model.JobStateDone, model.StateNone, ver, tblInfo) - } + job.FinishTableJob(model.JobStateDone, model.StateNone, ver, tblInfo) default: err = dbterror.ErrInvalidDDLJob.GenWithStackByArgs("constraint", tblInfo.State) } From 5ab124cc55eaa230ec00caed3d9eec211ae3bba4 Mon Sep 17 00:00:00 2001 From: jiyfhust Date: Sat, 28 Oct 2023 08:48:24 +0800 Subject: [PATCH 3/4] fix --- pkg/ddl/rollingback.go | 5 +---- 1 file changed, 1 insertion(+), 4 deletions(-) diff --git a/pkg/ddl/rollingback.go b/pkg/ddl/rollingback.go index abaeecf2a3396..2a39d1bd6f429 100644 --- a/pkg/ddl/rollingback.go +++ b/pkg/ddl/rollingback.go @@ -565,10 +565,7 @@ func rollingBackAddConstraint(d *ddlCtx, t *meta.Meta, job *model.Job) (ver int6 job.State = model.JobStateCancelled return ver, dbterror.ErrCancelledDDLJob } - // StatePublic means the check have already been added. - if constrInfoInMeta.State == model.StatePublic { - // Do we need handle it especially? - } + // Is there a case constrInfoInMeta.State become StatePublic that means the constraint have already been added successfully? for i, constr := range tblInfo.Constraints { if constr.Name.L == constrInfoInMeta.Name.L { tblInfo.Constraints = append(tblInfo.Constraints[0:i], tblInfo.Constraints[i+1:]...) From a9b1fd83fbaf43f628e4d64c1f5ba343f5b61a3f Mon Sep 17 00:00:00 2001 From: jiyfhust Date: Mon, 30 Oct 2023 10:59:10 +0800 Subject: [PATCH 4/4] fix test --- pkg/ddl/constraint_test.go | 18 ----------- pkg/ddl/rollingback.go | 1 - tests/integrationtest/r/ddl/constraint.result | 32 ++++++++++++++++++- tests/integrationtest/t/ddl/constraint.test | 21 +++++++++++- 4 files changed, 51 insertions(+), 21 deletions(-) diff --git a/pkg/ddl/constraint_test.go b/pkg/ddl/constraint_test.go index 7e00827d4b750..c1837bcd54d6b 100644 --- a/pkg/ddl/constraint_test.go +++ b/pkg/ddl/constraint_test.go @@ -287,21 +287,3 @@ func TestAlterEnforcedConstraintStateChange(t *testing.T) { tk.MustExec("alter table t alter constraint c1 enforced") tk.MustQuery("select * from t").Check(testkit.Rows("12")) } - -// Related issue TiDB#47567, #47631 and #47632. -func TestCheckConstraintIssue47567(t *testing.T) { - store := testkit.CreateMockStore(t) - tk := testkit.NewTestKit(t, store) - - tk.MustExec("set @@global.tidb_enable_check_constraint = 1") - tk.MustExec("use test") - tk.MustExec("CREATE TABLE `t` (`a` int(11) DEFAULT NULL)") - tk.MustExec("insert t values(1)") - tk.MustGetErrMsg("alter table t ADD CONSTRAINT chk CHECK (a > 1) ENFORCED", "[ddl:3819]Check constraint 'chk' is violated.") - tk.MustGetErrMsg("alter table t ADD CONSTRAINT chk CHECK (a > 1) ENFORCED", "[ddl:3819]Check constraint 'chk' is violated.") - tk.MustExec("alter table t ADD CONSTRAINT chk CHECK (a > 1) NOT ENFORCED") - tk.MustGetErrMsg("ALTER TABLE t ALTER CONSTRAINT chk ENFORCED;", "[ddl:3819]Check constraint 'chk' is violated.") - tk.MustQuery("select constraint_name from information_schema.CHECK_CONSTRAINTS where constraint_schema = 'test'").Check(testkit.Rows("chk")) - tk.MustExec("alter table t drop CONSTRAINT chk") - tk.MustQuery("select constraint_name from information_schema.CHECK_CONSTRAINTS where constraint_schema = 'test'").Check(testkit.Rows()) -} diff --git a/pkg/ddl/rollingback.go b/pkg/ddl/rollingback.go index 2a39d1bd6f429..44ee0f2d0b452 100644 --- a/pkg/ddl/rollingback.go +++ b/pkg/ddl/rollingback.go @@ -565,7 +565,6 @@ func rollingBackAddConstraint(d *ddlCtx, t *meta.Meta, job *model.Job) (ver int6 job.State = model.JobStateCancelled return ver, dbterror.ErrCancelledDDLJob } - // Is there a case constrInfoInMeta.State become StatePublic that means the constraint have already been added successfully? for i, constr := range tblInfo.Constraints { if constr.Name.L == constrInfoInMeta.Name.L { tblInfo.Constraints = append(tblInfo.Constraints[0:i], tblInfo.Constraints[i+1:]...) diff --git a/tests/integrationtest/r/ddl/constraint.result b/tests/integrationtest/r/ddl/constraint.result index 803643e47e192..06b0679415857 100644 --- a/tests/integrationtest/r/ddl/constraint.result +++ b/tests/integrationtest/r/ddl/constraint.result @@ -720,7 +720,6 @@ insert into t values(1), (2), (3); alter table t add constraint check(a < 2); Error 3819 (HY000): Check constraint 't_chk_1' is violated. alter table t add constraint check(a < 2) not enforced; -Error 3819 (HY000): Check constraint 't_chk_1' is violated. drop table if exists t; set @@global.tidb_enable_check_constraint = 1; create table t(a int not null check(a>0), b int, constraint haha check(a < b), check(a 0)) ) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin +drop table if exists t; +set @@global.tidb_enable_check_constraint = 1; +CREATE TABLE `t` (`a` int(11) DEFAULT NULL); +show create table t; +Table Create Table +t CREATE TABLE `t` ( + `a` int(11) DEFAULT NULL +) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin +insert t values(1); +select * from t; +a +1 +alter table t ADD CONSTRAINT chk CHECK (a > 1) ENFORCED; +Error 3819 (HY000): Check constraint 'chk' is violated. +alter table t ADD CONSTRAINT chk CHECK (a > 1) ENFORCED; +Error 3819 (HY000): Check constraint 'chk' is violated. +alter table t ADD CONSTRAINT chk CHECK (a > 1) NOT ENFORCED; +ALTER TABLE t ALTER CONSTRAINT chk ENFORCED; +Error 3819 (HY000): Check constraint 'chk' is violated. +show create table t; +Table Create Table +t CREATE TABLE `t` ( + `a` int(11) DEFAULT NULL, +CONSTRAINT `chk` CHECK ((`a` > 1)) /*!80016 NOT ENFORCED */ +) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin +alter table t drop CONSTRAINT chk; +show create table t; +Table Create Table +t CREATE TABLE `t` ( + `a` int(11) DEFAULT NULL +) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin set @@global.tidb_enable_check_constraint = 0; diff --git a/tests/integrationtest/t/ddl/constraint.test b/tests/integrationtest/t/ddl/constraint.test index 8a53e8a79ad37..d197575970820 100644 --- a/tests/integrationtest/t/ddl/constraint.test +++ b/tests/integrationtest/t/ddl/constraint.test @@ -658,4 +658,23 @@ show create table t; # _, err = tk.Exec("alter table t alter constraint haha enforced") # require.Errorf(t, err, "[table:3819]Check constraint 'haha' is violated.") -set @@global.tidb_enable_check_constraint = 0; \ No newline at end of file +# Related issue TiDB#47567, #47631 and #47632. +# TestCheckConstraintIssue47567 +drop table if exists t; +set @@global.tidb_enable_check_constraint = 1; +CREATE TABLE `t` (`a` int(11) DEFAULT NULL); +show create table t; +insert t values(1); +select * from t; +-- error 3940 +alter table t ADD CONSTRAINT chk CHECK (a > 1) ENFORCED; +-- error 3940 +alter table t ADD CONSTRAINT chk CHECK (a > 1) ENFORCED; +alter table t ADD CONSTRAINT chk CHECK (a > 1) NOT ENFORCED; +-- error 3940 +ALTER TABLE t ALTER CONSTRAINT chk ENFORCED; +show create table t; +alter table t drop CONSTRAINT chk; +show create table t; + +set @@global.tidb_enable_check_constraint = 0;