90 Star 491 Fork 151

平凯星辰(北京)科技有限公司/tidb

加入 Gitee
与超过 1200万 开发者一起发现、参与优秀开源项目,私有仓库也完全免费 :)
免费加入
文件
克隆/下载
ddl_api.go 57.04 KB
一键复制 编辑 原始数据 按行查看 历史
123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229230231232233234235236237238239240241242243244245246247248249250251252253254255256257258259260261262263264265266267268269270271272273274275276277278279280281282283284285286287288289290291292293294295296297298299300301302303304305306307308309310311312313314315316317318319320321322323324325326327328329330331332333334335336337338339340341342343344345346347348349350351352353354355356357358359360361362363364365366367368369370371372373374375376377378379380381382383384385386387388389390391392393394395396397398399400401402403404405406407408409410411412413414415416417418419420421422423424425426427428429430431432433434435436437438439440441442443444445446447448449450451452453454455456457458459460461462463464465466467468469470471472473474475476477478479480481482483484485486487488489490491492493494495496497498499500501502503504505506507508509510511512513514515516517518519520521522523524525526527528529530531532533534535536537538539540541542543544545546547548549550551552553554555556557558559560561562563564565566567568569570571572573574575576577578579580581582583584585586587588589590591592593594595596597598599600601602603604605606607608609610611612613614615616617618619620621622623624625626627628629630631632633634635636637638639640641642643644645646647648649650651652653654655656657658659660661662663664665666667668669670671672673674675676677678679680681682683684685686687688689690691692693694695696697698699700701702703704705706707708709710711712713714715716717718719720721722723724725726727728729730731732733734735736737738739740741742743744745746747748749750751752753754755756757758759760761762763764765766767768769770771772773774775776777778779780781782783784785786787788789790791792793794795796797798799800801802803804805806807808809810811812813814815816817818819820821822823824825826827828829830831832833834835836837838839840841842843844845846847848849850851852853854855856857858859860861862863864865866867868869870871872873874875876877878879880881882883884885886887888889890891892893894895896897898899900901902903904905906907908909910911912913914915916917918919920921922923924925926927928929930931932933934935936937938939940941942943944945946947948949950951952953954955956957958959960961962963964965966967968969970971972973974975976977978979980981982983984985986987988989990991992993994995996997998999100010011002100310041005100610071008100910101011101210131014101510161017101810191020102110221023102410251026102710281029103010311032103310341035103610371038103910401041104210431044104510461047104810491050105110521053105410551056105710581059106010611062106310641065106610671068106910701071107210731074107510761077107810791080108110821083108410851086108710881089109010911092109310941095109610971098109911001101110211031104110511061107110811091110111111121113111411151116111711181119112011211122112311241125112611271128112911301131113211331134113511361137113811391140114111421143114411451146114711481149115011511152115311541155115611571158115911601161116211631164116511661167116811691170117111721173117411751176117711781179118011811182118311841185118611871188118911901191119211931194119511961197119811991200120112021203120412051206120712081209121012111212121312141215121612171218121912201221122212231224122512261227122812291230123112321233123412351236123712381239124012411242124312441245124612471248124912501251125212531254125512561257125812591260126112621263126412651266126712681269127012711272127312741275127612771278127912801281128212831284128512861287128812891290129112921293129412951296129712981299130013011302130313041305130613071308130913101311131213131314131513161317131813191320132113221323132413251326132713281329133013311332133313341335133613371338133913401341134213431344134513461347134813491350135113521353135413551356135713581359136013611362136313641365136613671368136913701371137213731374137513761377137813791380138113821383138413851386138713881389139013911392139313941395139613971398139914001401140214031404140514061407140814091410141114121413141414151416141714181419142014211422142314241425142614271428142914301431143214331434143514361437143814391440144114421443144414451446144714481449145014511452145314541455145614571458145914601461146214631464146514661467146814691470147114721473147414751476147714781479148014811482148314841485148614871488148914901491149214931494149514961497149814991500150115021503150415051506150715081509151015111512151315141515151615171518151915201521152215231524152515261527152815291530153115321533153415351536153715381539154015411542154315441545154615471548154915501551155215531554155515561557155815591560156115621563156415651566156715681569157015711572157315741575157615771578157915801581158215831584158515861587158815891590159115921593159415951596159715981599160016011602160316041605160616071608160916101611161216131614161516161617161816191620162116221623162416251626162716281629163016311632163316341635163616371638163916401641164216431644164516461647164816491650165116521653165416551656165716581659166016611662166316641665166616671668166916701671167216731674167516761677167816791680168116821683168416851686168716881689169016911692169316941695169616971698169917001701170217031704170517061707170817091710171117121713171417151716171717181719172017211722172317241725172617271728172917301731173217331734173517361737173817391740174117421743174417451746174717481749175017511752175317541755175617571758175917601761176217631764176517661767176817691770177117721773177417751776177717781779178017811782178317841785178617871788178917901791179217931794179517961797179817991800180118021803180418051806180718081809181018111812181318141815181618171818181918201821182218231824182518261827182818291830183118321833183418351836183718381839184018411842184318441845184618471848184918501851
// Copyright 2013 The ql Authors. All rights reserved.
// Use of this source code is governed by a BSD-style
// license that can be found in the LICENSES/QL-LICENSE file.
// Copyright 2016 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 (
"bytes"
"fmt"
"strings"
"time"
"github.com/cznic/mathutil"
"github.com/juju/errors"
"github.com/pingcap/tidb/ast"
"github.com/pingcap/tidb/expression"
"github.com/pingcap/tidb/infoschema"
"github.com/pingcap/tidb/meta/autoid"
"github.com/pingcap/tidb/model"
"github.com/pingcap/tidb/mysql"
"github.com/pingcap/tidb/sessionctx"
"github.com/pingcap/tidb/sessionctx/variable"
"github.com/pingcap/tidb/table"
"github.com/pingcap/tidb/types"
"github.com/pingcap/tidb/util/charset"
)
func (d *ddl) CreateSchema(ctx sessionctx.Context, schema model.CIStr, charsetInfo *ast.CharsetOpt) (err error) {
is := d.GetInformationSchema()
_, ok := is.SchemaByName(schema)
if ok {
return infoschema.ErrDatabaseExists.GenByArgs(schema)
}
if err = checkTooLongSchema(schema); err != nil {
return errors.Trace(err)
}
schemaID, err := d.genGlobalID()
if err != nil {
return errors.Trace(err)
}
dbInfo := &model.DBInfo{
Name: schema,
}
if charsetInfo != nil {
err = checkCharsetAndCollation(charsetInfo.Chs, charsetInfo.Col)
if err != nil {
return errors.Trace(err)
}
dbInfo.Charset = charsetInfo.Chs
dbInfo.Collate = charsetInfo.Col
} else {
dbInfo.Charset, dbInfo.Collate = getDefaultCharsetAndCollate()
}
job := &model.Job{
SchemaID: schemaID,
Type: model.ActionCreateSchema,
BinlogInfo: &model.HistoryInfo{},
Args: []interface{}{dbInfo},
}
err = d.doDDLJob(ctx, job)
err = d.callHookOnChanged(err)
return errors.Trace(err)
}
func (d *ddl) DropSchema(ctx sessionctx.Context, schema model.CIStr) (err error) {
is := d.GetInformationSchema()
old, ok := is.SchemaByName(schema)
if !ok {
return errors.Trace(infoschema.ErrDatabaseNotExists)
}
job := &model.Job{
SchemaID: old.ID,
Type: model.ActionDropSchema,
BinlogInfo: &model.HistoryInfo{},
}
err = d.doDDLJob(ctx, job)
err = d.callHookOnChanged(err)
return errors.Trace(err)
}
func checkTooLongSchema(schema model.CIStr) error {
if len(schema.L) > mysql.MaxDatabaseNameLength {
return ErrTooLongIdent.GenByArgs(schema)
}
return nil
}
func checkTooLongTable(table model.CIStr) error {
if len(table.L) > mysql.MaxTableNameLength {
return ErrTooLongIdent.GenByArgs(table)
}
return nil
}
func checkTooLongIndex(index model.CIStr) error {
if len(index.L) > mysql.MaxIndexIdentifierLen {
return ErrTooLongIdent.GenByArgs(index)
}
return nil
}
func getDefaultCharsetAndCollate() (string, string) {
// TODO: TableDefaultCharset-->DatabaseDefaultCharset-->SystemDefaultCharset.
// TODO: Change TableOption parser to parse collate.
// This is a tmp solution.
return "utf8", "utf8_bin"
}
func setColumnFlagWithConstraint(colMap map[string]*table.Column, v *ast.Constraint) {
switch v.Tp {
case ast.ConstraintPrimaryKey:
for _, key := range v.Keys {
c, ok := colMap[key.Column.Name.L]
if !ok {
continue
}
c.Flag |= mysql.PriKeyFlag
// Primary key can not be NULL.
c.Flag |= mysql.NotNullFlag
}
case ast.ConstraintUniq, ast.ConstraintUniqIndex, ast.ConstraintUniqKey:
for i, key := range v.Keys {
c, ok := colMap[key.Column.Name.L]
if !ok {
continue
}
if i == 0 {
// Only the first column can be set
// if unique index has multi columns,
// the flag should be MultipleKeyFlag.
// See https://dev.mysql.com/doc/refman/5.7/en/show-columns.html
if len(v.Keys) > 1 {
c.Flag |= mysql.MultipleKeyFlag
} else {
c.Flag |= mysql.UniqueKeyFlag
}
}
}
case ast.ConstraintKey, ast.ConstraintIndex:
for i, key := range v.Keys {
c, ok := colMap[key.Column.Name.L]
if !ok {
continue
}
if i == 0 {
// Only the first column can be set.
c.Flag |= mysql.MultipleKeyFlag
}
}
}
}
func buildColumnsAndConstraints(ctx sessionctx.Context, colDefs []*ast.ColumnDef,
constraints []*ast.Constraint) ([]*table.Column, []*ast.Constraint, error) {
var cols []*table.Column
colMap := map[string]*table.Column{}
for i, colDef := range colDefs {
col, cts, err := buildColumnAndConstraint(ctx, i, colDef)
if err != nil {
return nil, nil, errors.Trace(err)
}
col.State = model.StatePublic
constraints = append(constraints, cts...)
cols = append(cols, col)
colMap[colDef.Name.Name.L] = col
}
// Traverse table Constraints and set col.flag.
for _, v := range constraints {
setColumnFlagWithConstraint(colMap, v)
}
return cols, constraints, nil
}
func setCharsetCollationFlenDecimal(tp *types.FieldType) error {
tp.Charset = strings.ToLower(tp.Charset)
tp.Collate = strings.ToLower(tp.Collate)
if len(tp.Charset) == 0 {
switch tp.Tp {
case mysql.TypeString, mysql.TypeVarchar, mysql.TypeVarString, mysql.TypeBlob, mysql.TypeTinyBlob, mysql.TypeMediumBlob, mysql.TypeLongBlob, mysql.TypeEnum, mysql.TypeSet:
tp.Charset, tp.Collate = getDefaultCharsetAndCollate()
default:
tp.Charset = charset.CharsetBin
tp.Collate = charset.CharsetBin
}
} else {
if !charset.ValidCharsetAndCollation(tp.Charset, tp.Collate) {
return errUnsupportedCharset.GenByArgs(tp.Charset, tp.Collate)
}
if len(tp.Collate) == 0 {
var err error
tp.Collate, err = charset.GetDefaultCollation(tp.Charset)
if err != nil {
return errors.Trace(err)
}
}
}
// Use default value for flen or decimal when they are unspecified.
defaultFlen, defaultDecimal := mysql.GetDefaultFieldLengthAndDecimal(tp.Tp)
if tp.Flen == types.UnspecifiedLength {
tp.Flen = defaultFlen
if mysql.HasUnsignedFlag(tp.Flag) && tp.Tp != mysql.TypeLonglong && mysql.IsIntegerType(tp.Tp) {
// Issue #4684: the flen of unsigned integer(except bigint) is 1 digit shorter than signed integer
// because it has no prefix "+" or "-" character.
tp.Flen--
}
}
if tp.Decimal == types.UnspecifiedLength {
tp.Decimal = defaultDecimal
}
return nil
}
func buildColumnAndConstraint(ctx sessionctx.Context, offset int,
colDef *ast.ColumnDef) (*table.Column, []*ast.Constraint, error) {
err := setCharsetCollationFlenDecimal(colDef.Tp)
if err != nil {
return nil, nil, errors.Trace(err)
}
col, cts, err := columnDefToCol(ctx, offset, colDef)
if err != nil {
return nil, nil, errors.Trace(err)
}
return col, cts, nil
}
// checkColumnCantHaveDefaultValue checks the column can have value as default or not.
// Now, TEXT/BLOB/JSON can't have not null value as default.
func checkColumnCantHaveDefaultValue(col *table.Column, value interface{}) (err error) {
if value != nil && (col.Tp == mysql.TypeJSON ||
col.Tp == mysql.TypeTinyBlob || col.Tp == mysql.TypeMediumBlob ||
col.Tp == mysql.TypeLongBlob || col.Tp == mysql.TypeBlob) {
// TEXT/BLOB/JSON can't have not null default values.
return errBlobCantHaveDefault.GenByArgs(col.Name.O)
}
return nil
}
// isExplicitTimeStamp is used to check if explicit_defaults_for_timestamp is on or off.
// Check out this link for more details.
// https://dev.mysql.com/doc/refman/5.7/en/server-system-variables.html#sysvar_explicit_defaults_for_timestamp
func isExplicitTimeStamp() bool {
// TODO: implement the behavior as MySQL when explicit_defaults_for_timestamp = off, then this function could return false.
return true
}
// columnDefToCol converts ColumnDef to Col and TableConstraints.
func columnDefToCol(ctx sessionctx.Context, offset int, colDef *ast.ColumnDef) (*table.Column, []*ast.Constraint, error) {
var constraints = make([]*ast.Constraint, 0)
col := table.ToColumn(&model.ColumnInfo{
Offset: offset,
Name: colDef.Name.Name,
FieldType: *colDef.Tp,
})
if !isExplicitTimeStamp() {
// Check and set TimestampFlag, OnUpdateNowFlag and NotNullFlag.
if col.Tp == mysql.TypeTimestamp {
col.Flag |= mysql.TimestampFlag
col.Flag |= mysql.OnUpdateNowFlag
col.Flag |= mysql.NotNullFlag
}
}
setOnUpdateNow := false
hasDefaultValue := false
if colDef.Options != nil {
length := types.UnspecifiedLength
keys := []*ast.IndexColName{
{
Column: colDef.Name,
Length: length,
},
}
for _, v := range colDef.Options {
switch v.Tp {
case ast.ColumnOptionNotNull:
col.Flag |= mysql.NotNullFlag
case ast.ColumnOptionNull:
col.Flag &= ^mysql.NotNullFlag
removeOnUpdateNowFlag(col)
case ast.ColumnOptionAutoIncrement:
col.Flag |= mysql.AutoIncrementFlag
case ast.ColumnOptionPrimaryKey:
constraint := &ast.Constraint{Tp: ast.ConstraintPrimaryKey, Keys: keys}
constraints = append(constraints, constraint)
col.Flag |= mysql.PriKeyFlag
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 ast.ColumnOptionDefaultValue:
value, err := getDefaultValue(ctx, v, colDef.Tp.Tp, colDef.Tp.Decimal)
if err != nil {
return nil, nil, ErrColumnBadNull.Gen("invalid default value - %s", err)
}
if err = checkColumnCantHaveDefaultValue(col, value); err != nil {
return nil, nil, errors.Trace(err)
}
col.DefaultValue = value
hasDefaultValue = true
removeOnUpdateNowFlag(col)
case ast.ColumnOptionOnUpdate:
// TODO: Support other time functions.
if !expression.IsCurrentTimestampExpr(v.Expr) {
return nil, nil, ErrInvalidOnUpdate.Gen("invalid ON UPDATE for - %s", col.Name)
}
col.Flag |= mysql.OnUpdateNowFlag
setOnUpdateNow = true
case ast.ColumnOptionComment:
err := setColumnComment(ctx, col, v)
if err != nil {
return nil, nil, errors.Trace(err)
}
case ast.ColumnOptionGenerated:
var buf = bytes.NewBuffer([]byte{})
v.Expr.Format(buf)
col.GeneratedExprString = buf.String()
col.GeneratedStored = v.Stored
_, dependColNames := findDependedColumnNames(colDef)
col.Dependences = dependColNames
case ast.ColumnOptionFulltext:
// TODO: Support this type.
}
}
}
setTimestampDefaultValue(col, hasDefaultValue, setOnUpdateNow)
// Set `NoDefaultValueFlag` if this field doesn't have a default value and
// it is `not null` and not an `AUTO_INCREMENT` field or `TIMESTAMP` field.
setNoDefaultValueFlag(col, hasDefaultValue)
if col.FieldType.EvalType().IsStringKind() && col.Charset == charset.CharsetBin {
col.Flag |= mysql.BinaryFlag
}
if col.Tp == mysql.TypeBit {
// For BIT field, it's charset is binary but does not have binary flag.
col.Flag &= ^mysql.BinaryFlag
col.Flag |= mysql.UnsignedFlag
}
if col.Tp == mysql.TypeYear {
// For Year field, it's charset is binary but does not have binary flag.
col.Flag &= ^mysql.BinaryFlag
col.Flag |= mysql.UnsignedFlag | mysql.ZerofillFlag
}
err := checkDefaultValue(ctx, col, hasDefaultValue)
if err != nil {
return nil, nil, errors.Trace(err)
}
return col, constraints, nil
}
func getDefaultValue(ctx sessionctx.Context, c *ast.ColumnOption, tp byte, fsp int) (interface{}, error) {
if tp == mysql.TypeTimestamp || tp == mysql.TypeDatetime {
vd, err := expression.GetTimeValue(ctx, c.Expr, tp, fsp)
value := vd.GetValue()
if err != nil {
return nil, errors.Trace(err)
}
// Value is nil means `default null`.
if value == nil {
return nil, nil
}
// If value is types.Time, convert it to string.
if vv, ok := value.(types.Time); ok {
return vv.String(), nil
}
return value, nil
}
v, err := expression.EvalAstExpr(ctx, c.Expr)
if err != nil {
return nil, errors.Trace(err)
}
if v.IsNull() {
return nil, nil
}
if v.Kind() == types.KindBinaryLiteral || v.Kind() == types.KindMysqlBit {
if tp == mysql.TypeBit ||
tp == mysql.TypeString || tp == mysql.TypeVarchar || tp == mysql.TypeVarString ||
tp == mysql.TypeBlob || tp == mysql.TypeLongBlob || tp == mysql.TypeMediumBlob || tp == mysql.TypeTinyBlob ||
tp == mysql.TypeJSON {
// For BinaryLiteral / string fields, when getting default value we cast the value into BinaryLiteral{}, thus we return
// its raw string content here.
return v.GetBinaryLiteral().ToString(), nil
}
// For other kind of fields (e.g. INT), we supply its integer value so that it acts as integers.
return v.GetBinaryLiteral().ToInt(ctx.GetSessionVars().StmtCtx)
}
if tp == mysql.TypeBit {
if v.Kind() == types.KindInt64 || v.Kind() == types.KindUint64 {
// For BIT fields, convert int into BinaryLiteral.
return types.NewBinaryLiteralFromUint(v.GetUint64(), -1).ToString(), nil
}
}
return v.ToString()
}
func removeOnUpdateNowFlag(c *table.Column) {
// For timestamp Col, if it is set null or default value,
// OnUpdateNowFlag should be removed.
if mysql.HasTimestampFlag(c.Flag) {
c.Flag &= ^mysql.OnUpdateNowFlag
}
}
func setTimestampDefaultValue(c *table.Column, hasDefaultValue bool, setOnUpdateNow bool) {
if hasDefaultValue {
return
}
// For timestamp Col, if is not set default value or not set null, use current timestamp.
if mysql.HasTimestampFlag(c.Flag) && mysql.HasNotNullFlag(c.Flag) {
if setOnUpdateNow {
c.DefaultValue = types.ZeroDatetimeStr
} else {
c.DefaultValue = strings.ToUpper(ast.CurrentTimestamp)
}
}
}
func setNoDefaultValueFlag(c *table.Column, hasDefaultValue bool) {
if hasDefaultValue {
return
}
if !mysql.HasNotNullFlag(c.Flag) {
return
}
// Check if it is an `AUTO_INCREMENT` field or `TIMESTAMP` field.
if !mysql.HasAutoIncrementFlag(c.Flag) && !mysql.HasTimestampFlag(c.Flag) {
c.Flag |= mysql.NoDefaultValueFlag
}
}
func checkDefaultValue(ctx sessionctx.Context, c *table.Column, hasDefaultValue bool) error {
if !hasDefaultValue {
return nil
}
if c.DefaultValue != nil {
_, err := table.GetColDefaultValue(ctx, c.ToInfo())
if types.ErrTruncated.Equal(err) {
return types.ErrInvalidDefault.GenByArgs(c.Name)
}
return errors.Trace(err)
}
// Set not null but default null is invalid.
if mysql.HasNotNullFlag(c.Flag) {
return types.ErrInvalidDefault.GenByArgs(c.Name)
}
return nil
}
func checkDuplicateColumn(colDefs []*ast.ColumnDef) error {
colNames := map[string]bool{}
for _, colDef := range colDefs {
nameLower := colDef.Name.Name.L
if colNames[nameLower] {
return infoschema.ErrColumnExists.GenByArgs(colDef.Name.Name)
}
colNames[nameLower] = true
}
return nil
}
func checkGeneratedColumn(colDefs []*ast.ColumnDef) error {
var colName2Generation = make(map[string]columnGenerationInDDL, len(colDefs))
for i, colDef := range colDefs {
generated, depCols := findDependedColumnNames(colDef)
if !generated {
colName2Generation[colDef.Name.Name.L] = columnGenerationInDDL{
position: i,
generated: false,
}
} else {
colName2Generation[colDef.Name.Name.L] = columnGenerationInDDL{
position: i,
generated: true,
dependences: depCols,
}
}
}
for _, colDef := range colDefs {
colName := colDef.Name.Name.L
if err := verifyColumnGeneration(colName2Generation, colName); err != nil {
return errors.Trace(err)
}
}
return nil
}
func checkTooLongColumn(colDefs []*ast.ColumnDef) error {
for _, colDef := range colDefs {
if len(colDef.Name.Name.O) > mysql.MaxColumnNameLength {
return ErrTooLongIdent.GenByArgs(colDef.Name.Name)
}
}
return nil
}
func checkTooManyColumns(colDefs []*ast.ColumnDef) error {
if len(colDefs) > TableColumnCountLimit {
return errTooManyFields
}
return nil
}
func checkDuplicateConstraint(namesMap map[string]bool, name string, foreign bool) error {
if name == "" {
return nil
}
nameLower := strings.ToLower(name)
if namesMap[nameLower] {
if foreign {
return infoschema.ErrCannotAddForeign
}
return ErrDupKeyName.Gen("duplicate key name %s", name)
}
namesMap[nameLower] = true
return nil
}
func setEmptyConstraintName(namesMap map[string]bool, constr *ast.Constraint, foreign bool) {
if constr.Name == "" && len(constr.Keys) > 0 {
colName := constr.Keys[0].Column.Name.L
constrName := colName
i := 2
if strings.EqualFold(constrName, mysql.PrimaryKeyName) {
constrName = fmt.Sprintf("%s_%d", constrName, 2)
i = 3
}
for namesMap[constrName] {
// We loop forever until we find constrName that haven't been used.
if foreign {
constrName = fmt.Sprintf("fk_%s_%d", colName, i)
} else {
constrName = fmt.Sprintf("%s_%d", colName, i)
}
i++
}
constr.Name = constrName
namesMap[constrName] = true
}
}
func checkConstraintNames(constraints []*ast.Constraint) error {
constrNames := map[string]bool{}
fkNames := map[string]bool{}
// Check not empty constraint name whether is duplicated.
for _, constr := range constraints {
if constr.Tp == ast.ConstraintForeignKey {
err := checkDuplicateConstraint(fkNames, constr.Name, true)
if err != nil {
return errors.Trace(err)
}
} else {
err := checkDuplicateConstraint(constrNames, constr.Name, false)
if err != nil {
return errors.Trace(err)
}
}
}
// Set empty constraint names.
for _, constr := range constraints {
if constr.Tp == ast.ConstraintForeignKey {
setEmptyConstraintName(fkNames, constr, true)
} else {
setEmptyConstraintName(constrNames, constr, false)
}
}
return nil
}
func buildTableInfo(ctx sessionctx.Context, d *ddl, tableName model.CIStr, cols []*table.Column, constraints []*ast.Constraint) (tbInfo *model.TableInfo, err error) {
tbInfo = &model.TableInfo{
Name: tableName,
}
// When this function is called by MockTableInfo, we should set a particular table id.
// So the `ddl` structure may be nil.
if d != nil {
tbInfo.ID, err = d.genGlobalID()
if err != nil {
return nil, errors.Trace(err)
}
}
for _, v := range cols {
v.ID = allocateColumnID(tbInfo)
tbInfo.Columns = append(tbInfo.Columns, v.ToInfo())
}
for _, constr := range constraints {
if constr.Tp == ast.ConstraintForeignKey {
for _, fk := range tbInfo.ForeignKeys {
if fk.Name.L == strings.ToLower(constr.Name) {
return nil, infoschema.ErrCannotAddForeign
}
}
var fk model.FKInfo
fk.Name = model.NewCIStr(constr.Name)
fk.RefTable = constr.Refer.Table.Name
fk.State = model.StatePublic
for _, key := range constr.Keys {
fk.Cols = append(fk.Cols, key.Column.Name)
}
for _, key := range constr.Refer.IndexColNames {
fk.RefCols = append(fk.RefCols, key.Column.Name)
}
fk.OnDelete = int(constr.Refer.OnDelete.ReferOpt)
fk.OnUpdate = int(constr.Refer.OnUpdate.ReferOpt)
if len(fk.Cols) != len(fk.RefCols) {
return nil, infoschema.ErrForeignKeyNotMatch.GenByArgs(tbInfo.Name.O)
}
if len(fk.Cols) == 0 {
// TODO: In MySQL, this case will report a parse error.
return nil, infoschema.ErrCannotAddForeign
}
tbInfo.ForeignKeys = append(tbInfo.ForeignKeys, &fk)
continue
}
if constr.Tp == ast.ConstraintPrimaryKey {
var col *table.Column
for _, key := range constr.Keys {
col = table.FindCol(cols, key.Column.Name.O)
if col == nil {
return nil, errKeyColumnDoesNotExits.Gen("key column %s doesn't exist in table", key.Column.Name)
}
// Virtual columns cannot be used in primary key.
if col.IsGenerated() && !col.GeneratedStored {
return nil, errUnsupportedOnGeneratedColumn.GenByArgs("Defining a virtual generated column as primary key")
}
}
if len(constr.Keys) == 1 {
switch col.Tp {
case mysql.TypeLong, mysql.TypeLonglong,
mysql.TypeTiny, mysql.TypeShort, mysql.TypeInt24:
tbInfo.PKIsHandle = true
// Avoid creating index for PK handle column.
continue
}
}
}
// build index info.
idxInfo, err := buildIndexInfo(tbInfo, model.NewCIStr(constr.Name), constr.Keys, model.StatePublic)
if err != nil {
return nil, errors.Trace(err)
}
//check if the index is primary or uniqiue.
switch constr.Tp {
case ast.ConstraintPrimaryKey:
idxInfo.Primary = true
idxInfo.Unique = true
idxInfo.Name = model.NewCIStr(mysql.PrimaryKeyName)
case ast.ConstraintUniq, ast.ConstraintUniqKey, ast.ConstraintUniqIndex:
idxInfo.Unique = true
}
// set index type.
if constr.Option != nil {
idxInfo.Comment, err = validateCommentLength(ctx.GetSessionVars(),
constr.Option.Comment,
maxCommentLength,
errTooLongIndexComment.GenByArgs(idxInfo.Name.String(), maxCommentLength))
if err != nil {
return nil, errors.Trace(err)
}
if constr.Option.Tp == model.IndexTypeInvalid {
// Use btree as default index type.
idxInfo.Tp = model.IndexTypeBtree
} else {
idxInfo.Tp = constr.Option.Tp
}
} else {
// Use btree as default index type.
idxInfo.Tp = model.IndexTypeBtree
}
idxInfo.ID = allocateIndexID(tbInfo)
tbInfo.Indices = append(tbInfo.Indices, idxInfo)
}
return
}
func (d *ddl) CreateTableWithLike(ctx sessionctx.Context, ident, referIdent ast.Ident, ifNotExists bool) error {
is := d.GetInformationSchema()
_, ok := is.SchemaByName(referIdent.Schema)
if !ok {
return infoschema.ErrTableNotExists.GenByArgs(referIdent.Schema, referIdent.Name)
}
referTbl, err := is.TableByName(referIdent.Schema, referIdent.Name)
if err != nil {
return infoschema.ErrTableNotExists.GenByArgs(referIdent.Schema, referIdent.Name)
}
schema, ok := is.SchemaByName(ident.Schema)
if !ok {
return infoschema.ErrDatabaseNotExists.GenByArgs(ident.Schema)
}
if is.TableExists(ident.Schema, ident.Name) {
if ifNotExists {
ctx.GetSessionVars().StmtCtx.AppendNote(infoschema.ErrTableExists.GenByArgs(ident))
return nil
}
return infoschema.ErrTableExists.GenByArgs(ident)
}
tblInfo := *referTbl.Meta()
tblInfo.Name = ident.Name
tblInfo.AutoIncID = 0
tblInfo.ForeignKeys = nil
tblInfo.ID, err = d.genGlobalID()
if err != nil {
return errors.Trace(err)
}
job := &model.Job{
SchemaID: schema.ID,
TableID: tblInfo.ID,
Type: model.ActionCreateTable,
BinlogInfo: &model.HistoryInfo{},
Args: []interface{}{tblInfo},
}
err = d.doDDLJob(ctx, job)
err = d.callHookOnChanged(err)
return errors.Trace(err)
}
func (d *ddl) CreateTable(ctx sessionctx.Context, s *ast.CreateTableStmt) (err error) {
ident := ast.Ident{Schema: s.Table.Schema, Name: s.Table.Name}
if s.ReferTable != nil {
referIdent := ast.Ident{Schema: s.ReferTable.Schema, Name: s.ReferTable.Name}
return d.CreateTableWithLike(ctx, ident, referIdent, s.IfNotExists)
}
colDefs := s.Cols
is := d.GetInformationSchema()
schema, ok := is.SchemaByName(ident.Schema)
if !ok {
return infoschema.ErrDatabaseNotExists.GenByArgs(ident.Schema)
}
if is.TableExists(ident.Schema, ident.Name) {
if s.IfNotExists {
ctx.GetSessionVars().StmtCtx.AppendNote(infoschema.ErrTableExists.GenByArgs(ident))
return nil
}
return infoschema.ErrTableExists.GenByArgs(ident)
}
if err = checkTooLongTable(ident.Name); err != nil {
return errors.Trace(err)
}
if err = checkDuplicateColumn(colDefs); err != nil {
return errors.Trace(err)
}
if err = checkGeneratedColumn(colDefs); err != nil {
return errors.Trace(err)
}
if err = checkTooLongColumn(colDefs); err != nil {
return errors.Trace(err)
}
if err = checkTooManyColumns(colDefs); err != nil {
return errors.Trace(err)
}
cols, newConstraints, err := buildColumnsAndConstraints(ctx, colDefs, s.Constraints)
if err != nil {
return errors.Trace(err)
}
err = checkConstraintNames(newConstraints)
if err != nil {
return errors.Trace(err)
}
tbInfo, err := buildTableInfo(ctx, d, ident.Name, cols, newConstraints)
if err != nil {
return errors.Trace(err)
}
if s.Partition != nil {
pi := &model.PartitionInfo{
Type: s.Partition.Tp,
Expr: s.Partition.Expr.Text(),
}
if s.Partition.Expr != nil {
buf := new(bytes.Buffer)
s.Partition.Expr.Format(buf)
pi.Expr = buf.String()
} else if s.Partition.ColumnNames != nil {
pi.Columns = make([]model.CIStr, 0, len(s.Partition.ColumnNames))
for _, cn := range s.Partition.ColumnNames {
pi.Columns = append(pi.Columns, cn.Name)
}
}
for _, def := range s.Partition.Definitions {
// TODO: generate multiple global ID for paritions.
pid, err1 := d.genGlobalID()
if err1 != nil {
return errors.Trace(err1)
}
piDef := model.PartitionDefinition{
Name: def.Name,
ID: pid,
Comment: def.Comment,
MaxValue: def.MaxValue,
}
for _, expr := range def.LessThan {
buf := new(bytes.Buffer)
expr.Format(buf)
piDef.LessThan = append(piDef.LessThan, buf.String())
}
pi.Definitions = append(pi.Definitions, piDef)
}
tbInfo.Partition = pi
}
job := &model.Job{
SchemaID: schema.ID,
TableID: tbInfo.ID,
Type: model.ActionCreateTable,
BinlogInfo: &model.HistoryInfo{},
Args: []interface{}{tbInfo},
}
err = handleTableOptions(s.Options, tbInfo)
if err != nil {
return errors.Trace(err)
}
err = checkCharsetAndCollation(tbInfo.Charset, tbInfo.Collate)
if err != nil {
return errors.Trace(err)
}
err = d.doDDLJob(ctx, job)
if err == nil {
if tbInfo.AutoIncID > 1 {
// Default tableAutoIncID base is 0.
// If the first id is expected to greater than 1, we need to do rebase.
err = d.handleAutoIncID(tbInfo, schema.ID)
}
}
// table exists, but if_not_exists flags is true, so we ignore this error.
if infoschema.ErrTableExists.Equal(err) && s.IfNotExists {
return nil
}
err = d.callHookOnChanged(err)
return errors.Trace(err)
}
func checkCharsetAndCollation(cs string, co string) error {
if !charset.ValidCharsetAndCollation(cs, co) {
return ErrUnknownCharacterSet.GenByArgs(cs)
}
return nil
}
// handleAutoIncID handles auto_increment option in DDL. It creates a ID counter for the table and initiates the counter to a proper value.
// For example if the option sets auto_increment to 10. The counter will be set to 9. So the next allocated ID will be 10.
func (d *ddl) handleAutoIncID(tbInfo *model.TableInfo, schemaID int64) error {
alloc := autoid.NewAllocator(d.store, tbInfo.GetDBID(schemaID))
tbInfo.State = model.StatePublic
tb, err := table.TableFromMeta(alloc, tbInfo)
if err != nil {
return errors.Trace(err)
}
// The operation of the minus 1 to make sure that the current value doesn't be used,
// the next Alloc operation will get this value.
// Its behavior is consistent with MySQL.
if err = tb.RebaseAutoID(nil, tbInfo.AutoIncID-1, false); err != nil {
return errors.Trace(err)
}
return nil
}
// handleTableOptions updates tableInfo according to table options.
func handleTableOptions(options []*ast.TableOption, tbInfo *model.TableInfo) error {
for _, op := range options {
switch op.Tp {
case ast.TableOptionAutoIncrement:
tbInfo.AutoIncID = int64(op.UintValue)
case ast.TableOptionComment:
tbInfo.Comment = op.StrValue
case ast.TableOptionCharset:
tbInfo.Charset = op.StrValue
case ast.TableOptionCollate:
tbInfo.Collate = op.StrValue
case ast.TableOptionShardRowID:
if hasAutoIncrementColumn(tbInfo) && op.UintValue != 0 {
return errUnsupportedShardRowIDBits
}
tbInfo.ShardRowIDBits = op.UintValue
if tbInfo.ShardRowIDBits > shardRowIDBitsMax {
tbInfo.ShardRowIDBits = shardRowIDBitsMax
}
}
}
return nil
}
func hasAutoIncrementColumn(tbInfo *model.TableInfo) bool {
for _, col := range tbInfo.Columns {
if mysql.HasAutoIncrementFlag(col.Flag) {
return true
}
}
return false
}
// isIgnorableSpec checks if the spec type is ignorable.
// Some specs are parsed by ignored. This is for compatibility.
func isIgnorableSpec(tp ast.AlterTableType) bool {
// AlterTableLock/AlterTableAlgorithm are ignored.
return tp == ast.AlterTableLock || tp == ast.AlterTableAlgorithm
}
func (d *ddl) AlterTable(ctx sessionctx.Context, ident ast.Ident, specs []*ast.AlterTableSpec) (err error) {
// Only handle valid specs.
validSpecs := make([]*ast.AlterTableSpec, 0, len(specs))
for _, spec := range specs {
if isIgnorableSpec(spec.Tp) {
continue
}
validSpecs = append(validSpecs, spec)
}
if len(validSpecs) != 1 {
// TODO: Hanlde len(validSpecs) == 0.
// Now we only allow one schema changing at the same time.
return errRunMultiSchemaChanges
}
for _, spec := range validSpecs {
switch spec.Tp {
case ast.AlterTableAddColumns:
if len(spec.NewColumns) != 1 {
return errRunMultiSchemaChanges
}
err = d.AddColumn(ctx, ident, spec)
case ast.AlterTableDropColumn:
err = d.DropColumn(ctx, ident, spec.OldColumnName.Name)
case ast.AlterTableDropIndex:
err = d.DropIndex(ctx, ident, model.NewCIStr(spec.Name))
case ast.AlterTableAddConstraint:
constr := spec.Constraint
switch spec.Constraint.Tp {
case ast.ConstraintKey, ast.ConstraintIndex:
err = d.CreateIndex(ctx, ident, false, model.NewCIStr(constr.Name), spec.Constraint.Keys, constr.Option)
case ast.ConstraintUniq, ast.ConstraintUniqIndex, ast.ConstraintUniqKey:
err = d.CreateIndex(ctx, ident, true, model.NewCIStr(constr.Name), spec.Constraint.Keys, constr.Option)
case ast.ConstraintForeignKey:
err = d.CreateForeignKey(ctx, ident, model.NewCIStr(constr.Name), spec.Constraint.Keys, spec.Constraint.Refer)
case ast.ConstraintPrimaryKey:
err = ErrUnsupportedModifyPrimaryKey.GenByArgs("add")
default:
// Nothing to do now.
}
case ast.AlterTableDropForeignKey:
err = d.DropForeignKey(ctx, ident, model.NewCIStr(spec.Name))
case ast.AlterTableModifyColumn:
err = d.ModifyColumn(ctx, ident, spec)
case ast.AlterTableChangeColumn:
err = d.ChangeColumn(ctx, ident, spec)
case ast.AlterTableAlterColumn:
err = d.AlterColumn(ctx, ident, spec)
case ast.AlterTableRenameTable:
newIdent := ast.Ident{Schema: spec.NewTable.Schema, Name: spec.NewTable.Name}
err = d.RenameTable(ctx, ident, newIdent)
case ast.AlterTableDropPrimaryKey:
err = ErrUnsupportedModifyPrimaryKey.GenByArgs("drop")
case ast.AlterTableOption:
for _, opt := range spec.Options {
switch opt.Tp {
case ast.TableOptionShardRowID:
if opt.UintValue > shardRowIDBitsMax {
opt.UintValue = shardRowIDBitsMax
}
err = d.ShardRowID(ctx, ident, opt.UintValue)
case ast.TableOptionAutoIncrement:
err = d.RebaseAutoID(ctx, ident, int64(opt.UintValue))
case ast.TableOptionComment:
spec.Comment = opt.StrValue
err = d.AlterTableComment(ctx, ident, spec)
}
if err != nil {
return errors.Trace(err)
}
}
default:
// Nothing to do now.
}
if err != nil {
return errors.Trace(err)
}
}
return nil
}
func (d *ddl) RebaseAutoID(ctx sessionctx.Context, ident ast.Ident, newBase int64) error {
is := d.GetInformationSchema()
schema, ok := is.SchemaByName(ident.Schema)
if !ok {
return infoschema.ErrDatabaseNotExists.GenByArgs(ident.Schema)
}
t, err := is.TableByName(ident.Schema, ident.Name)
if err != nil {
return errors.Trace(infoschema.ErrTableNotExists.GenByArgs(ident.Schema, ident.Name))
}
autoIncID, err := t.Allocator(ctx).NextGlobalAutoID(t.Meta().ID)
if err != nil {
return errors.Trace(err)
}
newBase = mathutil.MaxInt64(newBase, autoIncID)
job := &model.Job{
SchemaID: schema.ID,
TableID: t.Meta().ID,
Type: model.ActionRebaseAutoID,
BinlogInfo: &model.HistoryInfo{},
Args: []interface{}{newBase},
}
err = d.doDDLJob(ctx, job)
err = d.callHookOnChanged(err)
return errors.Trace(err)
}
// ShardRowID shards the implicit row ID by adding shard value to the row ID's first few bits.
func (d *ddl) ShardRowID(ctx sessionctx.Context, tableIdent ast.Ident, uVal uint64) error {
schema, t, err := d.getSchemaAndTableByIdent(tableIdent)
if err != nil {
return errors.Trace(err)
}
if hasAutoIncrementColumn(t.Meta()) && uVal != 0 {
return errUnsupportedShardRowIDBits
}
job := &model.Job{
Type: model.ActionShardRowID,
SchemaID: schema.ID,
TableID: t.Meta().ID,
BinlogInfo: &model.HistoryInfo{},
Args: []interface{}{uVal},
}
err = d.doDDLJob(ctx, job)
err = d.callHookOnChanged(err)
return errors.Trace(err)
}
func (d *ddl) getSchemaAndTableByIdent(tableIdent ast.Ident) (dbInfo *model.DBInfo, t table.Table, err error) {
is := d.GetInformationSchema()
schema, ok := is.SchemaByName(tableIdent.Schema)
if !ok {
return nil, nil, infoschema.ErrDatabaseNotExists.GenByArgs(tableIdent.Schema)
}
t, err = is.TableByName(tableIdent.Schema, tableIdent.Name)
if err != nil {
return nil, nil, infoschema.ErrTableNotExists.GenByArgs(tableIdent.Schema, tableIdent.Name)
}
return schema, t, nil
}
func checkColumnConstraint(constraints []*ast.ColumnOption) error {
for _, constraint := range constraints {
switch constraint.Tp {
case ast.ColumnOptionAutoIncrement, ast.ColumnOptionPrimaryKey, ast.ColumnOptionUniqKey:
return errUnsupportedAddColumn.Gen("unsupported add column constraint - %v", constraint.Tp)
}
}
return nil
}
// AddColumn will add a new column to the table.
func (d *ddl) AddColumn(ctx sessionctx.Context, ti ast.Ident, spec *ast.AlterTableSpec) error {
specNewColumn := spec.NewColumns[0]
// Check whether the added column constraints are supported.
err := checkColumnConstraint(specNewColumn.Options)
if err != nil {
return errors.Trace(err)
}
is := d.infoHandle.Get()
schema, ok := is.SchemaByName(ti.Schema)
if !ok {
return errors.Trace(infoschema.ErrDatabaseNotExists)
}
t, err := is.TableByName(ti.Schema, ti.Name)
if err != nil {
return errors.Trace(infoschema.ErrTableNotExists.GenByArgs(ti.Schema, ti.Name))
}
// Check whether added column has existed.
colName := specNewColumn.Name.Name.O
col := table.FindCol(t.Cols(), colName)
if col != nil {
return infoschema.ErrColumnExists.GenByArgs(colName)
}
// If new column is a generated column, do validation.
// NOTE: Because now we can only append columns to table,
// we dont't need check whether the column refers other
// generated columns occurring later in table.
for _, option := range specNewColumn.Options {
if option.Tp == ast.ColumnOptionGenerated {
referableColNames := make(map[string]struct{}, len(t.Cols()))
for _, col := range t.Cols() {
referableColNames[col.Name.L] = struct{}{}
}
_, dependColNames := findDependedColumnNames(specNewColumn)
if err = columnNamesCover(referableColNames, dependColNames); err != nil {
return errors.Trace(err)
}
}
}
if len(colName) > mysql.MaxColumnNameLength {
return ErrTooLongIdent.GenByArgs(colName)
}
// Ingore table constraints now, maybe return error later.
// We use length(t.Cols()) as the default offset firstly, we will change the
// column's offset later.
col, _, err = buildColumnAndConstraint(ctx, len(t.Cols()), specNewColumn)
if err != nil {
return errors.Trace(err)
}
col.OriginDefaultValue = col.DefaultValue
if col.OriginDefaultValue == nil && mysql.HasNotNullFlag(col.Flag) {
zeroVal := table.GetZeroValue(col.ToInfo())
col.OriginDefaultValue, err = zeroVal.ToString()
if err != nil {
return errors.Trace(err)
}
}
if col.OriginDefaultValue == strings.ToUpper(ast.CurrentTimestamp) &&
(col.Tp == mysql.TypeTimestamp || col.Tp == mysql.TypeDatetime) {
col.OriginDefaultValue = time.Now().Format(types.TimeFormat)
}
job := &model.Job{
SchemaID: schema.ID,
TableID: t.Meta().ID,
Type: model.ActionAddColumn,
BinlogInfo: &model.HistoryInfo{},
Args: []interface{}{col, spec.Position, 0},
}
err = d.doDDLJob(ctx, job)
err = d.callHookOnChanged(err)
return errors.Trace(err)
}
// DropColumn will drop a column from the table, now we don't support drop the column with index covered.
func (d *ddl) DropColumn(ctx sessionctx.Context, ti ast.Ident, colName model.CIStr) error {
is := d.infoHandle.Get()
schema, ok := is.SchemaByName(ti.Schema)
if !ok {
return errors.Trace(infoschema.ErrDatabaseNotExists)
}
t, err := is.TableByName(ti.Schema, ti.Name)
if err != nil {
return errors.Trace(infoschema.ErrTableNotExists.GenByArgs(ti.Schema, ti.Name))
}
// Check whether dropped column has existed.
col := table.FindCol(t.Cols(), colName.L)
if col == nil {
return ErrCantDropFieldOrKey.Gen("column %s doesn't exist", colName)
}
tblInfo := t.Meta()
if err = isDroppableColumn(tblInfo, colName); err != nil {
return errors.Trace(err)
}
// We don't support dropping column with PK handle covered now.
if col.IsPKHandleColumn(tblInfo) {
return errUnsupportedPKHandle
}
job := &model.Job{
SchemaID: schema.ID,
TableID: t.Meta().ID,
Type: model.ActionDropColumn,
BinlogInfo: &model.HistoryInfo{},
Args: []interface{}{colName},
}
err = d.doDDLJob(ctx, job)
err = d.callHookOnChanged(err)
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 modifiable(origin *types.FieldType, to *types.FieldType) error {
if to.Flen > 0 && to.Flen < origin.Flen {
msg := fmt.Sprintf("length %d is less than origin %d", to.Flen, origin.Flen)
return errUnsupportedModifyColumn.GenByArgs(msg)
}
if to.Decimal > 0 && to.Decimal < origin.Decimal {
msg := fmt.Sprintf("decimal %d is less than origin %d", to.Decimal, origin.Decimal)
return errUnsupportedModifyColumn.GenByArgs(msg)
}
if to.Charset != origin.Charset {
msg := fmt.Sprintf("charset %s not match origin %s", to.Charset, origin.Charset)
return errUnsupportedModifyColumn.GenByArgs(msg)
}
if to.Collate != origin.Collate {
msg := fmt.Sprintf("collate %s not match origin %s", to.Collate, origin.Collate)
return errUnsupportedModifyColumn.GenByArgs(msg)
}
toUnsigned := mysql.HasUnsignedFlag(to.Flag)
originUnsigned := mysql.HasUnsignedFlag(origin.Flag)
if originUnsigned != toUnsigned {
msg := fmt.Sprintf("unsigned %v not match origin %v", toUnsigned, originUnsigned)
return errUnsupportedModifyColumn.GenByArgs(msg)
}
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 nil
}
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 nil
}
case mysql.TypeEnum:
return errUnsupportedModifyColumn.GenByArgs("modify enum column is not supported")
default:
if origin.Tp == to.Tp {
return nil
}
}
msg := fmt.Sprintf("type %v not match origin %v", to.Tp, origin.Tp)
return errUnsupportedModifyColumn.GenByArgs(msg)
}
func setDefaultValue(ctx sessionctx.Context, col *table.Column, option *ast.ColumnOption) error {
value, err := getDefaultValue(ctx, option, col.Tp, col.Decimal)
if err != nil {
return ErrColumnBadNull.Gen("invalid default value - %s", err)
}
col.DefaultValue = value
return errors.Trace(checkDefaultValue(ctx, col, true))
}
func setColumnComment(ctx sessionctx.Context, col *table.Column, option *ast.ColumnOption) error {
value, err := expression.EvalAstExpr(ctx, option.Expr)
if err != nil {
return errors.Trace(err)
}
col.Comment, err = value.ToString()
return errors.Trace(err)
}
// setDefaultAndComment is only used in getModifiableColumnJob.
func setDefaultAndComment(ctx sessionctx.Context, col *table.Column, options []*ast.ColumnOption) error {
if len(options) == 0 {
return nil
}
var hasDefaultValue, setOnUpdateNow bool
for _, opt := range options {
switch opt.Tp {
case ast.ColumnOptionDefaultValue:
value, err := getDefaultValue(ctx, opt, col.Tp, col.Decimal)
if err != nil {
return ErrColumnBadNull.Gen("invalid default value - %s", err)
}
if err = checkColumnCantHaveDefaultValue(col, value); err != nil {
return errors.Trace(err)
}
col.DefaultValue = value
hasDefaultValue = true
case ast.ColumnOptionComment:
err := setColumnComment(ctx, col, opt)
if err != nil {
return errors.Trace(err)
}
case ast.ColumnOptionNotNull:
col.Flag |= mysql.NotNullFlag
case ast.ColumnOptionNull:
col.Flag &= ^mysql.NotNullFlag
case ast.ColumnOptionAutoIncrement:
col.Flag |= mysql.AutoIncrementFlag
case ast.ColumnOptionPrimaryKey, ast.ColumnOptionUniqKey:
return errUnsupportedModifyColumn.Gen("unsupported modify column constraint - %v", opt.Tp)
case ast.ColumnOptionOnUpdate:
// TODO: Support other time functions.
if !expression.IsCurrentTimestampExpr(opt.Expr) {
return ErrInvalidOnUpdate.Gen("invalid ON UPDATE for - %s", col.Name)
}
col.Flag |= mysql.OnUpdateNowFlag
setOnUpdateNow = true
case ast.ColumnOptionGenerated:
var buf = bytes.NewBuffer([]byte{})
opt.Expr.Format(buf)
col.GeneratedExprString = buf.String()
col.GeneratedStored = opt.Stored
col.Dependences = make(map[string]struct{})
for _, colName := range findColumnNamesInExpr(opt.Expr) {
col.Dependences[colName.Name.L] = struct{}{}
}
default:
// TODO: Support other types.
return errors.Trace(errUnsupportedModifyColumn.GenByArgs(opt.Tp))
}
}
setTimestampDefaultValue(col, hasDefaultValue, setOnUpdateNow)
// Set `NoDefaultValueFlag` if this field doesn't have a default value and
// it is `not null` and not an `AUTO_INCREMENT` field or `TIMESTAMP` field.
setNoDefaultValueFlag(col, hasDefaultValue)
if hasDefaultValue {
return errors.Trace(checkDefaultValue(ctx, col, true))
}
return nil
}
func (d *ddl) getModifiableColumnJob(ctx sessionctx.Context, ident ast.Ident, originalColName model.CIStr,
spec *ast.AlterTableSpec) (*model.Job, error) {
specNewColumn := spec.NewColumns[0]
is := d.infoHandle.Get()
schema, ok := is.SchemaByName(ident.Schema)
if !ok {
return nil, errors.Trace(infoschema.ErrDatabaseNotExists)
}
t, err := is.TableByName(ident.Schema, ident.Name)
if err != nil {
return nil, errors.Trace(infoschema.ErrTableNotExists.GenByArgs(ident.Schema, ident.Name))
}
col := table.FindCol(t.Cols(), originalColName.L)
if col == nil {
return nil, infoschema.ErrColumnNotExists.GenByArgs(originalColName, ident.Name)
}
newColName := specNewColumn.Name.Name
// If we want to rename the column name, we need to check whether it already exists.
if newColName.L != originalColName.L {
c := table.FindCol(t.Cols(), newColName.L)
if c != nil {
return nil, infoschema.ErrColumnExists.GenByArgs(newColName)
}
}
// Constraints in the new column means adding new constraints. Errors should thrown,
// which will be done by `setDefaultAndComment` later.
if specNewColumn.Tp == nil {
// Make sure the column definition is simple field type.
return nil, errors.Trace(errUnsupportedModifyColumn)
}
newCol := table.ToColumn(&model.ColumnInfo{
ID: col.ID,
// We use this PR(https://github.com/pingcap/tidb/pull/6274) as the dividing line to define whether it is a new version or an old version TiDB.
// The old version TiDB initializes the column's offset and state here.
// The new version TiDB doesn't initialize the column's offset and state, and it will do the initialization in run DDL function.
// When we do the rolling upgrade the following may happen:
// a new version TiDB builds the DDL job that doesn't be set the column's offset and state,
// and the old version TiDB is the DDL owner, it doesn't get offset and state from the store. Then it will encounter errors.
// So here we set offset and state to support the rolling upgrade.
Offset: col.Offset,
State: col.State,
OriginDefaultValue: col.OriginDefaultValue,
FieldType: *specNewColumn.Tp,
Name: newColName,
})
err = setCharsetCollationFlenDecimal(&newCol.FieldType)
if err != nil {
return nil, errors.Trace(err)
}
err = modifiable(&col.FieldType, &newCol.FieldType)
if err != nil {
return nil, errors.Trace(err)
}
if err = setDefaultAndComment(ctx, newCol, specNewColumn.Options); err != nil {
return nil, errors.Trace(err)
}
// Copy index related options to the new spec.
indexFlags := col.FieldType.Flag & (mysql.PriKeyFlag | mysql.UniqueKeyFlag | mysql.MultipleKeyFlag)
newCol.FieldType.Flag |= indexFlags
if mysql.HasPriKeyFlag(col.FieldType.Flag) {
newCol.FieldType.Flag |= mysql.NotNullFlag
// TODO: If user explicitly set NULL, we should throw error ErrPrimaryCantHaveNull.
}
// We don't support modifying column from not_auto_increment to auto_increment.
if !mysql.HasAutoIncrementFlag(col.Flag) && mysql.HasAutoIncrementFlag(newCol.Flag) {
return nil, errUnsupportedModifyColumn.GenByArgs("set auto_increment")
}
// We don't support modifying the type definitions from 'null' to 'not null' now.
if !mysql.HasNotNullFlag(col.Flag) && mysql.HasNotNullFlag(newCol.Flag) {
return nil, errUnsupportedModifyColumn.GenByArgs("null to not null")
}
// As same with MySQL, we don't support modifying the stored status for generated columns.
if err = checkModifyGeneratedColumn(t.Cols(), col, newCol); err != nil {
return nil, errors.Trace(err)
}
job := &model.Job{
SchemaID: schema.ID,
TableID: t.Meta().ID,
Type: model.ActionModifyColumn,
BinlogInfo: &model.HistoryInfo{},
Args: []interface{}{&newCol, originalColName, spec.Position},
}
return job, nil
}
// ChangeColumn renames an existing column and modifies the column's definition,
// currently we only support limited kind of changes
// that do not need to change or check data on the table.
func (d *ddl) ChangeColumn(ctx sessionctx.Context, ident ast.Ident, spec *ast.AlterTableSpec) error {
specNewColumn := spec.NewColumns[0]
if len(specNewColumn.Name.Schema.O) != 0 && ident.Schema.L != specNewColumn.Name.Schema.L {
return ErrWrongDBName.GenByArgs(specNewColumn.Name.Schema.O)
}
if len(spec.OldColumnName.Schema.O) != 0 && ident.Schema.L != spec.OldColumnName.Schema.L {
return ErrWrongDBName.GenByArgs(spec.OldColumnName.Schema.O)
}
if len(specNewColumn.Name.Table.O) != 0 && ident.Name.L != specNewColumn.Name.Table.L {
return ErrWrongTableName.GenByArgs(specNewColumn.Name.Table.O)
}
if len(spec.OldColumnName.Table.O) != 0 && ident.Name.L != spec.OldColumnName.Table.L {
return ErrWrongTableName.GenByArgs(spec.OldColumnName.Table.O)
}
job, err := d.getModifiableColumnJob(ctx, ident, spec.OldColumnName.Name, spec)
if err != nil {
return errors.Trace(err)
}
err = d.doDDLJob(ctx, job)
err = d.callHookOnChanged(err)
return errors.Trace(err)
}
// 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 sessionctx.Context, ident ast.Ident, spec *ast.AlterTableSpec) error {
specNewColumn := spec.NewColumns[0]
if len(specNewColumn.Name.Schema.O) != 0 && ident.Schema.L != specNewColumn.Name.Schema.L {
return ErrWrongDBName.GenByArgs(specNewColumn.Name.Schema.O)
}
if len(specNewColumn.Name.Table.O) != 0 && ident.Name.L != specNewColumn.Name.Table.L {
return ErrWrongTableName.GenByArgs(specNewColumn.Name.Table.O)
}
originalColName := specNewColumn.Name.Name
job, err := d.getModifiableColumnJob(ctx, ident, originalColName, spec)
if err != nil {
return errors.Trace(err)
}
err = d.doDDLJob(ctx, job)
err = d.callHookOnChanged(err)
return errors.Trace(err)
}
func (d *ddl) AlterColumn(ctx sessionctx.Context, ident ast.Ident, spec *ast.AlterTableSpec) error {
specNewColumn := spec.NewColumns[0]
is := d.infoHandle.Get()
schema, ok := is.SchemaByName(ident.Schema)
if !ok {
return infoschema.ErrTableNotExists.GenByArgs(ident.Schema, ident.Name)
}
t, err := is.TableByName(ident.Schema, ident.Name)
if err != nil {
return infoschema.ErrTableNotExists.GenByArgs(ident.Schema, ident.Name)
}
colName := specNewColumn.Name.Name
// Check whether alter column has existed.
col := table.FindCol(t.Cols(), colName.L)
if col == nil {
return errBadField.GenByArgs(colName, ident.Name)
}
// Clean the NoDefaultValueFlag value.
col.Flag &= ^mysql.NoDefaultValueFlag
if len(specNewColumn.Options) == 0 {
col.DefaultValue = nil
setNoDefaultValueFlag(col, false)
} else {
err = setDefaultValue(ctx, col, specNewColumn.Options[0])
if err != nil {
return errors.Trace(err)
}
}
job := &model.Job{
SchemaID: schema.ID,
TableID: t.Meta().ID,
Type: model.ActionSetDefaultValue,
BinlogInfo: &model.HistoryInfo{},
Args: []interface{}{col},
}
err = d.doDDLJob(ctx, job)
err = d.callHookOnChanged(err)
return errors.Trace(err)
}
// AlterTableComment updates the table comment information.
func (d *ddl) AlterTableComment(ctx sessionctx.Context, ident ast.Ident, spec *ast.AlterTableSpec) error {
is := d.infoHandle.Get()
schema, ok := is.SchemaByName(ident.Schema)
if !ok {
return infoschema.ErrDatabaseNotExists.GenByArgs(ident.Schema)
}
tb, err := is.TableByName(ident.Schema, ident.Name)
if err != nil {
return errors.Trace(infoschema.ErrTableNotExists.GenByArgs(ident.Schema, ident.Name))
}
job := &model.Job{
SchemaID: schema.ID,
TableID: tb.Meta().ID,
Type: model.ActionModifyTableComment,
BinlogInfo: &model.HistoryInfo{},
Args: []interface{}{spec.Comment},
}
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 sessionctx.Context, ti ast.Ident) (err error) {
is := d.GetInformationSchema()
schema, ok := is.SchemaByName(ti.Schema)
if !ok {
return infoschema.ErrDatabaseNotExists.GenByArgs(ti.Schema)
}
tb, err := is.TableByName(ti.Schema, ti.Name)
if err != nil {
return errors.Trace(infoschema.ErrTableNotExists.GenByArgs(ti.Schema, ti.Name))
}
job := &model.Job{
SchemaID: schema.ID,
TableID: tb.Meta().ID,
Type: model.ActionDropTable,
BinlogInfo: &model.HistoryInfo{},
}
err = d.doDDLJob(ctx, job)
err = d.callHookOnChanged(err)
return errors.Trace(err)
}
func (d *ddl) TruncateTable(ctx sessionctx.Context, ti ast.Ident) error {
is := d.GetInformationSchema()
schema, ok := is.SchemaByName(ti.Schema)
if !ok {
return infoschema.ErrDatabaseNotExists.GenByArgs(ti.Schema)
}
tb, err := is.TableByName(ti.Schema, ti.Name)
if err != nil {
return errors.Trace(infoschema.ErrTableNotExists.GenByArgs(ti.Schema, ti.Name))
}
newTableID, err := d.genGlobalID()
if err != nil {
return errors.Trace(err)
}
job := &model.Job{
SchemaID: schema.ID,
TableID: tb.Meta().ID,
Type: model.ActionTruncateTable,
BinlogInfo: &model.HistoryInfo{},
Args: []interface{}{newTableID},
}
err = d.doDDLJob(ctx, job)
err = d.callHookOnChanged(err)
return errors.Trace(err)
}
func (d *ddl) RenameTable(ctx sessionctx.Context, oldIdent, newIdent ast.Ident) error {
is := d.GetInformationSchema()
oldSchema, ok := is.SchemaByName(oldIdent.Schema)
if !ok {
return errFileNotFound.GenByArgs(oldIdent.Schema, oldIdent.Name)
}
oldTbl, err := is.TableByName(oldIdent.Schema, oldIdent.Name)
if err != nil {
return errFileNotFound.GenByArgs(oldIdent.Schema, oldIdent.Name)
}
newSchema, ok := is.SchemaByName(newIdent.Schema)
if !ok {
return errErrorOnRename.GenByArgs(oldIdent.Schema, oldIdent.Name, newIdent.Schema, newIdent.Name)
}
if is.TableExists(newIdent.Schema, newIdent.Name) {
return infoschema.ErrTableExists.GenByArgs(newIdent)
}
job := &model.Job{
SchemaID: newSchema.ID,
TableID: oldTbl.Meta().ID,
Type: model.ActionRenameTable,
BinlogInfo: &model.HistoryInfo{},
Args: []interface{}{oldSchema.ID, newIdent.Name},
}
err = d.doDDLJob(ctx, job)
err = d.callHookOnChanged(err)
return errors.Trace(err)
}
func getAnonymousIndex(t table.Table, colName model.CIStr) model.CIStr {
id := 2
l := len(t.Indices())
indexName := colName
for i := 0; i < l; i++ {
if t.Indices()[i].Meta().Name.L == indexName.L {
indexName = model.NewCIStr(fmt.Sprintf("%s_%d", colName.O, id))
i = -1
id++
}
}
return indexName
}
func (d *ddl) CreateIndex(ctx sessionctx.Context, ti ast.Ident, unique bool, indexName model.CIStr,
idxColNames []*ast.IndexColName, indexOption *ast.IndexOption) error {
is := d.infoHandle.Get()
schema, ok := is.SchemaByName(ti.Schema)
if !ok {
return infoschema.ErrDatabaseNotExists.GenByArgs(ti.Schema)
}
t, err := is.TableByName(ti.Schema, ti.Name)
if err != nil {
return errors.Trace(infoschema.ErrTableNotExists.GenByArgs(ti.Schema, ti.Name))
}
// Deal with anonymous index.
if len(indexName.L) == 0 {
indexName = getAnonymousIndex(t, idxColNames[0].Column.Name)
}
if indexInfo := findIndexByName(indexName.L, t.Meta().Indices); indexInfo != nil {
return ErrDupKeyName.Gen("index already exist %s", indexName)
}
if err = checkTooLongIndex(indexName); err != nil {
return errors.Trace(err)
}
if indexOption != nil {
// May be truncate comment here, when index comment too long and sql_mode is't strict.
indexOption.Comment, err = validateCommentLength(ctx.GetSessionVars(),
indexOption.Comment,
maxCommentLength,
errTooLongIndexComment.GenByArgs(indexName.String(), maxCommentLength))
if err != nil {
return errors.Trace(err)
}
}
job := &model.Job{
SchemaID: schema.ID,
TableID: t.Meta().ID,
Type: model.ActionAddIndex,
BinlogInfo: &model.HistoryInfo{},
Args: []interface{}{unique, indexName, idxColNames, indexOption},
}
err = d.doDDLJob(ctx, job)
err = d.callHookOnChanged(err)
return errors.Trace(err)
}
func buildFKInfo(fkName model.CIStr, keys []*ast.IndexColName, refer *ast.ReferenceDef) (*model.FKInfo, error) {
var fkInfo model.FKInfo
fkInfo.Name = fkName
fkInfo.RefTable = refer.Table.Name
fkInfo.Cols = make([]model.CIStr, len(keys))
for i, key := range keys {
fkInfo.Cols[i] = key.Column.Name
}
fkInfo.RefCols = make([]model.CIStr, len(refer.IndexColNames))
for i, key := range refer.IndexColNames {
fkInfo.RefCols[i] = key.Column.Name
}
fkInfo.OnDelete = int(refer.OnDelete.ReferOpt)
fkInfo.OnUpdate = int(refer.OnUpdate.ReferOpt)
return &fkInfo, nil
}
func (d *ddl) CreateForeignKey(ctx sessionctx.Context, ti ast.Ident, fkName model.CIStr, keys []*ast.IndexColName, refer *ast.ReferenceDef) error {
is := d.infoHandle.Get()
schema, ok := is.SchemaByName(ti.Schema)
if !ok {
return infoschema.ErrDatabaseNotExists.GenByArgs(ti.Schema)
}
t, err := is.TableByName(ti.Schema, ti.Name)
if err != nil {
return errors.Trace(infoschema.ErrTableNotExists.GenByArgs(ti.Schema, ti.Name))
}
fkInfo, err := buildFKInfo(fkName, keys, refer)
if err != nil {
return errors.Trace(err)
}
job := &model.Job{
SchemaID: schema.ID,
TableID: t.Meta().ID,
Type: model.ActionAddForeignKey,
BinlogInfo: &model.HistoryInfo{},
Args: []interface{}{fkInfo},
}
err = d.doDDLJob(ctx, job)
err = d.callHookOnChanged(err)
return errors.Trace(err)
}
func (d *ddl) DropForeignKey(ctx sessionctx.Context, ti ast.Ident, fkName model.CIStr) error {
is := d.infoHandle.Get()
schema, ok := is.SchemaByName(ti.Schema)
if !ok {
return infoschema.ErrDatabaseNotExists.GenByArgs(ti.Schema)
}
t, err := is.TableByName(ti.Schema, ti.Name)
if err != nil {
return errors.Trace(infoschema.ErrTableNotExists.GenByArgs(ti.Schema, ti.Name))
}
job := &model.Job{
SchemaID: schema.ID,
TableID: t.Meta().ID,
Type: model.ActionDropForeignKey,
BinlogInfo: &model.HistoryInfo{},
Args: []interface{}{fkName},
}
err = d.doDDLJob(ctx, job)
err = d.callHookOnChanged(err)
return errors.Trace(err)
}
func (d *ddl) DropIndex(ctx sessionctx.Context, ti ast.Ident, indexName model.CIStr) error {
is := d.infoHandle.Get()
schema, ok := is.SchemaByName(ti.Schema)
if !ok {
return errors.Trace(infoschema.ErrDatabaseNotExists)
}
t, err := is.TableByName(ti.Schema, ti.Name)
if err != nil {
return errors.Trace(infoschema.ErrTableNotExists.GenByArgs(ti.Schema, ti.Name))
}
if indexInfo := findIndexByName(indexName.L, t.Meta().Indices); indexInfo == nil {
return ErrCantDropFieldOrKey.Gen("index %s doesn't exist", indexName)
}
job := &model.Job{
SchemaID: schema.ID,
TableID: t.Meta().ID,
Type: model.ActionDropIndex,
BinlogInfo: &model.HistoryInfo{},
Args: []interface{}{indexName},
}
err = d.doDDLJob(ctx, job)
err = d.callHookOnChanged(err)
return errors.Trace(err)
}
func isDroppableColumn(tblInfo *model.TableInfo, colName model.CIStr) error {
// Check whether there are other columns depend on this column or not.
for _, col := range tblInfo.Columns {
for dep := range col.Dependences {
if dep == colName.L {
return errDependentByGeneratedColumn.GenByArgs(dep)
}
}
}
if len(tblInfo.Columns) == 1 {
return ErrCantRemoveAllFields.Gen("can't drop only column %s in table %s",
colName, tblInfo.Name)
}
// We don't support dropping column with index covered now.
// We must drop the index first, then drop the column.
if isColumnWithIndex(colName.L, tblInfo.Indices) {
return errCantDropColWithIndex.Gen("can't drop column %s with index covered now", colName)
}
return nil
}
// validateCommentLength checks comment length of table, column, index and partition.
// If comment length is more than the standard length truncate it
// and store the comment length upto the standard comment length size.
func validateCommentLength(vars *variable.SessionVars, comment string, maxLen int, err error) (string, error) {
if len(comment) > maxLen {
if vars.StrictSQLMode {
return "", err
}
vars.StmtCtx.AppendWarning(err)
return comment[:maxLen], nil
}
return comment, nil
}
Loading...
马建仓 AI 助手
尝试更多
代码解读
代码找茬
代码优化
Go
1
https://gitee.com/pingcap/tidb.git
git@gitee.com:pingcap/tidb.git
pingcap
tidb
tidb
v2.0.6

搜索帮助

0d507c66 1850385 C8b1a773 1850385