Skip to content

Commit

Permalink
ddl: support modify column (pingcap#1930)
Browse files Browse the repository at this point in the history
* ddl: support modify column
  • Loading branch information
coocood authored Nov 8, 2016
1 parent 2e8941a commit 1056c56
Show file tree
Hide file tree
Showing 11 changed files with 217 additions and 25 deletions.
1 change: 1 addition & 0 deletions ast/ddl.go
Original file line number Diff line number Diff line change
Expand Up @@ -603,6 +603,7 @@ const (
AlterTableDropPrimaryKey
AlterTableDropIndex
AlterTableDropForeignKey
AlterTableModifyColumn

// TODO: Add more actions
)
Expand Down
32 changes: 32 additions & 0 deletions ddl/column.go
Original file line number Diff line number Diff line change
Expand Up @@ -417,3 +417,35 @@ func (d *ddl) backfillColumn(t table.Table, columnInfo *model.ColumnInfo, handle

return nil
}

func (d *ddl) onModifyColumn(t *meta.Meta, job *model.Job) error {
tblInfo, err := d.getTableInfo(t, job)
if err != nil {
return errors.Trace(err)
}
newCol := &model.ColumnInfo{}
err = job.DecodeArgs(newCol)
if err != nil {
job.State = model.JobCancelled
return errors.Trace(err)
}
oldCol := findCol(tblInfo.Columns, newCol.Name.L)
if oldCol == nil || oldCol.State != model.StatePublic {
job.State = model.JobCancelled
return infoschema.ErrColumnNotExists.Gen("column %s doesn't exist", newCol.Name)
}
*oldCol = *newCol
err = t.UpdateTable(job.SchemaID, tblInfo)
if err != nil {
job.State = model.JobCancelled
return errors.Trace(err)
}
ver, err := updateSchemaVersion(t, job)
if err != nil {
return errors.Trace(err)
}
job.SchemaState = model.StatePublic
job.State = model.JobDone
addTableHistoryInfo(job, ver, tblInfo)
return nil
}
34 changes: 34 additions & 0 deletions ddl/column_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -24,6 +24,7 @@ import (
"github.com/pingcap/tidb/kv"
"github.com/pingcap/tidb/model"
"github.com/pingcap/tidb/mysql"
"github.com/pingcap/tidb/parser"
"github.com/pingcap/tidb/table"
"github.com/pingcap/tidb/table/tables"
"github.com/pingcap/tidb/tablecodec"
Expand Down Expand Up @@ -866,3 +867,36 @@ func (s *testColumnSuite) TestDropColumn(c *C) {
d.close()
s.d.start()
}

func (s *testColumnSuite) TestModifyColumn(c *C) {
d := newDDL(s.store, nil, nil, testLease)
cases := []struct {
origin string
to string
ok bool
}{
{"int", "bigint", true},
{"int", "int unsigned", false},
{"varchar(10)", "text", true},
{"varbinary(10)", "blob", true},
{"text", "blob", false},
{"varchar(10)", "varchar(8)", false},
{"varchar(10)", "varchar(11)", true},
}
for _, ca := range cases {
ftA := s.colDefStrToFieldType(c, ca.origin)
ftB := s.colDefStrToFieldType(c, ca.to)
c.Assert(d.modifiable(ftA, ftB), Equals, ca.ok)
}
d.close()
}

func (s *testColumnSuite) colDefStrToFieldType(c *C, str string) *types.FieldType {
sqlA := "alter table t modify column a " + str
stmt, err := parser.New().ParseOneStmt(sqlA, "", "")
c.Assert(err, IsNil)
colDef := stmt.(*ast.AlterTableStmt).Specs[0].Column
col, _, err := columnDefToCol(nil, 0, colDef)
c.Assert(err, IsNil)
return &col.FieldType
}
127 changes: 105 additions & 22 deletions ddl/ddl.go
Original file line number Diff line number Diff line change
Expand Up @@ -55,8 +55,9 @@ var (
errInvalidStoreVer = terror.ClassDDL.New(codeInvalidStoreVer, "invalid storage current version")

// We don't support dropping column with index covered now.
errCantDropColWithIndex = terror.ClassDDL.New(codeCantDropColWithIndex, "can't drop column with index")
errUnsupportedAddColumn = terror.ClassDDL.New(codeUnsupportedAddColumn, "unsupported add column")
errCantDropColWithIndex = terror.ClassDDL.New(codeCantDropColWithIndex, "can't drop column with index")
errUnsupportedAddColumn = terror.ClassDDL.New(codeUnsupportedAddColumn, "unsupported add column")
errUnsupportedModifyColumn = terror.ClassDDL.New(codeUnsupportedModifyColumn, "unsupported modify column")

errBlobKeyWithoutLength = terror.ClassDDL.New(codeBlobKeyWithoutLength, "index for BLOB/TEXT column must specificate a key length")
errIncorrectPrefixKey = terror.ClassDDL.New(codeIncorrectPrefixKey, "Incorrect prefix key; the used key part isn't a string, the used length is longer than the key part, or the storage engine doesn't support unique prefix keys")
Expand Down Expand Up @@ -433,19 +434,28 @@ func (d *ddl) buildColumnsAndConstraints(ctx context.Context, colDefs []*ast.Col
return cols, constraints, nil
}

func (d *ddl) buildColumnAndConstraint(ctx context.Context, offset int,
colDef *ast.ColumnDef) (*table.Column, []*ast.Constraint, error) {
// Set charset.
if len(colDef.Tp.Charset) == 0 {
switch colDef.Tp.Tp {
func (d *ddl) setCharsetCollationFlenDecimal(tp *types.FieldType) {
if len(tp.Charset) == 0 {
switch tp.Tp {
case mysql.TypeString, mysql.TypeVarchar, mysql.TypeVarString, mysql.TypeBlob, mysql.TypeTinyBlob, mysql.TypeMediumBlob, mysql.TypeLongBlob:
colDef.Tp.Charset, colDef.Tp.Collate = getDefaultCharsetAndCollate()
tp.Charset, tp.Collate = getDefaultCharsetAndCollate()
default:
colDef.Tp.Charset = charset.CharsetBin
colDef.Tp.Collate = charset.CharsetBin
tp.Charset = charset.CharsetBin
tp.Collate = charset.CharsetBin
}
}
// If flen is not assigned, assigned it by type.
if tp.Flen == types.UnspecifiedLength {
tp.Flen = mysql.GetDefaultFieldLength(tp.Tp)
}
if tp.Decimal == types.UnspecifiedLength {
tp.Decimal = mysql.GetDefaultDecimal(tp.Tp)
}
}

func (d *ddl) buildColumnAndConstraint(ctx context.Context, offset int,
colDef *ast.ColumnDef) (*table.Column, []*ast.Constraint, error) {
d.setCharsetCollationFlenDecimal(colDef.Tp)
col, cts, err := columnDefToCol(ctx, offset, colDef)
if err != nil {
return nil, nil, errors.Trace(err)
Expand Down Expand Up @@ -475,14 +485,6 @@ func columnDefToCol(ctx context.Context, offset int, colDef *ast.ColumnDef) (*ta
col.Flag |= mysql.NotNullFlag
}

// If flen is not assigned, assigned it by type.
if col.Flen == types.UnspecifiedLength {
col.Flen = mysql.GetDefaultFieldLength(col.Tp)
}
if col.Decimal == types.UnspecifiedLength {
col.Decimal = mysql.GetDefaultDecimal(col.Tp)
}

setOnUpdateNow := false
hasDefaultValue := false
if colDef.Options != nil {
Expand Down Expand Up @@ -952,6 +954,8 @@ func (d *ddl) AlterTable(ctx context.Context, ident ast.Ident, specs []*ast.Alte
}
case ast.AlterTableDropForeignKey:
err = d.DropForeignKey(ctx, ident, model.NewCIStr(spec.Name))
case ast.AlterTableModifyColumn:
err = d.ModifyColumn(ctx, ident, spec)
default:
// Nothing to do now.
}
Expand Down Expand Up @@ -1056,12 +1060,90 @@ func (d *ddl) DropColumn(ctx context.Context, ti ast.Ident, colName model.CIStr)
return errors.Trace(err)
}

// Modifiable checks if the 'origin' type can be modified to 'to' type with out the need to
// change or check existing data in the table.
// It returns true if the two types has the same Charset and Collation, the same sign, both are
// integer types or string types, and new Flen and Decimal must be greater than or equal to origin.
func (d *ddl) modifiable(origin *types.FieldType, to *types.FieldType) bool {
if to.Flen > 0 && to.Flen < origin.Flen {
return false
}
if to.Decimal > 0 && to.Decimal < origin.Decimal {
return false
}
if origin.Charset != to.Charset || origin.Collate != to.Collate {
return false
}
if mysql.HasUnsignedFlag(uint(origin.Flag)) != mysql.HasUnsignedFlag(uint(to.Flag)) {
return false
}
switch origin.Tp {
case mysql.TypeVarchar, mysql.TypeString, mysql.TypeVarString,
mysql.TypeBlob, mysql.TypeTinyBlob, mysql.TypeMediumBlob, mysql.TypeLongBlob:
switch to.Tp {
case mysql.TypeVarchar, mysql.TypeString, mysql.TypeVarString,
mysql.TypeBlob, mysql.TypeTinyBlob, mysql.TypeMediumBlob, mysql.TypeLongBlob:
return true
default:
return false
}
case mysql.TypeTiny, mysql.TypeShort, mysql.TypeInt24, mysql.TypeLong, mysql.TypeLonglong:
switch to.Tp {
case mysql.TypeTiny, mysql.TypeShort, mysql.TypeInt24, mysql.TypeLong, mysql.TypeLonglong:
return true
default:
return false
}
default:
return false
}
}

// ModifyColumn does modification on an existing column, currently we only support limited kind of changes
// that do not need to change or check data on the table.
func (d *ddl) ModifyColumn(ctx context.Context, ident ast.Ident, spec *ast.AlterTableSpec) error {
is := d.infoHandle.Get()
schema, ok := is.SchemaByName(ident.Schema)
if !ok {
return errors.Trace(infoschema.ErrDatabaseNotExists)
}
t, err := is.TableByName(ident.Schema, ident.Name)
if err != nil {
return errors.Trace(infoschema.ErrTableNotExists)
}
colName := spec.Column.Name.Name
col := table.FindCol(t.Cols(), colName.L)
if col == nil {
return infoschema.ErrColumnNotExists.Gen("column %s doesn't exist", colName.O)
}
if spec.Constraint != nil || spec.Position.Tp != ast.ColumnPositionNone ||
len(spec.Column.Options) != 0 || spec.Column.Tp == nil {
// Make sure the column definition is simple field type.
return errUnsupportedModifyColumn
}
d.setCharsetCollationFlenDecimal(spec.Column.Tp)
if !d.modifiable(&col.FieldType, spec.Column.Tp) {
return errUnsupportedModifyColumn
}
newCol := *col
newCol.FieldType = *spec.Column.Tp
job := &model.Job{
SchemaID: schema.ID,
TableID: t.Meta().ID,
Type: model.ActionModifyColumn,
Args: []interface{}{&newCol},
}
err = d.doDDLJob(ctx, job)
err = d.callHookOnChanged(err)
return errors.Trace(err)
}

// DropTable will proceed even if some table in the list does not exists.
func (d *ddl) DropTable(ctx context.Context, ti ast.Ident) (err error) {
is := d.GetInformationSchema()
schema, ok := is.SchemaByName(ti.Schema)
if !ok {
return infoschema.ErrDatabaseNotExists.Gen("database %s not exists", ti.Schema)
return infoschema.ErrDatabaseNotExists.Gen("database %s doesn't exist", ti.Schema)
}

tb, err := is.TableByName(ti.Schema, ti.Name)
Expand All @@ -1084,7 +1166,7 @@ func (d *ddl) TruncateTable(ctx context.Context, ti ast.Ident) error {
is := d.GetInformationSchema()
schema, ok := is.SchemaByName(ti.Schema)
if !ok {
return infoschema.ErrDatabaseNotExists.Gen("database %s not exists", ti.Schema)
return infoschema.ErrDatabaseNotExists.Gen("database %s doesn't exist", ti.Schema)
}
tb, err := is.TableByName(ti.Schema, ti.Name)
if err != nil {
Expand Down Expand Up @@ -1341,8 +1423,9 @@ const (
codeInvalidIndexState = 103
codeInvalidForeignKeyState = 104

codeCantDropColWithIndex = 201
codeUnsupportedAddColumn = 202
codeCantDropColWithIndex = 201
codeUnsupportedAddColumn = 202
codeUnsupportedModifyColumn = 203

codeBadNull = 1048
codeTooLongIdent = 1059
Expand Down
5 changes: 4 additions & 1 deletion ddl/ddl_db_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -575,7 +575,10 @@ LOOP:
if err != nil {
// if err is failed, the column number must be 4 now.
values := s.showColumns(c, "t2")
c.Assert(values, HasLen, 4, Commentf("err:%v", err))
if len(values) != 4 {
c.Log(errors.ErrorStack(err))
c.FailNow()
}
}
}
num += step
Expand Down
2 changes: 2 additions & 0 deletions ddl/ddl_worker.go
Original file line number Diff line number Diff line change
Expand Up @@ -349,6 +349,8 @@ func (d *ddl) runDDLJob(t *meta.Meta, job *model.Job) {
err = d.onAddColumn(t, job)
case model.ActionDropColumn:
err = d.onDropColumn(t, job)
case model.ActionModifyColumn:
err = d.onModifyColumn(t, job)
case model.ActionAddIndex:
err = d.onCreateIndex(t, job)
case model.ActionDropIndex:
Expand Down
25 changes: 24 additions & 1 deletion executor/executor_ddl_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -134,7 +134,7 @@ func (s *testSuite) TestCreateDropIndex(c *C) {
tk.MustExec("drop table drop_test")
}

func (s *testSuite) TestAlterTable(c *C) {
func (s *testSuite) TestAlterTableAddColumn(c *C) {
defer testleak.AfterTest(c)()
tk := testkit.NewTestKit(c, s.store)
tk.MustExec("use test")
Expand All @@ -156,3 +156,26 @@ func (s *testSuite) TestAddNotNullColumnNoDefault(c *C) {
tk.MustExec("insert nn (c1) values (3)")
tk.MustQuery("select * from nn").Check(testkit.Rows("1 0", "2 0", "3 0"))
}

func (s *testSuite) TestAlterTableModifyColumn(c *C) {
defer testleak.AfterTest(c)()
tk := testkit.NewTestKit(c, s.store)
tk.MustExec("use test")
tk.MustExec("create table if not exists mc (c1 int, c2 varchar(10))")
_, err := tk.Exec("alter table mc modify column c1 short")
c.Assert(err, NotNil)
tk.MustExec("alter table mc modify column c1 bigint")

_, err = tk.Exec("alter table mc modify column c2 blob")
c.Assert(err, NotNil)

_, err = tk.Exec("alter table mc modify column c2 varchar(8)")
c.Assert(err, NotNil)
tk.MustExec("alter table mc modify column c2 varchar(11)")
tk.MustExec("alter table mc modify column c2 text(13)")
tk.MustExec("alter table mc modify column c2 text")
result := tk.MustQuery("show create table mc")
createSQL := result.Rows()[0][1]
expected := "CREATE TABLE `mc` (\n `c1` bigint(21) DEFAULT NULL,\n `c2` text DEFAULT NULL\n) ENGINE=InnoDB"
c.Assert(createSQL, Equals, expected)
}
3 changes: 3 additions & 0 deletions model/ddl.go
Original file line number Diff line number Diff line change
Expand Up @@ -39,6 +39,7 @@ const (
ActionAddForeignKey
ActionDropForeignKey
ActionTruncateTable
ActionModifyColumn
)

func (action ActionType) String() string {
Expand All @@ -65,6 +66,8 @@ func (action ActionType) String() string {
return "drop foreign key"
case ActionTruncateTable:
return "truncate table"
case ActionModifyColumn:
return "modify column"
default:
return "none"
}
Expand Down
1 change: 1 addition & 0 deletions parser/misc.go
Original file line number Diff line number Diff line change
Expand Up @@ -297,6 +297,7 @@ var tokenMap = map[string]int{
"MIN_ROWS": minRows,
"MOD": mod,
"MODE": mode,
"MODIFY": modify,
"MONTH": month,
"MONTHNAME": monthname,
"NAMES": names,
Expand Down
11 changes: 10 additions & 1 deletion parser/parser.y
Original file line number Diff line number Diff line change
Expand Up @@ -171,6 +171,7 @@ import (
local "LOCAL"
level "LEVEL"
mode "MODE"
modify "MODIFY"
maxRows "MAX_ROWS"
minRows "MIN_ROWS"
noWriteToBinLog "NO_WRITE_TO_BINLOG"
Expand Down Expand Up @@ -808,6 +809,14 @@ AlterTableSpec:
{
$$ = &ast.AlterTableSpec{}
}
| "MODIFY" ColumnKeywordOpt ColumnDef ColumnPosition
{
$$ = &ast.AlterTableSpec{
Tp: ast.AlterTableModifyColumn,
Column: $3.(*ast.ColumnDef),
Position: $4.(*ast.ColumnPosition),
}
}

KeyOrIndex:
"KEY"|"INDEX"
Expand Down Expand Up @@ -1993,7 +2002,7 @@ UnReservedKeyword:
| "COLLATION" | "COMMENT" | "AVG_ROW_LENGTH" | "CONNECTION" | "CHECKSUM" | "COMPRESSION" | "KEY_BLOCK_SIZE" | "MAX_ROWS"
| "MIN_ROWS" | "NATIONAL" | "ROW" | "ROW_FORMAT" | "QUARTER" | "GRANTS" | "TRIGGERS" | "DELAY_KEY_WRITE" | "ISOLATION"
| "REPEATABLE" | "COMMITTED" | "UNCOMMITTED" | "ONLY" | "SERIALIZABLE" | "LEVEL" | "VARIABLES" | "SQL_CACHE" | "INDEXES" | "PROCESSLIST"
| "SQL_NO_CACHE" | "DISABLE" | "ENABLE" | "REVERSE" | "SPACE" | "PRIVILEGES" | "NO" | "BINLOG" | "FUNCTION" | "VIEW"
| "SQL_NO_CACHE" | "DISABLE" | "ENABLE" | "REVERSE" | "SPACE" | "PRIVILEGES" | "NO" | "BINLOG" | "FUNCTION" | "VIEW" | "MODIFY"

NotKeywordToken:
"ABS" | "ADDDATE" | "ADMIN" | "COALESCE" | "CONCAT" | "CONCAT_WS" | "CONNECTION_ID" | "CUR_TIME"| "COUNT" | "DAY"
Expand Down
1 change: 1 addition & 0 deletions parser/parser_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -239,6 +239,7 @@ func (s *testParserSuite) TestDMLStmt(c *C) {
{"ALTER TABLE t ADD COLUMN a SMALLINT UNSIGNED AFTER b", true},
{"ALTER TABLE t DISABLE KEYS", true},
{"ALTER TABLE t ENABLE KEYS", true},
{"ALTER TABLE t MODIFY COLUMN a varchar(255)", true},

// from join
{"SELECT * from t1, t2, t3", true},
Expand Down

0 comments on commit 1056c56

Please sign in to comment.