Skip to content

Commit

Permalink
ddl: fix create and alter check constraints problems (#47633)
Browse files Browse the repository at this point in the history
close #47567, close #47631, close #47632
  • Loading branch information
jiyfhust committed Nov 8, 2023
1 parent 59bd06c commit 14ff60d
Show file tree
Hide file tree
Showing 4 changed files with 127 additions and 47 deletions.
72 changes: 41 additions & 31 deletions pkg/ddl/constraint.go
Original file line number Diff line number Diff line change
Expand Up @@ -29,19 +29,14 @@ 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"
)

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) {
Expand Down Expand Up @@ -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)
}
Expand Down Expand Up @@ -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.
Expand All @@ -167,16 +175,11 @@ 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)
}
// 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)
}
Expand Down Expand Up @@ -212,29 +215,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)
}
Expand Down Expand Up @@ -336,7 +348,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) {
Expand All @@ -363,8 +375,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
Expand Down
49 changes: 35 additions & 14 deletions pkg/ddl/rollingback.go
Original file line number Diff line number Diff line change
Expand Up @@ -498,15 +498,16 @@ 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)
case model.ActionAddCheckConstraint:
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
Expand Down Expand Up @@ -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)
Expand All @@ -565,18 +565,17 @@ 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)
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
}
}
return ver, dbterror.ErrCancelledDDLJob
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) {
Expand All @@ -594,3 +593,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)
}
32 changes: 31 additions & 1 deletion tests/integrationtest/r/ddl/constraint.result
Original file line number Diff line number Diff line change
Expand Up @@ -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<b+1));
Expand Down Expand Up @@ -789,4 +788,35 @@ t CREATE TABLE `t` (
CONSTRAINT `haha` CHECK ((`a` < `b`)) /*!80016 NOT ENFORCED */,
CONSTRAINT `t_chk_1` 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;
21 changes: 20 additions & 1 deletion tests/integrationtest/t/ddl/constraint.test
Original file line number Diff line number Diff line change
Expand Up @@ -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;
# 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;

0 comments on commit 14ff60d

Please sign in to comment.