diff --git a/ast/misc.go b/ast/misc.go index 6d8afbeea1569..27ec5e9233ee5 100644 --- a/ast/misc.go +++ b/ast/misc.go @@ -13,7 +13,14 @@ package ast -import "github.com/pingcap/tidb/mysql" +import ( + "fmt" + + "github.com/pingcap/tidb/context" + "github.com/pingcap/tidb/model" + "github.com/pingcap/tidb/mysql" + "github.com/pingcap/tidb/sessionctx/db" +) var ( _ StmtNode = &AdminStmt{} @@ -473,3 +480,28 @@ func (n *GrantStmt) Accept(v Visitor) (Node, bool) { } return v.Leave(n) } + +// Ident is the table identifier composed of schema name and table name. +type Ident struct { + Schema model.CIStr + Name model.CIStr +} + +// Full returns an Ident which set schema to the current schema if it is empty. +func (i Ident) Full(ctx context.Context) (full Ident) { + full.Name = i.Name + if i.Schema.O != "" { + full.Schema = i.Schema + } else { + full.Schema = model.NewCIStr(db.GetCurrentSchema(ctx)) + } + return +} + +// String implements fmt.Stringer interface +func (i Ident) String() string { + if i.Schema.O == "" { + return i.Name.O + } + return fmt.Sprintf("%s.%s", i.Schema, i.Name) +} diff --git a/ddl/alter.go b/ddl/alter.go deleted file mode 100644 index 5cfcb0b7b7ab8..0000000000000 --- a/ddl/alter.go +++ /dev/null @@ -1,99 +0,0 @@ -// Copyright 2015 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, -// See the License for the specific language governing permissions and -// limitations under the License. - -package ddl - -import ( - "fmt" - - "github.com/pingcap/tidb/parser/coldef" -) - -// AlterTableSpecification.Action types. -// TODO: Add more actions. -const ( - AlterTableOpt int = iota + 1 - AlterAddColumn - AlterAddConstr - AlterDropColumn - AlterDropPrimaryKey - AlterDropIndex - AlterDropForeignKey -) - -// ColumnPosition Types. -const ( - ColumnPositionNone int = iota - ColumnPositionFirst - ColumnPositionAfter -) - -// ColumnPosition represents the position of the newly added column. -type ColumnPosition struct { - // ColumnPositionNone | ColumnPositionFirst | ColumnPositionAfter - Type int - // RelativeColumn is the column which is before the newly added column if type is ColumnPositionAfter. - RelativeColumn string -} - -// String implements fmt.Stringer. -func (cp *ColumnPosition) String() string { - switch cp.Type { - case ColumnPositionFirst: - return "FIRST" - case ColumnPositionAfter: - return fmt.Sprintf("AFTER %s", cp.RelativeColumn) - default: - return "" - } -} - -// AlterSpecification alter table specification. -type AlterSpecification struct { - Action int - Name string - Constraint *coldef.TableConstraint - TableOpts []*coldef.TableOpt - Column *coldef.ColumnDef - Position *ColumnPosition -} - -// String implements fmt.Stringer. -func (as *AlterSpecification) String() string { - switch as.Action { - case AlterTableOpt: - // TODO: Finish this - return "" - case AlterAddConstr: - if as.Constraint != nil { - return fmt.Sprintf("ADD %s", as.Constraint.String()) - } - return "" - case AlterDropColumn: - return fmt.Sprintf("DROP COLUMN %s", as.Name) - case AlterDropPrimaryKey: - return fmt.Sprintf("DROP PRIMARY KEY") - case AlterDropIndex: - return fmt.Sprintf("DROP INDEX %s", as.Name) - case AlterDropForeignKey: - return fmt.Sprintf("DROP FOREIGN KEY %s", as.Name) - case AlterAddColumn: - ps := as.Position.String() - if len(ps) > 0 { - return fmt.Sprintf("ADD Column %s %s", as.Column.String(), ps) - } - return fmt.Sprintf("ADD Column %s", as.Column.String()) - default: - return "" - } -} diff --git a/ddl/alter_test.go b/ddl/alter_test.go deleted file mode 100644 index daa6b5f5fffb3..0000000000000 --- a/ddl/alter_test.go +++ /dev/null @@ -1,76 +0,0 @@ -// Copyright 2015 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, -// See the License for the specific language governing permissions and -// limitations under the License. - -package ddl - -import ( - . "github.com/pingcap/check" - "github.com/pingcap/tidb/mysql" - "github.com/pingcap/tidb/parser/coldef" - "github.com/pingcap/tidb/util/types" -) - -func (s *testDDLSuite) TestAlterSpecification(c *C) { - tbl := []*AlterSpecification{ - { - Action: AlterTableOpt, - }, - { - Action: AlterDropColumn, - Name: "c1", - }, - {Action: AlterDropPrimaryKey}, - {Action: AlterDropForeignKey, - Name: "c"}, - {Action: AlterDropIndex, - Name: "index_c"}, - {Action: AlterAddConstr, - Constraint: nil}, - {Action: AlterAddConstr, - Constraint: &coldef.TableConstraint{ - Tp: coldef.ConstrPrimaryKey, - Keys: []*coldef.IndexColName{ - { - ColumnName: "a", - Length: 10, - }, - }, - }}, - {Action: AlterAddColumn, - Column: &coldef.ColumnDef{ - Name: "c", - Tp: types.NewFieldType(mysql.TypeLong), - }, - Position: &ColumnPosition{}}, - {Action: AlterAddColumn, - Column: &coldef.ColumnDef{ - Name: "c", - Tp: types.NewFieldType(mysql.TypeLong), - }, - Position: &ColumnPosition{Type: ColumnPositionFirst}}, - {Action: AlterAddColumn, - Column: &coldef.ColumnDef{ - Name: "c", - Tp: types.NewFieldType(mysql.TypeLong), - }, - Position: &ColumnPosition{Type: ColumnPositionAfter, - RelativeColumn: "c"}}, - - // Invalid action returns empty string - {Action: -1}, - } - - for _, spec := range tbl { - c.Assert(len(spec.String()), GreaterEqual, 0) - } -} diff --git a/ddl/column.go b/ddl/column.go index 5a1a91a2ed982..6cfa3aed69258 100644 --- a/ddl/column.go +++ b/ddl/column.go @@ -16,6 +16,7 @@ package ddl import ( "github.com/juju/errors" "github.com/ngaut/log" + "github.com/pingcap/tidb/ast" "github.com/pingcap/tidb/column" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/meta" @@ -55,16 +56,16 @@ func (d *ddl) adjustColumnOffset(columns []*model.ColumnInfo, indices []*model.I } } -func (d *ddl) addColumn(tblInfo *model.TableInfo, colInfo *model.ColumnInfo, pos *ColumnPosition) (*model.ColumnInfo, int, error) { +func (d *ddl) addColumn(tblInfo *model.TableInfo, colInfo *model.ColumnInfo, pos *ast.ColumnPosition) (*model.ColumnInfo, int, error) { // Check column name duplicate. cols := tblInfo.Columns position := len(cols) // Get column position. - if pos.Type == ColumnPositionFirst { + if pos.Tp == ast.ColumnPositionFirst { position = 0 - } else if pos.Type == ColumnPositionAfter { - c := findCol(cols, pos.RelativeColumn) + } else if pos.Tp == ast.ColumnPositionAfter { + c := findCol(cols, pos.RelativeColumn.Name.L) if c == nil { return nil, 0, errors.Errorf("No such column: %v", pos.RelativeColumn) } @@ -96,7 +97,7 @@ func (d *ddl) onAddColumn(t *meta.Meta, job *model.Job) error { } col := &model.ColumnInfo{} - pos := &ColumnPosition{} + pos := &ast.ColumnPosition{} offset := 0 err = job.DecodeArgs(col, pos, &offset) if err != nil { diff --git a/ddl/column_test.go b/ddl/column_test.go index c95288e660aa0..0962fcaf3517c 100644 --- a/ddl/column_test.go +++ b/ddl/column_test.go @@ -17,6 +17,7 @@ import ( "time" . "github.com/pingcap/check" + "github.com/pingcap/tidb/ast" "github.com/pingcap/tidb/column" "github.com/pingcap/tidb/context" "github.com/pingcap/tidb/kv" @@ -59,7 +60,7 @@ func (s *testColumnSuite) TearDownSuite(c *C) { } func testCreateColumn(c *C, ctx context.Context, d *ddl, dbInfo *model.DBInfo, tblInfo *model.TableInfo, - colName string, pos *ColumnPosition, defaultValue interface{}) *model.Job { + colName string, pos *ast.ColumnPosition, defaultValue interface{}) *model.Job { col := &model.ColumnInfo{ Name: model.NewCIStr(colName), Offset: len(tblInfo.Columns), @@ -133,7 +134,7 @@ func (s *testColumnSuite) TestColumn(c *C) { c.Assert(column.FindCol(t.Cols(), "c4"), IsNil) - job := testCreateColumn(c, ctx, s.d, s.dbInfo, tblInfo, "c4", &ColumnPosition{Type: ColumnPositionAfter, RelativeColumn: "c3"}, 100) + job := testCreateColumn(c, ctx, s.d, s.dbInfo, tblInfo, "c4", &ast.ColumnPosition{Tp: ast.ColumnPositionAfter, RelativeColumn: &ast.ColumnName{Name: model.NewCIStr("c3")}}, 100) testCheckJobDone(c, s.d, job, true) t = testGetTable(c, s.d, s.dbInfo.ID, tblInfo.ID) @@ -171,7 +172,7 @@ func (s *testColumnSuite) TestColumn(c *C) { c.Assert(values, HasLen, 3) c.Assert(values[2].GetInt64(), Equals, int64(13)) - job = testCreateColumn(c, ctx, s.d, s.dbInfo, tblInfo, "c4", &ColumnPosition{Type: ColumnPositionNone}, 111) + job = testCreateColumn(c, ctx, s.d, s.dbInfo, tblInfo, "c4", &ast.ColumnPosition{Tp: ast.ColumnPositionNone}, 111) testCheckJobDone(c, s.d, job, true) t = testGetTable(c, s.d, s.dbInfo.ID, tblInfo.ID) @@ -181,7 +182,7 @@ func (s *testColumnSuite) TestColumn(c *C) { c.Assert(values, HasLen, 4) c.Assert(values[3].GetInt64(), Equals, int64(111)) - job = testCreateColumn(c, ctx, s.d, s.dbInfo, tblInfo, "c5", &ColumnPosition{Type: ColumnPositionNone}, 101) + job = testCreateColumn(c, ctx, s.d, s.dbInfo, tblInfo, "c5", &ast.ColumnPosition{Tp: ast.ColumnPositionNone}, 101) testCheckJobDone(c, s.d, job, true) t = testGetTable(c, s.d, s.dbInfo.ID, tblInfo.ID) @@ -191,7 +192,7 @@ func (s *testColumnSuite) TestColumn(c *C) { c.Assert(values, HasLen, 5) c.Assert(values[4].GetInt64(), Equals, int64(101)) - job = testCreateColumn(c, ctx, s.d, s.dbInfo, tblInfo, "c6", &ColumnPosition{Type: ColumnPositionFirst}, 202) + job = testCreateColumn(c, ctx, s.d, s.dbInfo, tblInfo, "c6", &ast.ColumnPosition{Tp: ast.ColumnPositionFirst}, 202) testCheckJobDone(c, s.d, job, true) t = testGetTable(c, s.d, s.dbInfo.ID, tblInfo.ID) @@ -598,7 +599,7 @@ func (s *testColumnSuite) TestAddColumn(c *C) { d.close() d.start() - job := testCreateColumn(c, ctx, d, s.dbInfo, tblInfo, colName, &ColumnPosition{Type: ColumnPositionNone}, defaultColValue) + job := testCreateColumn(c, ctx, d, s.dbInfo, tblInfo, colName, &ast.ColumnPosition{Tp: ast.ColumnPositionNone}, defaultColValue) testCheckJobDone(c, d, job, true) _, err = ctx.GetTxn(true) diff --git a/ddl/ddl.go b/ddl/ddl.go index 6abc7ae2b6890..6c2bdadcf79a6 100644 --- a/ddl/ddl.go +++ b/ddl/ddl.go @@ -25,6 +25,7 @@ import ( "github.com/juju/errors" "github.com/ngaut/log" + "github.com/pingcap/tidb/ast" "github.com/pingcap/tidb/column" "github.com/pingcap/tidb/context" "github.com/pingcap/tidb/evaluator" @@ -33,9 +34,7 @@ import ( "github.com/pingcap/tidb/meta" "github.com/pingcap/tidb/model" "github.com/pingcap/tidb/mysql" - "github.com/pingcap/tidb/parser/coldef" "github.com/pingcap/tidb/sessionctx/variable" - "github.com/pingcap/tidb/table" "github.com/pingcap/tidb/util/charset" "github.com/pingcap/tidb/util/types" "github.com/twinj/uuid" @@ -43,14 +42,14 @@ import ( // DDL is responsible for updating schema in data store and maintaining in-memory InfoSchema cache. type DDL interface { - CreateSchema(ctx context.Context, name model.CIStr, charsetInfo *coldef.CharsetOpt) error + CreateSchema(ctx context.Context, name model.CIStr, charsetInfo *ast.CharsetOpt) error DropSchema(ctx context.Context, schema model.CIStr) error - CreateTable(ctx context.Context, ident table.Ident, cols []*coldef.ColumnDef, constrs []*coldef.TableConstraint) error - DropTable(ctx context.Context, tableIdent table.Ident) (err error) - CreateIndex(ctx context.Context, tableIdent table.Ident, unique bool, indexName model.CIStr, columnNames []*coldef.IndexColName) error - DropIndex(ctx context.Context, tableIdent table.Ident, indexName model.CIStr) error + CreateTable(ctx context.Context, ident ast.Ident, cols []*ast.ColumnDef, constrs []*ast.Constraint) error + DropTable(ctx context.Context, tableIdent ast.Ident) (err error) + CreateIndex(ctx context.Context, tableIdent ast.Ident, unique bool, indexName model.CIStr, columnNames []*ast.IndexColName) error + DropIndex(ctx context.Context, tableIdent ast.Ident, indexName model.CIStr) error GetInformationSchema() infoschema.InfoSchema - AlterTable(ctx context.Context, tableIdent table.Ident, spec []*AlterSpecification) error + AlterTable(ctx context.Context, tableIdent ast.Ident, spec []*ast.AlterTableSpec) error // SetLease will reset the lease time for online DDL change, it is a very dangerous function and you must guarantee that // all servers have the same lease time. SetLease(lease time.Duration) @@ -244,7 +243,7 @@ func (d *ddl) genGlobalID() (int64, error) { return globalID, errors.Trace(err) } -func (d *ddl) CreateSchema(ctx context.Context, schema model.CIStr, charsetInfo *coldef.CharsetOpt) (err error) { +func (d *ddl) CreateSchema(ctx context.Context, schema model.CIStr, charsetInfo *ast.CharsetOpt) (err error) { is := d.GetInformationSchema() _, ok := is.SchemaByName(schema) if ok { @@ -300,11 +299,11 @@ func getDefaultCharsetAndCollate() (string, string) { return "utf8", "utf8_unicode_ci" } -func setColumnFlagWithConstraint(colMap map[string]*column.Col, v *coldef.TableConstraint) { +func setColumnFlagWithConstraint(colMap map[string]*column.Col, v *ast.Constraint) { switch v.Tp { - case coldef.ConstrPrimaryKey: + case ast.ConstraintPrimaryKey: for _, key := range v.Keys { - c, ok := colMap[strings.ToLower(key.ColumnName)] + c, ok := colMap[key.Column.Name.L] if !ok { // TODO: table constraint on unknown column. continue @@ -313,9 +312,9 @@ func setColumnFlagWithConstraint(colMap map[string]*column.Col, v *coldef.TableC // Primary key can not be NULL. c.Flag |= mysql.NotNullFlag } - case coldef.ConstrUniq, coldef.ConstrUniqIndex, coldef.ConstrUniqKey: + case ast.ConstraintUniq, ast.ConstraintUniqIndex, ast.ConstraintUniqKey: for i, key := range v.Keys { - c, ok := colMap[strings.ToLower(key.ColumnName)] + c, ok := colMap[key.Column.Name.L] if !ok { // TODO: table constraint on unknown column. continue @@ -332,9 +331,9 @@ func setColumnFlagWithConstraint(colMap map[string]*column.Col, v *coldef.TableC } } } - case coldef.ConstrKey, coldef.ConstrIndex: + case ast.ConstraintKey, ast.ConstraintIndex: for i, key := range v.Keys { - c, ok := colMap[strings.ToLower(key.ColumnName)] + c, ok := colMap[key.Column.Name.L] if !ok { // TODO: table constraint on unknown column. continue @@ -347,8 +346,8 @@ func setColumnFlagWithConstraint(colMap map[string]*column.Col, v *coldef.TableC } } -func (d *ddl) buildColumnsAndConstraints(ctx context.Context, colDefs []*coldef.ColumnDef, - constraints []*coldef.TableConstraint) ([]*column.Col, []*coldef.TableConstraint, error) { +func (d *ddl) buildColumnsAndConstraints(ctx context.Context, colDefs []*ast.ColumnDef, + constraints []*ast.Constraint) ([]*column.Col, []*ast.Constraint, error) { var cols []*column.Col colMap := map[string]*column.Col{} for i, colDef := range colDefs { @@ -359,7 +358,7 @@ func (d *ddl) buildColumnsAndConstraints(ctx context.Context, colDefs []*coldef. col.State = model.StatePublic constraints = append(constraints, cts...) cols = append(cols, col) - colMap[strings.ToLower(colDef.Name)] = col + colMap[colDef.Name.Name.L] = col } // traverse table Constraints and set col.flag for _, v := range constraints { @@ -369,7 +368,7 @@ func (d *ddl) buildColumnsAndConstraints(ctx context.Context, colDefs []*coldef. } func (d *ddl) buildColumnAndConstraint(ctx context.Context, offset int, - colDef *coldef.ColumnDef) (*column.Col, []*coldef.TableConstraint, error) { + colDef *ast.ColumnDef) (*column.Col, []*ast.Constraint, error) { // Set charset. if len(colDef.Tp.Charset) == 0 { switch colDef.Tp.Tp { @@ -395,12 +394,12 @@ func (d *ddl) buildColumnAndConstraint(ctx context.Context, offset int, } // columnDefToCol converts ColumnDef to Col and TableConstraints. -func columnDefToCol(ctx context.Context, offset int, colDef *coldef.ColumnDef) (*column.Col, []*coldef.TableConstraint, error) { - constraints := []*coldef.TableConstraint{} +func columnDefToCol(ctx context.Context, offset int, colDef *ast.ColumnDef) (*column.Col, []*ast.Constraint, error) { + constraints := []*ast.Constraint{} col := &column.Col{ ColumnInfo: model.ColumnInfo{ Offset: offset, - Name: model.NewCIStr(colDef.Name), + Name: colDef.Name.Name, FieldType: *colDef.Tp, }, } @@ -422,45 +421,45 @@ func columnDefToCol(ctx context.Context, offset int, colDef *coldef.ColumnDef) ( setOnUpdateNow := false hasDefaultValue := false - if colDef.Constraints != nil { - keys := []*coldef.IndexColName{ + if colDef.Options != nil { + keys := []*ast.IndexColName{ { - colDef.Name, - colDef.Tp.Flen, + Column: colDef.Name, + Length: colDef.Tp.Flen, }, } - for _, v := range colDef.Constraints { + for _, v := range colDef.Options { switch v.Tp { - case coldef.ConstrNotNull: + case ast.ColumnOptionNotNull: col.Flag |= mysql.NotNullFlag - case coldef.ConstrNull: + case ast.ColumnOptionNull: col.Flag &= ^uint(mysql.NotNullFlag) removeOnUpdateNowFlag(col) - case coldef.ConstrAutoIncrement: + case ast.ColumnOptionAutoIncrement: col.Flag |= mysql.AutoIncrementFlag - case coldef.ConstrPrimaryKey: - constraint := &coldef.TableConstraint{Tp: coldef.ConstrPrimaryKey, Keys: keys} + case ast.ColumnOptionPrimaryKey: + constraint := &ast.Constraint{Tp: ast.ConstraintPrimaryKey, Keys: keys} constraints = append(constraints, constraint) col.Flag |= mysql.PriKeyFlag - case coldef.ConstrUniq: - constraint := &coldef.TableConstraint{Tp: coldef.ConstrUniq, ConstrName: colDef.Name, Keys: keys} + case ast.ColumnOptionUniq: + constraint := &ast.Constraint{Tp: ast.ConstraintUniq, Name: colDef.Name.Name.O, Keys: keys} constraints = append(constraints, constraint) col.Flag |= mysql.UniqueKeyFlag - case coldef.ConstrIndex: - constraint := &coldef.TableConstraint{Tp: coldef.ConstrIndex, ConstrName: colDef.Name, Keys: keys} + case ast.ColumnOptionIndex: + constraint := &ast.Constraint{Tp: ast.ConstraintIndex, Name: colDef.Name.Name.O, Keys: keys} constraints = append(constraints, constraint) - case coldef.ConstrUniqIndex: - constraint := &coldef.TableConstraint{Tp: coldef.ConstrUniqIndex, ConstrName: colDef.Name, Keys: keys} + case ast.ColumnOptionUniqIndex: + constraint := &ast.Constraint{Tp: ast.ConstraintUniqIndex, Name: colDef.Name.Name.O, Keys: keys} constraints = append(constraints, constraint) col.Flag |= mysql.UniqueKeyFlag - case coldef.ConstrKey: - constraint := &coldef.TableConstraint{Tp: coldef.ConstrKey, ConstrName: colDef.Name, Keys: keys} + case ast.ColumnOptionKey: + constraint := &ast.Constraint{Tp: ast.ConstraintKey, Name: colDef.Name.Name.O, Keys: keys} constraints = append(constraints, constraint) - case coldef.ConstrUniqKey: - constraint := &coldef.TableConstraint{Tp: coldef.ConstrUniqKey, ConstrName: colDef.Name, Keys: keys} + case ast.ColumnOptionUniqKey: + constraint := &ast.Constraint{Tp: ast.ConstraintUniqKey, Name: colDef.Name.Name.O, Keys: keys} constraints = append(constraints, constraint) col.Flag |= mysql.UniqueKeyFlag - case coldef.ConstrDefaultValue: + case ast.ColumnOptionDefaultValue: value, err := getDefaultValue(ctx, v, colDef.Tp.Tp, colDef.Tp.Decimal) if err != nil { return nil, nil, errors.Errorf("invalid default value - %s", errors.Trace(err)) @@ -468,16 +467,14 @@ func columnDefToCol(ctx context.Context, offset int, colDef *coldef.ColumnDef) ( col.DefaultValue = value hasDefaultValue = true removeOnUpdateNowFlag(col) - case coldef.ConstrOnUpdate: - if !evaluator.IsCurrentTimeExpr(v.Evalue) { + case ast.ColumnOptionOnUpdate: + if !evaluator.IsCurrentTimeExpr(v.Expr) { return nil, nil, errors.Errorf("invalid ON UPDATE for - %s", col.Name) } col.Flag |= mysql.OnUpdateNowFlag setOnUpdateNow = true - case coldef.ConstrFulltext: - // Do nothing. - case coldef.ConstrComment: + case ast.ColumnOptionFulltext, ast.ColumnOptionComment: // Do nothing. } } @@ -499,9 +496,9 @@ func columnDefToCol(ctx context.Context, offset int, colDef *coldef.ColumnDef) ( return col, constraints, nil } -func getDefaultValue(ctx context.Context, c *coldef.ConstraintOpt, tp byte, fsp int) (interface{}, error) { +func getDefaultValue(ctx context.Context, c *ast.ColumnOption, tp byte, fsp int) (interface{}, error) { if tp == mysql.TypeTimestamp || tp == mysql.TypeDatetime { - value, err := evaluator.GetTimeValue(ctx, c.Evalue, tp, fsp) + value, err := evaluator.GetTimeValue(ctx, c.Expr, tp, fsp) if err != nil { return nil, errors.Trace(err) } @@ -518,7 +515,7 @@ func getDefaultValue(ctx context.Context, c *coldef.ConstraintOpt, tp byte, fsp return value, nil } - v, err := evaluator.Eval(ctx, c.Evalue) + v, err := evaluator.Eval(ctx, c.Expr) if err != nil { return nil, errors.Trace(err) } @@ -580,10 +577,10 @@ func checkDefaultValue(c *column.Col, hasDefaultValue bool) error { return nil } -func checkDuplicateColumn(colDefs []*coldef.ColumnDef) error { +func checkDuplicateColumn(colDefs []*ast.ColumnDef) error { colNames := map[string]bool{} for _, colDef := range colDefs { - nameLower := strings.ToLower(colDef.Name) + nameLower := colDef.Name.Name.O if colNames[nameLower] { return errors.Errorf("CREATE TABLE: duplicate column %s", colDef.Name) } @@ -592,15 +589,15 @@ func checkDuplicateColumn(colDefs []*coldef.ColumnDef) error { return nil } -func checkConstraintNames(constraints []*coldef.TableConstraint) error { +func checkConstraintNames(constraints []*ast.Constraint) error { constrNames := map[string]bool{} // Check not empty constraint name whether is duplicated. for _, constr := range constraints { - if constr.ConstrName != "" { - nameLower := strings.ToLower(constr.ConstrName) + if constr.Name != "" { + nameLower := strings.ToLower(constr.Name) if constrNames[nameLower] { - return errors.Errorf("CREATE TABLE: duplicate key %s", constr.ConstrName) + return errors.Errorf("CREATE TABLE: duplicate key %s", constr.Name) } constrNames[nameLower] = true } @@ -608,8 +605,8 @@ func checkConstraintNames(constraints []*coldef.TableConstraint) error { // Set empty constraint names. for _, constr := range constraints { - if constr.ConstrName == "" && len(constr.Keys) > 0 { - colName := constr.Keys[0].ColumnName + if constr.Name == "" && len(constr.Keys) > 0 { + colName := constr.Keys[0].Column.Name.O constrName := colName i := 2 for constrNames[strings.ToLower(constrName)] { @@ -617,14 +614,14 @@ func checkConstraintNames(constraints []*coldef.TableConstraint) error { constrName = fmt.Sprintf("%s_%d", colName, i) i++ } - constr.ConstrName = constrName + constr.Name = constrName constrNames[constrName] = true } } return nil } -func (d *ddl) buildTableInfo(tableName model.CIStr, cols []*column.Col, constraints []*coldef.TableConstraint) (tbInfo *model.TableInfo, err error) { +func (d *ddl) buildTableInfo(tableName model.CIStr, cols []*column.Col, constraints []*ast.Constraint) (tbInfo *model.TableInfo, err error) { tbInfo = &model.TableInfo{ Name: tableName, } @@ -636,10 +633,10 @@ func (d *ddl) buildTableInfo(tableName model.CIStr, cols []*column.Col, constrai tbInfo.Columns = append(tbInfo.Columns, &v.ColumnInfo) } for _, constr := range constraints { - if constr.Tp == coldef.ConstrPrimaryKey { + if constr.Tp == ast.ConstraintPrimaryKey { if len(constr.Keys) == 1 { key := constr.Keys[0] - col := column.FindCol(cols, key.ColumnName) + col := column.FindCol(cols, key.Column.Name.O) if col == nil { return nil, errors.Errorf("No such column: %v", key) } @@ -656,27 +653,27 @@ func (d *ddl) buildTableInfo(tableName model.CIStr, cols []*column.Col, constrai // 2. add index indexColumns := make([]*model.IndexColumn, 0, len(constr.Keys)) for _, key := range constr.Keys { - col := column.FindCol(cols, key.ColumnName) + col := column.FindCol(cols, key.Column.Name.O) if col == nil { return nil, errors.Errorf("No such column: %v", key) } indexColumns = append(indexColumns, &model.IndexColumn{ - Name: model.NewCIStr(key.ColumnName), + Name: key.Column.Name, Offset: col.Offset, Length: key.Length, }) } idxInfo := &model.IndexInfo{ - Name: model.NewCIStr(constr.ConstrName), + Name: model.NewCIStr(constr.Name), Columns: indexColumns, State: model.StatePublic, } switch constr.Tp { - case coldef.ConstrPrimaryKey: + case ast.ConstraintPrimaryKey: idxInfo.Unique = true idxInfo.Primary = true idxInfo.Name = model.NewCIStr(column.PrimaryKeyName) - case coldef.ConstrUniq, coldef.ConstrUniqKey, coldef.ConstrUniqIndex: + case ast.ConstraintUniq, ast.ConstraintUniqKey, ast.ConstraintUniqIndex: idxInfo.Unique = true } idxInfo.ID, err = d.genGlobalID() @@ -688,7 +685,7 @@ func (d *ddl) buildTableInfo(tableName model.CIStr, cols []*column.Col, constrai return } -func (d *ddl) CreateTable(ctx context.Context, ident table.Ident, colDefs []*coldef.ColumnDef, constraints []*coldef.TableConstraint) (err error) { +func (d *ddl) CreateTable(ctx context.Context, ident ast.Ident, colDefs []*ast.ColumnDef, constraints []*ast.Constraint) (err error) { is := d.GetInformationSchema() schema, ok := is.SchemaByName(ident.Schema) if !ok { @@ -728,27 +725,27 @@ func (d *ddl) CreateTable(ctx context.Context, ident table.Ident, colDefs []*col return errors.Trace(err) } -func (d *ddl) AlterTable(ctx context.Context, ident table.Ident, specs []*AlterSpecification) (err error) { +func (d *ddl) AlterTable(ctx context.Context, ident ast.Ident, specs []*ast.AlterTableSpec) (err error) { // now we only allow one schema changes at the same time. if len(specs) != 1 { return errors.New("can't run multi schema changes in one DDL") } for _, spec := range specs { - switch spec.Action { - case AlterAddColumn: + switch spec.Tp { + case ast.AlterTableAddColumn: err = d.AddColumn(ctx, ident, spec) - case AlterDropColumn: - err = d.DropColumn(ctx, ident, model.NewCIStr(spec.Name)) - case AlterDropIndex: + case ast.AlterTableDropColumn: + err = d.DropColumn(ctx, ident, spec.DropColumn.Name) + case ast.AlterTableDropIndex: err = d.DropIndex(ctx, ident, model.NewCIStr(spec.Name)) - case AlterAddConstr: + case ast.AlterTableAddConstraint: constr := spec.Constraint switch spec.Constraint.Tp { - case coldef.ConstrKey, coldef.ConstrIndex: - err = d.CreateIndex(ctx, ident, false, model.NewCIStr(constr.ConstrName), spec.Constraint.Keys) - case coldef.ConstrUniq, coldef.ConstrUniqIndex, coldef.ConstrUniqKey: - err = d.CreateIndex(ctx, ident, true, model.NewCIStr(constr.ConstrName), spec.Constraint.Keys) + case ast.ConstraintKey, ast.ConstraintIndex: + err = d.CreateIndex(ctx, ident, false, model.NewCIStr(constr.Name), spec.Constraint.Keys) + case ast.ConstraintUniq, ast.ConstraintUniqIndex, ast.ConstraintUniqKey: + err = d.CreateIndex(ctx, ident, true, model.NewCIStr(constr.Name), spec.Constraint.Keys) default: // nothing to do now. } @@ -764,11 +761,11 @@ func (d *ddl) AlterTable(ctx context.Context, ident table.Ident, specs []*AlterS return nil } -func checkColumnConstraint(constraints []*coldef.ConstraintOpt) error { +func checkColumnConstraint(constraints []*ast.ColumnOption) error { for _, constraint := range constraints { switch constraint.Tp { - case coldef.ConstrAutoIncrement, coldef.ConstrForeignKey, coldef.ConstrPrimaryKey, coldef.ConstrUniq, coldef.ConstrUniqKey: - return errors.Errorf("unsupported add column constraint - %s", constraint) + case ast.ColumnOptionAutoIncrement, ast.ColumnOptionPrimaryKey, ast.ColumnOptionUniq, ast.ColumnOptionUniqKey: + return errors.Errorf("unsupported add column constraint - %v", constraint.Tp) } } @@ -776,9 +773,9 @@ func checkColumnConstraint(constraints []*coldef.ConstraintOpt) error { } // AddColumn will add a new column to the table. -func (d *ddl) AddColumn(ctx context.Context, ti table.Ident, spec *AlterSpecification) error { +func (d *ddl) AddColumn(ctx context.Context, ti ast.Ident, spec *ast.AlterTableSpec) error { // Check whether the added column constraints are supported. - err := checkColumnConstraint(spec.Column.Constraints) + err := checkColumnConstraint(spec.Column.Options) if err != nil { return errors.Trace(err) } @@ -795,7 +792,7 @@ func (d *ddl) AddColumn(ctx context.Context, ti table.Ident, spec *AlterSpecific } // Check whether added column has existed. - colName := spec.Column.Name + colName := spec.Column.Name.Name.O col := column.FindCol(t.Cols(), colName) if col != nil { return errors.Errorf("column %s already exists", colName) @@ -822,7 +819,7 @@ func (d *ddl) AddColumn(ctx context.Context, ti table.Ident, spec *AlterSpecific } // DropColumn will drop a column from the table, now we don't support drop the column with index covered. -func (d *ddl) DropColumn(ctx context.Context, ti table.Ident, colName model.CIStr) error { +func (d *ddl) DropColumn(ctx context.Context, ti ast.Ident, colName model.CIStr) error { is := d.infoHandle.Get() schema, ok := is.SchemaByName(ti.Schema) if !ok { @@ -853,7 +850,7 @@ func (d *ddl) DropColumn(ctx context.Context, ti table.Ident, colName model.CISt } // DropTable will proceed even if some table in the list does not exists. -func (d *ddl) DropTable(ctx context.Context, ti table.Ident) (err error) { +func (d *ddl) DropTable(ctx context.Context, ti ast.Ident) (err error) { is := d.GetInformationSchema() schema, ok := is.SchemaByName(ti.Schema) if !ok { @@ -876,7 +873,7 @@ func (d *ddl) DropTable(ctx context.Context, ti table.Ident) (err error) { return errors.Trace(err) } -func (d *ddl) CreateIndex(ctx context.Context, ti table.Ident, unique bool, indexName model.CIStr, idxColNames []*coldef.IndexColName) error { +func (d *ddl) CreateIndex(ctx context.Context, ti ast.Ident, unique bool, indexName model.CIStr, idxColNames []*ast.IndexColName) error { is := d.infoHandle.Get() schema, ok := is.SchemaByName(ti.Schema) if !ok { @@ -904,7 +901,7 @@ func (d *ddl) CreateIndex(ctx context.Context, ti table.Ident, unique bool, inde return errors.Trace(err) } -func (d *ddl) DropIndex(ctx context.Context, ti table.Ident, indexName model.CIStr) error { +func (d *ddl) DropIndex(ctx context.Context, ti ast.Ident, indexName model.CIStr) error { is := d.infoHandle.Get() schema, ok := is.SchemaByName(ti.Schema) if !ok { diff --git a/ddl/ddl_test.go b/ddl/ddl_test.go index 7c7642803645e..01cbf07dfb603 100644 --- a/ddl/ddl_test.go +++ b/ddl/ddl_test.go @@ -19,8 +19,8 @@ import ( "github.com/ngaut/log" . "github.com/pingcap/check" "github.com/pingcap/tidb" + "github.com/pingcap/tidb/ast" "github.com/pingcap/tidb/kv" - "github.com/pingcap/tidb/parser/coldef" ) func TestT(t *testing.T) { @@ -31,14 +31,14 @@ var _ = Suite(&testSuite{}) type testSuite struct { store kv.Storage - charsetInfo *coldef.CharsetOpt + charsetInfo *ast.CharsetOpt } func (ts *testSuite) SetUpSuite(c *C) { store, err := tidb.NewStore(tidb.EngineGoLevelDBMemory) c.Assert(err, IsNil) ts.store = store - ts.charsetInfo = &coldef.CharsetOpt{ + ts.charsetInfo = &ast.CharsetOpt{ Chs: "utf8", Col: "utf8_bin", } diff --git a/ddl/ddl_worker_test.go b/ddl/ddl_worker_test.go index 4c69fe2469c6c..c96621abc7bfd 100644 --- a/ddl/ddl_worker_test.go +++ b/ddl/ddl_worker_test.go @@ -19,11 +19,11 @@ import ( "time" . "github.com/pingcap/check" + "github.com/pingcap/tidb/ast" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/meta" "github.com/pingcap/tidb/model" "github.com/pingcap/tidb/mysql" - "github.com/pingcap/tidb/parser/coldef" "github.com/pingcap/tidb/store/localstore" "github.com/pingcap/tidb/store/localstore/goleveldb" "github.com/pingcap/tidb/terror" @@ -251,7 +251,7 @@ func (s *testDDLSuite) TestIndexError(c *C) { SchemaID: dbInfo.ID, TableID: tblInfo.ID, Type: model.ActionAddIndex, - Args: []interface{}{false, model.NewCIStr("t"), []*coldef.IndexColName{{ColumnName: "c", Length: 256}}}, + Args: []interface{}{false, model.NewCIStr("t"), []*ast.IndexColName{{Column: &ast.ColumnName{Name: model.NewCIStr("c")}, Length: 256}}}, } err = d.startDDLJob(ctx, job) c.Assert(err, NotNil) @@ -261,7 +261,7 @@ func (s *testDDLSuite) TestIndexError(c *C) { SchemaID: dbInfo.ID, TableID: tblInfo.ID, Type: model.ActionAddIndex, - Args: []interface{}{false, model.NewCIStr("c1_index"), []*coldef.IndexColName{{ColumnName: "c", Length: 256}}}, + Args: []interface{}{false, model.NewCIStr("c1_index"), []*ast.IndexColName{{Column: &ast.ColumnName{Name: model.NewCIStr("c")}, Length: 256}}}, } err = d.startDDLJob(ctx, job) c.Assert(err, NotNil) @@ -273,7 +273,7 @@ func (s *testDDLSuite) TestIndexError(c *C) { SchemaID: dbInfo.ID, TableID: tblInfo.ID, Type: model.ActionAddIndex, - Args: []interface{}{false, model.NewCIStr("c1_index"), []*coldef.IndexColName{{ColumnName: "c1", Length: 256}}}, + Args: []interface{}{false, model.NewCIStr("c1_index"), []*ast.IndexColName{{Column: &ast.ColumnName{Name: model.NewCIStr("c1")}, Length: 256}}}, } err = d.startDDLJob(ctx, job) c.Assert(err, NotNil) @@ -349,7 +349,7 @@ func (s *testDDLSuite) TestColumnError(c *C) { c.Assert(err, IsNil) col.FieldType = *types.NewFieldType(mysql.TypeLong) - pos := &ColumnPosition{Type: ColumnPositionAfter, RelativeColumn: "c5"} + pos := &ast.ColumnPosition{Tp: ast.ColumnPositionAfter, RelativeColumn: &ast.ColumnName{Name: model.NewCIStr("c5")}} job = &model.Job{ SchemaID: dbInfo.ID, diff --git a/ddl/index.go b/ddl/index.go index bd62948f9455c..d4f5fc49902f3 100644 --- a/ddl/index.go +++ b/ddl/index.go @@ -16,11 +16,11 @@ package ddl import ( "github.com/juju/errors" "github.com/ngaut/log" + "github.com/pingcap/tidb/ast" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/meta" "github.com/pingcap/tidb/model" "github.com/pingcap/tidb/mysql" - "github.com/pingcap/tidb/parser/coldef" "github.com/pingcap/tidb/table" "github.com/pingcap/tidb/table/tables" "github.com/pingcap/tidb/terror" @@ -28,13 +28,13 @@ import ( "github.com/pingcap/tidb/util/types" ) -func buildIndexInfo(tblInfo *model.TableInfo, unique bool, indexName model.CIStr, indexID int64, idxColNames []*coldef.IndexColName) (*model.IndexInfo, error) { +func buildIndexInfo(tblInfo *model.TableInfo, unique bool, indexName model.CIStr, indexID int64, idxColNames []*ast.IndexColName) (*model.IndexInfo, error) { // build offsets idxColumns := make([]*model.IndexColumn, 0, len(idxColNames)) for _, ic := range idxColNames { - col := findCol(tblInfo.Columns, ic.ColumnName) + col := findCol(tblInfo.Columns, ic.Column.Name.O) if col == nil { - return nil, errors.Errorf("CREATE INDEX: column does not exist: %s", ic.ColumnName) + return nil, errors.Errorf("CREATE INDEX: column does not exist: %s", ic.Column.Name.O) } idxColumns = append(idxColumns, &model.IndexColumn{ @@ -98,7 +98,7 @@ func (d *ddl) onCreateIndex(t *meta.Meta, job *model.Job) error { unique bool indexName model.CIStr indexID int64 - idxColNames []*coldef.IndexColName + idxColNames []*ast.IndexColName ) err = job.DecodeArgs(&unique, &indexName, &indexID, &idxColNames) diff --git a/ddl/index_test.go b/ddl/index_test.go index 43399e7df845f..e5d31c0544122 100644 --- a/ddl/index_test.go +++ b/ddl/index_test.go @@ -18,11 +18,11 @@ import ( "time" . "github.com/pingcap/check" + "github.com/pingcap/tidb/ast" "github.com/pingcap/tidb/column" "github.com/pingcap/tidb/context" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/model" - "github.com/pingcap/tidb/parser/coldef" "github.com/pingcap/tidb/table" "github.com/pingcap/tidb/table/tables" "github.com/pingcap/tidb/util/mock" @@ -66,7 +66,7 @@ func testCreateIndex(c *C, ctx context.Context, d *ddl, dbInfo *model.DBInfo, tb SchemaID: dbInfo.ID, TableID: tblInfo.ID, Type: model.ActionAddIndex, - Args: []interface{}{unique, model.NewCIStr(indexName), id, []*coldef.IndexColName{{ColumnName: colName, Length: 256}}}, + Args: []interface{}{unique, model.NewCIStr(indexName), id, []*ast.IndexColName{{Column: &ast.ColumnName{Name: model.NewCIStr(colName)}, Length: 256}}}, } err = d.startDDLJob(ctx, job) diff --git a/domain/domain_test.go b/domain/domain_test.go index f31e556479910..f521d1d757aba 100644 --- a/domain/domain_test.go +++ b/domain/domain_test.go @@ -19,8 +19,8 @@ import ( "time" . "github.com/pingcap/check" + "github.com/pingcap/tidb/ast" "github.com/pingcap/tidb/model" - "github.com/pingcap/tidb/parser/coldef" "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/store/localstore" "github.com/pingcap/tidb/store/localstore/goleveldb" @@ -49,7 +49,7 @@ func (*testSuite) TestT(c *C) { store = dom.Store() dd := dom.DDL() c.Assert(dd, NotNil) - cs := &coldef.CharsetOpt{ + cs := &ast.CharsetOpt{ Chs: "utf8", Col: "utf8_bin", } diff --git a/driver.go b/driver.go index 9073bd0c3cfd4..f8b95421ac623 100644 --- a/driver.go +++ b/driver.go @@ -31,7 +31,6 @@ import ( "github.com/juju/errors" "github.com/pingcap/tidb/ast" "github.com/pingcap/tidb/model" - "github.com/pingcap/tidb/parser/coldef" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/terror" "github.com/pingcap/tidb/util/types" @@ -203,7 +202,7 @@ func (d *sqlDriver) Open(dsn string) (driver.Conn, error) { DBName := model.NewCIStr(params.dbName) domain := sessionctx.GetDomain(s) - cs := &coldef.CharsetOpt{ + cs := &ast.CharsetOpt{ Chs: "utf8", Col: "utf8_bin", } diff --git a/executor/executor_ddl.go b/executor/executor_ddl.go index b43867e2f2429..328c60ae0ed1a 100644 --- a/executor/executor_ddl.go +++ b/executor/executor_ddl.go @@ -19,14 +19,11 @@ import ( "github.com/juju/errors" "github.com/pingcap/tidb/ast" "github.com/pingcap/tidb/context" - "github.com/pingcap/tidb/ddl" "github.com/pingcap/tidb/infoschema" "github.com/pingcap/tidb/model" "github.com/pingcap/tidb/mysql" - "github.com/pingcap/tidb/parser/coldef" "github.com/pingcap/tidb/privilege" "github.com/pingcap/tidb/sessionctx" - "github.com/pingcap/tidb/table" "github.com/pingcap/tidb/terror" ) @@ -88,9 +85,9 @@ func (e *DDLExec) executeTruncateTable(s *ast.TruncateTableStmt) error { } func (e *DDLExec) executeCreateDatabase(s *ast.CreateDatabaseStmt) error { - var opt *coldef.CharsetOpt + var opt *ast.CharsetOpt if len(s.Options) != 0 { - opt = &coldef.CharsetOpt{} + opt = &ast.CharsetOpt{} for _, val := range s.Options { switch val.Tp { case ast.DatabaseOptionCharset: @@ -110,26 +107,8 @@ func (e *DDLExec) executeCreateDatabase(s *ast.CreateDatabaseStmt) error { } func (e *DDLExec) executeCreateTable(s *ast.CreateTableStmt) error { - ident := table.Ident{Schema: s.Table.Schema, Name: s.Table.Name} - - var coldefs []*coldef.ColumnDef - for _, val := range s.Cols { - coldef, err := convertColumnDef(val) - if err != nil { - return errors.Trace(err) - } - coldefs = append(coldefs, coldef) - } - var constrs []*coldef.TableConstraint - for _, val := range s.Constraints { - constr, err := convertConstraint(val) - if err != nil { - return errors.Trace(err) - } - constrs = append(constrs, constr) - } - - err := sessionctx.GetDomain(e.ctx).DDL().CreateTable(e.ctx, ident, coldefs, constrs) + ident := ast.Ident{Schema: s.Table.Schema, Name: s.Table.Name} + err := sessionctx.GetDomain(e.ctx).DDL().CreateTable(e.ctx, ident, s.Cols, s.Constraints) if terror.ErrorEqual(err, infoschema.TableExists) { if s.IfNotExists { return nil @@ -139,105 +118,9 @@ func (e *DDLExec) executeCreateTable(s *ast.CreateTableStmt) error { return errors.Trace(err) } -func convertColumnDef(v *ast.ColumnDef) (*coldef.ColumnDef, error) { - oldColDef := &coldef.ColumnDef{ - Name: v.Name.Name.O, - Tp: v.Tp, - } - for _, val := range v.Options { - oldOpt, err := convertColumnOption(val) - if err != nil { - return nil, errors.Trace(err) - } - oldColDef.Constraints = append(oldColDef.Constraints, oldOpt) - } - return oldColDef, nil -} - -func convertColumnOption(v *ast.ColumnOption) (*coldef.ConstraintOpt, error) { - oldColumnOpt := &coldef.ConstraintOpt{} - switch v.Tp { - case ast.ColumnOptionAutoIncrement: - oldColumnOpt.Tp = coldef.ConstrAutoIncrement - case ast.ColumnOptionComment: - oldColumnOpt.Tp = coldef.ConstrComment - case ast.ColumnOptionDefaultValue: - oldColumnOpt.Tp = coldef.ConstrDefaultValue - case ast.ColumnOptionIndex: - oldColumnOpt.Tp = coldef.ConstrIndex - case ast.ColumnOptionKey: - oldColumnOpt.Tp = coldef.ConstrKey - case ast.ColumnOptionFulltext: - oldColumnOpt.Tp = coldef.ConstrFulltext - case ast.ColumnOptionNotNull: - oldColumnOpt.Tp = coldef.ConstrNotNull - case ast.ColumnOptionNoOption: - oldColumnOpt.Tp = coldef.ConstrNoConstr - case ast.ColumnOptionOnUpdate: - oldColumnOpt.Tp = coldef.ConstrOnUpdate - case ast.ColumnOptionPrimaryKey: - oldColumnOpt.Tp = coldef.ConstrPrimaryKey - case ast.ColumnOptionNull: - oldColumnOpt.Tp = coldef.ConstrNull - case ast.ColumnOptionUniq: - oldColumnOpt.Tp = coldef.ConstrUniq - case ast.ColumnOptionUniqIndex: - oldColumnOpt.Tp = coldef.ConstrUniqIndex - case ast.ColumnOptionUniqKey: - oldColumnOpt.Tp = coldef.ConstrUniqKey - } - oldColumnOpt.Evalue = v.Expr - return oldColumnOpt, nil -} - -func convertConstraint(v *ast.Constraint) (*coldef.TableConstraint, error) { - oldConstraint := &coldef.TableConstraint{ConstrName: v.Name} - switch v.Tp { - case ast.ConstraintNoConstraint: - oldConstraint.Tp = coldef.ConstrNoConstr - case ast.ConstraintPrimaryKey: - oldConstraint.Tp = coldef.ConstrPrimaryKey - case ast.ConstraintKey: - oldConstraint.Tp = coldef.ConstrKey - case ast.ConstraintIndex: - oldConstraint.Tp = coldef.ConstrIndex - case ast.ConstraintUniq: - oldConstraint.Tp = coldef.ConstrUniq - case ast.ConstraintUniqKey: - oldConstraint.Tp = coldef.ConstrUniqKey - case ast.ConstraintUniqIndex: - oldConstraint.Tp = coldef.ConstrUniqIndex - case ast.ConstraintForeignKey: - oldConstraint.Tp = coldef.ConstrForeignKey - case ast.ConstraintFulltext: - oldConstraint.Tp = coldef.ConstrFulltext - } - oldConstraint.Keys = convertIndexColNames(v.Keys) - if v.Refer != nil { - oldConstraint.Refer = &coldef.ReferenceDef{ - TableIdent: table.Ident{Schema: v.Refer.Table.Schema, Name: v.Refer.Table.Name}, - IndexColNames: convertIndexColNames(v.Refer.IndexColNames), - } - } - return oldConstraint, nil -} - -func convertIndexColNames(v []*ast.IndexColName) (out []*coldef.IndexColName) { - for _, val := range v { - oldIndexColKey := &coldef.IndexColName{ - ColumnName: val.Column.Name.O, - Length: val.Length, - } - out = append(out, oldIndexColKey) - } - return -} - func (e *DDLExec) executeCreateIndex(s *ast.CreateIndexStmt) error { - ident := table.Ident{Schema: s.Table.Schema, Name: s.Table.Name} - - colNames := convertIndexColNames(s.IndexColNames) - err := sessionctx.GetDomain(e.ctx).DDL().CreateIndex(e.ctx, ident, s.Unique, model.NewCIStr(s.IndexName), colNames) + ident := ast.Ident{Schema: s.Table.Schema, Name: s.Table.Name} + err := sessionctx.GetDomain(e.ctx).DDL().CreateIndex(e.ctx, ident, s.Unique, model.NewCIStr(s.IndexName), s.IndexColNames) return errors.Trace(err) } @@ -256,7 +139,7 @@ func (e *DDLExec) executeDropDatabase(s *ast.DropDatabaseStmt) error { func (e *DDLExec) executeDropTable(s *ast.DropTableStmt) error { var notExistTables []string for _, tn := range s.Tables { - fullti := table.Ident{Schema: tn.Schema, Name: tn.Name} + fullti := ast.Ident{Schema: tn.Schema, Name: tn.Name} schema, ok := e.is.SchemaByName(tn.Schema) if !ok { // TODO: we should return special error for table not exist, checking "not exist" is not enough, @@ -295,7 +178,7 @@ func (e *DDLExec) executeDropTable(s *ast.DropTableStmt) error { } func (e *DDLExec) executeDropIndex(s *ast.DropIndexStmt) error { - ti := table.Ident{Schema: s.Table.Schema, Name: s.Table.Name} + ti := ast.Ident{Schema: s.Table.Schema, Name: s.Table.Name} err := sessionctx.GetDomain(e.ctx).DDL().DropIndex(e.ctx, ti, model.NewCIStr(s.IndexName)) if (infoschema.DatabaseNotExists.Equal(err) || infoschema.TableNotExists.Equal(err)) && s.IfExists { err = nil @@ -304,112 +187,11 @@ func (e *DDLExec) executeDropIndex(s *ast.DropIndexStmt) error { } func (e *DDLExec) executeAlterTable(s *ast.AlterTableStmt) error { - ti := table.Ident{Schema: s.Table.Schema, Name: s.Table.Name} - var specs []*ddl.AlterSpecification - for _, v := range s.Specs { - spec, err := convertAlterTableSpec(v) - if err != nil { - return errors.Trace(err) - } - specs = append(specs, spec) - } - err := sessionctx.GetDomain(e.ctx).DDL().AlterTable(e.ctx, ti, specs) + ti := ast.Ident{Schema: s.Table.Schema, Name: s.Table.Name} + err := sessionctx.GetDomain(e.ctx).DDL().AlterTable(e.ctx, ti, s.Specs) return errors.Trace(err) } -func convertAlterTableSpec(v *ast.AlterTableSpec) (*ddl.AlterSpecification, error) { - oldAlterSpec := &ddl.AlterSpecification{ - Name: v.Name, - } - switch v.Tp { - case ast.AlterTableAddConstraint: - oldAlterSpec.Action = ddl.AlterAddConstr - case ast.AlterTableAddColumn: - oldAlterSpec.Action = ddl.AlterAddColumn - case ast.AlterTableDropColumn: - oldAlterSpec.Action = ddl.AlterDropColumn - case ast.AlterTableDropForeignKey: - oldAlterSpec.Action = ddl.AlterDropForeignKey - case ast.AlterTableDropIndex: - oldAlterSpec.Action = ddl.AlterDropIndex - case ast.AlterTableDropPrimaryKey: - oldAlterSpec.Action = ddl.AlterDropPrimaryKey - case ast.AlterTableOption: - oldAlterSpec.Action = ddl.AlterTableOpt - } - if v.Column != nil { - oldColDef, err := convertColumnDef(v.Column) - if err != nil { - return nil, errors.Trace(err) - } - oldAlterSpec.Column = oldColDef - } - if v.Position != nil { - oldAlterSpec.Position = &ddl.ColumnPosition{} - switch v.Position.Tp { - case ast.ColumnPositionNone: - oldAlterSpec.Position.Type = ddl.ColumnPositionNone - case ast.ColumnPositionFirst: - oldAlterSpec.Position.Type = ddl.ColumnPositionFirst - case ast.ColumnPositionAfter: - oldAlterSpec.Position.Type = ddl.ColumnPositionAfter - } - if v.Position.RelativeColumn != nil { - oldAlterSpec.Position.RelativeColumn = joinColumnName(v.Position.RelativeColumn) - } - } - if v.DropColumn != nil { - oldAlterSpec.Name = joinColumnName(v.DropColumn) - } - if v.Constraint != nil { - oldConstraint, err := convertConstraint(v.Constraint) - if err != nil { - return nil, errors.Trace(err) - } - oldAlterSpec.Constraint = oldConstraint - } - for _, val := range v.Options { - oldOpt := &coldef.TableOpt{ - StrValue: val.StrValue, - UintValue: val.UintValue, - } - switch val.Tp { - case ast.TableOptionNone: - oldOpt.Tp = coldef.TblOptNone - case ast.TableOptionEngine: - oldOpt.Tp = coldef.TblOptEngine - case ast.TableOptionCharset: - oldOpt.Tp = coldef.TblOptCharset - case ast.TableOptionCollate: - oldOpt.Tp = coldef.TblOptCollate - case ast.TableOptionAutoIncrement: - oldOpt.Tp = coldef.TblOptAutoIncrement - case ast.TableOptionComment: - oldOpt.Tp = coldef.TblOptComment - case ast.TableOptionAvgRowLength: - oldOpt.Tp = coldef.TblOptAvgRowLength - case ast.TableOptionCheckSum: - oldOpt.Tp = coldef.TblOptCheckSum - case ast.TableOptionCompression: - oldOpt.Tp = coldef.TblOptCompression - case ast.TableOptionConnection: - oldOpt.Tp = coldef.TblOptConnection - case ast.TableOptionPassword: - oldOpt.Tp = coldef.TblOptPassword - case ast.TableOptionKeyBlockSize: - oldOpt.Tp = coldef.TblOptKeyBlockSize - case ast.TableOptionMaxRows: - oldOpt.Tp = coldef.TblOptMaxRows - case ast.TableOptionMinRows: - oldOpt.Tp = coldef.TblOptMinRows - case ast.TableOptionDelayKeyWrite: - oldOpt.Tp = coldef.TblOptDelayKeyWrite - } - oldAlterSpec.TableOpts = append(oldAlterSpec.TableOpts, oldOpt) - } - return oldAlterSpec, nil -} - func joinColumnName(columnName *ast.ColumnName) string { var originStrs []string if columnName.Schema.O != "" { diff --git a/parser/coldef/col_def.go b/parser/coldef/col_def.go deleted file mode 100644 index ad1a9618fda9d..0000000000000 --- a/parser/coldef/col_def.go +++ /dev/null @@ -1,79 +0,0 @@ -// Copyright 2015 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, -// See the License for the specific language governing permissions and -// limitations under the License. - -package coldef - -import ( - "fmt" - "strings" - - "github.com/pingcap/tidb/table" - "github.com/pingcap/tidb/util/types" -) - -// ReferenceDef is used for parsing foreign key reference option from SQL. -// See: http://dev.mysql.com/doc/refman/5.7/en/create-table-foreign-keys.html -type ReferenceDef struct { - TableIdent table.Ident - IndexColNames []*IndexColName -} - -// String implements fmt.Stringer interface. -func (rd *ReferenceDef) String() string { - cns := make([]string, 0, len(rd.IndexColNames)) - for _, icn := range rd.IndexColNames { - cns = append(cns, icn.String()) - } - return fmt.Sprintf("REFERENCES %s (%s)", rd.TableIdent, strings.Join(cns, ", ")) -} - -// Clone clones a new ReferenceDef from old ReferenceDef. -func (rd *ReferenceDef) Clone() *ReferenceDef { - cnames := make([]*IndexColName, 0, len(rd.IndexColNames)) - for _, idxColName := range rd.IndexColNames { - t := *idxColName - cnames = append(cnames, &t) - } - return &ReferenceDef{TableIdent: rd.TableIdent, IndexColNames: cnames} -} - -// IndexColName is used for parsing index column name from SQL. -type IndexColName struct { - ColumnName string - Length int -} - -// String implements fmt.Stringer interface. -func (icn *IndexColName) String() string { - if icn.Length >= 0 { - return fmt.Sprintf("%s(%d)", icn.ColumnName, icn.Length) - } - return icn.ColumnName -} - -// ColumnDef is used for parsing column definition from SQL. -type ColumnDef struct { - Name string - Tp *types.FieldType - Constraints []*ConstraintOpt -} - -// String implements fmt.Stringer interface. -func (c *ColumnDef) String() string { - ans := []string{c.Name} - - for _, x := range c.Constraints { - ans = append(ans, x.String()) - } - return strings.Join(ans, " ") -} diff --git a/parser/coldef/opt.go b/parser/coldef/opt.go deleted file mode 100644 index c35cda20ddc53..0000000000000 --- a/parser/coldef/opt.go +++ /dev/null @@ -1,281 +0,0 @@ -// Copyright 2015 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, -// See the License for the specific language governing permissions and -// limitations under the License. - -package coldef - -import ( - "fmt" - "strings" - - "github.com/pingcap/tidb/ast" - "github.com/pingcap/tidb/mysql" -) - -// FloatOpt is used for parsing floating-point type option from SQL. -// TODO: add reference doc. -type FloatOpt struct { - Flen int - Decimal int -} - -// CharsetOpt is used for parsing charset option from SQL. -type CharsetOpt struct { - Chs string - Col string -} - -// String implements fmt.Stringer interface. -func (o *CharsetOpt) String() string { - var ss []string - if o.Chs != "" { - ss = append(ss, "CHARACTER SET = "+o.Chs) - } - if o.Col != "" { - ss = append(ss, "COLLATE = "+o.Col) - } - return strings.Join(ss, " ") -} - -// ConstraintOpt is used for parsing column constraint info from SQL. -type ConstraintOpt struct { - Tp int - Bvalue bool - Evalue ast.ExprNode -} - -// String implements fmt.Stringer interface. -func (c *ConstraintOpt) String() string { - switch c.Tp { - case ConstrNotNull: - return "NOT NULL" - case ConstrNull: - return "NULL" - case ConstrAutoIncrement: - return "AUTO_INCREMENT" - case ConstrPrimaryKey: - return "PRIMARY KEY" - case ConstrUniq: - return "UNIQUE" - case ConstrUniqKey: - return "UNIQUE KEY" - case ConstrDefaultValue: - return "DEFAULT " + ast.ToString(c.Evalue) - case ConstrOnUpdate: - return "ON UPDATE " + ast.ToString(c.Evalue) - default: - return "" - } -} - -// DB Options. -const ( - DBOptNone = iota - DBOptCharset - DBOptCollate -) - -// DatabaseOpt is used for parsing database option from SQL. -type DatabaseOpt struct { - Tp int - Value string -} - -// Constraints. -const ( - ConstrNoConstr = iota - ConstrPrimaryKey - ConstrForeignKey - ConstrNotNull - ConstrAutoIncrement - ConstrDefaultValue - ConstrUniq - ConstrIndex - ConstrUniqIndex - ConstrKey - ConstrUniqKey - ConstrNull - ConstrOnUpdate - ConstrFulltext - ConstrComment -) - -// LockType is select lock type. -type LockType int - -// Select Lock Type. -const ( - SelectLockNone LockType = iota - SelectLockForUpdate - SelectLockInShareMode -) - -// Table Options. -const ( - TblOptNone = iota - TblOptEngine - TblOptCharset - TblOptCollate - TblOptAutoIncrement - TblOptComment - TblOptAvgRowLength - TblOptCheckSum - TblOptCompression - TblOptConnection - TblOptPassword - TblOptKeyBlockSize - TblOptMaxRows - TblOptMinRows - TblOptDelayKeyWrite -) - -// TableOpt is used for parsing table option from SQL. -type TableOpt struct { - Tp int - StrValue string - UintValue uint64 -} - -// TableOption is the collection of table options. -// TODO: rename TableOpt or TableOption. -type TableOption struct { - Engine string - Charset string - Collate string - AutoIncrement uint64 // TODO: apply this value to autoid.Allocator. -} - -// String implements fmt.Stringer interface. -func (o *TableOption) String() string { - strs := []string{} - if o.Engine != "" { - x := fmt.Sprintf("ENGINE=%s", o.Engine) - strs = append(strs, x) - } - if o.Charset != "" { - x := fmt.Sprintf("CHARSET=%s", o.Charset) - strs = append(strs, x) - } - if o.Collate != "" { - x := fmt.Sprintf("COLLATE=%s", o.Collate) - strs = append(strs, x) - } - - return strings.Join(strs, " ") -} - -// TableConstraint is constraint for table definition. -type TableConstraint struct { - Tp int - ConstrName string - - // Used for PRIMARY KEY, UNIQUE, ...... - Keys []*IndexColName - - // Used for foreign key. - Refer *ReferenceDef -} - -// Clone clones a new TableConstraint from old TableConstraint. -func (tc *TableConstraint) Clone() *TableConstraint { - keys := make([]*IndexColName, 0, len(tc.Keys)) - for _, k := range tc.Keys { - keys = append(keys, k) - } - ntc := &TableConstraint{ - Tp: tc.Tp, - ConstrName: tc.ConstrName, - Keys: keys, - } - if tc.Refer != nil { - ntc.Refer = tc.Refer.Clone() - } - return ntc -} - -// String implements fmt.Stringer interface. -func (tc *TableConstraint) String() string { - tokens := []string{} - if tc.Tp == ConstrPrimaryKey { - tokens = append(tokens, "PRIMARY KEY") - } else { - if tc.Tp == ConstrKey { - tokens = append(tokens, "KEY") - } else if tc.Tp == ConstrIndex { - tokens = append(tokens, "INDEX") - } else if tc.Tp == ConstrUniq { - tokens = append(tokens, "UNIQUE") - } else if tc.Tp == ConstrUniqKey { - tokens = append(tokens, "UNIQUE KEY") - } else if tc.Tp == ConstrUniqIndex { - tokens = append(tokens, "UNIQUE INDEX") - } else if tc.Tp == ConstrForeignKey { - tokens = append(tokens, "FOREIGN KEY") - } - tokens = append(tokens, tc.ConstrName) - } - keysStr := make([]string, 0, len(tc.Keys)) - for _, v := range tc.Keys { - keysStr = append(keysStr, v.String()) - } - tokens = append(tokens, fmt.Sprintf("(%s)", strings.Join(keysStr, ", "))) - if tc.Refer != nil { - tokens = append(tokens, tc.Refer.String()) - } - return strings.Join(tokens, " ") -} - -// AuthOption is used for parsing create user statement. -// TODO: support auth_plugin -type AuthOption struct { - // AuthString/HashString can be empty, so we need to decide which one to use. - ByAuthString bool - AuthString string - HashString string -} - -// UserSpecification is used for parsing create user statement. -type UserSpecification struct { - User string - AuthOpt *AuthOption -} - -// PrivElem is the privilege type and optional column list. -type PrivElem struct { - Priv mysql.PrivilegeType - Cols []string -} - -const ( - // ObjectTypeNone is for empty object type. - ObjectTypeNone = iota - // ObjectTypeTable means the following object is a table. - ObjectTypeTable -) - -const ( - // GrantLevelNone is the dummy const for default value. - GrantLevelNone = iota - // GrantLevelGlobal means the privileges are administrative or apply to all databases on a given server. - GrantLevelGlobal - // GrantLevelDB means the privileges apply to all objects in a given database. - GrantLevelDB - // GrantLevelTable means the privileges apply to all columns in a given table. - GrantLevelTable -) - -// GrantLevel is used for store the privilege scope. -type GrantLevel struct { - Level int - DBName string - TableName string -} diff --git a/parser/parser.go b/parser/parser.go index 804f66ed289b9..c7b46bf1a1aa6 100644 --- a/parser/parser.go +++ b/parser/parser.go @@ -734,8 +734,8 @@ var ( 57372: 354, // by (3x) 57673: 355, // ByItem (3x) 57677: 356, // ColumnDef (3x) - 57691: 357, // Constraint (3x) - 57393: 358, // constraint (3x) + 57393: 357, // constraint (3x) + 57691: 358, // Constraint (3x) 57693: 359, // ConstraintKeywordOpt (3x) 57715: 360, // DeleteFromStmt (3x) 57740: 361, // FieldOpt (3x) @@ -1302,8 +1302,8 @@ var ( "by", "ByItem", "ColumnDef", - "Constraint", "constraint", + "Constraint", "ConstraintKeywordOpt", "DeleteFromStmt", "FieldOpt", @@ -2214,7 +2214,7 @@ var ( 698: {473, 1}, 699: {522, 1}, 700: {522, 3}, - 701: {357, 2}, + 701: {358, 2}, 702: {438, 1}, 703: {438, 1}, 704: {438, 4}, @@ -5315,7 +5315,7 @@ var ( {673, 673, 673, 673, 6: 673, 9: 673, 673, 673, 673, 673, 673, 673, 673, 673, 673, 673, 673, 673, 673, 673, 673, 673, 673, 673, 673, 673, 673, 673, 673, 673, 673, 673, 673, 673, 673, 673, 673, 673, 673, 673, 673, 673, 673, 673, 673, 673, 673, 673, 673, 673, 673, 673, 673, 673, 673, 673, 673, 673, 673, 673, 673, 673, 673, 673, 673, 673, 673, 673, 673, 673, 673, 673, 673, 673, 673, 673, 673, 673, 673, 673, 673, 673, 673, 673, 673, 673, 673, 673, 673, 673, 673, 673, 673, 673, 673, 673, 673, 673, 673, 673, 673, 673, 673, 673, 673, 673, 673, 673, 673, 673, 673, 673, 673, 673, 673, 673, 673, 123: 673}, {927, 976, 928, 945, 6: 934, 9: 950, 977, 979, 980, 978, 991, 942, 981, 982, 983, 941, 957, 959, 951, 944, 999, 968, 935, 938, 937, 1009, 989, 1018, 992, 1023, 1025, 1027, 949, 986, 1032, 961, 965, 966, 970, 1002, 930, 936, 939, 940, 974, 1028, 953, 955, 956, 964, 973, 931, 933, 932, 975, 994, 943, 946, 958, 988, 998, 947, 969, 984, 996, 993, 997, 954, 960, 962, 963, 990, 995, 967, 1000, 1001, 971, 929, 1033, 1003, 1004, 1005, 1006, 1008, 1007, 1010, 1011, 1012, 1013, 1014, 1015, 987, 1016, 1017, 924, 1019, 1020, 1021, 1022, 1024, 948, 1026, 1029, 1030, 952, 1031, 985, 972, 1034, 1035, 1036, 1037, 1038, 1039, 1040, 1041, 1042, 198: 1043, 926, 925, 295: 2047}, {125: 2048}, - {927, 976, 928, 945, 6: 934, 9: 950, 977, 979, 980, 978, 991, 942, 981, 982, 983, 941, 957, 959, 951, 944, 999, 968, 935, 938, 937, 1009, 989, 1018, 992, 1023, 1025, 1027, 949, 986, 1032, 961, 965, 966, 970, 1002, 930, 936, 939, 940, 974, 1028, 953, 955, 956, 964, 973, 931, 933, 932, 975, 994, 943, 946, 958, 988, 998, 947, 969, 984, 996, 993, 997, 954, 960, 962, 963, 990, 995, 967, 1000, 1001, 971, 929, 1033, 1003, 1004, 1005, 1006, 1008, 1007, 1010, 1011, 1012, 1013, 1014, 1015, 987, 1016, 1017, 924, 1019, 1020, 1021, 1022, 1024, 948, 1026, 1029, 1030, 952, 1031, 985, 972, 1034, 1035, 1036, 1037, 1038, 1039, 1040, 1041, 1042, 195: 834, 834, 2055, 1075, 926, 925, 234: 2050, 236: 834, 238: 834, 255: 834, 257: 834, 356: 2053, 2054, 2049, 2052, 438: 2056, 525: 2051}, + {927, 976, 928, 945, 6: 934, 9: 950, 977, 979, 980, 978, 991, 942, 981, 982, 983, 941, 957, 959, 951, 944, 999, 968, 935, 938, 937, 1009, 989, 1018, 992, 1023, 1025, 1027, 949, 986, 1032, 961, 965, 966, 970, 1002, 930, 936, 939, 940, 974, 1028, 953, 955, 956, 964, 973, 931, 933, 932, 975, 994, 943, 946, 958, 988, 998, 947, 969, 984, 996, 993, 997, 954, 960, 962, 963, 990, 995, 967, 1000, 1001, 971, 929, 1033, 1003, 1004, 1005, 1006, 1008, 1007, 1010, 1011, 1012, 1013, 1014, 1015, 987, 1016, 1017, 924, 1019, 1020, 1021, 1022, 1024, 948, 1026, 1029, 1030, 952, 1031, 985, 972, 1034, 1035, 1036, 1037, 1038, 1039, 1040, 1041, 1042, 195: 834, 834, 2055, 1075, 926, 925, 234: 2050, 236: 834, 238: 834, 255: 834, 257: 834, 356: 2053, 2049, 2054, 2052, 438: 2056, 525: 2051}, // 1195 {927, 976, 928, 945, 6: 934, 9: 950, 977, 979, 980, 978, 991, 942, 981, 982, 983, 941, 957, 959, 951, 944, 999, 968, 935, 938, 937, 1009, 989, 1018, 992, 1023, 1025, 1027, 949, 986, 1032, 961, 965, 966, 970, 1002, 930, 936, 939, 940, 974, 1028, 953, 955, 956, 964, 973, 931, 933, 932, 975, 994, 943, 946, 958, 988, 998, 947, 969, 984, 996, 993, 997, 954, 960, 962, 963, 990, 995, 967, 1000, 1001, 971, 929, 1033, 1003, 1004, 1005, 1006, 1008, 1007, 1010, 1011, 1012, 1013, 1014, 1015, 987, 1016, 1017, 924, 1019, 1020, 1021, 1022, 1024, 948, 1026, 1029, 1030, 952, 1031, 985, 972, 1034, 1035, 1036, 1037, 1038, 1039, 1040, 1041, 1042, 195: 833, 833, 198: 2309, 926, 925, 236: 833, 238: 833, 255: 833, 257: 833, 437: 2308}, {25: 2219, 27: 2216, 2215, 39: 2218, 55: 2198, 2192, 2191, 67: 2206, 72: 2212, 2217, 145: 2205, 189: 2200, 253: 87, 258: 2193, 2189, 261: 87, 263: 2190, 2208, 2197, 2204, 2175, 2176, 2195, 2177, 2188, 2210, 2214, 2209, 2187, 2213, 2194, 279: 2196, 2186, 2178, 2207, 2185, 2211, 2180, 2179, 2201, 453: 2184, 2202, 466: 2174, 478: 2182, 2183, 486: 2181, 492: 2199, 2172, 523: 2173, 529: 2203, 532: 2171}, @@ -5398,7 +5398,7 @@ var ( {7: 2115, 2077}, {4: 800, 800, 7: 800, 800}, {2123, 2124, 4: 132, 132, 760, 9: 2128, 2125, 2127, 2129, 2126, 2133, 2121, 2130, 2131, 2132, 128: 2120, 142: 760, 202: 760, 324: 2122, 341: 2135, 370: 2134, 2119}, - {927, 976, 928, 945, 6: 934, 9: 950, 977, 979, 980, 978, 991, 942, 981, 982, 983, 941, 957, 959, 951, 944, 999, 968, 935, 938, 937, 1009, 989, 1018, 992, 1023, 1025, 1027, 949, 986, 1032, 961, 965, 966, 970, 1002, 930, 936, 939, 940, 974, 1028, 953, 955, 956, 964, 973, 931, 933, 932, 975, 994, 943, 946, 958, 988, 998, 947, 969, 984, 996, 993, 997, 954, 960, 962, 963, 990, 995, 967, 1000, 1001, 971, 929, 1033, 1003, 1004, 1005, 1006, 1008, 1007, 1010, 1011, 1012, 1013, 1014, 1015, 987, 1016, 1017, 924, 1019, 1020, 1021, 1022, 1024, 948, 1026, 1029, 1030, 952, 1031, 985, 972, 1034, 1035, 1036, 1037, 1038, 1039, 1040, 1041, 1042, 195: 834, 834, 2055, 1075, 926, 925, 234: 2050, 236: 834, 238: 834, 255: 834, 257: 834, 356: 2053, 2054, 2049, 2052, 438: 2118}, + {927, 976, 928, 945, 6: 934, 9: 950, 977, 979, 980, 978, 991, 942, 981, 982, 983, 941, 957, 959, 951, 944, 999, 968, 935, 938, 937, 1009, 989, 1018, 992, 1023, 1025, 1027, 949, 986, 1032, 961, 965, 966, 970, 1002, 930, 936, 939, 940, 974, 1028, 953, 955, 956, 964, 973, 931, 933, 932, 975, 994, 943, 946, 958, 988, 998, 947, 969, 984, 996, 993, 997, 954, 960, 962, 963, 990, 995, 967, 1000, 1001, 971, 929, 1033, 1003, 1004, 1005, 1006, 1008, 1007, 1010, 1011, 1012, 1013, 1014, 1015, 987, 1016, 1017, 924, 1019, 1020, 1021, 1022, 1024, 948, 1026, 1029, 1030, 952, 1031, 985, 972, 1034, 1035, 1036, 1037, 1038, 1039, 1040, 1041, 1042, 195: 834, 834, 2055, 1075, 926, 925, 234: 2050, 236: 834, 238: 834, 255: 834, 257: 834, 356: 2053, 2049, 2054, 2052, 438: 2118}, {7: 148, 148}, // 1265 {4: 764, 764}, @@ -5661,7 +5661,7 @@ var ( // 1480 {4: 851, 851, 8: 2358}, {4: 850, 850, 8: 850}, - {841, 841, 841, 841, 6: 841, 9: 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 195: 834, 834, 236: 834, 238: 834, 255: 834, 257: 834, 357: 2352, 2049, 2052, 384: 2345, 2351}, + {841, 841, 841, 841, 6: 841, 9: 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 195: 834, 834, 236: 834, 238: 834, 255: 834, 257: 834, 357: 2049, 2352, 2052, 384: 2345, 2351}, {841, 841, 841, 841, 6: 841, 9: 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 841, 195: 2340, 236: 2343, 238: 2344, 255: 2342, 384: 2345, 2339, 488: 2341}, {4: 836, 836, 8: 836}, // 1485 diff --git a/privilege/privileges/privileges.go b/privilege/privileges/privileges.go index 05b7aa4210f81..7ff6588062754 100644 --- a/privilege/privileges/privileges.go +++ b/privilege/privileges/privileges.go @@ -18,10 +18,10 @@ import ( "strings" "github.com/juju/errors" + "github.com/pingcap/tidb/ast" "github.com/pingcap/tidb/context" "github.com/pingcap/tidb/model" "github.com/pingcap/tidb/mysql" - "github.com/pingcap/tidb/parser/coldef" "github.com/pingcap/tidb/privilege" "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/util/sqlexec" @@ -31,7 +31,7 @@ import ( var _ privilege.Checker = (*UserPrivileges)(nil) type privileges struct { - Level int + Level ast.GrantLevelType privs map[mysql.PrivilegeType]bool } @@ -52,11 +52,11 @@ func (ps *privileges) add(p mysql.PrivilegeType) { func (ps *privileges) String() string { switch ps.Level { - case coldef.GrantLevelGlobal: + case ast.GrantLevelGlobal: return ps.globalPrivToString() - case coldef.GrantLevelDB: + case ast.GrantLevelDB: return ps.dbPrivToString() - case coldef.GrantLevelTable: + case ast.GrantLevelTable: return ps.tablePrivToString() } return "" @@ -246,7 +246,7 @@ func (p *UserPrivileges) loadGlobalPrivileges(ctx context.Context) error { return errors.Trace(err) } defer rs.Close() - ps := &privileges{Level: coldef.GrantLevelGlobal} + ps := &privileges{Level: ast.GrantLevelGlobal} fs, err := rs.Fields() if err != nil { return errors.Trace(err) @@ -303,7 +303,7 @@ func (p *UserPrivileges) loadDBScopePrivileges(ctx context.Context) error { } // DB dbStr := row.Data[1].GetString() - ps[dbStr] = &privileges{Level: coldef.GrantLevelDB} + ps[dbStr] = &privileges{Level: ast.GrantLevelDB} for i := dbTablePrivColumnStartIndex; i < len(fs); i++ { d := row.Data[i] if d.Kind() != types.KindMysqlEnum { @@ -350,7 +350,7 @@ func (p *UserPrivileges) loadTableScopePrivileges(ctx context.Context) error { if !ok { ps[dbStr] = make(map[string]*privileges) } - ps[dbStr][tblStr] = &privileges{Level: coldef.GrantLevelTable} + ps[dbStr][tblStr] = &privileges{Level: ast.GrantLevelTable} // Table_priv tblPrivs := row.Data[6].GetMysqlSet() pvs := strings.Split(tblPrivs.Name, ",") diff --git a/table/table.go b/table/table.go index 17b8aef96e9ba..6f79aa153e62f 100644 --- a/table/table.go +++ b/table/table.go @@ -18,8 +18,6 @@ package table import ( - "fmt" - "github.com/juju/errors" "github.com/pingcap/tidb/column" "github.com/pingcap/tidb/context" @@ -28,7 +26,6 @@ import ( "github.com/pingcap/tidb/meta/autoid" "github.com/pingcap/tidb/model" "github.com/pingcap/tidb/mysql" - "github.com/pingcap/tidb/sessionctx/db" "github.com/pingcap/tidb/util/types" ) @@ -93,33 +90,6 @@ type Table interface { // Currently, it is assigned to tables.TableFromMeta in tidb package's init function. var TableFromMeta func(alloc autoid.Allocator, tblInfo *model.TableInfo) (Table, error) -// Ident is the table identifier composed of schema name and table name. -// TODO: Move out -type Ident struct { - Schema model.CIStr - Name model.CIStr -} - -// Full returns an Ident which set schema to the current schema if it is empty. -func (i Ident) Full(ctx context.Context) (full Ident) { - full.Name = i.Name - full.Schema = i.Schema - if i.Schema.O != "" { - full.Schema = i.Schema - } else { - full.Schema = model.NewCIStr(db.GetCurrentSchema(ctx)) - } - return -} - -// String implements fmt.Stringer interface -func (i Ident) String() string { - if i.Schema.O == "" { - return i.Name.O - } - return fmt.Sprintf("%s.%s", i.Schema, i.Name) -} - // GetColDefaultValue gets default value of the column. func GetColDefaultValue(ctx context.Context, col *model.ColumnInfo) (types.Datum, bool, error) { // Check no default value flag. diff --git a/table/table_test.go b/table/table_test.go index 9727d4a9a10ad..669d225d448c7 100644 --- a/table/table_test.go +++ b/table/table_test.go @@ -18,12 +18,12 @@ import ( . "github.com/pingcap/check" "github.com/pingcap/tidb" + "github.com/pingcap/tidb/ast" "github.com/pingcap/tidb/context" "github.com/pingcap/tidb/model" "github.com/pingcap/tidb/sessionctx/db" "github.com/pingcap/tidb/store/localstore" "github.com/pingcap/tidb/store/localstore/goleveldb" - "github.com/pingcap/tidb/table" ) func TestT(t *testing.T) { @@ -36,7 +36,7 @@ type testSuite struct { } func (*testSuite) TestT(c *C) { - var ident = table.Ident{ + var ident = ast.Ident{ Name: model.NewCIStr("t"), } c.Assert(ident.String(), Not(Equals), "")