Skip to content

Commit

Permalink
ddl:support the definition of null change to not null using alter table
Browse files Browse the repository at this point in the history
  • Loading branch information
ciscoxll committed Sep 23, 2018
1 parent 30ae420 commit f16f213
Show file tree
Hide file tree
Showing 6 changed files with 91 additions and 8 deletions.
58 changes: 55 additions & 3 deletions ddl/column.go
Original file line number Diff line number Diff line change
Expand Up @@ -14,12 +14,16 @@
package ddl

import (
"fmt"
"github.com/juju/errors"
"github.com/pingcap/tidb/ast"
"github.com/pingcap/tidb/ddl/util"
"github.com/pingcap/tidb/infoschema"
"github.com/pingcap/tidb/meta"
"github.com/pingcap/tidb/model"
"github.com/pingcap/tidb/mysql"
"github.com/pingcap/tidb/sessionctx"
"github.com/pingcap/tidb/util/sqlexec"
log "github.com/sirupsen/logrus"
)

Expand Down Expand Up @@ -267,7 +271,7 @@ func onSetDefaultValue(t *meta.Meta, job *model.Job) (ver int64, _ error) {
return updateColumn(t, job, newCol, &newCol.Name)
}

func onModifyColumn(t *meta.Meta, job *model.Job) (ver int64, _ error) {
func (w *worker) onModifyColumn(t *meta.Meta, job *model.Job) (ver int64, _ error) {
newCol := &model.ColumnInfo{}
oldColName := &model.CIStr{}
pos := &ast.ColumnPosition{}
Expand All @@ -277,11 +281,16 @@ func onModifyColumn(t *meta.Meta, job *model.Job) (ver int64, _ error) {
return ver, errors.Trace(err)
}

return doModifyColumn(t, job, newCol, oldColName, pos)
return w.doModifyColumn(t, job, newCol, oldColName, pos)
}

// doModifyColumn updates the column information and reorders all columns.
func doModifyColumn(t *meta.Meta, job *model.Job, newCol *model.ColumnInfo, oldName *model.CIStr, pos *ast.ColumnPosition) (ver int64, _ error) {
func (w *worker) doModifyColumn(t *meta.Meta, job *model.Job, newCol *model.ColumnInfo, oldName *model.CIStr, pos *ast.ColumnPosition) (ver int64, _ error) {
dbInfo, err := t.GetDatabase(job.SchemaID)
if err != nil {
return ver, errors.Trace(err)
}

tblInfo, err := getTableInfo(t, job, job.SchemaID)
if err != nil {
return ver, errors.Trace(err)
Expand All @@ -308,6 +317,35 @@ func doModifyColumn(t *meta.Meta, job *model.Job, newCol *model.ColumnInfo, oldN
// }
// }

if !mysql.HasNotNullFlag(oldCol.Flag) && mysql.HasNotNullFlag(newCol.Flag) {
// Get sessionctx from context resource pool.
ctx, err := w.sessPool.get()
if err != nil {
return ver, errors.Trace(err)
}
defer w.sessPool.put(ctx)

// Modify the type defined Flag to NotNullFlag.
tblInfo.Columns[oldCol.Offset].Flag = newCol.Flag
ver, err = updateVersionAndTableInfo(t, job, tblInfo, true)
if err != nil {
job.State = model.JobStateCancelled
return ver, errors.Trace(err)
}

err = CheckForNullValue(ctx, dbInfo.Name, tblInfo.Name, oldCol.Name, newCol.Name)
if err != nil {
// Execute rollback if an error occurs.
tblInfo.Columns[oldCol.Offset].Flag = oldCol.Flag
ver, err1 := updateVersionAndTableInfo(t, job, tblInfo, true)
if err != nil {
return ver, errors.Trace(err1)
}
job.State = model.JobStateCancelled
return ver, errors.Trace(err)
}
}

// We need the latest column's offset and state. This information can be obtained from the store.
newCol.Offset = oldCol.Offset
newCol.State = oldCol.State
Expand Down Expand Up @@ -379,6 +417,20 @@ func doModifyColumn(t *meta.Meta, job *model.Job, newCol *model.ColumnInfo, oldN
return ver, nil
}

// CheckForNullValue
func CheckForNullValue(ctx sessionctx.Context, schema, table, oldCol, newCol model.CIStr) error {
sql := fmt.Sprintf(`select * from %s.%s where %s is null; `, schema.L, table.L, oldCol.L)
rows, _, err := ctx.(sqlexec.RestrictedSQLExecutor).ExecRestrictedSQL(ctx, sql)
if err != nil {
return errors.Trace(err)
}

if len(rows) != 0 {
return ErrWarnDataTruncated.GenByArgs(newCol.L, len(rows))
}
return nil
}

func updateColumn(t *meta.Meta, job *model.Job, newCol *model.ColumnInfo, oldColName *model.CIStr) (ver int64, _ error) {
tblInfo, err := getTableInfo(t, job, job.SchemaID)
if err != nil {
Expand Down
12 changes: 12 additions & 0 deletions ddl/db_change_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -551,6 +551,18 @@ func (s *testStateChangeSuite) TestParallelAlterModifyColumn(c *C) {
s.testControlParallelExecSQL(c, sql, sql, f)
}

func (s *testStateChangeSuite) TestParallelColumnModifyingDefinition(c *C) {
sql1 := "insert into t(b) values (null);"
sql2 := "alter table t change b b2 bigint not null;"
f := func(c *C, err1, err2 error) {
c.Assert(err1, IsNil)
if err2 != nil {
c.Assert(err2.Error(), Equals, "[ddl:1265]Data truncated for column 'b2' at row 1")
}
}
s.testControlParallelExecSQL(c, sql1, sql2, f)
}

func (s *testStateChangeSuite) TestParallelChangeColumnName(c *C) {
sql1 := "ALTER TABLE t CHANGE a aa int;"
sql2 := "ALTER TABLE t CHANGE b aa int;"
Expand Down
16 changes: 14 additions & 2 deletions ddl/db_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -1226,8 +1226,8 @@ func (s *testDBSuite) TestChangeColumn(c *C) {
s.testErrorCode(c, sql, tmysql.ErrWrongDBName)
sql = "alter table t3 change t.a aa bigint"
s.testErrorCode(c, sql, tmysql.ErrWrongTableName)
sql = "alter table t3 change aa a bigint not null"
s.testErrorCode(c, sql, tmysql.ErrUnknown)
//sql = "alter table t3 change aa a bigint not null"
//s.testErrorCode(c, sql, tmysql.ErrUnknown)
sql = "alter table t3 modify en enum('a', 'z', 'b', 'c') not null default 'a'"
s.testErrorCode(c, sql, tmysql.ErrUnknown)
// Rename to an existing column.
Expand Down Expand Up @@ -3416,6 +3416,18 @@ func backgroundExecOnJobUpdatedExported(c *C, s *testDBSuite, hook *ddl.TestDDLC
return hook.OnJobUpdatedExported, c3IdxInfo
}

func (s *testDBSuite) TestColumnModifyingDefinition(c *C) {
s.tk = testkit.NewTestKit(c, s.store)
s.tk.MustExec("use test")
s.tk.MustExec("drop table if exists test2;")
s.tk.MustExec("create table test2 (c1 int, c2 int, c3 int default 1, index (c1));")
s.tk.MustExec("alter table test2 change c2 a bigint not null;")

s.tk.MustExec("drop table if exists test2;")
s.tk.MustExec("create table test2 (c1 int, c2 int, c3 int default 1, index (c1));")
s.tk.MustExec("insert into test2(c2) values (null);")
s.testErrorCode(c, "alter table test2 change c2 a bigint not null;", tmysql.WarnDataTruncated)
}
func (s *testDBSuite) TestPartitionAddIndex(c *C) {
tk := testkit.NewTestKit(c, s.store)
tk.MustExec("use test")
Expand Down
5 changes: 5 additions & 0 deletions ddl/ddl.go
Original file line number Diff line number Diff line change
Expand Up @@ -189,6 +189,9 @@ var (
// ErrUniqueKeyNeedAllFieldsInPf returns must include all columns in the table's partitioning function.
ErrUniqueKeyNeedAllFieldsInPf = terror.ClassDDL.New(codeUniqueKeyNeedAllFieldsInPf, mysql.MySQLErrName[mysql.ErrUniqueKeyNeedAllFieldsInPf])
errWrongExprInPartitionFunc = terror.ClassDDL.New(codeWrongExprInPartitionFunc, mysql.MySQLErrName[mysql.ErrWrongExprInPartitionFunc])

// ErrWarnDataTruncated returns data truncated for column '%s' at row %d.
ErrWarnDataTruncated = terror.ClassDDL.New(codeWarnDataTruncated,mysql.MySQLErrName[mysql.WarnDataTruncated])
)

// DDL is responsible for updating schema in data store and maintaining in-memory InfoSchema cache.
Expand Down Expand Up @@ -608,6 +611,7 @@ const (
codeUniqueKeyNeedAllFieldsInPf = terror.ErrCode(mysql.ErrUniqueKeyNeedAllFieldsInPf)
codePrimaryCantHaveNull = terror.ErrCode(mysql.ErrPrimaryCantHaveNull)
codeWrongExprInPartitionFunc = terror.ErrCode(mysql.ErrWrongExprInPartitionFunc)
codeWarnDataTruncated = terror.ErrCode(mysql.WarnDataTruncated)
)

func init() {
Expand Down Expand Up @@ -655,6 +659,7 @@ func init() {
codeUniqueKeyNeedAllFieldsInPf: mysql.ErrUniqueKeyNeedAllFieldsInPf,
codePrimaryCantHaveNull: mysql.ErrPrimaryCantHaveNull,
codeWrongExprInPartitionFunc: mysql.ErrWrongExprInPartitionFunc,
codeWarnDataTruncated: mysql.WarnDataTruncated,
}
terror.ErrClassToMySQLCodes[terror.ClassDDL] = ddlMySQLErrCodes
}
6 changes: 4 additions & 2 deletions ddl/ddl_api.go
Original file line number Diff line number Diff line change
Expand Up @@ -1635,9 +1635,11 @@ func (d *ddl) getModifiableColumnJob(ctx sessionctx.Context, ident ast.Ident, or
return nil, errUnsupportedModifyColumn.GenByArgs("set auto_increment")
}

// We don't support modifying the type definitions from 'null' to 'not null' now.
// We support modifying the type definitions of 'null ' to 'not null ' now.
if !mysql.HasNotNullFlag(col.Flag) && mysql.HasNotNullFlag(newCol.Flag) {
return nil, errUnsupportedModifyColumn.GenByArgs("null to not null")
if err = CheckForNullValue(ctx, ident.Schema, ident.Name, col.Name, newCol.Name); err != nil {
return nil, errors.Trace(err)
}
}
// As same with MySQL, we don't support modifying the stored status for generated columns.
if err = checkModifyGeneratedColumn(t.Cols(), col, newCol); err != nil {
Expand Down
2 changes: 1 addition & 1 deletion ddl/ddl_worker.go
Original file line number Diff line number Diff line change
Expand Up @@ -491,7 +491,7 @@ func (w *worker) runDDLJob(d *ddlCtx, t *meta.Meta, job *model.Job) (ver int64,
case model.ActionDropColumn:
ver, err = onDropColumn(t, job)
case model.ActionModifyColumn:
ver, err = onModifyColumn(t, job)
ver, err = w.onModifyColumn(t, job)
case model.ActionSetDefaultValue:
ver, err = onSetDefaultValue(t, job)
case model.ActionAddIndex:
Expand Down

0 comments on commit f16f213

Please sign in to comment.