From 79a9989dc841a6c92610bbdd524d16385c6fc23d Mon Sep 17 00:00:00 2001 From: jiyfhust Date: Wed, 8 Nov 2023 10:56:12 +0800 Subject: [PATCH] This is an automated cherry-pick of #47633 Signed-off-by: ti-chi-bot --- pkg/ddl/constraint.go | 427 +++++++++ pkg/ddl/rollingback.go | 617 +++++++++++++ tests/integrationtest/r/ddl/constraint.result | 822 ++++++++++++++++++ tests/integrationtest/t/ddl/constraint.test | 680 +++++++++++++++ 4 files changed, 2546 insertions(+) create mode 100644 pkg/ddl/constraint.go create mode 100644 pkg/ddl/rollingback.go create mode 100644 tests/integrationtest/r/ddl/constraint.result create mode 100644 tests/integrationtest/t/ddl/constraint.test diff --git a/pkg/ddl/constraint.go b/pkg/ddl/constraint.go new file mode 100644 index 0000000000000..1eaee3c618fa0 --- /dev/null +++ b/pkg/ddl/constraint.go @@ -0,0 +1,427 @@ +// Copyright 2023-2023 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package ddl + +import ( + "context" + "fmt" + "strings" + + "github.com/pingcap/errors" + "github.com/pingcap/failpoint" + "github.com/pingcap/tidb/pkg/infoschema" + "github.com/pingcap/tidb/pkg/kv" + "github.com/pingcap/tidb/pkg/meta" + "github.com/pingcap/tidb/pkg/parser/ast" + "github.com/pingcap/tidb/pkg/parser/format" + "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/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() { + return rollingBackAddConstraint(d, t, job) + } + + failpoint.Inject("errorBeforeDecodeArgs", func(val failpoint.Value) { + if val.(bool) { + failpoint.Return(ver, errors.New("occur an error before decode args")) + } + }) + + dbInfo, tblInfo, constraintInfoInMeta, constraintInfoInJob, err := checkAddCheckConstraint(t, job) + if err != nil { + return ver, errors.Trace(err) + } + if constraintInfoInMeta == nil { + // It's first time to run add constraint job, so there is no constraint info in meta. + // Use the raw constraint info from job directly and modify table info here. + constraintInfoInJob.ID = allocateConstraintID(tblInfo) + // Reset constraint name according to real-time constraints name at this point. + constrNames := map[string]bool{} + for _, constr := range tblInfo.Constraints { + constrNames[constr.Name.L] = true + } + setNameForConstraintInfo(tblInfo.Name.L, constrNames, []*model.ConstraintInfo{constraintInfoInJob}) + // Double check the constraint dependency. + existedColsMap := make(map[string]struct{}) + cols := tblInfo.Columns + for _, v := range cols { + if v.State == model.StatePublic { + existedColsMap[v.Name.L] = struct{}{} + } + } + dependedCols := constraintInfoInJob.ConstraintCols + for _, k := range dependedCols { + if _, ok := existedColsMap[k.L]; !ok { + // The table constraint depended on a non-existed column. + return ver, dbterror.ErrTableCheckConstraintReferUnknown.GenWithStackByArgs(constraintInfoInJob.Name, k) + } + } + + tblInfo.Constraints = append(tblInfo.Constraints, constraintInfoInJob) + constraintInfoInMeta = constraintInfoInJob + } + + // 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, true) + case model.StateWriteOnly: + job.SchemaState = model.StateWriteReorganization + constraintInfoInMeta.State = model.StateWriteReorganization + ver, err = updateVersionAndTableInfoWithCheck(d, t, job, tblInfo, true) + case model.StateWriteReorganization: + 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, true) + if err != nil { + return ver, errors.Trace(err) + } + // Finish this job. + job.FinishTableJob(model.JobStateDone, model.StatePublic, ver, tblInfo) + default: + err = dbterror.ErrInvalidDDLState.GenWithStackByArgs("constraint", constraintInfoInMeta.State) + } + + return ver, errors.Trace(err) +} + +func checkAddCheckConstraint(t *meta.Meta, job *model.Job) (*model.DBInfo, *model.TableInfo, *model.ConstraintInfo, *model.ConstraintInfo, error) { + schemaID := job.SchemaID + dbInfo, err := t.GetDatabase(job.SchemaID) + if err != nil { + return nil, nil, nil, nil, errors.Trace(err) + } + tblInfo, err := GetTableInfoAndCancelFaultJob(t, job, schemaID) + if err != nil { + return nil, nil, nil, nil, errors.Trace(err) + } + constraintInfo1 := &model.ConstraintInfo{} + err = job.DecodeArgs(constraintInfo1) + if err != nil { + job.State = model.JobStateCancelled + return nil, nil, nil, nil, errors.Trace(err) + } + // do the double-check with constraint existence. + constraintInfo2 := tblInfo.FindConstraintInfoByName(constraintInfo1.Name.L) + if constraintInfo2 != nil { + if constraintInfo2.State == model.StatePublic { + // We already have a constraint with the same constraint name. + job.State = model.JobStateCancelled + return nil, nil, nil, nil, infoschema.ErrColumnExists.GenWithStackByArgs(constraintInfo1.Name) + } + // if not, that means constraint was in intermediate state. + } + return dbInfo, tblInfo, constraintInfo2, constraintInfo1, nil +} + +// onDropCheckConstraint can be called from two case: +// 1: rollback in add constraint.(in rollback function the job.args will be changed) +// 2: user drop constraint ddl. +func onDropCheckConstraint(d *ddlCtx, t *meta.Meta, job *model.Job) (ver int64, _ error) { + tblInfo, constraintInfo, err := checkDropCheckConstraint(t, job) + if err != nil { + return ver, errors.Trace(err) + } + + switch constraintInfo.State { + case model.StatePublic: + job.SchemaState = model.StateWriteOnly + constraintInfo.State = model.StateWriteOnly + 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. + constraintInfo.State = model.StateNone + // remove the constraint from tableInfo. + for i, constr := range tblInfo.Constraints { + if constr.Name.L == constraintInfo.Name.L { + tblInfo.Constraints = append(tblInfo.Constraints[0:i], tblInfo.Constraints[i+1:]...) + } + } + ver, err = updateVersionAndTableInfo(d, t, job, tblInfo, true) + if err != nil { + return ver, errors.Trace(err) + } + job.FinishTableJob(model.JobStateDone, model.StateNone, ver, tblInfo) + default: + err = dbterror.ErrInvalidDDLJob.GenWithStackByArgs("constraint", tblInfo.State) + } + return ver, errors.Trace(err) +} + +func checkDropCheckConstraint(t *meta.Meta, job *model.Job) (*model.TableInfo, *model.ConstraintInfo, error) { + schemaID := job.SchemaID + tblInfo, err := GetTableInfoAndCancelFaultJob(t, job, schemaID) + if err != nil { + return nil, nil, errors.Trace(err) + } + + var constrName model.CIStr + err = job.DecodeArgs(&constrName) + if err != nil { + job.State = model.JobStateCancelled + return nil, nil, errors.Trace(err) + } + + // double check with constraint existence. + constraintInfo := tblInfo.FindConstraintInfoByName(constrName.L) + if constraintInfo == nil { + job.State = model.JobStateCancelled + return nil, nil, dbterror.ErrConstraintNotFound.GenWithStackByArgs(constrName) + } + return tblInfo, constraintInfo, nil +} + +func (w *worker) onAlterCheckConstraint(d *ddlCtx, t *meta.Meta, job *model.Job) (ver int64, err error) { + dbInfo, tblInfo, constraintInfo, enforced, err := checkAlterCheckConstraint(t, job) + if err != nil { + 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 { + switch constraintInfo.State { + case model.StatePublic: + job.SchemaState = model.StateWriteReorganization + constraintInfo.State = model.StateWriteReorganization + constraintInfo.Enforced = enforced + 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, true) + case model.StateWriteOnly: + err = w.verifyRemainRecordsForCheckConstraint(dbInfo, tblInfo, constraintInfo) + if err != nil { + if dbterror.ErrCheckConstraintIsViolated.Equal(err) { + job.State = model.JobStateRollingback + } + return ver, errors.Trace(err) + } + constraintInfo.State = model.StatePublic + ver, err = updateVersionAndTableInfoWithCheck(d, t, job, tblInfo, true) + if err != nil { + return ver, errors.Trace(err) + } + job.FinishTableJob(model.JobStateDone, model.StatePublic, ver, tblInfo) + } + } else { + constraintInfo.Enforced = enforced + ver, err = updateVersionAndTableInfoWithCheck(d, t, job, tblInfo, true) + if err != nil { + // update version and tableInfo error will cause retry. + return ver, errors.Trace(err) + } + job.FinishTableJob(model.JobStateDone, model.StatePublic, ver, tblInfo) + } + return ver, err +} + +func checkAlterCheckConstraint(t *meta.Meta, job *model.Job) (*model.DBInfo, *model.TableInfo, *model.ConstraintInfo, bool, error) { + schemaID := job.SchemaID + dbInfo, err := t.GetDatabase(job.SchemaID) + if err != nil { + return nil, nil, nil, false, errors.Trace(err) + } + tblInfo, err := GetTableInfoAndCancelFaultJob(t, job, schemaID) + if err != nil { + return nil, nil, nil, false, errors.Trace(err) + } + + var ( + enforced bool + constrName model.CIStr + ) + err = job.DecodeArgs(&constrName, &enforced) + if err != nil { + job.State = model.JobStateCancelled + return nil, nil, nil, false, errors.Trace(err) + } + // do the double check with constraint existence. + constraintInfo := tblInfo.FindConstraintInfoByName(constrName.L) + if constraintInfo == nil { + job.State = model.JobStateCancelled + return nil, nil, nil, false, dbterror.ErrConstraintNotFound.GenWithStackByArgs(constrName) + } + return dbInfo, tblInfo, constraintInfo, enforced, nil +} + +func allocateConstraintID(tblInfo *model.TableInfo) int64 { + tblInfo.MaxConstraintID++ + return tblInfo.MaxConstraintID +} + +func buildConstraintInfo(tblInfo *model.TableInfo, dependedCols []model.CIStr, constr *ast.Constraint, state model.SchemaState) (*model.ConstraintInfo, error) { + constraintName := model.NewCIStr(constr.Name) + if err := checkTooLongConstraint(constraintName); err != nil { + return nil, errors.Trace(err) + } + + // Restore check constraint expression to string. + var sb strings.Builder + restoreFlags := format.RestoreStringSingleQuotes | format.RestoreKeyWordLowercase | format.RestoreNameBackQuotes | + format.RestoreSpacesAroundBinaryOperation + restoreCtx := format.NewRestoreCtx(restoreFlags, &sb) + + sb.Reset() + err := constr.Expr.Restore(restoreCtx) + if err != nil { + return nil, errors.Trace(err) + } + + // Create constraint info. + constraintInfo := &model.ConstraintInfo{ + Name: constraintName, + Table: tblInfo.Name, + ConstraintCols: dependedCols, + ExprString: sb.String(), + Enforced: constr.Enforced, + InColumn: constr.InColumn, + State: state, + } + + return constraintInfo, nil +} + +func checkTooLongConstraint(constr model.CIStr) error { + if len(constr.L) > mysql.MaxConstraintIdentifierLen { + return dbterror.ErrTooLongIdent.GenWithStackByArgs(constr) + } + return nil +} + +// findDependentColsInExpr returns a set of string, which indicates +// the names of the columns that are dependent by exprNode. +func findDependentColsInExpr(expr ast.ExprNode) map[string]struct{} { + colNames := FindColumnNamesInExpr(expr) + colsMap := make(map[string]struct{}, len(colNames)) + for _, depCol := range colNames { + colsMap[depCol.Name.L] = struct{}{} + } + return colsMap +} + +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) { + failpoint.Return(nil) + } + }) + // Get sessionctx from ddl context resource pool in ddl worker. + var sctx sessionctx.Context + sctx, err := w.sessPool.Get() + if err != nil { + return errors.Trace(err) + } + defer w.sessPool.Put(sctx) + + // If there is any row can't pass the check expression, the add constraint action will error. + // It's no need to construct expression node out and pull the chunk rows through it. Here we + // can let the check expression restored string as the filter in where clause directly. + // Prepare internal SQL to fetch data from physical table under this filter. + sql := fmt.Sprintf("select 1 from `%s`.`%s` where not %s limit 1", dbInfo.Name.L, tableInfo.Name.L, constr.ExprString) + ctx := kv.WithInternalSourceType(context.Background(), kv.InternalTxnDDL) + rows, _, err := sctx.(sqlexec.RestrictedSQLExecutor).ExecRestrictedSQL(ctx, nil, sql) + if err != nil { + return errors.Trace(err) + } + rowCount := len(rows) + if rowCount != 0 { + return dbterror.ErrCheckConstraintIsViolated.GenWithStackByArgs(constr.Name.L) + } + return nil +} + +func setNameForConstraintInfo(tableLowerName string, namesMap map[string]bool, infos []*model.ConstraintInfo) { + cnt := 1 + constraintPrefix := tableLowerName + "_chk_" + for _, constrInfo := range infos { + if constrInfo.Name.O == "" { + constrName := fmt.Sprintf("%s%d", constraintPrefix, cnt) + for { + // loop until find constrName that haven't been used. + if !namesMap[constrName] { + namesMap[constrName] = true + break + } + cnt++ + constrName = fmt.Sprintf("%s%d", constraintPrefix, cnt) + } + constrInfo.Name = model.NewCIStr(constrName) + } + } +} + +// IsColumnDroppableWithCheckConstraint check whether the column in check-constraint whose dependent col is more than 1 +func IsColumnDroppableWithCheckConstraint(col model.CIStr, tblInfo *model.TableInfo) error { + for _, cons := range tblInfo.Constraints { + if len(cons.ConstraintCols) > 1 { + for _, colName := range cons.ConstraintCols { + if colName.L == col.L { + return dbterror.ErrCantDropColWithCheckConstraint.GenWithStackByArgs(cons.Name, col) + } + } + } + } + return nil +} + +// IsColumnRenameableWithCheckConstraint check whether the column is referenced in check-constraint +func IsColumnRenameableWithCheckConstraint(col model.CIStr, tblInfo *model.TableInfo) error { + for _, cons := range tblInfo.Constraints { + for _, colName := range cons.ConstraintCols { + if colName.L == col.L { + return dbterror.ErrCantDropColWithCheckConstraint.GenWithStackByArgs(cons.Name, col) + } + } + } + return nil +} diff --git a/pkg/ddl/rollingback.go b/pkg/ddl/rollingback.go new file mode 100644 index 0000000000000..44ee0f2d0b452 --- /dev/null +++ b/pkg/ddl/rollingback.go @@ -0,0 +1,617 @@ +// Copyright 2018 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package ddl + +import ( + "fmt" + + "github.com/pingcap/errors" + "github.com/pingcap/failpoint" + "github.com/pingcap/tidb/pkg/ddl/ingest" + "github.com/pingcap/tidb/pkg/meta" + "github.com/pingcap/tidb/pkg/parser/ast" + "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/mysql" + "github.com/pingcap/tidb/pkg/parser/terror" + "github.com/pingcap/tidb/pkg/sessionctx/variable" + "github.com/pingcap/tidb/pkg/util/dbterror" + "go.uber.org/zap" +) + +// UpdateColsNull2NotNull changes the null option of columns of an index. +func UpdateColsNull2NotNull(tblInfo *model.TableInfo, indexInfo *model.IndexInfo) error { + nullCols, err := getNullColInfos(tblInfo, indexInfo) + if err != nil { + return errors.Trace(err) + } + + for _, col := range nullCols { + col.AddFlag(mysql.NotNullFlag) + col.DelFlag(mysql.PreventNullInsertFlag) + } + return nil +} + +func convertAddIdxJob2RollbackJob( + d *ddlCtx, + t *meta.Meta, + job *model.Job, + tblInfo *model.TableInfo, + allIndexInfos []*model.IndexInfo, + err error, +) (int64, error) { + failpoint.Inject("mockConvertAddIdxJob2RollbackJobError", func(val failpoint.Value) { + if val.(bool) { + failpoint.Return(0, errors.New("mock convert add index job to rollback job error")) + } + }) + + originalState := allIndexInfos[0].State + idxNames := make([]model.CIStr, 0, len(allIndexInfos)) + ifExists := make([]bool, 0, len(allIndexInfos)) + for _, indexInfo := range allIndexInfos { + if indexInfo.Primary { + nullCols, err := getNullColInfos(tblInfo, indexInfo) + if err != nil { + return 0, errors.Trace(err) + } + for _, col := range nullCols { + // Field PreventNullInsertFlag flag reset. + col.DelFlag(mysql.PreventNullInsertFlag) + } + } + // If add index job rollbacks in write reorganization state, its need to delete all keys which has been added. + // Its work is the same as drop index job do. + // The write reorganization state in add index job that likes write only state in drop index job. + // So the next state is delete only state. + indexInfo.State = model.StateDeleteOnly + idxNames = append(idxNames, indexInfo.Name) + ifExists = append(ifExists, false) + } + + // the second and the third args will be used in onDropIndex. + job.Args = []interface{}{idxNames, ifExists, getPartitionIDs(tblInfo)} + job.SchemaState = model.StateDeleteOnly + ver, err1 := updateVersionAndTableInfo(d, t, job, tblInfo, originalState != model.StateDeleteOnly) + if err1 != nil { + return ver, errors.Trace(err1) + } + job.State = model.JobStateRollingback + // TODO(tangenta): get duplicate column and match index. + err = completeErr(err, allIndexInfos[0]) + if ingest.LitBackCtxMgr != nil { + ingest.LitBackCtxMgr.Unregister(job.ID) + } + return ver, errors.Trace(err) +} + +// convertNotReorgAddIdxJob2RollbackJob converts the add index job that are not started workers to rollingbackJob, +// to rollback add index operations. job.SnapshotVer == 0 indicates the workers are not started. +func convertNotReorgAddIdxJob2RollbackJob(d *ddlCtx, t *meta.Meta, job *model.Job, occuredErr error) (ver int64, err error) { + defer func() { + if ingest.LitBackCtxMgr != nil { + ingest.LitBackCtxMgr.Unregister(job.ID) + } + }() + schemaID := job.SchemaID + tblInfo, err := GetTableInfoAndCancelFaultJob(t, job, schemaID) + if err != nil { + return ver, errors.Trace(err) + } + + unique := make([]bool, 1) + indexName := make([]model.CIStr, 1) + indexPartSpecifications := make([][]*ast.IndexPartSpecification, 1) + indexOption := make([]*ast.IndexOption, 1) + + err = job.DecodeArgs(&unique[0], &indexName[0], &indexPartSpecifications[0], &indexOption[0]) + if err != nil { + err = job.DecodeArgs(&unique, &indexName, &indexPartSpecifications, &indexOption) + } + if err != nil { + job.State = model.JobStateCancelled + return ver, errors.Trace(err) + } + + var indexesInfo []*model.IndexInfo + for _, idxName := range indexName { + indexInfo := tblInfo.FindIndexByName(idxName.L) + if indexInfo != nil { + indexesInfo = append(indexesInfo, indexInfo) + } + } + if len(indexesInfo) == 0 { + job.State = model.JobStateCancelled + return ver, dbterror.ErrCancelledDDLJob + } + return convertAddIdxJob2RollbackJob(d, t, job, tblInfo, indexesInfo, occuredErr) +} + +// rollingbackModifyColumn change the modifying-column job into rolling back state. +// Since modifying column job has two types: normal-type and reorg-type, we should handle it respectively. +// normal-type has only two states: None -> Public +// reorg-type has five states: None -> Delete-only -> Write-only -> Write-org -> Public +func rollingbackModifyColumn(w *worker, d *ddlCtx, t *meta.Meta, job *model.Job) (ver int64, err error) { + if needNotifyAndStopReorgWorker(job) { + // column type change workers are started. we have to ask them to exit. + w.jobLogger(job).Info("run the cancelling DDL job", zap.String("job", job.String())) + d.notifyReorgWorkerJobStateChange(job) + // Give the this kind of ddl one more round to run, the dbterror.ErrCancelledDDLJob should be fetched from the bottom up. + return w.onModifyColumn(d, t, job) + } + _, tblInfo, oldCol, jp, err := getModifyColumnInfo(t, job) + if err != nil { + return ver, err + } + if !needChangeColumnData(oldCol, jp.newCol) { + // Normal-type rolling back + if job.SchemaState == model.StateNone { + // When change null to not null, although state is unchanged with none, the oldCol flag's has been changed to preNullInsertFlag. + // To roll back this kind of normal job, it is necessary to mark the state as JobStateRollingback to restore the old col's flag. + if jp.modifyColumnTp == mysql.TypeNull && tblInfo.Columns[oldCol.Offset].GetFlag()|mysql.PreventNullInsertFlag != 0 { + job.State = model.JobStateRollingback + return ver, dbterror.ErrCancelledDDLJob + } + // Normal job with stateNone can be cancelled directly. + job.State = model.JobStateCancelled + return ver, dbterror.ErrCancelledDDLJob + } + // StatePublic couldn't be cancelled. + job.State = model.JobStateRunning + return ver, nil + } + // reorg-type rolling back + if jp.changingCol == nil { + // The job hasn't been handled and we cancel it directly. + job.State = model.JobStateCancelled + return ver, dbterror.ErrCancelledDDLJob + } + // The job has been in its middle state (but the reorg worker hasn't started) and we roll it back here. + job.State = model.JobStateRollingback + return ver, dbterror.ErrCancelledDDLJob +} + +func rollingbackAddColumn(d *ddlCtx, t *meta.Meta, job *model.Job) (ver int64, err error) { + tblInfo, columnInfo, col, _, _, err := checkAddColumn(t, job) + if err != nil { + return ver, errors.Trace(err) + } + if columnInfo == nil { + job.State = model.JobStateCancelled + return ver, dbterror.ErrCancelledDDLJob + } + + originalState := columnInfo.State + columnInfo.State = model.StateDeleteOnly + job.SchemaState = model.StateDeleteOnly + + job.Args = []interface{}{col.Name} + ver, err = updateVersionAndTableInfo(d, t, job, tblInfo, originalState != columnInfo.State) + if err != nil { + return ver, errors.Trace(err) + } + + job.State = model.JobStateRollingback + return ver, dbterror.ErrCancelledDDLJob +} + +func rollingbackDropColumn(d *ddlCtx, t *meta.Meta, job *model.Job) (ver int64, err error) { + _, colInfo, idxInfos, _, err := checkDropColumn(d, t, job) + if err != nil { + return ver, errors.Trace(err) + } + + for _, indexInfo := range idxInfos { + switch indexInfo.State { + case model.StateWriteOnly, model.StateDeleteOnly, model.StateDeleteReorganization, model.StateNone: + // We can not rollback now, so just continue to drop index. + // In function isJobRollbackable will let job rollback when state is StateNone. + // When there is no index related to the drop column job it is OK, but when there has indices, we should + // make sure the job is not rollback. + job.State = model.JobStateRunning + return ver, nil + case model.StatePublic: + default: + return ver, dbterror.ErrInvalidDDLState.GenWithStackByArgs("index", indexInfo.State) + } + } + + // StatePublic means when the job is not running yet. + if colInfo.State == model.StatePublic { + job.State = model.JobStateCancelled + return ver, dbterror.ErrCancelledDDLJob + } + // In the state of drop column `write only -> delete only -> reorganization`, + // We can not rollback now, so just continue to drop column. + job.State = model.JobStateRunning + return ver, nil +} + +func rollingbackDropIndex(d *ddlCtx, t *meta.Meta, job *model.Job) (ver int64, err error) { + _, indexInfo, _, err := checkDropIndex(d, t, job) + if err != nil { + return ver, errors.Trace(err) + } + + switch indexInfo[0].State { + case model.StateWriteOnly, model.StateDeleteOnly, model.StateDeleteReorganization, model.StateNone: + // We can not rollback now, so just continue to drop index. + // Normally won't fetch here, because there is check when cancel ddl jobs. see function: isJobRollbackable. + job.State = model.JobStateRunning + return ver, nil + case model.StatePublic: + job.State = model.JobStateCancelled + return ver, dbterror.ErrCancelledDDLJob + default: + return ver, dbterror.ErrInvalidDDLState.GenWithStackByArgs("index", indexInfo[0].State) + } +} + +func rollingbackAddIndex(w *worker, d *ddlCtx, t *meta.Meta, job *model.Job, isPK bool) (ver int64, err error) { + if needNotifyAndStopReorgWorker(job) { + // add index workers are started. need to ask them to exit. + w.jobLogger(job).Info("run the cancelling DDL job", zap.String("job", job.String())) + d.notifyReorgWorkerJobStateChange(job) + ver, err = w.onCreateIndex(d, t, job, isPK) + } else { + // add index's reorg workers are not running, remove the indexInfo in tableInfo. + ver, err = convertNotReorgAddIdxJob2RollbackJob(d, t, job, dbterror.ErrCancelledDDLJob) + } + return +} + +func needNotifyAndStopReorgWorker(job *model.Job) bool { + if job.SchemaState == model.StateWriteReorganization && job.SnapshotVer != 0 { + // If the value of SnapshotVer isn't zero, it means the reorg workers have been started. + if job.MultiSchemaInfo != nil { + // However, if the sub-job is non-revertible, it means the reorg process is finished. + // We don't need to start another round to notify reorg workers to exit. + return job.MultiSchemaInfo.Revertible + } + return true + } + return false +} + +// rollbackExchangeTablePartition will clear the non-partitioned +// table's ExchangePartitionInfo state. +func rollbackExchangeTablePartition(d *ddlCtx, t *meta.Meta, job *model.Job, tblInfo *model.TableInfo) (ver int64, err error) { + tblInfo.ExchangePartitionInfo = nil + job.State = model.JobStateRollbackDone + job.SchemaState = model.StatePublic + if len(tblInfo.Constraints) == 0 { + return updateVersionAndTableInfo(d, t, job, tblInfo, true) + } + var ( + defID int64 + ptSchemaID int64 + ptID int64 + partName string + withValidation bool + ) + if err = job.DecodeArgs(&defID, &ptSchemaID, &ptID, &partName, &withValidation); err != nil { + return ver, errors.Trace(err) + } + pt, err := getTableInfo(t, ptID, ptSchemaID) + if err != nil { + return ver, errors.Trace(err) + } + pt.ExchangePartitionInfo = nil + var ptInfo []schemaIDAndTableInfo + ptInfo = append(ptInfo, schemaIDAndTableInfo{ + schemaID: ptSchemaID, + tblInfo: pt, + }) + ver, err = updateVersionAndTableInfo(d, t, job, tblInfo, true, ptInfo...) + return ver, errors.Trace(err) +} + +func rollingbackExchangeTablePartition(d *ddlCtx, t *meta.Meta, job *model.Job) (ver int64, err error) { + if job.SchemaState == model.StateNone { + // Nothing is changed + job.State = model.JobStateCancelled + return ver, dbterror.ErrCancelledDDLJob + } + var nt *model.TableInfo + nt, err = GetTableInfoAndCancelFaultJob(t, job, job.SchemaID) + if err != nil { + return ver, errors.Trace(err) + } + ver, err = rollbackExchangeTablePartition(d, t, job, nt) + return ver, errors.Trace(err) +} + +func convertAddTablePartitionJob2RollbackJob(d *ddlCtx, t *meta.Meta, job *model.Job, otherwiseErr error, tblInfo *model.TableInfo) (ver int64, err error) { + addingDefinitions := tblInfo.Partition.AddingDefinitions + partNames := make([]string, 0, len(addingDefinitions)) + for _, pd := range addingDefinitions { + partNames = append(partNames, pd.Name.L) + } + job.Args = []interface{}{partNames} + ver, err = updateVersionAndTableInfo(d, t, job, tblInfo, true) + if err != nil { + return ver, errors.Trace(err) + } + job.State = model.JobStateRollingback + return ver, errors.Trace(otherwiseErr) +} + +func rollingbackAddTablePartition(d *ddlCtx, t *meta.Meta, job *model.Job) (ver int64, err error) { + tblInfo, _, addingDefinitions, err := checkAddPartition(t, job) + if err != nil { + return ver, errors.Trace(err) + } + // addingDefinitions' len = 0 means the job hasn't reached the replica-only state. + if len(addingDefinitions) == 0 { + job.State = model.JobStateCancelled + return ver, errors.Trace(dbterror.ErrCancelledDDLJob) + } + // addingDefinitions is also in tblInfo, here pass the tblInfo as parameter directly. + return convertAddTablePartitionJob2RollbackJob(d, t, job, dbterror.ErrCancelledDDLJob, tblInfo) +} + +func rollingbackDropTableOrView(t *meta.Meta, job *model.Job) error { + tblInfo, err := checkTableExistAndCancelNonExistJob(t, job, job.SchemaID) + if err != nil { + return errors.Trace(err) + } + // To simplify the rollback logic, cannot be canceled after job start to run. + // Normally won't fetch here, because there is check when cancel ddl jobs. see function: isJobRollbackable. + if tblInfo.State == model.StatePublic { + job.State = model.JobStateCancelled + return dbterror.ErrCancelledDDLJob + } + job.State = model.JobStateRunning + return nil +} + +func rollingbackDropTablePartition(t *meta.Meta, job *model.Job) (ver int64, err error) { + _, err = GetTableInfoAndCancelFaultJob(t, job, job.SchemaID) + if err != nil { + return ver, errors.Trace(err) + } + return cancelOnlyNotHandledJob(job, model.StatePublic) +} + +func rollingbackDropSchema(t *meta.Meta, job *model.Job) error { + dbInfo, err := checkSchemaExistAndCancelNotExistJob(t, job) + if err != nil { + return errors.Trace(err) + } + // To simplify the rollback logic, cannot be canceled after job start to run. + // Normally won't fetch here, because there is check when cancel ddl jobs. see function: isJobRollbackable. + if dbInfo.State == model.StatePublic { + job.State = model.JobStateCancelled + return dbterror.ErrCancelledDDLJob + } + job.State = model.JobStateRunning + return nil +} + +func rollingbackRenameIndex(t *meta.Meta, job *model.Job) (ver int64, err error) { + tblInfo, from, _, err := checkRenameIndex(t, job) + if err != nil { + return ver, errors.Trace(err) + } + // Here rename index is done in a transaction, if the job is not completed, it can be canceled. + idx := tblInfo.FindIndexByName(from.L) + if idx.State == model.StatePublic { + job.State = model.JobStateCancelled + return ver, dbterror.ErrCancelledDDLJob + } + job.State = model.JobStateRunning + return ver, errors.Trace(err) +} + +func cancelOnlyNotHandledJob(job *model.Job, initialState model.SchemaState) (ver int64, err error) { + // We can only cancel the not handled job. + if job.SchemaState == initialState { + job.State = model.JobStateCancelled + return ver, dbterror.ErrCancelledDDLJob + } + + job.State = model.JobStateRunning + + return ver, nil +} + +func rollingbackTruncateTable(t *meta.Meta, job *model.Job) (ver int64, err error) { + _, err = GetTableInfoAndCancelFaultJob(t, job, job.SchemaID) + if err != nil { + return ver, errors.Trace(err) + } + return cancelOnlyNotHandledJob(job, model.StateNone) +} + +func rollingbackReorganizePartition(d *ddlCtx, t *meta.Meta, job *model.Job) (ver int64, err error) { + if job.SchemaState == model.StateNone { + job.State = model.JobStateCancelled + return ver, dbterror.ErrCancelledDDLJob + } + + tblInfo, err := GetTableInfoAndCancelFaultJob(t, job, job.SchemaID) + if err != nil { + return ver, errors.Trace(err) + } + + // addingDefinitions is also in tblInfo, here pass the tblInfo as parameter directly. + // TODO: Test this with reorganize partition p1 into (partition p1 ...)! + return convertAddTablePartitionJob2RollbackJob(d, t, job, dbterror.ErrCancelledDDLJob, tblInfo) +} + +func pauseReorgWorkers(w *worker, d *ddlCtx, job *model.Job) (err error) { + if needNotifyAndStopReorgWorker(job) { + w.jobLogger(job).Info("pausing the DDL job", zap.String("job", job.String())) + d.notifyReorgWorkerJobStateChange(job) + } + + return dbterror.ErrPausedDDLJob.GenWithStackByArgs(job.ID) +} + +func convertJob2RollbackJob(w *worker, d *ddlCtx, t *meta.Meta, job *model.Job) (ver int64, err error) { + switch job.Type { + case model.ActionAddColumn: + ver, err = rollingbackAddColumn(d, t, job) + case model.ActionAddIndex: + ver, err = rollingbackAddIndex(w, d, t, job, false) + case model.ActionAddPrimaryKey: + ver, err = rollingbackAddIndex(w, d, t, job, true) + case model.ActionAddTablePartition: + ver, err = rollingbackAddTablePartition(d, t, job) + case model.ActionReorganizePartition, model.ActionRemovePartitioning, + model.ActionAlterTablePartitioning: + ver, err = rollingbackReorganizePartition(d, t, job) + case model.ActionDropColumn: + ver, err = rollingbackDropColumn(d, t, job) + case model.ActionDropIndex, model.ActionDropPrimaryKey: + ver, err = rollingbackDropIndex(d, t, job) + case model.ActionDropTable, model.ActionDropView, model.ActionDropSequence: + err = rollingbackDropTableOrView(t, job) + case model.ActionDropTablePartition: + ver, err = rollingbackDropTablePartition(t, job) + case model.ActionExchangeTablePartition: + ver, err = rollingbackExchangeTablePartition(d, t, job) + case model.ActionDropSchema: + err = rollingbackDropSchema(t, job) + case model.ActionRenameIndex: + ver, err = rollingbackRenameIndex(t, job) + case model.ActionTruncateTable: + ver, err = rollingbackTruncateTable(t, job) + case model.ActionModifyColumn: + ver, err = rollingbackModifyColumn(w, d, t, job) + case model.ActionDropForeignKey, model.ActionTruncateTablePartition: + ver, err = cancelOnlyNotHandledJob(job, model.StatePublic) + case model.ActionRebaseAutoID, model.ActionShardRowID, model.ActionAddForeignKey, + model.ActionRenameTable, model.ActionRenameTables, + model.ActionModifyTableCharsetAndCollate, + model.ActionModifySchemaCharsetAndCollate, model.ActionRepairTable, + model.ActionModifyTableAutoIdCache, model.ActionAlterIndexVisibility, + 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 + } + + logger := w.jobLogger(job) + if err != nil { + if job.Error == nil { + job.Error = toTError(err) + } + job.ErrorCount++ + + if dbterror.ErrCancelledDDLJob.Equal(err) { + // The job is normally cancelled. + if !job.Error.Equal(dbterror.ErrCancelledDDLJob) { + job.Error = terror.GetErrClass(job.Error).Synthesize(terror.ErrCode(job.Error.Code()), + fmt.Sprintf("DDL job rollback, error msg: %s", terror.ToSQLError(job.Error).Message)) + } + } else { + // A job canceling meet other error. + // + // Once `convertJob2RollbackJob` meets an error, the job state can't be set as `JobStateRollingback` since + // job state and args may not be correctly overwritten. The job will be fetched to run with the cancelling + // state again. So we should check the error count here. + if err1 := loadDDLVars(w); err1 != nil { + logger.Error("load DDL global variable failed", zap.Error(err1)) + } + errorCount := variable.GetDDLErrorCountLimit() + if job.ErrorCount > errorCount { + logger.Warn("rollback DDL job error count exceed the limit, cancelled it now", zap.Int64("errorCountLimit", errorCount)) + job.Error = toTError(errors.Errorf("rollback DDL job error count exceed the limit %d, cancelled it now", errorCount)) + job.State = model.JobStateCancelled + } + } + + if !(job.State != model.JobStateRollingback && job.State != model.JobStateCancelled) { + logger.Info("the DDL job is cancelled normally", zap.String("job", job.String()), zap.Error(err)) + // If job is cancelled, we shouldn't return an error. + return ver, nil + } + logger.Error("run DDL job failed", zap.String("job", job.String()), zap.Error(err)) + } + + return +} + +func rollingBackAddConstraint(d *ddlCtx, t *meta.Meta, job *model.Job) (ver int64, err error) { + _, tblInfo, constrInfoInMeta, _, err := checkAddCheckConstraint(t, job) + if err != nil { + return ver, errors.Trace(err) + } + if constrInfoInMeta == nil { + // Add constraint hasn't stored constraint info into meta, so we can cancel the job + // directly without further rollback action. + job.State = model.JobStateCancelled + 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) { + _, constrInfoInMeta, err := checkDropCheckConstraint(t, job) + if err != nil { + return ver, errors.Trace(err) + } + + // StatePublic means when the job is not running yet. + if constrInfoInMeta.State == model.StatePublic { + job.State = model.JobStateCancelled + return ver, dbterror.ErrCancelledDDLJob + } + // Can not rollback like drop other element, so just continue to drop constraint. + 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) +} diff --git a/tests/integrationtest/r/ddl/constraint.result b/tests/integrationtest/r/ddl/constraint.result new file mode 100644 index 0000000000000..06b0679415857 --- /dev/null +++ b/tests/integrationtest/r/ddl/constraint.result @@ -0,0 +1,822 @@ +set @@global.tidb_enable_check_constraint = 1; +drop table if exists t; +create table t(a int check(a > 0), b int, c int, check(c > 0), check (b > c)); +alter table t drop column a; +show create table t; +Table Create Table +t CREATE TABLE `t` ( + `b` int(11) DEFAULT NULL, + `c` int(11) DEFAULT NULL, +CONSTRAINT `t_chk_1` CHECK ((`c` > 0)), +CONSTRAINT `t_chk_2` CHECK ((`b` > `c`)) +) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin +alter table t drop column b; +Error 3959 (HY000): Check constraint 't_chk_2' uses column 'b', hence column cannot be dropped or renamed. +alter table t rename column c to c1; +Error 3959 (HY000): Check constraint 't_chk_1' uses column 'c', hence column cannot be dropped or renamed. +drop table if exists t; +create table t(a int check(a > 0), b int check(b < 10) not enforced, c int); +alter table t alter constraint t_chk_1 not enforced; +insert into t values(-1, 1, 0); +alter table t alter constraint t_chk_2 enforced; +insert into t values(-1, 11, 0); +Error 3819 (HY000): Check constraint 't_chk_2' is violated. +alter table t add check(c = 0); +insert into t values(-1, 1, 1); +Error 3819 (HY000): Check constraint 't_chk_3' is violated. +alter table t alter constraint t_chk_3 not enforced; +insert into t values(-1, 1, 0); +drop table if exists t; +CREATE TABLE t1 (f1 TIMESTAMP CHECK (f1 + NOW() > '2011-11-21')); +Error 3814 (HY000): An expression of a check constraint 't1_chk_1' contains disallowed function: now. +CREATE TABLE t1 (f1 TIMESTAMP CHECK (f1 + CURRENT_TIMESTAMP() > '2011-11-21 01:02:03')); +Error 3814 (HY000): An expression of a check constraint 't1_chk_1' contains disallowed function: current_timestamp. +CREATE TABLE t1 (f1 TIMESTAMP CHECK (f1 + CURRENT_TIMESTAMP > '2011-11-21 01:02:03')); +Error 3814 (HY000): An expression of a check constraint 't1_chk_1' contains disallowed function: current_timestamp. +CREATE TABLE t1 (f1 DATETIME CHECK (f1 + CURDATE() > '2011-11-21')); +Error 3814 (HY000): An expression of a check constraint 't1_chk_1' contains disallowed function: curdate. +CREATE TABLE t1 (f1 DATETIME CHECK (f1 + CURTIME() > '23:11:21')); +Error 3814 (HY000): An expression of a check constraint 't1_chk_1' contains disallowed function: curtime. +CREATE TABLE t1 (f1 TIMESTAMP CHECK (f1 + CURRENT_DATE() > '2011-11-21')); +Error 3814 (HY000): An expression of a check constraint 't1_chk_1' contains disallowed function: current_date. +CREATE TABLE t1 (f1 TIMESTAMP CHECK (f1 + CURRENT_DATE > '2011-11-21')); +Error 3814 (HY000): An expression of a check constraint 't1_chk_1' contains disallowed function: current_date. +CREATE TABLE t1 (f1 TIMESTAMP CHECK (f1 + CURRENT_TIME() > '01:02:03')); +Error 3814 (HY000): An expression of a check constraint 't1_chk_1' contains disallowed function: current_time. +CREATE TABLE t1 (f1 TIMESTAMP CHECK (f1 + CURRENT_TIME > '01:02:03')); +Error 3814 (HY000): An expression of a check constraint 't1_chk_1' contains disallowed function: current_time. +CREATE TABLE t1 (f1 DATETIME CHECK (f1 + LOCALTIME() > '23:11:21')); +Error 3814 (HY000): An expression of a check constraint 't1_chk_1' contains disallowed function: localtime. +CREATE TABLE t1 (f1 DATETIME CHECK (f1 + LOCALTIME > '23:11:21')); +Error 3814 (HY000): An expression of a check constraint 't1_chk_1' contains disallowed function: localtime. +CREATE TABLE t1 (f1 TIMESTAMP CHECK (f1 + LOCALTIMESTAMP() > '2011-11-21 01:02:03')); +Error 3814 (HY000): An expression of a check constraint 't1_chk_1' contains disallowed function: localtimestamp. +CREATE TABLE t1 (f1 TIMESTAMP CHECK (f1 + LOCALTIMESTAMP > '2011-11-21 01:02:03')); +Error 3814 (HY000): An expression of a check constraint 't1_chk_1' contains disallowed function: localtimestamp. +CREATE TABLE t1 (f1 TIMESTAMP CHECK (f1 + UNIX_TIMESTAMP() > '2011-11-21 01:02:03')); +Error 3814 (HY000): An expression of a check constraint 't1_chk_1' contains disallowed function: unix_timestamp. +CREATE TABLE t1 (f1 DATETIME CHECK (f1 + UTC_DATE() > '2011-11-21')); +Error 3814 (HY000): An expression of a check constraint 't1_chk_1' contains disallowed function: utc_date. +CREATE TABLE t1 (f1 TIMESTAMP CHECK (f1 + UTC_TIMESTAMP() > '2011-11-21 01:02:03')); +Error 3814 (HY000): An expression of a check constraint 't1_chk_1' contains disallowed function: utc_timestamp. +CREATE TABLE t1 (f1 DATETIME CHECK (f1 + UTC_TIME() > '23:11:21')); +Error 3814 (HY000): An expression of a check constraint 't1_chk_1' contains disallowed function: utc_time. +CREATE TABLE t1 (f1 INT CHECK (f1 + CONNECTION_ID() < 929)); +Error 3814 (HY000): An expression of a check constraint 't1_chk_1' contains disallowed function: connection_id. +CREATE TABLE t1 (a VARCHAR(32) CHECK (CURRENT_USER() != a)); +Error 3814 (HY000): An expression of a check constraint 't1_chk_1' contains disallowed function: current_user. +CREATE TABLE t1 (a VARCHAR(32) CHECK (CURRENT_USER != a)); +Error 3814 (HY000): An expression of a check constraint 't1_chk_1' contains disallowed function: current_user. +CREATE TABLE t1 (a VARCHAR(32) CHECK (SESSION_USER() != a)); +Error 3814 (HY000): An expression of a check constraint 't1_chk_1' contains disallowed function: session_user. +CREATE TABLE t1 (a VARCHAR(32) CHECK (VERSION() != a)); +Error 3814 (HY000): An expression of a check constraint 't1_chk_1' contains disallowed function: version. +CREATE TABLE t1 (a VARCHAR(1024), b INT CHECK (b + FOUND_ROWS() > 2000)); +Error 3814 (HY000): An expression of a check constraint 't1_chk_1' contains disallowed function: found_rows. +CREATE TABLE t1 (a INT CHECK ((a + LAST_INSERT_ID()) < 929)); +Error 3814 (HY000): An expression of a check constraint 't1_chk_1' contains disallowed function: last_insert_id. +CREATE TABLE t1 (a VARCHAR(32) CHECK (SYSTEM_USER() != a)); +Error 3814 (HY000): An expression of a check constraint 't1_chk_1' contains disallowed function: system_user. +CREATE TABLE t1 (a VARCHAR(32) CHECK (USER() != a)); +Error 3814 (HY000): An expression of a check constraint 't1_chk_1' contains disallowed function: user. +CREATE TABLE t1 (f1 FLOAT CHECK (f1 + RAND() < 929.929)); +Error 3814 (HY000): An expression of a check constraint 't1_chk_1' contains disallowed function: rand. +CREATE TABLE t1 (a INT CHECK (a + ROW_COUNT() > 1000)); +Error 3814 (HY000): An expression of a check constraint 't1_chk_1' contains disallowed function: row_count. +CREATE TABLE t1 (a VARCHAR(1024), b VARCHAR(1024) CHECK (GET_LOCK(b,10) != 0)); +Error 3814 (HY000): An expression of a check constraint 't1_chk_1' contains disallowed function: get_lock. +CREATE TABLE t1 (a VARCHAR(1024), b VARCHAR(1024) CHECK (IS_FREE_LOCK(b) != 0)); +Error 3814 (HY000): An expression of a check constraint 't1_chk_1' contains disallowed function: is_free_lock. +CREATE TABLE t1 (a VARCHAR(1024), b VARCHAR(1024) CHECK (IS_USED_LOCK(b) != 0)); +Error 3814 (HY000): An expression of a check constraint 't1_chk_1' contains disallowed function: is_used_lock. +CREATE TABLE t1 (a VARCHAR(1024), b VARCHAR(1024) CHECK (RELEASE_LOCK(b) != 0)); +Error 3814 (HY000): An expression of a check constraint 't1_chk_1' contains disallowed function: release_lock. +CREATE TABLE t1 (a VARCHAR(1024), b VARCHAR(1024), CHECK (RELEASE_ALL_LOCKS() != 0)); +Error 3814 (HY000): An expression of a check constraint 't1_chk_1' contains disallowed function: release_all_locks. +CREATE TABLE t1 (f1 VARCHAR(1024), f2 VARCHAR(1024) CHECK (LOAD_FILE(f2) != NULL)); +Error 3814 (HY000): An expression of a check constraint 't1_chk_1' contains disallowed function: load_file. +CREATE TABLE t1 (id CHAR(40) CHECK(UUID() != id)); +Error 3814 (HY000): An expression of a check constraint 't1_chk_1' contains disallowed function: uuid. +CREATE TABLE t1 (id INT CHECK(UUID_SHORT() != id)); +Error 3814 (HY000): An expression of a check constraint 't1_chk_1' contains disallowed function: uuid_short. +CREATE TABLE t1 (id INT CHECK(SLEEP(id) != 0)); +Error 3814 (HY000): An expression of a check constraint 't1_chk_1' contains disallowed function: sleep. +set @a = 1; +CREATE TABLE t1 (f1 int CHECK (f1 > @a)); +Error 3816 (HY000): An expression of a check constraint 't1_chk_1' cannot refer to a user or system variable. +CREATE TABLE t1 (f1 int CHECK (f1 > @@session.tidb_mem_quota_query)); +Error 3816 (HY000): An expression of a check constraint 't1_chk_1' cannot refer to a user or system variable. +CREATE TABLE t1 (f1 int CHECK (f1 > @@global.tidb_mem_quota_query)); +Error 3816 (HY000): An expression of a check constraint 't1_chk_1' cannot refer to a user or system variable. +CREATE TABLE t1 (f1 int primary key auto_increment, f2 int, CHECK (f1 != f2)); +Error 3818 (HY000): Check constraint 't1_chk_1' cannot refer to an auto-increment column. +CREATE TABLE t1 (f1 INT CHECK (f1 = default(f1))); +Error 3814 (HY000): An expression of a check constraint 't1_chk_1' contains disallowed function: default. +CREATE TABLE t1 (id INT CHECK (id != (SELECT 1))); +Error 3815 (HY000): An expression of a check constraint 't1_chk_1' contains disallowed function. +CREATE TABLE t1 (a int check(a in (SELECT COALESCE(NULL, 1, 1)))); +Error 3815 (HY000): An expression of a check constraint 't1_chk_1' contains disallowed function. +drop table if exists t; +create table t1(f1 TIMESTAMP, f2 DATETIME, f3 INT, f4 VARCHAR(32), f5 FLOAT, f6 CHAR(40), f7 INT PRIMARY KEY AUTO_INCREMENT); +ALTER TABLE t1 ADD CHECK (f1 + NOW() > '2011-11-21'); +Error 3814 (HY000): An expression of a check constraint 't1_chk_1' contains disallowed function: now. +ALTER TABLE t1 ADD CHECK (f1 + CURRENT_TIMESTAMP() > '2011-11-21 01:02:03'); +Error 3814 (HY000): An expression of a check constraint 't1_chk_1' contains disallowed function: current_timestamp. +ALTER TABLE t1 ADD CHECK (f1 + CURRENT_TIMESTAMP > '2011-11-21 01:02:03'); +Error 3814 (HY000): An expression of a check constraint 't1_chk_1' contains disallowed function: current_timestamp. +ALTER TABLE t1 ADD CHECK (f2 + CURDATE() > '2011-11-21'); +Error 3814 (HY000): An expression of a check constraint 't1_chk_1' contains disallowed function: curdate. +ALTER TABLE t1 ADD CHECK (f2 + CURTIME() > '23:11:21'); +Error 3814 (HY000): An expression of a check constraint 't1_chk_1' contains disallowed function: curtime. +ALTER TABLE t1 ADD CHECK (f1 + CURRENT_DATE() > '2011-11-21'); +Error 3814 (HY000): An expression of a check constraint 't1_chk_1' contains disallowed function: current_date. +ALTER TABLE t1 ADD CHECK (f1 + CURRENT_DATE > '2011-11-21'); +Error 3814 (HY000): An expression of a check constraint 't1_chk_1' contains disallowed function: current_date. +ALTER TABLE t1 ADD CHECK (f1 + CURRENT_TIME() > '01:02:03'); +Error 3814 (HY000): An expression of a check constraint 't1_chk_1' contains disallowed function: current_time. +ALTER TABLE t1 ADD CHECK (f1 + CURRENT_TIME > '01:02:03'); +Error 3814 (HY000): An expression of a check constraint 't1_chk_1' contains disallowed function: current_time. +ALTER TABLE t1 ADD CHECK (f2 + LOCALTIME() > '23:11:21'); +Error 3814 (HY000): An expression of a check constraint 't1_chk_1' contains disallowed function: localtime. +ALTER TABLE t1 ADD CHECK (f2 + LOCALTIME > '23:11:21'); +Error 3814 (HY000): An expression of a check constraint 't1_chk_1' contains disallowed function: localtime. +ALTER TABLE t1 ADD CHECK (f1 + LOCALTIMESTAMP() > '2011-11-21 01:02:03'); +Error 3814 (HY000): An expression of a check constraint 't1_chk_1' contains disallowed function: localtimestamp. +ALTER TABLE t1 ADD CHECK (f1 + LOCALTIMESTAMP > '2011-11-21 01:02:03'); +Error 3814 (HY000): An expression of a check constraint 't1_chk_1' contains disallowed function: localtimestamp. +ALTER TABLE t1 ADD CHECK (f1 + UNIX_TIMESTAMP() > '2011-11-21 01:02:03'); +Error 3814 (HY000): An expression of a check constraint 't1_chk_1' contains disallowed function: unix_timestamp. +ALTER TABLE t1 ADD CHECK (f2 + UTC_DATE() > '2011-11-21'); +Error 3814 (HY000): An expression of a check constraint 't1_chk_1' contains disallowed function: utc_date. +ALTER TABLE t1 ADD CHECK (f1 + UTC_TIMESTAMP() > '2011-11-21 01:02:03'); +Error 3814 (HY000): An expression of a check constraint 't1_chk_1' contains disallowed function: utc_timestamp. +ALTER TABLE t1 ADD CHECK (f2 + UTC_TIME() > '23:11:21'); +Error 3814 (HY000): An expression of a check constraint 't1_chk_1' contains disallowed function: utc_time. +ALTER TABLE t1 ADD CHECK (f3 + CONNECTION_ID() < 929); +Error 3814 (HY000): An expression of a check constraint 't1_chk_1' contains disallowed function: connection_id. +ALTER TABLE t1 ADD CHECK (CURRENT_USER() != f4); +Error 3814 (HY000): An expression of a check constraint 't1_chk_1' contains disallowed function: current_user. +ALTER TABLE t1 ADD CHECK (CURRENT_USER != f4); +Error 3814 (HY000): An expression of a check constraint 't1_chk_1' contains disallowed function: current_user. +ALTER TABLE t1 ADD CHECK (SESSION_USER() != f4); +Error 3814 (HY000): An expression of a check constraint 't1_chk_1' contains disallowed function: session_user. +ALTER TABLE t1 ADD CHECK (VERSION() != f4); +Error 3814 (HY000): An expression of a check constraint 't1_chk_1' contains disallowed function: version. +ALTER TABLE t1 ADD CHECK (f3 + FOUND_ROWS() > 2000); +Error 3814 (HY000): An expression of a check constraint 't1_chk_1' contains disallowed function: found_rows. +ALTER TABLE t1 ADD CHECK ((f3 + LAST_INSERT_ID()) < 929); +Error 3814 (HY000): An expression of a check constraint 't1_chk_1' contains disallowed function: last_insert_id. +ALTER TABLE t1 ADD CHECK (SYSTEM_USER() != f4); +Error 3814 (HY000): An expression of a check constraint 't1_chk_1' contains disallowed function: system_user. +ALTER TABLE t1 ADD CHECK (USER() != f4); +Error 3814 (HY000): An expression of a check constraint 't1_chk_1' contains disallowed function: user. +ALTER TABLE t1 ADD CHECK (f5 + RAND() < 929.929); +Error 3814 (HY000): An expression of a check constraint 't1_chk_1' contains disallowed function: rand. +ALTER TABLE t1 ADD CHECK (f3 + ROW_COUNT() > 1000); +Error 3814 (HY000): An expression of a check constraint 't1_chk_1' contains disallowed function: row_count. +ALTER TABLE t1 ADD CHECK (GET_LOCK(f4,10) != 0); +Error 3814 (HY000): An expression of a check constraint 't1_chk_1' contains disallowed function: get_lock. +ALTER TABLE t1 ADD CHECK (IS_FREE_LOCK(f4) != 0); +Error 3814 (HY000): An expression of a check constraint 't1_chk_1' contains disallowed function: is_free_lock. +ALTER TABLE t1 ADD CHECK (IS_USED_LOCK(f4) != 0); +Error 3814 (HY000): An expression of a check constraint 't1_chk_1' contains disallowed function: is_used_lock. +ALTER TABLE t1 ADD CHECK (RELEASE_LOCK(f4) != 0); +Error 3814 (HY000): An expression of a check constraint 't1_chk_1' contains disallowed function: release_lock. +ALTER TABLE t1 ADD CHECK (RELEASE_ALL_LOCKS() != 0); +Error 3814 (HY000): An expression of a check constraint 't1_chk_1' contains disallowed function: release_all_locks. +ALTER TABLE t1 ADD CHECK (LOAD_FILE(f4) != NULL); +Error 3814 (HY000): An expression of a check constraint 't1_chk_1' contains disallowed function: load_file. +ALTER TABLE t1 ADD CHECK(UUID() != f6); +Error 3814 (HY000): An expression of a check constraint 't1_chk_1' contains disallowed function: uuid. +ALTER TABLE t1 ADD CHECK(UUID_SHORT() != f3); +Error 3814 (HY000): An expression of a check constraint 't1_chk_1' contains disallowed function: uuid_short. +ALTER TABLE t1 ADD CHECK(SLEEP(f3) != 0); +Error 3814 (HY000): An expression of a check constraint 't1_chk_1' contains disallowed function: sleep. +set @a = 1; +ALTER TABLE t1 ADD CHECK (f3 > @a); +Error 3816 (HY000): An expression of a check constraint 't1_chk_1' cannot refer to a user or system variable. +ALTER TABLE t1 ADD CHECK (f3 > @@session.tidb_mem_quota_query); +Error 3816 (HY000): An expression of a check constraint 't1_chk_1' cannot refer to a user or system variable. +ALTER TABLE t1 ADD CHECK (f3 > @@global.tidb_mem_quota_query); +Error 3816 (HY000): An expression of a check constraint 't1_chk_1' cannot refer to a user or system variable. +ALTER TABLE t1 ADD CHECK (f7 != f3); +Error 3818 (HY000): Check constraint 't1_chk_1' cannot refer to an auto-increment column. +ALTER TABLE t1 ADD CHECK (f1 = default(f1)); +Error 3814 (HY000): An expression of a check constraint 't1_chk_1' contains disallowed function: default. +ALTER TABLE t1 ADD CHECK (f3 != (SELECT 1)); +Error 3815 (HY000): An expression of a check constraint 't1_chk_1' contains disallowed function. +ALTER TABLE t1 ADD check (f3 in (SELECT COALESCE(NULL, 1, 1))); +Error 3815 (HY000): An expression of a check constraint 't1_chk_1' contains disallowed function. +drop table if exists t, s; +create table t(a int check(a > 10), b int constraint bbb check(b > 5), c int, check(c < 0)); +create table s like t; +show create table s; +Table Create Table +s CREATE TABLE `s` ( + `a` int(11) DEFAULT NULL, + `b` int(11) DEFAULT NULL, + `c` int(11) DEFAULT NULL, +CONSTRAINT `s_chk_1` CHECK ((`c` < 0)), +CONSTRAINT `s_chk_2` CHECK ((`a` > 10)), +CONSTRAINT `s_chk_3` CHECK ((`b` > 5)) +) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin +insert into s(a) values(1); +Error 3819 (HY000): Check constraint 's_chk_2' is violated. +insert into s(b) values(2); +Error 3819 (HY000): Check constraint 's_chk_3' is violated. +insert into s(c) values(3); +Error 3819 (HY000): Check constraint 's_chk_1' is violated. +alter table s add check(a > 0); +alter table s add constraint aaa check(a > 0); +show create table s; +Table Create Table +s CREATE TABLE `s` ( + `a` int(11) DEFAULT NULL, + `b` int(11) DEFAULT NULL, + `c` int(11) DEFAULT NULL, +CONSTRAINT `s_chk_1` CHECK ((`c` < 0)), +CONSTRAINT `s_chk_2` CHECK ((`a` > 10)), +CONSTRAINT `s_chk_3` CHECK ((`b` > 5)), +CONSTRAINT `s_chk_4` CHECK ((`a` > 0)), +CONSTRAINT `aaa` CHECK ((`a` > 0)) +) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin +drop table if exists t; +create table t(a int check(a > 10)); +insert into t values(1),(11),(15); +Error 3819 (HY000): Check constraint 't_chk_1' is violated. +insert ignore into t values(1),(11),(15); +show warnings; +Level Code Message +Warning 3819 Check constraint 't_chk_1' is violated. +select a from t; +a +11 +15 +update ignore t set a = a-2; +show warnings; +Level Code Message +Warning 3819 Check constraint 't_chk_1' is violated. +select a from t; +a +11 +13 +drop table if exists t, s, t1, t2; +create table t(a int, b int, index(a), index(a, b)); +create table s(a int, check (a > 0), foreign key (a) references t(a) on update cascade); +Error 3823 (HY000): Column 'a' cannot be used in a check constraint 's_chk_1': needed in a foreign key constraint referential action. +create table s(a int, b int, check (a > 0), foreign key (a, b) references t(a, b) on update cascade); +Error 3823 (HY000): Column 'a' cannot be used in a check constraint 's_chk_1': needed in a foreign key constraint referential action. +create table t1(a int, foreign key (a) references t(a) on update cascade); +alter table t1 add check ( a > 0 ); +Error 3823 (HY000): Column 'a' cannot be used in a check constraint 't1_chk_1': needed in a foreign key constraint referential action. +create table t2(a int, b int, foreign key (a, b) references t(a, b) on update cascade); +alter table t2 add check ( a > 0 ); +Error 3823 (HY000): Column 'a' cannot be used in a check constraint 't2_chk_1': needed in a foreign key constraint referential action. +drop table t, t1, t2; +drop table if exists t; +create table t(a int, check(a)); +Error 3812 (HY000): An expression of non-boolean type specified to a check constraint 't_chk_1'. +create table t(a int, check(1)); +Error 3812 (HY000): An expression of non-boolean type specified to a check constraint 't_chk_1'. +create table t(a int, check('1')); +Error 3812 (HY000): An expression of non-boolean type specified to a check constraint 't_chk_1'. +create table t(a int check(a)); +Error 3812 (HY000): An expression of non-boolean type specified to a check constraint 't_chk_1'. +create table t(a int, check(1+1)); +Error 3812 (HY000): An expression of non-boolean type specified to a check constraint 't_chk_1'. +create table t(a int, check(1/2)); +Error 3812 (HY000): An expression of non-boolean type specified to a check constraint 't_chk_1'. +create table t(a int, check(a + 1/2)); +Error 3812 (HY000): An expression of non-boolean type specified to a check constraint 't_chk_1'. +create table t(a int check(a + 1/2)); +Error 3812 (HY000): An expression of non-boolean type specified to a check constraint 't_chk_1'. +create table t(a int, check(true + 1)); +Error 3812 (HY000): An expression of non-boolean type specified to a check constraint 't_chk_1'. +create table t(a int, check(abs(1))); +Error 3812 (HY000): An expression of non-boolean type specified to a check constraint 't_chk_1'. +create table t(a int, check(length(a))); +Error 3812 (HY000): An expression of non-boolean type specified to a check constraint 't_chk_1'. +create table t(a int, check(length(1))); +Error 3812 (HY000): An expression of non-boolean type specified to a check constraint 't_chk_1'. +create table t(a int, check(floor(1.1))); +Error 3812 (HY000): An expression of non-boolean type specified to a check constraint 't_chk_1'. +create table t(a int, check(mod(3, 2))); +Error 3812 (HY000): An expression of non-boolean type specified to a check constraint 't_chk_1'. +create table t(a int, check('true')); +Error 3812 (HY000): An expression of non-boolean type specified to a check constraint 't_chk_1'. +create table t(a int, check(true)); +alter table t add check(a); +Error 3812 (HY000): An expression of non-boolean type specified to a check constraint 't_chk_2'. +alter table t add check(1); +Error 3812 (HY000): An expression of non-boolean type specified to a check constraint 't_chk_2'. +alter table t add check('1'); +Error 3812 (HY000): An expression of non-boolean type specified to a check constraint 't_chk_2'. +alter table t add check(1/2); +Error 3812 (HY000): An expression of non-boolean type specified to a check constraint 't_chk_2'. +alter table t add check(a + 1/2); +Error 3812 (HY000): An expression of non-boolean type specified to a check constraint 't_chk_2'. +alter table t add check(true + 1); +Error 3812 (HY000): An expression of non-boolean type specified to a check constraint 't_chk_2'. +alter table t add check(abs(1)); +Error 3812 (HY000): An expression of non-boolean type specified to a check constraint 't_chk_2'. +alter table t add check(length(a)); +Error 3812 (HY000): An expression of non-boolean type specified to a check constraint 't_chk_2'. +alter table t add check(length(1)); +Error 3812 (HY000): An expression of non-boolean type specified to a check constraint 't_chk_2'. +alter table t add check(length(1)); +Error 3812 (HY000): An expression of non-boolean type specified to a check constraint 't_chk_2'. +alter table t add check(mod(3, 2)); +Error 3812 (HY000): An expression of non-boolean type specified to a check constraint 't_chk_2'. +alter table t add check('true'); +Error 3812 (HY000): An expression of non-boolean type specified to a check constraint 't_chk_2'. +drop table if exists t; +create table t(a int); +alter table t add check(b > 0); +Error 1054 (42S22): Unknown column 'b' in 'check constraint t_chk_1 expression' +drop table if exists t; +create table t(a json, b varchar(20)); +alter table t add check (JSON_VALID(a)); +alter table t add check (REGEXP_LIKE(b,'@')); +drop table if exists t; +create table t(a int constraint aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa check(a > 0)); +drop table t; +create table t(a int, constraint aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa check(a > 0)); +drop table t; +create table t(a int constraint aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa check(a > 0)); +Error 1059 (42000): Identifier name 'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa' is too long +create table t(a int, constraint aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa check(a > 0)); +Error 1059 (42000): Identifier name 'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa' is too long +create table aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa(a int check(a > 0)); +Error 1059 (42000): Identifier name 'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa_chk_1' is too long +create table aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa(a int, check(a > 0)); +Error 1059 (42000): Identifier name 'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa_chk_1' is too long +create table t(a int); +alter table t add constraint aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa check(a > 0); +Error 1059 (42000): Identifier name 'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa' is too long +alter table t add constraint aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa check(a > 0); +drop table t; +create table aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa(a int); +alter table aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa add check(a > 0); +Error 1059 (42000): Identifier name 'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa_chk_1' is too long +drop table if exists t; +create table t(a int); +alter table t add constraint `cafe` check(a > 0); +alter table t add constraint `CAFE` check(a > 0); +Error 3822 (HY000): Duplicate check constraint name 'cafe'. +alter table t add constraint `café` check(a > 0); +drop table if exists t, s; +create table t(a int check(a > 0)); +insert into t values(1); +create table s(a int); +insert into s values(-1); +insert into t values(-1); +Error 3819 (HY000): Check constraint 't_chk_1' is violated. +insert into t values(1); +insert into t(a) values(-1); +Error 3819 (HY000): Check constraint 't_chk_1' is violated. +insert into t(a) values(1); +insert into t set a = -1; +Error 3819 (HY000): Check constraint 't_chk_1' is violated. +insert into t set a = 1; +insert into t(a) select -1; +Error 3819 (HY000): Check constraint 't_chk_1' is violated. +insert into t(a) select 1; +insert into t(a) select a from s; +Error 3819 (HY000): Check constraint 't_chk_1' is violated. +insert into t(a) select a + 2 from s; +update t set a = -1; +Error 3819 (HY000): Check constraint 't_chk_1' is violated. +update t set a = 1; +update t set a = a-1; +Error 3819 (HY000): Check constraint 't_chk_1' is violated. +update t set a = a+1; +update t set a = -1 where a > 0; +Error 3819 (HY000): Check constraint 't_chk_1' is violated. +update t set a = 1 where a > 0; +update t set a = (select a from s where a < 0) where a > 0; +Error 3819 (HY000): Check constraint 't_chk_1' is violated. +update t set a = (select a + 2 from s where a < 0) where a > 0; +update t as a, s as b set a.a=b.a; +Error 3819 (HY000): Check constraint 't_chk_1' is violated. +update t as a, s as b set a.a=b.a + 2; +replace into t(a) values(-1); +Error 3819 (HY000): Check constraint 't_chk_1' is violated. +replace into t(a) values(1); +replace into t(a) select -1; +Error 3819 (HY000): Check constraint 't_chk_1' is violated. +replace into t(a) select 1; +replace into t set a = -1; +Error 3819 (HY000): Check constraint 't_chk_1' is violated. +replace into t set a = 1; +drop table if exists t; +create table t( a int, b int as (a+1), CHECK (b > 0)); +insert into t(a) values(0); +insert into t(a) values(-2); +Error 3819 (HY000): Check constraint 't_chk_1' is violated. +show create table t; +Table Create Table +t CREATE TABLE `t` ( + `a` int(11) DEFAULT NULL, + `b` int(11) GENERATED ALWAYS AS (`a` + 1) VIRTUAL, +CONSTRAINT `t_chk_1` CHECK ((`b` > 0)) +) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin +alter table t alter constraint t_chk_1 not enforced; +insert into t(a) values(-2); +alter table t drop constraint t_chk_1; +drop table t; +create table t(a int, b int as (a+1)); +alter table t add check(b > 0); +insert into t(a) values(0); +insert into t(a) values(-2); +Error 3819 (HY000): Check constraint 't_chk_1' is violated. +alter table t alter constraint t_chk_1 not enforced; +insert into t(a) values(-2); +alter table t drop constraint t_chk_1; +alter table t add check(b > 0); +Error 3819 (HY000): Check constraint 't_chk_1' is violated. +drop table if exists t; +create table t(a int, CHECK (a < -999)); +drop temporary table if exists t; +create temporary table t(a int, CHECK (a > 0)); +insert into t(a) values(1); +insert into t(a) values(-2); +Error 3819 (HY000): Check constraint 't_chk_1' is violated. +drop temporary table t; +create temporary table t(a int, CHECK (a > 0) not enforced); +insert into t values(-1); +create global temporary table tt(a int, check(a > 0)) on commit delete rows; +insert into tt(a) values(1); +insert into tt(a) values(-2); +Error 3819 (HY000): Check constraint 'tt_chk_1' is violated. +alter table tt alter constraint tt_chk_1 not enforced; +insert into tt(a) values(-2); +alter table tt drop constraint tt_chk_1; +drop temporary table t; +drop global temporary table tt; +drop table if exists t; +create table t(a int CHECK (a != 0)); +prepare stmt from 'insert into t values(?)'; +set @a = 1; +execute stmt using @a; + +set @a = 0; +execute stmt using @a; +Error 3819 (HY000): Check constraint 't_chk_1' is violated. +deallocate prepare stmt; +drop table if exists t, s; +create table t(a int primary key, b int, check (b > 0)); +insert into t values(1, 1); +insert into t values(1, -10) on duplicate key update b = -1; +Error 3819 (HY000): Check constraint 't_chk_1' is violated. +insert ignore into t values(1, -10) on duplicate key update b = -1; +select * from t; +a b +1 1 +create table s(a int primary key, check (a > 0)); +insert into s values(1); +insert into s values(1) on duplicate key update a = -1; +Error 3819 (HY000): Check constraint 's_chk_1' is violated. +insert ignore into s values(1) on duplicate key update a = -1; +select * from s; +a +1 +drop table if exists t1, t2; +CREATE TABLE t1 (CHECK (c1 <> c2), c1 INT CHECK (c1 > 10), c2 INT CONSTRAINT c2_positive CHECK (c2 > 0)); +insert into t1 values (2, 2); +Error 3819 (HY000): Check constraint 't1_chk_1' is violated. +insert into t1 values (9, 2); +Error 3819 (HY000): Check constraint 't1_chk_2' is violated. +insert into t1 values (14, -4); +Error 3819 (HY000): Check constraint 'c2_positive' is violated. +insert into t1(c1) values (9); +Error 3819 (HY000): Check constraint 't1_chk_2' is violated. +insert into t1(c2) values (-3); +Error 3819 (HY000): Check constraint 'c2_positive' is violated. +insert into t1 values (14, 4); +insert into t1 values (null, 4); +insert into t1 values (13, null); +insert into t1 values (null, null); +insert into t1(c1) values (null); +insert into t1(c2) values (null); +CREATE TABLE t2 (CHECK (c1 <> c2), c1 INT CHECK (c1 > 10), c2 INT CONSTRAINT c2_positive CHECK (c2 > 0), c3 int as (c1 + c2) check(c3 > 15)); +insert into t2(c1, c2) values (11, 1); +Error 3819 (HY000): Check constraint 't2_chk_3' is violated. +insert into t2(c1, c2) values (12, 7); +drop table if exists t1, t2; +CREATE TABLE t1 (CHECK (c1 <> c2), c1 INT CHECK (c1 > 10), c2 INT CONSTRAINT c2_positive CHECK (c2 > 0)); +insert into t1 values (11, 12), (12, 13), (13, 14), (14, 15), (15, 16); +update t1 set c2 = -c2; +Error 3819 (HY000): Check constraint 'c2_positive' is violated. +update t1 set c2 = c1; +Error 3819 (HY000): Check constraint 't1_chk_1' is violated. +update t1 set c1 = c1 - 10; +Error 3819 (HY000): Check constraint 't1_chk_2' is violated. +update t1 set c2 = -10 where c2 = 12; +Error 3819 (HY000): Check constraint 'c2_positive' is violated. +CREATE TABLE t2 (CHECK (c1 <> c2), c1 INT CHECK (c1 > 10), c2 INT CONSTRAINT c2_positive CHECK (c2 > 0), c3 int as (c1 + c2) check(c3 > 15)); +insert into t2(c1, c2) values (11, 12), (12, 13), (13, 14), (14, 15), (15, 16); +update t2 set c2 = c2 - 10; +Error 3819 (HY000): Check constraint 't2_chk_3' is violated. +update t2 set c2 = c2 - 5; +drop table if exists t1, t2; +CREATE TABLE t1 (CHECK (c1 <> c2), c1 INT CHECK (c1 > 10), c2 INT CONSTRAINT c2_positive CHECK (c2 > 0)) partition by hash(c2) partitions 5; +insert into t1 values (11, 12), (12, 13), (13, 14), (14, 15), (15, 16); +update t1 set c2 = -c2; +Error 3819 (HY000): Check constraint 'c2_positive' is violated. +update t1 set c2 = c1; +Error 3819 (HY000): Check constraint 't1_chk_1' is violated. +update t1 set c1 = c1 - 10; +Error 3819 (HY000): Check constraint 't1_chk_2' is violated. +update t1 set c2 = -10 where c2 = 12; +Error 3819 (HY000): Check constraint 'c2_positive' is violated. +CREATE TABLE t2 (CHECK (c1 <> c2), c1 INT CHECK (c1 > 10), c2 INT CONSTRAINT c2_positive CHECK (c2 > 0), c3 int as (c1 + c2) check(c3 > 15)) partition by hash(c2) partitions 5; +insert into t2(c1, c2) values (11, 12), (12, 13), (13, 14), (14, 15), (15, 16); +update t2 set c2 = c2 - 10; +Error 3819 (HY000): Check constraint 't2_chk_3' is violated. +update t2 set c2 = c2 - 5; +drop table if exists t; +create table t(a int check (a>1), b int, constraint my_constr check(a<10)); +show create table t; +Table Create Table +t CREATE TABLE `t` ( + `a` int(11) DEFAULT NULL, + `b` int(11) DEFAULT NULL, +CONSTRAINT `my_constr` CHECK ((`a` < 10)), +CONSTRAINT `t_chk_1` CHECK ((`a` > 1)) +) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin +alter table t add constraint my_constr2 check (a 1)), +CONSTRAINT `my_constr2` CHECK ((`a` < `b`)) /*!80016 NOT ENFORCED */ +) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin +alter table t drop constraint t_chk_1; +show create table t; +Table Create Table +t CREATE TABLE `t` ( + `a` int(11) DEFAULT NULL, + `b` int(11) DEFAULT NULL, +CONSTRAINT `my_constr` CHECK ((`a` < 10)), +CONSTRAINT `my_constr2` CHECK ((`a` < `b`)) /*!80016 NOT ENFORCED */ +) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin +drop table if exists t; +DROP TABLE IF EXISTS t0, t1, t2; +CREATE TABLE t0(c1 NUMERIC CHECK(true)); +CREATE TABLE t1(c1 NUMERIC, CHECK(true)); +CREATE TABLE t2(c1 NUMERIC); +ALTER TABLE t2 ADD CONSTRAINT CHECK(true); +DROP TABLE IF EXISTS t0, t1, t2; +CREATE TABLE t0(c1 NUMERIC CHECK(false)); +CREATE TABLE t1(c1 NUMERIC, CHECK(false)); +CREATE TABLE t2(c1 NUMERIC); +ALTER TABLE t2 ADD CONSTRAINT CHECK(false); +create table t(a int check(a > 0)); +show warnings; +Level Code Message +show create table t; +Table Create Table +t CREATE TABLE `t` ( + `a` int(11) DEFAULT NULL, +CONSTRAINT `t_chk_1` CHECK ((`a` > 0)) +) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin +drop table t; +set @@global.tidb_enable_check_constraint = 1; +create table t(a int check(a > 0)); +show warnings; +Level Code Message +show create table t; +Table Create Table +t CREATE TABLE `t` ( + `a` int(11) DEFAULT NULL, +CONSTRAINT `t_chk_1` CHECK ((`a` > 0)) +) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin +alter table t add constraint chk check(true); +show warnings; +Level Code Message +alter table t alter constraint chk not enforced; +show warnings; +Level Code Message +alter table t drop constraint chk; +show warnings; +Level Code Message +set @@global.tidb_enable_check_constraint = 0; +alter table t drop constraint t_chk_1; +show warnings; +Level Code Message +Warning 1105 the switch of check constraint is off +alter table t alter constraint t_chk_1 not enforced; +show warnings; +Level Code Message +Warning 1105 the switch of check constraint is off +show create table t; +Table Create Table +t CREATE TABLE `t` ( + `a` int(11) DEFAULT NULL, +CONSTRAINT `t_chk_1` CHECK ((`a` > 0)) +) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin +set @@global.tidb_enable_check_constraint = 1; +drop table if exists t; +create table t(a int not null check(a>0)); +show create table t; +Table Create Table +t CREATE TABLE `t` ( + `a` int(11) NOT NULL, +CONSTRAINT `t_chk_1` CHECK ((`a` > 0)) +) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin +drop table t; +create table t(a bigint key constraint my_constr check(a<10), b int constraint check(b > 1) not enforced); +show create table t; +Table Create Table +t CREATE TABLE `t` ( + `a` bigint(20) NOT NULL, + `b` int(11) DEFAULT NULL, + PRIMARY KEY (`a`) /*T![clustered_index] CLUSTERED */, +CONSTRAINT `my_constr` CHECK ((`a` < 10)), +CONSTRAINT `t_chk_1` CHECK ((`b` > 1)) /*!80016 NOT ENFORCED */ +) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin +drop table t; +create table t(a int constraint check(a > 1) not enforced, constraint my_constr check(a < 10)); +show create table t; +Table Create Table +t CREATE TABLE `t` ( + `a` int(11) DEFAULT NULL, +CONSTRAINT `my_constr` CHECK ((`a` < 10)), +CONSTRAINT `t_chk_1` CHECK ((`a` > 1)) /*!80016 NOT ENFORCED */ +) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin +drop table t; +create table t(a int not null check(b>0)); +Error 3813 (HY000): Column check constraint 't_chk_1' references other column. +create table t(a int not null check(b>a)); +Error 3813 (HY000): Column check constraint 't_chk_1' references other column. +create table t(a int not null check(a>0), b int, constraint check(c>b)); +Error 3820 (HY000): Check constraint 't_chk_1' refers to non-existing column 'c'. +create table t(a int not null check(a>0), b int, constraint check(a>b)); +show create table t; +Table Create Table +t CREATE TABLE `t` ( + `a` int(11) NOT NULL, + `b` int(11) DEFAULT NULL, +CONSTRAINT `t_chk_1` CHECK ((`a` > `b`)), +CONSTRAINT `t_chk_2` CHECK ((`a` > 0)) +) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin +drop table t; +create table t(a int not null check(a > '12345')); +show create table t; +Table Create Table +t CREATE TABLE `t` ( + `a` int(11) NOT NULL, +CONSTRAINT `t_chk_1` CHECK ((`a` > _utf8mb4'12345')) +) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin +drop table t; +create table t(a int not null primary key check(a > '12345')); +show create table t; +Table Create Table +t CREATE TABLE `t` ( + `a` int(11) NOT NULL, + PRIMARY KEY (`a`) /*T![clustered_index] CLUSTERED */, +CONSTRAINT `t_chk_1` CHECK ((`a` > _utf8mb4'12345')) +) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin +drop table t; +create table t(a varchar(10) not null primary key check(a > '12345')); +show create table t; +Table Create Table +t CREATE TABLE `t` ( + `a` varchar(10) NOT NULL, + PRIMARY KEY (`a`) /*T![clustered_index] NONCLUSTERED */, +CONSTRAINT `t_chk_1` CHECK ((`a` > _utf8mb4'12345')) +) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin +drop table t; +drop table if exists t; +create table t(a int not null check(a>0)); +alter table t add constraint haha check(a<10); +show create table t; +Table Create Table +t CREATE TABLE `t` ( + `a` int(11) NOT NULL, +CONSTRAINT `t_chk_1` CHECK ((`a` > 0)), +CONSTRAINT `haha` CHECK ((`a` < 10)) +) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin +alter table t add constraint check(a<11) not enforced; +show create table t; +Table Create Table +t CREATE TABLE `t` ( + `a` int(11) NOT NULL, +CONSTRAINT `t_chk_1` CHECK ((`a` > 0)), +CONSTRAINT `haha` CHECK ((`a` < 10)), +CONSTRAINT `t_chk_2` CHECK ((`a` < 11)) /*!80016 NOT ENFORCED */ +) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin +alter table t add constraint haha check(a); +Error 3822 (HY000): Duplicate check constraint name 'haha'. +alter table t add constraint check(b); +Error 1054 (42S22): Unknown column 'b' in 'check constraint t_chk_3 expression' +alter table t add constraint check(a*2 < a+1) not enforced; +drop table t; +create table t(a int); +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; +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 +alter table t drop constraint not_exist_constraint; +Error 3940 (HY000): Constraint 'not_exist_constraint' does not exist. +alter table t drop constraint haha; +show create table t; +Table Create Table +t CREATE TABLE `t` ( + `a` int(11) NOT NULL, + `b` int(11) DEFAULT NULL, +CONSTRAINT `t_chk_1` CHECK ((`a` < `b` + 1)), +CONSTRAINT `t_chk_2` CHECK ((`a` > 0)) +) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin +alter table t drop constraint t_chk_2; +show create table t; +Table Create Table +t CREATE TABLE `t` ( + `a` int(11) NOT NULL, + `b` int(11) DEFAULT NULL, +CONSTRAINT `t_chk_1` CHECK ((`a` < `b` + 1)) +) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin +drop table t; +create table t(a int check(a > 0)); +insert into t values(0); +Error 3819 (HY000): Check constraint 't_chk_1' is violated. +alter table t drop constraint t_chk_1; +insert into t values(0); +drop table if exists t; +set @@global.tidb_enable_check_constraint = 1; +create table t(a int not null check(a>0) not enforced, b int, constraint haha check(a < b)); +show create table t; +Table Create Table +t CREATE TABLE `t` ( + `a` int(11) NOT NULL, + `b` int(11) DEFAULT NULL, +CONSTRAINT `haha` CHECK ((`a` < `b`)), +CONSTRAINT `t_chk_1` CHECK ((`a` > 0)) /*!80016 NOT ENFORCED */ +) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin +alter table t alter constraint unknown not enforced; +Error 3940 (HY000): Constraint 'unknown' does not exist. +alter table t alter constraint haha not enforced; +show create table t; +Table Create Table +t CREATE TABLE `t` ( + `a` int(11) NOT NULL, + `b` int(11) DEFAULT NULL, +CONSTRAINT `haha` CHECK ((`a` < `b`)) /*!80016 NOT ENFORCED */, +CONSTRAINT `t_chk_1` CHECK ((`a` > 0)) /*!80016 NOT ENFORCED */ +) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin +alter table t alter constraint t_chk_1 enforced; +show create table t; +Table Create Table +t CREATE TABLE `t` ( + `a` int(11) NOT NULL, + `b` int(11) DEFAULT NULL, +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; diff --git a/tests/integrationtest/t/ddl/constraint.test b/tests/integrationtest/t/ddl/constraint.test new file mode 100644 index 0000000000000..d197575970820 --- /dev/null +++ b/tests/integrationtest/t/ddl/constraint.test @@ -0,0 +1,680 @@ +set @@global.tidb_enable_check_constraint = 1; + +# TestDropColumnWithCheckConstraints +drop table if exists t; +create table t(a int check(a > 0), b int, c int, check(c > 0), check (b > c)); +alter table t drop column a; +show create table t; +-- error 3959 +alter table t drop column b; +-- error 3959 +alter table t rename column c to c1; + +# TestCheckConstraintsNotEnforcedWorks +drop table if exists t; +create table t(a int check(a > 0), b int check(b < 10) not enforced, c int); +alter table t alter constraint t_chk_1 not enforced; +insert into t values(-1, 1, 0); +alter table t alter constraint t_chk_2 enforced; +-- error 3819 +insert into t values(-1, 11, 0); +alter table t add check(c = 0); +-- error 3819 +insert into t values(-1, 1, 1); +alter table t alter constraint t_chk_3 not enforced; +insert into t values(-1, 1, 0); + +# TestUnsupportedCheckConstraintsExprWhenCreateTable +drop table if exists t; +-- error 3814 +CREATE TABLE t1 (f1 TIMESTAMP CHECK (f1 + NOW() > '2011-11-21')); +-- error 3814 +CREATE TABLE t1 (f1 TIMESTAMP CHECK (f1 + CURRENT_TIMESTAMP() > '2011-11-21 01:02:03')); +-- error 3814 +CREATE TABLE t1 (f1 TIMESTAMP CHECK (f1 + CURRENT_TIMESTAMP > '2011-11-21 01:02:03')); +-- error 3814 +CREATE TABLE t1 (f1 DATETIME CHECK (f1 + CURDATE() > '2011-11-21')); +-- error 3814 +CREATE TABLE t1 (f1 DATETIME CHECK (f1 + CURTIME() > '23:11:21')); +-- error 3814 +CREATE TABLE t1 (f1 TIMESTAMP CHECK (f1 + CURRENT_DATE() > '2011-11-21')); +-- error 3814 +CREATE TABLE t1 (f1 TIMESTAMP CHECK (f1 + CURRENT_DATE > '2011-11-21')); +-- error 3814 +CREATE TABLE t1 (f1 TIMESTAMP CHECK (f1 + CURRENT_TIME() > '01:02:03')); +-- error 3814 +CREATE TABLE t1 (f1 TIMESTAMP CHECK (f1 + CURRENT_TIME > '01:02:03')); +-- error 3814 +CREATE TABLE t1 (f1 DATETIME CHECK (f1 + LOCALTIME() > '23:11:21')); +-- error 3814 +CREATE TABLE t1 (f1 DATETIME CHECK (f1 + LOCALTIME > '23:11:21')); +-- error 3814 +CREATE TABLE t1 (f1 TIMESTAMP CHECK (f1 + LOCALTIMESTAMP() > '2011-11-21 01:02:03')); +-- error 3814 +CREATE TABLE t1 (f1 TIMESTAMP CHECK (f1 + LOCALTIMESTAMP > '2011-11-21 01:02:03')); +-- error 3814 +CREATE TABLE t1 (f1 TIMESTAMP CHECK (f1 + UNIX_TIMESTAMP() > '2011-11-21 01:02:03')); +-- error 3814 +CREATE TABLE t1 (f1 DATETIME CHECK (f1 + UTC_DATE() > '2011-11-21')); +-- error 3814 +CREATE TABLE t1 (f1 TIMESTAMP CHECK (f1 + UTC_TIMESTAMP() > '2011-11-21 01:02:03')); +-- error 3814 +CREATE TABLE t1 (f1 DATETIME CHECK (f1 + UTC_TIME() > '23:11:21')); +-- error 3814 +CREATE TABLE t1 (f1 INT CHECK (f1 + CONNECTION_ID() < 929)); +-- error 3814 +CREATE TABLE t1 (a VARCHAR(32) CHECK (CURRENT_USER() != a)); +-- error 3814 +CREATE TABLE t1 (a VARCHAR(32) CHECK (CURRENT_USER != a)); +-- error 3814 +CREATE TABLE t1 (a VARCHAR(32) CHECK (SESSION_USER() != a)); +-- error 3814 +CREATE TABLE t1 (a VARCHAR(32) CHECK (VERSION() != a)); +-- error 3814 +CREATE TABLE t1 (a VARCHAR(1024), b INT CHECK (b + FOUND_ROWS() > 2000)); +-- error 3814 +CREATE TABLE t1 (a INT CHECK ((a + LAST_INSERT_ID()) < 929)); +-- error 3814 +CREATE TABLE t1 (a VARCHAR(32) CHECK (SYSTEM_USER() != a)); +-- error 3814 +CREATE TABLE t1 (a VARCHAR(32) CHECK (USER() != a)); +-- error 3814 +CREATE TABLE t1 (f1 FLOAT CHECK (f1 + RAND() < 929.929)); +-- error 3814 +CREATE TABLE t1 (a INT CHECK (a + ROW_COUNT() > 1000)); +-- error 3814 +CREATE TABLE t1 (a VARCHAR(1024), b VARCHAR(1024) CHECK (GET_LOCK(b,10) != 0)); +-- error 3814 +CREATE TABLE t1 (a VARCHAR(1024), b VARCHAR(1024) CHECK (IS_FREE_LOCK(b) != 0)); +-- error 3814 +CREATE TABLE t1 (a VARCHAR(1024), b VARCHAR(1024) CHECK (IS_USED_LOCK(b) != 0)); +-- error 3814 +CREATE TABLE t1 (a VARCHAR(1024), b VARCHAR(1024) CHECK (RELEASE_LOCK(b) != 0)); +-- error 3814 +CREATE TABLE t1 (a VARCHAR(1024), b VARCHAR(1024), CHECK (RELEASE_ALL_LOCKS() != 0)); +-- error 3814 +CREATE TABLE t1 (f1 VARCHAR(1024), f2 VARCHAR(1024) CHECK (LOAD_FILE(f2) != NULL)); +-- error 3814 +CREATE TABLE t1 (id CHAR(40) CHECK(UUID() != id)); +-- error 3814 +CREATE TABLE t1 (id INT CHECK(UUID_SHORT() != id)); +-- error 3814 +CREATE TABLE t1 (id INT CHECK(SLEEP(id) != 0)); +set @a = 1; +-- error 3816 +CREATE TABLE t1 (f1 int CHECK (f1 > @a)); +-- error 3816 +CREATE TABLE t1 (f1 int CHECK (f1 > @@session.tidb_mem_quota_query)); +-- error 3816 +CREATE TABLE t1 (f1 int CHECK (f1 > @@global.tidb_mem_quota_query)); +-- error 3818 +CREATE TABLE t1 (f1 int primary key auto_increment, f2 int, CHECK (f1 != f2)); +-- error 3814 +CREATE TABLE t1 (f1 INT CHECK (f1 = default(f1))); +-- error 3815 +CREATE TABLE t1 (id INT CHECK (id != (SELECT 1))); +-- error 3815 +CREATE TABLE t1 (a int check(a in (SELECT COALESCE(NULL, 1, 1)))); + +# TestUnsupportedCheckConstraintsExprWhenAlterTable +drop table if exists t; +create table t1(f1 TIMESTAMP, f2 DATETIME, f3 INT, f4 VARCHAR(32), f5 FLOAT, f6 CHAR(40), f7 INT PRIMARY KEY AUTO_INCREMENT); +-- error 3814 +ALTER TABLE t1 ADD CHECK (f1 + NOW() > '2011-11-21'); +-- error 3814 +ALTER TABLE t1 ADD CHECK (f1 + CURRENT_TIMESTAMP() > '2011-11-21 01:02:03'); +-- error 3814 +ALTER TABLE t1 ADD CHECK (f1 + CURRENT_TIMESTAMP > '2011-11-21 01:02:03'); +-- error 3814 +ALTER TABLE t1 ADD CHECK (f2 + CURDATE() > '2011-11-21'); +-- error 3814 +ALTER TABLE t1 ADD CHECK (f2 + CURTIME() > '23:11:21'); +-- error 3814 +ALTER TABLE t1 ADD CHECK (f1 + CURRENT_DATE() > '2011-11-21'); +-- error 3814 +ALTER TABLE t1 ADD CHECK (f1 + CURRENT_DATE > '2011-11-21'); +-- error 3814 +ALTER TABLE t1 ADD CHECK (f1 + CURRENT_TIME() > '01:02:03'); +-- error 3814 +ALTER TABLE t1 ADD CHECK (f1 + CURRENT_TIME > '01:02:03'); +-- error 3814 +ALTER TABLE t1 ADD CHECK (f2 + LOCALTIME() > '23:11:21'); +-- error 3814 +ALTER TABLE t1 ADD CHECK (f2 + LOCALTIME > '23:11:21'); +-- error 3814 +ALTER TABLE t1 ADD CHECK (f1 + LOCALTIMESTAMP() > '2011-11-21 01:02:03'); +-- error 3814 +ALTER TABLE t1 ADD CHECK (f1 + LOCALTIMESTAMP > '2011-11-21 01:02:03'); +-- error 3814 +ALTER TABLE t1 ADD CHECK (f1 + UNIX_TIMESTAMP() > '2011-11-21 01:02:03'); +-- error 3814 +ALTER TABLE t1 ADD CHECK (f2 + UTC_DATE() > '2011-11-21'); +-- error 3814 +ALTER TABLE t1 ADD CHECK (f1 + UTC_TIMESTAMP() > '2011-11-21 01:02:03'); +-- error 3814 +ALTER TABLE t1 ADD CHECK (f2 + UTC_TIME() > '23:11:21'); +-- error 3814 +ALTER TABLE t1 ADD CHECK (f3 + CONNECTION_ID() < 929); +-- error 3814 +ALTER TABLE t1 ADD CHECK (CURRENT_USER() != f4); +-- error 3814 +ALTER TABLE t1 ADD CHECK (CURRENT_USER != f4); +-- error 3814 +ALTER TABLE t1 ADD CHECK (SESSION_USER() != f4); +-- error 3814 +ALTER TABLE t1 ADD CHECK (VERSION() != f4); +-- error 3814 +ALTER TABLE t1 ADD CHECK (f3 + FOUND_ROWS() > 2000); +-- error 3814 +ALTER TABLE t1 ADD CHECK ((f3 + LAST_INSERT_ID()) < 929); +-- error 3814 +ALTER TABLE t1 ADD CHECK (SYSTEM_USER() != f4); +-- error 3814 +ALTER TABLE t1 ADD CHECK (USER() != f4); +-- error 3814 +ALTER TABLE t1 ADD CHECK (f5 + RAND() < 929.929); +-- error 3814 +ALTER TABLE t1 ADD CHECK (f3 + ROW_COUNT() > 1000); +-- error 3814 +ALTER TABLE t1 ADD CHECK (GET_LOCK(f4,10) != 0); +-- error 3814 +ALTER TABLE t1 ADD CHECK (IS_FREE_LOCK(f4) != 0); +-- error 3814 +ALTER TABLE t1 ADD CHECK (IS_USED_LOCK(f4) != 0); +-- error 3814 +ALTER TABLE t1 ADD CHECK (RELEASE_LOCK(f4) != 0); +-- error 3814 +ALTER TABLE t1 ADD CHECK (RELEASE_ALL_LOCKS() != 0); +-- error 3814 +ALTER TABLE t1 ADD CHECK (LOAD_FILE(f4) != NULL); +-- error 3814 +ALTER TABLE t1 ADD CHECK(UUID() != f6); +-- error 3814 +ALTER TABLE t1 ADD CHECK(UUID_SHORT() != f3); +-- error 3814 +ALTER TABLE t1 ADD CHECK(SLEEP(f3) != 0); +set @a = 1; +-- error 3816 +ALTER TABLE t1 ADD CHECK (f3 > @a); +-- error 3816 +ALTER TABLE t1 ADD CHECK (f3 > @@session.tidb_mem_quota_query); +-- error 3816 +ALTER TABLE t1 ADD CHECK (f3 > @@global.tidb_mem_quota_query); +-- error 3818 +ALTER TABLE t1 ADD CHECK (f7 != f3); +-- error 3814 +ALTER TABLE t1 ADD CHECK (f1 = default(f1)); +-- error 3815 +ALTER TABLE t1 ADD CHECK (f3 != (SELECT 1)); +-- error 3815 +ALTER TABLE t1 ADD check (f3 in (SELECT COALESCE(NULL, 1, 1))); + +# TestNameInCreateTableLike +drop table if exists t, s; +create table t(a int check(a > 10), b int constraint bbb check(b > 5), c int, check(c < 0)); +create table s like t; +show create table s; +-- error 3819 +insert into s(a) values(1); +-- error 3819 +insert into s(b) values(2); +-- error 3819 +insert into s(c) values(3); +alter table s add check(a > 0); +alter table s add constraint aaa check(a > 0); +show create table s; + +# TestInsertUpdateIgnoreWarningMessage +drop table if exists t; +create table t(a int check(a > 10)); +-- error 3819 +insert into t values(1),(11),(15); +insert ignore into t values(1),(11),(15); +show warnings; +select a from t; +update ignore t set a = a-2; +show warnings; +select a from t; + +# TestCheckConstraintForeignKey +drop table if exists t, s, t1, t2; +create table t(a int, b int, index(a), index(a, b)); +-- error 3823 +create table s(a int, check (a > 0), foreign key (a) references t(a) on update cascade); +-- error 3823 +create table s(a int, b int, check (a > 0), foreign key (a, b) references t(a, b) on update cascade); +create table t1(a int, foreign key (a) references t(a) on update cascade); +-- error 3823 +alter table t1 add check ( a > 0 ); +create table t2(a int, b int, foreign key (a, b) references t(a, b) on update cascade); +-- error 3823 +alter table t2 add check ( a > 0 ); +drop table t, t1, t2; + +# TestCheckConstrainNonBoolExpr +drop table if exists t; +-- error 3812 +create table t(a int, check(a)); +-- error 3812 +create table t(a int, check(1)); +-- error 3812 +create table t(a int, check('1')); +-- error 3812 +create table t(a int check(a)); +-- error 3812 +create table t(a int, check(1+1)); +-- error 3812 +create table t(a int, check(1/2)); +-- error 3812 +create table t(a int, check(a + 1/2)); +-- error 3812 +create table t(a int check(a + 1/2)); +-- error 3812 +create table t(a int, check(true + 1)); +-- error 3812 +create table t(a int, check(abs(1))); +-- error 3812 +create table t(a int, check(length(a))); +-- error 3812 +create table t(a int, check(length(1))); +-- error 3812 +create table t(a int, check(floor(1.1))); +-- error 3812 +create table t(a int, check(mod(3, 2))); +-- error 3812 +create table t(a int, check('true')); +create table t(a int, check(true)); +-- error 3812 +alter table t add check(a); +-- error 3812 +alter table t add check(1); +-- error 3812 +alter table t add check('1'); +-- error 3812 +alter table t add check(1/2); +-- error 3812 +alter table t add check(a + 1/2); +-- error 3812 +alter table t add check(true + 1); +-- error 3812 +alter table t add check(abs(1)); +-- error 3812 +alter table t add check(length(a)); +-- error 3812 +alter table t add check(length(1)); +-- error 3812 +alter table t add check(length(1)); +-- error 3812 +alter table t add check(mod(3, 2)); +-- error 3812 +alter table t add check('true'); + +# TestAlterAddCheckConstrainColumnBadErr +drop table if exists t; +create table t(a int); +-- error 1054 +alter table t add check(b > 0); + +# TestCheckConstraintBoolExpr +drop table if exists t; +create table t(a json, b varchar(20)); +alter table t add check (JSON_VALID(a)); +alter table t add check (REGEXP_LIKE(b,'@')); + +# TestCheckConstraintNameMaxLength +drop table if exists t; +create table t(a int constraint aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa check(a > 0)); +drop table t; +create table t(a int, constraint aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa check(a > 0)); +drop table t; +-- error 1059 +create table t(a int constraint aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa check(a > 0)); +-- error 1059 +create table t(a int, constraint aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa check(a > 0)); +-- error 1059 +create table aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa(a int check(a > 0)); +-- error 1059 +create table aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa(a int, check(a > 0)); +create table t(a int); +-- error 1059 +alter table t add constraint aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa check(a > 0); +alter table t add constraint aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa check(a > 0); +drop table t; +create table aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa(a int); +-- error 1059 +alter table aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa add check(a > 0); + +# TestCheckConstraintNameCaseAndAccentSensitivity +drop table if exists t; +create table t(a int); +alter table t add constraint `cafe` check(a > 0); +-- error 3822 +alter table t add constraint `CAFE` check(a > 0); +alter table t add constraint `café` check(a > 0); + +# TestCheckConstraintEvaluated +drop table if exists t, s; +create table t(a int check(a > 0)); +insert into t values(1); +create table s(a int); +insert into s values(-1); +-- error 3819 +insert into t values(-1); +insert into t values(1); +-- error 3819 +insert into t(a) values(-1); +insert into t(a) values(1); +-- error 3819 +insert into t set a = -1; +insert into t set a = 1; +-- error 3819 +insert into t(a) select -1; +insert into t(a) select 1; +-- error 3819 +insert into t(a) select a from s; +insert into t(a) select a + 2 from s; +-- error 3819 +update t set a = -1; +update t set a = 1; +-- error 3819 +update t set a = a-1; +update t set a = a+1; +-- error 3819 +update t set a = -1 where a > 0; +update t set a = 1 where a > 0; +-- error 3819 +update t set a = (select a from s where a < 0) where a > 0; +update t set a = (select a + 2 from s where a < 0) where a > 0; +-- error 3819 +update t as a, s as b set a.a=b.a; +update t as a, s as b set a.a=b.a + 2; +-- error 3819 +replace into t(a) values(-1); +replace into t(a) values(1); +-- error 3819 +replace into t(a) select -1; +replace into t(a) select 1; +-- error 3819 +replace into t set a = -1; +replace into t set a = 1; + +# TestGenerateColumnCheckConstraint +drop table if exists t; +create table t( a int, b int as (a+1), CHECK (b > 0)); +insert into t(a) values(0); +-- error 3819 +insert into t(a) values(-2); +show create table t; +alter table t alter constraint t_chk_1 not enforced; +insert into t(a) values(-2); +alter table t drop constraint t_chk_1; +drop table t; +create table t(a int, b int as (a+1)); +alter table t add check(b > 0); +insert into t(a) values(0); +-- error 3819 +insert into t(a) values(-2); +alter table t alter constraint t_chk_1 not enforced; +insert into t(a) values(-2); +alter table t drop constraint t_chk_1; +-- error 3819 +alter table t add check(b > 0); + +# TestTemporaryTableCheckConstraint +drop table if exists t; +create table t(a int, CHECK (a < -999)); +drop temporary table if exists t; +create temporary table t(a int, CHECK (a > 0)); +insert into t(a) values(1); +-- error 3819 +insert into t(a) values(-2); +drop temporary table t; +create temporary table t(a int, CHECK (a > 0) not enforced); +insert into t values(-1); +create global temporary table tt(a int, check(a > 0)) on commit delete rows; +insert into tt(a) values(1); +-- error 3819 +insert into tt(a) values(-2); +alter table tt alter constraint tt_chk_1 not enforced; +insert into tt(a) values(-2); +alter table tt drop constraint tt_chk_1; +drop temporary table t; +drop global temporary table tt; + +# TestCheckConstraintWithPrepareInsertCheckConstraint +drop table if exists t; +create table t(a int CHECK (a != 0)); +prepare stmt from 'insert into t values(?)'; +set @a = 1; +execute stmt using @a; +set @a = 0; +-- error 3819 +execute stmt using @a; +deallocate prepare stmt; + +# TestCheckConstraintOnDuplicateKeyUpdate +drop table if exists t, s; +create table t(a int primary key, b int, check (b > 0)); +insert into t values(1, 1); +-- error 3819 +insert into t values(1, -10) on duplicate key update b = -1; +insert ignore into t values(1, -10) on duplicate key update b = -1; +select * from t; +create table s(a int primary key, check (a > 0)); +insert into s values(1); +-- error 3819 +insert into s values(1) on duplicate key update a = -1; +insert ignore into s values(1) on duplicate key update a = -1; +select * from s; + +# TestCheckConstraintOnInsert +drop table if exists t1, t2; +CREATE TABLE t1 (CHECK (c1 <> c2), c1 INT CHECK (c1 > 10), c2 INT CONSTRAINT c2_positive CHECK (c2 > 0)); +-- error 3819 +insert into t1 values (2, 2); +-- error 3819 +insert into t1 values (9, 2); +-- error 3819 +insert into t1 values (14, -4); +-- error 3819 +insert into t1(c1) values (9); +-- error 3819 +insert into t1(c2) values (-3); +insert into t1 values (14, 4); +insert into t1 values (null, 4); +insert into t1 values (13, null); +insert into t1 values (null, null); +insert into t1(c1) values (null); +insert into t1(c2) values (null); +CREATE TABLE t2 (CHECK (c1 <> c2), c1 INT CHECK (c1 > 10), c2 INT CONSTRAINT c2_positive CHECK (c2 > 0), c3 int as (c1 + c2) check(c3 > 15)); +-- error 3819 +insert into t2(c1, c2) values (11, 1); +insert into t2(c1, c2) values (12, 7); + +# TestCheckConstraintOnUpdate +drop table if exists t1, t2; +CREATE TABLE t1 (CHECK (c1 <> c2), c1 INT CHECK (c1 > 10), c2 INT CONSTRAINT c2_positive CHECK (c2 > 0)); +insert into t1 values (11, 12), (12, 13), (13, 14), (14, 15), (15, 16); +-- error 3819 +update t1 set c2 = -c2; +-- error 3819 +update t1 set c2 = c1; +-- error 3819 +update t1 set c1 = c1 - 10; +-- error 3819 +update t1 set c2 = -10 where c2 = 12; +CREATE TABLE t2 (CHECK (c1 <> c2), c1 INT CHECK (c1 > 10), c2 INT CONSTRAINT c2_positive CHECK (c2 > 0), c3 int as (c1 + c2) check(c3 > 15)); +insert into t2(c1, c2) values (11, 12), (12, 13), (13, 14), (14, 15), (15, 16); +-- error 3819 +update t2 set c2 = c2 - 10; +update t2 set c2 = c2 - 5; + +# TestCheckConstraintOnUpdateWithPartition +drop table if exists t1, t2; +CREATE TABLE t1 (CHECK (c1 <> c2), c1 INT CHECK (c1 > 10), c2 INT CONSTRAINT c2_positive CHECK (c2 > 0)) partition by hash(c2) partitions 5; +insert into t1 values (11, 12), (12, 13), (13, 14), (14, 15), (15, 16); +-- error 3819 +update t1 set c2 = -c2; +-- error 3819 +update t1 set c2 = c1; +-- error 3819 +update t1 set c1 = c1 - 10; +-- error 3819 +update t1 set c2 = -10 where c2 = 12; +CREATE TABLE t2 (CHECK (c1 <> c2), c1 INT CHECK (c1 > 10), c2 INT CONSTRAINT c2_positive CHECK (c2 > 0), c3 int as (c1 + c2) check(c3 > 15)) partition by hash(c2) partitions 5; +insert into t2(c1, c2) values (11, 12), (12, 13), (13, 14), (14, 15), (15, 16); +-- error 3819 +update t2 set c2 = c2 - 10; +update t2 set c2 = c2 - 5; + +# TestShowCheckConstraint +drop table if exists t; +create table t(a int check (a>1), b int, constraint my_constr check(a<10)); +show create table t; +alter table t add constraint my_constr2 check (a 0)); +show warnings; +show create table t; +drop table t; +set @@global.tidb_enable_check_constraint = 1; +create table t(a int check(a > 0)); +show warnings; +show create table t; +alter table t add constraint chk check(true); +show warnings; +alter table t alter constraint chk not enforced; +show warnings; +alter table t drop constraint chk; +show warnings; +set @@global.tidb_enable_check_constraint = 0; +alter table t drop constraint t_chk_1; +show warnings; +alter table t alter constraint t_chk_1 not enforced; +show warnings; +show create table t; +set @@global.tidb_enable_check_constraint = 1; + +# TestCreateTableWithCheckConstraints +drop table if exists t; +create table t(a int not null check(a>0)); +show create table t; +drop table t; +create table t(a bigint key constraint my_constr check(a<10), b int constraint check(b > 1) not enforced); +show create table t; +drop table t; +create table t(a int constraint check(a > 1) not enforced, constraint my_constr check(a < 10)); +show create table t; +drop table t; +-- error 3813 +create table t(a int not null check(b>0)); +-- error 3813 +create table t(a int not null check(b>a)); +-- error 3820 +create table t(a int not null check(a>0), b int, constraint check(c>b)); +create table t(a int not null check(a>0), b int, constraint check(a>b)); +show create table t; +drop table t; +create table t(a int not null check(a > '12345')); +show create table t; +drop table t; +create table t(a int not null primary key check(a > '12345')); +show create table t; +drop table t; +create table t(a varchar(10) not null primary key check(a > '12345')); +show create table t; +drop table t; + +# TestAlterTableAddCheckConstraints +drop table if exists t; +create table t(a int not null check(a>0)); +alter table t add constraint haha check(a<10); +show create table t; +alter table t add constraint check(a<11) not enforced; +show create table t; +-- error 3822 +alter table t add constraint haha check(a); +-- error 1054 +alter table t add constraint check(b); +alter table t add constraint check(a*2 < a+1) not enforced; +drop table t; +create table t(a int); +insert into t values(1), (2), (3); +-- error 3819 +alter table t add constraint check(a < 2); +-- error 3819 +alter table t add constraint check(a < 2) not enforced; + +# TestAlterTableDropCheckConstraints +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)); +-- error 3819 +insert into t values(0); +alter table t drop constraint t_chk_1; +insert into t values(0); + +# TestAlterTableAlterCheckConstraints +drop table if exists t; +set @@global.tidb_enable_check_constraint = 1; +create table t(a int not null check(a>0) not enforced, b int, constraint haha check(a < b)); +show create table t; +-- error 3940 +alter table t alter constraint unknown not enforced; +alter table t alter constraint haha not enforced; +show create table t; +alter table t alter constraint t_chk_1 enforced; +show create table t; +# Alter table alter constraint will violate check. +# Here a=1, b=0 doesn't satisfy "a < b" constraint. +# Since "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;