1 Star 0 Fork 0

jiangplus / tidb-parser

加入 Gitee
与超过 1200万 开发者一起发现、参与优秀开源项目,私有仓库也完全免费 :)
免费加入
克隆/下载
ddl.go 63.50 KB
一键复制 编辑 原始数据 按行查看 历史
123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229230231232233234235236237238239240241242243244245246247248249250251252253254255256257258259260261262263264265266267268269270271272273274275276277278279280281282283284285286287288289290291292293294295296297298299300301302303304305306307308309310311312313314315316317318319320321322323324325326327328329330331332333334335336337338339340341342343344345346347348349350351352353354355356357358359360361362363364365366367368369370371372373374375376377378379380381382383384385386387388389390391392393394395396397398399400401402403404405406407408409410411412413414415416417418419420421422423424425426427428429430431432433434435436437438439440441442443444445446447448449450451452453454455456457458459460461462463464465466467468469470471472473474475476477478479480481482483484485486487488489490491492493494495496497498499500501502503504505506507508509510511512513514515516517518519520521522523524525526527528529530531532533534535536537538539540541542543544545546547548549550551552553554555556557558559560561562563564565566567568569570571572573574575576577578579580581582583584585586587588589590591592593594595596597598599600601602603604605606607608609610611612613614615616617618619620621622623624625626627628629630631632633634635636637638639640641642643644645646647648649650651652653654655656657658659660661662663664665666667668669670671672673674675676677678679680681682683684685686687688689690691692693694695696697698699700701702703704705706707708709710711712713714715716717718719720721722723724725726727728729730731732733734735736737738739740741742743744745746747748749750751752753754755756757758759760761762763764765766767768769770771772773774775776777778779780781782783784785786787788789790791792793794795796797798799800801802803804805806807808809810811812813814815816817818819820821822823824825826827828829830831832833834835836837838839840841842843844845846847848849850851852853854855856857858859860861862863864865866867868869870871872873874875876877878879880881882883884885886887888889890891892893894895896897898899900901902903904905906907908909910911912913914915916917918919920921922923924925926927928929930931932933934935936937938939940941942943944945946947948949950951952953954955956957958959960961962963964965966967968969970971972973974975976977978979980981982983984985986987988989990991992993994995996997998999100010011002100310041005100610071008100910101011101210131014101510161017101810191020102110221023102410251026102710281029103010311032103310341035103610371038103910401041104210431044104510461047104810491050105110521053105410551056105710581059106010611062106310641065106610671068106910701071107210731074107510761077107810791080108110821083108410851086108710881089109010911092109310941095109610971098109911001101110211031104110511061107110811091110111111121113111411151116111711181119112011211122112311241125112611271128112911301131113211331134113511361137113811391140114111421143114411451146114711481149115011511152115311541155115611571158115911601161116211631164116511661167116811691170117111721173117411751176117711781179118011811182118311841185118611871188118911901191119211931194119511961197119811991200120112021203120412051206120712081209121012111212121312141215121612171218121912201221122212231224122512261227122812291230123112321233123412351236123712381239124012411242124312441245124612471248124912501251125212531254125512561257125812591260126112621263126412651266126712681269127012711272127312741275127612771278127912801281128212831284128512861287128812891290129112921293129412951296129712981299130013011302130313041305130613071308130913101311131213131314131513161317131813191320132113221323132413251326132713281329133013311332133313341335133613371338133913401341134213431344134513461347134813491350135113521353135413551356135713581359136013611362136313641365136613671368136913701371137213731374137513761377137813791380138113821383138413851386138713881389139013911392139313941395139613971398139914001401140214031404140514061407140814091410141114121413141414151416141714181419142014211422142314241425142614271428142914301431143214331434143514361437143814391440144114421443144414451446144714481449145014511452145314541455145614571458145914601461146214631464146514661467146814691470147114721473147414751476147714781479148014811482148314841485148614871488148914901491149214931494149514961497149814991500150115021503150415051506150715081509151015111512151315141515151615171518151915201521152215231524152515261527152815291530153115321533153415351536153715381539154015411542154315441545154615471548154915501551155215531554155515561557155815591560156115621563156415651566156715681569157015711572157315741575157615771578157915801581158215831584158515861587158815891590159115921593159415951596159715981599160016011602160316041605160616071608160916101611161216131614161516161617161816191620162116221623162416251626162716281629163016311632163316341635163616371638163916401641164216431644164516461647164816491650165116521653165416551656165716581659166016611662166316641665166616671668166916701671167216731674167516761677167816791680168116821683168416851686168716881689169016911692169316941695169616971698169917001701170217031704170517061707170817091710171117121713171417151716171717181719172017211722172317241725172617271728172917301731173217331734173517361737173817391740174117421743174417451746174717481749175017511752175317541755175617571758175917601761176217631764176517661767176817691770177117721773177417751776177717781779178017811782178317841785178617871788178917901791179217931794179517961797179817991800180118021803180418051806180718081809181018111812181318141815181618171818181918201821182218231824182518261827182818291830183118321833183418351836183718381839184018411842184318441845184618471848184918501851185218531854185518561857185818591860186118621863186418651866186718681869187018711872187318741875187618771878187918801881188218831884188518861887188818891890189118921893189418951896189718981899190019011902190319041905190619071908190919101911191219131914191519161917191819191920192119221923192419251926192719281929193019311932193319341935193619371938193919401941194219431944194519461947194819491950195119521953195419551956195719581959196019611962196319641965196619671968196919701971197219731974197519761977197819791980198119821983198419851986198719881989199019911992199319941995199619971998199920002001200220032004200520062007200820092010201120122013201420152016201720182019202020212022202320242025202620272028202920302031203220332034203520362037203820392040204120422043204420452046204720482049205020512052205320542055205620572058205920602061206220632064206520662067206820692070207120722073207420752076207720782079208020812082208320842085208620872088208920902091209220932094209520962097209820992100210121022103210421052106210721082109211021112112211321142115211621172118211921202121212221232124212521262127212821292130213121322133213421352136213721382139214021412142214321442145214621472148214921502151215221532154215521562157215821592160216121622163216421652166216721682169217021712172217321742175217621772178217921802181218221832184218521862187218821892190219121922193219421952196219721982199220022012202220322042205220622072208220922102211221222132214221522162217221822192220222122222223222422252226222722282229223022312232223322342235223622372238223922402241224222432244224522462247224822492250225122522253225422552256225722582259226022612262226322642265226622672268226922702271227222732274227522762277227822792280228122822283228422852286228722882289229022912292229322942295229622972298229923002301230223032304230523062307230823092310231123122313231423152316231723182319232023212322232323242325232623272328232923302331233223332334233523362337233823392340234123422343234423452346234723482349235023512352235323542355235623572358235923602361236223632364236523662367236823692370237123722373237423752376237723782379238023812382238323842385238623872388238923902391239223932394239523962397239823992400240124022403240424052406240724082409241024112412241324142415241624172418241924202421242224232424242524262427242824292430243124322433243424352436243724382439244024412442244324442445244624472448244924502451245224532454245524562457245824592460246124622463246424652466246724682469247024712472247324742475247624772478247924802481248224832484248524862487248824892490249124922493249424952496249724982499250025012502250325042505250625072508250925102511251225132514251525162517
// 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 ast
import (
"strings"
"github.com/pingcap/errors"
"github.com/pingcap/parser/auth"
. "github.com/pingcap/parser/format"
"github.com/pingcap/parser/model"
"github.com/pingcap/parser/mysql"
"github.com/pingcap/parser/terror"
"github.com/pingcap/parser/types"
)
var (
_ DDLNode = &AlterTableStmt{}
_ DDLNode = &CreateDatabaseStmt{}
_ DDLNode = &CreateIndexStmt{}
_ DDLNode = &CreateTableStmt{}
_ DDLNode = &CreateViewStmt{}
_ DDLNode = &DropDatabaseStmt{}
_ DDLNode = &DropIndexStmt{}
_ DDLNode = &DropTableStmt{}
_ DDLNode = &RenameTableStmt{}
_ DDLNode = &TruncateTableStmt{}
_ Node = &AlterTableSpec{}
_ Node = &ColumnDef{}
_ Node = &ColumnOption{}
_ Node = &ColumnPosition{}
_ Node = &Constraint{}
_ Node = &IndexColName{}
_ Node = &ReferenceDef{}
)
// CharsetOpt is used for parsing charset option from SQL.
type CharsetOpt struct {
Chs string
Col string
}
// DatabaseOptionType is the type for database options.
type DatabaseOptionType int
// Database option types.
const (
DatabaseOptionNone DatabaseOptionType = iota
DatabaseOptionCharset
DatabaseOptionCollate
)
// DatabaseOption represents database option.
type DatabaseOption struct {
Tp DatabaseOptionType
Value string
}
// Restore implements Node interface.
func (n *DatabaseOption) Restore(ctx *RestoreCtx) error {
switch n.Tp {
case DatabaseOptionCharset:
ctx.WriteKeyWord("CHARACTER SET")
ctx.WritePlain(" = ")
ctx.WritePlain(n.Value)
case DatabaseOptionCollate:
ctx.WriteKeyWord("COLLATE")
ctx.WritePlain(" = ")
ctx.WritePlain(n.Value)
default:
return errors.Errorf("invalid DatabaseOptionType: %d", n.Tp)
}
return nil
}
// CreateDatabaseStmt is a statement to create a database.
// See https://dev.mysql.com/doc/refman/5.7/en/create-database.html
type CreateDatabaseStmt struct {
ddlNode
IfNotExists bool
Name string
Options []*DatabaseOption
}
// Restore implements Node interface.
func (n *CreateDatabaseStmt) Restore(ctx *RestoreCtx) error {
ctx.WriteKeyWord("CREATE DATABASE ")
if n.IfNotExists {
ctx.WriteKeyWord("IF NOT EXISTS ")
}
ctx.WriteName(n.Name)
for i, option := range n.Options {
ctx.WritePlain(" ")
err := option.Restore(ctx)
if err != nil {
return errors.Annotatef(err, "An error occurred while splicing CreateDatabaseStmt DatabaseOption: [%v]", i)
}
}
return nil
}
// Accept implements Node Accept interface.
func (n *CreateDatabaseStmt) Accept(v Visitor) (Node, bool) {
newNode, skipChildren := v.Enter(n)
if skipChildren {
return v.Leave(newNode)
}
n = newNode.(*CreateDatabaseStmt)
return v.Leave(n)
}
// AlterDatabaseStmt is a statement to change the structure of a database.
// See https://dev.mysql.com/doc/refman/5.7/en/alter-database.html
type AlterDatabaseStmt struct {
ddlNode
Name string
AlterDefaultDatabase bool
Options []*DatabaseOption
}
// Restore implements Node interface.
func (n *AlterDatabaseStmt) Restore(ctx *RestoreCtx) error {
ctx.WriteKeyWord("ALTER DATABASE")
if !n.AlterDefaultDatabase {
ctx.WritePlain(" ")
ctx.WriteName(n.Name)
}
for i, option := range n.Options {
ctx.WritePlain(" ")
err := option.Restore(ctx)
if err != nil {
return errors.Annotatef(err, "An error occurred while splicing AlterDatabaseStmt DatabaseOption: [%v]", i)
}
}
return nil
}
// Accept implements Node Accept interface.
func (n *AlterDatabaseStmt) Accept(v Visitor) (Node, bool) {
newNode, skipChildren := v.Enter(n)
if skipChildren {
return v.Leave(newNode)
}
n = newNode.(*AlterDatabaseStmt)
return v.Leave(n)
}
// DropDatabaseStmt is a statement to drop a database and all tables in the database.
// See https://dev.mysql.com/doc/refman/5.7/en/drop-database.html
type DropDatabaseStmt struct {
ddlNode
IfExists bool
Name string
}
// Restore implements Node interface.
func (n *DropDatabaseStmt) Restore(ctx *RestoreCtx) error {
ctx.WriteKeyWord("DROP DATABASE ")
if n.IfExists {
ctx.WriteKeyWord("IF EXISTS ")
}
ctx.WriteName(n.Name)
return nil
}
// Accept implements Node Accept interface.
func (n *DropDatabaseStmt) Accept(v Visitor) (Node, bool) {
newNode, skipChildren := v.Enter(n)
if skipChildren {
return v.Leave(newNode)
}
n = newNode.(*DropDatabaseStmt)
return v.Leave(n)
}
// IndexColName is used for parsing index column name from SQL.
type IndexColName struct {
node
Column *ColumnName
Length int
}
// Restore implements Node interface.
func (n *IndexColName) Restore(ctx *RestoreCtx) error {
if err := n.Column.Restore(ctx); err != nil {
return errors.Annotate(err, "An error occurred while splicing IndexColName")
}
if n.Length > 0 {
ctx.WritePlainf("(%d)", n.Length)
}
return nil
}
// Accept implements Node Accept interface.
func (n *IndexColName) Accept(v Visitor) (Node, bool) {
newNode, skipChildren := v.Enter(n)
if skipChildren {
return v.Leave(newNode)
}
n = newNode.(*IndexColName)
node, ok := n.Column.Accept(v)
if !ok {
return n, false
}
n.Column = node.(*ColumnName)
return v.Leave(n)
}
// MatchType is the type for reference match type.
type MatchType int
// match type
const (
MatchNone MatchType = iota
MatchFull
MatchPartial
MatchSimple
)
// 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 {
node
Table *TableName
IndexColNames []*IndexColName
OnDelete *OnDeleteOpt
OnUpdate *OnUpdateOpt
Match MatchType
}
// Restore implements Node interface.
func (n *ReferenceDef) Restore(ctx *RestoreCtx) error {
if n.Table != nil {
ctx.WriteKeyWord("REFERENCES ")
if err := n.Table.Restore(ctx); err != nil {
return errors.Annotate(err, "An error occurred while splicing ReferenceDef")
}
}
ctx.WritePlain("(")
for i, indexColNames := range n.IndexColNames {
if i > 0 {
ctx.WritePlain(", ")
}
if err := indexColNames.Restore(ctx); err != nil {
return errors.Annotatef(err, "An error occurred while splicing IndexColNames: [%v]", i)
}
}
ctx.WritePlain(")")
if n.Match != MatchNone {
ctx.WriteKeyWord(" MATCH ")
switch n.Match {
case MatchFull:
ctx.WriteKeyWord("FULL")
case MatchPartial:
ctx.WriteKeyWord("PARTIAL")
case MatchSimple:
ctx.WriteKeyWord("SIMPLE")
}
}
if n.OnDelete.ReferOpt != ReferOptionNoOption {
ctx.WritePlain(" ")
if err := n.OnDelete.Restore(ctx); err != nil {
return errors.Annotate(err, "An error occurred while splicing OnDelete")
}
}
if n.OnUpdate.ReferOpt != ReferOptionNoOption {
ctx.WritePlain(" ")
if err := n.OnUpdate.Restore(ctx); err != nil {
return errors.Annotate(err, "An error occurred while splicing OnUpdate")
}
}
return nil
}
// Accept implements Node Accept interface.
func (n *ReferenceDef) Accept(v Visitor) (Node, bool) {
newNode, skipChildren := v.Enter(n)
if skipChildren {
return v.Leave(newNode)
}
n = newNode.(*ReferenceDef)
node, ok := n.Table.Accept(v)
if !ok {
return n, false
}
n.Table = node.(*TableName)
for i, val := range n.IndexColNames {
node, ok = val.Accept(v)
if !ok {
return n, false
}
n.IndexColNames[i] = node.(*IndexColName)
}
onDelete, ok := n.OnDelete.Accept(v)
if !ok {
return n, false
}
n.OnDelete = onDelete.(*OnDeleteOpt)
onUpdate, ok := n.OnUpdate.Accept(v)
if !ok {
return n, false
}
n.OnUpdate = onUpdate.(*OnUpdateOpt)
return v.Leave(n)
}
// ReferOptionType is the type for refer options.
type ReferOptionType int
// Refer option types.
const (
ReferOptionNoOption ReferOptionType = iota
ReferOptionRestrict
ReferOptionCascade
ReferOptionSetNull
ReferOptionNoAction
ReferOptionSetDefault
)
// String implements fmt.Stringer interface.
func (r ReferOptionType) String() string {
switch r {
case ReferOptionRestrict:
return "RESTRICT"
case ReferOptionCascade:
return "CASCADE"
case ReferOptionSetNull:
return "SET NULL"
case ReferOptionNoAction:
return "NO ACTION"
case ReferOptionSetDefault:
return "SET DEFAULT"
}
return ""
}
// OnDeleteOpt is used for optional on delete clause.
type OnDeleteOpt struct {
node
ReferOpt ReferOptionType
}
// Restore implements Node interface.
func (n *OnDeleteOpt) Restore(ctx *RestoreCtx) error {
if n.ReferOpt != ReferOptionNoOption {
ctx.WriteKeyWord("ON DELETE ")
ctx.WriteKeyWord(n.ReferOpt.String())
}
return nil
}
// Accept implements Node Accept interface.
func (n *OnDeleteOpt) Accept(v Visitor) (Node, bool) {
newNode, skipChildren := v.Enter(n)
if skipChildren {
return v.Leave(newNode)
}
n = newNode.(*OnDeleteOpt)
return v.Leave(n)
}
// OnUpdateOpt is used for optional on update clause.
type OnUpdateOpt struct {
node
ReferOpt ReferOptionType
}
// Restore implements Node interface.
func (n *OnUpdateOpt) Restore(ctx *RestoreCtx) error {
if n.ReferOpt != ReferOptionNoOption {
ctx.WriteKeyWord("ON UPDATE ")
ctx.WriteKeyWord(n.ReferOpt.String())
}
return nil
}
// Accept implements Node Accept interface.
func (n *OnUpdateOpt) Accept(v Visitor) (Node, bool) {
newNode, skipChildren := v.Enter(n)
if skipChildren {
return v.Leave(newNode)
}
n = newNode.(*OnUpdateOpt)
return v.Leave(n)
}
// ColumnOptionType is the type for ColumnOption.
type ColumnOptionType int
// ColumnOption types.
const (
ColumnOptionNoOption ColumnOptionType = iota
ColumnOptionPrimaryKey
ColumnOptionNotNull
ColumnOptionAutoIncrement
ColumnOptionDefaultValue
ColumnOptionUniqKey
ColumnOptionNull
ColumnOptionOnUpdate // For Timestamp and Datetime only.
ColumnOptionFulltext
ColumnOptionComment
ColumnOptionGenerated
ColumnOptionReference
ColumnOptionCollate
ColumnOptionCheck
)
var (
invalidOptionForGeneratedColumn = map[ColumnOptionType]struct{}{
ColumnOptionAutoIncrement: {},
ColumnOptionOnUpdate: {},
ColumnOptionDefaultValue: {},
}
)
// ColumnOption is used for parsing column constraint info from SQL.
type ColumnOption struct {
node
Tp ColumnOptionType
// Expr is used for ColumnOptionDefaultValue/ColumnOptionOnUpdateColumnOptionGenerated.
// For ColumnOptionDefaultValue or ColumnOptionOnUpdate, it's the target value.
// For ColumnOptionGenerated, it's the target expression.
Expr ExprNode
// Stored is only for ColumnOptionGenerated, default is false.
Stored bool
// Refer is used for foreign key.
Refer *ReferenceDef
StrValue string
// Enforced is only for Check, default is true.
Enforced bool
}
// Restore implements Node interface.
func (n *ColumnOption) Restore(ctx *RestoreCtx) error {
switch n.Tp {
case ColumnOptionNoOption:
return nil
case ColumnOptionPrimaryKey:
ctx.WriteKeyWord("PRIMARY KEY")
case ColumnOptionNotNull:
ctx.WriteKeyWord("NOT NULL")
case ColumnOptionAutoIncrement:
ctx.WriteKeyWord("AUTO_INCREMENT")
case ColumnOptionDefaultValue:
ctx.WriteKeyWord("DEFAULT ")
if err := n.Expr.Restore(ctx); err != nil {
return errors.Annotate(err, "An error occurred while splicing ColumnOption DefaultValue Expr")
}
case ColumnOptionUniqKey:
ctx.WriteKeyWord("UNIQUE KEY")
case ColumnOptionNull:
ctx.WriteKeyWord("NULL")
case ColumnOptionOnUpdate:
ctx.WriteKeyWord("ON UPDATE ")
if err := n.Expr.Restore(ctx); err != nil {
return errors.Annotate(err, "An error occurred while splicing ColumnOption ON UPDATE Expr")
}
case ColumnOptionFulltext:
return errors.New("TiDB Parser ignore the `ColumnOptionFulltext` type now")
case ColumnOptionComment:
ctx.WriteKeyWord("COMMENT ")
if err := n.Expr.Restore(ctx); err != nil {
return errors.Annotate(err, "An error occurred while splicing ColumnOption COMMENT Expr")
}
case ColumnOptionGenerated:
ctx.WriteKeyWord("GENERATED ALWAYS AS")
ctx.WritePlain("(")
if err := n.Expr.Restore(ctx); err != nil {
return errors.Annotate(err, "An error occurred while splicing ColumnOption GENERATED ALWAYS Expr")
}
ctx.WritePlain(")")
if n.Stored {
ctx.WriteKeyWord(" STORED")
} else {
ctx.WriteKeyWord(" VIRTUAL")
}
case ColumnOptionReference:
if err := n.Refer.Restore(ctx); err != nil {
return errors.Annotate(err, "An error occurred while splicing ColumnOption ReferenceDef")
}
case ColumnOptionCollate:
if n.StrValue == "" {
return errors.New("Empty ColumnOption COLLATE")
}
ctx.WriteKeyWord("COLLATE ")
ctx.WritePlain(n.StrValue)
case ColumnOptionCheck:
ctx.WriteKeyWord("CHECK")
ctx.WritePlain("(")
if err := n.Expr.Restore(ctx); err != nil {
return errors.Trace(err)
}
ctx.WritePlain(")")
if n.Enforced {
ctx.WriteKeyWord(" ENFORCED")
} else {
ctx.WriteKeyWord(" NOT ENFORCED")
}
default:
return errors.New("An error occurred while splicing ColumnOption")
}
return nil
}
// Accept implements Node Accept interface.
func (n *ColumnOption) Accept(v Visitor) (Node, bool) {
newNode, skipChildren := v.Enter(n)
if skipChildren {
return v.Leave(newNode)
}
n = newNode.(*ColumnOption)
if n.Expr != nil {
node, ok := n.Expr.Accept(v)
if !ok {
return n, false
}
n.Expr = node.(ExprNode)
}
return v.Leave(n)
}
// IndexOption is the index options.
// KEY_BLOCK_SIZE [=] value
// | index_type
// | WITH PARSER parser_name
// | COMMENT 'string'
// See http://dev.mysql.com/doc/refman/5.7/en/create-table.html
type IndexOption struct {
node
KeyBlockSize uint64
Tp model.IndexType
Comment string
}
// Restore implements Node interface.
func (n *IndexOption) Restore(ctx *RestoreCtx) error {
hasPrevOption := false
if n.KeyBlockSize > 0 {
ctx.WriteKeyWord("KEY_BLOCK_SIZE")
ctx.WritePlainf("=%d", n.KeyBlockSize)
hasPrevOption = true
}
if n.Tp != model.IndexTypeInvalid {
if hasPrevOption {
ctx.WritePlain(" ")
}
ctx.WriteKeyWord("USING ")
ctx.WritePlain(n.Tp.String())
hasPrevOption = true
}
if n.Comment != "" {
if hasPrevOption {
ctx.WritePlain(" ")
}
ctx.WriteKeyWord("COMMENT ")
ctx.WriteString(n.Comment)
}
return nil
}
// Accept implements Node Accept interface.
func (n *IndexOption) Accept(v Visitor) (Node, bool) {
newNode, skipChildren := v.Enter(n)
if skipChildren {
return v.Leave(newNode)
}
n = newNode.(*IndexOption)
return v.Leave(n)
}
// ConstraintType is the type for Constraint.
type ConstraintType int
// ConstraintTypes
const (
ConstraintNoConstraint ConstraintType = iota
ConstraintPrimaryKey
ConstraintKey
ConstraintIndex
ConstraintUniq
ConstraintUniqKey
ConstraintUniqIndex
ConstraintForeignKey
ConstraintFulltext
ConstraintCheck
)
// Constraint is constraint for table definition.
type Constraint struct {
node
Tp ConstraintType
Name string
Keys []*IndexColName // Used for PRIMARY KEY, UNIQUE, ......
Refer *ReferenceDef // Used for foreign key.
Option *IndexOption // Index Options
Expr ExprNode // Used for Check
Enforced bool // Used for Check
}
// Restore implements Node interface.
func (n *Constraint) Restore(ctx *RestoreCtx) error {
switch n.Tp {
case ConstraintNoConstraint:
return nil
case ConstraintPrimaryKey:
ctx.WriteKeyWord("PRIMARY KEY")
case ConstraintKey:
ctx.WriteKeyWord("KEY")
case ConstraintIndex:
ctx.WriteKeyWord("INDEX")
case ConstraintUniq:
ctx.WriteKeyWord("UNIQUE")
case ConstraintUniqKey:
ctx.WriteKeyWord("UNIQUE KEY")
case ConstraintUniqIndex:
ctx.WriteKeyWord("UNIQUE INDEX")
case ConstraintFulltext:
ctx.WriteKeyWord("FULLTEXT")
case ConstraintCheck:
if n.Name != "" {
ctx.WriteKeyWord("CONSTRAINT ")
ctx.WriteName(n.Name)
ctx.WritePlain(" ")
}
ctx.WriteKeyWord("CHECK")
ctx.WritePlain("(")
if err := n.Expr.Restore(ctx); err != nil {
return errors.Trace(err)
}
ctx.WritePlain(") ")
if n.Enforced {
ctx.WriteKeyWord("ENFORCED")
} else {
ctx.WriteKeyWord("NOT ENFORCED")
}
return nil
}
if n.Tp == ConstraintForeignKey {
ctx.WriteKeyWord("CONSTRAINT ")
if n.Name != "" {
ctx.WriteName(n.Name)
ctx.WritePlain(" ")
}
ctx.WriteKeyWord("FOREIGN KEY ")
} else if n.Name != "" {
ctx.WritePlain(" ")
ctx.WriteName(n.Name)
}
ctx.WritePlain("(")
for i, keys := range n.Keys {
if i > 0 {
ctx.WritePlain(", ")
}
if err := keys.Restore(ctx); err != nil {
return errors.Annotatef(err, "An error occurred while splicing Constraint Keys: [%v]", i)
}
}
ctx.WritePlain(")")
if n.Refer != nil {
ctx.WritePlain(" ")
if err := n.Refer.Restore(ctx); err != nil {
return errors.Annotate(err, "An error occurred while splicing Constraint Refer")
}
}
if n.Option != nil {
ctx.WritePlain(" ")
if err := n.Option.Restore(ctx); err != nil {
return errors.Annotate(err, "An error occurred while splicing Constraint Option")
}
}
return nil
}
// Accept implements Node Accept interface.
func (n *Constraint) Accept(v Visitor) (Node, bool) {
newNode, skipChildren := v.Enter(n)
if skipChildren {
return v.Leave(newNode)
}
n = newNode.(*Constraint)
for i, val := range n.Keys {
node, ok := val.Accept(v)
if !ok {
return n, false
}
n.Keys[i] = node.(*IndexColName)
}
if n.Refer != nil {
node, ok := n.Refer.Accept(v)
if !ok {
return n, false
}
n.Refer = node.(*ReferenceDef)
}
if n.Option != nil {
node, ok := n.Option.Accept(v)
if !ok {
return n, false
}
n.Option = node.(*IndexOption)
}
return v.Leave(n)
}
// ColumnDef is used for parsing column definition from SQL.
type ColumnDef struct {
node
Name *ColumnName
Tp *types.FieldType
Options []*ColumnOption
}
// Restore implements Node interface.
func (n *ColumnDef) Restore(ctx *RestoreCtx) error {
if err := n.Name.Restore(ctx); err != nil {
return errors.Annotate(err, "An error occurred while splicing ColumnDef Name")
}
if n.Tp != nil {
ctx.WritePlain(" ")
if err := n.Tp.Restore(ctx); err != nil {
return errors.Annotate(err, "An error occurred while splicing ColumnDef Type")
}
}
for i, options := range n.Options {
ctx.WritePlain(" ")
if err := options.Restore(ctx); err != nil {
return errors.Annotatef(err, "An error occurred while splicing ColumnDef ColumnOption: [%v]", i)
}
}
return nil
}
// Accept implements Node Accept interface.
func (n *ColumnDef) Accept(v Visitor) (Node, bool) {
newNode, skipChildren := v.Enter(n)
if skipChildren {
return v.Leave(newNode)
}
n = newNode.(*ColumnDef)
node, ok := n.Name.Accept(v)
if !ok {
return n, false
}
n.Name = node.(*ColumnName)
for i, val := range n.Options {
node, ok := val.Accept(v)
if !ok {
return n, false
}
n.Options[i] = node.(*ColumnOption)
}
return v.Leave(n)
}
// Validate checks if a column definition is legal.
// For example, generated column definitions that contain such
// column options as `ON UPDATE`, `AUTO_INCREMENT`, `DEFAULT`
// are illegal.
func (n *ColumnDef) Validate() bool {
generatedCol := false
illegalOpt4gc := false
for _, opt := range n.Options {
if opt.Tp == ColumnOptionGenerated {
generatedCol = true
}
_, found := invalidOptionForGeneratedColumn[opt.Tp]
illegalOpt4gc = illegalOpt4gc || found
}
return !(generatedCol && illegalOpt4gc)
}
// CreateTableStmt is a statement to create a table.
// See https://dev.mysql.com/doc/refman/5.7/en/create-table.html
type CreateTableStmt struct {
ddlNode
IfNotExists bool
Table *TableName
ReferTable *TableName
Cols []*ColumnDef
Constraints []*Constraint
Options []*TableOption
Partition *PartitionOptions
OnDuplicate OnDuplicateKeyHandlingType
Select ResultSetNode
}
// Restore implements Node interface.
func (n *CreateTableStmt) Restore(ctx *RestoreCtx) error {
ctx.WriteKeyWord("CREATE TABLE ")
if n.IfNotExists {
ctx.WriteKeyWord("IF NOT EXISTS ")
}
if err := n.Table.Restore(ctx); err != nil {
return errors.Annotate(err, "An error occurred while splicing CreateTableStmt Table")
}
ctx.WritePlain(" ")
if n.ReferTable != nil {
ctx.WriteKeyWord("LIKE ")
if err := n.ReferTable.Restore(ctx); err != nil {
return errors.Annotate(err, "An error occurred while splicing CreateTableStmt ReferTable")
}
}
lenCols := len(n.Cols)
lenConstraints := len(n.Constraints)
if lenCols+lenConstraints > 0 {
ctx.WritePlain("(")
for i, col := range n.Cols {
if i > 0 {
ctx.WritePlain(",")
}
if err := col.Restore(ctx); err != nil {
return errors.Annotatef(err, "An error occurred while splicing CreateTableStmt ColumnDef: [%v]", i)
}
}
for i, constraint := range n.Constraints {
if i > 0 || lenCols >= 1 {
ctx.WritePlain(",")
}
if err := constraint.Restore(ctx); err != nil {
return errors.Annotatef(err, "An error occurred while splicing CreateTableStmt Constraints: [%v]", i)
}
}
ctx.WritePlain(")")
}
for i, option := range n.Options {
ctx.WritePlain(" ")
if err := option.Restore(ctx); err != nil {
return errors.Annotatef(err, "An error occurred while splicing CreateTableStmt TableOption: [%v]", i)
}
}
if n.Partition != nil {
ctx.WritePlain(" ")
if err := n.Partition.Restore(ctx); err != nil {
return errors.Annotate(err, "An error occurred while splicing CreateTableStmt Partition")
}
}
if n.Select != nil {
switch n.OnDuplicate {
case OnDuplicateKeyHandlingError:
ctx.WriteKeyWord(" AS ")
case OnDuplicateKeyHandlingIgnore:
ctx.WriteKeyWord(" IGNORE AS ")
case OnDuplicateKeyHandlingReplace:
ctx.WriteKeyWord(" REPLACE AS ")
}
if err := n.Select.Restore(ctx); err != nil {
return errors.Annotate(err, "An error occurred while splicing CreateTableStmt Select")
}
}
return nil
}
// Accept implements Node Accept interface.
func (n *CreateTableStmt) Accept(v Visitor) (Node, bool) {
newNode, skipChildren := v.Enter(n)
if skipChildren {
return v.Leave(newNode)
}
n = newNode.(*CreateTableStmt)
node, ok := n.Table.Accept(v)
if !ok {
return n, false
}
n.Table = node.(*TableName)
if n.ReferTable != nil {
node, ok = n.ReferTable.Accept(v)
if !ok {
return n, false
}
n.ReferTable = node.(*TableName)
}
for i, val := range n.Cols {
node, ok = val.Accept(v)
if !ok {
return n, false
}
n.Cols[i] = node.(*ColumnDef)
}
for i, val := range n.Constraints {
node, ok = val.Accept(v)
if !ok {
return n, false
}
n.Constraints[i] = node.(*Constraint)
}
if n.Select != nil {
node, ok := n.Select.Accept(v)
if !ok {
return n, false
}
n.Select = node.(ResultSetNode)
}
if n.Partition != nil {
node, ok := n.Partition.Accept(v)
if !ok {
return n, false
}
n.Partition = node.(*PartitionOptions)
}
return v.Leave(n)
}
// DropTableStmt is a statement to drop one or more tables.
// See https://dev.mysql.com/doc/refman/5.7/en/drop-table.html
type DropTableStmt struct {
ddlNode
IfExists bool
Tables []*TableName
IsView bool
}
// Restore implements Node interface.
func (n *DropTableStmt) Restore(ctx *RestoreCtx) error {
if n.IsView {
ctx.WriteKeyWord("DROP VIEW ")
} else {
ctx.WriteKeyWord("DROP TABLE ")
}
if n.IfExists {
ctx.WriteKeyWord("IF EXISTS ")
}
for index, table := range n.Tables {
if index != 0 {
ctx.WritePlain(", ")
}
if err := table.Restore(ctx); err != nil {
return errors.Annotate(err, "An error occurred while restore DropTableStmt.Tables "+string(index))
}
}
return nil
}
// Accept implements Node Accept interface.
func (n *DropTableStmt) Accept(v Visitor) (Node, bool) {
newNode, skipChildren := v.Enter(n)
if skipChildren {
return v.Leave(newNode)
}
n = newNode.(*DropTableStmt)
for i, val := range n.Tables {
node, ok := val.Accept(v)
if !ok {
return n, false
}
n.Tables[i] = node.(*TableName)
}
return v.Leave(n)
}
// RenameTableStmt is a statement to rename a table.
// See http://dev.mysql.com/doc/refman/5.7/en/rename-table.html
type RenameTableStmt struct {
ddlNode
OldTable *TableName
NewTable *TableName
// TableToTables is only useful for syncer which depends heavily on tidb parser to do some dirty work for now.
// TODO: Refactor this when you are going to add full support for multiple schema changes.
TableToTables []*TableToTable
}
// Restore implements Node interface.
func (n *RenameTableStmt) Restore(ctx *RestoreCtx) error {
ctx.WriteKeyWord("RENAME TABLE ")
for index, table2table := range n.TableToTables {
if index != 0 {
ctx.WritePlain(", ")
}
if err := table2table.Restore(ctx); err != nil {
return errors.Annotate(err, "An error occurred while restore RenameTableStmt.TableToTables")
}
}
return nil
}
// Accept implements Node Accept interface.
func (n *RenameTableStmt) Accept(v Visitor) (Node, bool) {
newNode, skipChildren := v.Enter(n)
if skipChildren {
return v.Leave(newNode)
}
n = newNode.(*RenameTableStmt)
node, ok := n.OldTable.Accept(v)
if !ok {
return n, false
}
n.OldTable = node.(*TableName)
node, ok = n.NewTable.Accept(v)
if !ok {
return n, false
}
n.NewTable = node.(*TableName)
for i, t := range n.TableToTables {
node, ok := t.Accept(v)
if !ok {
return n, false
}
n.TableToTables[i] = node.(*TableToTable)
}
return v.Leave(n)
}
// TableToTable represents renaming old table to new table used in RenameTableStmt.
type TableToTable struct {
node
OldTable *TableName
NewTable *TableName
}
// Restore implements Node interface.
func (n *TableToTable) Restore(ctx *RestoreCtx) error {
if err := n.OldTable.Restore(ctx); err != nil {
return errors.Annotate(err, "An error occurred while restore TableToTable.OldTable")
}
ctx.WriteKeyWord(" TO ")
if err := n.NewTable.Restore(ctx); err != nil {
return errors.Annotate(err, "An error occurred while restore TableToTable.NewTable")
}
return nil
}
// Accept implements Node Accept interface.
func (n *TableToTable) Accept(v Visitor) (Node, bool) {
newNode, skipChildren := v.Enter(n)
if skipChildren {
return v.Leave(newNode)
}
n = newNode.(*TableToTable)
node, ok := n.OldTable.Accept(v)
if !ok {
return n, false
}
n.OldTable = node.(*TableName)
node, ok = n.NewTable.Accept(v)
if !ok {
return n, false
}
n.NewTable = node.(*TableName)
return v.Leave(n)
}
// CreateViewStmt is a statement to create a View.
// See https://dev.mysql.com/doc/refman/5.7/en/create-view.html
type CreateViewStmt struct {
ddlNode
OrReplace bool
ViewName *TableName
Cols []model.CIStr
Select StmtNode
SchemaCols []model.CIStr
Algorithm model.ViewAlgorithm
Definer *auth.UserIdentity
Security model.ViewSecurity
CheckOption model.ViewCheckOption
}
// Restore implements Node interface.
func (n *CreateViewStmt) Restore(ctx *RestoreCtx) error {
ctx.WriteKeyWord("CREATE ")
if n.OrReplace {
ctx.WriteKeyWord("OR REPLACE ")
}
ctx.WriteKeyWord("ALGORITHM")
ctx.WritePlain(" = ")
ctx.WriteKeyWord(n.Algorithm.String())
ctx.WriteKeyWord(" DEFINER")
ctx.WritePlain(" = ")
// todo Use n.Definer.Restore(ctx) to replace this part
if n.Definer.CurrentUser {
ctx.WriteKeyWord("current_user")
} else {
ctx.WriteName(n.Definer.Username)
if n.Definer.Hostname != "" {
ctx.WritePlain("@")
ctx.WriteName(n.Definer.Hostname)
}
}
ctx.WriteKeyWord(" SQL SECURITY ")
ctx.WriteKeyWord(n.Security.String())
ctx.WriteKeyWord(" VIEW ")
if err := n.ViewName.Restore(ctx); err != nil {
return errors.Annotate(err, "An error occurred while create CreateViewStmt.ViewName")
}
for i, col := range n.Cols {
if i == 0 {
ctx.WritePlain(" (")
} else {
ctx.WritePlain(",")
}
ctx.WriteName(col.O)
if i == len(n.Cols)-1 {
ctx.WritePlain(")")
}
}
ctx.WriteKeyWord(" AS ")
if err := n.Select.Restore(ctx); err != nil {
return errors.Annotate(err, "An error occurred while create CreateViewStmt.Select")
}
if n.CheckOption != model.CheckOptionCascaded {
ctx.WriteKeyWord(" WITH ")
ctx.WriteKeyWord(n.CheckOption.String())
ctx.WriteKeyWord(" CHECK OPTION")
}
return nil
}
// Accept implements Node Accept interface.
func (n *CreateViewStmt) Accept(v Visitor) (Node, bool) {
newNode, skipChildren := v.Enter(n)
if skipChildren {
return v.Leave(newNode)
}
n = newNode.(*CreateViewStmt)
node, ok := n.ViewName.Accept(v)
if !ok {
return n, false
}
n.ViewName = node.(*TableName)
selnode, ok := n.Select.Accept(v)
if !ok {
return n, false
}
n.Select = selnode.(StmtNode)
return v.Leave(n)
}
// CreateIndexStmt is a statement to create an index.
// See https://dev.mysql.com/doc/refman/5.7/en/create-index.html
type CreateIndexStmt struct {
ddlNode
IndexName string
Table *TableName
Unique bool
IndexColNames []*IndexColName
IndexOption *IndexOption
}
// Restore implements Node interface.
func (n *CreateIndexStmt) Restore(ctx *RestoreCtx) error {
ctx.WriteKeyWord("CREATE ")
if n.Unique {
ctx.WriteKeyWord("UNIQUE ")
}
ctx.WriteKeyWord("INDEX ")
ctx.WriteName(n.IndexName)
ctx.WriteKeyWord(" ON ")
if err := n.Table.Restore(ctx); err != nil {
return errors.Annotate(err, "An error occurred while restore CreateIndexStmt.Table")
}
ctx.WritePlain(" (")
for i, indexColName := range n.IndexColNames {
if i != 0 {
ctx.WritePlain(", ")
}
if err := indexColName.Restore(ctx); err != nil {
return errors.Annotatef(err, "An error occurred while restore CreateIndexStmt.IndexColNames: [%v]", i)
}
}
ctx.WritePlain(")")
if n.IndexOption.Tp != model.IndexTypeInvalid || n.IndexOption.KeyBlockSize > 0 || n.IndexOption.Comment != "" {
ctx.WritePlain(" ")
if err := n.IndexOption.Restore(ctx); err != nil {
return errors.Annotate(err, "An error occurred while restore CreateIndexStmt.IndexOption")
}
}
return nil
}
// Accept implements Node Accept interface.
func (n *CreateIndexStmt) Accept(v Visitor) (Node, bool) {
newNode, skipChildren := v.Enter(n)
if skipChildren {
return v.Leave(newNode)
}
n = newNode.(*CreateIndexStmt)
node, ok := n.Table.Accept(v)
if !ok {
return n, false
}
n.Table = node.(*TableName)
for i, val := range n.IndexColNames {
node, ok = val.Accept(v)
if !ok {
return n, false
}
n.IndexColNames[i] = node.(*IndexColName)
}
if n.IndexOption != nil {
node, ok := n.IndexOption.Accept(v)
if !ok {
return n, false
}
n.IndexOption = node.(*IndexOption)
}
return v.Leave(n)
}
// DropIndexStmt is a statement to drop the index.
// See https://dev.mysql.com/doc/refman/5.7/en/drop-index.html
type DropIndexStmt struct {
ddlNode
IfExists bool
IndexName string
Table *TableName
}
// Restore implements Node interface.
func (n *DropIndexStmt) Restore(ctx *RestoreCtx) error {
ctx.WriteKeyWord("DROP INDEX ")
if n.IfExists {
ctx.WriteKeyWord("IF EXISTS ")
}
ctx.WriteName(n.IndexName)
ctx.WriteKeyWord(" ON ")
if err := n.Table.Restore(ctx); err != nil {
return errors.Annotate(err, "An error occurred while add index")
}
return nil
}
// Accept implements Node Accept interface.
func (n *DropIndexStmt) Accept(v Visitor) (Node, bool) {
newNode, skipChildren := v.Enter(n)
if skipChildren {
return v.Leave(newNode)
}
n = newNode.(*DropIndexStmt)
node, ok := n.Table.Accept(v)
if !ok {
return n, false
}
n.Table = node.(*TableName)
return v.Leave(n)
}
// TableOptionType is the type for TableOption
type TableOptionType int
// TableOption types.
const (
TableOptionNone TableOptionType = iota
TableOptionEngine
TableOptionCharset
TableOptionCollate
TableOptionAutoIncrement
TableOptionComment
TableOptionAvgRowLength
TableOptionCheckSum
TableOptionCompression
TableOptionConnection
TableOptionPassword
TableOptionKeyBlockSize
TableOptionMaxRows
TableOptionMinRows
TableOptionDelayKeyWrite
TableOptionRowFormat
TableOptionStatsPersistent
TableOptionShardRowID
TableOptionPreSplitRegion
TableOptionPackKeys
TableOptionTablespace
TableOptionNodegroup
TableOptionDataDirectory
TableOptionIndexDirectory
)
// RowFormat types
const (
RowFormatDefault uint64 = iota + 1
RowFormatDynamic
RowFormatFixed
RowFormatCompressed
RowFormatRedundant
RowFormatCompact
TokuDBRowFormatDefault
TokuDBRowFormatFast
TokuDBRowFormatSmall
TokuDBRowFormatZlib
TokuDBRowFormatQuickLZ
TokuDBRowFormatLzma
TokuDBRowFormatSnappy
TokuDBRowFormatUncompressed
)
// OnDuplicateKeyHandlingType is the option that handle unique key values in 'CREATE TABLE ... SELECT' or `LOAD DATA`.
// See https://dev.mysql.com/doc/refman/5.7/en/create-table-select.html
// See https://dev.mysql.com/doc/refman/5.7/en/load-data.html
type OnDuplicateKeyHandlingType int
// OnDuplicateKeyHandling types
const (
OnDuplicateKeyHandlingError OnDuplicateKeyHandlingType = iota
OnDuplicateKeyHandlingIgnore
OnDuplicateKeyHandlingReplace
)
// TableOption is used for parsing table option from SQL.
type TableOption struct {
Tp TableOptionType
StrValue string
UintValue uint64
}
func (n *TableOption) Restore(ctx *RestoreCtx) error {
switch n.Tp {
case TableOptionEngine:
ctx.WriteKeyWord("ENGINE ")
ctx.WritePlain("= ")
if n.StrValue != "" {
ctx.WritePlain(n.StrValue)
} else {
ctx.WritePlain("''")
}
case TableOptionCharset:
ctx.WriteKeyWord("DEFAULT CHARACTER SET ")
ctx.WritePlain("= ")
ctx.WriteKeyWord(n.StrValue)
case TableOptionCollate:
ctx.WriteKeyWord("DEFAULT COLLATE ")
ctx.WritePlain("= ")
ctx.WriteKeyWord(n.StrValue)
case TableOptionAutoIncrement:
ctx.WriteKeyWord("AUTO_INCREMENT ")
ctx.WritePlain("= ")
ctx.WritePlainf("%d", n.UintValue)
case TableOptionComment:
ctx.WriteKeyWord("COMMENT ")
ctx.WritePlain("= ")
ctx.WriteString(n.StrValue)
case TableOptionAvgRowLength:
ctx.WriteKeyWord("AVG_ROW_LENGTH ")
ctx.WritePlain("= ")
ctx.WritePlainf("%d", n.UintValue)
case TableOptionCheckSum:
ctx.WriteKeyWord("CHECKSUM ")
ctx.WritePlain("= ")
ctx.WritePlainf("%d", n.UintValue)
case TableOptionCompression:
ctx.WriteKeyWord("COMPRESSION ")
ctx.WritePlain("= ")
ctx.WriteString(n.StrValue)
case TableOptionConnection:
ctx.WriteKeyWord("CONNECTION ")
ctx.WritePlain("= ")
ctx.WriteString(n.StrValue)
case TableOptionPassword:
ctx.WriteKeyWord("PASSWORD ")
ctx.WritePlain("= ")
ctx.WriteString(n.StrValue)
case TableOptionKeyBlockSize:
ctx.WriteKeyWord("KEY_BLOCK_SIZE ")
ctx.WritePlain("= ")
ctx.WritePlainf("%d", n.UintValue)
case TableOptionMaxRows:
ctx.WriteKeyWord("MAX_ROWS ")
ctx.WritePlain("= ")
ctx.WritePlainf("%d", n.UintValue)
case TableOptionMinRows:
ctx.WriteKeyWord("MIN_ROWS ")
ctx.WritePlain("= ")
ctx.WritePlainf("%d", n.UintValue)
case TableOptionDelayKeyWrite:
ctx.WriteKeyWord("DELAY_KEY_WRITE ")
ctx.WritePlain("= ")
ctx.WritePlainf("%d", n.UintValue)
case TableOptionRowFormat:
ctx.WriteKeyWord("ROW_FORMAT ")
ctx.WritePlain("= ")
switch n.UintValue {
case RowFormatDefault:
ctx.WriteKeyWord("DEFAULT")
case RowFormatDynamic:
ctx.WriteKeyWord("DYNAMIC")
case RowFormatFixed:
ctx.WriteKeyWord("FIXED")
case RowFormatCompressed:
ctx.WriteKeyWord("COMPRESSED")
case RowFormatRedundant:
ctx.WriteKeyWord("REDUNDANT")
case RowFormatCompact:
ctx.WriteKeyWord("COMPACT")
case TokuDBRowFormatDefault:
ctx.WriteKeyWord("TOKUDB_DEFAULT")
case TokuDBRowFormatFast:
ctx.WriteKeyWord("TOKUDB_FAST")
case TokuDBRowFormatSmall:
ctx.WriteKeyWord("TOKUDB_SMALL")
case TokuDBRowFormatZlib:
ctx.WriteKeyWord("TOKUDB_ZLIB")
case TokuDBRowFormatQuickLZ:
ctx.WriteKeyWord("TOKUDB_QUICKLZ")
case TokuDBRowFormatLzma:
ctx.WriteKeyWord("TOKUDB_LZMA")
case TokuDBRowFormatSnappy:
ctx.WriteKeyWord("TOKUDB_SNAPPY")
case TokuDBRowFormatUncompressed:
ctx.WriteKeyWord("TOKUDB_UNCOMPRESSED")
default:
return errors.Errorf("invalid TableOption: TableOptionRowFormat: %d", n.UintValue)
}
case TableOptionStatsPersistent:
// TODO: not support
ctx.WriteKeyWord("STATS_PERSISTENT ")
ctx.WritePlain("= ")
ctx.WriteKeyWord("DEFAULT")
ctx.WritePlain(" /* TableOptionStatsPersistent is not supported */ ")
case TableOptionShardRowID:
ctx.WriteKeyWord("SHARD_ROW_ID_BITS ")
ctx.WritePlainf("= %d", n.UintValue)
case TableOptionPreSplitRegion:
ctx.WriteKeyWord("PRE_SPLIT_REGIONS ")
ctx.WritePlainf("= %d", n.UintValue)
case TableOptionPackKeys:
// TODO: not support
ctx.WriteKeyWord("PACK_KEYS ")
ctx.WritePlain("= ")
ctx.WriteKeyWord("DEFAULT")
ctx.WritePlain(" /* TableOptionPackKeys is not supported */ ")
case TableOptionTablespace:
ctx.WriteKeyWord("TABLESPACE ")
ctx.WritePlain("= ")
ctx.WriteName(n.StrValue)
case TableOptionNodegroup:
ctx.WriteKeyWord("NODEGROUP ")
ctx.WritePlainf("= %d", n.UintValue)
case TableOptionDataDirectory:
ctx.WriteKeyWord("DATA DIRECTORY ")
ctx.WritePlain("= ")
ctx.WriteString(n.StrValue)
case TableOptionIndexDirectory:
ctx.WriteKeyWord("INDEX DIRECTORY ")
ctx.WritePlain("= ")
ctx.WriteString(n.StrValue)
default:
return errors.Errorf("invalid TableOption: %d", n.Tp)
}
return nil
}
// ColumnPositionType is the type for ColumnPosition.
type ColumnPositionType int
// ColumnPosition Types
const (
ColumnPositionNone ColumnPositionType = iota
ColumnPositionFirst
ColumnPositionAfter
)
// ColumnPosition represent the position of the newly added column
type ColumnPosition struct {
node
// Tp is either ColumnPositionNone, ColumnPositionFirst or ColumnPositionAfter.
Tp ColumnPositionType
// RelativeColumn is the column the newly added column after if type is ColumnPositionAfter
RelativeColumn *ColumnName
}
// Restore implements Node interface.
func (n *ColumnPosition) Restore(ctx *RestoreCtx) error {
switch n.Tp {
case ColumnPositionNone:
// do nothing
case ColumnPositionFirst:
ctx.WriteKeyWord("FIRST")
case ColumnPositionAfter:
ctx.WriteKeyWord("AFTER ")
if err := n.RelativeColumn.Restore(ctx); err != nil {
return errors.Annotate(err, "An error occurred while restore ColumnPosition.RelativeColumn")
}
default:
return errors.Errorf("invalid ColumnPositionType: %d", n.Tp)
}
return nil
}
// Accept implements Node Accept interface.
func (n *ColumnPosition) Accept(v Visitor) (Node, bool) {
newNode, skipChildren := v.Enter(n)
if skipChildren {
return v.Leave(newNode)
}
n = newNode.(*ColumnPosition)
if n.RelativeColumn != nil {
node, ok := n.RelativeColumn.Accept(v)
if !ok {
return n, false
}
n.RelativeColumn = node.(*ColumnName)
}
return v.Leave(n)
}
// AlterTableType is the type for AlterTableSpec.
type AlterTableType int
// AlterTable types.
const (
AlterTableOption AlterTableType = iota + 1
AlterTableAddColumns
AlterTableAddConstraint
AlterTableDropColumn
AlterTableDropPrimaryKey
AlterTableDropIndex
AlterTableDropForeignKey
AlterTableModifyColumn
AlterTableChangeColumn
AlterTableRenameTable
AlterTableAlterColumn
AlterTableLock
AlterTableAlgorithm
AlterTableRenameIndex
AlterTableForce
AlterTableAddPartitions
AlterTableCoalescePartitions
AlterTableDropPartition
AlterTableTruncatePartition
AlterTablePartition
AlterTableEnableKeys
AlterTableDisableKeys
// TODO: Add more actions
)
// LockType is the type for AlterTableSpec.
// See https://dev.mysql.com/doc/refman/5.7/en/alter-table.html#alter-table-concurrency
type LockType byte
func (n LockType) String() string {
switch n {
case LockTypeNone:
return "NONE"
case LockTypeDefault:
return "DEFAULT"
case LockTypeShared:
return "SHARED"
case LockTypeExclusive:
return "EXCLUSIVE"
}
return ""
}
// Lock Types.
const (
LockTypeNone LockType = iota + 1
LockTypeDefault
LockTypeShared
LockTypeExclusive
)
// AlterAlgorithm is the algorithm of the DDL operations.
// See https://dev.mysql.com/doc/refman/8.0/en/alter-table.html#alter-table-performance.
type AlterAlgorithm byte
// DDL alter algorithms.
// For now, TiDB only supported inplace and instance algorithms. If the user specify `copy`,
// will get an error.
const (
AlterAlgorithmDefault AlterAlgorithm = iota
AlterAlgorithmCopy
AlterAlgorithmInplace
AlterAlgorithmInstant
)
func (a AlterAlgorithm) String() string {
switch a {
case AlterAlgorithmDefault:
return "DEFAULT"
case AlterAlgorithmCopy:
return "COPY"
case AlterAlgorithmInplace:
return "INPLACE"
case AlterAlgorithmInstant:
return "INSTANT"
default:
return "DEFAULT"
}
}
// AlterTableSpec represents alter table specification.
type AlterTableSpec struct {
node
Tp AlterTableType
Name string
Constraint *Constraint
Options []*TableOption
NewTable *TableName
NewColumns []*ColumnDef
OldColumnName *ColumnName
Position *ColumnPosition
LockType LockType
Algorithm AlterAlgorithm
Comment string
FromKey model.CIStr
ToKey model.CIStr
Partition *PartitionOptions
PartitionNames []model.CIStr
PartDefinitions []*PartitionDefinition
Num uint64
}
// Restore implements Node interface.
func (n *AlterTableSpec) Restore(ctx *RestoreCtx) error {
switch n.Tp {
case AlterTableOption:
switch {
case len(n.Options) == 2 &&
n.Options[0].Tp == TableOptionCharset &&
n.Options[1].Tp == TableOptionCollate:
ctx.WriteKeyWord("CONVERT TO CHARACTER SET ")
ctx.WriteKeyWord(n.Options[0].StrValue)
ctx.WriteKeyWord(" COLLATE ")
ctx.WriteKeyWord(n.Options[1].StrValue)
default:
for i, opt := range n.Options {
if i != 0 {
ctx.WritePlain(", ")
}
if err := opt.Restore(ctx); err != nil {
return errors.Annotatef(err, "An error occurred while restore AlterTableSpec.Options[%d]", i)
}
}
}
case AlterTableAddColumns:
ctx.WriteKeyWord("ADD COLUMN ")
if n.Position != nil && len(n.NewColumns) == 1 {
if err := n.NewColumns[0].Restore(ctx); err != nil {
return errors.Annotatef(err, "An error occurred while restore AlterTableSpec.NewColumns[%d]", 0)
}
if n.Position.Tp != ColumnPositionNone {
ctx.WritePlain(" ")
}
if err := n.Position.Restore(ctx); err != nil {
return errors.Annotate(err, "An error occurred while restore AlterTableSpec.Position")
}
} else {
ctx.WritePlain("(")
for i, col := range n.NewColumns {
if i != 0 {
ctx.WritePlain(", ")
}
if err := col.Restore(ctx); err != nil {
return errors.Annotatef(err, "An error occurred while restore AlterTableSpec.NewColumns[%d]", i)
}
}
ctx.WritePlain(")")
}
case AlterTableAddConstraint:
ctx.WriteKeyWord("ADD ")
if err := n.Constraint.Restore(ctx); err != nil {
return errors.Annotate(err, "An error occurred while restore AlterTableSpec.Constraint")
}
case AlterTableDropColumn:
ctx.WriteKeyWord("DROP COLUMN ")
if err := n.OldColumnName.Restore(ctx); err != nil {
return errors.Annotate(err, "An error occurred while restore AlterTableSpec.OldColumnName")
}
// TODO: RestrictOrCascadeOpt not support
case AlterTableDropPrimaryKey:
ctx.WriteKeyWord("DROP PRIMARY KEY")
case AlterTableDropIndex:
ctx.WriteKeyWord("DROP INDEX ")
ctx.WriteName(n.Name)
case AlterTableDropForeignKey:
ctx.WriteKeyWord("DROP FOREIGN KEY ")
ctx.WriteName(n.Name)
case AlterTableModifyColumn:
ctx.WriteKeyWord("MODIFY COLUMN ")
if err := n.NewColumns[0].Restore(ctx); err != nil {
return errors.Annotate(err, "An error occurred while restore AlterTableSpec.NewColumns[0]")
}
if n.Position.Tp != ColumnPositionNone {
ctx.WritePlain(" ")
}
if err := n.Position.Restore(ctx); err != nil {
return errors.Annotate(err, "An error occurred while restore AlterTableSpec.Position")
}
case AlterTableChangeColumn:
ctx.WriteKeyWord("CHANGE COLUMN ")
if err := n.OldColumnName.Restore(ctx); err != nil {
return errors.Annotate(err, "An error occurred while restore AlterTableSpec.OldColumnName")
}
ctx.WritePlain(" ")
if err := n.NewColumns[0].Restore(ctx); err != nil {
return errors.Annotate(err, "An error occurred while restore AlterTableSpec.NewColumns[0]")
}
if n.Position.Tp != ColumnPositionNone {
ctx.WritePlain(" ")
}
if err := n.Position.Restore(ctx); err != nil {
return errors.Annotate(err, "An error occurred while restore AlterTableSpec.Position")
}
case AlterTableRenameTable:
ctx.WriteKeyWord("RENAME AS ")
if err := n.NewTable.Restore(ctx); err != nil {
return errors.Annotate(err, "An error occurred while restore AlterTableSpec.NewTable")
}
case AlterTableAlterColumn:
ctx.WriteKeyWord("ALTER COLUMN ")
if err := n.NewColumns[0].Restore(ctx); err != nil {
return errors.Annotate(err, "An error occurred while restore AlterTableSpec.NewColumns[0]")
}
if len(n.NewColumns[0].Options) == 1 {
ctx.WriteKeyWord("SET DEFAULT ")
expr := n.NewColumns[0].Options[0].Expr
if valueExpr, ok := expr.(ValueExpr); ok {
if err := valueExpr.Restore(ctx); err != nil {
return errors.Annotate(err, "An error occurred while restore AlterTableSpec.NewColumns[0].Options[0].Expr")
}
} else {
ctx.WritePlain("(")
if err := expr.Restore(ctx); err != nil {
return errors.Annotate(err, "An error occurred while restore AlterTableSpec.NewColumns[0].Options[0].Expr")
}
ctx.WritePlain(")")
}
} else {
ctx.WriteKeyWord(" DROP DEFAULT")
}
case AlterTableLock:
ctx.WriteKeyWord("LOCK ")
ctx.WritePlain("= ")
ctx.WriteKeyWord(n.LockType.String())
case AlterTableAlgorithm:
ctx.WriteKeyWord("ALGORITHM ")
ctx.WritePlain("= ")
ctx.WriteKeyWord(n.Algorithm.String())
case AlterTableRenameIndex:
ctx.WriteKeyWord("RENAME INDEX ")
ctx.WriteName(n.FromKey.O)
ctx.WriteKeyWord(" TO ")
ctx.WriteName(n.ToKey.O)
case AlterTableForce:
// TODO: not support
ctx.WriteKeyWord("FORCE")
ctx.WritePlain(" /* AlterTableForce is not supported */ ")
case AlterTableAddPartitions:
ctx.WriteKeyWord("ADD PARTITION")
if n.PartDefinitions != nil {
ctx.WritePlain(" (")
for i, def := range n.PartDefinitions {
if i != 0 {
ctx.WritePlain(", ")
}
if err := def.Restore(ctx); err != nil {
return errors.Annotatef(err, "An error occurred while restore AlterTableSpec.PartDefinitions[%d]", i)
}
}
ctx.WritePlain(")")
} else if n.Num != 0 {
ctx.WriteKeyWord(" PARTITIONS ")
ctx.WritePlainf("%d", n.Num)
}
case AlterTableCoalescePartitions:
ctx.WriteKeyWord("COALESCE PARTITION ")
ctx.WritePlainf("%d", n.Num)
case AlterTableDropPartition:
ctx.WriteKeyWord("DROP PARTITION ")
for i, name := range n.PartitionNames {
if i != 0 {
ctx.WritePlain(",")
}
ctx.WriteName(name.O)
}
case AlterTableTruncatePartition:
ctx.WriteKeyWord("TRUNCATE PARTITION ")
for i, name := range n.PartitionNames {
if i != 0 {
ctx.WritePlain(",")
}
ctx.WriteName(name.O)
}
case AlterTablePartition:
if err := n.Partition.Restore(ctx); err != nil {
return errors.Annotate(err, "An error occurred while restore AlterTableSpec.Partition")
}
case AlterTableEnableKeys:
ctx.WriteKeyWord("ENABLE KEYS")
case AlterTableDisableKeys:
ctx.WriteKeyWord("DISABLE KEYS")
default:
// TODO: not support
ctx.WritePlainf(" /* AlterTableType(%d) is not supported */ ", n.Tp)
}
return nil
}
// Accept implements Node Accept interface.
func (n *AlterTableSpec) Accept(v Visitor) (Node, bool) {
newNode, skipChildren := v.Enter(n)
if skipChildren {
return v.Leave(newNode)
}
n = newNode.(*AlterTableSpec)
if n.Constraint != nil {
node, ok := n.Constraint.Accept(v)
if !ok {
return n, false
}
n.Constraint = node.(*Constraint)
}
if n.NewTable != nil {
node, ok := n.NewTable.Accept(v)
if !ok {
return n, false
}
n.NewTable = node.(*TableName)
}
for _, col := range n.NewColumns {
node, ok := col.Accept(v)
if !ok {
return n, false
}
col = node.(*ColumnDef)
}
if n.OldColumnName != nil {
node, ok := n.OldColumnName.Accept(v)
if !ok {
return n, false
}
n.OldColumnName = node.(*ColumnName)
}
if n.Position != nil {
node, ok := n.Position.Accept(v)
if !ok {
return n, false
}
n.Position = node.(*ColumnPosition)
}
return v.Leave(n)
}
// AlterTableStmt is a statement to change the structure of a table.
// See https://dev.mysql.com/doc/refman/5.7/en/alter-table.html
type AlterTableStmt struct {
ddlNode
Table *TableName
Specs []*AlterTableSpec
}
// Restore implements Node interface.
func (n *AlterTableStmt) Restore(ctx *RestoreCtx) error {
ctx.WriteKeyWord("ALTER TABLE ")
if err := n.Table.Restore(ctx); err != nil {
return errors.Annotate(err, "An error occurred while restore AlterTableStmt.Table")
}
for i, spec := range n.Specs {
if i == 0 || spec.Tp == AlterTablePartition {
ctx.WritePlain(" ")
} else {
ctx.WritePlain(", ")
}
if err := spec.Restore(ctx); err != nil {
return errors.Annotatef(err, "An error occurred while restore AlterTableStmt.Specs[%d]", i)
}
}
return nil
}
// Accept implements Node Accept interface.
func (n *AlterTableStmt) Accept(v Visitor) (Node, bool) {
newNode, skipChildren := v.Enter(n)
if skipChildren {
return v.Leave(newNode)
}
n = newNode.(*AlterTableStmt)
node, ok := n.Table.Accept(v)
if !ok {
return n, false
}
n.Table = node.(*TableName)
for i, val := range n.Specs {
node, ok = val.Accept(v)
if !ok {
return n, false
}
n.Specs[i] = node.(*AlterTableSpec)
}
return v.Leave(n)
}
// TruncateTableStmt is a statement to empty a table completely.
// See https://dev.mysql.com/doc/refman/5.7/en/truncate-table.html
type TruncateTableStmt struct {
ddlNode
Table *TableName
}
// Restore implements Node interface.
func (n *TruncateTableStmt) Restore(ctx *RestoreCtx) error {
ctx.WriteKeyWord("TRUNCATE TABLE ")
if err := n.Table.Restore(ctx); err != nil {
return errors.Annotate(err, "An error occurred while restore TruncateTableStmt.Table")
}
return nil
}
// Accept implements Node Accept interface.
func (n *TruncateTableStmt) Accept(v Visitor) (Node, bool) {
newNode, skipChildren := v.Enter(n)
if skipChildren {
return v.Leave(newNode)
}
n = newNode.(*TruncateTableStmt)
node, ok := n.Table.Accept(v)
if !ok {
return n, false
}
n.Table = node.(*TableName)
return v.Leave(n)
}
var (
ErrNoParts = terror.ClassDDL.NewStd(mysql.ErrNoParts)
ErrPartitionColumnList = terror.ClassDDL.NewStd(mysql.ErrPartitionColumnList)
ErrPartitionRequiresValues = terror.ClassDDL.NewStd(mysql.ErrPartitionRequiresValues)
ErrPartitionsMustBeDefined = terror.ClassDDL.NewStd(mysql.ErrPartitionsMustBeDefined)
ErrPartitionWrongNoPart = terror.ClassDDL.NewStd(mysql.ErrPartitionWrongNoPart)
ErrPartitionWrongNoSubpart = terror.ClassDDL.NewStd(mysql.ErrPartitionWrongNoSubpart)
ErrPartitionWrongValues = terror.ClassDDL.NewStd(mysql.ErrPartitionWrongValues)
ErrRowSinglePartitionField = terror.ClassDDL.NewStd(mysql.ErrRowSinglePartitionField)
ErrSubpartition = terror.ClassDDL.NewStd(mysql.ErrSubpartition)
ErrSystemVersioningWrongPartitions = terror.ClassDDL.NewStd(mysql.ErrSystemVersioningWrongPartitions)
ErrTooManyValues = terror.ClassDDL.NewStd(mysql.ErrTooManyValues)
ErrWrongPartitionTypeExpectedSystemTime = terror.ClassDDL.NewStd(mysql.ErrWrongPartitionTypeExpectedSystemTime)
)
type SubPartitionDefinition struct {
Name model.CIStr
Options []*TableOption
}
func (spd *SubPartitionDefinition) Restore(ctx *RestoreCtx) error {
ctx.WriteKeyWord("SUBPARTITION ")
ctx.WriteName(spd.Name.O)
for i, opt := range spd.Options {
ctx.WritePlain(" ")
if err := opt.Restore(ctx); err != nil {
return errors.Annotatef(err, "An error occurred while restore SubPartitionDefinition.Options[%d]", i)
}
}
return nil
}
type PartitionDefinitionClause interface {
restore(ctx *RestoreCtx) error
acceptInPlace(v Visitor) bool
// Validate checks if the clause is consistent with the given options.
// `pt` can be 0 and `columns` can be -1 to skip checking the clause against
// the partition type or number of columns in the expression list.
Validate(pt model.PartitionType, columns int) error
}
type PartitionDefinitionClauseNone struct{}
func (n *PartitionDefinitionClauseNone) restore(ctx *RestoreCtx) error {
return nil
}
func (n *PartitionDefinitionClauseNone) acceptInPlace(v Visitor) bool {
return true
}
func (n *PartitionDefinitionClauseNone) Validate(pt model.PartitionType, columns int) error {
switch pt {
case 0:
case model.PartitionTypeRange:
return ErrPartitionRequiresValues.GenWithStackByArgs("RANGE", "LESS THAN")
case model.PartitionTypeList:
return ErrPartitionRequiresValues.GenWithStackByArgs("LIST", "IN")
case model.PartitionTypeSystemTime:
return ErrSystemVersioningWrongPartitions
}
return nil
}
type PartitionDefinitionClauseLessThan struct {
Exprs []ExprNode
}
func (n *PartitionDefinitionClauseLessThan) restore(ctx *RestoreCtx) error {
ctx.WriteKeyWord(" VALUES LESS THAN ")
ctx.WritePlain("(")
for i, expr := range n.Exprs {
if i != 0 {
ctx.WritePlain(", ")
}
if err := expr.Restore(ctx); err != nil {
return errors.Annotatef(err, "An error occurred while restore PartitionDefinitionClauseLessThan.Exprs[%d]", i)
}
}
ctx.WritePlain(")")
return nil
}
func (n *PartitionDefinitionClauseLessThan) acceptInPlace(v Visitor) bool {
for i, expr := range n.Exprs {
newExpr, ok := expr.Accept(v)
if !ok {
return false
}
n.Exprs[i] = newExpr.(ExprNode)
}
return true
}
func (n *PartitionDefinitionClauseLessThan) Validate(pt model.PartitionType, columns int) error {
switch pt {
case model.PartitionTypeRange, 0:
default:
return ErrPartitionWrongValues.GenWithStackByArgs("RANGE", "LESS THAN")
}
switch {
case columns == 0 && len(n.Exprs) != 1:
return ErrTooManyValues.GenWithStackByArgs("RANGE")
case columns > 0 && len(n.Exprs) != columns:
return ErrPartitionColumnList
}
return nil
}
type PartitionDefinitionClauseIn struct {
Values [][]ExprNode
}
func (n *PartitionDefinitionClauseIn) restore(ctx *RestoreCtx) error {
// we special-case an empty list of values to mean MariaDB's "DEFAULT" clause.
if len(n.Values) == 0 {
ctx.WriteKeyWord(" DEFAULT")
return nil
}
ctx.WriteKeyWord(" VALUES IN ")
ctx.WritePlain("(")
for i, valList := range n.Values {
if i != 0 {
ctx.WritePlain(", ")
}
if len(valList) == 1 {
if err := valList[0].Restore(ctx); err != nil {
return errors.Annotatef(err, "An error occurred while restore PartitionDefinitionClauseIn.Values[%d][0]", i)
}
} else {
ctx.WritePlain("(")
for j, val := range valList {
if j != 0 {
ctx.WritePlain(", ")
}
if err := val.Restore(ctx); err != nil {
return errors.Annotatef(err, "An error occurred while restore PartitionDefinitionClauseIn.Values[%d][%d]", i, j)
}
}
ctx.WritePlain(")")
}
}
ctx.WritePlain(")")
return nil
}
func (n *PartitionDefinitionClauseIn) acceptInPlace(v Visitor) bool {
for _, valList := range n.Values {
for j, val := range valList {
newVal, ok := val.Accept(v)
if !ok {
return false
}
valList[j] = newVal.(ExprNode)
}
}
return true
}
func (n *PartitionDefinitionClauseIn) Validate(pt model.PartitionType, columns int) error {
switch pt {
case model.PartitionTypeList, 0:
default:
return ErrPartitionWrongValues.GenWithStackByArgs("LIST", "IN")
}
if len(n.Values) == 0 {
return nil
}
expectedColCount := len(n.Values[0])
for _, val := range n.Values[1:] {
if len(val) != expectedColCount {
return ErrPartitionColumnList
}
}
switch {
case columns == 0 && expectedColCount != 1:
return ErrRowSinglePartitionField
case columns > 0 && expectedColCount != columns:
return ErrPartitionColumnList
}
return nil
}
type PartitionDefinitionClauseHistory struct {
Current bool
}
func (n *PartitionDefinitionClauseHistory) restore(ctx *RestoreCtx) error {
if n.Current {
ctx.WriteKeyWord(" CURRENT")
} else {
ctx.WriteKeyWord(" HISTORY")
}
return nil
}
func (n *PartitionDefinitionClauseHistory) acceptInPlace(v Visitor) bool {
return true
}
func (n *PartitionDefinitionClauseHistory) Validate(pt model.PartitionType, columns int) error {
switch pt {
case 0, model.PartitionTypeSystemTime:
default:
return ErrWrongPartitionTypeExpectedSystemTime
}
return nil
}
// PartitionDefinition defines a single partition.
type PartitionDefinition struct {
Name model.CIStr
Clause PartitionDefinitionClause
Options []*TableOption
Sub []*SubPartitionDefinition
}
// Comment returns the comment option given to this definition.
// The second return value indicates if the comment option exists.
func (n *PartitionDefinition) Comment() (string, bool) {
for _, opt := range n.Options {
if opt.Tp == TableOptionComment {
return opt.StrValue, true
}
}
return "", false
}
func (n *PartitionDefinition) acceptInPlace(v Visitor) bool {
return n.Clause.acceptInPlace(v)
}
// Restore implements Node interface.
func (n *PartitionDefinition) Restore(ctx *RestoreCtx) error {
ctx.WriteKeyWord("PARTITION ")
ctx.WriteName(n.Name.O)
if err := n.Clause.restore(ctx); err != nil {
return errors.Annotate(err, "An error occurred while restore PartitionDefinition.Clause")
}
for i, opt := range n.Options {
ctx.WritePlain(" ")
if err := opt.Restore(ctx); err != nil {
return errors.Annotatef(err, "An error occurred while restore PartitionDefinition.Options[%d]", i)
}
}
if len(n.Sub) > 0 {
ctx.WritePlain(" (")
for i, spd := range n.Sub {
if i != 0 {
ctx.WritePlain(",")
}
if err := spd.Restore(ctx); err != nil {
return errors.Annotatef(err, "An error occurred while restore PartitionDefinition.Sub[%d]", i)
}
}
ctx.WritePlain(")")
}
return nil
}
// PartitionMethod describes how partitions or subpartitions are constructed.
type PartitionMethod struct {
// Tp is the type of the partition function
Tp model.PartitionType
// Linear is a modifier to the HASH and KEY type for choosing a different
// algorithm
Linear bool
// Expr is an expression used as argument of HASH, RANGE, LIST and
// SYSTEM_TIME types
Expr ExprNode
// ColumnNames is a list of column names used as argument of KEY,
// RANGE COLUMNS and LIST COLUMNS types
ColumnNames []*ColumnName
// Unit is a time unit used as argument of SYSTEM_TIME type
Unit ValueExpr
// Limit is a row count used as argument of the SYSTEM_TIME type
Limit uint64
// Num is the number of (sub)partitions required by the method.
Num uint64
}
// Restore implements the Node interface
func (n *PartitionMethod) Restore(ctx *RestoreCtx) error {
if n.Linear {
ctx.WriteKeyWord("LINEAR ")
}
ctx.WriteKeyWord(n.Tp.String())
switch {
case n.Tp == model.PartitionTypeSystemTime:
if n.Expr != nil && n.Unit != nil {
ctx.WriteKeyWord(" INTERVAL ")
if err := n.Expr.Restore(ctx); err != nil {
return errors.Annotate(err, "An error occurred while restore PartitionMethod.Expr")
}
// Here the Unit string should not be quoted.
// TODO: This is a temporary workaround that should be changed once something like "Keyword Expression" is implemented.
var sb strings.Builder
if err := n.Unit.Restore(NewRestoreCtx(0, &sb)); err != nil {
return errors.Annotate(err, "An error occurred while restore PartitionMethod.Unit")
}
ctx.WritePlain(" ")
ctx.WriteKeyWord(sb.String())
}
case n.Expr != nil:
ctx.WritePlain(" (")
if err := n.Expr.Restore(ctx); err != nil {
return errors.Annotate(err, "An error occurred while restore PartitionMethod.Expr")
}
ctx.WritePlain(")")
default:
if n.Tp == model.PartitionTypeRange || n.Tp == model.PartitionTypeList {
ctx.WriteKeyWord(" COLUMNS")
}
ctx.WritePlain(" (")
for i, col := range n.ColumnNames {
if i > 0 {
ctx.WritePlain(",")
}
if err := col.Restore(ctx); err != nil {
return errors.Annotatef(err, "An error occurred while splicing PartitionMethod.ColumnName[%d]", i)
}
}
ctx.WritePlain(")")
}
if n.Limit > 0 {
ctx.WriteKeyWord(" LIMIT ")
ctx.WritePlainf("%d", n.Limit)
}
return nil
}
// acceptInPlace is like Node.Accept but does not allow replacing the node itself.
func (n *PartitionMethod) acceptInPlace(v Visitor) bool {
if n.Expr != nil {
expr, ok := n.Expr.Accept(v)
if !ok {
return false
}
n.Expr = expr.(ExprNode)
}
for i, colName := range n.ColumnNames {
newColName, ok := colName.Accept(v)
if !ok {
return false
}
n.ColumnNames[i] = newColName.(*ColumnName)
}
if n.Unit != nil {
unit, ok := n.Unit.Accept(v)
if !ok {
return false
}
n.Unit = unit.(ValueExpr)
}
return true
}
// PartitionOptions specifies the partition options.
type PartitionOptions struct {
node
PartitionMethod
Sub *PartitionMethod
Definitions []*PartitionDefinition
}
// Validate checks if the partition is well-formed.
func (n *PartitionOptions) Validate() error {
// if both a partition list and the partition numbers are specified, their values must match
if n.Num != 0 && len(n.Definitions) != 0 && n.Num != uint64(len(n.Definitions)) {
return ErrPartitionWrongNoPart
}
// now check the subpartition count
if len(n.Definitions) > 0 {
// ensure the subpartition count for every partitions are the same
// then normalize n.Num and n.Sub.Num so equality comparison works.
n.Num = uint64(len(n.Definitions))
subDefCount := len(n.Definitions[0].Sub)
for _, pd := range n.Definitions[1:] {
if len(pd.Sub) != subDefCount {
return ErrPartitionWrongNoSubpart
}
}
if n.Sub != nil {
if n.Sub.Num != 0 && subDefCount != 0 && n.Sub.Num != uint64(subDefCount) {
return ErrPartitionWrongNoSubpart
}
if subDefCount != 0 {
n.Sub.Num = uint64(subDefCount)
}
} else if subDefCount != 0 {
return ErrSubpartition
}
}
switch n.Tp {
case model.PartitionTypeHash, model.PartitionTypeKey:
if n.Num == 0 {
n.Num = 1
}
case model.PartitionTypeRange, model.PartitionTypeList:
if len(n.Definitions) == 0 {
return ErrPartitionsMustBeDefined.GenWithStackByArgs(n.Tp)
}
case model.PartitionTypeSystemTime:
if len(n.Definitions) < 2 {
return ErrSystemVersioningWrongPartitions
}
}
for _, pd := range n.Definitions {
// ensure the partition definition types match the methods,
// e.g. RANGE partitions only allows VALUES LESS THAN
if err := pd.Clause.Validate(n.Tp, len(n.ColumnNames)); err != nil {
return err
}
}
return nil
}
func (n *PartitionOptions) Restore(ctx *RestoreCtx) error {
ctx.WriteKeyWord("PARTITION BY ")
if err := n.PartitionMethod.Restore(ctx); err != nil {
return errors.Annotate(err, "An error occurred while restore PartitionOptions.PartitionMethod")
}
if n.Num > 0 && len(n.Definitions) == 0 {
ctx.WriteKeyWord(" PARTITIONS ")
ctx.WritePlainf("%d", n.Num)
}
if n.Sub != nil {
ctx.WriteKeyWord(" SUBPARTITION BY ")
if err := n.Sub.Restore(ctx); err != nil {
return errors.Annotate(err, "An error occurred while restore PartitionOptions.Sub")
}
if n.Sub.Num > 0 {
ctx.WriteKeyWord(" SUBPARTITIONS ")
ctx.WritePlainf("%d", n.Sub.Num)
}
}
if len(n.Definitions) > 0 {
ctx.WritePlain(" (")
for i, def := range n.Definitions {
if i > 0 {
ctx.WritePlain(",")
}
if err := def.Restore(ctx); err != nil {
return errors.Annotatef(err, "An error occurred while restore PartitionOptions.Definitions[%d]", i)
}
}
ctx.WritePlain(")")
}
return nil
}
func (n *PartitionOptions) Accept(v Visitor) (Node, bool) {
newNode, skipChildren := v.Enter(n)
if skipChildren {
return v.Leave(newNode)
}
n = newNode.(*PartitionOptions)
if !n.PartitionMethod.acceptInPlace(v) {
return n, false
}
if n.Sub != nil && !n.Sub.acceptInPlace(v) {
return n, false
}
for _, def := range n.Definitions {
if !def.acceptInPlace(v) {
return n, false
}
}
return v.Leave(n)
}
// RecoverTableStmt is a statement to recover dropped table.
type RecoverTableStmt struct {
ddlNode
JobID int64
Table *TableName
JobNum int64
}
// Restore implements Node interface.
func (n *RecoverTableStmt) Restore(ctx *RestoreCtx) error {
ctx.WriteKeyWord("RECOVER TABLE ")
if n.JobID != 0 {
ctx.WriteKeyWord("BY JOB ")
ctx.WritePlainf("%d", n.JobID)
} else {
if err := n.Table.Restore(ctx); err != nil {
return errors.Annotate(err, "An error occurred while splicing RecoverTableStmt Table")
}
if n.JobNum > 0 {
ctx.WritePlainf(" %d", n.JobNum)
}
}
return nil
}
// Accept implements Node Accept interface.
func (n *RecoverTableStmt) Accept(v Visitor) (Node, bool) {
newNode, skipChildren := v.Enter(n)
if skipChildren {
return v.Leave(newNode)
}
n = newNode.(*RecoverTableStmt)
if n.Table != nil {
node, ok := n.Table.Accept(v)
if !ok {
return n, false
}
n.Table = node.(*TableName)
}
return v.Leave(n)
}
马建仓 AI 助手
尝试更多
代码解读
代码找茬
代码优化
1
https://gitee.com/jiangplus/tidb-parser.git
git@gitee.com:jiangplus/tidb-parser.git
jiangplus
tidb-parser
tidb-parser
v3.0.11

搜索帮助

344bd9b3 5694891 D2dac590 5694891